Github user ifesdjeen commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/224#discussion_r197129915
--- Diff:
src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java ---
@@ -90,41 +98,53 @@ public NetworkTopologyStrategy(String keyspaceName,
TokenMetadata tokenMetadata,
/** Number of replicas left to fill from this DC. */
int rfLeft;
int acceptableRackRepeats;
+ int transients;
- DatacenterEndpoints(int rf, int rackCount, int nodeCount,
Set<InetAddressAndPort> endpoints, Set<Pair<String, String>> racks)
+ DatacenterEndpoints(ReplicationFactor rf, int rackCount, int
nodeCount, ReplicaSet replicas, Set<Pair<String, String>> racks)
{
- this.endpoints = endpoints;
+ this.replicas = replicas;
this.racks = racks;
// If there aren't enough nodes in this DC to fill the RF, the
number of nodes is the effective RF.
- this.rfLeft = Math.min(rf, nodeCount);
+ this.rfLeft = Math.min(rf.replicas, nodeCount);
// If there aren't enough racks in this DC to fill the RF,
we'll still use at least one node from each rack,
// and the difference is to be filled by the first encountered
nodes.
- acceptableRackRepeats = rf - rackCount;
+ acceptableRackRepeats = rf.replicas - rackCount;
+
+ // if we have fewer replicas than rf calls for, reduce
transients accordingly
+ int reduceTransients = rf.replicas - this.rfLeft;
+ transients = Math.max(rf.trans - reduceTransients, 0);
+ ReplicationFactor.validate(rfLeft, transients);
}
/**
- * Attempts to add an endpoint to the replicas for this
datacenter, adding to the endpoints set if successful.
+ * Attempts to add an endpoint to the replicas for this
datacenter, adding to the replicas set if successful.
* Returns true if the endpoint was added, and this datacenter
does not require further replicas.
*/
- boolean addEndpointAndCheckIfDone(InetAddressAndPort ep,
Pair<String,String> location)
+ boolean addEndpointAndCheckIfDone(InetAddressAndPort ep,
Pair<String,String> location, Range<Token> replicatedRange)
{
if (done())
return false;
+ if (replicas.containsEndpoint(ep))
--- End diff --
Previously this has been handled with `endpoints.add` which was using Set
as well, and it was throwing with `assert` rather than skipping.
Did we change some subtle semantics here?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]