cassandra-pr mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bdeggleston <...@git.apache.org>
Subject [GitHub] cassandra pull request #224: 14405 replicas
Date Fri, 18 May 2018 21:26:31 GMT
Github user bdeggleston commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/224#discussion_r189397262
  
    --- Diff: src/java/org/apache/cassandra/locator/Replica.java ---
    @@ -0,0 +1,221 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.cassandra.locator;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Objects;
    +import java.util.Set;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Iterables;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import org.apache.cassandra.db.ConsistencyLevel;
    +import org.apache.cassandra.dht.Range;
    +import org.apache.cassandra.dht.Token;
    +import org.apache.cassandra.exceptions.UnavailableException;
    +
    +/**
    + * Decorated Endpoint
    + */
    +public class Replica
    +{
    +    private final InetAddressAndPort endpoint;
    +    private final Range<Token> range;
    +    private final boolean full;
    +
    +    public Replica(InetAddressAndPort endpoint, Range<Token> range, boolean full)
    +    {
    +        Preconditions.checkNotNull(endpoint);
    +        this.endpoint = endpoint;
    +        this.range = range;
    +        this.full = full;
    +    }
    +
    +    public Replica(InetAddressAndPort endpoint, Token start, Token end, boolean full)
    +    {
    +        this(endpoint, new Range<>(start, end), full);
    +    }
    +
    +    public boolean equals(Object o)
    +    {
    +        if (this == o) return true;
    +        if (o == null || getClass() != o.getClass()) return false;
    +        Replica replica = (Replica) o;
    +        return full == replica.full &&
    +               Objects.equals(endpoint, replica.endpoint) &&
    +               Objects.equals(range, replica.range);
    +    }
    +
    +    public int hashCode()
    +    {
    +
    +        return Objects.hash(endpoint, range, full);
    +    }
    +
    +    @Override
    +    public String toString()
    +    {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(full ? "Full" : "Transient");
    +        sb.append('(').append(getEndpoint()).append(',').append(range).append(')');
    +        return sb.toString();
    +    }
    +
    +    public final InetAddressAndPort getEndpoint()
    +    {
    +        return endpoint;
    +    }
    +
    +    public Range<Token> getRange()
    +    {
    +        return range;
    +    }
    +
    +    public boolean isFull()
    +    {
    +        return full;
    +    }
    +
    +    public final boolean isTransient()
    +    {
    +        return !isFull();
    +    }
    +
    +    public ReplicaSet subtract(Replica that)
    +    {
    +        assert isFull() && that.isFull();  // FIXME: this
    +        Set<Range<Token>> ranges = range.subtract(that.range);
    +        ReplicaSet replicatedRanges = new ReplicaSet(ranges.size());
    +        for (Range<Token> range: ranges)
    +        {
    +            replicatedRanges.add(new Replica(getEndpoint(), range, isFull()));
    +        }
    +        return replicatedRanges;
    +    }
    +
    +    /**
    +     * Subtract the ranges of the given replicas from the range of this replica,
    +     * returning a set of replicas with the endpoint and transient information of
    +     * this replica, and the ranges resulting from the subtraction.
    +     */
    +    public ReplicaSet subtractByRange(Replicas toSubtract)
    +    {
    +        if (isFull() && Iterables.all(toSubtract, Replica::isFull))
    +        {
    +            Set<Range<Token>> subtractedRanges = getRange().subtractAll(toSubtract.asRangeSet());
    +            ReplicaSet replicaSet = new ReplicaSet(subtractedRanges.size());
    +            for (Range<Token> range: subtractedRanges)
    +            {
    +                replicaSet.add(new Replica(getEndpoint(), range, isFull()));
    +            }
    +            return replicaSet;
    +        }
    +        else
    +        {
    +            // FIXME: add support for transient replicas
    +            throw new UnsupportedOperationException("transient replicas are currently
unsupported");
    +        }
    +    }
    +
    +    public ReplicaList normalizeByRange()
    +    {
    +        List<Range<Token>> normalized = Range.normalize(Collections.singleton(getRange()));
    +        ReplicaList replicas = new ReplicaList(normalized.size());
    +        for (Range<Token> normalizedRange: normalized)
    +        {
    +            replicas.add(new Replica(getEndpoint(), normalizedRange, isFull()));
    +        }
    +        return replicas;
    +    }
    +
    +    public boolean contains(Range<Token> that)
    +    {
    +        return getRange().contains(that);
    +    }
    +
    +    public boolean intersectsOnRange(Replica replica)
    +    {
    +        return getRange().intersects(replica.getRange());
    +    }
    +
    +    public Replica decorateSubrange(Range<Token> subrange)
    +    {
    +        Preconditions.checkArgument(range.contains(subrange));
    +        return new Replica(getEndpoint(), subrange, isFull());
    +    }
    +
    +    public static Replica full(InetAddressAndPort endpoint, Range<Token> range)
    +    {
    +        return new Replica(endpoint, range, true);
    +    }
    +
    +    /**
    +     * We need to assume an endpoint is a full replica in a with unknown ranges in a
    +     * few cases, so this returns one that throw an exception if you try to get it's
range
    +     */
    +    public static Replica fullStandin(InetAddressAndPort endpoint)
    --- End diff --
    
    I agree it sucks. The problem is that there batchlog and hint related functions that don't
work particularly well with the Replica idea, but use the snitch and some write path stuff
that were converted to use Replicas to prevent a lot of conversion/copying on the hot path.
So using fake replicas solved that problem, but not that well. Let me take another look at
the places that use stand ins though, there's probably something less crappy that can be done.


---

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


Mime
View raw message