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

    https://github.com/apache/cassandra/pull/265#discussion_r217112723
  
    --- Diff: src/java/org/apache/cassandra/locator/ReplicaLayout.java ---
    @@ -18,364 +18,296 @@
     
     package org.apache.cassandra.locator;
     
    -import java.util.Collection;
    -import java.util.HashSet;
    -import java.util.List;
    -import java.util.Set;
    -import java.util.function.Predicate;
    -
    -import com.google.common.annotations.VisibleForTesting;
    -import com.google.common.base.Predicates;
    -import com.google.common.collect.Iterables;
    -
     import org.apache.cassandra.config.DatabaseDescriptor;
    -import org.apache.cassandra.db.ConsistencyLevel;
    -import org.apache.cassandra.db.DecoratedKey;
     import org.apache.cassandra.db.Keyspace;
     import org.apache.cassandra.db.PartitionPosition;
     import org.apache.cassandra.dht.AbstractBounds;
     import org.apache.cassandra.dht.Token;
    -import org.apache.cassandra.exceptions.UnavailableException;
     import org.apache.cassandra.gms.FailureDetector;
    -import org.apache.cassandra.net.IAsyncCallback;
    -import org.apache.cassandra.service.StorageProxy;
     import org.apache.cassandra.service.StorageService;
    -import org.apache.cassandra.service.reads.AlwaysSpeculativeRetryPolicy;
    -import org.apache.cassandra.service.reads.SpeculativeRetryPolicy;
     import org.apache.cassandra.utils.FBUtilities;
     
    -import static com.google.common.collect.Iterables.any;
    +import java.util.Set;
    +import java.util.function.Predicate;
     
     /**
    - * Encapsulates knowledge about the ring necessary for performing a 
specific operation, with static accessors
    - * for building the relevant layout.
    + * The relevant replicas for an operation over a given range or token.
      *
    - * Constitutes:
    - *  - the 'natural' replicas replicating the range or token relevant for 
the operation
    - *  - if for performing a write, any 'pending' replicas that are taking 
ownership of the range, and must receive updates
    - *  - the 'selected' replicas, those that should be targeted for any 
operation
    - *  - 'all' replicas represents natural+pending
    - *
    - * @param <E> the type of Endpoints this ReplayLayout holds (either 
EndpointsForToken or EndpointsForRange)
    - * @param <L> the type of itself, including its type parameters, for 
return type of modifying methods
    + * @param <E>
      */
    -public abstract class ReplicaLayout<E extends Endpoints<E>, L extends 
ReplicaLayout<E, L>>
    +public abstract class ReplicaLayout<E extends Endpoints<E>>
     {
    -    private volatile E all;
    -    protected final E natural;
    -    protected final E pending;
    -    protected final E selected;
    -
    -    protected final Keyspace keyspace;
    -    protected final ConsistencyLevel consistencyLevel;
    -
    -    private ReplicaLayout(Keyspace keyspace, ConsistencyLevel 
consistencyLevel, E natural, E pending, E selected)
    -    {
    -        this(keyspace, consistencyLevel, natural, pending, selected, null);
    -    }
    +    private final E natural;
     
    -    private ReplicaLayout(Keyspace keyspace, ConsistencyLevel 
consistencyLevel, E natural, E pending, E selected, E all)
    +    ReplicaLayout(E natural)
         {
    -        assert selected != null;
    -        assert pending == null || !Endpoints.haveConflicts(natural, 
pending);
    -        this.keyspace = keyspace;
    -        this.consistencyLevel = consistencyLevel;
             this.natural = natural;
    -        this.pending = pending;
    -        this.selected = selected;
    -        // if we logically have no pending endpoints (they are null), then 
'all' our endpoints are natural
    -        if (all == null && pending == null)
    -            all = natural;
    -        this.all = all;
         }
     
    -    public Replica getReplicaFor(InetAddressAndPort endpoint)
    -    {
    -        return natural.byEndpoint().get(endpoint);
    -    }
    -
    -    public E natural()
    +    /**
    +     * The 'natural' owners of the ring position(s), as implied by the 
current ring layout.
    +     * This excludes any pending owners, i.e. those that are in the 
process of taking ownership of a range, but
    +     * have not yet finished obtaining their view of the range.
    +     */
    +    public final E natural()
         {
             return natural;
         }
     
    -    public E all()
    -    {
    -        E result = all;
    -        if (result == null)
    -            all = result = Endpoints.concat(natural, pending);
    -        return result;
    -    }
    -
    -    public E selected()
    -    {
    -        return selected;
    -    }
    -
         /**
    -     * @return the pending replicas - will be null for read layouts
    -     * TODO: ideally we would enforce at compile time that read layouts 
have no pending to access
    +     * All relevant owners of the ring position(s) for this operation, as 
implied by the current ring layout.
    +     * For writes, this will include pending owners, and for reads it will 
be equivalent to natural()
          */
    -    public E pending()
    -    {
    -        return pending;
    -    }
    -
    -    public int blockFor()
    -    {
    -        return pending == null
    -                ? consistencyLevel.blockFor(keyspace)
    -                : consistencyLevel.blockForWrite(keyspace, pending);
    -    }
    -
    -    public Keyspace keyspace()
    +    public E all()
         {
    -        return keyspace;
    +        return natural;
         }
     
    -    public ConsistencyLevel consistencyLevel()
    +    public String toString()
         {
    -        return consistencyLevel;
    +        return "ReplicaLayout [ natural: " + natural + " ]";
         }
     
    -    abstract public L withSelected(E replicas);
    -
    -    abstract public L withConsistencyLevel(ConsistencyLevel cl);
    -
    -    public L forNaturalUncontacted()
    +    public static class ForTokenRead extends 
ReplicaLayout<EndpointsForToken> implements ForToken
         {
    -        E more;
    -        if (consistencyLevel.isDatacenterLocal() && 
keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
    +        public ForTokenRead(EndpointsForToken natural)
             {
    -            IEndpointSnitch snitch = 
keyspace.getReplicationStrategy().snitch;
    -            String localDC = DatabaseDescriptor.getLocalDataCenter();
    -
    -            more = natural.filter(replica -> !selected.contains(replica) &&
    -                    snitch.getDatacenter(replica).equals(localDC));
    -        } else
    +            super(natural);
    +        }
    +        @Override
    +        public Token token()
             {
    -            more = natural.filter(replica -> !selected.contains(replica));
    +            return natural().token();
             }
     
    -        return withSelected(more);
    +        public ReplicaLayout.ForTokenRead filter(Predicate<Replica> filter)
    +        {
    +            EndpointsForToken filtered = natural().filter(filter);
    +            if (filtered == natural()) return this;
    +            return new ReplicaLayout.ForTokenRead(filtered);
    +        }
         }
     
    -    public static class ForRange extends ReplicaLayout<EndpointsForRange, 
ForRange>
    +    public static class ForRangeRead extends 
ReplicaLayout<EndpointsForRange> implements ForRange
         {
    -        public final AbstractBounds<PartitionPosition> range;
    +        final AbstractBounds<PartitionPosition> range;
     
    -        @VisibleForTesting
    -        public ForRange(Keyspace keyspace, ConsistencyLevel 
consistencyLevel, AbstractBounds<PartitionPosition> range, EndpointsForRange 
natural, EndpointsForRange selected)
    +        public ForRangeRead(AbstractBounds<PartitionPosition> range, 
EndpointsForRange natural)
             {
    -            // Range queries do not contact pending replicas
    -            super(keyspace, consistencyLevel, natural, null, selected);
    +            super(natural);
                 this.range = range;
             }
     
             @Override
    -        public ForRange withSelected(EndpointsForRange newSelected)
    +        public AbstractBounds<PartitionPosition> range()
    --- End diff --
    
    It wasn't unused, but it can be made so - and it is now.  But for 
consistency with ReplicaLayout.ForTokenRead, I think it's fairly harmless.
    
    A wider question is whether we actually want a ReplicaLayout for reads at 
all.  They're only a single Endpoints, and I do wonder if it's worth it.  But, 
again, for consistency with ReplicaPlan I think it's a very small price to pay.


---

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

Reply via email to