cassandra-pr mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From belliottsmith <...@git.apache.org>
Subject [GitHub] cassandra pull request #267: Consolidate batch write code
Date Thu, 20 Sep 2018 11:54:17 GMT
Github user belliottsmith commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/267#discussion_r219129563
  
    --- 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();
    --- End diff --
    
    should EndpointFilter be brought into ReplicaPlans?  It's not used anywhere else, and
it seems that the two pieces of logic should be proximal


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


Mime
View raw message