Github user belliottsmith commented on a diff in the pull request: https://github.com/apache/cassandra/pull/267#discussion_r219133467 --- Diff: src/java/org/apache/cassandra/locator/ReplicaPlans.java --- @@ -61,26 +73,62 @@ return forSingleReplicaWrite(keyspace, token, replica); } + public static ReplicaPlan.ForTokenWrite forLocalBatchlogWrite() + { + Token token = DatabaseDescriptor.getPartitioner().getMinimumToken(); + Keyspace systemKeypsace = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME); + Replica localSystemReplica = SystemReplicas.getSystemReplica(FBUtilities.getBroadcastAddressAndPort()); + + ReplicaLayout.ForTokenWrite liveAndDown = ReplicaLayout.forTokenWrite( + EndpointsForToken.of(token, localSystemReplica), + EndpointsForToken.empty(token) + ); + + return forWrite(systemKeypsace, ConsistencyLevel.ONE, liveAndDown, liveAndDown, writeAll); + } + /** * Requires that the provided endpoints are alive. Converts them to their relevant system replicas. * Note that the liveAndDown collection and live are equal to the provided endpoints. - * - * The semantics are a bit weird, in that CL=ONE iff we have one node provided, and otherwise is equal to TWO. - * How these CL were chosen, and why we drop the CL if only one live node is available, are both unclear. */ - public static ReplicaPlan.ForTokenWrite forBatchlogWrite(Keyspace keyspace, Collection<InetAddressAndPort> endpoints) throws UnavailableException + public static ReplicaPlan.ForTokenWrite forBatchlogWrite(String localDataCenter, ConsistencyLevel consistencyLevel) throws UnavailableException { // A single case we write not for range or token, but multiple mutations to many tokens Token token = DatabaseDescriptor.getPartitioner().getMinimumToken(); + TokenMetadata.Topology topology = StorageService.instance.getTokenMetadata().cachedOnlyTokenMap().getTopology(); + Multimap<String, InetAddressAndPort> localEndpoints = HashMultimap.create(topology.getDatacenterRacks().get(localDataCenter)); + String localRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddressAndPort()); + + // Replicas are picked manually: + // - replicas should be alive according to the failure detector + // - replicas should be in the local datacenter + // - choose min(2, number of qualifying candiates above) + // - allow the local node to be the only replica only if it's a single-node DC + Collection<InetAddressAndPort> chosenEndpoints = new BatchlogManager.EndpointFilter(localRack, localEndpoints).filter(); + + if (chosenEndpoints.isEmpty()) + { + if (consistencyLevel == ConsistencyLevel.ANY) + chosenEndpoints = Collections.singleton(FBUtilities.getBroadcastAddressAndPort()); + else + throw UnavailableException.create(ConsistencyLevel.ONE, 1, 0); + } + ReplicaLayout.ForTokenWrite liveAndDown = ReplicaLayout.forTokenWrite( - SystemReplicas.getSystemReplicas(endpoints).forToken(token), + SystemReplicas.getSystemReplicas(chosenEndpoints).forToken(token), EndpointsForToken.empty(token) ); - ConsistencyLevel consistencyLevel = liveAndDown.all().size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO; + + // Batchlog is hosted by either one node or two nodes from different racks. + consistencyLevel = liveAndDown.all().size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO; --- End diff -- Are we sure it's OK (although consistent with prior behaviour), to only require CL.ONE if all nodes besides us are down? The logic above suggests we only require CL.ONE if we're a single-node DC, but we could have multiple failing nodes in our DC, and be the only one left. It seems like we should probably claim insufficient consistency in this case, though this question probably deserves a separate ticket for discussion.
--- --------------------------------------------------------------------- To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org For additional commands, e-mail: pr-h...@cassandra.apache.org