Github user aweisberg commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/224#discussion_r188688173
--- Diff: src/java/org/apache/cassandra/service/StorageService.java ---
@@ -4307,32 +4300,38 @@ private void
calculateToFromStreams(Collection<Token> newTokens, List<String> ke
if (addressList.size() > 1)
throw new IllegalStateException("Multiple
strict sources found for " + toFetch);
- InetAddressAndPort sourceIp =
addressList.iterator().next();
+ InetAddressAndPort sourceIp =
addressList.iterator().next().getEndpoint();
if (Gossiper.instance.isEnabled() &&
!Gossiper.instance.getEndpointStateForEndpoint(sourceIp).isAlive())
throw new RuntimeException("A node
required to move the data consistently is down ("+sourceIp+"). If you wish to
move the data from a potentially inconsistent replica, restart the node with
-Dcassandra.consistent.rangemovement=false");
}
}
// calculating endpoints to stream current ranges to
if needed
// in some situations node will handle current ranges
as part of the new ranges
- Multimap<InetAddressAndPort, Range<Token>>
endpointRanges = HashMultimap.create();
+ ReplicaMultimap<InetAddressAndPort, ReplicaSet>
endpointRanges = ReplicaMultimap.set();
for (Range<Token> toStream : rangesPerKeyspace.left)
{
- Set<InetAddressAndPort> currentEndpoints =
ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right,
tokenMetaClone));
- Set<InetAddressAndPort> newEndpoints =
ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right,
tokenMetaCloneAllSettled));
+ Set<Replica> currentEndpoints =
ImmutableSet.copyOf(strategy.calculateNaturalReplicas(toStream.right,
tokenMetaClone));
+ Set<Replica> newEndpoints =
ImmutableSet.copyOf(strategy.calculateNaturalReplicas(toStream.right,
tokenMetaCloneAllSettled));
+
+ Replicas.checkFull(currentEndpoints);
+ Replicas.checkFull(newEndpoints);
+
logger.debug("Range: {} Current endpoints: {} New
endpoints: {}", toStream, currentEndpoints, newEndpoints);
- for (InetAddressAndPort address :
Sets.difference(newEndpoints, currentEndpoints))
+ for (Replica replica :
Sets.difference(newEndpoints, currentEndpoints))
--- End diff --
I am not sure this works since it should be by address. I think I found
cases that were suspect when it also compared by range.
I thought we weren't going to do regular collections of Replicas for the
most part so it would be explicit?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]