// setting 'moving' application state
Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.moving(newToken));
logger_.info(String.format("Moving %s from %s to %s.", localAddress, getLocalToken(), newToken));
IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
Map<String, Multimap<InetAddress, Range>> rangesToFetch = new HashMap<String, Multimap<InetAddress, Range>>();
Map<String, Multimap<Range, InetAddress>> rangesToStreamByTable = new HashMap<String, Multimap<Range, InetAddress>>();
TokenMetadata tokenMetaClone = tokenMetadata_.cloneAfterAllSettled();
// for each of the non system tables calculating new ranges
// which current node will handle after move to the new token
for (String table : tablesToProcess)
{
// replication strategy of the current keyspace (aka table)
AbstractReplicationStrategy strategy = Table.open(table).getReplicationStrategy();
// getting collection of the currently used ranges by this keyspace
Collection<Range> currentRanges = getRangesForEndpoint(table, localAddress);
// collection of ranges which this node will serve after move to the new token
Collection<Range> updatedRanges = strategy.getPendingAddressRanges(tokenMetadata_, newToken, localAddress);
// ring ranges and endpoints associated with them
// this used to determine what nodes should we ping about range data
Multimap<Range, InetAddress> rangeAddresses = strategy.getRangeAddresses(tokenMetadata_);
// calculated parts of the ranges to request/stream from/to nodes in the ring
Pair<Set<Range>, Set<Range>> rangesPerTable = calculateStreamAndFetchRanges(currentRanges, updatedRanges);
/**
* In this loop we are going through all ranges "to fetch" and determining
* nodes in the ring responsible for data we are interested in
*/
Multimap<Range, InetAddress> rangesToFetchWithPreferredEndpoints = ArrayListMultimap.create();
for (Range toFetch : rangesPerTable.right)
{
for (Range range : rangeAddresses.keySet())
{
if (range.contains(toFetch))
{
List<InetAddress> endpoints = snitch.getSortedListByProximity(localAddress, rangeAddresses.get(range));
// storing range and preferred endpoint set
rangesToFetchWithPreferredEndpoints.putAll(toFetch, endpoints);
}
}
}