You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/06/21 15:07:00 UTC

[jira] [Commented] (CASSANDRA-8346) Paxos operation can use stale data during multiple range movements

    [ https://issues.apache.org/jira/browse/CASSANDRA-8346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16519472#comment-16519472 ] 

ASF GitHub Bot commented on CASSANDRA-8346:
-------------------------------------------

Github user ifesdjeen commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/224#discussion_r197133241
  
    --- Diff: src/java/org/apache/cassandra/service/StorageProxy.java ---
    @@ -344,47 +343,43 @@ private static void recordCasContention(int contentions)
                 casWriteMetrics.contention.update(contentions);
         }
     
    -    private static Predicate<InetAddressAndPort> sameDCPredicateFor(final String dc)
    +    private static Predicate<Replica> sameDCPredicateFor(final String dc)
         {
             final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
    -        return new Predicate<InetAddressAndPort>()
    -        {
    -            public boolean apply(InetAddressAndPort host)
    -            {
    -                return dc.equals(snitch.getDatacenter(host));
    -            }
    -        };
    +        return replica -> dc.equals(snitch.getDatacenter(replica));
         }
     
         private static PaxosParticipants getPaxosParticipants(TableMetadata metadata, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
         {
             Token tk = key.getToken();
    -        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(metadata.keyspace, tk);
    -        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, metadata.keyspace);
    +        ReplicaList naturalReplicas = StorageService.instance.getNaturalReplicas(metadata.keyspace, tk);
    +        ReplicaList pendingReplicas = new ReplicaList(StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, metadata.keyspace));
             if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
             {
    -            // Restrict naturalEndpoints and pendingEndpoints to node in the local DC only
    +            // Restrict naturalReplicas and pendingReplicas to node in the local DC only
                 String localDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
    -            Predicate<InetAddressAndPort> isLocalDc = sameDCPredicateFor(localDc);
    -            naturalEndpoints = ImmutableList.copyOf(Iterables.filter(naturalEndpoints, isLocalDc));
    -            pendingEndpoints = ImmutableList.copyOf(Iterables.filter(pendingEndpoints, isLocalDc));
    +            Predicate<Replica> isLocalDc = sameDCPredicateFor(localDc);
    +            naturalReplicas = ReplicaList.immutableCopyOf(naturalReplicas.filter(isLocalDc));
    +            pendingReplicas = ReplicaList.immutableCopyOf(pendingReplicas.filter(isLocalDc));
             }
    -        int participants = pendingEndpoints.size() + naturalEndpoints.size();
    +        int participants = pendingReplicas.size() + naturalReplicas.size();
             int requiredParticipants = participants / 2 + 1; // See CASSANDRA-8346, CASSANDRA-833
    -        List<InetAddressAndPort> liveEndpoints = ImmutableList.copyOf(Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), IAsyncCallback.isAlive));
    -        if (liveEndpoints.size() < requiredParticipants)
    -            throw new UnavailableException(consistencyForPaxos, requiredParticipants, liveEndpoints.size());
    +
    +        Replicas concatenated = Replicas.concatNaturalAndPending(naturalReplicas, pendingReplicas);
    +        ReplicaList liveReplicas = ReplicaList.immutableCopyOf(Replicas.filter(concatenated, IAsyncCallback.isReplicaAlive));
    --- End diff --
    
    Same here (with Immutable).


> Paxos operation can use stale data during multiple range movements
> ------------------------------------------------------------------
>
>                 Key: CASSANDRA-8346
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8346
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sylvain Lebresne
>            Assignee: Sylvain Lebresne
>            Priority: Major
>             Fix For: 2.0.12
>
>         Attachments: 8346.txt
>
>
> Paxos operations correctly account for pending ranges for all operation pertaining to the Paxos state, but those pending ranges are not taken into account when reading the data to check for the conditions or during a serial read. It's thus possible to break the LWT guarantees by reading a stale value.  This require 2 node movements (on the same token range) to be a problem though.
> Basically, we have {{RF}} replicas + {{P}} pending nodes. For the Paxos prepare/propose phases, the number of required participants (the "Paxos QUORUM") is {{(RF + P + 1) / 2}} ({{SP.getPaxosParticipants}}), but the read done to check conditions or for serial reads is done at a "normal" QUORUM (or LOCAL_QUORUM), and so a weaker {{(RF + 1) / 2}}. We have a problem if it's possible that said read can read only from nodes that were not part of the paxos participants, and so we have a problem if:
> {noformat}
> "normal quorum" == (RF + 1) / 2 <= (RF + P) - ((RF + P + 1) / 2) == "participants considered - blocked for"
> {noformat}
> We're good if {{P = 0}} or {{P = 1}} since this inequality gives us respectively {{RF + 1 <= RF - 1}} and {{RF + 1 <= RF}}, both of which are impossible. But at {{P = 2}} (2 pending nodes), this inequality is equivalent to {{RF <= RF}} and so we might read stale data.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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