You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by aw...@apache.org on 2018/09/01 01:35:53 UTC

[01/18] cassandra git commit: Transient Replication and Cheap Quorums

Repository: cassandra
Updated Branches:
  refs/heads/trunk 5b645de13 -> f7431b432


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/DataResolverTransientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/DataResolverTransientTest.java b/test/unit/org/apache/cassandra/service/reads/DataResolverTransientTest.java
new file mode 100644
index 0000000..8119400
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/reads/DataResolverTransientTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.service.reads;
+
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.primitives.Ints;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.DecoratedKey;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.EmptyIterators;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.SimpleBuilders;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.reads.repair.TestableReadRepair;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+import static org.apache.cassandra.locator.ReplicaUtils.full;
+import static org.apache.cassandra.locator.ReplicaUtils.trans;
+
+/**
+ * Tests DataResolvers handing of transient replicas
+ */
+public class DataResolverTransientTest extends AbstractReadResponseTest
+{
+    private static DecoratedKey key;
+
+    @Before
+    public void setUp()
+    {
+        key = Util.dk("key1");
+    }
+
+    private static PartitionUpdate.Builder update(TableMetadata metadata, String key, Row... rows)
+    {
+        PartitionUpdate.Builder builder = new PartitionUpdate.Builder(metadata, dk(key), metadata.regularAndStaticColumns(), rows.length, false);
+        for (Row row: rows)
+        {
+            builder.add(row);
+        }
+        return builder;
+    }
+
+    private static PartitionUpdate.Builder update(Row... rows)
+    {
+        return update(cfm, "key1", rows);
+    }
+
+    private static Row.SimpleBuilder rowBuilder(int clustering)
+    {
+        return new SimpleBuilders.RowBuilder(cfm, Integer.toString(clustering));
+    }
+
+    private static Row row(long timestamp, int clustering, int value)
+    {
+        return rowBuilder(clustering).timestamp(timestamp).add("c1", Integer.toString(value)).build();
+    }
+
+    private static DeletionTime deletion(long timeMillis)
+    {
+        TimeUnit MILLIS = TimeUnit.MILLISECONDS;
+        return new DeletionTime(MILLIS.toMicros(timeMillis), Ints.checkedCast(MILLIS.toSeconds(timeMillis)));
+    }
+
+    /**
+     * Tests that the given update doesn't cause data resolver to attempt to repair a transient replica
+     */
+    private void assertNoTransientRepairs(PartitionUpdate update)
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(update.metadata(), nowInSec, key);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(key.getToken(), full(EP1), full(EP2), trans(EP3));
+        TestableReadRepair repair = new TestableReadRepair(command, QUORUM);
+        DataResolver resolver = new DataResolver(command, plan(targetReplicas, ConsistencyLevel.QUORUM), repair, 0);
+
+        Assert.assertFalse(resolver.isDataPresent());
+        resolver.preprocess(response(command, EP1, iter(update), false));
+        resolver.preprocess(response(command, EP2, iter(update), false));
+        resolver.preprocess(response(command, EP3, EmptyIterators.unfilteredPartition(update.metadata()), false));
+
+        Assert.assertFalse(repair.dataWasConsumed());
+        assertPartitionsEqual(filter(iter(update)), resolver.resolve());
+        Assert.assertTrue(repair.dataWasConsumed());
+        Assert.assertTrue(repair.sent.toString(), repair.sent.isEmpty());
+    }
+
+    @Test
+    public void emptyRowRepair()
+    {
+        assertNoTransientRepairs(update(row(1000, 4, 4), row(1000, 5, 5)).build());
+    }
+
+    @Test
+    public void emptyPartitionDeletionRepairs()
+    {
+        PartitionUpdate.Builder builder = update();
+        builder.addPartitionDeletion(deletion(1999));
+        assertNoTransientRepairs(builder.build());
+    }
+
+    /**
+     * Partition level deletion responses shouldn't sent data to a transient replica
+     */
+    @Test
+    public void emptyRowDeletionRepairs()
+    {
+        PartitionUpdate.Builder builder = update();
+        builder.add(rowBuilder(1).timestamp(1999).delete().build());
+        assertNoTransientRepairs(builder.build());
+    }
+
+    @Test
+    public void emptyComplexDeletionRepair()
+    {
+
+        long[] ts = {1000, 2000};
+
+        Row.Builder builder = BTreeRow.unsortedBuilder();
+        builder.newRow(Clustering.EMPTY);
+        builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
+        assertNoTransientRepairs(update(cfm2, "key", builder.build()).build());
+
+    }
+
+    @Test
+    public void emptyRangeTombstoneRepairs()
+    {
+        Slice slice = Slice.make(Clustering.make(ByteBufferUtil.bytes("a")), Clustering.make(ByteBufferUtil.bytes("b")));
+        PartitionUpdate.Builder builder = update();
+        builder.add(new RangeTombstone(slice, deletion(2000)));
+        assertNoTransientRepairs(builder.build());
+    }
+
+    /**
+     * If the full replicas need to repair each other, repairs shouldn't be sent to transient replicas
+     */
+    @Test
+    public void fullRepairsIgnoreTransientReplicas()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk(5));
+        EndpointsForToken targetReplicas = EndpointsForToken.of(key.getToken(), full(EP1), full(EP2), trans(EP3));
+        TestableReadRepair repair = new TestableReadRepair(command, QUORUM);
+        DataResolver resolver = new DataResolver(command, plan(targetReplicas, QUORUM), repair, 0);
+
+        Assert.assertFalse(resolver.isDataPresent());
+        resolver.preprocess(response(command, EP1, iter(update(row(1000, 5, 5)).build()), false));
+        resolver.preprocess(response(command, EP2, iter(update(row(2000, 4, 4)).build()), false));
+        resolver.preprocess(response(command, EP3, EmptyIterators.unfilteredPartition(cfm), false));
+
+        Assert.assertFalse(repair.dataWasConsumed());
+
+        consume(resolver.resolve());
+
+        Assert.assertTrue(repair.dataWasConsumed());
+
+        Assert.assertTrue(repair.sent.containsKey(EP1));
+        Assert.assertTrue(repair.sent.containsKey(EP2));
+        Assert.assertFalse(repair.sent.containsKey(EP3));
+    }
+
+    /**
+     * If the transient replica has new data, the full replicas shoould be repaired, the transient one should not
+     */
+    @Test
+    public void transientMismatchesRepairFullReplicas()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk(5));
+        EndpointsForToken targetReplicas = EndpointsForToken.of(key.getToken(), full(EP1), full(EP2), trans(EP3));
+        TestableReadRepair<?, ?> repair = new TestableReadRepair(command, QUORUM);
+        DataResolver resolver = new DataResolver(command, plan(targetReplicas, QUORUM), repair, 0);
+
+        Assert.assertFalse(resolver.isDataPresent());
+        PartitionUpdate transData = update(row(1000, 5, 5)).build();
+        resolver.preprocess(response(command, EP1, EmptyIterators.unfilteredPartition(cfm), false));
+        resolver.preprocess(response(command, EP2, EmptyIterators.unfilteredPartition(cfm), false));
+        resolver.preprocess(response(command, EP3, iter(transData), false));
+
+        Assert.assertFalse(repair.dataWasConsumed());
+
+        assertPartitionsEqual(filter(iter(transData)), resolver.resolve());
+
+        Assert.assertTrue(repair.dataWasConsumed());
+
+        assertPartitionsEqual(filter(iter(transData)), filter(iter(repair.sent.get(EP1).getPartitionUpdate(cfm))));
+        assertPartitionsEqual(filter(iter(transData)), filter(iter(repair.sent.get(EP2).getPartitionUpdate(cfm))));
+        Assert.assertFalse(repair.sent.containsKey(EP3));
+
+    }
+
+    private ReplicaLayout.ForToken plan(EndpointsForToken replicas, ConsistencyLevel consistencyLevel)
+    {
+        return new ReplicaLayout.ForToken(ks, consistencyLevel, replicas.token(), replicas, null, replicas);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java b/test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
new file mode 100644
index 0000000..5306a74
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.service.reads;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.SimpleBuilders;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.reads.repair.NoopReadRepair;
+import org.apache.cassandra.service.reads.repair.TestableReadRepair;
+
+import static org.apache.cassandra.locator.ReplicaUtils.full;
+import static org.apache.cassandra.locator.ReplicaUtils.trans;
+
+public class DigestResolverTest extends AbstractReadResponseTest
+{
+    private static PartitionUpdate.Builder update(TableMetadata metadata, String key, Row... rows)
+    {
+        PartitionUpdate.Builder builder = new PartitionUpdate.Builder(metadata, dk(key), metadata.regularAndStaticColumns(), rows.length, false);
+        for (Row row: rows)
+        {
+            builder.add(row);
+        }
+        return builder;
+    }
+
+    private static PartitionUpdate.Builder update(Row... rows)
+    {
+        return update(cfm, "key1", rows);
+    }
+
+    private static Row row(long timestamp, int clustering, int value)
+    {
+        SimpleBuilders.RowBuilder builder = new SimpleBuilders.RowBuilder(cfm, Integer.toString(clustering));
+        builder.timestamp(timestamp).add("c1", Integer.toString(value));
+        return builder.build();
+    }
+
+    @Test
+    public void noRepairNeeded()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        TestableReadRepair readRepair = new TestableReadRepair(command, ConsistencyLevel.QUORUM);
+        DigestResolver resolver = new DigestResolver(command, plan(ConsistencyLevel.QUORUM, targetReplicas), readRepair, 0);
+
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+
+        Assert.assertFalse(resolver.isDataPresent());
+        resolver.preprocess(response(command, EP2, iter(response), true));
+        resolver.preprocess(response(command, EP1, iter(response), false));
+        Assert.assertTrue(resolver.isDataPresent());
+        Assert.assertTrue(resolver.responsesMatch());
+
+        assertPartitionsEqual(filter(iter(response)), resolver.getData());
+    }
+
+    @Test
+    public void digestMismatch()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        DigestResolver resolver = new DigestResolver(command, plan(ConsistencyLevel.QUORUM, targetReplicas), NoopReadRepair.instance,0);
+
+        PartitionUpdate response1 = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        PartitionUpdate response2 = update(row(2000, 4, 5)).build();
+
+        Assert.assertFalse(resolver.isDataPresent());
+        resolver.preprocess(response(command, EP2, iter(response1), true));
+        resolver.preprocess(response(command, EP1, iter(response2), false));
+        Assert.assertTrue(resolver.isDataPresent());
+        Assert.assertFalse(resolver.responsesMatch());
+        Assert.assertFalse(resolver.hasTransientResponse());
+    }
+
+    /**
+     * A full response and a transient response, with the transient response being a subset of the full one
+     */
+    @Test
+    public void agreeingTransient()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), trans(EP2));
+        TestableReadRepair readRepair = new TestableReadRepair(command, ConsistencyLevel.QUORUM);
+        DigestResolver resolver = new DigestResolver(command, plan(ConsistencyLevel.QUORUM, targetReplicas), readRepair, 0);
+
+        PartitionUpdate response1 = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        PartitionUpdate response2 = update(row(1000, 5, 5)).build();
+
+        Assert.assertFalse(resolver.isDataPresent());
+        resolver.preprocess(response(command, EP1, iter(response1), false));
+        resolver.preprocess(response(command, EP2, iter(response2), false));
+        Assert.assertTrue(resolver.isDataPresent());
+        Assert.assertTrue(resolver.responsesMatch());
+        Assert.assertTrue(resolver.hasTransientResponse());
+        Assert.assertTrue(readRepair.sent.isEmpty());
+    }
+
+    /**
+     * Transient responses shouldn't be classified as the single dataResponse
+     */
+    @Test
+    public void transientResponse()
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), trans(EP2));
+        DigestResolver resolver = new DigestResolver(command, plan(ConsistencyLevel.QUORUM, targetReplicas), NoopReadRepair.instance, 0);
+
+        PartitionUpdate response2 = update(row(1000, 5, 5)).build();
+        Assert.assertFalse(resolver.isDataPresent());
+        Assert.assertFalse(resolver.hasTransientResponse());
+        resolver.preprocess(response(command, EP2, iter(response2), false));
+        Assert.assertFalse(resolver.isDataPresent());
+        Assert.assertTrue(resolver.hasTransientResponse());
+    }
+
+    private ReplicaLayout.ForToken plan(ConsistencyLevel consistencyLevel, EndpointsForToken replicas)
+    {
+        return new ReplicaLayout.ForToken(ks, consistencyLevel, replicas.token(), replicas, null, replicas);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java
index de7b2e4..3b102f2 100644
--- a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java
@@ -18,10 +18,12 @@
 
 package org.apache.cassandra.service.reads;
 
-import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.ImmutableList;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -35,12 +37,15 @@ import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.service.reads.AbstractReadExecutor;
 
+import static org.apache.cassandra.locator.ReplicaUtils.full;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -48,7 +53,8 @@ public class ReadExecutorTest
 {
     static Keyspace ks;
     static ColumnFamilyStore cfs;
-    static List<InetAddressAndPort> targets;
+    static EndpointsForToken targets;
+    static Token dummy;
 
     @BeforeClass
     public static void setUpClass() throws Throwable
@@ -57,8 +63,13 @@ public class ReadExecutorTest
         SchemaLoader.createKeyspace("Foo", KeyspaceParams.simple(3), SchemaLoader.standardCFMD("Foo", "Bar"));
         ks = Keyspace.open("Foo");
         cfs = ks.getColumnFamilyStore("Bar");
-        targets = ImmutableList.of(InetAddressAndPort.getByName("127.0.0.255"), InetAddressAndPort.getByName("127.0.0.254"), InetAddressAndPort.getByName("127.0.0.253"));
-        cfs.sampleLatencyNanos = 0;
+        dummy = Murmur3Partitioner.instance.getMinimumToken();
+        targets = EndpointsForToken.of(dummy,
+                full(InetAddressAndPort.getByName("127.0.0.255")),
+                full(InetAddressAndPort.getByName("127.0.0.254")),
+                full(InetAddressAndPort.getByName("127.0.0.253"))
+        );
+        cfs.sampleReadLatencyNanos = 0;
     }
 
     @Before
@@ -78,7 +89,7 @@ public class ReadExecutorTest
     {
         assertEquals(0, cfs.metric.speculativeInsufficientReplicas.getCount());
         assertEquals(0, ks.metric.speculativeInsufficientReplicas.getCount());
-        AbstractReadExecutor executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime(), true);
+        AbstractReadExecutor executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(cfs, new MockSinglePartitionReadCommand(), plan(targets, ConsistencyLevel.LOCAL_QUORUM), System.nanoTime(), true);
         executor.maybeTryAdditionalReplicas();
         try
         {
@@ -93,7 +104,7 @@ public class ReadExecutorTest
         assertEquals(1, ks.metric.speculativeInsufficientReplicas.getCount());
 
         //Shouldn't increment
-        executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime(), false);
+        executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(cfs, new MockSinglePartitionReadCommand(), plan(targets, ConsistencyLevel.LOCAL_QUORUM), System.nanoTime(), false);
         executor.maybeTryAdditionalReplicas();
         try
         {
@@ -119,7 +130,7 @@ public class ReadExecutorTest
         assertEquals(0, cfs.metric.speculativeFailedRetries.getCount());
         assertEquals(0, ks.metric.speculativeRetries.getCount());
         assertEquals(0, ks.metric.speculativeFailedRetries.getCount());
-        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(TimeUnit.DAYS.toMillis(365)), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime());
+        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(cfs, new MockSinglePartitionReadCommand(TimeUnit.DAYS.toMillis(365)), plan(ConsistencyLevel.LOCAL_QUORUM, targets, targets.subList(0, 2)), System.nanoTime());
         executor.maybeTryAdditionalReplicas();
         new Thread()
         {
@@ -127,8 +138,8 @@ public class ReadExecutorTest
             public void run()
             {
                 //Failures end the read promptly but don't require mock data to be suppleid
-                executor.handler.onFailure(targets.get(0), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
-                executor.handler.onFailure(targets.get(1), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
+                executor.handler.onFailure(targets.get(0).endpoint(), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
+                executor.handler.onFailure(targets.get(1).endpoint(), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
                 executor.handler.condition.signalAll();
             }
         }.start();
@@ -160,7 +171,7 @@ public class ReadExecutorTest
         assertEquals(0, cfs.metric.speculativeFailedRetries.getCount());
         assertEquals(0, ks.metric.speculativeRetries.getCount());
         assertEquals(0, ks.metric.speculativeFailedRetries.getCount());
-        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime());
+        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(cfs, new MockSinglePartitionReadCommand(), plan(ConsistencyLevel.LOCAL_QUORUM, targets, targets.subList(0, 2)), System.nanoTime());
         executor.maybeTryAdditionalReplicas();
         try
         {
@@ -188,7 +199,7 @@ public class ReadExecutorTest
 
         MockSinglePartitionReadCommand(long timeout)
         {
-            super(false, 0, cfs.metadata(), 0, null, null, null, Util.dk("ry@n_luvs_teh_y@nk33z"), null, null);
+            super(false, 0, false, cfs.metadata(), 0, null, null, null, Util.dk("ry@n_luvs_teh_y@nk33z"), null, null);
             this.timeout = timeout;
         }
 
@@ -213,4 +224,13 @@ public class ReadExecutorTest
 
     }
 
+    private ReplicaLayout.ForToken plan(EndpointsForToken targets, ConsistencyLevel consistencyLevel)
+    {
+        return plan(consistencyLevel, targets, targets);
+    }
+
+    private ReplicaLayout.ForToken plan(ConsistencyLevel consistencyLevel, EndpointsForToken natural, EndpointsForToken selected)
+    {
+        return new ReplicaLayout.ForToken(ks, consistencyLevel, natural.token(), natural, null, selected);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java
index 9717c4e..7e6ee29 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java
@@ -8,7 +8,6 @@ import java.util.function.Consumer;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
 import org.junit.Assert;
@@ -39,7 +38,11 @@ import org.apache.cassandra.db.rows.BufferCell;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceMetadata;
@@ -49,6 +52,9 @@ import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.ReplicaUtils.FULL_RANGE;
+
 @Ignore
 public abstract  class AbstractReadRepairTest
 {
@@ -60,6 +66,12 @@ public abstract  class AbstractReadRepairTest
     static InetAddressAndPort target3;
     static List<InetAddressAndPort> targets;
 
+    static Replica replica1;
+    static Replica replica2;
+    static Replica replica3;
+    static EndpointsForRange replicas;
+    static ReplicaLayout<?, ?> replicaLayout;
+
     static long now = TimeUnit.NANOSECONDS.toMicros(System.nanoTime());
     static DecoratedKey key;
     static Cell cell1;
@@ -191,7 +203,8 @@ public abstract  class AbstractReadRepairTest
         ks = Keyspace.open(ksName);
         cfs = ks.getColumnFamilyStore("tbl");
 
-        cfs.sampleLatencyNanos = 0;
+        cfs.sampleReadLatencyNanos = 0;
+        cfs.transientWriteLatencyNanos = 0;
 
         target1 = InetAddressAndPort.getByName("127.0.0.255");
         target2 = InetAddressAndPort.getByName("127.0.0.254");
@@ -199,6 +212,13 @@ public abstract  class AbstractReadRepairTest
 
         targets = ImmutableList.of(target1, target2, target3);
 
+        replica1 = fullReplica(target1, FULL_RANGE);
+        replica2 = fullReplica(target2, FULL_RANGE);
+        replica3 = fullReplica(target3, FULL_RANGE);
+        replicas = EndpointsForRange.of(replica1, replica2, replica3);
+
+        replicaLayout = replicaLayout(ConsistencyLevel.QUORUM, replicas);
+
         // default test values
         key  = dk(5);
         cell1 = cell("v", "val1", now);
@@ -220,14 +240,26 @@ public abstract  class AbstractReadRepairTest
     public void setUp()
     {
         assert configured : "configureClass must be called in a @BeforeClass method";
-        cfs.sampleLatencyNanos = 0;
+
+        cfs.sampleReadLatencyNanos = 0;
+        cfs.transientWriteLatencyNanos = 0;
+    }
+
+    static ReplicaLayout.ForRange replicaLayout(EndpointsForRange replicas, EndpointsForRange targets)
+    {
+        return new ReplicaLayout.ForRange(ks, ConsistencyLevel.QUORUM, ReplicaUtils.FULL_BOUNDS, replicas, targets);
+    }
+
+    static ReplicaLayout.ForRange replicaLayout(ConsistencyLevel consistencyLevel, EndpointsForRange replicas)
+    {
+        return new ReplicaLayout.ForRange(ks, consistencyLevel, ReplicaUtils.FULL_BOUNDS, replicas, replicas);
     }
 
-    public abstract InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency);
+    public abstract InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, ReplicaLayout<?, ?> replicaLayout, long queryStartNanoTime);
 
-    public InstrumentedReadRepair createInstrumentedReadRepair()
+    public InstrumentedReadRepair createInstrumentedReadRepair(ReplicaLayout<?, ?> replicaLayout)
     {
-        return createInstrumentedReadRepair(command, System.nanoTime(), ConsistencyLevel.QUORUM);
+        return createInstrumentedReadRepair(command, replicaLayout, System.nanoTime());
 
     }
 
@@ -238,12 +270,11 @@ public abstract  class AbstractReadRepairTest
     @Test
     public void readSpeculationCycle()
     {
-        InstrumentedReadRepair repair = createInstrumentedReadRepair();
+        InstrumentedReadRepair repair = createInstrumentedReadRepair(replicaLayout(replicas, EndpointsForRange.of(replica1, replica2)));
         ResultConsumer consumer = new ResultConsumer();
 
-
         Assert.assertEquals(epSet(), repair.getReadRecipients());
-        repair.startRepair(null, targets, Lists.newArrayList(target1, target2), consumer);
+        repair.startRepair(null, consumer);
 
         Assert.assertEquals(epSet(target1, target2), repair.getReadRecipients());
         repair.maybeSendAdditionalReads();
@@ -258,11 +289,11 @@ public abstract  class AbstractReadRepairTest
     @Test
     public void noSpeculationRequired()
     {
-        InstrumentedReadRepair repair = createInstrumentedReadRepair();
+        InstrumentedReadRepair repair = createInstrumentedReadRepair(replicaLayout(replicas, EndpointsForRange.of(replica1, replica2)));
         ResultConsumer consumer = new ResultConsumer();
 
         Assert.assertEquals(epSet(), repair.getReadRecipients());
-        repair.startRepair(null, targets, Lists.newArrayList(target1, target2), consumer);
+        repair.startRepair(null, consumer);
 
         Assert.assertEquals(epSet(target1, target2), repair.getReadRecipients());
         repair.getReadCallback().response(msg(target1, cell1));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java
index b06e88a..a574d02 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java
@@ -18,10 +18,8 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -31,24 +29,26 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.service.reads.ReadCallback;
 
 public class BlockingReadRepairTest extends AbstractReadRepairTest
 {
-
-    private static class InstrumentedReadRepairHandler extends BlockingPartitionRepair
+    private static class InstrumentedReadRepairHandler<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends BlockingPartitionRepair<E, L>
     {
-        public InstrumentedReadRepairHandler(Keyspace keyspace, DecoratedKey key, ConsistencyLevel consistency, Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor, InetAddressAndPort[] participants)
+        public InstrumentedReadRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor, L replicaLayout)
         {
-            super(keyspace, key, consistency, repairs, maxBlockFor, participants);
+            super(Util.dk("not a real usable value"), repairs, maxBlockFor, replicaLayout);
         }
 
         Map<InetAddressAndPort, Mutation> mutationsSent = new HashMap<>();
@@ -58,13 +58,6 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
             mutationsSent.put(endpoint, message.payload);
         }
 
-        List<InetAddressAndPort> candidates = targets;
-
-        protected List<InetAddressAndPort> getCandidateEndpoints()
-        {
-            return candidates;
-        }
-
         @Override
         protected boolean isLocal(InetAddressAndPort endpoint)
         {
@@ -78,23 +71,22 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
         configureClass(ReadRepairStrategy.BLOCKING);
     }
 
-    private static InstrumentedReadRepairHandler createRepairHandler(Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor, Collection<InetAddressAndPort> participants)
+    private static InstrumentedReadRepairHandler createRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor, ReplicaLayout<?, ?> replicaLayout)
     {
-        InetAddressAndPort[] participantArray = new InetAddressAndPort[participants.size()];
-        participants.toArray(participantArray);
-        return new InstrumentedReadRepairHandler(ks, key, ConsistencyLevel.LOCAL_QUORUM, repairs, maxBlockFor, participantArray);
+        return new InstrumentedReadRepairHandler(repairs, maxBlockFor, replicaLayout);
     }
 
-    private static InstrumentedReadRepairHandler createRepairHandler(Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor)
+    private static InstrumentedReadRepairHandler createRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor)
     {
-        return createRepairHandler(repairs, maxBlockFor, repairs.keySet());
+        EndpointsForRange replicas = EndpointsForRange.copyOf(Lists.newArrayList(repairs.keySet()));
+        return createRepairHandler(repairs, maxBlockFor, replicaLayout(replicas, replicas));
     }
 
-    private static class InstrumentedBlockingReadRepair extends BlockingReadRepair implements InstrumentedReadRepair
+    private static class InstrumentedBlockingReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends BlockingReadRepair<E, L> implements InstrumentedReadRepair<E, L>
     {
-        public InstrumentedBlockingReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+        public InstrumentedBlockingReadRepair(ReadCommand command, L replicaLayout, long queryStartNanoTime)
         {
-            super(command, queryStartNanoTime, consistency);
+            super(command, replicaLayout, queryStartNanoTime);
         }
 
         Set<InetAddressAndPort> readCommandRecipients = new HashSet<>();
@@ -109,12 +101,6 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
         }
 
         @Override
-        Iterable<InetAddressAndPort> getCandidatesForToken(Token token)
-        {
-            return targets;
-        }
-
-        @Override
         public Set<InetAddressAndPort> getReadRecipients()
         {
             return readCommandRecipients;
@@ -128,9 +114,9 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     }
 
     @Override
-    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, ReplicaLayout<?, ?> replicaLayout, long queryStartNanoTime)
     {
-        return new InstrumentedBlockingReadRepair(command, queryStartNanoTime, consistency);
+        return new InstrumentedBlockingReadRepair(command, replicaLayout, queryStartNanoTime);
     }
 
     @Test
@@ -152,12 +138,12 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
         Mutation repair2 = mutation(cell1);
 
         // check that the correct repairs are calculated
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, repair1);
-        repairs.put(target2, repair2);
-
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, repair1);
+        repairs.put(replica2, repair2);
 
-        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2);
+        ReplicaLayout.ForRange replicaLayout = replicaLayout(replicas, EndpointsForRange.copyOf(Lists.newArrayList(repairs.keySet())));
+        InstrumentedReadRepairHandler<?, ?> handler = createRepairHandler(repairs, 2, replicaLayout);
 
         Assert.assertTrue(handler.mutationsSent.isEmpty());
 
@@ -188,9 +174,9 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     @Test
     public void noAdditionalMutationRequired() throws Exception
     {
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, mutation(cell2));
-        repairs.put(target2, mutation(cell1));
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, mutation(cell2));
+        repairs.put(replica2, mutation(cell1));
 
         InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2);
         handler.sendInitialRepairs();
@@ -209,15 +195,14 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     @Test
     public void noAdditionalMutationPossible() throws Exception
     {
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, mutation(cell2));
-        repairs.put(target2, mutation(cell1));
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, mutation(cell2));
+        repairs.put(replica2, mutation(cell1));
 
         InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2);
         handler.sendInitialRepairs();
 
         // we've already sent mutations to all candidates, so we shouldn't send any more
-        handler.candidates = Lists.newArrayList(target1, target2);
         handler.mutationsSent.clear();
         handler.maybeSendAdditionalWrites(0, TimeUnit.NANOSECONDS);
         Assert.assertTrue(handler.mutationsSent.isEmpty());
@@ -232,12 +217,11 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     {
         Mutation repair1 = mutation(cell2);
 
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, repair1);
-        Collection<InetAddressAndPort> participants = Lists.newArrayList(target1, target2);
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, repair1);
 
         // check that the correct initial mutations are sent out
-        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, participants);
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, replicaLayout(replicas, EndpointsForRange.of(replica1, replica2)));
         handler.sendInitialRepairs();
         Assert.assertEquals(1, handler.mutationsSent.size());
         Assert.assertTrue(handler.mutationsSent.containsKey(target1));
@@ -252,10 +236,10 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     @Test
     public void onlyBlockOnQuorum()
     {
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, mutation(cell1));
-        repairs.put(target2, mutation(cell2));
-        repairs.put(target3, mutation(cell3));
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, mutation(cell1));
+        repairs.put(replica2, mutation(cell2));
+        repairs.put(replica3, mutation(cell3));
         Assert.assertEquals(3, repairs.size());
 
         InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2);
@@ -277,30 +261,29 @@ public class BlockingReadRepairTest extends AbstractReadRepairTest
     @Test
     public void remoteDCTest() throws Exception
     {
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, mutation(cell1));
-
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, mutation(cell1));
 
-        InetAddressAndPort remote1 = InetAddressAndPort.getByName("10.0.0.1");
-        InetAddressAndPort remote2 = InetAddressAndPort.getByName("10.0.0.2");
+        Replica remote1 = ReplicaUtils.full(InetAddressAndPort.getByName("10.0.0.1"));
+        Replica remote2 = ReplicaUtils.full(InetAddressAndPort.getByName("10.0.0.2"));
         repairs.put(remote1, mutation(cell1));
 
-        Collection<InetAddressAndPort> participants = Lists.newArrayList(target1, target2, remote1, remote2);
-
-        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, participants);
+        EndpointsForRange participants = EndpointsForRange.of(replica1, replica2, remote1, remote2);
+        ReplicaLayout.ForRange replicaLayout = new ReplicaLayout.ForRange(ks, ConsistencyLevel.LOCAL_QUORUM, ReplicaUtils.FULL_BOUNDS, participants, participants);
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, replicaLayout);
         handler.sendInitialRepairs();
         Assert.assertEquals(2, handler.mutationsSent.size());
-        Assert.assertTrue(handler.mutationsSent.containsKey(target1));
-        Assert.assertTrue(handler.mutationsSent.containsKey(remote1));
+        Assert.assertTrue(handler.mutationsSent.containsKey(replica1.endpoint()));
+        Assert.assertTrue(handler.mutationsSent.containsKey(remote1.endpoint()));
 
         Assert.assertEquals(1, handler.waitingOn());
         Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
 
-        handler.ack(remote1);
+        handler.ack(remote1.endpoint());
         Assert.assertEquals(1, handler.waitingOn());
         Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
 
-        handler.ack(target1);
+        handler.ack(replica1.endpoint());
         Assert.assertEquals(0, handler.waitingOn());
         Assert.assertTrue(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java
index 1f07c2b..c345ee6 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java
@@ -26,20 +26,23 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.Lists;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.OverrideConfigurationLoader;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.diag.DiagnosticEventService;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.service.reads.ReadCallback;
 import org.apache.cassandra.service.reads.repair.ReadRepairEvent.ReadRepairEventType;
@@ -72,12 +75,13 @@ public class DiagEventsBlockingReadRepairTest extends AbstractReadRepairTest
         Mutation repair2 = mutation(cell1);
 
         // check that the correct repairs are calculated
-        Map<InetAddressAndPort, Mutation> repairs = new HashMap<>();
-        repairs.put(target1, repair1);
-        repairs.put(target2, repair2);
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(replica1, repair1);
+        repairs.put(replica2, repair2);
 
 
-        DiagnosticPartitionReadRepairHandler handler = createRepairHandler(repairs, 2);
+        ReplicaLayout.ForRange replicaLayout = replicaLayout(replicas, EndpointsForRange.copyOf(Lists.newArrayList(repairs.keySet())));
+        DiagnosticPartitionReadRepairHandler handler = createRepairHandler(repairs, 2, replicaLayout);
 
         Assert.assertTrue(handler.updatesByEp.isEmpty());
 
@@ -102,17 +106,20 @@ public class DiagEventsBlockingReadRepairTest extends AbstractReadRepairTest
         Assert.assertTrue(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
     }
 
-    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, ReplicaLayout<?, ?> replicaLayout, long queryStartNanoTime)
     {
-        return new DiagnosticBlockingRepairHandler(command, queryStartNanoTime, consistency);
+        return new DiagnosticBlockingRepairHandler(command, replicaLayout, queryStartNanoTime);
     }
 
-    private static DiagnosticPartitionReadRepairHandler createRepairHandler(Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor)
+    private static DiagnosticPartitionReadRepairHandler createRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor, ReplicaLayout<?, ?> replicaLayout)
     {
-        Set<InetAddressAndPort> participants = repairs.keySet();
-        InetAddressAndPort[] participantArray = new InetAddressAndPort[participants.size()];
-        participants.toArray(participantArray);
-        return new DiagnosticPartitionReadRepairHandler(ks, key, ConsistencyLevel.LOCAL_QUORUM, repairs, maxBlockFor, participantArray);
+        return new DiagnosticPartitionReadRepairHandler(key, repairs, maxBlockFor, replicaLayout);
+    }
+
+    private static DiagnosticPartitionReadRepairHandler createRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor)
+    {
+        EndpointsForRange replicas = EndpointsForRange.copyOf(Lists.newArrayList(repairs.keySet()));
+        return createRepairHandler(repairs, maxBlockFor, replicaLayout(replicas, replicas));
     }
 
     private static class DiagnosticBlockingRepairHandler extends BlockingReadRepair implements InstrumentedReadRepair
@@ -120,9 +127,9 @@ public class DiagEventsBlockingReadRepairTest extends AbstractReadRepairTest
         private Set<InetAddressAndPort> recipients = Collections.emptySet();
         private ReadCallback readCallback = null;
 
-        DiagnosticBlockingRepairHandler(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+        DiagnosticBlockingRepairHandler(ReadCommand command, ReplicaLayout<?, ?> replicaLayout, long queryStartNanoTime)
         {
-            super(command, queryStartNanoTime, consistency);
+            super(command, replicaLayout, queryStartNanoTime);
             DiagnosticEventService.instance().subscribe(ReadRepairEvent.class, this::onRepairEvent);
         }
 
@@ -130,7 +137,7 @@ public class DiagEventsBlockingReadRepairTest extends AbstractReadRepairTest
         {
             if (e.getType() == ReadRepairEventType.START_REPAIR) recipients = new HashSet<>(e.destinations);
             else if (e.getType() == ReadRepairEventType.SPECULATED_READ) recipients.addAll(e.destinations);
-            Assert.assertEquals(targets, e.allEndpoints);
+            Assert.assertEquals(new HashSet<>(targets), new HashSet<>(e.allEndpoints));
             Assert.assertNotNull(e.toMap());
         }
 
@@ -156,13 +163,13 @@ public class DiagEventsBlockingReadRepairTest extends AbstractReadRepairTest
         }
     }
 
-    private static class DiagnosticPartitionReadRepairHandler extends BlockingPartitionRepair
+    private static class DiagnosticPartitionReadRepairHandler<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends BlockingPartitionRepair<E, L>
     {
         private final Map<InetAddressAndPort, String> updatesByEp = new HashMap<>();
 
-        DiagnosticPartitionReadRepairHandler(Keyspace keyspace, DecoratedKey key, ConsistencyLevel consistency, Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor, InetAddressAndPort[] participants)
+        DiagnosticPartitionReadRepairHandler(DecoratedKey key, Map<Replica, Mutation> repairs, int maxBlockFor, L replicaLayout)
         {
-            super(keyspace, key, consistency, repairs, maxBlockFor, participants);
+            super(key, repairs, maxBlockFor, replicaLayout);
             DiagnosticEventService.instance().subscribe(PartitionRepairEvent.class, this::onRepairEvent);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/InstrumentedReadRepair.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/InstrumentedReadRepair.java b/test/unit/org/apache/cassandra/service/reads/repair/InstrumentedReadRepair.java
index 2fb8ffc..f3d2866 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/InstrumentedReadRepair.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/InstrumentedReadRepair.java
@@ -20,10 +20,12 @@ package org.apache.cassandra.service.reads.repair;
 
 import java.util.Set;
 
+import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.service.reads.ReadCallback;
 
-public interface InstrumentedReadRepair extends ReadRepair
+public interface InstrumentedReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends ReadRepair<E, L>
 {
     Set<InetAddressAndPort> getReadRecipients();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepairTest.java
index efce59a..9bb7eed 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepairTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepairTest.java
@@ -26,20 +26,20 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.service.reads.ReadCallback;
 
 public class ReadOnlyReadRepairTest extends AbstractReadRepairTest
 {
-    private static class InstrumentedReadOnlyReadRepair extends ReadOnlyReadRepair implements InstrumentedReadRepair
+    private static class InstrumentedReadOnlyReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends ReadOnlyReadRepair implements InstrumentedReadRepair
     {
-        public InstrumentedReadOnlyReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+        public InstrumentedReadOnlyReadRepair(ReadCommand command, L replicaLayout, long queryStartNanoTime)
         {
-            super(command, queryStartNanoTime, consistency);
+            super(command, replicaLayout, queryStartNanoTime);
         }
 
         Set<InetAddressAndPort> readCommandRecipients = new HashSet<>();
@@ -54,12 +54,6 @@ public class ReadOnlyReadRepairTest extends AbstractReadRepairTest
         }
 
         @Override
-        Iterable<InetAddressAndPort> getCandidatesForToken(Token token)
-        {
-            return targets;
-        }
-
-        @Override
         public Set<InetAddressAndPort> getReadRecipients()
         {
             return readCommandRecipients;
@@ -79,22 +73,24 @@ public class ReadOnlyReadRepairTest extends AbstractReadRepairTest
     }
 
     @Override
-    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+    public InstrumentedReadRepair createInstrumentedReadRepair(ReadCommand command, ReplicaLayout<?, ?> replicaLayout, long queryStartNanoTime)
     {
-        return new InstrumentedReadOnlyReadRepair(command, queryStartNanoTime, consistency);
+        return new InstrumentedReadOnlyReadRepair(command, replicaLayout, queryStartNanoTime);
     }
 
     @Test
     public void getMergeListener()
     {
-        InstrumentedReadRepair repair = createInstrumentedReadRepair();
-        Assert.assertSame(UnfilteredPartitionIterators.MergeListener.NOOP, repair.getMergeListener(new InetAddressAndPort[]{}));
+        ReplicaLayout<?, ?> replicaLayout = replicaLayout(replicas, replicas);
+        InstrumentedReadRepair repair = createInstrumentedReadRepair(replicaLayout);
+        Assert.assertSame(UnfilteredPartitionIterators.MergeListener.NOOP, repair.getMergeListener(replicaLayout));
     }
 
     @Test(expected = UnsupportedOperationException.class)
     public void repairPartitionFailure()
     {
-        InstrumentedReadRepair repair = createInstrumentedReadRepair();
-        repair.repairPartition(dk(1), Collections.emptyMap(), new InetAddressAndPort[]{});
+        ReplicaLayout<?, ?> replicaLayout = replicaLayout(replicas, replicas);
+        InstrumentedReadRepair repair = createInstrumentedReadRepair(replicaLayout);
+        repair.repairPartition(null, Collections.emptyMap(), replicaLayout);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java
new file mode 100644
index 0000000..e4ba25d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java
@@ -0,0 +1,353 @@
+/*
+ * 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.service.reads.repair;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.ReplicaUtils;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Tables;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.locator.ReplicaUtils.full;
+
+public class ReadRepairTest
+{
+    static Keyspace ks;
+    static ColumnFamilyStore cfs;
+    static TableMetadata cfm;
+    static Replica target1;
+    static Replica target2;
+    static Replica target3;
+    static EndpointsForRange targets;
+
+    private static class InstrumentedReadRepairHandler<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends BlockingPartitionRepair<E, L>
+    {
+        public InstrumentedReadRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor, L replicaLayout)
+        {
+            super(Util.dk("not a valid key"), repairs, maxBlockFor, replicaLayout);
+        }
+
+        Map<InetAddressAndPort, Mutation> mutationsSent = new HashMap<>();
+
+        protected void sendRR(MessageOut<Mutation> message, InetAddressAndPort endpoint)
+        {
+            mutationsSent.put(endpoint, message.payload);
+        }
+
+        @Override
+        protected boolean isLocal(InetAddressAndPort endpoint)
+        {
+            return targets.endpoints().contains(endpoint);
+        }
+    }
+
+    static long now = TimeUnit.NANOSECONDS.toMicros(System.nanoTime());
+    static DecoratedKey key;
+    static Cell cell1;
+    static Cell cell2;
+    static Cell cell3;
+    static Mutation resolved;
+
+    private static void assertRowsEqual(Row expected, Row actual)
+    {
+        try
+        {
+            Assert.assertEquals(expected == null, actual == null);
+            if (expected == null)
+                return;
+            Assert.assertEquals(expected.clustering(), actual.clustering());
+            Assert.assertEquals(expected.deletion(), actual.deletion());
+            Assert.assertArrayEquals(Iterables.toArray(expected.cells(), Cell.class), Iterables.toArray(expected.cells(), Cell.class));
+        } catch (Throwable t)
+        {
+            throw new AssertionError(String.format("Row comparison failed, expected %s got %s", expected, actual), t);
+        }
+    }
+
+    @BeforeClass
+    public static void setUpClass() throws Throwable
+    {
+        SchemaLoader.loadSchema();
+        String ksName = "ks";
+
+        cfm = CreateTableStatement.parse("CREATE TABLE tbl (k int primary key, v text)", ksName).build();
+        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(3), Tables.of(cfm));
+        MigrationManager.announceNewKeyspace(ksm, false);
+
+        ks = Keyspace.open(ksName);
+        cfs = ks.getColumnFamilyStore("tbl");
+
+        cfs.sampleReadLatencyNanos = 0;
+
+        target1 = full(InetAddressAndPort.getByName("127.0.0.255"));
+        target2 = full(InetAddressAndPort.getByName("127.0.0.254"));
+        target3 = full(InetAddressAndPort.getByName("127.0.0.253"));
+
+        targets = EndpointsForRange.of(target1, target2, target3);
+
+        // default test values
+        key  = dk(5);
+        cell1 = cell("v", "val1", now);
+        cell2 = cell("v", "val2", now);
+        cell3 = cell("v", "val3", now);
+        resolved = mutation(cell1, cell2);
+    }
+
+    private static DecoratedKey dk(int v)
+    {
+        return DatabaseDescriptor.getPartitioner().decorateKey(ByteBufferUtil.bytes(v));
+    }
+
+    private static Cell cell(String name, String value, long timestamp)
+    {
+        return BufferCell.live(cfm.getColumn(ColumnIdentifier.getInterned(name, false)), timestamp, ByteBufferUtil.bytes(value));
+    }
+
+    private static Mutation mutation(Cell... cells)
+    {
+        Row.Builder builder = BTreeRow.unsortedBuilder();
+        builder.newRow(Clustering.EMPTY);
+        for (Cell cell: cells)
+        {
+            builder.addCell(cell);
+        }
+        return new Mutation(PartitionUpdate.singleRowUpdate(cfm, key, builder.build()));
+    }
+
+    private static InstrumentedReadRepairHandler createRepairHandler(Map<Replica, Mutation> repairs, int maxBlockFor, EndpointsForRange all, EndpointsForRange targets)
+    {
+        ReplicaLayout.ForRange replicaLayout = new ReplicaLayout.ForRange(ks, ConsistencyLevel.LOCAL_QUORUM, ReplicaUtils.FULL_BOUNDS, all, targets);
+        return new InstrumentedReadRepairHandler(repairs, maxBlockFor, replicaLayout);
+    }
+
+    @Test
+    public void consistencyLevelTest() throws Exception
+    {
+        Assert.assertTrue(ConsistencyLevel.QUORUM.satisfies(ConsistencyLevel.QUORUM, ks));
+        Assert.assertTrue(ConsistencyLevel.THREE.satisfies(ConsistencyLevel.QUORUM, ks));
+        Assert.assertTrue(ConsistencyLevel.TWO.satisfies(ConsistencyLevel.QUORUM, ks));
+        Assert.assertFalse(ConsistencyLevel.ONE.satisfies(ConsistencyLevel.QUORUM, ks));
+        Assert.assertFalse(ConsistencyLevel.ANY.satisfies(ConsistencyLevel.QUORUM, ks));
+    }
+
+    private static void assertMutationEqual(Mutation expected, Mutation actual)
+    {
+        Assert.assertEquals(expected.getKeyspaceName(), actual.getKeyspaceName());
+        Assert.assertEquals(expected.key(), actual.key());
+        Assert.assertEquals(expected.key(), actual.key());
+        PartitionUpdate expectedUpdate = Iterables.getOnlyElement(expected.getPartitionUpdates());
+        PartitionUpdate actualUpdate = Iterables.getOnlyElement(actual.getPartitionUpdates());
+        assertRowsEqual(Iterables.getOnlyElement(expectedUpdate), Iterables.getOnlyElement(actualUpdate));
+    }
+
+    @Test
+    public void additionalMutationRequired() throws Exception
+    {
+        Mutation repair1 = mutation(cell2);
+        Mutation repair2 = mutation(cell1);
+
+        // check that the correct repairs are calculated
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, repair1);
+        repairs.put(target2, repair2);
+
+        InstrumentedReadRepairHandler<?, ?> handler = createRepairHandler(repairs, 2,
+                                                                    targets, EndpointsForRange.of(target1, target2));
+
+        Assert.assertTrue(handler.mutationsSent.isEmpty());
+
+        // check that the correct mutations are sent
+        handler.sendInitialRepairs();
+        Assert.assertEquals(2, handler.mutationsSent.size());
+        assertMutationEqual(repair1, handler.mutationsSent.get(target1.endpoint()));
+        assertMutationEqual(repair2, handler.mutationsSent.get(target2.endpoint()));
+
+        // check that a combined mutation is speculatively sent to the 3rd target
+        handler.mutationsSent.clear();
+        handler.maybeSendAdditionalWrites(0, TimeUnit.NANOSECONDS);
+        Assert.assertEquals(1, handler.mutationsSent.size());
+        assertMutationEqual(resolved, handler.mutationsSent.get(target3.endpoint()));
+
+        // check repairs stop blocking after receiving 2 acks
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+        handler.ack(target1.endpoint());
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+        handler.ack(target3.endpoint());
+        Assert.assertTrue(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+    }
+
+    /**
+     * If we've received enough acks, we shouldn't send any additional mutations
+     */
+    @Test
+    public void noAdditionalMutationRequired() throws Exception
+    {
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, mutation(cell2));
+        repairs.put(target2, mutation(cell1));
+
+        EndpointsForRange replicas = EndpointsForRange.of(target1, target2);
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, replicas, targets);
+        handler.sendInitialRepairs();
+        handler.ack(target1.endpoint());
+        handler.ack(target2.endpoint());
+
+        // both replicas have acked, we shouldn't send anything else out
+        handler.mutationsSent.clear();
+        handler.maybeSendAdditionalWrites(0, TimeUnit.NANOSECONDS);
+        Assert.assertTrue(handler.mutationsSent.isEmpty());
+    }
+
+    /**
+     * If there are no additional nodes we can send mutations to, we... shouldn't
+     */
+    @Test
+    public void noAdditionalMutationPossible() throws Exception
+    {
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, mutation(cell2));
+        repairs.put(target2, mutation(cell1));
+
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, EndpointsForRange.of(target1, target2),
+                                                                    EndpointsForRange.of(target1, target2));
+        handler.sendInitialRepairs();
+
+        // we've already sent mutations to all candidates, so we shouldn't send any more
+        handler.mutationsSent.clear();
+        handler.maybeSendAdditionalWrites(0, TimeUnit.NANOSECONDS);
+        Assert.assertTrue(handler.mutationsSent.isEmpty());
+    }
+
+    /**
+     * If we didn't send a repair to a replica because there wasn't a diff with the
+     * resolved column family, we shouldn't send it a speculative mutation
+     */
+    @Test
+    public void mutationsArentSentToInSyncNodes() throws Exception
+    {
+        Mutation repair1 = mutation(cell2);
+
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, repair1);
+
+        // check that the correct initial mutations are sent out
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, targets, EndpointsForRange.of(target1, target2));
+        handler.sendInitialRepairs();
+        Assert.assertEquals(1, handler.mutationsSent.size());
+        Assert.assertTrue(handler.mutationsSent.containsKey(target1.endpoint()));
+
+        // check that speculative mutations aren't sent to target2
+        handler.mutationsSent.clear();
+        handler.maybeSendAdditionalWrites(0, TimeUnit.NANOSECONDS);
+
+        Assert.assertEquals(1, handler.mutationsSent.size());
+        Assert.assertTrue(handler.mutationsSent.containsKey(target3.endpoint()));
+    }
+
+    @Test
+    public void onlyBlockOnQuorum()
+    {
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, mutation(cell1));
+        repairs.put(target2, mutation(cell2));
+        repairs.put(target3, mutation(cell3));
+        Assert.assertEquals(3, repairs.size());
+
+        EndpointsForRange replicas = EndpointsForRange.of(target1, target2, target3);
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, replicas, replicas);
+        handler.sendInitialRepairs();
+
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+        handler.ack(target1.endpoint());
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+
+        // here we should stop blocking, even though we've sent 3 repairs
+        handler.ack(target2.endpoint());
+        Assert.assertTrue(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+
+    }
+
+    /**
+     * For dc local consistency levels, noop mutations and responses from remote dcs should not affect effective blockFor
+     */
+    @Test
+    public void remoteDCTest() throws Exception
+    {
+        Map<Replica, Mutation> repairs = new HashMap<>();
+        repairs.put(target1, mutation(cell1));
+
+        Replica remote1 = full(InetAddressAndPort.getByName("10.0.0.1"));
+        Replica remote2 = full(InetAddressAndPort.getByName("10.0.0.2"));
+        repairs.put(remote1, mutation(cell1));
+
+        EndpointsForRange participants = EndpointsForRange.of(target1, target2, remote1, remote2);
+        EndpointsForRange targets = EndpointsForRange.of(target1, target2);
+
+        InstrumentedReadRepairHandler handler = createRepairHandler(repairs, 2, participants, targets);
+        handler.sendInitialRepairs();
+        Assert.assertEquals(2, handler.mutationsSent.size());
+        Assert.assertTrue(handler.mutationsSent.containsKey(target1.endpoint()));
+        Assert.assertTrue(handler.mutationsSent.containsKey(remote1.endpoint()));
+
+        Assert.assertEquals(1, handler.waitingOn());
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+
+        handler.ack(remote1.endpoint());
+        Assert.assertEquals(1, handler.waitingOn());
+        Assert.assertFalse(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+
+        handler.ack(target1.endpoint());
+        Assert.assertEquals(0, handler.waitingOn());
+        Assert.assertTrue(handler.awaitRepairs(0, TimeUnit.NANOSECONDS));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/repair/TestableReadRepair.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/repair/TestableReadRepair.java b/test/unit/org/apache/cassandra/service/reads/repair/TestableReadRepair.java
index f97980b..2a2dec2 100644
--- a/test/unit/org/apache/cassandra/service/reads/repair/TestableReadRepair.java
+++ b/test/unit/org/apache/cassandra/service/reads/repair/TestableReadRepair.java
@@ -29,17 +29,25 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.service.reads.DigestResolver;
 
-public class TestableReadRepair implements ReadRepair
+public class TestableReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> implements ReadRepair<E, L>
 {
     public final Map<InetAddressAndPort, Mutation> sent = new HashMap<>();
 
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
 
+    private boolean partitionListenerClosed = false;
+    private boolean rowListenerClosed = true;
+
     public TestableReadRepair(ReadCommand command, ConsistencyLevel consistency)
     {
         this.command = command;
@@ -47,13 +55,35 @@ public class TestableReadRepair implements ReadRepair
     }
 
     @Override
-    public UnfilteredPartitionIterators.MergeListener getMergeListener(InetAddressAndPort[] endpoints)
+    public UnfilteredPartitionIterators.MergeListener getMergeListener(L endpoints)
     {
-        return new PartitionIteratorMergeListener(endpoints, command, consistency, this);
+        return new PartitionIteratorMergeListener(endpoints, command, consistency, this) {
+            @Override
+            public void close()
+            {
+                super.close();
+                partitionListenerClosed = true;
+            }
+
+            @Override
+            public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
+            {
+                assert rowListenerClosed;
+                rowListenerClosed = false;
+                return new RowIteratorMergeListener(partitionKey, columns(versions), isReversed(versions), endpoints, command, consistency, TestableReadRepair.this) {
+                    @Override
+                    public void close()
+                    {
+                        super.close();
+                        rowListenerClosed = true;
+                    }
+                };
+            }
+        };
     }
 
     @Override
-    public void startRepair(DigestResolver digestResolver, List<InetAddressAndPort> allEndpoints, List<InetAddressAndPort> contactedEndpoints, Consumer<PartitionIterator> resultConsumer)
+    public void startRepair(DigestResolver<E, L> digestResolver, Consumer<PartitionIterator> resultConsumer)
     {
 
     }
@@ -83,13 +113,19 @@ public class TestableReadRepair implements ReadRepair
     }
 
     @Override
-    public void repairPartition(DecoratedKey key, Map<InetAddressAndPort, Mutation> mutations, InetAddressAndPort[] destinations)
+    public void repairPartition(DecoratedKey partitionKey, Map<Replica, Mutation> mutations, L replicaLayout)
     {
-        sent.putAll(mutations);
+        for (Map.Entry<Replica, Mutation> entry: mutations.entrySet())
+            sent.put(entry.getKey().endpoint(), entry.getValue());
     }
 
     public Mutation getForEndpoint(InetAddressAndPort endpoint)
     {
         return sent.get(endpoint);
     }
-}
+
+    public boolean dataWasConsumed()
+    {
+        return partitionListenerClosed && rowListenerClosed;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index bc501be..909e221 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -144,7 +145,7 @@ public class StreamingTransferTest
         ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
 
         StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER)
-                                                  .requestRanges(LOCAL, KEYSPACE2, ranges)
+                                                  .requestRanges(LOCAL, KEYSPACE2, RangesAtEndpoint.toDummyList(ranges), RangesAtEndpoint.toDummyList(Collections.emptyList()))
                                                   .execute();
 
         UUID planId = futureResult.planId;
@@ -238,13 +239,13 @@ public class StreamingTransferTest
         List<Range<Token>> ranges = new ArrayList<>();
         // wrapped range
         ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key0"))));
-        StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName());
+        StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferRanges(LOCAL, cfs.keyspace.getName(), RangesAtEndpoint.toDummyList(ranges), cfs.getTableName());
         streamPlan.execute().get();
 
         //cannot add ranges after stream session is finished
         try
         {
-            streamPlan.transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName());
+            streamPlan.transferRanges(LOCAL, cfs.keyspace.getName(), RangesAtEndpoint.toDummyList(ranges), cfs.getTableName());
             fail("Should have thrown exception");
         }
         catch (RuntimeException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java b/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
index 2842374..f8f6b12 100644
--- a/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
@@ -95,7 +95,7 @@ public class AccumulatorTest
 
         assertEquals("0", accu.get(3));
 
-        Iterator<String> iter = accu.iterator();
+        Iterator<String> iter = accu.snapshot().iterator();
 
         assertEquals("3", iter.next());
         assertEquals("2", iter.next());


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


[13/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index cb2ea46..c63f4f3 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -20,10 +20,12 @@ package org.apache.cassandra.locator;
 import java.util.*;
 import java.util.Map.Entry;
 
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.dht.Datacenters;
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
@@ -49,14 +51,17 @@ import com.google.common.collect.Multimap;
  */
 public class NetworkTopologyStrategy extends AbstractReplicationStrategy
 {
-    private final Map<String, Integer> datacenters;
+    private final Map<String, ReplicationFactor> datacenters;
+    private final ReplicationFactor aggregateRf;
     private static final Logger logger = LoggerFactory.getLogger(NetworkTopologyStrategy.class);
 
     public NetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
     {
         super(keyspaceName, tokenMetadata, snitch, configOptions);
 
-        Map<String, Integer> newDatacenters = new HashMap<String, Integer>();
+        int replicas = 0;
+        int trans = 0;
+        Map<String, ReplicationFactor> newDatacenters = new HashMap<>();
         if (configOptions != null)
         {
             for (Entry<String, String> entry : configOptions.entrySet())
@@ -64,12 +69,15 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
                 String dc = entry.getKey();
                 if (dc.equalsIgnoreCase("replication_factor"))
                     throw new ConfigurationException("replication_factor is an option for SimpleStrategy, not NetworkTopologyStrategy");
-                Integer replicas = Integer.valueOf(entry.getValue());
-                newDatacenters.put(dc, replicas);
+                ReplicationFactor rf = ReplicationFactor.fromString(entry.getValue());
+                replicas += rf.allReplicas;
+                trans += rf.transientReplicas();
+                newDatacenters.put(dc, rf);
             }
         }
 
         datacenters = Collections.unmodifiableMap(newDatacenters);
+        aggregateRf = ReplicationFactor.withTransient(replicas, trans);
         logger.info("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
     }
 
@@ -79,7 +87,8 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
     private static final class DatacenterEndpoints
     {
         /** List accepted endpoints get pushed into. */
-        Set<InetAddressAndPort> endpoints;
+        EndpointsForRange.Mutable replicas;
+
         /**
          * Racks encountered so far. Replicas are put into separate racks while possible.
          * For efficiency the set is shared between the instances, using the location pair (dc, rack) to make sure
@@ -90,41 +99,51 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         /** Number of replicas left to fill from this DC. */
         int rfLeft;
         int acceptableRackRepeats;
+        int transients;
 
-        DatacenterEndpoints(int rf, int rackCount, int nodeCount, Set<InetAddressAndPort> endpoints, Set<Pair<String, String>> racks)
+        DatacenterEndpoints(ReplicationFactor rf, int rackCount, int nodeCount, EndpointsForRange.Mutable replicas, Set<Pair<String, String>> racks)
         {
-            this.endpoints = endpoints;
+            this.replicas = replicas;
             this.racks = racks;
             // If there aren't enough nodes in this DC to fill the RF, the number of nodes is the effective RF.
-            this.rfLeft = Math.min(rf, nodeCount);
+            this.rfLeft = Math.min(rf.allReplicas, nodeCount);
             // If there aren't enough racks in this DC to fill the RF, we'll still use at least one node from each rack,
             // and the difference is to be filled by the first encountered nodes.
-            acceptableRackRepeats = rf - rackCount;
+            acceptableRackRepeats = rf.allReplicas - rackCount;
+
+            // if we have fewer replicas than rf calls for, reduce transients accordingly
+            int reduceTransients = rf.allReplicas - this.rfLeft;
+            transients = Math.max(rf.transientReplicas() - reduceTransients, 0);
+            ReplicationFactor.validate(rfLeft, transients);
         }
 
         /**
-         * Attempts to add an endpoint to the replicas for this datacenter, adding to the endpoints set if successful.
+         * Attempts to add an endpoint to the replicas for this datacenter, adding to the replicas set if successful.
          * Returns true if the endpoint was added, and this datacenter does not require further replicas.
          */
-        boolean addEndpointAndCheckIfDone(InetAddressAndPort ep, Pair<String,String> location)
+        boolean addEndpointAndCheckIfDone(InetAddressAndPort ep, Pair<String,String> location, Range<Token> replicatedRange)
         {
             if (done())
                 return false;
 
+            if (replicas.endpoints().contains(ep))
+                // Cannot repeat a node.
+                return false;
+
+            Replica replica = new Replica(ep, replicatedRange, rfLeft > transients);
+
             if (racks.add(location))
             {
                 // New rack.
                 --rfLeft;
-                boolean added = endpoints.add(ep);
-                assert added;
+                replicas.add(replica, Conflict.NONE);
                 return done();
             }
             if (acceptableRackRepeats <= 0)
                 // There must be rfLeft distinct racks left, do not add any more rack repeats.
                 return false;
-            if (!endpoints.add(ep))
-                // Cannot repeat a node.
-                return false;
+
+            replicas.add(replica, Conflict.NONE);
             // Added a node that is from an already met rack to match RF when there aren't enough racks.
             --acceptableRackRepeats;
             --rfLeft;
@@ -141,10 +160,15 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
     /**
      * calculate endpoints in one pass through the tokens by tracking our progress in each DC.
      */
-    public List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata)
+    public EndpointsForRange calculateNaturalReplicas(Token searchToken, TokenMetadata tokenMetadata)
     {
         // we want to preserve insertion order so that the first added endpoint becomes primary
-        Set<InetAddressAndPort> replicas = new LinkedHashSet<>();
+        ArrayList<Token> sortedTokens = tokenMetadata.sortedTokens();
+        Token replicaEnd = TokenMetadata.firstToken(sortedTokens, searchToken);
+        Token replicaStart = tokenMetadata.getPredecessor(replicaEnd);
+        Range<Token> replicatedRange = new Range<>(replicaStart, replicaEnd);
+
+        EndpointsForRange.Mutable builder = new EndpointsForRange.Mutable(replicatedRange);
         Set<Pair<String, String>> seenRacks = new HashSet<>();
 
         Topology topology = tokenMetadata.getTopology();
@@ -158,31 +182,31 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         Map<String, DatacenterEndpoints> dcs = new HashMap<>(datacenters.size() * 2);
 
         // Create a DatacenterEndpoints object for each non-empty DC.
-        for (Map.Entry<String, Integer> en : datacenters.entrySet())
+        for (Map.Entry<String, ReplicationFactor> en : datacenters.entrySet())
         {
             String dc = en.getKey();
-            int rf = en.getValue();
+            ReplicationFactor rf = en.getValue();
             int nodeCount = sizeOrZero(allEndpoints.get(dc));
 
-            if (rf <= 0 || nodeCount <= 0)
+            if (rf.allReplicas <= 0 || nodeCount <= 0)
                 continue;
 
-            DatacenterEndpoints dcEndpoints = new DatacenterEndpoints(rf, sizeOrZero(racks.get(dc)), nodeCount, replicas, seenRacks);
+            DatacenterEndpoints dcEndpoints = new DatacenterEndpoints(rf, sizeOrZero(racks.get(dc)), nodeCount, builder, seenRacks);
             dcs.put(dc, dcEndpoints);
             ++dcsToFill;
         }
 
-        Iterator<Token> tokenIter = TokenMetadata.ringIterator(tokenMetadata.sortedTokens(), searchToken, false);
+        Iterator<Token> tokenIter = TokenMetadata.ringIterator(sortedTokens, searchToken, false);
         while (dcsToFill > 0 && tokenIter.hasNext())
         {
             Token next = tokenIter.next();
             InetAddressAndPort ep = tokenMetadata.getEndpoint(next);
             Pair<String, String> location = topology.getLocation(ep);
             DatacenterEndpoints dcEndpoints = dcs.get(location.left);
-            if (dcEndpoints != null && dcEndpoints.addEndpointAndCheckIfDone(ep, location))
+            if (dcEndpoints != null && dcEndpoints.addEndpointAndCheckIfDone(ep, location, replicatedRange))
                 --dcsToFill;
         }
-        return new ArrayList<>(replicas);
+        return builder.asImmutableView();
     }
 
     private int sizeOrZero(Multimap<?, ?> collection)
@@ -195,18 +219,15 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         return collection != null ? collection.size() : 0;
     }
 
-    public int getReplicationFactor()
+    public ReplicationFactor getReplicationFactor()
     {
-        int total = 0;
-        for (int repFactor : datacenters.values())
-            total += repFactor;
-        return total;
+        return aggregateRf;
     }
 
-    public int getReplicationFactor(String dc)
+    public ReplicationFactor getReplicationFactor(String dc)
     {
-        Integer replicas = datacenters.get(dc);
-        return replicas == null ? 0 : replicas;
+        ReplicationFactor replicas = datacenters.get(dc);
+        return replicas == null ? ReplicationFactor.ZERO : replicas;
     }
 
     public Set<String> getDatacenters()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
index 93e629e..449c51e 100644
--- a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
@@ -21,9 +21,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 
@@ -36,27 +36,32 @@ import org.apache.cassandra.dht.Token;
  */
 public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
 {
+    private final ReplicationFactor rf;
     public OldNetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
         super(keyspaceName, tokenMetadata, snitch, configOptions);
+        this.rf = ReplicationFactor.fromString(this.configOptions.get("replication_factor"));
     }
 
-    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
     {
-        int replicas = getReplicationFactor();
-        List<InetAddressAndPort> endpoints = new ArrayList<>(replicas);
         ArrayList<Token> tokens = metadata.sortedTokens();
-
         if (tokens.isEmpty())
-            return endpoints;
+            return EndpointsForRange.empty(new Range<>(metadata.partitioner.getMinimumToken(), metadata.partitioner.getMinimumToken()));
 
         Iterator<Token> iter = TokenMetadata.ringIterator(tokens, token, false);
         Token primaryToken = iter.next();
-        endpoints.add(metadata.getEndpoint(primaryToken));
+        Token previousToken = metadata.getPredecessor(primaryToken);
+        Range<Token> tokenRange = new Range<>(previousToken, primaryToken);
+
+        EndpointsForRange.Builder replicas = EndpointsForRange.builder(tokenRange, rf.allReplicas);
+
+        assert !rf.hasTransientReplicas() : "support transient replicas";
+        replicas.add(new Replica(metadata.getEndpoint(primaryToken), previousToken, primaryToken, true));
 
         boolean bDataCenter = false;
         boolean bOtherRack = false;
-        while (endpoints.size() < replicas && iter.hasNext())
+        while (replicas.size() < rf.allReplicas && iter.hasNext())
         {
             // First try to find one in a different data center
             Token t = iter.next();
@@ -65,7 +70,7 @@ public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
                 // If we have already found something in a diff datacenter no need to find another
                 if (!bDataCenter)
                 {
-                    endpoints.add(metadata.getEndpoint(t));
+                    replicas.add(new Replica(metadata.getEndpoint(t), previousToken, primaryToken, true));
                     bDataCenter = true;
                 }
                 continue;
@@ -77,7 +82,7 @@ public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
                 // If we have already found something in a diff rack no need to find another
                 if (!bOtherRack)
                 {
-                    endpoints.add(metadata.getEndpoint(t));
+                    replicas.add(new Replica(metadata.getEndpoint(t), previousToken, primaryToken, true));
                     bOtherRack = true;
                 }
             }
@@ -86,23 +91,24 @@ public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
 
         // If we found N number of nodes we are good. This loop wil just exit. Otherwise just
         // loop through the list and add until we have N nodes.
-        if (endpoints.size() < replicas)
+        if (replicas.size() < rf.allReplicas)
         {
             iter = TokenMetadata.ringIterator(tokens, token, false);
-            while (endpoints.size() < replicas && iter.hasNext())
+            while (replicas.size() < rf.allReplicas && iter.hasNext())
             {
                 Token t = iter.next();
-                if (!endpoints.contains(metadata.getEndpoint(t)))
-                    endpoints.add(metadata.getEndpoint(t));
+                Replica replica = new Replica(metadata.getEndpoint(t), previousToken, primaryToken, true);
+                if (!replicas.containsEndpoint(replica.endpoint()))
+                    replicas.add(replica);
             }
         }
 
-        return endpoints;
+        return replicas.build();
     }
 
-    public int getReplicationFactor()
+    public ReplicationFactor getReplicationFactor()
     {
-        return Integer.parseInt(this.configOptions.get("replication_factor"));
+        return rf;
     }
 
     public void validateOptions() throws ConfigurationException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
index 92307a3..b8b7bc6 100644
--- a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
+++ b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
@@ -23,166 +23,147 @@ package org.apache.cassandra.locator;
 import com.google.common.collect.Iterators;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 
 import java.util.*;
 
-public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<InetAddressAndPort>>>
+public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, EndpointsForRange.Mutable>>
 {
-    private static final Logger logger = LoggerFactory.getLogger(PendingRangeMaps.class);
-
     /**
      * We have for NavigableMap to be able to search for ranges containing a token efficiently.
      *
      * First two are for non-wrap-around ranges, and the last two are for wrap-around ranges.
      */
     // ascendingMap will sort the ranges by the ascending order of right token
-    final NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMap;
+    private final NavigableMap<Range<Token>, EndpointsForRange.Mutable> ascendingMap;
+
     /**
      * sorting end ascending, if ends are same, sorting begin descending, so that token (end, end) will
      * come before (begin, end] with the same end, and (begin, end) will be selected in the tailMap.
      */
-    static final Comparator<Range<Token>> ascendingComparator = new Comparator<Range<Token>>()
-        {
-            @Override
-            public int compare(Range<Token> o1, Range<Token> o2)
-            {
-                int res = o1.right.compareTo(o2.right);
-                if (res != 0)
-                    return res;
+    private static final Comparator<Range<Token>> ascendingComparator = (o1, o2) -> {
+        int res = o1.right.compareTo(o2.right);
+        if (res != 0)
+            return res;
 
-                return o2.left.compareTo(o1.left);
-            }
-        };
+        return o2.left.compareTo(o1.left);
+    };
 
     // ascendingMap will sort the ranges by the descending order of left token
-    final NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMap;
+    private final NavigableMap<Range<Token>, EndpointsForRange.Mutable> descendingMap;
+
     /**
      * sorting begin descending, if begins are same, sorting end descending, so that token (begin, begin) will
      * come after (begin, end] with the same begin, and (begin, end) won't be selected in the tailMap.
      */
-    static final Comparator<Range<Token>> descendingComparator = new Comparator<Range<Token>>()
-        {
-            @Override
-            public int compare(Range<Token> o1, Range<Token> o2)
-            {
-                int res = o2.left.compareTo(o1.left);
-                if (res != 0)
-                    return res;
+    private static final Comparator<Range<Token>> descendingComparator = (o1, o2) -> {
+        int res = o2.left.compareTo(o1.left);
+        if (res != 0)
+            return res;
 
-                // if left tokens are same, sort by the descending of the right tokens.
-                return o2.right.compareTo(o1.right);
-            }
-        };
+        // if left tokens are same, sort by the descending of the right tokens.
+        return o2.right.compareTo(o1.right);
+    };
 
     // these two maps are for warp around ranges.
-    final NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMapForWrapAround;
+    private final NavigableMap<Range<Token>, EndpointsForRange.Mutable> ascendingMapForWrapAround;
+
     /**
      * for wrap around range (begin, end], which begin > end.
      * Sorting end ascending, if ends are same, sorting begin ascending,
      * so that token (end, end) will come before (begin, end] with the same end, and (begin, end] will be selected in
      * the tailMap.
      */
-    static final Comparator<Range<Token>> ascendingComparatorForWrapAround = new Comparator<Range<Token>>()
-    {
-        @Override
-        public int compare(Range<Token> o1, Range<Token> o2)
-        {
-            int res = o1.right.compareTo(o2.right);
-            if (res != 0)
-                return res;
+    private static final Comparator<Range<Token>> ascendingComparatorForWrapAround = (o1, o2) -> {
+        int res = o1.right.compareTo(o2.right);
+        if (res != 0)
+            return res;
 
-            return o1.left.compareTo(o2.left);
-        }
+        return o1.left.compareTo(o2.left);
     };
 
-    final NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMapForWrapAround;
+    private final NavigableMap<Range<Token>, EndpointsForRange.Mutable> descendingMapForWrapAround;
+
     /**
      * for wrap around ranges, which begin > end.
      * Sorting end ascending, so that token (begin, begin) will come after (begin, end] with the same begin,
      * and (begin, end) won't be selected in the tailMap.
      */
-    static final Comparator<Range<Token>> descendingComparatorForWrapAround = new Comparator<Range<Token>>()
-    {
-        @Override
-        public int compare(Range<Token> o1, Range<Token> o2)
-        {
-            int res = o2.left.compareTo(o1.left);
-            if (res != 0)
-                return res;
-            return o1.right.compareTo(o2.right);
-        }
+    private static final Comparator<Range<Token>> descendingComparatorForWrapAround = (o1, o2) -> {
+        int res = o2.left.compareTo(o1.left);
+        if (res != 0)
+            return res;
+        return o1.right.compareTo(o2.right);
     };
 
     public PendingRangeMaps()
     {
-        this.ascendingMap = new TreeMap<Range<Token>, List<InetAddressAndPort>>(ascendingComparator);
-        this.descendingMap = new TreeMap<Range<Token>, List<InetAddressAndPort>>(descendingComparator);
-        this.ascendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddressAndPort>>(ascendingComparatorForWrapAround);
-        this.descendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddressAndPort>>(descendingComparatorForWrapAround);
+        this.ascendingMap = new TreeMap<>(ascendingComparator);
+        this.descendingMap = new TreeMap<>(descendingComparator);
+        this.ascendingMapForWrapAround = new TreeMap<>(ascendingComparatorForWrapAround);
+        this.descendingMapForWrapAround = new TreeMap<>(descendingComparatorForWrapAround);
     }
 
     static final void addToMap(Range<Token> range,
-                               InetAddressAndPort address,
-                               NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingMap,
-                               NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingMap)
+                               Replica replica,
+                               NavigableMap<Range<Token>, EndpointsForRange.Mutable> ascendingMap,
+                               NavigableMap<Range<Token>, EndpointsForRange.Mutable> descendingMap)
     {
-        List<InetAddressAndPort> addresses = ascendingMap.get(range);
-        if (addresses == null)
+        EndpointsForRange.Mutable replicas = ascendingMap.get(range);
+        if (replicas == null)
         {
-            addresses = new ArrayList<>(1);
-            ascendingMap.put(range, addresses);
-            descendingMap.put(range, addresses);
+            replicas = new EndpointsForRange.Mutable(range,1);
+            ascendingMap.put(range, replicas);
+            descendingMap.put(range, replicas);
         }
-        addresses.add(address);
+        replicas.add(replica, Conflict.DUPLICATE);
     }
 
-    public void addPendingRange(Range<Token> range, InetAddressAndPort address)
+    public void addPendingRange(Range<Token> range, Replica replica)
     {
         if (Range.isWrapAround(range.left, range.right))
         {
-            addToMap(range, address, ascendingMapForWrapAround, descendingMapForWrapAround);
+            addToMap(range, replica, ascendingMapForWrapAround, descendingMapForWrapAround);
         }
         else
         {
-            addToMap(range, address, ascendingMap, descendingMap);
+            addToMap(range, replica, ascendingMap, descendingMap);
         }
     }
 
-    static final void addIntersections(Set<InetAddressAndPort> endpointsToAdd,
-                                       NavigableMap<Range<Token>, List<InetAddressAndPort>> smallerMap,
-                                       NavigableMap<Range<Token>, List<InetAddressAndPort>> biggerMap)
+    static final void addIntersections(EndpointsForToken.Builder replicasToAdd,
+                                       NavigableMap<Range<Token>, EndpointsForRange.Mutable> smallerMap,
+                                       NavigableMap<Range<Token>, EndpointsForRange.Mutable> biggerMap)
     {
         // find the intersection of two sets
         for (Range<Token> range : smallerMap.keySet())
         {
-            List<InetAddressAndPort> addresses = biggerMap.get(range);
-            if (addresses != null)
+            EndpointsForRange.Mutable replicas = biggerMap.get(range);
+            if (replicas != null)
             {
-                endpointsToAdd.addAll(addresses);
+                replicasToAdd.addAll(replicas);
             }
         }
     }
 
-    public Collection<InetAddressAndPort> pendingEndpointsFor(Token token)
+    public EndpointsForToken pendingEndpointsFor(Token token)
     {
-        Set<InetAddressAndPort> endpoints = new HashSet<>();
+        EndpointsForToken.Builder replicas = EndpointsForToken.builder(token);
 
-        Range searchRange = new Range(token, token);
+        Range<Token> searchRange = new Range<>(token, token);
 
         // search for non-wrap-around maps
-        NavigableMap<Range<Token>, List<InetAddressAndPort>> ascendingTailMap = ascendingMap.tailMap(searchRange, true);
-        NavigableMap<Range<Token>, List<InetAddressAndPort>> descendingTailMap = descendingMap.tailMap(searchRange, false);
+        NavigableMap<Range<Token>, EndpointsForRange.Mutable> ascendingTailMap = ascendingMap.tailMap(searchRange, true);
+        NavigableMap<Range<Token>, EndpointsForRange.Mutable> descendingTailMap = descendingMap.tailMap(searchRange, false);
 
         // add intersections of two maps
         if (ascendingTailMap.size() < descendingTailMap.size())
         {
-            addIntersections(endpoints, ascendingTailMap, descendingTailMap);
+            addIntersections(replicas, ascendingTailMap, descendingTailMap);
         }
         else
         {
-            addIntersections(endpoints, descendingTailMap, ascendingTailMap);
+            addIntersections(replicas, descendingTailMap, ascendingTailMap);
         }
 
         // search for wrap-around sets
@@ -190,29 +171,29 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
         descendingTailMap = descendingMapForWrapAround.tailMap(searchRange, false);
 
         // add them since they are all necessary.
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : ascendingTailMap.entrySet())
+        for (Map.Entry<Range<Token>, EndpointsForRange.Mutable> entry : ascendingTailMap.entrySet())
         {
-            endpoints.addAll(entry.getValue());
+            replicas.addAll(entry.getValue());
         }
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : descendingTailMap.entrySet())
+        for (Map.Entry<Range<Token>, EndpointsForRange.Mutable> entry : descendingTailMap.entrySet())
         {
-            endpoints.addAll(entry.getValue());
+            replicas.addAll(entry.getValue());
         }
 
-        return endpoints;
+        return replicas.build();
     }
 
     public String printPendingRanges()
     {
         StringBuilder sb = new StringBuilder();
 
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : this)
+        for (Map.Entry<Range<Token>, EndpointsForRange.Mutable> entry : this)
         {
             Range<Token> range = entry.getKey();
 
-            for (InetAddressAndPort address : entry.getValue())
+            for (Replica replica : entry.getValue())
             {
-                sb.append(address).append(':').append(range);
+                sb.append(replica).append(':').append(range);
                 sb.append(System.getProperty("line.separator"));
             }
         }
@@ -221,7 +202,7 @@ public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<I
     }
 
     @Override
-    public Iterator<Map.Entry<Range<Token>, List<InetAddressAndPort>>> iterator()
+    public Iterator<Map.Entry<Range<Token>, EndpointsForRange.Mutable>> iterator()
     {
         return Iterators.concat(ascendingMap.entrySet().iterator(), ascendingMapForWrapAround.entrySet().iterator());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/RangesAtEndpoint.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/RangesAtEndpoint.java b/src/java/org/apache/cassandra/locator/RangesAtEndpoint.java
new file mode 100644
index 0000000..74828ad
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/RangesAtEndpoint.java
@@ -0,0 +1,313 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collector;
+import java.util.stream.Collectors;
+
+import static org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict.*;
+
+/**
+ * A ReplicaCollection for Ranges occurring at an endpoint. All Replica will be for the same endpoint,
+ * and must be unique Ranges (though overlapping ranges are presently permitted, these should probably not be permitted to occur)
+ */
+public class RangesAtEndpoint extends AbstractReplicaCollection<RangesAtEndpoint>
+{
+    private static final Map<Range<Token>, Replica> EMPTY_MAP = Collections.unmodifiableMap(new LinkedHashMap<>());
+
+    private final InetAddressAndPort endpoint;
+    private volatile Map<Range<Token>, Replica> byRange;
+    private volatile RangesAtEndpoint fullRanges;
+    private volatile RangesAtEndpoint transRanges;
+
+    private RangesAtEndpoint(InetAddressAndPort endpoint, List<Replica> list, boolean isSnapshot)
+    {
+        this(endpoint, list, isSnapshot, null);
+    }
+    private RangesAtEndpoint(InetAddressAndPort endpoint, List<Replica> list, boolean isSnapshot, Map<Range<Token>, Replica> byRange)
+    {
+        super(list, isSnapshot);
+        this.endpoint = endpoint;
+        this.byRange = byRange;
+        assert endpoint != null;
+    }
+
+    public InetAddressAndPort endpoint()
+    {
+        return endpoint;
+    }
+
+    @Override
+    public Set<InetAddressAndPort> endpoints()
+    {
+        return Collections.unmodifiableSet(list.isEmpty()
+                ? Collections.emptySet()
+                : Collections.singleton(endpoint)
+        );
+    }
+
+    public Set<Range<Token>> ranges()
+    {
+        return byRange().keySet();
+    }
+
+    public Map<Range<Token>, Replica> byRange()
+    {
+        Map<Range<Token>, Replica> map = byRange;
+        if (map == null)
+            byRange = map = buildByRange(list);
+        return map;
+    }
+
+    @Override
+    protected RangesAtEndpoint snapshot(List<Replica> subList)
+    {
+        if (subList.isEmpty()) return empty(endpoint);
+        return new RangesAtEndpoint(endpoint, subList, true);
+    }
+
+    @Override
+    public RangesAtEndpoint self()
+    {
+        return this;
+    }
+
+    @Override
+    public ReplicaCollection.Mutable<RangesAtEndpoint> newMutable(int initialCapacity)
+    {
+        return new Mutable(endpoint, initialCapacity);
+    }
+
+    @Override
+    public boolean contains(Replica replica)
+    {
+        return replica != null
+                && Objects.equals(
+                        byRange().get(replica.range()),
+                        replica);
+    }
+
+    public RangesAtEndpoint full()
+    {
+        RangesAtEndpoint coll = fullRanges;
+        if (fullRanges == null)
+            fullRanges = coll = filter(Replica::isFull);
+        return coll;
+    }
+
+    public RangesAtEndpoint trans()
+    {
+        RangesAtEndpoint coll = transRanges;
+        if (transRanges == null)
+            transRanges = coll = filter(Replica::isTransient);
+        return coll;
+    }
+
+    public Collection<Range<Token>> fullRanges()
+    {
+        return full().ranges();
+    }
+
+    public Collection<Range<Token>> transientRanges()
+    {
+        return trans().ranges();
+    }
+
+    public boolean contains(Range<Token> range, boolean isFull)
+    {
+        Replica replica = byRange().get(range);
+        return replica != null && replica.isFull() == isFull;
+    }
+
+    private static Map<Range<Token>, Replica> buildByRange(List<Replica> list)
+    {
+        // TODO: implement a delegating map that uses our superclass' list, and is immutable
+        Map<Range<Token>, Replica> byRange = new LinkedHashMap<>(list.size());
+        for (Replica replica : list)
+        {
+            Replica prev = byRange.put(replica.range(), replica);
+            assert prev == null : "duplicate range in RangesAtEndpoint: " + prev + " and " + replica;
+        }
+
+        return Collections.unmodifiableMap(byRange);
+    }
+
+    public static Collector<Replica, Builder, RangesAtEndpoint> collector(InetAddressAndPort endpoint)
+    {
+        return collector(ImmutableSet.of(), () -> new Builder(endpoint));
+    }
+
+    public static class Mutable extends RangesAtEndpoint implements ReplicaCollection.Mutable<RangesAtEndpoint>
+    {
+        boolean hasSnapshot;
+        public Mutable(InetAddressAndPort endpoint) { this(endpoint, 0); }
+        public Mutable(InetAddressAndPort endpoint, int capacity) { super(endpoint, new ArrayList<>(capacity), false, new LinkedHashMap<>()); }
+
+        public void add(Replica replica, Conflict ignoreConflict)
+        {
+            if (hasSnapshot) throw new IllegalStateException();
+            Preconditions.checkNotNull(replica);
+            if (!Objects.equals(super.endpoint, replica.endpoint()))
+                throw new IllegalArgumentException("Replica " + replica + " has incorrect endpoint (expected " + super.endpoint + ")");
+
+            Replica prev = super.byRange.put(replica.range(), replica);
+            if (prev != null)
+            {
+                super.byRange.put(replica.range(), prev); // restore prev
+                switch (ignoreConflict)
+                {
+                    case DUPLICATE:
+                        if (prev.equals(replica))
+                            break;
+                    case NONE:
+                        throw new IllegalArgumentException("Conflicting replica added (expected unique ranges): " + replica + "; existing: " + prev);
+                    case ALL:
+                }
+                return;
+            }
+
+            list.add(replica);
+        }
+
+        @Override
+        public Map<Range<Token>, Replica> byRange()
+        {
+            // our internal map is modifiable, but it is unsafe to modify the map externally
+            // it would be possible to implement a safe modifiable map, but it is probably not valuable
+            return Collections.unmodifiableMap(super.byRange());
+        }
+
+        public RangesAtEndpoint get(boolean isSnapshot)
+        {
+            return new RangesAtEndpoint(super.endpoint, super.list, isSnapshot, Collections.unmodifiableMap(super.byRange));
+        }
+
+        public RangesAtEndpoint asImmutableView()
+        {
+            return get(false);
+        }
+
+        public RangesAtEndpoint asSnapshot()
+        {
+            hasSnapshot = true;
+            return get(true);
+        }
+    }
+
+    public static class Builder extends ReplicaCollection.Builder<RangesAtEndpoint, Mutable, RangesAtEndpoint.Builder>
+    {
+        public Builder(InetAddressAndPort endpoint) { this(endpoint, 0); }
+        public Builder(InetAddressAndPort endpoint, int capacity) { super(new Mutable(endpoint, capacity)); }
+    }
+
+    public static RangesAtEndpoint.Builder builder(InetAddressAndPort endpoint)
+    {
+        return new RangesAtEndpoint.Builder(endpoint);
+    }
+
+    public static RangesAtEndpoint.Builder builder(InetAddressAndPort endpoint, int capacity)
+    {
+        return new RangesAtEndpoint.Builder(endpoint, capacity);
+    }
+
+    public static RangesAtEndpoint empty(InetAddressAndPort endpoint)
+    {
+        return new RangesAtEndpoint(endpoint, EMPTY_LIST, true, EMPTY_MAP);
+    }
+
+    public static RangesAtEndpoint of(Replica replica)
+    {
+        ArrayList<Replica> one = new ArrayList<>(1);
+        one.add(replica);
+        return new RangesAtEndpoint(replica.endpoint(), one, true, Collections.unmodifiableMap(Collections.singletonMap(replica.range(), replica)));
+    }
+
+    public static RangesAtEndpoint of(Replica ... replicas)
+    {
+        return copyOf(Arrays.asList(replicas));
+    }
+
+    public static RangesAtEndpoint copyOf(List<Replica> replicas)
+    {
+        if (replicas.isEmpty())
+            throw new IllegalArgumentException("Must specify a non-empty collection of replicas");
+        return builder(replicas.get(0).endpoint(), replicas.size()).addAll(replicas).build();
+    }
+
+
+    /**
+     * Use of this method to synthesize Replicas is almost always wrong. In repair it turns out the concerns of transient
+     * vs non-transient are handled at a higher level, but eventually repair needs to ask streaming to actually move
+     * the data and at that point it doesn't have a great handle on what the replicas are and it doesn't really matter.
+     *
+     * Streaming expects to be given Replicas with each replica indicating what type of data (transient or not transient)
+     * should be sent.
+     *
+     * So in this one instance we can lie to streaming and pretend all the replicas are full and use a dummy address
+     * and it doesn't matter because streaming doesn't rely on the address for anything other than debugging and full
+     * is a valid value for transientness because streaming is selecting candidate tables from the repair/unrepaired
+     * set already.
+     * @param ranges
+     * @return
+     */
+    @VisibleForTesting
+    public static RangesAtEndpoint toDummyList(Collection<Range<Token>> ranges)
+    {
+        InetAddressAndPort dummy;
+        try
+        {
+            dummy = InetAddressAndPort.getByNameOverrideDefaults("0.0.0.0", 0);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        //For repair we are less concerned with full vs transient since repair is already dealing with those concerns.
+        //Always say full and then if the repair is incremental or not will determine what is streamed.
+        return ranges.stream()
+                .map(range -> new Replica(dummy, range, true))
+                .collect(collector(dummy));
+    }
+
+    /**
+     * @return concatenate two DISJOINT collections together
+     */
+    public static RangesAtEndpoint concat(RangesAtEndpoint replicas, RangesAtEndpoint extraReplicas)
+    {
+        return AbstractReplicaCollection.concat(replicas, extraReplicas, NONE);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/RangesByEndpoint.java b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
new file mode 100644
index 0000000..698b133
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/RangesByEndpoint.java
@@ -0,0 +1,54 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class RangesByEndpoint extends ReplicaMultimap<InetAddressAndPort, RangesAtEndpoint>
+{
+    public RangesByEndpoint(Map<InetAddressAndPort, RangesAtEndpoint> map)
+    {
+        super(map);
+    }
+
+    public RangesAtEndpoint get(InetAddressAndPort endpoint)
+    {
+        Preconditions.checkNotNull(endpoint);
+        return map.getOrDefault(endpoint, RangesAtEndpoint.empty(endpoint));
+    }
+
+    public static class Mutable extends ReplicaMultimap.Mutable<InetAddressAndPort, RangesAtEndpoint.Mutable>
+    {
+        @Override
+        protected RangesAtEndpoint.Mutable newMutable(InetAddressAndPort endpoint)
+        {
+            return new RangesAtEndpoint.Mutable(endpoint);
+        }
+
+        public RangesByEndpoint asImmutableView()
+        {
+            return new RangesByEndpoint(Collections.unmodifiableMap(Maps.transformValues(map, RangesAtEndpoint.Mutable::asImmutableView)));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/Replica.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Replica.java b/src/java/org/apache/cassandra/locator/Replica.java
new file mode 100644
index 0000000..37b6050
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/Replica.java
@@ -0,0 +1,196 @@
+/*
+ * 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.Objects;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * A Replica represents an owning node for a copy of a portion of the token ring.
+ *
+ * It consists of:
+ *  - the logical token range that is being replicated (i.e. for the first logical replica only, this will be equal
+ *      to one of its owned portions of the token ring; all other replicas will have this token range also)
+ *  - an endpoint (IP and port)
+ *  - whether the range is replicated in full, or transiently (CASSANDRA-14404)
+ *
+ * In general, it is preferred to use a Replica to a Range&lt;Token&gt;, particularly when users of the concept depend on
+ * knowledge of the full/transient status of the copy.
+ *
+ * That means you should avoid unwrapping and rewrapping these things and think hard about subtraction
+ * and such and what the result is WRT to transientness. Definitely avoid creating fake Replicas with misinformation
+ * about endpoints, ranges, or transientness.
+ */
+public final class Replica implements Comparable<Replica>
+{
+    private final Range<Token> range;
+    private final InetAddressAndPort endpoint;
+    private final boolean full;
+
+    public Replica(InetAddressAndPort endpoint, Range<Token> range, boolean full)
+    {
+        Preconditions.checkNotNull(endpoint);
+        Preconditions.checkNotNull(range);
+        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);
+    }
+
+    @Override
+    public int compareTo(Replica o)
+    {
+        int c = range.compareTo(o.range);
+        if (c == 0)
+            c = endpoint.compareTo(o.endpoint);
+        if (c == 0)
+            c =  Boolean.compare(full, o.full);
+        return c;
+    }
+
+    public int hashCode()
+    {
+        return Objects.hash(endpoint, range, full);
+    }
+
+    @Override
+    public String toString()
+    {
+        return (full ? "Full" : "Transient") + '(' + endpoint() + ',' + range + ')';
+    }
+
+    public final InetAddressAndPort endpoint()
+    {
+        return endpoint;
+    }
+
+    public boolean isLocal()
+    {
+        return endpoint.equals(FBUtilities.getBroadcastAddressAndPort());
+    }
+
+    public Range<Token> range()
+    {
+        return range;
+    }
+
+    public boolean isFull()
+    {
+        return full;
+    }
+
+    public final boolean isTransient()
+    {
+        return !isFull();
+    }
+
+    /**
+     * This is used exclusively in TokenMetadata to check if a portion of a range is already replicated
+     * by an endpoint so that we only mark as pending the portion that is either not replicated sufficiently (transient
+     * when we need full) or at all.
+     *
+     * If it's not replicated at all it needs to be pending because there is no data.
+     * If it's replicated but only transiently and we need to replicate it fully it must be marked as pending until it
+     * is available fully otherwise a read might treat this replica as full and not read from a full replica that has
+     * the data.
+     */
+    public RangesAtEndpoint subtractSameReplication(RangesAtEndpoint toSubtract)
+    {
+        Set<Range<Token>> subtractedRanges = range().subtractAll(toSubtract.filter(r -> r.isFull() == isFull()).ranges());
+        RangesAtEndpoint.Builder result = RangesAtEndpoint.builder(endpoint, subtractedRanges.size());
+        for (Range<Token> range : subtractedRanges)
+        {
+            result.add(decorateSubrange(range));
+        }
+        return result.build();
+    }
+
+    /**
+     * Don't use this method and ignore transient status unless you are explicitly handling it outside this method.
+     *
+     * This helper method is used by StorageService.calculateStreamAndFetchRanges to perform subtraction.
+     * It ignores transient status because it's already being handled in calculateStreamAndFetchRanges.
+     */
+    public RangesAtEndpoint subtractIgnoreTransientStatus(Range<Token> subtract)
+    {
+        Set<Range<Token>> ranges = this.range.subtract(subtract);
+        RangesAtEndpoint.Builder result = RangesAtEndpoint.builder(endpoint, ranges.size());
+        for (Range<Token> subrange : ranges)
+            result.add(decorateSubrange(subrange));
+        return result.build();
+    }
+
+    public boolean contains(Range<Token> that)
+    {
+        return range().contains(that);
+    }
+
+    public boolean intersectsOnRange(Replica replica)
+    {
+        return range().intersects(replica.range());
+    }
+
+    public Replica decorateSubrange(Range<Token> subrange)
+    {
+        Preconditions.checkArgument(range.contains(subrange));
+        return new Replica(endpoint(), subrange, isFull());
+    }
+
+    public static Replica fullReplica(InetAddressAndPort endpoint, Range<Token> range)
+    {
+        return new Replica(endpoint, range, true);
+    }
+
+    public static Replica fullReplica(InetAddressAndPort endpoint, Token start, Token end)
+    {
+        return fullReplica(endpoint, new Range<>(start, end));
+    }
+
+    public static Replica transientReplica(InetAddressAndPort endpoint, Range<Token> range)
+    {
+        return new Replica(endpoint, range, false);
+    }
+
+    public static Replica transientReplica(InetAddressAndPort endpoint, Token start, Token end)
+    {
+        return transientReplica(endpoint, new Range<>(start, end));
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/ReplicaCollection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReplicaCollection.java b/src/java/org/apache/cassandra/locator/ReplicaCollection.java
new file mode 100644
index 0000000..6833f4b
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/ReplicaCollection.java
@@ -0,0 +1,160 @@
+/*
+ * 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 org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+/**
+ * A collection like class for Replica objects. Represents both a well defined order on the contained Replica objects,
+ * and efficient methods for accessing the contained Replicas, directly and as a projection onto their endpoints and ranges.
+ */
+public interface ReplicaCollection<C extends ReplicaCollection<C>> extends Iterable<Replica>
+{
+    /**
+     * @return a Set of the endpoints of the contained Replicas.
+     * Iteration order is maintained where there is a 1:1 relationship between endpoint and Replica
+     * Typically this collection offers O(1) access methods, and this is true for all but ReplicaList.
+     */
+    public abstract Set<InetAddressAndPort> endpoints();
+
+    /**
+     * @param i a value in the range [0..size())
+     * @return the i'th Replica, in our iteration order
+     */
+    public abstract Replica get(int i);
+
+    /**
+     * @return the number of Replica contained
+     */
+    public abstract int size();
+
+    /**
+     * @return true iff size() == 0
+     */
+    public abstract boolean isEmpty();
+
+    /**
+     * @return true iff a Replica in this collection is equal to the provided Replica.
+     * Typically this method is expected to take O(1) time, and this is true for all but ReplicaList.
+     */
+    public abstract boolean contains(Replica replica);
+
+    /**
+     * @return a *eagerly constructed* copy of this collection containing the Replica that match the provided predicate.
+     * An effort will be made to either return ourself, or a subList, where possible.
+     * It is guaranteed that no changes to any upstream Mutable will affect the state of the result.
+     */
+    public abstract C filter(Predicate<Replica> predicate);
+
+    /**
+     * @return a *eagerly constructed* copy of this collection containing the Replica that match the provided predicate.
+     * An effort will be made to either return ourself, or a subList, where possible.
+     * It is guaranteed that no changes to any upstream Mutable will affect the state of the result.
+     * Only the first maxSize items will be returned.
+     */
+    public abstract C filter(Predicate<Replica> predicate, int maxSize);
+
+    /**
+     * @return an *eagerly constructed* copy of this collection containing the Replica at positions [start..end);
+     * An effort will be made to either return ourself, or a subList, where possible.
+     * It is guaranteed that no changes to any upstream Mutable will affect the state of the result.
+     */
+    public abstract C subList(int start, int end);
+
+    /**
+     * @return an *eagerly constructed* copy of this collection containing the Replica re-ordered according to this comparator
+     * It is guaranteed that no changes to any upstream Mutable will affect the state of the result.
+     */
+    public abstract C sorted(Comparator<Replica> comparator);
+
+    public abstract Iterator<Replica> iterator();
+    public abstract Stream<Replica> stream();
+
+    public abstract boolean equals(Object o);
+    public abstract int hashCode();
+    public abstract String toString();
+
+    /**
+     * A mutable extension of a ReplicaCollection.  This is append-only, so it is safe to select a subList,
+     * or at any time take an asImmutableView() snapshot.
+     */
+    public interface Mutable<C extends ReplicaCollection<C>> extends ReplicaCollection<C>
+    {
+        /**
+         * @return an Immutable clone that mirrors any modifications to this Mutable instance.
+         */
+        C asImmutableView();
+
+        /**
+         * @return an Immutable clone that assumes this Mutable will never be modified again.
+         * If this is not true, behaviour is undefined.
+         */
+        C asSnapshot();
+
+        enum Conflict { NONE, DUPLICATE, ALL}
+
+        /**
+         * @param replica add this replica to the end of the collection
+         * @param ignoreConflict if false, fail on any conflicting additions (as defined by C's semantics)
+         */
+        void add(Replica replica, Conflict ignoreConflict);
+
+        default public void add(Replica replica)
+        {
+            add(replica, Conflict.NONE);
+        }
+
+        default public void addAll(Iterable<Replica> replicas, Conflict ignoreConflicts)
+        {
+            for (Replica replica : replicas)
+                add(replica, ignoreConflicts);
+        }
+
+        default public void addAll(Iterable<Replica> replicas)
+        {
+            addAll(replicas, Conflict.NONE);
+        }
+    }
+
+    public static class Builder<C extends ReplicaCollection<C>, M extends Mutable<C>, B extends Builder<C, M, B>>
+    {
+        Mutable<C> mutable;
+        public Builder(Mutable<C> mutable) { this.mutable = mutable; }
+
+        public int size() { return mutable.size(); }
+        public B add(Replica replica) { mutable.add(replica); return (B) this; }
+        public B add(Replica replica, Conflict ignoreConflict) { mutable.add(replica, ignoreConflict); return (B) this; }
+        public B addAll(Iterable<Replica> replica) { mutable.addAll(replica); return (B) this; }
+        public B addAll(Iterable<Replica> replica, Conflict ignoreConflict) { mutable.addAll(replica, ignoreConflict); return (B) this; }
+
+        public C build()
+        {
+            C result = mutable.asSnapshot();
+            mutable = null;
+            return result;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/ReplicaLayout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReplicaLayout.java b/src/java/org/apache/cassandra/locator/ReplicaLayout.java
new file mode 100644
index 0000000..946a7f8
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/ReplicaLayout.java
@@ -0,0 +1,381 @@
+/*
+ * 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.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;
+
+/**
+ * Encapsulates knowledge about the ring necessary for performing a specific operation, with static accessors
+ * for building the relevant layout.
+ *
+ * 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
+ */
+public abstract class ReplicaLayout<E extends Endpoints<E>, L extends ReplicaLayout<E, L>>
+{
+    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 ReplicaLayout(Keyspace keyspace, ConsistencyLevel consistencyLevel, E natural, E pending, E selected, E all)
+    {
+        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()
+    {
+        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
+     */
+    public E pending()
+    {
+        return pending;
+    }
+
+    public int blockFor()
+    {
+        return pending == null
+                ? consistencyLevel.blockFor(keyspace)
+                : consistencyLevel.blockForWrite(keyspace, pending);
+    }
+
+    public Keyspace keyspace()
+    {
+        return keyspace;
+    }
+
+    public ConsistencyLevel consistencyLevel()
+    {
+        return consistencyLevel;
+    }
+
+    abstract public L withSelected(E replicas);
+
+    abstract public L withConsistencyLevel(ConsistencyLevel cl);
+
+    public L forNaturalUncontacted()
+    {
+        E more;
+        if (consistencyLevel.isDatacenterLocal() && keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
+        {
+            IEndpointSnitch snitch = keyspace.getReplicationStrategy().snitch;
+            String localDC = DatabaseDescriptor.getLocalDataCenter();
+
+            more = natural.filter(replica -> !selected.contains(replica) &&
+                    snitch.getDatacenter(replica).equals(localDC));
+        } else
+        {
+            more = natural.filter(replica -> !selected.contains(replica));
+        }
+
+        return withSelected(more);
+    }
+
+    public static class ForRange extends ReplicaLayout<EndpointsForRange, ForRange>
+    {
+        public final AbstractBounds<PartitionPosition> range;
+
+        @VisibleForTesting
+        public ForRange(Keyspace keyspace, ConsistencyLevel consistencyLevel, AbstractBounds<PartitionPosition> range, EndpointsForRange natural, EndpointsForRange selected)
+        {
+            // Range queries do not contact pending replicas
+            super(keyspace, consistencyLevel, natural, null, selected);
+            this.range = range;
+        }
+
+        @Override
+        public ForRange withSelected(EndpointsForRange newSelected)
+        {
+            return new ForRange(keyspace, consistencyLevel, range, natural, newSelected);
+        }
+
+        @Override
+        public ForRange withConsistencyLevel(ConsistencyLevel cl)
+        {
+            return new ForRange(keyspace, cl, range, natural, selected);
+        }
+    }
+
+    public static class ForToken extends ReplicaLayout<EndpointsForToken, ForToken>
+    {
+        public final Token token;
+
+        @VisibleForTesting
+        public ForToken(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, EndpointsForToken natural, EndpointsForToken pending, EndpointsForToken selected)
+        {
+            super(keyspace, consistencyLevel, natural, pending, selected);
+            this.token = token;
+        }
+
+        public ForToken(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, EndpointsForToken natural, EndpointsForToken pending, EndpointsForToken selected, EndpointsForToken all)
+        {
+            super(keyspace, consistencyLevel, natural, pending, selected, all);
+            this.token = token;
+        }
+
+        public ForToken withSelected(EndpointsForToken newSelected)
+        {
+            return new ForToken(keyspace, consistencyLevel, token, natural, pending, newSelected);
+        }
+
+        @Override
+        public ForToken withConsistencyLevel(ConsistencyLevel cl)
+        {
+            return new ForToken(keyspace, cl, token, natural, pending, selected);
+        }
+    }
+
+    public static class ForPaxos extends ForToken
+    {
+        private final int requiredParticipants;
+
+        private ForPaxos(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, int requiredParticipants, EndpointsForToken natural, EndpointsForToken pending, EndpointsForToken selected, EndpointsForToken all)
+        {
+            super(keyspace, consistencyLevel, token, natural, pending, selected, all);
+            this.requiredParticipants = requiredParticipants;
+        }
+
+        public int getRequiredParticipants()
+        {
+            return requiredParticipants;
+        }
+    }
+
+    public static ForToken forSingleReplica(Keyspace keyspace, Token token, Replica replica)
+    {
+        EndpointsForToken singleReplica = EndpointsForToken.of(token, replica);
+        return new ForToken(keyspace, ConsistencyLevel.ONE, token, singleReplica, EndpointsForToken.empty(token), singleReplica, singleReplica);
+    }
+
+    public static ForRange forSingleReplica(Keyspace keyspace, AbstractBounds<PartitionPosition> range, Replica replica)
+    {
+        EndpointsForRange singleReplica = EndpointsForRange.of(replica);
+        return new ForRange(keyspace, ConsistencyLevel.ONE, range, singleReplica, singleReplica);
+    }
+
+    public static ForToken forCounterWrite(Keyspace keyspace, Token token, Replica replica)
+    {
+        return forSingleReplica(keyspace, token, replica);
+    }
+
+    public static ForToken forBatchlogWrite(Keyspace keyspace, Collection<InetAddressAndPort> endpoints) throws UnavailableException
+    {
+        // A single case we write not for range or token, but multiple mutations to many tokens
+        Token token = DatabaseDescriptor.getPartitioner().getMinimumToken();
+        EndpointsForToken natural = EndpointsForToken.copyOf(token, SystemReplicas.getSystemReplicas(endpoints));
+        EndpointsForToken pending = EndpointsForToken.empty(token);
+        ConsistencyLevel consistencyLevel = natural.size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO;
+
+        return forWriteWithDownNodes(keyspace, consistencyLevel, token, natural, pending);
+    }
+
+    public static ForToken forWriteWithDownNodes(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token) throws UnavailableException
+    {
+        return forWrite(keyspace, consistencyLevel, token, Predicates.alwaysTrue());
+    }
+
+    public static ForToken forWrite(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, Predicate<InetAddressAndPort> isAlive) throws UnavailableException
+    {
+        EndpointsForToken natural = StorageService.getNaturalReplicasForToken(keyspace.getName(), token);
+        EndpointsForToken pending = StorageService.instance.getTokenMetadata().pendingEndpointsForToken(token, keyspace.getName());
+        return forWrite(keyspace, consistencyLevel, token, natural, pending, isAlive);
+    }
+
+    public static ForToken forWriteWithDownNodes(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, EndpointsForToken natural, EndpointsForToken pending) throws UnavailableException
+    {
+        return forWrite(keyspace, consistencyLevel, token, natural, pending, Predicates.alwaysTrue());
+    }
+
+    public static ForToken forWrite(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, EndpointsForToken natural, EndpointsForToken pending, Predicate<InetAddressAndPort> isAlive) throws UnavailableException
+    {
+        if (Endpoints.haveConflicts(natural, pending))
+        {
+            natural = Endpoints.resolveConflictsInNatural(natural, pending);
+            pending = Endpoints.resolveConflictsInPending(natural, pending);
+        }
+
+        if (!any(natural, Replica::isTransient) && !any(pending, Replica::isTransient))
+        {
+            EndpointsForToken selected = Endpoints.concat(natural, pending).filter(r -> isAlive.test(r.endpoint()));
+            return new ForToken(keyspace, consistencyLevel, token, natural, pending, selected);
+        }
+
+        return forWrite(keyspace, consistencyLevel, token, consistencyLevel.blockForWrite(keyspace, pending), natural, pending, isAlive);
+    }
+
+    public static ReplicaLayout.ForPaxos forPaxos(Keyspace keyspace, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
+    {
+        Token tk = key.getToken();
+        EndpointsForToken natural = StorageService.getNaturalReplicasForToken(keyspace.getName(), tk);
+        EndpointsForToken pending = StorageService.instance.getTokenMetadata().pendingEndpointsForToken(tk, keyspace.getName());
+        if (Endpoints.haveConflicts(natural, pending))
+        {
+            natural = Endpoints.resolveConflictsInNatural(natural, pending);
+            pending = Endpoints.resolveConflictsInPending(natural, pending);
+        }
+
+        // TODO CASSANDRA-14547
+        Replicas.temporaryAssertFull(natural);
+        Replicas.temporaryAssertFull(pending);
+
+        if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
+        {
+            // Restrict natural and pending to node in the local DC only
+            String localDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
+            IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+            Predicate<Replica> isLocalDc = replica -> localDc.equals(snitch.getDatacenter(replica));
+
+            natural = natural.filter(isLocalDc);
+            pending = pending.filter(isLocalDc);
+        }
+
+        int participants = pending.size() + natural.size();
+        int requiredParticipants = participants / 2 + 1; // See CASSANDRA-8346, CASSANDRA-833
+
+        EndpointsForToken all = Endpoints.concat(natural, pending);
+        EndpointsForToken selected = all.filter(IAsyncCallback.isReplicaAlive);
+        if (selected.size() < requiredParticipants)
+            throw UnavailableException.create(consistencyForPaxos, requiredParticipants, selected.size());
+
+        // We cannot allow CAS operations with 2 or more pending endpoints, see #8346.
+        // Note that we fake an impossible number of required nodes in the unavailable exception
+        // to nail home the point that it's an impossible operation no matter how many nodes are live.
+        if (pending.size() > 1)
+            throw new UnavailableException(String.format("Cannot perform LWT operation as there is more than one (%d) pending range movement", pending.size()),
+                                           consistencyForPaxos,
+                                           participants + 1,
+                                           selected.size());
+
+        return new ReplicaLayout.ForPaxos(keyspace, consistencyForPaxos, key.getToken(), requiredParticipants, natural, pending, selected, all);
+    }
+
+    /**
+     * We want to send mutations to as many full replicas as we can, and just as many transient replicas
+     * as we need to meet blockFor.
+     */
+    @VisibleForTesting
+    public static ForToken forWrite(Keyspace keyspace, ConsistencyLevel consistencyLevel, Token token, int blockFor, EndpointsForToken natural, EndpointsForToken pending, Predicate<InetAddressAndPort> livePredicate) throws UnavailableException
+    {
+        EndpointsForToken all = Endpoints.concat(natural, pending);
+        EndpointsForToken selected = all
+                .select()
+                .add(r -> r.isFull() && livePredicate.test(r.endpoint()))
+                .add(r -> r.isTransient() && livePredicate.test(r.endpoint()), blockFor)
+                .get();
+
+        consistencyLevel.assureSufficientLiveNodesForWrite(keyspace, selected, pending);
+
+        return new ForToken(keyspace, consistencyLevel, token, natural, pending, selected, all);
+    }
+
+    public static ForToken forRead(Keyspace keyspace, Token token, ConsistencyLevel consistencyLevel, SpeculativeRetryPolicy retry)
+    {
+        EndpointsForToken natural = StorageProxy.getLiveSortedReplicasForToken(keyspace, token);
+        EndpointsForToken selected = consistencyLevel.filterForQuery(keyspace, natural, retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE));
+
+        // Throw UAE early if we don't have enough replicas.
+        consistencyLevel.assureSufficientLiveNodesForRead(keyspace, selected);
+
+        return new ForToken(keyspace, consistencyLevel, token, natural, null, selected);
+    }
+
+    public static ForRange forRangeRead(Keyspace keyspace, ConsistencyLevel consistencyLevel, AbstractBounds<PartitionPosition> range, EndpointsForRange natural, EndpointsForRange selected)
+    {
+        return new ForRange(keyspace, consistencyLevel, range, natural, selected);
+    }
+
+    public String toString()
+    {
+        return "ReplicaLayout [ CL: " + consistencyLevel + " keyspace: " + keyspace + " natural: " + natural + "pending: " + pending + " selected: " + selected + " ]";
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/ReplicaMultimap.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReplicaMultimap.java b/src/java/org/apache/cassandra/locator/ReplicaMultimap.java
new file mode 100644
index 0000000..3e3fcb4
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/ReplicaMultimap.java
@@ -0,0 +1,127 @@
+/*
+ * 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.AbstractMap;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+public abstract class ReplicaMultimap<K, C extends ReplicaCollection<?>>
+{
+    final Map<K, C> map;
+    ReplicaMultimap(Map<K, C> map)
+    {
+        this.map = map;
+    }
+
+    public abstract C get(K key);
+    public C getIfPresent(K key) { return map.get(key); }
+
+    public static abstract class Mutable
+            <K, MutableCollection extends ReplicaCollection.Mutable<?>>
+            extends ReplicaMultimap<K, MutableCollection>
+    {
+        protected abstract MutableCollection newMutable(K key);
+
+        Mutable()
+        {
+            super(new HashMap<>());
+        }
+
+        public MutableCollection get(K key)
+        {
+            Preconditions.checkNotNull(key);
+            return map.computeIfAbsent(key, k -> newMutable(key));
+        }
+
+        public void put(K key, Replica replica)
+        {
+            Preconditions.checkNotNull(key);
+            Preconditions.checkNotNull(replica);
+            get(key).add(replica);
+        }
+    }
+
+    public Iterable<Replica> flattenValues()
+    {
+        return Iterables.concat(map.values());
+    }
+
+    public Iterable<Map.Entry<K, Replica>> flattenEntries()
+    {
+        return () -> {
+            Stream<Map.Entry<K, Replica>> s = map.entrySet()
+                                                 .stream()
+                                                 .flatMap(entry -> entry.getValue()
+                                                                        .stream()
+                                                                        .map(replica -> (Map.Entry<K, Replica>)new AbstractMap.SimpleImmutableEntry<>(entry.getKey(), replica)));
+            return s.iterator();
+        };
+    }
+
+    public boolean isEmpty()
+    {
+        return map.isEmpty();
+    }
+
+    public boolean containsKey(Object key)
+    {
+        return map.containsKey(key);
+    }
+
+    public Set<K> keySet()
+    {
+        return map.keySet();
+    }
+
+    public Set<Map.Entry<K, C>> entrySet()
+    {
+        return map.entrySet();
+    }
+
+    public Map<K, C> asMap()
+    {
+        return map;
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ReplicaMultimap<?, ?> that = (ReplicaMultimap<?, ?>) o;
+        return Objects.equals(map, that.map);
+    }
+
+    public int hashCode()
+    {
+        return map.hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return map.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/Replicas.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Replicas.java b/src/java/org/apache/cassandra/locator/Replicas.java
new file mode 100644
index 0000000..299e6ec
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/Replicas.java
@@ -0,0 +1,83 @@
+/*
+ * 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.List;
+
+import com.google.common.collect.Iterables;
+
+import static com.google.common.collect.Iterables.all;
+
+public class Replicas
+{
+
+    public static int countFull(ReplicaCollection<?> liveReplicas)
+    {
+        int count = 0;
+        for (Replica replica : liveReplicas)
+            if (replica.isFull())
+                ++count;
+        return count;
+    }
+
+    /**
+     * A placeholder for areas of the code that cannot yet handle transient replicas, but should do so in future
+     */
+    public static void temporaryAssertFull(Replica replica)
+    {
+        if (!replica.isFull())
+        {
+            throw new UnsupportedOperationException("transient replicas are currently unsupported: " + replica);
+        }
+    }
+
+    /**
+     * A placeholder for areas of the code that cannot yet handle transient replicas, but should do so in future
+     */
+    public static void temporaryAssertFull(Iterable<Replica> replicas)
+    {
+        if (!all(replicas, Replica::isFull))
+        {
+            throw new UnsupportedOperationException("transient replicas are currently unsupported: " + Iterables.toString(replicas));
+        }
+    }
+
+    /**
+     * For areas of the code that should never see a transient replica
+     */
+    public static void assertFull(Iterable<Replica> replicas)
+    {
+        if (!all(replicas, Replica::isFull))
+        {
+            throw new UnsupportedOperationException("transient replicas are currently unsupported: " + Iterables.toString(replicas));
+        }
+    }
+
+    public static List<String> stringify(ReplicaCollection<?> replicas, boolean withPort)
+    {
+        List<String> stringEndpoints = new ArrayList<>(replicas.size());
+        for (Replica replica: replicas)
+        {
+            stringEndpoints.add(replica.endpoint().getHostAddress(withPort));
+        }
+        return stringEndpoints;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/ReplicationFactor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/ReplicationFactor.java b/src/java/org/apache/cassandra/locator/ReplicationFactor.java
new file mode 100644
index 0000000..c0ed31f
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/ReplicationFactor.java
@@ -0,0 +1,130 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicates;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class ReplicationFactor
+{
+    public static final ReplicationFactor ZERO = new ReplicationFactor(0);
+
+    public final int allReplicas;
+    public final int fullReplicas;
+
+    private ReplicationFactor(int allReplicas, int transientReplicas)
+    {
+        validate(allReplicas, transientReplicas);
+        this.allReplicas = allReplicas;
+        this.fullReplicas = allReplicas - transientReplicas;
+    }
+
+    public int transientReplicas()
+    {
+        return allReplicas - fullReplicas;
+    }
+
+    public boolean hasTransientReplicas()
+    {
+        return allReplicas != fullReplicas;
+    }
+
+    private ReplicationFactor(int allReplicas)
+    {
+        this(allReplicas, 0);
+    }
+
+    static void validate(int totalRF, int transientRF)
+    {
+        Preconditions.checkArgument(transientRF == 0 || DatabaseDescriptor.isTransientReplicationEnabled(),
+                                    "Transient replication is not enabled on this node");
+        Preconditions.checkArgument(totalRF >= 0,
+                                    "Replication factor must be non-negative, found %s", totalRF);
+        Preconditions.checkArgument(transientRF == 0 || transientRF < totalRF,
+                                    "Transient replicas must be zero, or less than total replication factor. For %s/%s", totalRF, transientRF);
+        if (transientRF > 0)
+        {
+            Preconditions.checkArgument(DatabaseDescriptor.getNumTokens() == 1,
+                                        "Transient nodes are not allowed with multiple tokens");
+            Stream<InetAddressAndPort> endpoints = Stream.concat(Gossiper.instance.getLiveMembers().stream(), Gossiper.instance.getUnreachableMembers().stream());
+            List<InetAddressAndPort> badVersionEndpoints = endpoints.filter(Predicates.not(FBUtilities.getBroadcastAddressAndPort()::equals))
+                                                                    .filter(endpoint -> Gossiper.instance.getReleaseVersion(endpoint) != null && Gossiper.instance.getReleaseVersion(endpoint).major < 4)
+                                                                    .collect(Collectors.toList());
+            if (!badVersionEndpoints.isEmpty())
+                throw new AssertionError("Transient replication is not supported in mixed version clusters with nodes < 4.0. Bad nodes: " + badVersionEndpoints);
+        }
+        else if (transientRF < 0)
+        {
+            throw new AssertionError(String.format("Amount of transient nodes should be strictly positive, but was: '%d'", transientRF));
+        }
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ReplicationFactor that = (ReplicationFactor) o;
+        return allReplicas == that.allReplicas && fullReplicas == that.fullReplicas;
+    }
+
+    public int hashCode()
+    {
+        return Objects.hash(allReplicas, fullReplicas);
+    }
+
+    public static ReplicationFactor fullOnly(int totalReplicas)
+    {
+        return new ReplicationFactor(totalReplicas);
+    }
+
+    public static ReplicationFactor withTransient(int totalReplicas, int transientReplicas)
+    {
+        return new ReplicationFactor(totalReplicas, transientReplicas);
+    }
+
+    public static ReplicationFactor fromString(String s)
+    {
+        if (s.contains("/"))
+        {
+            String[] parts = s.split("/");
+            Preconditions.checkArgument(parts.length == 2,
+                                        "Replication factor format is <replicas> or <replicas>/<transient>");
+            return new ReplicationFactor(Integer.valueOf(parts[0]), Integer.valueOf(parts[1]));
+        }
+        else
+        {
+            return new ReplicationFactor(Integer.valueOf(s), 0);
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return "rf(" + allReplicas + (hasTransientReplicas() ? '/' + transientReplicas() : "") + ')';
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/SimpleSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleSnitch.java b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
index e31fc6b..d605b6e 100644
--- a/src/java/org/apache/cassandra/locator/SimpleSnitch.java
+++ b/src/java/org/apache/cassandra/locator/SimpleSnitch.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.util.List;
-
 /**
  * A simple endpoint snitch implementation that treats Strategy order as proximity,
  * allowing non-read-repaired reads to prefer a single endpoint, which improves
@@ -37,12 +35,14 @@ public class SimpleSnitch extends AbstractEndpointSnitch
     }
 
     @Override
-    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
+    public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddress)
     {
         // Optimization to avoid walking the list
+        return unsortedAddress;
     }
 
-    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+    @Override
+    public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2)
     {
         // Making all endpoints equal ensures we won't change the original ordering (since
         // Collections.sort is guaranteed to be stable)


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


[17/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
index 72b5e2a..acfe71a 100644
--- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
+++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
@@ -19,8 +19,9 @@
 package org.apache.cassandra.db;
 
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Comparator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,6 +31,8 @@ import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Splitter;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.PendingRangeCalculatorService;
 import org.apache.cassandra.service.StorageService;
@@ -68,7 +71,7 @@ public class DiskBoundaryManager
 
     private static DiskBoundaries getDiskBoundaryValue(ColumnFamilyStore cfs)
     {
-        Collection<Range<Token>> localRanges;
+        RangesAtEndpoint localRanges;
 
         long ringVersion;
         TokenMetadata tmd;
@@ -87,7 +90,7 @@ public class DiskBoundaryManager
                 // Reason we use use the future settled TMD is that if we decommission a node, we want to stream
                 // from that node to the correct location on disk, if we didn't, we would put new files in the wrong places.
                 // We do this to minimize the amount of data we need to move in rebalancedisks once everything settled
-                localRanges = cfs.keyspace.getReplicationStrategy().getAddressRanges(tmd.cloneAfterAllSettled()).get(FBUtilities.getBroadcastAddressAndPort());
+                localRanges = cfs.keyspace.getReplicationStrategy().getAddressReplicas(tmd.cloneAfterAllSettled(), FBUtilities.getBroadcastAddressAndPort());
             }
             logger.debug("Got local ranges {} (ringVersion = {})", localRanges, ringVersion);
         }
@@ -106,9 +109,18 @@ public class DiskBoundaryManager
         if (localRanges == null || localRanges.isEmpty())
             return new DiskBoundaries(dirs, null, ringVersion, directoriesVersion);
 
-        List<Range<Token>> sortedLocalRanges = Range.sort(localRanges);
+        // note that Range.sort unwraps any wraparound ranges, so we need to sort them here
+        List<Range<Token>> fullLocalRanges = Range.sort(localRanges.stream()
+                                                                   .filter(Replica::isFull)
+                                                                   .map(Replica::range)
+                                                                   .collect(Collectors.toList()));
+        List<Range<Token>> transientLocalRanges = Range.sort(localRanges.stream()
+                                                                        .filter(Replica::isTransient)
+                                                                        .map(Replica::range)
+                                                                        .collect(Collectors.toList()));
+
+        List<PartitionPosition> positions = getDiskBoundaries(fullLocalRanges, transientLocalRanges, cfs.getPartitioner(), dirs);
 
-        List<PartitionPosition> positions = getDiskBoundaries(sortedLocalRanges, cfs.getPartitioner(), dirs);
         return new DiskBoundaries(dirs, positions, ringVersion, directoriesVersion);
     }
 
@@ -121,15 +133,26 @@ public class DiskBoundaryManager
      *
      * The final entry in the returned list will always be the partitioner maximum tokens upper key bound
      */
-    private static List<PartitionPosition> getDiskBoundaries(List<Range<Token>> sortedLocalRanges, IPartitioner partitioner, Directories.DataDirectory[] dataDirectories)
+    private static List<PartitionPosition> getDiskBoundaries(List<Range<Token>> fullRanges, List<Range<Token>> transientRanges, IPartitioner partitioner, Directories.DataDirectory[] dataDirectories)
     {
         assert partitioner.splitter().isPresent();
+
         Splitter splitter = partitioner.splitter().get();
         boolean dontSplitRanges = DatabaseDescriptor.getNumTokens() > 1;
-        List<Token> boundaries = splitter.splitOwnedRanges(dataDirectories.length, sortedLocalRanges, dontSplitRanges);
+
+        List<Splitter.WeightedRange> weightedRanges = new ArrayList<>(fullRanges.size() + transientRanges.size());
+        for (Range<Token> r : fullRanges)
+            weightedRanges.add(new Splitter.WeightedRange(1.0, r));
+
+        for (Range<Token> r : transientRanges)
+            weightedRanges.add(new Splitter.WeightedRange(0.1, r));
+
+        weightedRanges.sort(Comparator.comparing(Splitter.WeightedRange::left));
+
+        List<Token> boundaries = splitter.splitOwnedRanges(dataDirectories.length, weightedRanges, dontSplitRanges);
         // If we can't split by ranges, split evenly to ensure utilisation of all disks
         if (dontSplitRanges && boundaries.size() < dataDirectories.length)
-            boundaries = splitter.splitOwnedRanges(dataDirectories.length, sortedLocalRanges, false);
+            boundaries = splitter.splitOwnedRanges(dataDirectories.length, weightedRanges, false);
 
         List<PartitionPosition> diskBoundaries = new ArrayList<>();
         for (int i = 0; i < boundaries.size() - 1; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index c162697..436b7ef 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -503,6 +503,7 @@ public class Memtable implements Comparable<Memtable>
                                                 toFlush.size(),
                                                 ActiveRepairService.UNREPAIRED_SSTABLE,
                                                 ActiveRepairService.NO_PENDING_REPAIR,
+                                                false,
                                                 sstableMetadataCollector,
                                                 new SerializationHeader(true, cfs.metadata(), columns, stats), txn);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/MutationVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 8386048..9660f65 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.cassandra.exceptions.WriteTimeoutException;
@@ -38,7 +37,7 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
         Tracing.trace("Payload application resulted in WriteTimeout, not replying");
     }
 
-    public void doVerb(MessageIn<Mutation> message, int id)  throws IOException
+    public void doVerb(MessageIn<Mutation> message, int id)
     {
         // Check if there were any forwarding headers in this message
         InetAddressAndPort from = (InetAddressAndPort)message.parameters.get(ParameterType.FORWARD_FROM);
@@ -69,7 +68,7 @@ public class MutationVerbHandler implements IVerbHandler<Mutation>
         }
     }
 
-    private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from) throws IOException
+    private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from)
     {
         // tell the recipients who to send their ack to
         MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(ParameterType.FORWARD_FROM, from);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 2bfb434..7eab016 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -24,7 +24,6 @@ import java.util.List;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.filter.*;
@@ -61,6 +60,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
 
     private PartitionRangeReadCommand(boolean isDigest,
                                      int digestVersion,
+                                     boolean acceptsTransient,
                                      TableMetadata metadata,
                                      int nowInSec,
                                      ColumnFilter columnFilter,
@@ -69,7 +69,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
                                      DataRange dataRange,
                                      IndexMetadata index)
     {
-        super(Kind.PARTITION_RANGE, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
+        super(Kind.PARTITION_RANGE, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         this.dataRange = dataRange;
     }
 
@@ -82,6 +82,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(false,
                                              0,
+                                             false,
                                              metadata,
                                              nowInSec,
                                              columnFilter,
@@ -103,6 +104,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(false,
                                              0,
+                                             false,
                                              metadata,
                                              nowInSec,
                                              ColumnFilter.all(metadata),
@@ -151,6 +153,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
         // on the ring.
         return new PartitionRangeReadCommand(isDigestQuery(),
                                              digestVersion(),
+                                             acceptsTransient(),
                                              metadata(),
                                              nowInSec(),
                                              columnFilter(),
@@ -164,6 +167,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(isDigestQuery(),
                                              digestVersion(),
+                                             acceptsTransient(),
                                              metadata(),
                                              nowInSec(),
                                              columnFilter(),
@@ -177,6 +181,21 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(true,
                                              digestVersion(),
+                                             false,
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand copyAsTransientQuery()
+    {
+        return new PartitionRangeReadCommand(false,
+                                             0,
+                                             true,
                                              metadata(),
                                              nowInSec(),
                                              columnFilter(),
@@ -191,6 +210,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(isDigestQuery(),
                                              digestVersion(),
+                                             acceptsTransient(),
                                              metadata(),
                                              nowInSec(),
                                              columnFilter(),
@@ -205,6 +225,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
     {
         return new PartitionRangeReadCommand(isDigestQuery(),
                                              digestVersion(),
+                                             acceptsTransient(),
                                              metadata(),
                                              nowInSec(),
                                              columnFilter(),
@@ -406,6 +427,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
                                        int version,
                                        boolean isDigest,
                                        int digestVersion,
+                                       boolean acceptsTransient,
                                        TableMetadata metadata,
                                        int nowInSec,
                                        ColumnFilter columnFilter,
@@ -415,7 +437,7 @@ public class PartitionRangeReadCommand extends ReadCommand implements PartitionR
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);
-            return new PartitionRangeReadCommand(isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
+            return new PartitionRangeReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 0262140..736e3a3 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.db.transform.RTBoundCloser;
 import org.apache.cassandra.db.transform.RTBoundValidator;
 import org.apache.cassandra.db.transform.StoppingTransformation;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.UnknownIndexException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.IndexNotAvailableException;
@@ -68,6 +67,7 @@ public abstract class ReadCommand extends AbstractReadQuery
     private final Kind kind;
 
     private final boolean isDigestQuery;
+    private final boolean acceptsTransient;
     // if a digest query, the version for which the digest is expected. Ignored if not a digest.
     private int digestVersion;
 
@@ -80,6 +80,7 @@ public abstract class ReadCommand extends AbstractReadQuery
                                                 int version,
                                                 boolean isDigest,
                                                 int digestVersion,
+                                                boolean acceptsTransient,
                                                 TableMetadata metadata,
                                                 int nowInSec,
                                                 ColumnFilter columnFilter,
@@ -104,6 +105,7 @@ public abstract class ReadCommand extends AbstractReadQuery
     protected ReadCommand(Kind kind,
                           boolean isDigestQuery,
                           int digestVersion,
+                          boolean acceptsTransient,
                           TableMetadata metadata,
                           int nowInSec,
                           ColumnFilter columnFilter,
@@ -115,6 +117,7 @@ public abstract class ReadCommand extends AbstractReadQuery
         this.kind = kind;
         this.isDigestQuery = isDigestQuery;
         this.digestVersion = digestVersion;
+        this.acceptsTransient = acceptsTransient;
         this.index = index;
     }
 
@@ -176,6 +179,14 @@ public abstract class ReadCommand extends AbstractReadQuery
     }
 
     /**
+     * @return Whether this query expects only a transient data response, or a full response
+     */
+    public boolean acceptsTransient()
+    {
+        return acceptsTransient;
+    }
+
+    /**
      * Index (metadata) chosen for this query. Can be null.
      *
      * @return index (metadata) chosen for this query
@@ -210,6 +221,7 @@ public abstract class ReadCommand extends AbstractReadQuery
      * Returns a copy of this command with isDigestQuery set to true.
      */
     public abstract ReadCommand copyAsDigestQuery();
+    public abstract ReadCommand copyAsTransientQuery();
 
     protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadExecutionController executionController);
 
@@ -569,6 +581,16 @@ public abstract class ReadCommand extends AbstractReadQuery
             return (flags & 0x01) != 0;
         }
 
+        private static boolean acceptsTransient(int flags)
+        {
+            return (flags & 0x08) != 0;
+        }
+
+        private static int acceptsTransientFlag(boolean acceptsTransient)
+        {
+            return acceptsTransient ? 0x08 : 0;
+        }
+
         // We don't set this flag anymore, but still look if we receive a
         // command with it set in case someone is using thrift a mixed 3.0/4.0+
         // cluster (which is unsupported). This is also a reminder for not
@@ -592,7 +614,11 @@ public abstract class ReadCommand extends AbstractReadQuery
         public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
         {
             out.writeByte(command.kind.ordinal());
-            out.writeByte(digestFlag(command.isDigestQuery()) | indexFlag(null != command.indexMetadata()));
+            out.writeByte(
+                    digestFlag(command.isDigestQuery())
+                    | indexFlag(null != command.indexMetadata())
+                    | acceptsTransientFlag(command.acceptsTransient())
+            );
             if (command.isDigestQuery())
                 out.writeUnsignedVInt(command.digestVersion());
             command.metadata().id.serialize(out);
@@ -611,6 +637,7 @@ public abstract class ReadCommand extends AbstractReadQuery
             Kind kind = Kind.values()[in.readByte()];
             int flags = in.readByte();
             boolean isDigest = isDigest(flags);
+            boolean acceptsTransient = acceptsTransient(flags);
             // Shouldn't happen or it's a user error (see comment above) but
             // better complain loudly than doing the wrong thing.
             if (isForThrift(flags))
@@ -628,7 +655,7 @@ public abstract class ReadCommand extends AbstractReadQuery
             DataLimits limits = DataLimits.serializer.deserialize(in, version,  metadata.comparator);
             IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
 
-            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
+            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         }
 
         private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, TableMetadata metadata) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/SSTableImporter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SSTableImporter.java b/src/java/org/apache/cassandra/db/SSTableImporter.java
index c919d25..7597f82 100644
--- a/src/java/org/apache/cassandra/db/SSTableImporter.java
+++ b/src/java/org/apache/cassandra/db/SSTableImporter.java
@@ -349,9 +349,9 @@ public class SSTableImporter
             }
             if (options.clearRepaired)
             {
-                descriptor.getMetadataSerializer().mutateRepaired(descriptor,
-                                                                  ActiveRepairService.UNREPAIRED_SSTABLE,
-                                                                  null);
+                descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, ActiveRepairService.UNREPAIRED_SSTABLE,
+                                                                        null,
+                                                                        false);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 97ab210..c81185e 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
@@ -71,6 +70,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     @VisibleForTesting
     protected SinglePartitionReadCommand(boolean isDigest,
                                          int digestVersion,
+                                         boolean acceptsTransient,
                                          TableMetadata metadata,
                                          int nowInSec,
                                          ColumnFilter columnFilter,
@@ -80,7 +80,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
                                          ClusteringIndexFilter clusteringIndexFilter,
                                          IndexMetadata index)
     {
-        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
+        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         assert partitionKey.getPartitioner() == metadata.partitioner;
         this.partitionKey = partitionKey;
         this.clusteringIndexFilter = clusteringIndexFilter;
@@ -111,6 +111,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     {
         return new SinglePartitionReadCommand(false,
                                               0,
+                                              false,
                                               metadata,
                                               nowInSec,
                                               columnFilter,
@@ -286,6 +287,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     {
         return new SinglePartitionReadCommand(isDigestQuery(),
                                               digestVersion(),
+                                              acceptsTransient(),
                                               metadata(),
                                               nowInSec(),
                                               columnFilter(),
@@ -300,6 +302,22 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     {
         return new SinglePartitionReadCommand(true,
                                               digestVersion(),
+                                              acceptsTransient(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand copyAsTransientQuery()
+    {
+        return new SinglePartitionReadCommand(false,
+                                              0,
+                                              true,
                                               metadata(),
                                               nowInSec(),
                                               columnFilter(),
@@ -315,6 +333,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
     {
         return new SinglePartitionReadCommand(isDigestQuery(),
                                               digestVersion(),
+                                              acceptsTransient(),
                                               metadata(),
                                               nowInSec(),
                                               columnFilter(),
@@ -1064,6 +1083,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
                                        int version,
                                        boolean isDigest,
                                        int digestVersion,
+                                       boolean acceptsTransient,
                                        TableMetadata metadata,
                                        int nowInSec,
                                        ColumnFilter columnFilter,
@@ -1074,7 +1094,7 @@ public class SinglePartitionReadCommand extends ReadCommand implements SinglePar
         {
             DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readValue(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
-            return new SinglePartitionReadCommand(isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
+            return new SinglePartitionReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index fb9e889..ff070a3 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -29,13 +29,15 @@ import java.util.stream.StreamSupport;
 import javax.management.openmbean.OpenDataException;
 import javax.management.openmbean.TabularData;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
 import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,6 +73,8 @@ import static java.util.Collections.singletonMap;
 
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
 
 public final class SystemKeyspace
 {
@@ -95,12 +99,10 @@ public final class SystemKeyspace
     public static final String LOCAL = "local";
     public static final String PEERS_V2 = "peers_v2";
     public static final String PEER_EVENTS_V2 = "peer_events_v2";
-    public static final String RANGE_XFERS = "range_xfers";
     public static final String COMPACTION_HISTORY = "compaction_history";
     public static final String SSTABLE_ACTIVITY = "sstable_activity";
     public static final String SIZE_ESTIMATES = "size_estimates";
-    public static final String AVAILABLE_RANGES = "available_ranges";
-    public static final String TRANSFERRED_RANGES = "transferred_ranges";
+    public static final String AVAILABLE_RANGES_V2 = "available_ranges_v2";
     public static final String TRANSFERRED_RANGES_V2 = "transferred_ranges_v2";
     public static final String VIEW_BUILDS_IN_PROGRESS = "view_builds_in_progress";
     public static final String BUILT_VIEWS = "built_views";
@@ -110,6 +112,8 @@ public final class SystemKeyspace
     @Deprecated public static final String LEGACY_PEERS = "peers";
     @Deprecated public static final String LEGACY_PEER_EVENTS = "peer_events";
     @Deprecated public static final String LEGACY_TRANSFERRED_RANGES = "transferred_ranges";
+    @Deprecated public static final String LEGACY_AVAILABLE_RANGES = "available_ranges";
+
 
     public static final TableMetadata Batches =
         parse(BATCHES,
@@ -207,15 +211,6 @@ public final class SystemKeyspace
                 + "PRIMARY KEY ((peer), peer_port))")
                 .build();
 
-    private static final TableMetadata RangeXfers =
-        parse(RANGE_XFERS,
-                "ranges requested for transfer",
-                "CREATE TABLE %s ("
-                + "token_bytes blob,"
-                + "requested_at timestamp,"
-                + "PRIMARY KEY ((token_bytes)))")
-                .build();
-
     private static final TableMetadata CompactionHistory =
         parse(COMPACTION_HISTORY,
                 "week-long compaction history",
@@ -256,14 +251,15 @@ public final class SystemKeyspace
                 + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
                 .build();
 
-    private static final TableMetadata AvailableRanges =
-        parse(AVAILABLE_RANGES,
-                "available keyspace/ranges during bootstrap/replace that are ready to be served",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "ranges set<blob>,"
-                + "PRIMARY KEY ((keyspace_name)))")
-                .build();
+    private static final TableMetadata AvailableRangesV2 =
+    parse(AVAILABLE_RANGES_V2,
+          "available keyspace/ranges during bootstrap/replace that are ready to be served",
+          "CREATE TABLE %s ("
+          + "keyspace_name text,"
+          + "full_ranges set<blob>,"
+          + "transient_ranges set<blob>,"
+          + "PRIMARY KEY ((keyspace_name)))")
+    .build();
 
     private static final TableMetadata TransferredRangesV2 =
         parse(TRANSFERRED_RANGES_V2,
@@ -366,6 +362,16 @@ public final class SystemKeyspace
             + "PRIMARY KEY ((operation, keyspace_name), peer))")
             .build();
 
+    @Deprecated
+    private static final TableMetadata LegacyAvailableRanges =
+        parse(LEGACY_AVAILABLE_RANGES,
+              "available keyspace/ranges during bootstrap/replace that are ready to be served",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "ranges set<blob>,"
+              + "PRIMARY KEY ((keyspace_name)))")
+        .build();
+
     private static TableMetadata.Builder parse(String table, String description, String cql)
     {
         return CreateTableStatement.parse(format(cql, table), SchemaConstants.SYSTEM_KEYSPACE_NAME)
@@ -390,11 +396,11 @@ public final class SystemKeyspace
                          LegacyPeers,
                          PeerEventsV2,
                          LegacyPeerEvents,
-                         RangeXfers,
                          CompactionHistory,
                          SSTableActivity,
                          SizeEstimates,
-                         AvailableRanges,
+                         AvailableRangesV2,
+                         LegacyAvailableRanges,
                          TransferredRangesV2,
                          LegacyTransferredRanges,
                          ViewBuildsInProgress,
@@ -1270,36 +1276,38 @@ public final class SystemKeyspace
         executeInternal(cql, keyspace, table);
     }
 
-    public static synchronized void updateAvailableRanges(String keyspace, Collection<Range<Token>> completedRanges)
+    public static synchronized void updateAvailableRanges(String keyspace, Collection<Range<Token>> completedFullRanges, Collection<Range<Token>> completedTransientRanges)
     {
-        String cql = "UPDATE system.%s SET ranges = ranges + ? WHERE keyspace_name = ?";
-        Set<ByteBuffer> rangesToUpdate = new HashSet<>(completedRanges.size());
-        for (Range<Token> range : completedRanges)
-        {
-            rangesToUpdate.add(rangeToBytes(range));
-        }
-        executeInternal(format(cql, AVAILABLE_RANGES), rangesToUpdate, keyspace);
+        String cql = "UPDATE system.%s SET full_ranges = full_ranges + ?, transient_ranges = transient_ranges + ? WHERE keyspace_name = ?";
+        executeInternal(format(cql, AVAILABLE_RANGES_V2),
+                        completedFullRanges.stream().map(SystemKeyspace::rangeToBytes).collect(Collectors.toSet()),
+                        completedTransientRanges.stream().map(SystemKeyspace::rangeToBytes).collect(Collectors.toSet()),
+                        keyspace);
     }
 
-    public static synchronized Set<Range<Token>> getAvailableRanges(String keyspace, IPartitioner partitioner)
+    public static synchronized RangesAtEndpoint getAvailableRanges(String keyspace, IPartitioner partitioner)
     {
-        Set<Range<Token>> result = new HashSet<>();
         String query = "SELECT * FROM system.%s WHERE keyspace_name=?";
-        UntypedResultSet rs = executeInternal(format(query, AVAILABLE_RANGES), keyspace);
+        UntypedResultSet rs = executeInternal(format(query, AVAILABLE_RANGES_V2), keyspace);
+        InetAddressAndPort endpoint = InetAddressAndPort.getLocalHost();
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(endpoint);
         for (UntypedResultSet.Row row : rs)
         {
-            Set<ByteBuffer> rawRanges = row.getSet("ranges", BytesType.instance);
-            for (ByteBuffer rawRange : rawRanges)
-            {
-                result.add(byteBufferToRange(rawRange, partitioner));
-            }
+            Optional.ofNullable(row.getSet("full_ranges", BytesType.instance))
+                    .ifPresent(full_ranges -> full_ranges.stream()
+                            .map(buf -> byteBufferToRange(buf, partitioner))
+                            .forEach(range -> builder.add(fullReplica(endpoint, range))));
+            Optional.ofNullable(row.getSet("transient_ranges", BytesType.instance))
+                    .ifPresent(transient_ranges -> transient_ranges.stream()
+                            .map(buf -> byteBufferToRange(buf, partitioner))
+                            .forEach(range -> builder.add(transientReplica(endpoint, range))));
         }
-        return ImmutableSet.copyOf(result);
+        return builder.build();
     }
 
     public static void resetAvailableRanges()
     {
-        ColumnFamilyStore availableRanges = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(AVAILABLE_RANGES);
+        ColumnFamilyStore availableRanges = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(AVAILABLE_RANGES_V2);
         availableRanges.truncateBlocking();
     }
 
@@ -1405,7 +1413,13 @@ public final class SystemKeyspace
         return result.one().getString("release_version");
     }
 
-    private static ByteBuffer rangeToBytes(Range<Token> range)
+    @VisibleForTesting
+    public static Set<Range<Token>> rawRangesToRangeSet(Set<ByteBuffer> rawRanges, IPartitioner partitioner)
+    {
+        return rawRanges.stream().map(buf -> byteBufferToRange(buf, partitioner)).collect(Collectors.toSet());
+    }
+
+    static ByteBuffer rangeToBytes(Range<Token> range)
     {
         try (DataOutputBuffer out = new DataOutputBuffer())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
index ea5ff59..e0a58ba 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java
@@ -18,6 +18,11 @@
 
 package org.apache.cassandra.db;
 
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +50,9 @@ public class SystemKeyspaceMigrator40
     static final String peerEventsName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.PEER_EVENTS_V2);
     static final String legacyTransferredRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.LEGACY_TRANSFERRED_RANGES);
     static final String transferredRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.TRANSFERRED_RANGES_V2);
+    static final String legacyAvailableRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.LEGACY_AVAILABLE_RANGES);
+    static final String availableRangesName = String.format("%s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.AVAILABLE_RANGES_V2);
+
 
     private static final Logger logger = LoggerFactory.getLogger(SystemKeyspaceMigrator40.class);
 
@@ -55,6 +63,7 @@ public class SystemKeyspaceMigrator40
         migratePeers();
         migratePeerEvents();
         migrateTransferredRanges();
+        migrateAvailableRanges();
     }
 
     private static void migratePeers()
@@ -181,4 +190,40 @@ public class SystemKeyspaceMigrator40
         logger.info("Migrated {} rows from legacy {} to {}", transferred, legacyTransferredRangesName, transferredRangesName);
     }
 
+    static void migrateAvailableRanges()
+    {
+        ColumnFamilyStore newAvailableRanges = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.AVAILABLE_RANGES_V2);
+
+        if (!newAvailableRanges.isEmpty())
+            return;
+
+        logger.info("{} table was empty, migrating legacy {} to {}", availableRangesName, legacyAvailableRangesName, availableRangesName);
+
+        String query = String.format("SELECT * FROM %s",
+                                     legacyAvailableRangesName);
+
+        String insert = String.format("INSERT INTO %s ("
+                                      + "keyspace_name, "
+                                      + "full_ranges, "
+                                      + "transient_ranges) "
+                                      + " values ( ?, ?, ? )",
+                                      availableRangesName);
+
+        UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000);
+        int transferred = 0;
+        for (UntypedResultSet.Row row : rows)
+        {
+            logger.debug("Transferring row {}", transferred);
+            String keyspace = row.getString("keyspace_name");
+            Set<ByteBuffer> ranges = Optional.ofNullable(row.getSet("ranges", BytesType.instance)).orElse(Collections.emptySet());
+            QueryProcessor.executeInternal(insert,
+                                           keyspace,
+                                           ranges,
+                                           Collections.emptySet());
+            transferred++;
+        }
+
+        logger.info("Migrated {} rows from legacy {} to {}", transferred, legacyAvailableRangesName, availableRangesName);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/TableCQLHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TableCQLHelper.java b/src/java/org/apache/cassandra/db/TableCQLHelper.java
index 550a6d6..f97bebc 100644
--- a/src/java/org/apache/cassandra/db/TableCQLHelper.java
+++ b/src/java/org/apache/cassandra/db/TableCQLHelper.java
@@ -310,6 +310,7 @@ public class TableCQLHelper
         builder.append("\n\tAND max_index_interval = ").append(tableParams.maxIndexInterval);
         builder.append("\n\tAND memtable_flush_period_in_ms = ").append(tableParams.memtableFlushPeriodInMs);
         builder.append("\n\tAND speculative_retry = '").append(tableParams.speculativeRetry).append("'");
+        builder.append("\n\tAND speculative_write_threshold = '").append(tableParams.speculativeWriteThreshold).append("'");
         builder.append("\n\tAND comment = ").append(singleQuote(tableParams.comment));
         builder.append("\n\tAND caching = ").append(toCQL(tableParams.caching.asMap()));
         builder.append("\n\tAND compaction = ").append(toCQL(tableParams.compaction.asMap()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 59bdce6..28ea90a 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -530,12 +530,13 @@ public abstract class AbstractCompactionStrategy
                                                        long keyCount,
                                                        long repairedAt,
                                                        UUID pendingRepair,
+                                                       boolean isTransient,
                                                        MetadataCollector meta,
                                                        SerializationHeader header,
                                                        Collection<Index> indexes,
                                                        LifecycleTransaction txn)
     {
-        return SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, cfs.metadata, meta, header, indexes, txn);
+        return SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, cfs.metadata, meta, header, indexes, txn);
     }
 
     public boolean supportsEarlyOpen()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
index dc16261..24bea06 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java
@@ -158,11 +158,11 @@ public abstract class AbstractStrategyHolder
      * groups they deal with. IOW, if one holder returns true for a given isRepaired/isPendingRepair combo,
      * none of the others should.
      */
-    public abstract boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair);
+    public abstract boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair, boolean isTransient);
 
     public boolean managesSSTable(SSTableReader sstable)
     {
-        return managesRepairedGroup(sstable.isRepaired(), sstable.isPendingRepair());
+        return managesRepairedGroup(sstable.isRepaired(), sstable.isPendingRepair(), sstable.isTransient());
     }
 
     public abstract AbstractCompactionStrategy getStrategyFor(SSTableReader sstable);
@@ -193,6 +193,7 @@ public abstract class AbstractStrategyHolder
                                                                 long keyCount,
                                                                 long repairedAt,
                                                                 UUID pendingRepair,
+                                                                boolean isTransient,
                                                                 MetadataCollector collector,
                                                                 SerializationHeader header,
                                                                 Collection<Index> indexes,
@@ -203,4 +204,6 @@ public abstract class AbstractStrategyHolder
      * if it's not held by this holder
      */
     public abstract int getStrategyIndex(AbstractCompactionStrategy strategy);
+
+    public abstract boolean containsSSTable(SSTableReader sstable);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index a872fea..2a56650 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -23,7 +23,7 @@ import java.lang.management.ManagementFactory;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.LongPredicate;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -34,6 +34,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
+
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,7 +46,6 @@ import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.repair.ValidationPartitionIterator;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -71,7 +73,6 @@ import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.CompactionMetrics;
 import org.apache.cassandra.metrics.TableMetrics;
-import org.apache.cassandra.repair.Validator;
 import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
@@ -81,6 +82,8 @@ import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.Refs;
 
 import static java.util.Collections.singleton;
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
 
 /**
  * <p>
@@ -509,7 +512,10 @@ public class CompactionManager implements CompactionManagerMBean
             return AllSSTableOpStatus.ABORTED;
         }
         // if local ranges is empty, it means no data should remain
-        final Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
+        final RangesAtEndpoint replicas = StorageService.instance.getLocalReplicas(keyspace.getName());
+        final Set<Range<Token>> allRanges = replicas.ranges();
+        final Set<Range<Token>> transientRanges = replicas.filter(Replica::isTransient).ranges();
+        final Set<Range<Token>> fullRanges = replicas.filter(Replica::isFull).ranges();
         final boolean hasIndexes = cfStore.indexManager.hasIndexes();
 
         return parallelAllSSTableOperation(cfStore, new OneSSTableOperation()
@@ -525,8 +531,8 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(LifecycleTransaction txn) throws IOException
             {
-                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges, FBUtilities.nowInSeconds());
-                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
+                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, allRanges, transientRanges, txn.onlyOne().isRepaired(), FBUtilities.nowInSeconds());
+                doCleanupOne(cfStore, txn, cleanupStrategy, replicas.ranges(), fullRanges, transientRanges, hasIndexes);
             }
         }, jobs, OperationType.CLEANUP);
     }
@@ -574,9 +580,8 @@ public class CompactionManager implements CompactionManagerMBean
             logger.info("Partitioner does not support splitting");
             return AllSSTableOpStatus.ABORTED;
         }
-        final Collection<Range<Token>> r = StorageService.instance.getLocalRanges(cfs.keyspace.getName());
 
-        if (r.isEmpty())
+        if (StorageService.instance.getLocalReplicas(cfs.keyspace.getName()).isEmpty())
         {
             logger.info("Relocate cannot run before a node has joined the ring");
             return AllSSTableOpStatus.ABORTED;
@@ -643,7 +648,11 @@ public class CompactionManager implements CompactionManagerMBean
     /**
      * Splits the given token ranges of the given sstables into a pending repair silo
      */
-    public ListenableFuture<?> submitPendingAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, Refs<SSTableReader> sstables, LifecycleTransaction txn, UUID sessionId)
+    public ListenableFuture<?> submitPendingAntiCompaction(ColumnFamilyStore cfs,
+                                                           RangesAtEndpoint tokenRanges,
+                                                           Refs<SSTableReader> sstables,
+                                                           LifecycleTransaction txn,
+                                                           UUID sessionId)
     {
         Runnable runnable = new WrappedRunnable()
         {
@@ -651,7 +660,7 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 try (TableMetrics.TableTimer.Context ctx = cfs.metric.anticompactionTime.time())
                 {
-                    performAnticompaction(cfs, ranges, sstables, txn, ActiveRepairService.UNREPAIRED_SSTABLE, sessionId, sessionId);
+                    performAnticompaction(cfs, tokenRanges, sstables, txn, sessionId);
                 }
             }
         };
@@ -673,48 +682,69 @@ public class CompactionManager implements CompactionManagerMBean
     }
 
     /**
+     * for sstables that are fully contained in the given ranges, just rewrite their metadata with
+     * the pending repair id and remove them from the transaction
+     */
+    private static void mutateFullyContainedSSTables(ColumnFamilyStore cfs,
+                                                     Refs<SSTableReader> refs,
+                                                     Iterator<SSTableReader> sstableIterator,
+                                                     Collection<Range<Token>> ranges,
+                                                     LifecycleTransaction txn,
+                                                     UUID sessionID,
+                                                     boolean isTransient) throws IOException
+    {
+        if (ranges.isEmpty())
+            return;
+
+        List<Range<Token>> normalizedRanges = Range.normalize(ranges);
+
+        Set<SSTableReader> fullyContainedSSTables = findSSTablesToAnticompact(sstableIterator, normalizedRanges, sessionID);
+
+        cfs.metric.bytesMutatedAnticompaction.inc(SSTableReader.getTotalBytes(fullyContainedSSTables));
+        cfs.getCompactionStrategyManager().mutateRepaired(fullyContainedSSTables, UNREPAIRED_SSTABLE, sessionID, isTransient);
+        // since we're just re-writing the sstable metdata for the fully contained sstables, we don't want
+        // them obsoleted when the anti-compaction is complete. So they're removed from the transaction here
+        txn.cancel(fullyContainedSSTables);
+        refs.release(fullyContainedSSTables);
+    }
+
+    /**
      * Make sure the {validatedForRepair} are marked for compaction before calling this.
      *
      * Caller must reference the validatedForRepair sstables (via ParentRepairSession.getActiveRepairedSSTableRefs(..)).
      *
      * @param cfs
-     * @param ranges Ranges that the repair was carried out on
+     * @param ranges token ranges to be repaired
      * @param validatedForRepair SSTables containing the repaired ranges. Should be referenced before passing them.
-     * @param parentRepairSession parent repair session ID
+     * @param sessionID the repair session we're anti-compacting for
      * @throws InterruptedException
      * @throws IOException
      */
     public void performAnticompaction(ColumnFamilyStore cfs,
-                                      Collection<Range<Token>> ranges,
+                                      RangesAtEndpoint ranges,
                                       Refs<SSTableReader> validatedForRepair,
                                       LifecycleTransaction txn,
-                                      long repairedAt,
-                                      UUID pendingRepair,
-                                      UUID parentRepairSession) throws InterruptedException, IOException
+                                      UUID sessionID) throws IOException
     {
         try
         {
-            ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(parentRepairSession);
+            ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
             Preconditions.checkArgument(!prs.isPreview(), "Cannot anticompact for previews");
+            Preconditions.checkArgument(!ranges.isEmpty(), "No ranges to anti-compact");
 
             if (logger.isInfoEnabled())
-                logger.info("{} Starting anticompaction for {}.{} on {}/{} sstables", PreviewKind.NONE.logPrefix(parentRepairSession), cfs.keyspace.getName(), cfs.getTableName(), validatedForRepair.size(), cfs.getLiveSSTables().size());
+                logger.info("{} Starting anticompaction for {}.{} on {}/{} sstables", PreviewKind.NONE.logPrefix(sessionID), cfs.keyspace.getName(), cfs.getTableName(), validatedForRepair.size(), cfs.getLiveSSTables().size());
             if (logger.isTraceEnabled())
-                logger.trace("{} Starting anticompaction for ranges {}", PreviewKind.NONE.logPrefix(parentRepairSession), ranges);
-            Set<SSTableReader> sstables = new HashSet<>(validatedForRepair);
-
-            Iterator<SSTableReader> sstableIterator = sstables.iterator();
-            List<Range<Token>> normalizedRanges = Range.normalize(ranges);
+                logger.trace("{} Starting anticompaction for ranges {}", PreviewKind.NONE.logPrefix(sessionID), ranges);
 
-            Set<SSTableReader> fullyContainedSSTables = findSSTablesToAnticompact(sstableIterator, normalizedRanges, parentRepairSession);
+            Set<SSTableReader> sstables = new HashSet<>(validatedForRepair);
+            validateSSTableBoundsForAnticompaction(sessionID, sstables, ranges);
+            mutateFullyContainedSSTables(cfs, validatedForRepair, sstables.iterator(), ranges.fullRanges(), txn, sessionID, false);
+            mutateFullyContainedSSTables(cfs, validatedForRepair, sstables.iterator(), ranges.transientRanges(), txn, sessionID, true);
 
-            cfs.metric.bytesMutatedAnticompaction.inc(SSTableReader.getTotalBytes(fullyContainedSSTables));
-            cfs.getCompactionStrategyManager().mutateRepaired(fullyContainedSSTables, repairedAt, pendingRepair);
-            txn.cancel(fullyContainedSSTables);
-            validatedForRepair.release(fullyContainedSSTables);
             assert txn.originals().equals(sstables);
             if (!sstables.isEmpty())
-                doAntiCompaction(cfs, ranges, txn, repairedAt, pendingRepair);
+                doAntiCompaction(cfs, ranges, txn, sessionID);
             txn.finish();
         }
         finally
@@ -723,7 +753,28 @@ public class CompactionManager implements CompactionManagerMBean
             txn.close();
         }
 
-        logger.info("{} Completed anticompaction successfully", PreviewKind.NONE.logPrefix(parentRepairSession));
+        logger.info("{} Completed anticompaction successfully", PreviewKind.NONE.logPrefix(sessionID));
+    }
+
+    static void validateSSTableBoundsForAnticompaction(UUID sessionID,
+                                                       Collection<SSTableReader> sstables,
+                                                       RangesAtEndpoint ranges)
+    {
+        List<Range<Token>> normalizedRanges = Range.normalize(ranges.ranges());
+        for (SSTableReader sstable : sstables)
+        {
+            Bounds<Token> bounds = new Bounds<>(sstable.first.getToken(), sstable.last.getToken());
+
+            if (!Iterables.any(normalizedRanges, r -> (r.contains(bounds.left) && r.contains(bounds.right)) || r.intersects(bounds)))
+            {
+                // this should never happen - in PendingAntiCompaction#getSSTables we select all sstables that intersect the repaired ranges, that can't have changed here
+                String message = String.format("%s SSTable %s (%s) does not intersect repaired ranges %s, this sstable should not have been included.",
+                                               PreviewKind.NONE.logPrefix(sessionID), sstable, bounds, normalizedRanges);
+                logger.error(message);
+                throw new IllegalStateException(message);
+            }
+        }
+
     }
 
     @VisibleForTesting
@@ -736,8 +787,6 @@ public class CompactionManager implements CompactionManagerMBean
 
             Bounds<Token> sstableBounds = new Bounds<>(sstable.first.getToken(), sstable.last.getToken());
 
-            boolean shouldAnticompact = false;
-
             for (Range<Token> r : normalizedRanges)
             {
                 // ranges are normalized - no wrap around - if first and last are contained we know that all tokens are contained in the range
@@ -746,23 +795,13 @@ public class CompactionManager implements CompactionManagerMBean
                     logger.info("{} SSTable {} fully contained in range {}, mutating repairedAt instead of anticompacting", PreviewKind.NONE.logPrefix(parentRepairSession), sstable, r);
                     fullyContainedSSTables.add(sstable);
                     sstableIterator.remove();
-                    shouldAnticompact = true;
                     break;
                 }
                 else if (r.intersects(sstableBounds))
                 {
                     logger.info("{} SSTable {} ({}) will be anticompacted on range {}", PreviewKind.NONE.logPrefix(parentRepairSession), sstable, sstableBounds, r);
-                    shouldAnticompact = true;
                 }
             }
-
-            if (!shouldAnticompact)
-            {
-                // this should never happen - in PendingAntiCompaction#getSSTables we select all sstables that intersect the repaired ranges, that can't have changed here
-                String message = String.format("%s SSTable %s (%s) does not intersect repaired ranges %s, this sstable should not have been included.", PreviewKind.NONE.logPrefix(parentRepairSession), sstable, sstableBounds, normalizedRanges);
-                logger.error(message);
-                throw new IllegalStateException(message);
-            }
         }
         return fullyContainedSSTables;
     }
@@ -914,7 +953,10 @@ public class CompactionManager implements CompactionManagerMBean
         {
             ColumnFamilyStore cfs = entry.getKey();
             Keyspace keyspace = cfs.keyspace;
-            Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
+            final RangesAtEndpoint replicas = StorageService.instance.getLocalReplicas(keyspace.getName());
+            final Set<Range<Token>> allRanges = replicas.ranges();
+            final Set<Range<Token>> transientRanges = replicas.filter(Replica::isTransient).ranges();
+            final Set<Range<Token>> fullRanges = replicas.filter(Replica::isFull).ranges();
             boolean hasIndexes = cfs.indexManager.hasIndexes();
             SSTableReader sstable = lookupSSTable(cfs, entry.getValue());
 
@@ -924,10 +966,10 @@ public class CompactionManager implements CompactionManagerMBean
             }
             else
             {
-                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfs, ranges, FBUtilities.nowInSeconds());
+                CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfs, allRanges, transientRanges, sstable.isRepaired(), FBUtilities.nowInSeconds());
                 try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstable, OperationType.CLEANUP))
                 {
-                    doCleanupOne(cfs, txn, cleanupStrategy, ranges, hasIndexes);
+                    doCleanupOne(cfs, txn, cleanupStrategy, allRanges, fullRanges, transientRanges, hasIndexes);
                 }
                 catch (IOException e)
                 {
@@ -1104,22 +1146,33 @@ public class CompactionManager implements CompactionManagerMBean
      *
      * @throws IOException
      */
-    private void doCleanupOne(final ColumnFamilyStore cfs, LifecycleTransaction txn, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
+    private void doCleanupOne(final ColumnFamilyStore cfs,
+                              LifecycleTransaction txn,
+                              CleanupStrategy cleanupStrategy,
+                              Collection<Range<Token>> allRanges,
+                              Collection<Range<Token>> fullRanges,
+                              Collection<Range<Token>> transientRanges,
+                              boolean hasIndexes) throws IOException
     {
         assert !cfs.isIndex();
 
         SSTableReader sstable = txn.onlyOne();
 
         // if ranges is empty and no index, entire sstable is discarded
-        if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
+        if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(allRanges))
         {
             txn.obsoleteOriginals();
             txn.finish();
             return;
         }
-        if (!needsCleanup(sstable, ranges))
+
+        boolean needsCleanupFull = needsCleanup(sstable, fullRanges);
+        boolean needsCleanupTransient = needsCleanup(sstable, transientRanges);
+        //If there are no ranges for which the table needs cleanup either due to lack of intersection or lack
+        //of the table being repaired.
+        if (!needsCleanupFull && (!needsCleanupTransient || !sstable.isRepaired()))
         {
-            logger.trace("Skipping {} for cleanup; all rows should be kept", sstable);
+            logger.trace("Skipping {} for cleanup; all rows should be kept. Needs cleanup full ranges: {} Needs cleanup transient ranges: {} Repaired: {}", sstable, needsCleanupFull, needsCleanupTransient, sstable.isRepaired());
             return;
         }
 
@@ -1150,7 +1203,7 @@ public class CompactionManager implements CompactionManagerMBean
              CompactionIterator ci = new CompactionIterator(OperationType.CLEANUP, Collections.singletonList(scanner), controller, nowInSec, UUIDGen.getTimeUUID(), metrics))
         {
             StatsMetadata metadata = sstable.getSSTableMetadata();
-            writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, sstable, txn));
+            writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, metadata.isTransient, sstable, txn));
             long lastBytesScanned = 0;
 
             while (ci.hasNext())
@@ -1218,11 +1271,18 @@ public class CompactionManager implements CompactionManagerMBean
             this.nowInSec = nowInSec;
         }
 
-        public static CleanupStrategy get(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, int nowInSec)
+        public static CleanupStrategy get(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, Collection<Range<Token>> transientRanges, boolean isRepaired, int nowInSec)
         {
-            return cfs.indexManager.hasIndexes()
-                 ? new Full(cfs, ranges, nowInSec)
-                 : new Bounded(cfs, ranges, nowInSec);
+            if (cfs.indexManager.hasIndexes())
+            {
+                if (!transientRanges.isEmpty())
+                {
+                    //Shouldn't have been possible to create this situation
+                    throw new AssertionError("Can't have indexes and transient ranges");
+                }
+                return new Full(cfs, ranges, nowInSec);
+            }
+            return new Bounded(cfs, ranges, transientRanges, isRepaired, nowInSec);
         }
 
         public abstract ISSTableScanner getScanner(SSTableReader sstable);
@@ -1230,7 +1290,10 @@ public class CompactionManager implements CompactionManagerMBean
 
         private static final class Bounded extends CleanupStrategy
         {
-            public Bounded(final ColumnFamilyStore cfs, Collection<Range<Token>> ranges, int nowInSec)
+            private final Collection<Range<Token>> transientRanges;
+            private final boolean isRepaired;
+
+            public Bounded(final ColumnFamilyStore cfs, Collection<Range<Token>> ranges, Collection<Range<Token>> transientRanges, boolean isRepaired, int nowInSec)
             {
                 super(ranges, nowInSec);
                 cacheCleanupExecutor.submit(new Runnable()
@@ -1241,12 +1304,23 @@ public class CompactionManager implements CompactionManagerMBean
                         cfs.cleanupCache();
                     }
                 });
+                this.transientRanges = transientRanges;
+                this.isRepaired = isRepaired;
             }
 
             @Override
             public ISSTableScanner getScanner(SSTableReader sstable)
             {
-                return sstable.getScanner(ranges);
+                //If transient replication is enabled and there are transient ranges
+                //then cleanup should remove any partitions that are repaired and in the transient range
+                //as they should already be synchronized at other full replicas.
+                //So just don't scan the portion of the table containing the repaired transient ranges
+                Collection<Range<Token>> rangesToScan = ranges;
+                if (isRepaired)
+                {
+                    rangesToScan = Collections2.filter(ranges, range -> !transientRanges.contains(range));
+                }
+                return sstable.getScanner(rangesToScan);
             }
 
             @Override
@@ -1291,6 +1365,7 @@ public class CompactionManager implements CompactionManagerMBean
                                              long expectedBloomFilterSize,
                                              long repairedAt,
                                              UUID pendingRepair,
+                                             boolean isTransient,
                                              SSTableReader sstable,
                                              LifecycleTransaction txn)
     {
@@ -1301,6 +1376,7 @@ public class CompactionManager implements CompactionManagerMBean
                                     expectedBloomFilterSize,
                                     repairedAt,
                                     pendingRepair,
+                                    isTransient,
                                     sstable.getSSTableLevel(),
                                     sstable.header,
                                     cfs.indexManager.listIndexes(),
@@ -1312,6 +1388,7 @@ public class CompactionManager implements CompactionManagerMBean
                                                               int expectedBloomFilterSize,
                                                               long repairedAt,
                                                               UUID pendingRepair,
+                                                              boolean isTransient,
                                                               Collection<SSTableReader> sstables,
                                                               LifecycleTransaction txn)
     {
@@ -1335,6 +1412,7 @@ public class CompactionManager implements CompactionManagerMBean
                                     (long) expectedBloomFilterSize,
                                     repairedAt,
                                     pendingRepair,
+                                    isTransient,
                                     cfs.metadata,
                                     new MetadataCollector(sstables, cfs.metadata().comparator, minLevel),
                                     SerializationHeader.make(cfs.metadata(), sstables),
@@ -1347,16 +1425,19 @@ public class CompactionManager implements CompactionManagerMBean
      * will store the non-repaired ranges. Once anticompation is completed, the original sstable is marked as compacted
      * and subsequently deleted.
      * @param cfs
-     * @param repaired a transaction over the repaired sstables to anticompacy
-     * @param ranges Repaired ranges to be placed into one of the new sstables. The repaired table will be tracked via
-     * the {@link org.apache.cassandra.io.sstable.metadata.StatsMetadata#repairedAt} field.
+     * @param txn a transaction over the repaired sstables to anticompact
+     * @param ranges full and transient ranges to be placed into one of the new sstables. The repaired table will be tracked via
+     *   the {@link org.apache.cassandra.io.sstable.metadata.StatsMetadata#pendingRepair} field.
      */
-    private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, LifecycleTransaction repaired, long repairedAt, UUID pendingRepair)
+    private void doAntiCompaction(ColumnFamilyStore cfs,
+                                  RangesAtEndpoint ranges,
+                                  LifecycleTransaction txn,
+                                  UUID pendingRepair)
     {
-        logger.info("Performing anticompaction on {} sstables", repaired.originals().size());
+        logger.info("Performing anticompaction on {} sstables", txn.originals().size());
 
         //Group SSTables
-        Set<SSTableReader> sstables = repaired.originals();
+        Set<SSTableReader> sstables = txn.originals();
 
         // Repairs can take place on both unrepaired (incremental + full) and repaired (full) data.
         // Although anti-compaction could work on repaired sstables as well and would result in having more accurate
@@ -1366,101 +1447,111 @@ public class CompactionManager implements CompactionManagerMBean
         cfs.metric.bytesAnticompacted.inc(SSTableReader.getTotalBytes(unrepairedSSTables));
         Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategyManager().groupSSTablesForAntiCompaction(unrepairedSSTables);
 
-        // iterate over sstables to check if the repaired / unrepaired ranges intersect them.
+        // iterate over sstables to check if the full / transient / unrepaired ranges intersect them.
         int antiCompactedSSTableCount = 0;
         for (Collection<SSTableReader> sstableGroup : groupedSSTables)
         {
-            try (LifecycleTransaction txn = repaired.split(sstableGroup))
+            try (LifecycleTransaction groupTxn = txn.split(sstableGroup))
             {
-                int antiCompacted = antiCompactGroup(cfs, ranges, txn, repairedAt, pendingRepair);
+                int antiCompacted = antiCompactGroup(cfs, ranges, groupTxn, pendingRepair);
                 antiCompactedSSTableCount += antiCompacted;
             }
         }
 
         String format = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";
-        logger.info(format, repaired.originals().size(), antiCompactedSSTableCount);
+        logger.info(format, txn.originals().size(), antiCompactedSSTableCount);
     }
 
-    private int antiCompactGroup(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
-                             LifecycleTransaction anticompactionGroup, long repairedAt, UUID pendingRepair)
+    private int antiCompactGroup(ColumnFamilyStore cfs,
+                                 RangesAtEndpoint ranges,
+                                 LifecycleTransaction txn,
+                                 UUID pendingRepair)
     {
+        Preconditions.checkArgument(!ranges.isEmpty(), "need at least one full or transient range");
         long groupMaxDataAge = -1;
 
-        for (Iterator<SSTableReader> i = anticompactionGroup.originals().iterator(); i.hasNext();)
+        for (Iterator<SSTableReader> i = txn.originals().iterator(); i.hasNext();)
         {
             SSTableReader sstable = i.next();
             if (groupMaxDataAge < sstable.maxDataAge)
                 groupMaxDataAge = sstable.maxDataAge;
         }
 
-        if (anticompactionGroup.originals().size() == 0)
+        if (txn.originals().size() == 0)
         {
             logger.info("No valid anticompactions for this group, All sstables were compacted and are no longer available");
             return 0;
         }
 
-        logger.info("Anticompacting {}", anticompactionGroup);
-        Set<SSTableReader> sstableAsSet = anticompactionGroup.originals();
+        logger.info("Anticompacting {}", txn);
+        Set<SSTableReader> sstableAsSet = txn.originals();
 
         File destination = cfs.getDirectories().getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
-        long repairedKeyCount = 0;
-        long unrepairedKeyCount = 0;
         int nowInSec = FBUtilities.nowInSeconds();
 
         CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
-        try (SSTableRewriter repairedSSTableWriter = SSTableRewriter.constructWithoutEarlyOpening(anticompactionGroup, false, groupMaxDataAge);
-             SSTableRewriter unRepairedSSTableWriter = SSTableRewriter.constructWithoutEarlyOpening(anticompactionGroup, false, groupMaxDataAge);
-             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup.originals());
+        try (SSTableRewriter fullWriter = SSTableRewriter.constructWithoutEarlyOpening(txn, false, groupMaxDataAge);
+             SSTableRewriter transWriter = SSTableRewriter.constructWithoutEarlyOpening(txn, false, groupMaxDataAge);
+             SSTableRewriter unrepairedWriter = SSTableRewriter.constructWithoutEarlyOpening(txn, false, groupMaxDataAge);
+
+             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(txn.originals());
              CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs, nowInSec));
              CompactionIterator ci = new CompactionIterator(OperationType.ANTICOMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID(), metrics))
         {
             int expectedBloomFilterSize = Math.max(cfs.metadata().params.minIndexInterval, (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
 
-            repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, pendingRepair, sstableAsSet, anticompactionGroup));
-            unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, null, sstableAsSet, anticompactionGroup));
-            Range.OrderedRangeContainmentChecker containmentChecker = new Range.OrderedRangeContainmentChecker(ranges);
+            fullWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, false, sstableAsSet, txn));
+            transWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, true, sstableAsSet, txn));
+            unrepairedWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, false, sstableAsSet, txn));
+
+            Predicate<Token> fullChecker = !ranges.fullRanges().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.fullRanges()) : t -> false;
+            Predicate<Token> transChecker = !ranges.transientRanges().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.transientRanges()) : t -> false;
             while (ci.hasNext())
             {
                 try (UnfilteredRowIterator partition = ci.next())
                 {
-                    // if current range from sstable is repaired, save it into the new repaired sstable
-                    if (containmentChecker.contains(partition.partitionKey().getToken()))
+                    Token token = partition.partitionKey().getToken();
+                    // if this row is contained in the full or transient ranges, append it to the appropriate sstable
+                    if (fullChecker.test(token))
                     {
-                        repairedSSTableWriter.append(partition);
-                        repairedKeyCount++;
+                        fullWriter.append(partition);
+                    }
+                    else if (transChecker.test(token))
+                    {
+                        transWriter.append(partition);
                     }
-                    // otherwise save into the new 'non-repaired' table
                     else
                     {
-                        unRepairedSSTableWriter.append(partition);
-                        unrepairedKeyCount++;
+                        // otherwise, append it to the unrepaired sstable
+                        unrepairedWriter.append(partition);
                     }
                 }
             }
 
             List<SSTableReader> anticompactedSSTables = new ArrayList<>();
-            // since both writers are operating over the same Transaction, we cannot use the convenience Transactional.finish() method,
+            // since all writers are operating over the same Transaction, we cannot use the convenience Transactional.finish() method,
             // as on the second finish() we would prepareToCommit() on a Transaction that has already been committed, which is forbidden by the API
             // (since it indicates misuse). We call permitRedundantTransitions so that calls that transition to a state already occupied are permitted.
-            anticompactionGroup.permitRedundantTransitions();
-            repairedSSTableWriter.setRepairedAt(repairedAt).prepareToCommit();
-            unRepairedSSTableWriter.prepareToCommit();
-            anticompactedSSTables.addAll(repairedSSTableWriter.finished());
-            anticompactedSSTables.addAll(unRepairedSSTableWriter.finished());
-            repairedSSTableWriter.commit();
-            unRepairedSSTableWriter.commit();
-
-            logger.trace("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
-                                                                       repairedKeyCount + unrepairedKeyCount,
-                                                                       cfs.keyspace.getName(),
-                                                                       cfs.getTableName(),
-                                                                       anticompactionGroup);
+            txn.permitRedundantTransitions();
+
+            fullWriter.prepareToCommit();
+            transWriter.prepareToCommit();
+            unrepairedWriter.prepareToCommit();
+
+            anticompactedSSTables.addAll(fullWriter.finished());
+            anticompactedSSTables.addAll(transWriter.finished());
+            anticompactedSSTables.addAll(unrepairedWriter.finished());
+
+            fullWriter.commit();
+            transWriter.commit();
+            unrepairedWriter.commit();
+
             return anticompactedSSTables.size();
         }
         catch (Throwable e)
         {
             JVMStabilityInspector.inspectThrowable(e);
-            logger.error("Error anticompacting " + anticompactionGroup, e);
+            logger.error("Error anticompacting " + txn, e);
         }
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
index 8fba121..8ce93fa 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SerializationHeader;
@@ -71,11 +72,19 @@ public class CompactionStrategyHolder extends AbstractStrategyHolder
     }
 
     @Override
-    public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair)
+    public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair, boolean isTransient)
     {
-        Preconditions.checkArgument(!isPendingRepair || !isRepaired,
-                                    "SSTables cannot be both repaired and pending repair");
-        return !isPendingRepair && (isRepaired == this.isRepaired);
+        if (!isPendingRepair)
+        {
+            Preconditions.checkArgument(!isTransient, "isTransient can only be true for sstables pending repairs");
+            return this.isRepaired == isRepaired;
+        }
+        else
+        {
+            Preconditions.checkArgument(!isRepaired, "SSTables cannot be both repaired and pending repair");
+            return false;
+
+        }
     }
 
     @Override
@@ -206,7 +215,15 @@ public class CompactionStrategyHolder extends AbstractStrategyHolder
     }
 
     @Override
-    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, MetadataCollector collector, SerializationHeader header, Collection<Index> indexes, LifecycleTransaction txn)
+    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor,
+                                                       long keyCount,
+                                                       long repairedAt,
+                                                       UUID pendingRepair,
+                                                       boolean isTransient,
+                                                       MetadataCollector collector,
+                                                       SerializationHeader header,
+                                                       Collection<Index> indexes,
+                                                       LifecycleTransaction txn)
     {
         if (isRepaired)
         {
@@ -226,6 +243,7 @@ public class CompactionStrategyHolder extends AbstractStrategyHolder
                                                  keyCount,
                                                  repairedAt,
                                                  pendingRepair,
+                                                 isTransient,
                                                  collector,
                                                  header,
                                                  indexes,
@@ -237,4 +255,10 @@ public class CompactionStrategyHolder extends AbstractStrategyHolder
     {
         return strategies.indexOf(strategy);
     }
+
+    @Override
+    public boolean containsSSTable(SSTableReader sstable)
+    {
+        return Iterables.any(strategies, acs -> acs.getSSTables().contains(sstable));
+    }
 }


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


[03/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
index 294731a..4f7cde0 100644
--- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
@@ -20,8 +20,6 @@ package org.apache.cassandra.service;
 
 import java.util.*;
 
-import javax.xml.crypto.Data;
-
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 import org.junit.Assert;
@@ -36,13 +34,13 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
-import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -107,13 +105,13 @@ public class ActiveRepairServiceTest
     public void testGetNeighborsPlusOne() throws Throwable
     {
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         expected.remove(FBUtilities.getBroadcastAddressAndPort());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
         Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null).endpoints());
         }
         assertEquals(expected, neighbors);
     }
@@ -124,19 +122,19 @@ public class ActiveRepairServiceTest
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
         Set<InetAddressAndPort> expected = new HashSet<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
+        for (Replica replica : ars.getAddressReplicas().get(FBUtilities.getBroadcastAddressAndPort()))
         {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replica.range()).endpoints());
         }
         expected.remove(FBUtilities.getBroadcastAddressAndPort());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
         Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null).endpoints());
         }
         assertEquals(expected, neighbors);
     }
@@ -147,18 +145,18 @@ public class ActiveRepairServiceTest
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        Set<InetAddressAndPort> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         expected.remove(FBUtilities.getBroadcastAddressAndPort());
         // remove remote endpoints
         TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
         HashSet<InetAddressAndPort> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
 
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
         Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null).endpoints());
         }
         assertEquals(expected, neighbors);
     }
@@ -169,12 +167,12 @@ public class ActiveRepairServiceTest
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
         Set<InetAddressAndPort> expected = new HashSet<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
+        for (Replica replica : ars.getAddressReplicas().get(FBUtilities.getBroadcastAddressAndPort()))
         {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replica.range()).endpoints());
         }
         expected.remove(FBUtilities.getBroadcastAddressAndPort());
         // remove remote endpoints
@@ -182,11 +180,11 @@ public class ActiveRepairServiceTest
         HashSet<InetAddressAndPort> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
 
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
         Set<InetAddressAndPort> neighbors = new HashSet<>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null).endpoints());
         }
         assertEquals(expected, neighbors);
     }
@@ -197,30 +195,30 @@ public class ActiveRepairServiceTest
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf*2 nodes, and ensure that only neighbors specified by the hosts are returned
-        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
         List<InetAddressAndPort> expected = new ArrayList<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddressAndPort()))
+        for (Replica replicas : ars.getAddressReplicas().get(FBUtilities.getBroadcastAddressAndPort()))
         {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicas.range()).endpoints());
         }
 
         expected.remove(FBUtilities.getBroadcastAddressAndPort());
         Collection<String> hosts = Arrays.asList(FBUtilities.getBroadcastAddressAndPort().toString(),expected.get(0).toString());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
 
         assertEquals(expected.get(0), ActiveRepairService.getNeighbors(KEYSPACE5, ranges,
                                                                        ranges.iterator().next(),
-                                                                       null, hosts).iterator().next());
+                                                                       null, hosts).endpoints().iterator().next());
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void testGetNeighborsSpecifiedHostsWithNoLocalHost() throws Throwable
     {
-        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor().allReplicas);
         //Dont give local endpoint
         Collection<String> hosts = Arrays.asList("127.0.0.3");
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE5).ranges();
         ActiveRepairService.getNeighbors(KEYSPACE5, ranges, ranges.iterator().next(), null, hosts);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/BootstrapTransientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/BootstrapTransientTest.java b/test/unit/org/apache/cassandra/service/BootstrapTransientTest.java
new file mode 100644
index 0000000..63973ea
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/BootstrapTransientTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.service;
+
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.locator.EndpointsByReplica;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.OrderPreservingPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.RangeStreamer;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+import static org.apache.cassandra.service.StorageServiceTest.assertMultimapEqualsIgnoreOrder;
+
+/**
+ * This is also fairly effectively testing source retrieval for bootstrap as well since RangeStreamer
+ * is used to calculate the endpoints to fetch from and check they are alive for both RangeRelocator (move) and
+ * bootstrap (RangeRelocator).
+ */
+public class BootstrapTransientTest
+{
+    static InetAddressAndPort aAddress;
+    static InetAddressAndPort bAddress;
+    static InetAddressAndPort cAddress;
+    static InetAddressAndPort dAddress;
+
+    @BeforeClass
+    public static void setUpClass() throws Exception
+    {
+        aAddress = InetAddressAndPort.getByName("127.0.0.1");
+        bAddress = InetAddressAndPort.getByName("127.0.0.2");
+        cAddress = InetAddressAndPort.getByName("127.0.0.3");
+        dAddress = InetAddressAndPort.getByName("127.0.0.4");
+    }
+
+    private final List<InetAddressAndPort> downNodes = new ArrayList<>();
+    Predicate<Replica> alivePredicate = replica -> !downNodes.contains(replica.endpoint());
+
+    private final List<InetAddressAndPort> sourceFilterDownNodes = new ArrayList<>();
+    private final Collection<Predicate<Replica>> sourceFilters = Collections.singleton(replica -> !sourceFilterDownNodes.contains(replica.endpoint()));
+
+    @After
+    public void clearDownNode()
+    {
+        downNodes.clear();
+        sourceFilterDownNodes.clear();
+    }
+
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    Token tenToken = new OrderPreservingPartitioner.StringToken("00010");
+    Token twentyToken = new OrderPreservingPartitioner.StringToken("00020");
+    Token thirtyToken = new OrderPreservingPartitioner.StringToken("00030");
+    Token fourtyToken = new OrderPreservingPartitioner.StringToken("00040");
+
+    Range<Token> aRange = new Range<>(thirtyToken, tenToken);
+    Range<Token> bRange = new Range<>(tenToken, twentyToken);
+    Range<Token> cRange = new Range<>(twentyToken, thirtyToken);
+    Range<Token> dRange = new Range<>(thirtyToken, fourtyToken);
+
+    RangesAtEndpoint toFetch = RangesAtEndpoint.of(new Replica(dAddress, dRange, true),
+                                                   new Replica(dAddress, cRange, true),
+                                                   new Replica(dAddress, bRange, false));
+
+    @Test
+    public void testRangeStreamerRangesToFetch() throws Exception
+    {
+        EndpointsByReplica expectedResult = new EndpointsByReplica(ImmutableMap.of(
+        fullReplica(dAddress, dRange), EndpointsForRange.builder(aRange).add(fullReplica(bAddress, aRange)).add(transientReplica(cAddress, aRange)).build(),
+        fullReplica(dAddress, cRange), EndpointsForRange.builder(cRange).add(fullReplica(cAddress, cRange)).add(transientReplica(bAddress, cRange)).build(),
+        transientReplica(dAddress, bRange), EndpointsForRange.builder(bRange).add(transientReplica(aAddress, bRange)).build()));
+
+        invokeCalculateRangesToFetchWithPreferredEndpoints(toFetch, constructTMDs(), expectedResult);
+    }
+
+    private Pair<TokenMetadata, TokenMetadata> constructTMDs()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(aRange.right, aAddress);
+        tmd.updateNormalToken(bRange.right, bAddress);
+        tmd.updateNormalToken(cRange.right, cAddress);
+        TokenMetadata updated = tmd.cloneOnlyTokenMap();
+        updated.updateNormalToken(dRange.right, dAddress);
+
+        return Pair.create(tmd, updated);
+    }
+
+    private void invokeCalculateRangesToFetchWithPreferredEndpoints(ReplicaCollection<?> toFetch,
+                                                                    Pair<TokenMetadata, TokenMetadata> tmds,
+                                                                    EndpointsByReplica expectedResult)
+    {
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+
+        EndpointsByReplica result = RangeStreamer.calculateRangesToFetchWithPreferredEndpoints((address, replicas) -> replicas,
+                                                                                                                   simpleStrategy(tmds.left),
+                                                                                                                   toFetch,
+                                                                                                                   true,
+                                                                                                                   tmds.left,
+                                                                                                                   tmds.right,
+                                                                                                                   alivePredicate,
+                                                                                                                   "OldNetworkTopologyStrategyTest",
+                                                                                                                   sourceFilters);
+        result.asMap().forEach((replica, list) -> System.out.printf("Replica %s, sources %s%n", replica, list));
+        assertMultimapEqualsIgnoreOrder(expectedResult, result);
+
+    }
+
+    private AbstractReplicationStrategy simpleStrategy(TokenMetadata tmd)
+    {
+        IEndpointSnitch snitch = new AbstractEndpointSnitch()
+        {
+            public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2)
+            {
+                return 0;
+            }
+
+            public String getRack(InetAddressAndPort endpoint)
+            {
+                return "R1";
+            }
+
+            public String getDatacenter(InetAddressAndPort endpoint)
+            {
+                return "DC1";
+            }
+        };
+
+        return new SimpleStrategy("MoveTransientTest",
+                                  tmd,
+                                  snitch,
+                                  com.google.common.collect.ImmutableMap.of("replication_factor", "3/1"));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index 8ddc4f0..3c4748e 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -130,10 +130,10 @@ public class LeaveAndBootstrapTest
             strategy = getStrategy(keyspaceName, tmd);
             for (Token token : keyTokens)
             {
-                int replicationFactor = strategy.getReplicationFactor();
+                int replicationFactor = strategy.getReplicationFactor().allReplicas;
 
-                HashSet<InetAddressAndPort> actual = new HashSet<>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
-                HashSet<InetAddressAndPort> expected = new HashSet<>();
+                Set<InetAddressAndPort> actual = tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalReplicas(token, tmd.cloneOnlyTokenMap()).forToken(token)).endpoints();
+                Set<InetAddressAndPort> expected = new HashSet<>();
 
                 for (int i = 0; i < replicationFactor; i++)
                 {
@@ -198,8 +198,6 @@ public class LeaveAndBootstrapTest
                     ApplicationState.STATUS,
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(7))));
 
-        Collection<InetAddressAndPort> endpoints = null;
-
         /* don't require test update every time a new keyspace is added to test/conf/cassandra.yaml */
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i=1; i<=4; i++)
@@ -263,18 +261,18 @@ public class LeaveAndBootstrapTest
 
             for (int i = 0; i < keyTokens.size(); i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(i))).endpoints();
                 assertEquals(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).size(), endpoints.size());
                 assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).containsAll(endpoints));
             }
 
             // just to be sure that things still work according to the old tests, run them:
-            if (strategy.getReplicationFactor() != 3)
+            if (strategy.getReplicationFactor().allReplicas != 3)
                 continue;
             // tokens 5, 15 and 25 should go three nodes
             for (int i=0; i<3; ++i)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(i))).endpoints();
                 assertEquals(3, endpoints.size());
                 assertTrue(endpoints.contains(hosts.get(i+1)));
                 assertTrue(endpoints.contains(hosts.get(i+2)));
@@ -282,7 +280,7 @@ public class LeaveAndBootstrapTest
             }
 
             // token 35 should go to nodes 4, 5, 6, 7 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
+            Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(3))).endpoints();
             assertEquals(5, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(4)));
             assertTrue(endpoints.contains(hosts.get(5)));
@@ -291,7 +289,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot1));
 
             // token 45 should go to nodes 5, 6, 7, 0, boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(4))).endpoints();
             assertEquals(6, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(hosts.get(6)));
@@ -301,7 +299,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 55 should go to nodes 6, 7, 8, 0, 1, boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(5))).endpoints();
             assertEquals(7, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(6)));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -312,7 +310,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 65 should go to nodes 7, 8, 9, 0, 1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(6))).endpoints();
             assertEquals(6, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(7)));
             assertTrue(endpoints.contains(hosts.get(8)));
@@ -322,7 +320,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 75 should to go nodes 8, 9, 0, 1, 2 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(7))).endpoints();
             assertEquals(6, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(8)));
             assertTrue(endpoints.contains(hosts.get(9)));
@@ -332,7 +330,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 85 should go to nodes 9, 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(8))).endpoints();
             assertEquals(4, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(9)));
             assertTrue(endpoints.contains(hosts.get(0)));
@@ -340,7 +338,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 95 should go to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(9))).endpoints();
             assertEquals(3, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));
@@ -385,18 +383,18 @@ public class LeaveAndBootstrapTest
 
             for (int i = 0; i < keyTokens.size(); i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(i))).endpoints();
                 assertEquals(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).size(), endpoints.size());
                 assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).containsAll(endpoints));
             }
 
-            if (strategy.getReplicationFactor() != 3)
+            if (strategy.getReplicationFactor().allReplicas != 3)
                 continue;
             // leave this stuff in to guarantee the old tests work the way they were supposed to.
             // tokens 5, 15 and 25 should go three nodes
             for (int i=0; i<3; ++i)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(i))).endpoints();
                 assertEquals(3, endpoints.size());
                 assertTrue(endpoints.contains(hosts.get(i+1)));
                 assertTrue(endpoints.contains(hosts.get(i+2)));
@@ -404,21 +402,21 @@ public class LeaveAndBootstrapTest
             }
 
             // token 35 goes to nodes 4, 5 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
+            Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(3))).endpoints();
             assertEquals(3, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(4)));
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(boot1));
 
             // token 45 goes to nodes 5, boot1 and node7
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(4))).endpoints();
             assertEquals(3, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(boot1));
             assertTrue(endpoints.contains(hosts.get(7)));
 
             // token 55 goes to boot1, 7, boot2, 8 and 0
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(5))).endpoints();
             assertEquals(5, endpoints.size());
             assertTrue(endpoints.contains(boot1));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -427,7 +425,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(0)));
 
             // token 65 goes to nodes 7, boot2, 8, 0 and 1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(6))).endpoints();
             assertEquals(5, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(7)));
             assertTrue(endpoints.contains(boot2));
@@ -436,7 +434,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(1)));
 
             // token 75 goes to nodes boot2, 8, 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(7))).endpoints();
             assertEquals(5, endpoints.size());
             assertTrue(endpoints.contains(boot2));
             assertTrue(endpoints.contains(hosts.get(8)));
@@ -445,14 +443,14 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 85 goes to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(8))).endpoints();
             assertEquals(3, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 95 goes to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(9))).endpoints();
             assertEquals(3, endpoints.size());
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/MoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index 7321fba..731a25d 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -23,13 +23,21 @@ import java.net.UnknownHostException;
 import java.util.*;
 
 import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Multimap;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.RangesByEndpoint;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -492,24 +500,25 @@ public class MoveTest
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(token)), host);
     }
 
-    private Map.Entry<Range<Token>, Collection<InetAddressAndPort>> generatePendingMapEntry(int start, int end, String... endpoints) throws UnknownHostException
+    private Map.Entry<Range<Token>, EndpointsForRange> generatePendingMapEntry(int start, int end, String... endpoints) throws UnknownHostException
     {
-        Map<Range<Token>, Collection<InetAddressAndPort>> pendingRanges = new HashMap<>();
-        pendingRanges.put(generateRange(start, end), makeAddrs(endpoints));
+        Map<Range<Token>, EndpointsForRange> pendingRanges = new HashMap<>();
+        Range<Token> range = generateRange(start, end);
+        pendingRanges.put(range, makeReplicas(range, endpoints));
         return pendingRanges.entrySet().iterator().next();
     }
 
-    private Map<Range<Token>, Collection<InetAddressAndPort>> generatePendingRanges(Map.Entry<Range<Token>, Collection<InetAddressAndPort>>... entries)
+    private Map<Range<Token>, EndpointsForRange> generatePendingRanges(Map.Entry<Range<Token>, EndpointsForRange>... entries)
     {
-        Map<Range<Token>, Collection<InetAddressAndPort>> pendingRanges = new HashMap<>();
-        for(Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : entries)
+        Map<Range<Token>, EndpointsForRange> pendingRanges = new HashMap<>();
+        for(Map.Entry<Range<Token>, EndpointsForRange> entry : entries)
         {
             pendingRanges.put(entry.getKey(), entry.getValue());
         }
         return pendingRanges;
     }
 
-    private void assertPendingRanges(TokenMetadata tmd, Map<Range<Token>,  Collection<InetAddressAndPort>> pendingRanges, String keyspaceName) throws ConfigurationException
+    private void assertPendingRanges(TokenMetadata tmd, Map<Range<Token>, EndpointsForRange> pendingRanges, String keyspaceName) throws ConfigurationException
     {
         boolean keyspaceFound = false;
         for (String nonSystemKeyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
@@ -523,15 +532,15 @@ public class MoveTest
         assert keyspaceFound;
     }
 
-    private void assertMaps(Map<Range<Token>, Collection<InetAddressAndPort>> expected, PendingRangeMaps actual)
+    private void assertMaps(Map<Range<Token>, EndpointsForRange> expected, PendingRangeMaps actual)
     {
         int sizeOfActual = 0;
-        Iterator<Map.Entry<Range<Token>, List<InetAddressAndPort>>> iterator = actual.iterator();
+        Iterator<Map.Entry<Range<Token>, EndpointsForRange.Mutable>> iterator = actual.iterator();
         while(iterator.hasNext())
         {
-            Map.Entry<Range<Token>, List<InetAddressAndPort>> actualEntry = iterator.next();
+            Map.Entry<Range<Token>, EndpointsForRange.Mutable> actualEntry = iterator.next();
             assertNotNull(expected.get(actualEntry.getKey()));
-            assertEquals(new HashSet<>(expected.get(actualEntry.getKey())), new HashSet<>(actualEntry.getValue()));
+            assertEquals(ImmutableSet.copyOf(expected.get(actualEntry.getKey())), ImmutableSet.copyOf(actualEntry.getValue()));
             sizeOfActual++;
         }
 
@@ -589,9 +598,9 @@ public class MoveTest
             int numMoved = 0;
             for (Token token : keyTokens)
             {
-                int replicationFactor = strategy.getReplicationFactor();
+                int replicationFactor = strategy.getReplicationFactor().allReplicas;
 
-                HashSet<InetAddressAndPort> actual = new HashSet<>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
+                EndpointsForToken actual = tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalReplicas(token, tmd.cloneOnlyTokenMap()).forToken(token));
                 HashSet<InetAddressAndPort> expected = new HashSet<>();
 
                 for (int i = 0; i < replicationFactor; i++)
@@ -600,10 +609,10 @@ public class MoveTest
                 }
 
                 if (expected.size() == actual.size()) {
-                	assertEquals("mismatched endpoint sets", expected, actual);
+                	assertEquals("mismatched endpoint sets", expected, actual.endpoints());
                 } else {
                 	expected.add(hosts.get(MOVING_NODE));
-                	assertEquals("mismatched endpoint sets", expected, actual);
+                	assertEquals("mismatched endpoint sets", expected, actual.endpoints());
                 	numMoved++;
                 }
             }
@@ -648,8 +657,6 @@ public class MoveTest
             newTokens.put(movingIndex, newToken);
         }
 
-        Collection<InetAddressAndPort> endpoints;
-
         tmd = tmd.cloneAfterAllSettled();
         ss.setTokenMetadataUnsafe(tmd);
 
@@ -693,37 +700,18 @@ public class MoveTest
         *  }
         */
 
-        Multimap<InetAddressAndPort, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get(Simple_RF1_KeyspaceName).getAddressRanges();
-        Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
-        assertEquals(1, collectionSize(ranges1));
-        assertEquals(generateRange(97, 0), ranges1.iterator().next());
-        Collection<Range<Token>> ranges2 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
-        assertEquals(1, collectionSize(ranges2));
-        assertEquals(generateRange(0, 10), ranges2.iterator().next());
-        Collection<Range<Token>> ranges3 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
-        assertEquals(1, collectionSize(ranges3));
-        assertEquals(generateRange(10, 20), ranges3.iterator().next());
-        Collection<Range<Token>> ranges4 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
-        assertEquals(1, collectionSize(ranges4));
-        assertEquals(generateRange(20, 30), ranges4.iterator().next());
-        Collection<Range<Token>> ranges5 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
-        assertEquals(1, collectionSize(ranges5));
-        assertEquals(generateRange(30, 40), ranges5.iterator().next());
-        Collection<Range<Token>> ranges6 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
-        assertEquals(1, collectionSize(ranges6));
-        assertEquals(generateRange(40, 50), ranges6.iterator().next());
-        Collection<Range<Token>> ranges7 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
-        assertEquals(1, collectionSize(ranges7));
-        assertEquals(generateRange(50, 67), ranges7.iterator().next());
-        Collection<Range<Token>> ranges8 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
-        assertEquals(1, collectionSize(ranges8));
-        assertEquals(generateRange(67, 70), ranges8.iterator().next());
-        Collection<Range<Token>> ranges9 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
-        assertEquals(1, collectionSize(ranges9));
-        assertEquals(generateRange(70, 87), ranges9.iterator().next());
-        Collection<Range<Token>> ranges10 = keyspace1ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
-        assertEquals(1, collectionSize(ranges10));
-        assertEquals(generateRange(87, 97), ranges10.iterator().next());
+        RangesByEndpoint keyspace1ranges = keyspaceStrategyMap.get(Simple_RF1_KeyspaceName).getAddressReplicas();
+
+        assertRanges(keyspace1ranges, "127.0.0.1", 97, 0);
+        assertRanges(keyspace1ranges, "127.0.0.2", 0, 10);
+        assertRanges(keyspace1ranges, "127.0.0.3", 10, 20);
+        assertRanges(keyspace1ranges, "127.0.0.4", 20, 30);
+        assertRanges(keyspace1ranges, "127.0.0.5", 30, 40);
+        assertRanges(keyspace1ranges, "127.0.0.6", 40, 50);
+        assertRanges(keyspace1ranges, "127.0.0.7", 50, 67);
+        assertRanges(keyspace1ranges, "127.0.0.8", 67, 70);
+        assertRanges(keyspace1ranges, "127.0.0.9", 70, 87);
+        assertRanges(keyspace1ranges, "127.0.0.10", 87, 97);
 
 
         /**
@@ -742,37 +730,17 @@ public class MoveTest
         * }
         */
 
-        Multimap<InetAddressAndPort, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressRanges();
-        ranges1 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
-        assertEquals(collectionSize(ranges1), 5);
-        assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 50, 67, 87, 97, 67, 70)));
-        ranges2 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
-        assertEquals(collectionSize(ranges2), 5);
-        assertTrue(ranges2.equals(generateRanges(97, 0, 70, 87, 87, 97, 0, 10, 67, 70)));
-        ranges3 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
-        assertEquals(collectionSize(ranges3), 5);
-        assertTrue(ranges3.equals(generateRanges(97, 0, 70, 87, 87, 97, 0, 10, 10, 20)));
-        ranges4 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
-        assertEquals(collectionSize(ranges4), 5);
-        assertTrue(ranges4.equals(generateRanges(97, 0, 20, 30, 87, 97, 0, 10, 10, 20)));
-        ranges5 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
-        assertEquals(collectionSize(ranges5), 5);
-        assertTrue(ranges5.equals(generateRanges(97, 0, 30, 40, 20, 30, 0, 10, 10, 20)));
-        ranges6 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
-        assertEquals(collectionSize(ranges6), 5);
-        assertTrue(ranges6.equals(generateRanges(40, 50, 30, 40, 20, 30, 0, 10, 10, 20)));
-        ranges7 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
-        assertEquals(collectionSize(ranges7), 5);
-        assertTrue(ranges7.equals(generateRanges(40, 50, 30, 40, 50, 67, 20, 30, 10, 20)));
-        ranges8 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
-        assertEquals(collectionSize(ranges8), 5);
-        assertTrue(ranges8.equals(generateRanges(40, 50, 30, 40, 50, 67, 20, 30, 67, 70)));
-        ranges9 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
-        assertEquals(collectionSize(ranges9), 5);
-        assertTrue(ranges9.equals(generateRanges(40, 50, 70, 87, 30, 40, 50, 67, 67, 70)));
-        ranges10 = keyspace3ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
-        assertEquals(collectionSize(ranges10), 5);
-        assertTrue(ranges10.equals(generateRanges(40, 50, 70, 87, 50, 67, 87, 97, 67, 70)));
+        RangesByEndpoint keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressReplicas();
+        assertRanges(keyspace3ranges, "127.0.0.1", 97, 0, 70, 87, 50, 67, 87, 97, 67, 70);
+        assertRanges(keyspace3ranges, "127.0.0.2", 97, 0, 70, 87, 87, 97, 0, 10, 67, 70);
+        assertRanges(keyspace3ranges, "127.0.0.3", 97, 0, 70, 87, 87, 97, 0, 10, 10, 20);
+        assertRanges(keyspace3ranges, "127.0.0.4", 97, 0, 20, 30, 87, 97, 0, 10, 10, 20);
+        assertRanges(keyspace3ranges, "127.0.0.5", 97, 0, 30, 40, 20, 30, 0, 10, 10, 20);
+        assertRanges(keyspace3ranges, "127.0.0.6", 40, 50, 30, 40, 20, 30, 0, 10, 10, 20);
+        assertRanges(keyspace3ranges, "127.0.0.7", 40, 50, 30, 40, 50, 67, 20, 30, 10, 20);
+        assertRanges(keyspace3ranges, "127.0.0.8", 40, 50, 30, 40, 50, 67, 20, 30, 67, 70);
+        assertRanges(keyspace3ranges, "127.0.0.9", 40, 50, 70, 87, 30, 40, 50, 67, 67, 70);
+        assertRanges(keyspace3ranges, "127.0.0.10", 40, 50, 70, 87, 50, 67, 87, 97, 67, 70);
 
 
         /**
@@ -790,37 +758,18 @@ public class MoveTest
          *      /127.0.0.10=[(70,87], (87,97], (67,70]]
          *  }
          */
-        Multimap<InetAddressAndPort, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get(Simple_RF3_KeyspaceName).getAddressRanges();
-        ranges1 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.1"));
-        assertEquals(collectionSize(ranges1), 3);
-        assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 87, 97)));
-        ranges2 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.2"));
-        assertEquals(collectionSize(ranges2), 3);
-        assertTrue(ranges2.equals(generateRanges(97, 0, 87, 97, 0, 10)));
-        ranges3 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.3"));
-        assertEquals(collectionSize(ranges3), 3);
-        assertTrue(ranges3.equals(generateRanges(97, 0, 0, 10, 10, 20)));
-        ranges4 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.4"));
-        assertEquals(collectionSize(ranges4), 3);
-        assertTrue(ranges4.equals(generateRanges(20, 30, 0, 10, 10, 20)));
-        ranges5 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.5"));
-        assertEquals(collectionSize(ranges5), 3);
-        assertTrue(ranges5.equals(generateRanges(30, 40, 20, 30, 10, 20)));
-        ranges6 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.6"));
-        assertEquals(collectionSize(ranges6), 3);
-        assertTrue(ranges6.equals(generateRanges(40, 50, 30, 40, 20, 30)));
-        ranges7 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.7"));
-        assertEquals(collectionSize(ranges7), 3);
-        assertTrue(ranges7.equals(generateRanges(40, 50, 30, 40, 50, 67)));
-        ranges8 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.8"));
-        assertEquals(collectionSize(ranges8), 3);
-        assertTrue(ranges8.equals(generateRanges(40, 50, 50, 67, 67, 70)));
-        ranges9 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.9"));
-        assertEquals(collectionSize(ranges9), 3);
-        assertTrue(ranges9.equals(generateRanges(70, 87, 50, 67, 67, 70)));
-        ranges10 = keyspace4ranges.get(InetAddressAndPort.getByName("127.0.0.10"));
-        assertEquals(collectionSize(ranges10), 3);
-        assertTrue(ranges10.equals(generateRanges(70, 87, 87, 97, 67, 70)));
+        RangesByEndpoint keyspace4ranges = keyspaceStrategyMap.get(Simple_RF3_KeyspaceName).getAddressReplicas();
+
+        assertRanges(keyspace4ranges, "127.0.0.1", 97, 0, 70, 87, 87, 97);
+        assertRanges(keyspace4ranges, "127.0.0.2", 97, 0, 87, 97, 0, 10);
+        assertRanges(keyspace4ranges, "127.0.0.3", 97, 0, 0, 10, 10, 20);
+        assertRanges(keyspace4ranges, "127.0.0.4", 20, 30, 0, 10, 10, 20);
+        assertRanges(keyspace4ranges, "127.0.0.5", 30, 40, 20, 30, 10, 20);
+        assertRanges(keyspace4ranges, "127.0.0.6", 40, 50, 30, 40, 20, 30);
+        assertRanges(keyspace4ranges, "127.0.0.7", 40, 50, 30, 40, 50, 67);
+        assertRanges(keyspace4ranges, "127.0.0.8", 40, 50, 50, 67, 67, 70);
+        assertRanges(keyspace4ranges, "127.0.0.9", 70, 87, 50, 67, 67, 70);
+        assertRanges(keyspace4ranges, "127.0.0.10", 70, 87, 87, 97, 67, 70);
 
         // pre-calculate the results.
         Map<String, Multimap<Token, InetAddressAndPort>> expectedEndpoints = new HashMap<>();
@@ -876,79 +825,80 @@ public class MoveTest
 
             for (Token token : keyTokens)
             {
-                endpoints = tmd.getWriteEndpoints(token, keyspaceName, strategy.getNaturalEndpoints(token));
+                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(token, keyspaceName, strategy.getNaturalReplicasForToken(token)).endpoints();
                 assertEquals(expectedEndpoints.get(keyspaceName).get(token).size(), endpoints.size());
                 assertTrue(expectedEndpoints.get(keyspaceName).get(token).containsAll(endpoints));
             }
 
             // just to be sure that things still work according to the old tests, run them:
-            if (strategy.getReplicationFactor() != 3)
+            if (strategy.getReplicationFactor().allReplicas != 3)
                 continue;
 
+            ReplicaCollection<?> replicas = null;
             // tokens 5, 15 and 25 should go three nodes
             for (int i = 0; i < 3; i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
-                assertEquals(3, endpoints.size());
-                assertTrue(endpoints.contains(hosts.get(i+1)));
-                assertTrue(endpoints.contains(hosts.get(i+2)));
-                assertTrue(endpoints.contains(hosts.get(i+3)));
+                replicas = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(i)));
+                assertEquals(3, replicas.size());
+                assertTrue(replicas.endpoints().contains(hosts.get(i + 1)));
+                assertTrue(replicas.endpoints().contains(hosts.get(i + 2)));
+                assertTrue(replicas.endpoints().contains(hosts.get(i + 3)));
             }
 
             // token 35 should go to nodes 4, 5, 6 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
-            assertEquals(4, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(4)));
-            assertTrue(endpoints.contains(hosts.get(5)));
-            assertTrue(endpoints.contains(hosts.get(6)));
-            assertTrue(endpoints.contains(boot1));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(3)));
+            assertEquals(4, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(4)));
+            assertTrue(replicas.endpoints().contains(hosts.get(5)));
+            assertTrue(replicas.endpoints().contains(hosts.get(6)));
+            assertTrue(replicas.endpoints().contains(boot1));
 
             // token 45 should go to nodes 5, 6, 7 boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
-            assertEquals(4, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(5)));
-            assertTrue(endpoints.contains(hosts.get(6)));
-            assertTrue(endpoints.contains(hosts.get(7)));
-            assertTrue(endpoints.contains(boot1));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(4)));
+            assertEquals(4, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(5)));
+            assertTrue(replicas.endpoints().contains(hosts.get(6)));
+            assertTrue(replicas.endpoints().contains(hosts.get(7)));
+            assertTrue(replicas.endpoints().contains(boot1));
 
             // token 55 should go to nodes 6, 7, 8 boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
-            assertEquals(5, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(6)));
-            assertTrue(endpoints.contains(hosts.get(7)));
-            assertTrue(endpoints.contains(hosts.get(8)));
-            assertTrue(endpoints.contains(boot1));
-            assertTrue(endpoints.contains(boot2));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(5)));
+            assertEquals(5, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(6)));
+            assertTrue(replicas.endpoints().contains(hosts.get(7)));
+            assertTrue(replicas.endpoints().contains(hosts.get(8)));
+            assertTrue(replicas.endpoints().contains(boot1));
+            assertTrue(replicas.endpoints().contains(boot2));
 
             // token 65 should go to nodes 6, 7, 8 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
-            assertEquals(4, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(6)));
-            assertTrue(endpoints.contains(hosts.get(7)));
-            assertTrue(endpoints.contains(hosts.get(8)));
-            assertTrue(endpoints.contains(boot2));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(6)));
+            assertEquals(4, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(6)));
+            assertTrue(replicas.endpoints().contains(hosts.get(7)));
+            assertTrue(replicas.endpoints().contains(hosts.get(8)));
+            assertTrue(replicas.endpoints().contains(boot2));
 
             // token 75 should to go nodes 8, 9, 0 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
-            assertEquals(4, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(8)));
-            assertTrue(endpoints.contains(hosts.get(9)));
-            assertTrue(endpoints.contains(hosts.get(0)));
-            assertTrue(endpoints.contains(boot2));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(7)));
+            assertEquals(4, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(8)));
+            assertTrue(replicas.endpoints().contains(hosts.get(9)));
+            assertTrue(replicas.endpoints().contains(hosts.get(0)));
+            assertTrue(replicas.endpoints().contains(boot2));
 
             // token 85 should go to nodes 8, 9 and 0
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
-            assertEquals(3, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(8)));
-            assertTrue(endpoints.contains(hosts.get(9)));
-            assertTrue(endpoints.contains(hosts.get(0)));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(8)));
+            assertEquals(3, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(8)));
+            assertTrue(replicas.endpoints().contains(hosts.get(9)));
+            assertTrue(replicas.endpoints().contains(hosts.get(0)));
 
             // token 95 should go to nodes 9, 0 and 1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
-            assertEquals(3, endpoints.size());
-            assertTrue(endpoints.contains(hosts.get(9)));
-            assertTrue(endpoints.contains(hosts.get(0)));
-            assertTrue(endpoints.contains(hosts.get(1)));
+            replicas = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalReplicasForToken(keyTokens.get(9)));
+            assertEquals(3, replicas.size());
+            assertTrue(replicas.endpoints().contains(hosts.get(9)));
+            assertTrue(replicas.endpoints().contains(hosts.get(0)));
+            assertTrue(replicas.endpoints().contains(hosts.get(1)));
         }
 
         // all moving nodes are back to the normal state
@@ -1009,6 +959,14 @@ public class MoveTest
         return addrs;
     }
 
+    private static EndpointsForRange makeReplicas(Range<Token> range, String... hosts) throws UnknownHostException
+    {
+        EndpointsForRange.Builder replicas = EndpointsForRange.builder(range, hosts.length);
+        for (String host : hosts)
+            replicas.add(Replica.fullReplica(InetAddressAndPort.getByName(host), range));
+        return replicas.build();
+    }
+
     private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd)
     {
         KeyspaceMetadata ksmd = Schema.instance.getKeyspaceMetadata(keyspaceName);
@@ -1025,7 +983,7 @@ public class MoveTest
         return new BigIntegerToken(String.valueOf(10 * position + 7));
     }
 
-    private int collectionSize(Collection<?> collection)
+    private static int collectionSize(Collection<?> collection)
     {
         if (collection.isEmpty())
             return 0;
@@ -1057,8 +1015,52 @@ public class MoveTest
         return ranges;
     }
 
-    private Range<Token> generateRange(int left, int right)
+    private static Token tk(int v)
+    {
+        return new BigIntegerToken(String.valueOf(v));
+    }
+
+    private static Range<Token> generateRange(int left, int right)
+    {
+        return new Range<Token>(tk(left), tk(right));
+    }
+
+    private static Replica replica(InetAddressAndPort endpoint, int left, int right, boolean full)
+    {
+        return new Replica(endpoint, tk(left), tk(right), full);
+    }
+
+    private static InetAddressAndPort inet(String name)
     {
-        return new Range<Token>(new BigIntegerToken(String.valueOf(left)), new BigIntegerToken(String.valueOf(right)));
+        try
+        {
+            return InetAddressAndPort.getByName(name);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
+    private static Replica replica(InetAddressAndPort endpoint, int left, int right)
+    {
+        return replica(endpoint, left, right, true);
+    }
+
+    private static void assertRanges(RangesByEndpoint epReplicas, String endpoint, int... rangePairs)
+    {
+        if (rangePairs.length % 2 == 1)
+            throw new RuntimeException("assertRanges argument count should be even");
+
+        InetAddressAndPort ep = inet(endpoint);
+        List<Replica> expected = new ArrayList<>(rangePairs.length/2);
+        for (int i=0; i<rangePairs.length; i+=2)
+            expected.add(replica(ep, rangePairs[i], rangePairs[i+1]));
+
+        RangesAtEndpoint actual = epReplicas.get(ep);
+        assertEquals(expected.size(), actual.size());
+        for (Replica replica : expected)
+            if (!actual.contains(replica))
+                assertEquals(RangesAtEndpoint.copyOf(expected), actual);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/MoveTransientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTransientTest.java b/test/unit/org/apache/cassandra/service/MoveTransientTest.java
new file mode 100644
index 0000000..1e24735
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/MoveTransientTest.java
@@ -0,0 +1,638 @@
+/*
+ * 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.service;
+
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Predicate;
+import org.apache.cassandra.locator.EndpointsByReplica;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.RangesByEndpoint;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.RangeStreamer;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+import static org.apache.cassandra.service.StorageServiceTest.assertMultimapEqualsIgnoreOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This is also fairly effectively testing source retrieval for bootstrap as well since RangeStreamer
+ * is used to calculate the endpoints to fetch from and check they are alive for both RangeRelocator (move) and
+ * bootstrap (RangeRelocator).
+ */
+public class MoveTransientTest
+{
+    private static final Logger logger = LoggerFactory.getLogger(MoveTransientTest.class);
+
+    static InetAddressAndPort aAddress;
+    static InetAddressAndPort bAddress;
+    static InetAddressAndPort cAddress;
+    static InetAddressAndPort dAddress;
+    static InetAddressAndPort eAddress;
+
+    @BeforeClass
+    public static void setUpClass() throws Exception
+    {
+        aAddress = InetAddressAndPort.getByName("127.0.0.1");
+        bAddress = InetAddressAndPort.getByName("127.0.0.2");
+        cAddress = InetAddressAndPort.getByName("127.0.0.3");
+        dAddress = InetAddressAndPort.getByName("127.0.0.4");
+        eAddress = InetAddressAndPort.getByName("127.0.0.5");
+    }
+
+    private final List<InetAddressAndPort> downNodes = new ArrayList();
+    Predicate<Replica> alivePredicate = replica -> !downNodes.contains(replica.endpoint());
+
+    private final List<InetAddressAndPort> sourceFilterDownNodes = new ArrayList<>();
+    private final Collection<Predicate<Replica>> sourceFilters = Collections.singleton(replica -> !sourceFilterDownNodes.contains(replica.endpoint()));
+
+    @After
+    public void clearDownNode()
+    {
+        downNodes.clear();
+        sourceFilterDownNodes.clear();
+    }
+
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    Token oneToken = new RandomPartitioner.BigIntegerToken("1");
+    Token twoToken = new RandomPartitioner.BigIntegerToken("2");
+    Token threeToken = new RandomPartitioner.BigIntegerToken("3");
+    Token fourToken = new RandomPartitioner.BigIntegerToken("4");
+    Token sixToken = new RandomPartitioner.BigIntegerToken("6");
+    Token sevenToken = new RandomPartitioner.BigIntegerToken("7");
+    Token nineToken = new RandomPartitioner.BigIntegerToken("9");
+    Token elevenToken = new RandomPartitioner.BigIntegerToken("11");
+    Token fourteenToken = new RandomPartitioner.BigIntegerToken("14");
+
+    Range<Token> aRange = new Range(oneToken, threeToken);
+    Range<Token> bRange = new Range(threeToken, sixToken);
+    Range<Token> cRange = new Range(sixToken, nineToken);
+    Range<Token> dRange = new Range(nineToken, elevenToken);
+    Range<Token> eRange = new Range(elevenToken, oneToken);
+
+
+    RangesAtEndpoint current = RangesAtEndpoint.of(new Replica(aAddress, aRange, true),
+                                       new Replica(aAddress, eRange, true),
+                                       new Replica(aAddress, dRange, false));
+
+
+    /**
+     * Ring with start A 1-3 B 3-6 C 6-9 D 9-1
+     * A's token moves from 3 to 4.
+     * <p>
+     * Result is A gains some range
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testCalculateStreamAndFetchRangesMoveForward() throws Exception
+    {
+        calculateStreamAndFetchRangesMoveForward();
+    }
+
+    private Pair<RangesAtEndpoint, RangesAtEndpoint> calculateStreamAndFetchRangesMoveForward() throws Exception
+    {
+        Range<Token> aPrimeRange = new Range<>(oneToken, fourToken);
+
+        RangesAtEndpoint updated = RangesAtEndpoint.of(
+                new Replica(aAddress, aPrimeRange, true),
+                new Replica(aAddress, eRange, true),
+                new Replica(aAddress, dRange, false)
+        );
+
+        Pair<RangesAtEndpoint, RangesAtEndpoint> result = StorageService.calculateStreamAndFetchRanges(current, updated);
+        assertContentsIgnoreOrder(result.left);
+        assertContentsIgnoreOrder(result.right, fullReplica(aAddress, threeToken, fourToken));
+        return result;
+    }
+
+    /**
+     * Ring with start A 1-3 B 3-6 C 6-9 D 9-11 E 11-1
+     * A's token moves from 3 to 14
+     * <p>
+     * Result is A loses range and it must be streamed
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testCalculateStreamAndFetchRangesMoveBackwardBetween() throws Exception
+    {
+        calculateStreamAndFetchRangesMoveBackwardBetween();
+    }
+
+    public Pair<RangesAtEndpoint, RangesAtEndpoint> calculateStreamAndFetchRangesMoveBackwardBetween() throws Exception
+    {
+        Range<Token> aPrimeRange = new Range<>(elevenToken, fourteenToken);
+
+        RangesAtEndpoint updated = RangesAtEndpoint.of(
+            new Replica(aAddress, aPrimeRange, true),
+            new Replica(aAddress, dRange, true),
+            new Replica(aAddress, cRange, false)
+        );
+
+
+        Pair<RangesAtEndpoint, RangesAtEndpoint> result = StorageService.calculateStreamAndFetchRanges(current, updated);
+        assertContentsIgnoreOrder(result.left, fullReplica(aAddress, oneToken, threeToken), fullReplica(aAddress, fourteenToken, oneToken));
+        assertContentsIgnoreOrder(result.right, transientReplica(aAddress, sixToken, nineToken), fullReplica(aAddress, nineToken, elevenToken));
+        return result;
+    }
+
+    /**
+     * Ring with start A 1-3 B 3-6 C 6-9 D 9-11 E 11-1
+     * A's token moves from 3 to 2
+     *
+     * Result is A loses range and it must be streamed
+     * @throws Exception
+     */
+    @Test
+    public void testCalculateStreamAndFetchRangesMoveBackward() throws Exception
+    {
+        calculateStreamAndFetchRangesMoveBackward();
+    }
+
+    private Pair<RangesAtEndpoint, RangesAtEndpoint> calculateStreamAndFetchRangesMoveBackward() throws Exception
+    {
+        Range<Token> aPrimeRange = new Range<>(oneToken, twoToken);
+
+        RangesAtEndpoint updated = RangesAtEndpoint.of(
+            new Replica(aAddress, aPrimeRange, true),
+            new Replica(aAddress, eRange, true),
+            new Replica(aAddress, dRange, false)
+        );
+
+        Pair<RangesAtEndpoint, RangesAtEndpoint> result = StorageService.calculateStreamAndFetchRanges(current, updated);
+
+        //Moving backwards has no impact on any replica. We already fully replicate counter clockwise
+        //The transient replica does transiently replicate slightly more, but that is addressed by cleanup
+        assertContentsIgnoreOrder(result.left, fullReplica(aAddress, twoToken, threeToken));
+        assertContentsIgnoreOrder(result.right);
+
+        return result;
+    }
+
+    /**
+     * Ring with start A 1-3 B 3-6 C 6-9 D 9-11 E 11-1
+     * A's moves from 3 to 7
+     *
+     * @throws Exception
+     */
+    private Pair<RangesAtEndpoint, RangesAtEndpoint> calculateStreamAndFetchRangesMoveForwardBetween() throws Exception
+    {
+        Range<Token> aPrimeRange = new Range<>(sixToken, sevenToken);
+        Range<Token> bPrimeRange = new Range<>(oneToken, sixToken);
+
+
+        RangesAtEndpoint updated = RangesAtEndpoint.of(
+            new Replica(aAddress, aPrimeRange, true),
+            new Replica(aAddress, bPrimeRange, true),
+            new Replica(aAddress, eRange, false)
+        );
+
+        Pair<RangesAtEndpoint, RangesAtEndpoint> result = StorageService.calculateStreamAndFetchRanges(current, updated);
+
+        assertContentsIgnoreOrder(result.left, fullReplica(aAddress, elevenToken, oneToken), transientReplica(aAddress, nineToken, elevenToken));
+        assertContentsIgnoreOrder(result.right, fullReplica(aAddress, threeToken, sixToken), fullReplica(aAddress, sixToken, sevenToken));
+        return result;
+    }
+
+    /**
+     * Ring with start A 1-3 B 3-6 C 6-9 D 9-11 E 11-1
+     * A's token moves from 3 to 7
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testCalculateStreamAndFetchRangesMoveForwardBetween() throws Exception
+    {
+        calculateStreamAndFetchRangesMoveForwardBetween();
+    }
+
+    /**
+     * Construct the ring state for calculateStreamAndFetchRangesMoveBackwardBetween
+     * Where are A moves from 3 to 14
+     * @return
+     */
+    private Pair<TokenMetadata, TokenMetadata> constructTMDsMoveBackwardBetween()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(aRange.right, aAddress);
+        tmd.updateNormalToken(bRange.right, bAddress);
+        tmd.updateNormalToken(cRange.right, cAddress);
+        tmd.updateNormalToken(dRange.right, dAddress);
+        tmd.updateNormalToken(eRange.right, eAddress);
+        tmd.addMovingEndpoint(fourteenToken, aAddress);
+        TokenMetadata updated = tmd.cloneAfterAllSettled();
+
+        return Pair.create(tmd, updated);
+    }
+
+
+    /**
+     * Construct the ring state for calculateStreamAndFetchRangesMoveForwardBetween
+     * Where are A moves from 3 to 7
+     * @return
+     */
+    private Pair<TokenMetadata, TokenMetadata> constructTMDsMoveForwardBetween()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(aRange.right, aAddress);
+        tmd.updateNormalToken(bRange.right, bAddress);
+        tmd.updateNormalToken(cRange.right, cAddress);
+        tmd.updateNormalToken(dRange.right, dAddress);
+        tmd.updateNormalToken(eRange.right, eAddress);
+        tmd.addMovingEndpoint(sevenToken, aAddress);
+        TokenMetadata updated = tmd.cloneAfterAllSettled();
+
+        return Pair.create(tmd, updated);
+    }
+
+    private Pair<TokenMetadata, TokenMetadata> constructTMDsMoveBackward()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(aRange.right, aAddress);
+        tmd.updateNormalToken(bRange.right, bAddress);
+        tmd.updateNormalToken(cRange.right, cAddress);
+        tmd.updateNormalToken(dRange.right, dAddress);
+        tmd.updateNormalToken(eRange.right, eAddress);
+        tmd.addMovingEndpoint(twoToken, aAddress);
+        TokenMetadata updated = tmd.cloneAfterAllSettled();
+
+        return Pair.create(tmd, updated);
+    }
+
+    private Pair<TokenMetadata, TokenMetadata> constructTMDsMoveForward()
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(aRange.right, aAddress);
+        tmd.updateNormalToken(bRange.right, bAddress);
+        tmd.updateNormalToken(cRange.right, cAddress);
+        tmd.updateNormalToken(dRange.right, dAddress);
+        tmd.updateNormalToken(eRange.right, eAddress);
+        tmd.addMovingEndpoint(fourToken, aAddress);
+        TokenMetadata updated = tmd.cloneAfterAllSettled();
+
+        return Pair.create(tmd, updated);
+    }
+
+
+    @Test
+    public void testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints() throws Exception
+    {
+        EndpointsByReplica.Mutable expectedResult = new EndpointsByReplica.Mutable();
+
+        InetAddressAndPort cOrB = (downNodes.contains(cAddress) || sourceFilterDownNodes.contains(cAddress)) ? bAddress : cAddress;
+
+        //Need to pull the full replica and the transient replica that is losing the range
+        expectedResult.put(fullReplica(aAddress, sixToken, sevenToken),  fullReplica(dAddress, sixToken, nineToken));
+        expectedResult.put(fullReplica(aAddress, sixToken, sevenToken), transientReplica(eAddress, sixToken, nineToken));
+
+        //Same need both here as well
+        expectedResult.put(fullReplica(aAddress, threeToken, sixToken), fullReplica(cOrB, threeToken, sixToken));
+        expectedResult.put(fullReplica(aAddress, threeToken, sixToken), transientReplica(dAddress, threeToken, sixToken));
+
+        invokeCalculateRangesToFetchWithPreferredEndpoints(calculateStreamAndFetchRangesMoveForwardBetween().right,
+                                                           constructTMDsMoveForwardBetween(),
+                                                           expectedResult.asImmutableView());
+    }
+
+    @Test
+    public void testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpointsDownNodes() throws Exception
+    {
+        for (InetAddressAndPort downNode : new InetAddressAndPort[] {dAddress, eAddress})
+        {
+            downNodes.clear();
+            downNodes.add(downNode);
+            boolean threw = false;
+            try
+            {
+                testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+            }
+            catch (IllegalStateException ise)
+            {
+                ise.printStackTrace();
+                assertTrue(downNode.toString(),
+                           ise.getMessage().startsWith("A node required to move the data consistently is down:")
+                                    && ise.getMessage().contains(downNode.toString()));
+                threw = true;
+            }
+            assertTrue("Didn't throw for " + downNode, threw);
+        }
+
+        //Shouldn't throw because another full replica is available
+        downNodes.clear();
+        downNodes.add(cAddress);
+        testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+    @Test
+    public void testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpointsDownNodesSourceFilter() throws Exception
+    {
+        for (InetAddressAndPort downNode : new InetAddressAndPort[] {dAddress, eAddress})
+        {
+            sourceFilterDownNodes.clear();
+            sourceFilterDownNodes.add(downNode);
+            boolean threw = false;
+            try
+            {
+                testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+            }
+            catch (IllegalStateException ise)
+            {
+                ise.printStackTrace();
+                assertTrue(downNode.toString(),
+                           ise.getMessage().startsWith("Necessary replicas for strict consistency were removed by source filters:")
+                           && ise.getMessage().contains(downNode.toString()));
+                threw = true;
+            }
+            assertTrue("Didn't throw for " + downNode, threw);
+        }
+
+        //Shouldn't throw because another full replica is available
+        sourceFilterDownNodes.clear();
+        sourceFilterDownNodes.add(cAddress);
+        testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+    @Test
+    public void testMoveBackwardBetweenCalculateRangesToFetchWithPreferredEndpoints() throws Exception
+    {
+        EndpointsByReplica.Mutable expectedResult = new EndpointsByReplica.Mutable();
+
+        //Need to pull the full replica and the transient replica that is losing the range
+        expectedResult.put(fullReplica(aAddress, nineToken, elevenToken), fullReplica(eAddress, nineToken, elevenToken));
+        expectedResult.put(transientReplica(aAddress, sixToken, nineToken), transientReplica(eAddress, sixToken, nineToken));
+
+        invokeCalculateRangesToFetchWithPreferredEndpoints(calculateStreamAndFetchRangesMoveBackwardBetween().right,
+                                                           constructTMDsMoveBackwardBetween(),
+                                                           expectedResult.asImmutableView());
+
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testMoveBackwardBetweenCalculateRangesToFetchWithPreferredEndpointsDownNodes() throws Exception
+    {
+        //Any replica can be the full replica so this will always fail on the transient range
+        downNodes.add(eAddress);
+        testMoveBackwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testMoveBackwardBetweenCalculateRangesToFetchWithPreferredEndpointsDownNodesSourceFilter() throws Exception
+    {
+        //Any replica can be the full replica so this will always fail on the transient range
+        sourceFilterDownNodes.add(eAddress);
+        testMoveBackwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+
+    //There is no down node version of this test because nothing needs to be fetched
+    @Test
+    public void testMoveBackwardCalculateRangesToFetchWithPreferredEndpoints() throws Exception
+    {
+        //Moving backwards should fetch nothing and fetch ranges is emptys so this doesn't test a ton
+        EndpointsByReplica.Mutable expectedResult = new EndpointsByReplica.Mutable();
+
+        invokeCalculateRangesToFetchWithPreferredEndpoints(calculateStreamAndFetchRangesMoveBackward().right,
+                                                           constructTMDsMoveBackward(),
+                                                           expectedResult.asImmutableView());
+
+    }
+
+    @Test
+    public void testMoveForwardCalculateRangesToFetchWithPreferredEndpoints() throws Exception
+    {
+        EndpointsByReplica.Mutable expectedResult = new EndpointsByReplica.Mutable();
+
+        InetAddressAndPort cOrBAddress = (downNodes.contains(cAddress) || sourceFilterDownNodes.contains(cAddress)) ? bAddress : cAddress;
+
+        //Need to pull the full replica and the transient replica that is losing the range
+        expectedResult.put(fullReplica(aAddress, threeToken, fourToken), fullReplica(cOrBAddress, threeToken, sixToken));
+        expectedResult.put(fullReplica(aAddress, threeToken, fourToken), transientReplica(dAddress, threeToken, sixToken));
+
+        invokeCalculateRangesToFetchWithPreferredEndpoints(calculateStreamAndFetchRangesMoveForward().right,
+                                                           constructTMDsMoveForward(),
+                                                           expectedResult.asImmutableView());
+
+    }
+
+    @Test
+    public void testMoveForwardCalculateRangesToFetchWithPreferredEndpointsDownNodes() throws Exception
+    {
+        downNodes.add(dAddress);
+        boolean threw = false;
+        try
+        {
+            testMoveForwardCalculateRangesToFetchWithPreferredEndpoints();
+        }
+        catch (IllegalStateException ise)
+        {
+            ise.printStackTrace();
+            assertTrue(dAddress.toString(),
+                       ise.getMessage().startsWith("A node required to move the data consistently is down:")
+                       && ise.getMessage().contains(dAddress.toString()));
+            threw = true;
+        }
+        assertTrue("Didn't throw for " + dAddress, threw);
+
+        //Shouldn't throw because another full replica is available
+        downNodes.clear();
+        downNodes.add(cAddress);
+        testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+    @Test
+    public void testMoveForwardCalculateRangesToFetchWithPreferredEndpointsDownNodesSourceFilter() throws Exception
+    {
+        sourceFilterDownNodes.add(dAddress);
+        boolean threw = false;
+        try
+        {
+            testMoveForwardCalculateRangesToFetchWithPreferredEndpoints();
+        }
+        catch (IllegalStateException ise)
+        {
+            ise.printStackTrace();
+            assertTrue(dAddress.toString(),
+                       ise.getMessage().startsWith("Necessary replicas for strict consistency were removed by source filters:")
+                       && ise.getMessage().contains(dAddress.toString()));
+            threw = true;
+        }
+        assertTrue("Didn't throw for " + dAddress, threw);
+
+        //Shouldn't throw because another full replica is available
+        sourceFilterDownNodes.clear();
+        sourceFilterDownNodes.add(cAddress);
+        testMoveForwardBetweenCalculateRangesToFetchWithPreferredEndpoints();
+    }
+
+    private void invokeCalculateRangesToFetchWithPreferredEndpoints(RangesAtEndpoint toFetch,
+                                                                    Pair<TokenMetadata, TokenMetadata> tmds,
+                                                                    EndpointsByReplica expectedResult)
+    {
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+
+        EndpointsByReplica result = RangeStreamer.calculateRangesToFetchWithPreferredEndpoints((address, replicas) -> replicas.sorted((a, b) -> b.endpoint().compareTo(a.endpoint())),
+                                                                                                                   simpleStrategy(tmds.left),
+                                                                                                                   toFetch,
+                                                                                                                   true,
+                                                                                                                   tmds.left,
+                                                                                                                   tmds.right,
+                                                                                                                   alivePredicate,
+                                                                                                                   "OldNetworkTopologyStrategyTest",
+                                                                                                                   sourceFilters);
+        logger.info("Ranges to fetch with preferred endpoints");
+        logger.info(result.toString());
+        assertMultimapEqualsIgnoreOrder(expectedResult, result);
+
+    }
+
+    private AbstractReplicationStrategy simpleStrategy(TokenMetadata tmd)
+    {
+        IEndpointSnitch snitch = new AbstractEndpointSnitch()
+        {
+            public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2)
+            {
+                return 0;
+            }
+
+            public String getRack(InetAddressAndPort endpoint)
+            {
+                return "R1";
+            }
+
+            public String getDatacenter(InetAddressAndPort endpoint)
+            {
+                return "DC1";
+            }
+        };
+
+        return new SimpleStrategy("MoveTransientTest",
+                                  tmd,
+                                  snitch,
+                                  com.google.common.collect.ImmutableMap.of("replication_factor", "3/1"));
+    }
+
+    @Test
+    public void testMoveForwardBetweenCalculateRangesToStreamWithPreferredEndpoints() throws Exception
+    {
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        RangesByEndpoint.Mutable expectedResult = new RangesByEndpoint.Mutable();
+
+        //Need to pull the full replica and the transient replica that is losing the range
+        expectedResult.put(bAddress, transientReplica(bAddress, nineToken, elevenToken));
+        expectedResult.put(bAddress, fullReplica(bAddress, elevenToken, oneToken));
+
+        invokeCalculateRangesToStreamWithPreferredEndpoints(calculateStreamAndFetchRangesMoveForwardBetween().left,
+                                                            constructTMDsMoveForwardBetween(),
+                                                            expectedResult.asImmutableView());
+    }
+
+    @Test
+    public void testMoveBackwardBetweenCalculateRangesToStreamWithPreferredEndpoints() throws Exception
+    {
+        RangesByEndpoint.Mutable expectedResult = new RangesByEndpoint.Mutable();
+
+        expectedResult.put(bAddress, fullReplica(bAddress, fourteenToken, oneToken));
+
+        expectedResult.put(dAddress, transientReplica(dAddress, oneToken, threeToken));
+
+        expectedResult.put(cAddress, fullReplica(cAddress, oneToken, threeToken));
+        expectedResult.put(cAddress, transientReplica(cAddress, fourteenToken, oneToken));
+
+        invokeCalculateRangesToStreamWithPreferredEndpoints(calculateStreamAndFetchRangesMoveBackwardBetween().left,
+                                                            constructTMDsMoveBackwardBetween(),
+                                                            expectedResult.asImmutableView());
+    }
+
+    @Test
+    public void testMoveBackwardCalculateRangesToStreamWithPreferredEndpoints() throws Exception
+    {
+        RangesByEndpoint.Mutable expectedResult = new RangesByEndpoint.Mutable();
+        expectedResult.put(cAddress, fullReplica(cAddress, twoToken, threeToken));
+        expectedResult.put(dAddress, transientReplica(dAddress, twoToken, threeToken));
+
+        invokeCalculateRangesToStreamWithPreferredEndpoints(calculateStreamAndFetchRangesMoveBackward().left,
+                                                            constructTMDsMoveBackward(),
+                                                            expectedResult.asImmutableView());
+    }
+
+    @Test
+    public void testMoveForwardCalculateRangesToStreamWithPreferredEndpoints() throws Exception
+    {
+        //Nothing to stream moving forward because we are acquiring more range not losing range
+        RangesByEndpoint.Mutable expectedResult = new RangesByEndpoint.Mutable();
+
+        invokeCalculateRangesToStreamWithPreferredEndpoints(calculateStreamAndFetchRangesMoveForward().left,
+                                                            constructTMDsMoveForward(),
+                                                            expectedResult.asImmutableView());
+    }
+
+    private void invokeCalculateRangesToStreamWithPreferredEndpoints(RangesAtEndpoint toStream,
+                                                                     Pair<TokenMetadata, TokenMetadata> tmds,
+                                                                     RangesByEndpoint expectedResult)
+    {
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        StorageService.RangeRelocator relocator = new StorageService.RangeRelocator();
+        RangesByEndpoint result = relocator.calculateRangesToStreamWithEndpoints(toStream,
+                                                                                 simpleStrategy(tmds.left),
+                                                                                 tmds.left,
+                                                                                 tmds.right);
+        logger.info("Ranges to stream by endpoint");
+        logger.info(result.toString());
+        assertMultimapEqualsIgnoreOrder(expectedResult, result);
+    }
+
+    private static void assertContentsIgnoreOrder(RangesAtEndpoint ranges, Replica ... replicas)
+    {
+        assertEquals(ranges.size(), replicas.length);
+        for (Replica replica : replicas)
+            if (!ranges.contains(replica))
+                assertEquals(RangesAtEndpoint.of(replicas), ranges);
+    }
+
+}


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


[07/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/NoopReadRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/NoopReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/NoopReadRepair.java
index a43e3eb..4af4a92 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/NoopReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/NoopReadRepair.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import java.util.List;
 import java.util.Map;
 import java.util.function.Consumer;
 
@@ -27,24 +26,28 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.service.reads.DigestResolver;
 
 /**
  * Bypasses the read repair path for short read protection and testing
  */
-public class NoopReadRepair implements ReadRepair
+public class NoopReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> implements ReadRepair<E, L>
 {
     public static final NoopReadRepair instance = new NoopReadRepair();
 
     private NoopReadRepair() {}
 
-    public UnfilteredPartitionIterators.MergeListener getMergeListener(InetAddressAndPort[] endpoints)
+    @Override
+    public UnfilteredPartitionIterators.MergeListener getMergeListener(L replicas)
     {
         return UnfilteredPartitionIterators.MergeListener.NOOP;
     }
 
-    public void startRepair(DigestResolver digestResolver, List<InetAddressAndPort> allEndpoints, List<InetAddressAndPort> contactedEndpoints, Consumer<PartitionIterator> resultConsumer)
+    @Override
+    public void startRepair(DigestResolver<E, L> digestResolver, Consumer<PartitionIterator> resultConsumer)
     {
         resultConsumer.accept(digestResolver.getData());
     }
@@ -72,7 +75,7 @@ public class NoopReadRepair implements ReadRepair
     }
 
     @Override
-    public void repairPartition(DecoratedKey key, Map<InetAddressAndPort, Mutation> mutations, InetAddressAndPort[] destinations)
+    public void repairPartition(DecoratedKey partitionKey, Map<Replica, Mutation> mutations, L replicaLayout)
     {
 
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java b/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
index 6cf761a..4cae3ae 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/PartitionIteratorMergeListener.java
@@ -28,18 +28,18 @@ import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
 
 public class PartitionIteratorMergeListener implements UnfilteredPartitionIterators.MergeListener
 {
-    private final InetAddressAndPort[] sources;
+    private final ReplicaLayout replicaLayout;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
     private final ReadRepair readRepair;
 
-    public PartitionIteratorMergeListener(InetAddressAndPort[] sources, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair)
+    public PartitionIteratorMergeListener(ReplicaLayout replicaLayout, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair)
     {
-        this.sources = sources;
+        this.replicaLayout = replicaLayout;
         this.command = command;
         this.consistency = consistency;
         this.readRepair = readRepair;
@@ -47,10 +47,10 @@ public class PartitionIteratorMergeListener implements UnfilteredPartitionIterat
 
     public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List<UnfilteredRowIterator> versions)
     {
-        return new RowIteratorMergeListener(partitionKey, columns(versions), isReversed(versions), sources, command, consistency, readRepair);
+        return new RowIteratorMergeListener(partitionKey, columns(versions), isReversed(versions), replicaLayout, command, consistency, readRepair);
     }
 
-    private RegularAndStaticColumns columns(List<UnfilteredRowIterator> versions)
+    protected RegularAndStaticColumns columns(List<UnfilteredRowIterator> versions)
     {
         Columns statics = Columns.NONE;
         Columns regulars = Columns.NONE;
@@ -66,7 +66,7 @@ public class PartitionIteratorMergeListener implements UnfilteredPartitionIterat
         return new RegularAndStaticColumns(statics, regulars);
     }
 
-    private boolean isReversed(List<UnfilteredRowIterator> versions)
+    protected boolean isReversed(List<UnfilteredRowIterator> versions)
     {
         for (UnfilteredRowIterator iter : versions)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepair.java
index d994b23..c13e2d6 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadOnlyReadRepair.java
@@ -21,27 +21,28 @@ package org.apache.cassandra.service.reads.repair;
 import java.util.Map;
 
 import com.codahale.metrics.Meter;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 
 /**
  * Only performs the collection of data responses and reconciliation of them, doesn't send repair mutations
  * to replicas. This preserves write atomicity, but doesn't provide monotonic quorum reads
  */
-public class ReadOnlyReadRepair extends AbstractReadRepair
+public class ReadOnlyReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends AbstractReadRepair<E, L>
 {
-    public ReadOnlyReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+    ReadOnlyReadRepair(ReadCommand command, L replicaLayout, long queryStartNanoTime)
     {
-        super(command, queryStartNanoTime, consistency);
+        super(command, replicaLayout, queryStartNanoTime);
     }
 
     @Override
-    public UnfilteredPartitionIterators.MergeListener getMergeListener(InetAddressAndPort[] endpoints)
+    public UnfilteredPartitionIterators.MergeListener getMergeListener(L replicaLayout)
     {
         return UnfilteredPartitionIterators.MergeListener.NOOP;
     }
@@ -59,7 +60,7 @@ public class ReadOnlyReadRepair extends AbstractReadRepair
     }
 
     @Override
-    public void repairPartition(DecoratedKey key, Map<InetAddressAndPort, Mutation> mutations, InetAddressAndPort[] destinations)
+    public void repairPartition(DecoratedKey partitionKey, Map<Replica, Mutation> mutations, L replicaLayout)
     {
         throw new UnsupportedOperationException("ReadOnlyReadRepair shouldn't be trying to repair partitions");
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
index 97f0f67..168f003 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepair.java
@@ -17,44 +17,45 @@
  */
 package org.apache.cassandra.service.reads.repair;
 
-import java.util.List;
 import java.util.Map;
 import java.util.function.Consumer;
 
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.locator.Endpoints;
+
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.service.reads.DigestResolver;
 
-public interface ReadRepair
+public interface ReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>>
 {
     public interface Factory
     {
-        ReadRepair create(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency);
+        <E extends Endpoints<E>, L extends ReplicaLayout<E, L>> ReadRepair<E, L> create(ReadCommand command, L replicaLayout, long queryStartNanoTime);
+    }
+
+    static <E extends Endpoints<E>, L extends ReplicaLayout<E, L>> ReadRepair<E, L> create(ReadCommand command, L replicaPlan, long queryStartNanoTime)
+    {
+        return command.metadata().params.readRepair.create(command, replicaPlan, queryStartNanoTime);
     }
 
     /**
      * Used by DataResolver to generate corrections as the partition iterator is consumed
      */
-    UnfilteredPartitionIterators.MergeListener getMergeListener(InetAddressAndPort[] endpoints);
+    UnfilteredPartitionIterators.MergeListener getMergeListener(L replicaLayout);
 
     /**
      * Called when the digests from the initial read don't match. Reads may block on the
      * repair started by this method.
      * @param digestResolver supplied so we can get the original data response
-     * @param allEndpoints all available replicas for this read
-     * @param contactedEndpoints the replicas we actually sent requests to
      * @param resultConsumer hook for the repair to set it's result on completion
      */
-    public void startRepair(DigestResolver digestResolver,
-                            List<InetAddressAndPort> allEndpoints,
-                            List<InetAddressAndPort> contactedEndpoints,
-                            Consumer<PartitionIterator> resultConsumer);
+    public void startRepair(DigestResolver<E, L> digestResolver, Consumer<PartitionIterator> resultConsumer);
 
     /**
      * Block on the reads (or timeout) sent out in {@link ReadRepair#startRepair}
@@ -81,17 +82,13 @@ public interface ReadRepair
     public void maybeSendAdditionalWrites();
 
     /**
-     * Hook for the merge listener to start repairs on individual partitions.
-     */
-    void repairPartition(DecoratedKey key, Map<InetAddressAndPort, Mutation> mutations, InetAddressAndPort[] destinations);
-
-    /**
      * Block on any mutations (or timeout) we sent out to repair replicas in {@link ReadRepair#repairPartition}
      */
     public void awaitWrites();
 
-    static ReadRepair create(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
-    {
-        return command.metadata().params.readRepair.create(command, queryStartNanoTime, consistency);
-    }
+    /**
+     * Repairs a partition _after_ receiving data responses. This method receives replica list, since
+     * we will block repair only on the replicas that have responded.
+     */
+    void repairPartition(DecoratedKey partitionKey, Map<Replica, Mutation> mutations, L replicaLayout);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
index 1117822..6eff395 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairDiagnostics.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.service.reads.repair;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
@@ -38,8 +39,8 @@ final class ReadRepairDiagnostics
     {
     }
 
-    static void startRepair(AbstractReadRepair readRepair, List<InetAddressAndPort> endpointDestinations,
-                            DigestResolver digestResolver, List<InetAddressAndPort> allEndpoints)
+    static void startRepair(AbstractReadRepair readRepair, Collection<InetAddressAndPort> endpointDestinations,
+                            DigestResolver digestResolver, Collection<InetAddressAndPort> allEndpoints)
     {
         if (service.isEnabled(ReadRepairEvent.class, ReadRepairEventType.START_REPAIR))
             service.publish(new ReadRepairEvent(ReadRepairEventType.START_REPAIR,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
index 152f7e6..9e14362 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairEvent.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.service.reads.repair;
 
 import java.io.Serializable;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -48,9 +49,9 @@ final class ReadRepairEvent extends DiagnosticEvent
     private final ConsistencyLevel consistency;
     private final SpeculativeRetryPolicy.Kind speculativeRetry;
     @VisibleForTesting
-    final List<InetAddressAndPort> destinations;
+    final Collection<InetAddressAndPort> destinations;
     @VisibleForTesting
-    final List<InetAddressAndPort> allEndpoints;
+    final Collection<InetAddressAndPort> allEndpoints;
     @Nullable
     private final DigestResolverDebugResult[] digestsByEndpoint;
 
@@ -60,13 +61,13 @@ final class ReadRepairEvent extends DiagnosticEvent
         SPECULATED_READ
     }
 
-    ReadRepairEvent(ReadRepairEventType type, AbstractReadRepair readRepair, List<InetAddressAndPort> destinations,
-                    List<InetAddressAndPort> allEndpoints, DigestResolver digestResolver)
+    ReadRepairEvent(ReadRepairEventType type, AbstractReadRepair readRepair, Collection<InetAddressAndPort> destinations,
+                    Collection<InetAddressAndPort> allEndpoints, DigestResolver digestResolver)
     {
         this.keyspace = readRepair.cfs.keyspace;
         this.tableName = readRepair.cfs.getTableName();
         this.cqlCommand = readRepair.command.toCQLString();
-        this.consistency = readRepair.consistency;
+        this.consistency = readRepair.replicaLayout.consistencyLevel();
         this.speculativeRetry = readRepair.cfs.metadata().params.speculativeRetry.kind();
         this.destinations = destinations;
         this.allEndpoints = allEndpoints;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
index 5945633..28c0e9e 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java
@@ -18,26 +18,25 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.ReplicaLayout;
 
 public enum ReadRepairStrategy implements ReadRepair.Factory
 {
     NONE
     {
-        @Override
-        public ReadRepair create(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+        public <E extends Endpoints<E>, L extends ReplicaLayout<E, L>> ReadRepair<E, L> create(ReadCommand command, L replicaLayout, long queryStartNanoTime)
         {
-            return new ReadOnlyReadRepair(command, queryStartNanoTime, consistency);
+            return new ReadOnlyReadRepair<>(command, replicaLayout, queryStartNanoTime);
         }
     },
 
     BLOCKING
     {
-        @Override
-        public ReadRepair create(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+        public <E extends Endpoints<E>, L extends ReplicaLayout<E, L>> ReadRepair<E, L> create(ReadCommand command, L replicaLayout, long queryStartNanoTime)
         {
-            return new BlockingReadRepair(command, queryStartNanoTime, consistency);
+            return new BlockingReadRepair<>(command, replicaLayout, queryStartNanoTime);
         }
     };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/RowIteratorMergeListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/RowIteratorMergeListener.java b/src/java/org/apache/cassandra/service/reads/repair/RowIteratorMergeListener.java
index cb6707d..b0c019a 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/RowIteratorMergeListener.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/RowIteratorMergeListener.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.service.reads.repair;
 import java.util.Arrays;
 import java.util.Map;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
 import org.apache.cassandra.db.Clustering;
@@ -43,7 +44,9 @@ import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.RowDiffListener;
 import org.apache.cassandra.db.rows.Rows;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.schema.ColumnMetadata;
 
 public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeListener
@@ -51,14 +54,14 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
     private final DecoratedKey partitionKey;
     private final RegularAndStaticColumns columns;
     private final boolean isReversed;
-    private final InetAddressAndPort[] sources;
     private final ReadCommand command;
     private final ConsistencyLevel consistency;
 
     private final PartitionUpdate.Builder[] repairs;
-
+    private final Replica[] sources;
     private final Row.Builder[] currentRows;
     private final RowDiffListener diffListener;
+    private final ReplicaLayout layout;
 
     // The partition level deletion for the merge row.
     private DeletionTime partitionLevelDeletion;
@@ -71,16 +74,21 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
 
     private final ReadRepair readRepair;
 
-    public RowIteratorMergeListener(DecoratedKey partitionKey, RegularAndStaticColumns columns, boolean isReversed, InetAddressAndPort[] sources, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair)
+    public RowIteratorMergeListener(DecoratedKey partitionKey, RegularAndStaticColumns columns, boolean isReversed, ReplicaLayout layout, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair)
     {
         this.partitionKey = partitionKey;
         this.columns = columns;
         this.isReversed = isReversed;
-        this.sources = sources;
-        repairs = new PartitionUpdate.Builder[sources.length];
-        currentRows = new Row.Builder[sources.length];
-        sourceDeletionTime = new DeletionTime[sources.length];
-        markerToRepair = new ClusteringBound[sources.length];
+        Endpoints<?> sources = layout.selected();
+        this.sources = new Replica[sources.size()];
+        for (int i = 0; i < sources.size(); i++)
+            this.sources[i] = sources.get(i);
+
+        this.layout = layout;
+        repairs = new PartitionUpdate.Builder[sources.size()];
+        currentRows = new Row.Builder[sources.size()];
+        sourceDeletionTime = new DeletionTime[sources.size()];
+        markerToRepair = new ClusteringBound[sources.size()];
         this.command = command;
         this.consistency = consistency;
         this.readRepair = readRepair;
@@ -89,25 +97,25 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
         {
             public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
             {
-                if (merged != null && !merged.equals(original))
+                if (merged != null && !merged.equals(original) && !isTransient(i))
                     currentRow(i, clustering).addPrimaryKeyLivenessInfo(merged);
             }
 
             public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
             {
-                if (merged != null && !merged.equals(original))
+                if (merged != null && !merged.equals(original) && !isTransient(i))
                     currentRow(i, clustering).addRowDeletion(merged);
             }
 
             public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original)
             {
-                if (merged != null && !merged.equals(original))
+                if (merged != null && !merged.equals(original) && !isTransient(i))
                     currentRow(i, clustering).addComplexDeletion(column, merged);
             }
 
             public void onCell(int i, Clustering clustering, Cell merged, Cell original)
             {
-                if (merged != null && !merged.equals(original) && isQueried(merged))
+                if (merged != null && !merged.equals(original) && isQueried(merged) && !isTransient(i))
                     currentRow(i, clustering).addCell(merged);
             }
 
@@ -126,6 +134,11 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
         };
     }
 
+    private boolean isTransient(int i)
+    {
+        return sources[i].isTransient();
+    }
+
     private PartitionUpdate.Builder update(int i)
     {
         if (repairs[i] == null)
@@ -159,6 +172,9 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
         this.partitionLevelDeletion = mergedDeletion;
         for (int i = 0; i < versions.length; i++)
         {
+            if (isTransient(i))
+                continue;
+
             if (mergedDeletion.supersedes(versions[i]))
                 update(i).addPartitionDeletion(mergedDeletion);
         }
@@ -193,6 +209,9 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
 
         for (int i = 0; i < versions.length; i++)
         {
+            if (isTransient(i))
+                continue;
+
             RangeTombstoneMarker marker = versions[i];
 
             // Update what the source now thinks is the current deletion
@@ -245,12 +264,12 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
                     if (!marker.isBoundary() && marker.isOpen(isReversed)) // (1)
                     {
                         assert currentDeletion.equals(marker.openDeletionTime(isReversed))
-                            : String.format("currentDeletion=%s, marker=%s", currentDeletion, marker.toString(command.metadata()));
+                        : String.format("currentDeletion=%s, marker=%s", currentDeletion, marker.toString(command.metadata()));
                     }
                     else // (2)
                     {
                         assert marker.isClose(isReversed) && currentDeletion.equals(marker.closeDeletionTime(isReversed))
-                            : String.format("currentDeletion=%s, marker=%s", currentDeletion, marker.toString(command.metadata()));
+                        : String.format("currentDeletion=%s, marker=%s", currentDeletion, marker.toString(command.metadata()));
                     }
 
                     // and so unless it's a boundary whose opening deletion time is still equal to the current
@@ -306,13 +325,14 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
 
     public void close()
     {
-        Map<InetAddressAndPort, Mutation> mutations = null;
+        Map<Replica, Mutation> mutations = null;
         for (int i = 0; i < repairs.length; i++)
         {
             if (repairs[i] == null)
                 continue;
 
-            Mutation mutation = BlockingReadRepairs.createRepairMutation(repairs[i].build(), consistency, sources[i], false);
+            Preconditions.checkState(!isTransient(i), "cannot read repair transient replicas");
+            Mutation mutation = BlockingReadRepairs.createRepairMutation(repairs[i].build(), consistency, sources[i].endpoint(), false);
             if (mutation == null)
                 continue;
 
@@ -324,7 +344,7 @@ public class RowIteratorMergeListener implements UnfilteredRowIterators.MergeLis
 
         if (mutations != null)
         {
-            readRepair.repairPartition(partitionKey, mutations, sources);
+            readRepair.repairPartition(partitionKey, mutations, layout);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
index 609d2a0..38c25dc 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
 
 import javax.annotation.Nullable;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,8 +44,10 @@ public class DefaultConnectionFactory implements StreamConnectionFactory
 
     private static final int DEFAULT_CHANNEL_BUFFER_SIZE = 1 << 22;
 
-    private static final long MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30);
-    private static final int MAX_CONNECT_ATTEMPTS = 3;
+    @VisibleForTesting
+    public static long MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30);
+    @VisibleForTesting
+    public static int MAX_CONNECT_ATTEMPTS = 3;
 
     @Override
     public Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/streaming/StreamPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
index b56f165..2f6deb5 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -19,11 +19,14 @@ package org.apache.cassandra.streaming;
 
 import java.util.*;
 
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.utils.UUIDGen;
 
+import static com.google.common.collect.Iterables.all;
 import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
 
 /**
@@ -69,12 +72,13 @@ public class StreamPlan
      *
      * @param from endpoint address to fetch data from.
      * @param keyspace name of keyspace
-     * @param ranges ranges to fetch
+     * @param fullRanges ranges to fetch that from provides the full version of
+     * @param transientRanges ranges to fetch that from provides only transient data of
      * @return this object for chaining
      */
-    public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Collection<Range<Token>> ranges)
+    public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, RangesAtEndpoint fullRanges, RangesAtEndpoint transientRanges)
     {
-        return requestRanges(from, keyspace, ranges, EMPTY_COLUMN_FAMILIES);
+        return requestRanges(from, keyspace, fullRanges, transientRanges, EMPTY_COLUMN_FAMILIES);
     }
 
     /**
@@ -82,14 +86,20 @@ public class StreamPlan
      *
      * @param from endpoint address to fetch data from.
      * @param keyspace name of keyspace
-     * @param ranges ranges to fetch
+     * @param fullRanges ranges to fetch that from provides the full data for
+     * @param transientRanges ranges to fetch that from provides only transient data for
      * @param columnFamilies specific column families
      * @return this object for chaining
      */
-    public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    public StreamPlan requestRanges(InetAddressAndPort from, String keyspace, RangesAtEndpoint fullRanges, RangesAtEndpoint transientRanges, String... columnFamilies)
     {
+        //It should either be a dummy address for repair or if it's a bootstrap/move/rebuild it should be this node
+        assert all(fullRanges, Replica::isLocal) || all(fullRanges, range -> range.endpoint().getHostAddress(true).equals("0.0.0.0:0")) :
+             fullRanges.toString();
+        assert all(transientRanges, Replica::isLocal) || all(transientRanges, range -> range.endpoint().getHostAddress(true).equals("0.0.0.0:0")) :
+        transientRanges.toString();
         StreamSession session = coordinator.getOrCreateNextSession(from);
-        session.addStreamRequest(keyspace, ranges, Arrays.asList(columnFamilies));
+        session.addStreamRequest(keyspace, fullRanges, transientRanges, Arrays.asList(columnFamilies));
         return this;
     }
 
@@ -98,14 +108,14 @@ public class StreamPlan
      *
      * @param to endpoint address of receiver
      * @param keyspace name of keyspace
-     * @param ranges ranges to send
+     * @param replicas ranges to send
      * @param columnFamilies specific column families
      * @return this object for chaining
      */
-    public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    public StreamPlan transferRanges(InetAddressAndPort to, String keyspace, RangesAtEndpoint replicas, String... columnFamilies)
     {
         StreamSession session = coordinator.getOrCreateNextSession(to);
-        session.addTransferRanges(keyspace, ranges, Arrays.asList(columnFamilies), flushBeforeTransfer);
+        session.addTransferRanges(keyspace, replicas, Arrays.asList(columnFamilies), flushBeforeTransfer);
         return this;
     }
 
@@ -182,4 +192,10 @@ public class StreamPlan
     {
         return flushBeforeTransfer;
     }
+
+    @VisibleForTesting
+    public StreamCoordinator getCoordinator()
+    {
+        return coordinator;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/streaming/StreamRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequest.java b/src/java/org/apache/cassandra/streaming/StreamRequest.java
index 4a3761e..f37268f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequest.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequest.java
@@ -29,6 +29,10 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
 
 public class StreamRequest
@@ -36,12 +40,23 @@ public class StreamRequest
     public static final IVersionedSerializer<StreamRequest> serializer = new StreamRequestSerializer();
 
     public final String keyspace;
-    public final Collection<Range<Token>> ranges;
+    //Full replicas and transient replicas are split based on the transient status of the remote we are fetching
+    //from. We preserve this distinction so on completion we can log to a system table whether we got the data transiently
+    //or fully from some remote. This is an important distinction for resumable bootstrap. The Replicas in these collections
+    //are local replicas (or dummy if this is triggered by repair) and don't encode the necessary information about
+    //what the remote provided.
+    public final RangesAtEndpoint full;
+    public final RangesAtEndpoint transientReplicas;
     public final Collection<String> columnFamilies = new HashSet<>();
-    public StreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
+
+    public StreamRequest(String keyspace, RangesAtEndpoint full, RangesAtEndpoint transientReplicas, Collection<String> columnFamilies)
     {
         this.keyspace = keyspace;
-        this.ranges = ranges;
+        if (!full.endpoint().equals(transientReplicas.endpoint()))
+            throw new IllegalStateException("Mismatching endpoints: " + full + ", " + transientReplicas);
+
+        this.full = full;
+        this.transientReplicas = transientReplicas;
         this.columnFamilies.addAll(columnFamilies);
     }
 
@@ -50,49 +65,82 @@ public class StreamRequest
         public void serialize(StreamRequest request, DataOutputPlus out, int version) throws IOException
         {
             out.writeUTF(request.keyspace);
-            out.writeInt(request.ranges.size());
-            for (Range<Token> range : request.ranges)
-            {
-                MessagingService.validatePartitioner(range);
-                Token.serializer.serialize(range.left, out, version);
-                Token.serializer.serialize(range.right, out, version);
-            }
             out.writeInt(request.columnFamilies.size());
+
+            CompactEndpointSerializationHelper.streamingInstance.serialize(request.full.endpoint(), out, version);
+            serializeReplicas(request.full, out, version);
+            serializeReplicas(request.transientReplicas, out, version);
             for (String cf : request.columnFamilies)
                 out.writeUTF(cf);
         }
 
-        public StreamRequest deserialize(DataInputPlus in, int version) throws IOException
+        private void serializeReplicas(RangesAtEndpoint replicas, DataOutputPlus out, int version) throws IOException
         {
-            String keyspace = in.readUTF();
-            int rangeCount = in.readInt();
-            List<Range<Token>> ranges = new ArrayList<>(rangeCount);
-            for (int i = 0; i < rangeCount; i++)
+            out.writeInt(replicas.size());
+
+            for (Replica replica : replicas)
             {
-                Token left = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
-                Token right = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
-                ranges.add(new Range<>(left, right));
+                MessagingService.validatePartitioner(replica.range());
+                Token.serializer.serialize(replica.range().left, out, version);
+                Token.serializer.serialize(replica.range().right, out, version);
             }
+        }
+
+        public StreamRequest deserialize(DataInputPlus in, int version) throws IOException
+        {
+            String keyspace = in.readUTF();
             int cfCount = in.readInt();
+            InetAddressAndPort endpoint = CompactEndpointSerializationHelper.streamingInstance.deserialize(in, version);
+
+            RangesAtEndpoint full = deserializeReplicas(in, version, endpoint, true);
+            RangesAtEndpoint transientReplicas = deserializeReplicas(in, version, endpoint, false);
             List<String> columnFamilies = new ArrayList<>(cfCount);
             for (int i = 0; i < cfCount; i++)
                 columnFamilies.add(in.readUTF());
-            return new StreamRequest(keyspace, ranges, columnFamilies);
+            return new StreamRequest(keyspace, full, transientReplicas, columnFamilies);
         }
 
-        public long serializedSize(StreamRequest request, int version)
+        RangesAtEndpoint deserializeReplicas(DataInputPlus in, int version, InetAddressAndPort endpoint, boolean isFull) throws IOException
         {
-            int size = TypeSizes.sizeof(request.keyspace);
-            size += TypeSizes.sizeof(request.ranges.size());
-            for (Range<Token> range : request.ranges)
+            int replicaCount = in.readInt();
+
+            RangesAtEndpoint.Builder replicas = RangesAtEndpoint.builder(endpoint, replicaCount);
+            for (int i = 0; i < replicaCount; i++)
             {
-                size += Token.serializer.serializedSize(range.left, version);
-                size += Token.serializer.serializedSize(range.right, version);
+                //TODO, super need to review the usage of streaming vs not streaming endpoint serialization helper
+                //to make sure I'm not using the wrong one some of the time, like do repair messages use the
+                //streaming version?
+                Token left = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
+                Token right = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
+                replicas.add(new Replica(endpoint, new Range<>(left, right), isFull));
             }
+            return replicas.build();
+        }
+
+        public long serializedSize(StreamRequest request, int version)
+        {
+            int size = TypeSizes.sizeof(request.keyspace);
             size += TypeSizes.sizeof(request.columnFamilies.size());
+            size += CompactEndpointSerializationHelper.streamingInstance.serializedSize(request.full.endpoint(), version);
+            size += replicasSerializedSize(request.transientReplicas, version);
+            size += replicasSerializedSize(request.full, version);
             for (String cf : request.columnFamilies)
                 size += TypeSizes.sizeof(cf);
             return size;
         }
+
+        private long replicasSerializedSize(RangesAtEndpoint replicas, int version)
+        {
+            long size = 0;
+            size += TypeSizes.sizeof(replicas.size());
+
+            for (Replica replica : replicas)
+            {
+                size += Token.serializer.serializedSize(replica.range().left, version);
+                size += Token.serializer.serializedSize(replica.range().right, version);
+            }
+            return size;
+        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 393cd24..ec80772 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -29,6 +29,7 @@ import com.google.common.util.concurrent.Futures;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,6 +41,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.metrics.StreamingMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.async.OutboundConnectionIdentifier;
@@ -49,6 +51,8 @@ import org.apache.cassandra.streaming.messages.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
+import static com.google.common.collect.Iterables.all;
+
 /**
  * Handles the streaming a one or more streams to and from a specific remote node.
  *
@@ -243,7 +247,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
 
     public StreamReceiver getAggregator(TableId tableId)
     {
-        assert receivers.containsKey(tableId);
+        assert receivers.containsKey(tableId) : "Missing tableId " + tableId;
         return receivers.get(tableId).getReceiver();
     }
 
@@ -297,38 +301,52 @@ public class StreamSession implements IEndpointStateChangeSubscriber
      * Request data fetch task to this session.
      *
      * @param keyspace Requesting keyspace
-     * @param ranges Ranges to retrieve data
+     * @param fullRanges Ranges to retrieve data that will return full data from the source
+     * @param transientRanges Ranges to retrieve data that will return transient data from the source
      * @param columnFamilies ColumnFamily names. Can be empty if requesting all CF under the keyspace.
      */
-    public void addStreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
+    public void addStreamRequest(String keyspace, RangesAtEndpoint fullRanges, RangesAtEndpoint transientRanges, Collection<String> columnFamilies)
     {
-        requests.add(new StreamRequest(keyspace, ranges, columnFamilies));
+        //It should either be a dummy address for repair or if it's a bootstrap/move/rebuild it should be this node
+        assert all(fullRanges, Replica::isLocal) || all(fullRanges, range -> range.endpoint().getHostAddress(true).equals("0.0.0.0:0")) : fullRanges.toString();
+        assert all(transientRanges, Replica::isLocal) || all(transientRanges, range -> range.endpoint().getHostAddress(true).equals("0.0.0.0:0")) : transientRanges.toString();
+        requests.add(new StreamRequest(keyspace, fullRanges, transientRanges, columnFamilies));
     }
 
     /**
      * Set up transfer for specific keyspace/ranges/CFs
      *
      * @param keyspace Transfer keyspace
-     * @param ranges Transfer ranges
+     * @param replicas Transfer ranges
      * @param columnFamilies Transfer ColumnFamilies
      * @param flushTables flush tables?
      */
-    synchronized void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables)
+    synchronized void addTransferRanges(String keyspace, RangesAtEndpoint replicas, Collection<String> columnFamilies, boolean flushTables)
     {
         failIfFinished();
         Collection<ColumnFamilyStore> stores = getColumnFamilyStores(keyspace, columnFamilies);
         if (flushTables)
             flushSSTables(stores);
 
-        List<Range<Token>> normalizedRanges = Range.normalize(ranges);
-        List<OutgoingStream> streams = getOutgoingStreamsForRanges(normalizedRanges, stores, pendingRepair, previewKind);
+        //Was it safe to remove this normalize, sorting seems not to matter, merging? Maybe we should have?
+        //Do we need to unwrap here also or is that just making it worse?
+        //Range and if it's transient
+        RangesAtEndpoint.Builder unwrappedRanges = RangesAtEndpoint.builder(replicas.endpoint(), replicas.size());
+        for (Replica replica : replicas)
+        {
+            for (Range<Token> unwrapped : replica.range().unwrap())
+            {
+                unwrappedRanges.add(new Replica(replica.endpoint(), unwrapped, replica.isFull()));
+            }
+        }
+        List<OutgoingStream> streams = getOutgoingStreamsForRanges(unwrappedRanges.build(), stores, pendingRepair, previewKind);
         addTransferStreams(streams);
         Set<Range<Token>> toBeUpdated = transferredRangesPerKeyspace.get(keyspace);
         if (toBeUpdated == null)
         {
             toBeUpdated = new HashSet<>();
         }
-        toBeUpdated.addAll(ranges);
+        toBeUpdated.addAll(replicas.ranges());
         transferredRangesPerKeyspace.put(keyspace, toBeUpdated);
     }
 
@@ -355,14 +373,14 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     }
 
     @VisibleForTesting
-    public List<OutgoingStream> getOutgoingStreamsForRanges(Collection<Range<Token>> ranges, Collection<ColumnFamilyStore> stores, UUID pendingRepair, PreviewKind previewKind)
+    public List<OutgoingStream> getOutgoingStreamsForRanges(RangesAtEndpoint replicas, Collection<ColumnFamilyStore> stores, UUID pendingRepair, PreviewKind previewKind)
     {
         List<OutgoingStream> streams = new ArrayList<>();
         try
         {
             for (ColumnFamilyStore cfs: stores)
             {
-                streams.addAll(cfs.getStreamManager().createOutgoingStreams(this, ranges, pendingRepair, previewKind));
+                streams.addAll(cfs.getStreamManager().createOutgoingStreams(this, replicas, pendingRepair, previewKind));
             }
         }
         catch (Throwable t)
@@ -561,7 +579,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
 
         for (StreamRequest request : requests)
-            addTransferRanges(request.keyspace, request.ranges, request.columnFamilies, true); // always flush on stream request
+            addTransferRanges(request.keyspace, RangesAtEndpoint.concat(request.full, request.transientReplicas), request.columnFamilies, true); // always flush on stream request
         for (StreamSummary summary : summaries)
             prepareReceiving(summary);
 
@@ -812,4 +830,16 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         }
         maybeCompleted();
     }
+
+    @VisibleForTesting
+    public int getNumRequests()
+    {
+        return requests.size();
+    }
+
+    @VisibleForTesting
+    public int getNumTransfers()
+    {
+        return transferredRangesPerKeyspace.size();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/streaming/TableStreamManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/TableStreamManager.java b/src/java/org/apache/cassandra/streaming/TableStreamManager.java
index 11512e9..d97fabc 100644
--- a/src/java/org/apache/cassandra/streaming/TableStreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/TableStreamManager.java
@@ -21,8 +21,7 @@ package org.apache.cassandra.streaming;
 import java.util.Collection;
 import java.util.UUID;
 
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.streaming.messages.StreamMessageHeader;
 
 /**
@@ -46,12 +45,12 @@ public interface TableStreamManager
 
     /**
      * Returns a collection of {@link OutgoingStream}s that contains the data selected by the
-     * given ranges, pendingRepair, and preview.
+     * given replicas, pendingRepair, and preview.
      *
      * There aren't any requirements on how data is divided between the outgoing streams
      */
     Collection<OutgoingStream> createOutgoingStreams(StreamSession session,
-                                                     Collection<Range<Token>> ranges,
+                                                     RangesAtEndpoint replicas,
                                                      UUID pendingRepair,
                                                      PreviewKind previewKind);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 03b8af0..54187d1 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -816,6 +816,11 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.getNaturalEndpoints(keyspace, cf, key);
     }
 
+    public List<String> getReplicas(String keyspace, String cf, String key)
+    {
+        return ssProxy.getReplicas(keyspace, cf, key);
+    }
+
     public List<String> getSSTables(String keyspace, String cf, String key, boolean hexFormat)
     {
         ColumnFamilyStoreMBean cfsProxy = getCfsProxy(keyspace, cf);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index c2193d4..1d09b0f 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -190,6 +190,8 @@ public class NodeTool
                 ReloadSslCertificates.class,
                 EnableAuditLog.class,
                 DisableAuditLog.class,
+                GetReplicas.class,
+                DisableAuditLog.class,
                 EnableOldProtocolVersions.class,
                 DisableOldProtocolVersions.class
         );

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
index 8056ff8..31d80fa 100644
--- a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
@@ -88,11 +88,11 @@ public class SSTableRepairedAtSetter
             if (setIsRepaired)
             {
                 FileTime f = Files.getLastModifiedTime(new File(descriptor.filenameFor(Component.DATA)).toPath());
-                descriptor.getMetadataSerializer().mutateRepaired(descriptor, f.toMillis(), null);
+                descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, f.toMillis(), null, false);
             }
             else
             {
-                descriptor.getMetadataSerializer().mutateRepaired(descriptor, 0, null);
+                descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, 0, null, false);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/tools/nodetool/GetReplicas.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetReplicas.java b/src/java/org/apache/cassandra/tools/nodetool/GetReplicas.java
new file mode 100644
index 0000000..4c401fc
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetReplicas.java
@@ -0,0 +1,47 @@
+/*
+ * 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.tools.nodetool;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@Command(name = "getreplicas", description = "Print replicas for a given key")
+public class GetReplicas extends NodeTool.NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table> <key>", description = "The keyspace, the table, and the partition key for which we need to find replicas")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "getreplicas requires keyspace, table and partition key arguments");
+        String ks = args.get(0);
+        String table = args.get(1);
+        String key = args.get(2);
+
+        System.out.println(probe.getReplicas(ks, table, key));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/tracing/TraceState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java b/src/java/org/apache/cassandra/tracing/TraceState.java
index a53846c..1e0813c 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -161,7 +161,7 @@ public abstract class TraceState implements ProgressEventNotifier
         trace(MessageFormatter.format(format, arg1, arg2).getMessage());
     }
 
-    public void trace(String format, Object[] args)
+    public void trace(String format, Object... args)
     {
         trace(MessageFormatter.arrayFormat(format, args).getMessage());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 84af41c..8e0b19f 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -68,7 +68,7 @@ public class ErrorMessage extends Message.Response
                         ConsistencyLevel cl = CBUtil.readConsistencyLevel(body);
                         int required = body.readInt();
                         int alive = body.readInt();
-                        te = new UnavailableException(cl, required, alive);
+                        te = UnavailableException.create(cl, required, alive);
                     }
                     break;
                 case OVERLOADED:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/utils/Pair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Pair.java b/src/java/org/apache/cassandra/utils/Pair.java
index ea8b8fc..cb09529 100644
--- a/src/java/org/apache/cassandra/utils/Pair.java
+++ b/src/java/org/apache/cassandra/utils/Pair.java
@@ -53,6 +53,18 @@ public class Pair<T1, T2>
         return "(" + left + "," + right + ")";
     }
 
+    //For functional interfaces
+    public T1 left()
+    {
+        return left;
+    }
+
+    //For functional interfaces
+    public T2 right()
+    {
+        return right;
+    }
+
     public static <X, Y> Pair<X, Y> create(X x, Y y)
     {
         return new Pair<X, Y>(x, y);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java b/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
index e80faca..0c097a6 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
@@ -18,6 +18,8 @@
 */
 package org.apache.cassandra.utils.concurrent;
 
+import java.util.AbstractCollection;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
@@ -27,7 +29,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
  *
  * @param <E>
  */
-public class Accumulator<E> implements Iterable<E>
+public class Accumulator<E>
 {
     private volatile int nextIndex;
     private volatile int presentCount;
@@ -105,7 +107,7 @@ public class Accumulator<E> implements Iterable<E>
         return values.length;
     }
 
-    public Iterator<E> iterator()
+    private Iterator<E> iterator(int count)
     {
         return new Iterator<E>()
         {
@@ -113,7 +115,7 @@ public class Accumulator<E> implements Iterable<E>
 
             public boolean hasNext()
             {
-                return p < presentCount;
+                return p < count;
             }
 
             public E next()
@@ -135,4 +137,23 @@ public class Accumulator<E> implements Iterable<E>
             throw new IndexOutOfBoundsException();
         return (E) values[i];
     }
+
+    public Collection<E> snapshot()
+    {
+        int count = presentCount;
+        return new AbstractCollection<E>()
+        {
+            @Override
+            public Iterator<E> iterator()
+            {
+                return Accumulator.this.iterator(count);
+            }
+
+            @Override
+            public int size()
+            {
+                return count;
+            }
+        };
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db
index ceaa5a3..8fad34f 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db
index 6968720..ae35335 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32
index f1c192b..8a92f3c 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32
@@ -1 +1 @@
-4004129384
\ No newline at end of file
+2977407251
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db
index af16195..d50fdeb 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db
index 970e385..7341864 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt
index bb800f8..b03b283 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt
@@ -1,8 +1,8 @@
-Digest.crc32
 Filter.db
-CompressionInfo.db
+Digest.crc32
 Index.db
-Summary.db
-Data.db
 TOC.txt
+Summary.db
 Statistics.db
+CompressionInfo.db
+Data.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db
index f5ad4d0..f0a1cfb 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db
index 7217716..b487fe8 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32
index 4f1391a..ca286e0 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32
@@ -1 +1 @@
-4072239034
\ No newline at end of file
+2759187708
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db
index 6dd3da6..c981a22 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db
index 3a0e63f..33fccc9 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt
index bb800f8..b03b283 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt
@@ -1,8 +1,8 @@
-Digest.crc32
 Filter.db
-CompressionInfo.db
+Digest.crc32
 Index.db
-Summary.db
-Data.db
 TOC.txt
+Summary.db
 Statistics.db
+CompressionInfo.db
+Data.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db
index c665dfb..11219d0 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32
index c6c24a7..985d6dc 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32
@@ -1 +1 @@
-3772296151
\ No newline at end of file
+462858821
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db
index 6741430..3c68ac5 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt
index bb800f8..b03b283 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt
@@ -1,8 +1,8 @@
-Digest.crc32
 Filter.db
-CompressionInfo.db
+Digest.crc32
 Index.db
-Summary.db
-Data.db
 TOC.txt
+Summary.db
 Statistics.db
+CompressionInfo.db
+Data.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db
index d9fe576..620cdf2 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32
index de7baed..bc5f671 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32
@@ -1 +1 @@
-4035692752
\ No newline at end of file
+3987542254
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db
index e9556d1..689bec8 100644
Binary files a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db and b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt
index bb800f8..b03b283 100644
--- a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt
+++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt
@@ -1,8 +1,8 @@
-Digest.crc32
 Filter.db
-CompressionInfo.db
+Digest.crc32
 Index.db
-Summary.db
-Data.db
 TOC.txt
+Summary.db
 Statistics.db
+CompressionInfo.db
+Data.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index a5025a3..94a3bd3 100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@ -54,19 +54,21 @@ public class DynamicEndpointSnitchLongTest
             DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
             InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
 
-            List<InetAddressAndPort> hosts = new ArrayList<>();
+            EndpointsForRange.Builder replicasBuilder = EndpointsForRange.builder(ReplicaUtils.FULL_RANGE);
             // We want a big list of hosts so  sorting takes time, making it much more likely to reproduce the
             // problem we're looking for.
             for (int i = 0; i < 100; i++)
                 for (int j = 0; j < 256; j++)
-                    hosts.add(InetAddressAndPort.getByAddress(new byte[]{ 127, 0, (byte)i, (byte)j}));
+                    replicasBuilder.add(ReplicaUtils.full(InetAddressAndPort.getByAddress(new byte[]{ 127, 0, (byte)i, (byte)j})));
 
-            ScoreUpdater updater = new ScoreUpdater(dsnitch, hosts);
+            EndpointsForRange replicas = replicasBuilder.build();
+
+            ScoreUpdater updater = new ScoreUpdater(dsnitch, replicas);
             updater.start();
 
-            List<InetAddressAndPort> result = null;
+            EndpointsForRange result = replicas;
             for (int i = 0; i < ITERATIONS; i++)
-                result = dsnitch.getSortedListByProximity(self, hosts);
+                result = dsnitch.sortedByProximity(self, result);
 
             updater.stopped = true;
             updater.join();
@@ -84,10 +86,10 @@ public class DynamicEndpointSnitchLongTest
         public volatile boolean stopped;
 
         private final DynamicEndpointSnitch dsnitch;
-        private final List<InetAddressAndPort> hosts;
+        private final EndpointsForRange hosts;
         private final Random random = new Random();
 
-        public ScoreUpdater(DynamicEndpointSnitch dsnitch, List<InetAddressAndPort> hosts)
+        public ScoreUpdater(DynamicEndpointSnitch dsnitch, EndpointsForRange hosts)
         {
             this.dsnitch = dsnitch;
             this.hosts = hosts;
@@ -97,9 +99,9 @@ public class DynamicEndpointSnitchLongTest
         {
             while (!stopped)
             {
-                InetAddressAndPort host = hosts.get(random.nextInt(hosts.size()));
+                Replica host = hosts.get(random.nextInt(hosts.size()));
                 int score = random.nextInt(SCORE_RANGE);
-                dsnitch.receiveTiming(host, score);
+                dsnitch.receiveTiming(host.endpoint(), score);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
index 01e67f0..e37045a 100644
--- a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
+++ b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
@@ -33,11 +33,10 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.CQLSSTableWriter;
 import org.apache.cassandra.io.sstable.SSTableLoader;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadataRef;
@@ -121,8 +120,8 @@ public class LongStreamingTest
             private String ks;
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
-                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
+                for (Replica range : StorageService.instance.getLocalReplicas(KS))
+                    addRangeForEndpoint(range.range(), FBUtilities.getBroadcastAddressAndPort());
 
                 this.ks = keyspace;
             }
@@ -148,8 +147,8 @@ public class LongStreamingTest
             private String ks;
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
-                    addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
+                for (Replica range : StorageService.instance.getLocalReplicas(KS))
+                    addRangeForEndpoint(range.range(), FBUtilities.getBroadcastAddressAndPort());
 
                 this.ks = keyspace;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
index 68cfd7e..73a2b71 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
@@ -21,18 +21,22 @@
 package org.apache.cassandra.test.microbench;
 
 import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.PendingRangeMaps;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.openjdk.jmh.annotations.*;
 import org.openjdk.jmh.infra.Blackhole;
 
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
@@ -50,7 +54,7 @@ public class PendingRangesBench
     PendingRangeMaps pendingRangeMaps;
     int maxToken = 256 * 100;
 
-    Multimap<Range<Token>, InetAddressAndPort> oldPendingRanges;
+    Multimap<Range<Token>, Replica> oldPendingRanges;
 
     private Range<Token> genRange(String left, String right)
     {
@@ -63,15 +67,17 @@ public class PendingRangesBench
         pendingRangeMaps = new PendingRangeMaps();
         oldPendingRanges = HashMultimap.create();
 
-        InetAddressAndPort[] addresses = { InetAddressAndPort.getByName("127.0.0.1"), InetAddressAndPort.getByName("127.0.0.2")};
+        List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"),
+                                                                InetAddressAndPort.getByName("127.0.0.2"));
 
         for (int i = 0; i < maxToken; i++)
         {
             for (int j = 0; j < ThreadLocalRandom.current().nextInt(2); j ++)
             {
                 Range<Token> range = genRange(Integer.toString(i * 10 + 5), Integer.toString(i * 10 + 15));
-                pendingRangeMaps.addPendingRange(range, addresses[j]);
-                oldPendingRanges.put(range, addresses[j]);
+                Replica replica = Replica.fullReplica(endpoints.get(j), range);
+                pendingRangeMaps.addPendingRange(range, replica);
+                oldPendingRanges.put(range, replica);
             }
         }
 
@@ -79,8 +85,9 @@ public class PendingRangesBench
         for (int j = 0; j < ThreadLocalRandom.current().nextInt(2); j ++)
         {
             Range<Token> range = genRange(Integer.toString(maxToken * 10 + 5), Integer.toString(5));
-            pendingRangeMaps.addPendingRange(range, addresses[j]);
-            oldPendingRanges.put(range, addresses[j]);
+            Replica replica = Replica.fullReplica(endpoints.get(j), range);
+            pendingRangeMaps.addPendingRange(range, replica);
+            oldPendingRanges.put(range, replica);
         }
     }
 
@@ -97,13 +104,13 @@ public class PendingRangesBench
     {
         int randomToken = ThreadLocalRandom.current().nextInt(maxToken * 10 + 5);
         Token searchToken = new RandomPartitioner.BigIntegerToken(Integer.toString(randomToken));
-        Set<InetAddressAndPort> endpoints = new HashSet<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : oldPendingRanges.asMap().entrySet())
+        Set<Replica> replicas = new HashSet<>();
+        for (Map.Entry<Range<Token>, Collection<Replica>> entry : oldPendingRanges.asMap().entrySet())
         {
             if (entry.getKey().contains(searchToken))
-                endpoints.addAll(entry.getValue());
+                replicas.addAll(entry.getValue());
         }
-        bh.consume(endpoints);
+        bh.consume(replicas);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 1201efa..bc2c19c 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -32,9 +32,11 @@ import java.util.function.Supplier;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import org.apache.commons.lang3.StringUtils;
 
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,6 +75,7 @@ import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 public class Util
@@ -446,6 +449,14 @@ public class Util
         }
     }
 
+    public static void consume(UnfilteredPartitionIterator iterator)
+    {
+        while (iterator.hasNext())
+        {
+            consume(iterator.next());
+        }
+    }
+
     public static int size(PartitionIterator iter)
     {
         int size = 0;
@@ -478,6 +489,15 @@ public class Util
             && Iterators.elementsEqual(a, b);
     }
 
+    public static boolean sameContent(RowIterator a, RowIterator b)
+    {
+        return Objects.equals(a.metadata(), b.metadata())
+               && Objects.equals(a.isReverseOrder(), b.isReverseOrder())
+               && Objects.equals(a.partitionKey(), b.partitionKey())
+               && Objects.equals(a.staticRow(), b.staticRow())
+               && Iterators.elementsEqual(a, b);
+    }
+
     public static boolean sameContent(Mutation a, Mutation b)
     {
         if (!a.key().equals(b.key()) || !a.getTableIds().equals(b.getTableIds()))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
index 07ab3dc..782e3b1 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
@@ -104,6 +104,7 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.io.util.DataOutputPlus",
     "org.apache.cassandra.io.util.DiskOptimizationStrategy",
     "org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy",
+    "org.apache.cassandra.locator.Replica",
     "org.apache.cassandra.locator.SimpleSeedProvider",
     "org.apache.cassandra.locator.SeedProvider",
     "org.apache.cassandra.net.BackPressureStrategy",
@@ -134,7 +135,9 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptionsCustomizer",
     "org.apache.cassandra.ConsoleAppenderBeanInfo",
     "org.apache.cassandra.ConsoleAppenderCustomizer",
-    "org.apache.cassandra.locator.InetAddressAndPort"
+    "org.apache.cassandra.locator.InetAddressAndPort",
+    "org.apache.cassandra.cql3.statements.schema.AlterKeyspaceStatement",
+    "org.apache.cassandra.cql3.statements.schema.CreateKeyspaceStatement"
     };
 
     static final Set<String> checkedClasses = new HashSet<>(Arrays.asList(validClasses));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 4a1a365..2fbbc28 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.index.SecondaryIndexManager;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.functions.FunctionName;
@@ -148,7 +149,7 @@ public abstract class CQLTester
         {
             @Override public String getRack(InetAddressAndPort endpoint) { return RACK1; }
             @Override public String getDatacenter(InetAddressAndPort endpoint) { return DATA_CENTER; }
-            @Override public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2) { return 0; }
+            @Override public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
         });
 
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index 184c5ad..37605d6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -37,6 +37,7 @@ import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.schema.*;
@@ -527,7 +528,7 @@ public class CreateTest extends CQLTester
             public String getDatacenter(InetAddressAndPort endpoint) { return "us-east-1"; }
 
             @Override
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2) { return 0; }
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; }
         });
 
         // this forces the dc above to be added to the list of known datacenters (fixes static init problem


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


[18/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
Transient Replication and Cheap Quorums

Patch by Blake Eggleston, Benedict Elliott Smith, Marcus Eriksson, Alex Petrov, Ariel Weisberg; Reviewed by Blake Eggleston, Marcus Eriksson, Benedict Elliott Smith, Alex Petrov, Ariel Weisberg for CASSANDRA-14404

Co-authored-by: Blake Eggleston <bd...@gmail.com>
Co-authored-by: Benedict Elliott Smith <be...@apache.org>
Co-authored-by: Marcus Eriksson <ma...@apache.org>
Co-authored-by: Alex Petrov <ol...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f7431b43
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f7431b43
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f7431b43

Branch: refs/heads/trunk
Commit: f7431b432875e334170ccdb19934d05545d2cebd
Parents: 5b645de
Author: Ariel Weisberg <ar...@weisberg.ws>
Authored: Thu Jul 5 18:10:40 2018 -0400
Committer: Ariel Weisberg <aw...@apple.com>
Committed: Fri Aug 31 21:34:22 2018 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   4 +
 conf/cassandra.yaml                             |   4 +
 doc/source/architecture/dynamo.rst              |  29 +
 doc/source/cql/ddl.rst                          |  14 +-
 ...iver-internal-only-3.12.0.post0-5838e2fd.zip | Bin 0 -> 269418 bytes
 pylib/cqlshlib/cql3handling.py                  |   1 +
 pylib/cqlshlib/cqlshhandling.py                 |   1 +
 pylib/cqlshlib/test/test_cqlsh_completion.py    |   6 +-
 pylib/cqlshlib/test/test_cqlsh_output.py        |   3 +-
 .../cassandra/batchlog/BatchlogManager.java     |  45 +-
 .../org/apache/cassandra/config/Config.java     |   2 +
 .../cassandra/config/DatabaseDescriptor.java    |  35 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  13 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../cql3/statements/BatchUpdatesCollector.java  |   2 +-
 .../cql3/statements/ModificationStatement.java  |   4 +-
 .../statements/SingleTableUpdatesCollector.java |   2 +-
 .../cql3/statements/UpdatesCollector.java       |   5 +-
 .../schema/AlterKeyspaceStatement.java          |  86 +-
 .../statements/schema/AlterTableStatement.java  |   7 +
 .../statements/schema/CreateIndexStatement.java |   5 +
 .../statements/schema/CreateTableStatement.java |   9 +
 .../statements/schema/CreateViewStatement.java  |   5 +
 .../cql3/statements/schema/TableAttributes.java |   3 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  24 +-
 .../apache/cassandra/db/ConsistencyLevel.java   | 211 +++--
 .../cassandra/db/DiskBoundaryManager.java       |  39 +-
 src/java/org/apache/cassandra/db/Memtable.java  |   1 +
 .../cassandra/db/MutationVerbHandler.java       |   5 +-
 .../cassandra/db/PartitionRangeReadCommand.java |  28 +-
 .../org/apache/cassandra/db/ReadCommand.java    |  33 +-
 .../apache/cassandra/db/SSTableImporter.java    |   6 +-
 .../db/SinglePartitionReadCommand.java          |  26 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  98 ++-
 .../cassandra/db/SystemKeyspaceMigrator40.java  |  45 +
 .../org/apache/cassandra/db/TableCQLHelper.java |   1 +
 .../compaction/AbstractCompactionStrategy.java  |   3 +-
 .../db/compaction/AbstractStrategyHolder.java   |   7 +-
 .../db/compaction/CompactionManager.java        | 295 ++++---
 .../db/compaction/CompactionStrategyHolder.java |  34 +-
 .../compaction/CompactionStrategyManager.java   | 108 +--
 .../cassandra/db/compaction/CompactionTask.java |  26 +-
 .../db/compaction/PendingRepairHolder.java      |  42 +-
 .../db/compaction/PendingRepairManager.java     |  45 +-
 .../cassandra/db/compaction/Scrubber.java       |   4 +-
 .../cassandra/db/compaction/Upgrader.java       |  10 +-
 .../cassandra/db/compaction/Verifier.java       |   3 +-
 .../writers/CompactionAwareWriter.java          |   2 +
 .../writers/DefaultCompactionWriter.java        |   1 +
 .../writers/MajorLeveledCompactionWriter.java   |   1 +
 .../writers/MaxSSTableSizeWriter.java           |   1 +
 .../SplittingSizeTieredCompactionWriter.java    |   1 +
 .../db/partitions/PartitionIterators.java       |  12 -
 .../repair/CassandraKeyspaceRepairManager.java  |  10 +-
 .../db/repair/PendingAntiCompaction.java        |  22 +-
 .../db/streaming/CassandraOutgoingFile.java     |  11 +-
 .../db/streaming/CassandraStreamManager.java    |  36 +-
 .../db/streaming/CassandraStreamReader.java     |   2 +-
 .../apache/cassandra/db/view/TableViews.java    |   5 +
 .../apache/cassandra/db/view/ViewBuilder.java   |  19 +-
 .../apache/cassandra/db/view/ViewManager.java   |   2 +-
 .../org/apache/cassandra/db/view/ViewUtils.java |  64 +-
 src/java/org/apache/cassandra/dht/Range.java    |  27 +-
 .../cassandra/dht/RangeFetchMapCalculator.java  |  58 +-
 .../org/apache/cassandra/dht/RangeStreamer.java | 571 ++++++++----
 src/java/org/apache/cassandra/dht/Splitter.java |  95 +-
 .../apache/cassandra/dht/StreamStateStore.java  |  25 +-
 .../ReplicationAwareTokenAllocator.java         |   2 +-
 .../dht/tokenallocator/TokenAllocation.java     |   6 +-
 .../exceptions/UnavailableException.java        |  20 +-
 .../org/apache/cassandra/gms/EndpointState.java |   5 +
 .../apache/cassandra/hints/HintsService.java    |  21 +-
 .../io/sstable/AbstractSSTableSimpleWriter.java |   3 +-
 .../apache/cassandra/io/sstable/SSTable.java    |  13 +
 .../cassandra/io/sstable/SSTableLoader.java     |   2 +-
 .../cassandra/io/sstable/SSTableTxnWriter.java  |  14 +-
 .../io/sstable/SimpleSSTableMultiWriter.java    |   3 +-
 .../sstable/format/RangeAwareSSTableWriter.java |   8 +-
 .../io/sstable/format/SSTableReader.java        |   5 +
 .../io/sstable/format/SSTableWriter.java        |  17 +-
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  17 +-
 .../io/sstable/format/big/BigTableWriter.java   |   3 +-
 .../sstable/metadata/IMetadataSerializer.java   |   4 +-
 .../io/sstable/metadata/MetadataCollector.java  |   8 +-
 .../io/sstable/metadata/MetadataSerializer.java |   4 +-
 .../io/sstable/metadata/StatsMetadata.java      |  52 +-
 .../locator/AbstractEndpointSnitch.java         |  38 +-
 .../locator/AbstractNetworkTopologySnitch.java  |   5 +-
 .../locator/AbstractReplicaCollection.java      | 264 ++++++
 .../locator/AbstractReplicationStrategy.java    | 142 +--
 .../locator/DynamicEndpointSnitch.java          |  67 +-
 .../org/apache/cassandra/locator/Ec2Snitch.java |   2 +-
 .../org/apache/cassandra/locator/Endpoints.java | 157 ++++
 .../cassandra/locator/EndpointsByRange.java     |  63 ++
 .../cassandra/locator/EndpointsByReplica.java   |  61 ++
 .../cassandra/locator/EndpointsForRange.java    | 188 ++++
 .../cassandra/locator/EndpointsForToken.java    | 172 ++++
 .../cassandra/locator/IEndpointSnitch.java      |  18 +-
 .../cassandra/locator/InetAddressAndPort.java   |   5 +-
 .../apache/cassandra/locator/LocalStrategy.java |  29 +-
 .../locator/NetworkTopologyStrategy.java        |  87 +-
 .../locator/OldNetworkTopologyStrategy.java     |  40 +-
 .../cassandra/locator/PendingRangeMaps.java     | 161 ++--
 .../cassandra/locator/RangesAtEndpoint.java     | 313 +++++++
 .../cassandra/locator/RangesByEndpoint.java     |  54 ++
 .../org/apache/cassandra/locator/Replica.java   | 196 +++++
 .../cassandra/locator/ReplicaCollection.java    | 160 ++++
 .../apache/cassandra/locator/ReplicaLayout.java | 381 ++++++++
 .../cassandra/locator/ReplicaMultimap.java      | 127 +++
 .../org/apache/cassandra/locator/Replicas.java  |  83 ++
 .../cassandra/locator/ReplicationFactor.java    | 130 +++
 .../apache/cassandra/locator/SimpleSnitch.java  |   8 +-
 .../cassandra/locator/SimpleStrategy.java       |  37 +-
 .../cassandra/locator/SystemReplicas.java       |  62 ++
 .../apache/cassandra/locator/TokenMetadata.java | 102 ++-
 .../cassandra/metrics/KeyspaceMetrics.java      |  43 +-
 .../cassandra/metrics/ReadRepairMetrics.java    |   1 +
 .../apache/cassandra/metrics/TableMetrics.java  |  17 +-
 .../apache/cassandra/net/IAsyncCallback.java    |  11 +-
 .../apache/cassandra/net/MessagingService.java  |  38 +-
 .../apache/cassandra/net/WriteCallbackInfo.java |  15 +-
 .../cassandra/repair/AbstractSyncTask.java      |  31 +
 .../repair/AsymmetricLocalSyncTask.java         |   7 +-
 .../repair/AsymmetricRemoteSyncTask.java        |   6 +
 .../cassandra/repair/AsymmetricSyncTask.java    |  10 +-
 .../apache/cassandra/repair/CommonRange.java    |  82 ++
 .../cassandra/repair/KeyspaceRepairManager.java |   8 +-
 .../apache/cassandra/repair/LocalSyncTask.java  | 135 ---
 .../apache/cassandra/repair/RemoteSyncTask.java |  74 --
 .../org/apache/cassandra/repair/RepairJob.java  |  42 +-
 .../apache/cassandra/repair/RepairRunnable.java |  87 +-
 .../apache/cassandra/repair/RepairSession.java  |  57 +-
 .../cassandra/repair/StreamingRepairTask.java   |   8 +-
 .../repair/SymmetricLocalSyncTask.java          | 142 +++
 .../repair/SymmetricRemoteSyncTask.java         |  84 ++
 .../cassandra/repair/SymmetricSyncTask.java     |  94 ++
 .../org/apache/cassandra/repair/SyncTask.java   |  97 ---
 .../repair/SystemDistributedKeyspace.java       |   6 +-
 .../repair/consistent/LocalSessions.java        |  36 +-
 .../apache/cassandra/schema/KeyspaceParams.java |   5 +
 .../cassandra/schema/ReplicationParams.java     |   9 +-
 .../apache/cassandra/schema/SchemaKeyspace.java |   4 +
 .../apache/cassandra/schema/TableMetadata.java  |   6 +
 .../apache/cassandra/schema/TableParams.java    |  11 +
 .../service/AbstractWriteResponseHandler.java   |  95 +-
 .../cassandra/service/ActiveRepairService.java  |  52 +-
 .../service/BatchlogResponseHandler.java        |   2 +-
 .../DatacenterSyncWriteResponseHandler.java     |  21 +-
 .../service/DatacenterWriteResponseHandler.java |  26 +-
 .../service/PendingRangeCalculatorService.java  |   2 +-
 .../apache/cassandra/service/StorageProxy.java  | 672 ++++++---------
 .../cassandra/service/StorageService.java       | 860 ++++++++++++-------
 .../cassandra/service/StorageServiceMBean.java  |   2 +
 .../cassandra/service/WriteResponseHandler.java |  25 +-
 .../service/reads/AbstractReadExecutor.java     | 243 +++---
 .../cassandra/service/reads/DataResolver.java   |  83 +-
 .../cassandra/service/reads/DigestResolver.java |  79 +-
 .../cassandra/service/reads/ReadCallback.java   |  57 +-
 .../service/reads/ResponseResolver.java         |  32 +-
 .../reads/ShortReadPartitionsProtection.java    |  36 +-
 .../service/reads/ShortReadProtection.java      |   3 +-
 .../service/reads/ShortReadRowsProtection.java  |   6 +-
 .../reads/repair/AbstractReadRepair.java        |  90 +-
 .../reads/repair/BlockingPartitionRepair.java   |  73 +-
 .../reads/repair/BlockingReadRepair.java        |  29 +-
 .../reads/repair/BlockingReadRepairs.java       |  19 -
 .../service/reads/repair/NoopReadRepair.java    |  15 +-
 .../repair/PartitionIteratorMergeListener.java  |  14 +-
 .../reads/repair/ReadOnlyReadRepair.java        |  15 +-
 .../service/reads/repair/ReadRepair.java        |  39 +-
 .../reads/repair/ReadRepairDiagnostics.java     |   5 +-
 .../service/reads/repair/ReadRepairEvent.java   |  11 +-
 .../reads/repair/ReadRepairStrategy.java        |  13 +-
 .../reads/repair/RowIteratorMergeListener.java  |  58 +-
 .../streaming/DefaultConnectionFactory.java     |   7 +-
 .../apache/cassandra/streaming/StreamPlan.java  |  38 +-
 .../cassandra/streaming/StreamRequest.java      |  98 ++-
 .../cassandra/streaming/StreamSession.java      |  54 +-
 .../cassandra/streaming/TableStreamManager.java |   7 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   5 +
 .../org/apache/cassandra/tools/NodeTool.java    |   2 +
 .../tools/SSTableRepairedAtSetter.java          |   4 +-
 .../cassandra/tools/nodetool/GetReplicas.java   |  47 +
 .../apache/cassandra/tracing/TraceState.java    |   2 +-
 .../transport/messages/ErrorMessage.java        |   2 +-
 src/java/org/apache/cassandra/utils/Pair.java   |  12 +
 .../cassandra/utils/concurrent/Accumulator.java |  27 +-
 .../legacy_na_clust/na-1-big-CompressionInfo.db | Bin 87 -> 87 bytes
 .../legacy_na_clust/na-1-big-Data.db            | Bin 5259 -> 5214 bytes
 .../legacy_na_clust/na-1-big-Digest.crc32       |   2 +-
 .../legacy_na_clust/na-1-big-Index.db           | Bin 157553 -> 157553 bytes
 .../legacy_na_clust/na-1-big-Statistics.db      | Bin 7095 -> 7096 bytes
 .../legacy_na_clust/na-1-big-TOC.txt            |   8 +-
 .../na-1-big-CompressionInfo.db                 | Bin 79 -> 79 bytes
 .../legacy_na_clust_counter/na-1-big-Data.db    | Bin 5888 -> 5759 bytes
 .../na-1-big-Digest.crc32                       |   2 +-
 .../legacy_na_clust_counter/na-1-big-Index.db   | Bin 157553 -> 157553 bytes
 .../na-1-big-Statistics.db                      | Bin 7104 -> 7105 bytes
 .../legacy_na_clust_counter/na-1-big-TOC.txt    |   8 +-
 .../legacy_na_simple/na-1-big-Data.db           | Bin 89 -> 88 bytes
 .../legacy_na_simple/na-1-big-Digest.crc32      |   2 +-
 .../legacy_na_simple/na-1-big-Statistics.db     | Bin 4648 -> 4649 bytes
 .../legacy_na_simple/na-1-big-TOC.txt           |   8 +-
 .../legacy_na_simple_counter/na-1-big-Data.db   | Bin 140 -> 138 bytes
 .../na-1-big-Digest.crc32                       |   2 +-
 .../na-1-big-Statistics.db                      | Bin 4657 -> 4658 bytes
 .../legacy_na_simple_counter/na-1-big-TOC.txt   |   8 +-
 .../locator/DynamicEndpointSnitchLongTest.java  |  20 +-
 .../cassandra/streaming/LongStreamingTest.java  |  11 +-
 .../test/microbench/PendingRangesBench.java     |  27 +-
 test/unit/org/apache/cassandra/Util.java        |  20 +
 .../config/DatabaseDescriptorRefTest.java       |   5 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |   3 +-
 .../cql3/validation/operations/CreateTest.java  |   3 +-
 .../org/apache/cassandra/db/CleanupTest.java    |   9 +-
 .../cassandra/db/CleanupTransientTest.java      | 195 +++++
 .../org/apache/cassandra/db/ImportTest.java     |   2 +-
 .../db/RepairedDataTombstonesTest.java          |   2 +-
 .../apache/cassandra/db/RowUpdateBuilder.java   |   6 +
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 .../db/SystemKeyspaceMigrator40Test.java        |  26 +
 .../apache/cassandra/db/TableCQLHelperTest.java |   3 +
 .../org/apache/cassandra/db/VerifyTest.java     |   4 +-
 .../compaction/AbstractPendingRepairTest.java   |  13 +-
 .../db/compaction/AntiCompactionTest.java       | 234 +++--
 ...pactionStrategyManagerPendingRepairTest.java | 163 +++-
 .../CompactionStrategyManagerTest.java          |  50 +-
 .../db/compaction/CompactionTaskTest.java       |  10 +-
 .../db/compaction/CompactionsCQLTest.java       |   4 +-
 .../LeveledCompactionStrategyTest.java          |   2 +-
 .../db/compaction/PendingRepairManagerTest.java |  28 +-
 .../db/compaction/SingleSSTableLCSTaskTest.java |   6 +-
 .../db/lifecycle/LogTransactionTest.java        |   2 +-
 .../db/lifecycle/RealTransactionsTest.java      |   1 +
 ...tionManagerGetSSTablesForValidationTest.java |   4 +-
 .../db/repair/PendingAntiCompactionTest.java    |  39 +-
 .../db/streaming/CassandraOutgoingFileTest.java |   1 +
 .../streaming/CassandraStreamManagerTest.java   |  16 +-
 .../db/streaming/StreamRequestTest.java         |  98 +++
 .../apache/cassandra/db/view/ViewUtilsTest.java |  23 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |  34 +-
 .../dht/RangeFetchMapCalculatorTest.java        | 138 ++-
 .../org/apache/cassandra/dht/RangeTest.java     |  12 +-
 .../org/apache/cassandra/dht/SplitterTest.java  |  63 +-
 .../cassandra/dht/StreamStateStoreTest.java     |   5 +-
 .../gms/PendingRangeCalculatorServiceTest.java  |   2 +-
 .../io/sstable/BigTableWriterTest.java          |   2 +-
 .../io/sstable/CQLSSTableWriterTest.java        |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   9 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |   7 +-
 .../io/sstable/SSTableRewriterTest.java         |   2 +-
 .../cassandra/io/sstable/SSTableUtils.java      |   2 +-
 .../cassandra/io/sstable/SSTableWriterTest.java |  60 ++
 .../io/sstable/SSTableWriterTestBase.java       |  11 +-
 .../format/SSTableFlushObserverTest.java        |   2 +-
 .../metadata/MetadataSerializerTest.java        |   2 +-
 .../locator/DynamicEndpointSnitchTest.java      |  42 +-
 .../locator/NetworkTopologyStrategyTest.java    |  92 +-
 .../locator/OldNetworkTopologyStrategyTest.java |  28 +-
 .../cassandra/locator/PendingRangeMapsTest.java |  49 +-
 .../locator/ReplicaCollectionTest.java          | 468 ++++++++++
 .../apache/cassandra/locator/ReplicaUtils.java  |  44 +
 .../locator/ReplicationFactorTest.java          |  73 ++
 .../ReplicationStrategyEndpointCacheTest.java   |  56 +-
 .../cassandra/locator/SimpleStrategyTest.java   |  81 +-
 .../cassandra/locator/TokenMetadataTest.java    |   8 +-
 .../cassandra/net/WriteCallbackInfoTest.java    |   7 +-
 .../async/OutboundMessagingConnectionTest.java  |   3 +-
 .../cassandra/repair/LocalSyncTaskTest.java     | 191 ----
 .../cassandra/repair/RepairRunnableTest.java    |  12 +-
 .../cassandra/repair/RepairSessionTest.java     |   6 +-
 .../repair/SymmetricLocalSyncTaskTest.java      | 232 +++++
 .../repair/SymmetricRemoteSyncTaskTest.java     |  71 ++
 .../repair/consistent/LocalSessionAccessor.java |   3 +-
 .../repair/consistent/LocalSessionTest.java     |   9 +-
 .../org/apache/cassandra/schema/MockSchema.java |   2 +-
 .../service/ActiveRepairServiceTest.java        |  50 +-
 .../service/BootstrapTransientTest.java         | 179 ++++
 .../service/LeaveAndBootstrapTest.java          |  48 +-
 .../org/apache/cassandra/service/MoveTest.java  | 324 +++----
 .../cassandra/service/MoveTransientTest.java    | 638 ++++++++++++++
 .../cassandra/service/StorageServiceTest.java   | 148 ++++
 .../service/WriteResponseHandlerTest.java       |  58 +-
 .../WriteResponseHandlerTransientTest.java      | 224 +++++
 .../service/reads/AbstractReadResponseTest.java | 300 +++++++
 .../service/reads/DataResolverTest.java         | 486 +++++------
 .../reads/DataResolverTransientTest.java        | 226 +++++
 .../service/reads/DigestResolverTest.java       | 144 ++++
 .../service/reads/ReadExecutorTest.java         |  46 +-
 .../reads/repair/AbstractReadRepairTest.java    |  53 +-
 .../reads/repair/BlockingReadRepairTest.java    | 113 ++-
 .../DiagEventsBlockingReadRepairTest.java       |  45 +-
 .../reads/repair/InstrumentedReadRepair.java    |   4 +-
 .../reads/repair/ReadOnlyReadRepairTest.java    |  30 +-
 .../service/reads/repair/ReadRepairTest.java    | 353 ++++++++
 .../reads/repair/TestableReadRepair.java        |  50 +-
 .../streaming/StreamingTransferTest.java        |   7 +-
 .../utils/concurrent/AccumulatorTest.java       |   2 +-
 300 files changed, 11945 insertions(+), 4347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9e76586..b53b986 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Transient Replication and Cheap Quorums (CASSANDRA-14404)
  * Log server-generated timestamp and nowInSeconds used by queries in FQL (CASSANDRA-14675)
  * Add diagnostic events for read repairs (CASSANDRA-14668)
  * Use consistent nowInSeconds and timestamps values within a request (CASSANDRA-14671)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index aa8281c..5066378 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -38,6 +38,10 @@ using the provided 'sstableupgrade' tool.
 
 New features
 ------------
+   - *Experimental* support for Transient Replication and Cheap Quorums introduced by CASSANDRA-14404
+     The intended audience for this functionality is expert users of Cassandra who are prepared
+     to validate every aspect of the database for their application and deployment practices. Future
+     releases of Cassandra will make this feature suitable for a wider audience.
    - *Experimental* support for Java 11 has been added. JVM options that differ between or are
      specific for Java 8 and 11 have been moved from jvm.options into jvm8.options and jvm11.options.
      IMPORTANT: Running C* on Java 11 is *experimental* and do it at your own risk.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 064ee4f..503a0fa 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1052,6 +1052,10 @@ enable_scripted_user_defined_functions: false
 # Materialized views are considered experimental and are not recommended for production use.
 enable_materialized_views: true
 
+# Enables creation of transiently replicated keyspaces on this node.
+# Transient replication is experimental and is not recommended for production use.
+#enable_transient_replication: true
+
 # The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation.
 # Lowering this value on Windows can provide much tighter latency and better throughput, however
 # some virtualized environments may see a negative performance impact from changing this setting

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/doc/source/architecture/dynamo.rst
----------------------------------------------------------------------
diff --git a/doc/source/architecture/dynamo.rst b/doc/source/architecture/dynamo.rst
index 365a695..12c586e 100644
--- a/doc/source/architecture/dynamo.rst
+++ b/doc/source/architecture/dynamo.rst
@@ -74,6 +74,35 @@ nodes in each rack, the data load on the smallest rack may be much higher.  Simi
 into a new rack, it will be considered a replica for the entire ring.  For this reason, many operators choose to
 configure all nodes on a single "rack".
 
+.. _transient-replication:
+
+Transient Replication
+~~~~~~~~~~~~~~~~~~~~~
+
+Transient replication allows you to configure a subset of replicas to only replicate data that hasn't been incrementally
+repaired. This allows you to decouple data redundancy from availability. For instance, if you have a keyspace replicated
+at rf 3, and alter it to rf 5 with 2 transient replicas, you go from being able to tolerate one failed replica to being
+able to tolerate two, without corresponding increase in storage usage. This is because 3 nodes will replicate all the data
+for a given token range, and the other 2 will only replicate data that hasn't been incrementally repaired.
+
+To use transient replication, you first need to enable it in ``cassandra.yaml``. Once enabled, both SimpleStrategy and
+NetworkTopologyStrategy can be configured to transiently replicate data. You configure it by specifying replication factor
+as ``<total_replicas>/<transient_replicas`` Both SimpleStrategy and NetworkTopologyStrategy support configuring transient
+replication.
+
+Transiently replicated keyspaces only support tables created with read_repair set to NONE and monotonic reads are not currently supported.
+You also can't use LWT, logged batches, and counters in 4.0. You will possibly never be able to use materialized views
+with transiently replicated keyspaces and probably never be able to use 2i with them.
+
+Transient replication is an experimental feature that may not be ready for production use. The expected audienced is experienced
+users of Cassandra capable of fully validating a deployment of their particular application. That means being able check
+that operations like reads, writes, decommission, remove, rebuild, repair, and replace all work with your queries, data,
+configuration, operational practices, and availability requirements.
+
+It is anticipated that 4.next will support monotonic reads with transient replication as well as LWT, logged batches, and
+counters.
+
+
 Tunable Consistency
 ^^^^^^^^^^^^^^^^^^^
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/doc/source/cql/ddl.rst
----------------------------------------------------------------------
diff --git a/doc/source/cql/ddl.rst b/doc/source/cql/ddl.rst
index 9afd638..2d9a50a 100644
--- a/doc/source/cql/ddl.rst
+++ b/doc/source/cql/ddl.rst
@@ -105,6 +105,14 @@ strategy is used. By default, Cassandra support the following ``'class'``:
 Attempting to create a keyspace that already exists will return an error unless the ``IF NOT EXISTS`` option is used. If
 it is used, the statement will be a no-op if the keyspace already exists.
 
+If :ref:`transient replication <transient-replication>` has been enabled, transient replicas can be configured for both
+SimpleStrategy and NetworkTopologyStrategy by defining replication factors in the format ``'<total_replicas>/<transient_replicas>'``
+
+For instance, this keyspace will have 3 replicas in DC1, 1 of which is transient, and 5 replicas in DC2, 2 of which are transient::
+
+    CREATE KEYSPACE some_keysopace
+               WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1' : '3/1'', 'DC2' : '5/2'};
+
 .. _use-statement:
 
 USE
@@ -455,6 +463,9 @@ A table supports the following options:
 | ``speculative_retry``          | *simple* | 99PERCENTILE| :ref:`Speculative retry options                           |
 |                                |          |             | <speculative-retry-options>`.                             |
 +--------------------------------+----------+-------------+-----------------------------------------------------------+
+| ``speculative_write_threshold``| *simple* | 99PERCENTILE| :ref:`Speculative retry options                           |
+|                                |          |             | <speculative-retry-options>`.                             |
++--------------------------------+----------+-------------+-----------------------------------------------------------+
 | ``gc_grace_seconds``           | *simple* | 864000      | Time to wait before garbage collecting tombstones         |
 |                                |          |             | (deletion markers).                                       |
 +--------------------------------+----------+-------------+-----------------------------------------------------------+
@@ -485,7 +496,8 @@ Speculative retry options
 By default, Cassandra read coordinators only query as many replicas as necessary to satisfy
 consistency levels: one for consistency level ``ONE``, a quorum for ``QUORUM``, and so on.
 ``speculative_retry`` determines when coordinators may query additional replicas, which is useful
-when replicas are slow or unresponsive.  The following are legal values (case-insensitive):
+when replicas are slow or unresponsive.  ``speculative_write_threshold`` specifies the threshold at which
+a cheap quorum write will be upgraded to include transient replicas.  The following are legal values (case-insensitive):
 
 ============================ ======================== =============================================================================
  Format                       Example                  Description

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/lib/cassandra-driver-internal-only-3.12.0.post0-5838e2fd.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.12.0.post0-5838e2fd.zip b/lib/cassandra-driver-internal-only-3.12.0.post0-5838e2fd.zip
new file mode 100644
index 0000000..8d627a9
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.12.0.post0-5838e2fd.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 5595e2a..405e88e 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -49,6 +49,7 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         ('max_index_interval', None),
         ('default_time_to_live', None),
         ('speculative_retry', None),
+        ('speculative_write_threshold', None),
         ('memtable_flush_period_in_ms', None),
         ('cdc', None),
         ('read_repair', None),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/pylib/cqlshlib/cqlshhandling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cqlshhandling.py b/pylib/cqlshlib/cqlshhandling.py
index 9545876..7abd6ce 100644
--- a/pylib/cqlshlib/cqlshhandling.py
+++ b/pylib/cqlshlib/cqlshhandling.py
@@ -112,6 +112,7 @@ cqlsh_consistency_level_syntax_rules = r'''
                      | "SERIAL"
                      | "LOCAL_SERIAL"
                      | "LOCAL_ONE"
+                     | "NODE_LOCAL"
                      ;
 '''
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/pylib/cqlshlib/test/test_cqlsh_completion.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py
index fa9490d..794c591 100644
--- a/pylib/cqlshlib/test/test_cqlsh_completion.py
+++ b/pylib/cqlshlib/test/test_cqlsh_completion.py
@@ -594,7 +594,7 @@ class TestCqlshCompletion(CqlshCompletionCase):
                                      'memtable_flush_period_in_ms',
                                      'CLUSTERING',
                                      'COMPACT', 'caching', 'comment',
-                                     'min_index_interval', 'speculative_retry', 'cdc'])
+                                     'min_index_interval', 'speculative_retry', 'speculative_write_threshold', 'cdc'])
         self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH ',
                             choices=['bloom_filter_fp_chance', 'compaction',
                                      'compression',
@@ -603,7 +603,7 @@ class TestCqlshCompletion(CqlshCompletionCase):
                                      'memtable_flush_period_in_ms',
                                      'CLUSTERING',
                                      'COMPACT', 'caching', 'comment',
-                                     'min_index_interval', 'speculative_retry', 'cdc'])
+                                     'min_index_interval', 'speculative_retry', 'speculative_write_threshold', 'cdc'])
         self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH bloom_filter_fp_chance ',
                             immediate='= ')
         self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH bloom_filter_fp_chance = ',
@@ -650,7 +650,7 @@ class TestCqlshCompletion(CqlshCompletionCase):
                                      'memtable_flush_period_in_ms',
                                      'CLUSTERING',
                                      'COMPACT', 'caching', 'comment',
-                                     'min_index_interval', 'speculative_retry', 'cdc'])
+                                     'min_index_interval', 'speculative_retry', 'speculative_write_threshold', 'cdc'])
         self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
                             + "{'class': 'DateTieredCompactionStrategy', '",
                             choices=['base_time_seconds', 'max_sstable_age_days',

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/pylib/cqlshlib/test/test_cqlsh_output.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 2f0d9bf..46546f0 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -622,7 +622,8 @@ class TestCqlshOutput(BaseTestCase):
                 AND max_index_interval = 2048
                 AND memtable_flush_period_in_ms = 0
                 AND min_index_interval = 128
-                AND speculative_retry = '99PERCENTILE';
+                AND speculative_retry = '99PERCENTILE'
+                AND speculative_write_threshold = '99PERCENTILE';
 
         """ % quote_name(get_keyspace()))
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index 4809bd7..8dda54e 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -26,14 +26,20 @@ import java.util.concurrent.*;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicates;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.EndpointsForToken;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.Replicas;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
@@ -419,7 +425,7 @@ public class BatchlogManager implements BatchlogManagerMBean
                 if (handler != null)
                 {
                     hintedNodes.addAll(handler.undelivered);
-                    HintsService.instance.write(transform(handler.undelivered, StorageService.instance::getHostIdForEndpoint),
+                    HintsService.instance.write(Collections2.transform(handler.undelivered, StorageService.instance::getHostIdForEndpoint),
                                                 Hint.create(undeliveredMutation, writtenAt));
                 }
             }
@@ -449,35 +455,41 @@ public class BatchlogManager implements BatchlogManagerMBean
                                                                                      long writtenAt,
                                                                                      Set<InetAddressAndPort> hintedNodes)
         {
-            Set<InetAddressAndPort> liveEndpoints = new HashSet<>();
             String ks = mutation.getKeyspaceName();
+            Keyspace keyspace = Keyspace.open(ks);
             Token tk = mutation.key().getToken();
 
-            for (InetAddressAndPort endpoint : StorageService.instance.getNaturalAndPendingEndpoints(ks, tk))
+            EndpointsForToken replicas = StorageService.instance.getNaturalAndPendingReplicasForToken(ks, tk);
+            Replicas.temporaryAssertFull(replicas); // TODO in CASSANDRA-14549
+
+            EndpointsForToken.Builder liveReplicasBuilder = EndpointsForToken.builder(tk);
+            for (Replica replica : replicas)
             {
-                if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
+                if (replica.isLocal())
                 {
                     mutation.apply();
                 }
-                else if (FailureDetector.instance.isAlive(endpoint))
+                else if (FailureDetector.instance.isAlive(replica.endpoint()))
                 {
-                    liveEndpoints.add(endpoint); // will try delivering directly instead of writing a hint.
+                    liveReplicasBuilder.add(replica); // will try delivering directly instead of writing a hint.
                 }
                 else
                 {
-                    hintedNodes.add(endpoint);
-                    HintsService.instance.write(StorageService.instance.getHostIdForEndpoint(endpoint),
+                    hintedNodes.add(replica.endpoint());
+                    HintsService.instance.write(StorageService.instance.getHostIdForEndpoint(replica.endpoint()),
                                                 Hint.create(mutation, writtenAt));
                 }
             }
 
-            if (liveEndpoints.isEmpty())
+            EndpointsForToken liveReplicas = liveReplicasBuilder.build();
+            if (liveReplicas.isEmpty())
                 return null;
 
-            ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(liveEndpoints, System.nanoTime());
+            Replicas.temporaryAssertFull(liveReplicas);
+            ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(keyspace, liveReplicas, System.nanoTime());
             MessageOut<Mutation> message = mutation.createMessage();
-            for (InetAddressAndPort endpoint : liveEndpoints)
-                MessagingService.instance().sendRR(message, endpoint, handler, false);
+            for (Replica replica : liveReplicas)
+                MessagingService.instance().sendWriteRR(message, replica, handler, false);
             return handler;
         }
 
@@ -497,16 +509,17 @@ public class BatchlogManager implements BatchlogManagerMBean
         {
             private final Set<InetAddressAndPort> undelivered = Collections.newSetFromMap(new ConcurrentHashMap<>());
 
-            ReplayWriteResponseHandler(Collection<InetAddressAndPort> writeEndpoints, long queryStartNanoTime)
+            ReplayWriteResponseHandler(Keyspace keyspace, EndpointsForToken writeReplicas, long queryStartNanoTime)
             {
-                super(writeEndpoints, Collections.<InetAddressAndPort>emptySet(), null, null, null, WriteType.UNLOGGED_BATCH, queryStartNanoTime);
-                undelivered.addAll(writeEndpoints);
+                super(ReplicaLayout.forWriteWithDownNodes(keyspace, null, writeReplicas.token(), writeReplicas, EndpointsForToken.empty(writeReplicas.token())),
+                      null, WriteType.UNLOGGED_BATCH, queryStartNanoTime);
+                Iterables.addAll(undelivered, writeReplicas.endpoints());
             }
 
             @Override
             protected int totalBlockFor()
             {
-                return this.naturalEndpoints.size();
+                return this.replicaLayout.selected().size();
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index a13070c..783dcc1 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -339,6 +339,8 @@ public class Config
 
     public boolean enable_materialized_views = true;
 
+    public boolean enable_transient_replication = false;
+
     /**
      * Optionally disable asynchronous UDF execution.
      * Disabling asynchronous UDF execution also implicitly disables the security-manager!

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index af13f9c..75b3fc3 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -59,6 +59,7 @@ import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.locator.EndpointSnitchInfo;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.SeedProvider;
 import org.apache.cassandra.net.BackPressureStrategy;
 import org.apache.cassandra.net.RateBasedBackPressure;
@@ -122,7 +123,7 @@ public class DatabaseDescriptor
     private static long indexSummaryCapacityInMB;
 
     private static String localDC;
-    private static Comparator<InetAddressAndPort> localComparator;
+    private static Comparator<Replica> localComparator;
     private static EncryptionContext encryptionContext;
     private static boolean hasLoggedConfig;
 
@@ -991,18 +992,14 @@ public class DatabaseDescriptor
         EndpointSnitchInfo.create();
 
         localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddressAndPort());
-        localComparator = new Comparator<InetAddressAndPort>()
-        {
-            public int compare(InetAddressAndPort endpoint1, InetAddressAndPort endpoint2)
-            {
-                boolean local1 = localDC.equals(snitch.getDatacenter(endpoint1));
-                boolean local2 = localDC.equals(snitch.getDatacenter(endpoint2));
-                if (local1 && !local2)
-                    return -1;
-                if (local2 && !local1)
-                    return 1;
-                return 0;
-            }
+        localComparator = (replica1, replica2) -> {
+            boolean local1 = localDC.equals(snitch.getDatacenter(replica1));
+            boolean local2 = localDC.equals(snitch.getDatacenter(replica2));
+            if (local1 && !local2)
+                return -1;
+            if (local2 && !local1)
+                return 1;
+            return 0;
         };
     }
 
@@ -2308,7 +2305,7 @@ public class DatabaseDescriptor
         return localDC;
     }
 
-    public static Comparator<InetAddressAndPort> getLocalComparator()
+    public static Comparator<Replica> getLocalComparator()
     {
         return localComparator;
     }
@@ -2459,6 +2456,16 @@ public class DatabaseDescriptor
         return conf.enable_materialized_views;
     }
 
+    public static boolean isTransientReplicationEnabled()
+    {
+        return conf.enable_transient_replication;
+    }
+
+    public static void setTransientReplicationEnabledUnsafe(boolean enabled)
+    {
+        conf.enable_transient_replication = enabled;
+    }
+
     public static long getUserDefinedFunctionFailTimeout()
     {
         return conf.user_defined_function_fail_timeout;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 79e19c1..45db947 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -202,7 +202,18 @@ public class QueryProcessor implements QueryHandler
         statement.authorize(clientState);
         statement.validate(clientState);
 
-        ResultMessage result = statement.execute(queryState, options, queryStartNanoTime);
+        ResultMessage result;
+        if (options.getConsistency() == ConsistencyLevel.NODE_LOCAL)
+        {
+            assert Boolean.getBoolean("cassandra.enable_nodelocal_queries") : "Node local consistency level is highly dangerous and should be used only for debugging purposes";
+            assert statement instanceof SelectStatement : "Only SELECT statements are permitted for node-local execution";
+            logger.info("Statement {} executed with NODE_LOCAL consistency level.", statement);
+            result = statement.executeLocally(queryState, options);
+        }
+        else
+        {
+            result = statement.execute(queryState, options, queryStartNanoTime);
+        }
         return result == null ? new ResultMessage.Void() : result;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index e925735..fa637ef 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -261,7 +261,7 @@ public class BatchStatement implements CQLStatement
         return statements;
     }
 
-    private Collection<? extends IMutation> getMutations(BatchQueryOptions options,
+    private List<? extends IMutation> getMutations(BatchQueryOptions options,
                                                          boolean local,
                                                          long batchTimestamp,
                                                          int nowInSeconds,
@@ -401,7 +401,7 @@ public class BatchStatement implements CQLStatement
         return new ResultMessage.Void();
     }
 
-    private void executeWithoutConditions(Collection<? extends IMutation> mutations, ConsistencyLevel cl, long queryStartNanoTime) throws RequestExecutionException, RequestValidationException
+    private void executeWithoutConditions(List<? extends IMutation> mutations, ConsistencyLevel cl, long queryStartNanoTime) throws RequestExecutionException, RequestValidationException
     {
         if (mutations.isEmpty())
             return;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
index 96d9f5a..8f70ffc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchUpdatesCollector.java
@@ -104,7 +104,7 @@ final class BatchUpdatesCollector implements UpdatesCollector
      * Returns a collection containing all the mutations.
      * @return a collection containing all the mutations.
      */
-    public Collection<IMutation> toMutations()
+    public List<IMutation> toMutations()
     {
         //TODO: The case where all statement where on the same keyspace is pretty common, optimize for that?
         List<IMutation> ms = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 13fc659..a8367f0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -465,7 +465,7 @@ public abstract class ModificationStatement implements CQLStatement
         else
             cl.validateForWrite(metadata.keyspace);
 
-        Collection<? extends IMutation> mutations =
+        List<? extends IMutation> mutations =
             getMutations(options,
                          false,
                          options.getTimestamp(queryState),
@@ -676,7 +676,7 @@ public abstract class ModificationStatement implements CQLStatement
      *
      * @return list of the mutations
      */
-    private Collection<? extends IMutation> getMutations(QueryOptions options,
+    private List<? extends IMutation> getMutations(QueryOptions options,
                                                          boolean local,
                                                          long timestamp,
                                                          int nowInSeconds,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java
index 1def3fd..6ef551d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java
@@ -82,7 +82,7 @@ final class SingleTableUpdatesCollector implements UpdatesCollector
      * Returns a collection containing all the mutations.
      * @return a collection containing all the mutations.
      */
-    public Collection<IMutation> toMutations()
+    public List<IMutation> toMutations()
     {
         List<IMutation> ms = new ArrayList<>();
         for (PartitionUpdate.Builder builder : puBuilders.values())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
index 30db7ca..c3dd334 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
@@ -18,17 +18,16 @@
 
 package org.apache.cassandra.cql3.statements;
 
-import java.util.Collection;
+import java.util.List;
 
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.schema.TableMetadata;
 
 public interface UpdatesCollector
 {
     PartitionUpdate.Builder getPartitionUpdateBuilder(TableMetadata metadata, DecoratedKey dk, ConsistencyLevel consistency);
-    Collection<IMutation> toMutations();
+    List<IMutation> toMutations();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
index 12e73d0..2f0c188 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
@@ -17,16 +17,27 @@
  */
 package org.apache.cassandra.cql3.statements.schema;
 
+import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.ReplicationFactor;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
 import org.apache.cassandra.schema.Keyspaces;
@@ -34,9 +45,13 @@ import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event.SchemaChange;
 import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.utils.FBUtilities;
 
 public final class AlterKeyspaceStatement extends AlterSchemaStatement
 {
+    private static final boolean allow_alter_rf_during_range_movement = Boolean.getBoolean(Config.PROPERTY_PREFIX + "allow_alter_rf_during_range_movement");
+    private static final boolean allow_unsafe_transient_changes = Boolean.getBoolean(Config.PROPERTY_PREFIX + "allow_unsafe_transient_changes");
+
     private final KeyspaceAttributes attrs;
 
     public AlterKeyspaceStatement(String keyspaceName, KeyspaceAttributes attrs)
@@ -60,6 +75,9 @@ public final class AlterKeyspaceStatement extends AlterSchemaStatement
 
         newKeyspace.params.validate(keyspaceName);
 
+        validateNoRangeMovements();
+        validateTransientReplication(keyspace.createReplicationStrategy(), newKeyspace.createReplicationStrategy());
+
         return schema.withAddedOrUpdated(newKeyspace);
     }
 
@@ -84,11 +102,77 @@ public final class AlterKeyspaceStatement extends AlterSchemaStatement
         AbstractReplicationStrategy before = keyspaceDiff.before.createReplicationStrategy();
         AbstractReplicationStrategy after = keyspaceDiff.after.createReplicationStrategy();
 
-        return before.getReplicationFactor() < after.getReplicationFactor()
+        return before.getReplicationFactor().fullReplicas < after.getReplicationFactor().fullReplicas
              ? ImmutableSet.of("When increasing replication factor you need to run a full (-full) repair to distribute the data.")
              : ImmutableSet.of();
     }
 
+    private void validateNoRangeMovements()
+    {
+        if (allow_alter_rf_during_range_movement)
+            return;
+
+        Stream<InetAddressAndPort> endpoints = Stream.concat(Gossiper.instance.getLiveMembers().stream(), Gossiper.instance.getUnreachableMembers().stream());
+        List<InetAddressAndPort> notNormalEndpoints = endpoints.filter(endpoint -> !FBUtilities.getBroadcastAddressAndPort().equals(endpoint) &&
+                                                                                   !Gossiper.instance.getEndpointStateForEndpoint(endpoint).isNormalState())
+                                                               .collect(Collectors.toList());
+        if (!notNormalEndpoints.isEmpty())
+        {
+            throw new ConfigurationException("Cannot alter RF while some endpoints are not in normal state (no range movements): " + notNormalEndpoints);
+        }
+    }
+
+    private void validateTransientReplication(AbstractReplicationStrategy oldStrategy, AbstractReplicationStrategy newStrategy)
+    {
+        //If there is no read traffic there are some extra alterations you can safely make, but this is so atypical
+        //that a good default is to not allow unsafe changes
+        if (allow_unsafe_transient_changes)
+            return;
+
+        ReplicationFactor oldRF = oldStrategy.getReplicationFactor();
+        ReplicationFactor newRF = newStrategy.getReplicationFactor();
+
+        int oldTrans = oldRF.transientReplicas();
+        int oldFull = oldRF.fullReplicas;
+        int newTrans = newRF.transientReplicas();
+        int newFull = newRF.fullReplicas;
+
+        if (newTrans > 0)
+        {
+            if (DatabaseDescriptor.getNumTokens() > 1)
+                throw new ConfigurationException(String.format("Transient replication is not supported with vnodes yet"));
+
+            Keyspace ks = Keyspace.open(keyspaceName);
+            for (ColumnFamilyStore cfs : ks.getColumnFamilyStores())
+            {
+                if (cfs.viewManager.hasViews())
+                {
+                    throw new ConfigurationException("Cannot use transient replication on keyspaces using materialized views");
+                }
+
+                if (cfs.indexManager.hasIndexes())
+                {
+                    throw new ConfigurationException("Cannot use transient replication on keyspaces using secondary indexes");
+                }
+            }
+        }
+
+        //This is true right now because the transition from transient -> full lacks the pending state
+        //necessary for correctness. What would happen if we allowed this is that we would attempt
+        //to read from a transient replica as if it were a full replica.
+        if (oldFull > newFull && oldTrans > 0)
+            throw new ConfigurationException("Can't add full replicas if there are any transient replicas. You must first remove all transient replicas, then change the # of full replicas, then add back the transient replicas");
+
+        //Don't increase transient replication factor by more than one at a time if changing number of replicas
+        //Just like with changing full replicas it's not safe to do this as you could read from too many replicas
+        //that don't have the necessary data. W/O transient replication this alteration was allowed and it's not clear
+        //if it should be.
+        //This is structured so you can convert as many full replicas to transient replicas as you want.
+        boolean numReplicasChanged = oldTrans + oldFull != newTrans + newFull;
+        if (numReplicasChanged && (newTrans > oldTrans && newTrans != oldTrans + 1))
+            throw new ConfigurationException("Can only safely increase number of transients one at a time with incremental repair run in between each time");
+    }
+
     @Override
     public AuditLogContext getAuditLogContext()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
index 3ec75b2..5044119 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
 import org.apache.cassandra.transport.Event.SchemaChange;
 import org.apache.cassandra.transport.Event.SchemaChange.Change;
 import org.apache.cassandra.transport.Event.SchemaChange.Target;
@@ -360,6 +361,12 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
                           "before being replayed.");
             }
 
+            if (keyspace.createReplicationStrategy().hasTransientReplicas()
+                && params.readRepair != ReadRepairStrategy.NONE)
+            {
+                throw ire("read_repair must be set to 'NONE' for transiently replicated keyspaces");
+            }
+
             return keyspace.withSwapped(keyspace.tables.withSwapped(table.withSwapped(params)));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
index df41358..dbca160 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
@@ -28,7 +28,9 @@ import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QualifiedName;
 import org.apache.cassandra.cql3.statements.schema.IndexTarget.Type;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.service.ClientState;
@@ -88,6 +90,9 @@ public final class CreateIndexStatement extends AlterSchemaStatement
         if (table.isView())
             throw ire("Secondary indexes on materialized views aren't supported");
 
+        if (Keyspace.open(table.keyspace).getReplicationStrategy().hasTransientReplicas())
+            throw new InvalidRequestException("Secondary indexes are not supported on transiently replicated keyspaces");
+
         List<IndexTarget> indexTargets = Lists.newArrayList(transform(rawIndexTargets, t -> t.prepare(table)));
 
         if (indexTargets.isEmpty() && !attrs.isCustom)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
index 62fcafe..be7907f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
@@ -27,11 +27,14 @@ import org.apache.cassandra.auth.DataResource;
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.reads.repair.ReadRepairStrategy;
 import org.apache.cassandra.transport.Event.SchemaChange;
 import org.apache.cassandra.transport.Event.SchemaChange.Change;
 import org.apache.cassandra.transport.Event.SchemaChange.Target;
@@ -98,6 +101,12 @@ public final class CreateTableStatement extends AlterSchemaStatement
         TableMetadata table = builder(keyspace.types).build();
         table.validate();
 
+        if (keyspace.createReplicationStrategy().hasTransientReplicas()
+            && table.params.readRepair != ReadRepairStrategy.NONE)
+        {
+            throw ire("read_repair must be set to 'NONE' for transiently replicated keyspaces");
+        }
+
         return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.with(table)));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
index 5f62001..bf6bcff 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
@@ -31,9 +31,11 @@ import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.selection.RawSelector;
 import org.apache.cassandra.cql3.selection.Selectable;
 import org.apache.cassandra.cql3.statements.StatementType;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.service.ClientState;
@@ -107,6 +109,9 @@ public final class CreateViewStatement extends AlterSchemaStatement
         if (null == keyspace)
             throw ire("Keyspace '%s' doesn't exist", keyspaceName);
 
+        if (keyspace.createReplicationStrategy().hasTransientReplicas())
+            throw new InvalidRequestException("Materialized views are not supported on transiently replicated keyspaces");
+
         TableMetadata table = keyspace.tables.getNullable(tableName);
         if (null == table)
             throw ire("Base table '%s' doesn't exist", tableName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
index c8e464a..4e66307 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
@@ -128,6 +128,9 @@ public final class TableAttributes extends PropertyDefinitions
         if (hasOption(Option.SPECULATIVE_RETRY))
             builder.speculativeRetry(SpeculativeRetryPolicy.fromString(getString(Option.SPECULATIVE_RETRY)));
 
+        if (hasOption(Option.SPECULATIVE_WRITE_THRESHOLD))
+            builder.speculativeWriteThreshold(SpeculativeRetryPolicy.fromString(getString(Option.SPECULATIVE_WRITE_THRESHOLD)));
+
         if (hasOption(Option.CRC_CHECK_CHANCE))
             builder.crcCheckChance(getDouble(Option.CRC_CHECK_CHANCE));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 5e38584..56851e2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -30,7 +30,6 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
-import javax.annotation.Nullable;
 import javax.management.*;
 import javax.management.openmbean.*;
 
@@ -68,7 +67,6 @@ import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.KeyIterator;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.*;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -81,7 +79,6 @@ import org.apache.cassandra.metrics.TableMetrics;
 import org.apache.cassandra.repair.TableRepairManager;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
-import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.TableStreamManager;
@@ -205,7 +202,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final Directories directories;
 
     public final TableMetrics metric;
-    public volatile long sampleLatencyNanos;
+    public volatile long sampleReadLatencyNanos;
+    public volatile long transientWriteLatencyNanos;
 
     private final CassandraTableWriteHandler writeHandler;
     private final CassandraStreamManager streamManager;
@@ -384,7 +382,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         viewManager = keyspace.viewManager.forTable(metadata.id);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
-        sampleLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
+        sampleReadLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
+        transientWriteLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getWriteRpcTimeout() / 2);
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
@@ -454,7 +453,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         try
         {
-            sampleLatencyNanos = metadata().params.speculativeRetry.calculateThreshold(metric.coordinatorReadLatency);
+            sampleReadLatencyNanos = metadata().params.speculativeRetry.calculateThreshold(metric.coordinatorReadLatency);
+            transientWriteLatencyNanos = metadata().params.speculativeWriteThreshold.calculateThreshold(metric.coordinatorWriteLatency);
         }
         catch (Throwable e)
         {
@@ -487,15 +487,15 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return directories;
     }
 
-    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, int sstableLevel, SerializationHeader header, LifecycleTransaction txn)
+    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, int sstableLevel, SerializationHeader header, LifecycleTransaction txn)
     {
         MetadataCollector collector = new MetadataCollector(metadata().comparator).sstableLevel(sstableLevel);
-        return createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, collector, header, txn);
+        return createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, collector, header, txn);
     }
 
-    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, MetadataCollector metadataCollector, SerializationHeader header, LifecycleTransaction txn)
+    public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, MetadataCollector metadataCollector, SerializationHeader header, LifecycleTransaction txn)
     {
-        return getCompactionStrategyManager().createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, metadataCollector, header, indexManager.listIndexes(), txn);
+        return getCompactionStrategyManager().createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadataCollector, header, indexManager.listIndexes(), txn);
     }
 
     public boolean supportsEarlyOpen()
@@ -1402,7 +1402,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         // cleanup size estimation only counts bytes for keys local to this node
         long expectedFileSize = 0;
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalReplicas(keyspace.getName()).ranges();
         for (SSTableReader sstable : sstables)
         {
             List<SSTableReader.PartitionPositionBounds> positions = sstable.getPositionsForRanges(ranges);
@@ -1677,7 +1677,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public void cleanupCache()
     {
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalReplicas(keyspace.getName()).ranges();
 
         for (Iterator<RowCacheKey> keyIter = CacheService.instance.rowCache.keyIterator();
              keyIter.hasNext(); )

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index d37da0a..35ba198 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -17,16 +17,18 @@
  */
 package org.apache.cassandra.db;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.Replicas;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -47,7 +49,8 @@ public enum ConsistencyLevel
     EACH_QUORUM (7),
     SERIAL      (8),
     LOCAL_SERIAL(9),
-    LOCAL_ONE   (10, true);
+    LOCAL_ONE   (10, true),
+    NODE_LOCAL  (11, true);
 
     private static final Logger logger = LoggerFactory.getLogger(ConsistencyLevel.class);
 
@@ -89,13 +92,13 @@ public enum ConsistencyLevel
 
     private int quorumFor(Keyspace keyspace)
     {
-        return (keyspace.getReplicationStrategy().getReplicationFactor() / 2) + 1;
+        return (keyspace.getReplicationStrategy().getReplicationFactor().allReplicas / 2) + 1;
     }
 
     private int localQuorumFor(Keyspace keyspace, String dc)
     {
         return (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
-             ? (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1
+             ? (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc).allReplicas / 2) + 1
              : quorumFor(keyspace);
     }
 
@@ -116,7 +119,7 @@ public enum ConsistencyLevel
             case SERIAL:
                 return quorumFor(keyspace);
             case ALL:
-                return keyspace.getReplicationStrategy().getReplicationFactor();
+                return keyspace.getReplicationStrategy().getReplicationFactor().allReplicas;
             case LOCAL_QUORUM:
             case LOCAL_SERIAL:
                 return localQuorumFor(keyspace, DatabaseDescriptor.getLocalDataCenter());
@@ -138,6 +141,28 @@ public enum ConsistencyLevel
         }
     }
 
+    public int blockForWrite(Keyspace keyspace, Endpoints<?> pending)
+    {
+        assert pending != null;
+
+        int blockFor = blockFor(keyspace);
+        switch (this)
+        {
+            case ANY:
+                break;
+            case LOCAL_ONE: case LOCAL_QUORUM: case LOCAL_SERIAL:
+                // we will only count local replicas towards our response count, as these queries only care about local guarantees
+                blockFor += countDCLocalReplicas(pending).allReplicas();
+                break;
+            case ONE: case TWO: case THREE:
+            case QUORUM: case EACH_QUORUM:
+            case SERIAL:
+            case ALL:
+                blockFor += pending.size();
+        }
+        return blockFor;
+    }
+
     /**
      * Determine if this consistency level meets or exceeds the consistency requirements of the given cl for the given keyspace
      */
@@ -156,40 +181,75 @@ public enum ConsistencyLevel
         return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint));
     }
 
-    public int countLocalEndpoints(Iterable<InetAddressAndPort> liveEndpoints)
+    public static boolean isLocal(Replica replica)
+    {
+        return isLocal(replica.endpoint());
+    }
+
+    private static ReplicaCount countDCLocalReplicas(ReplicaCollection<?> liveReplicas)
     {
-        int count = 0;
-        for (InetAddressAndPort endpoint : liveEndpoints)
-            if (isLocal(endpoint))
-                count++;
+        ReplicaCount count = new ReplicaCount();
+        for (Replica replica : liveReplicas)
+            if (isLocal(replica))
+                count.increment(replica);
         return count;
     }
 
-    private Map<String, Integer> countPerDCEndpoints(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints)
+    private static class ReplicaCount
+    {
+        int fullReplicas;
+        int transientReplicas;
+
+        int allReplicas()
+        {
+            return fullReplicas + transientReplicas;
+        }
+
+        void increment(Replica replica)
+        {
+            if (replica.isFull()) ++fullReplicas;
+            else ++transientReplicas;
+        }
+
+        boolean isSufficient(int allReplicas, int fullReplicas)
+        {
+            return this.fullReplicas >= fullReplicas
+                    && this.allReplicas() >= allReplicas;
+        }
+    }
+
+    private static Map<String, ReplicaCount> countPerDCEndpoints(Keyspace keyspace, Iterable<Replica> liveReplicas)
     {
         NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
-        Map<String, Integer> dcEndpoints = new HashMap<String, Integer>();
+        Map<String, ReplicaCount> dcEndpoints = new HashMap<>();
         for (String dc: strategy.getDatacenters())
-            dcEndpoints.put(dc, 0);
+            dcEndpoints.put(dc, new ReplicaCount());
 
-        for (InetAddressAndPort endpoint : liveEndpoints)
+        for (Replica replica : liveReplicas)
         {
-            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint);
-            dcEndpoints.put(dc, dcEndpoints.get(dc) + 1);
+            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica);
+            dcEndpoints.get(dc).increment(replica);
         }
         return dcEndpoints;
     }
 
-    public List<InetAddressAndPort> filterForQuery(Keyspace keyspace, List<InetAddressAndPort> liveEndpoints)
+    public <E extends Endpoints<E>> E filterForQuery(Keyspace keyspace, E liveReplicas)
+    {
+        return filterForQuery(keyspace, liveReplicas, false);
+    }
+
+    public <E extends Endpoints<E>> E filterForQuery(Keyspace keyspace, E liveReplicas, boolean alwaysSpeculate)
     {
         /*
          * If we are doing an each quorum query, we have to make sure that the endpoints we select
          * provide a quorum for each data center. If we are not using a NetworkTopologyStrategy,
          * we should fall through and grab a quorum in the replication strategy.
+         *
+         * We do not speculate for EACH_QUORUM.
          */
         if (this == EACH_QUORUM && keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
-            return filterForEachQuorum(keyspace, liveEndpoints);
+            return filterForEachQuorum(keyspace, liveReplicas);
 
         /*
          * Endpoints are expected to be restricted to live replicas, sorted by snitch preference.
@@ -198,36 +258,34 @@ public enum ConsistencyLevel
          * the blockFor first ones).
          */
         if (isDCLocal)
-            liveEndpoints.sort(DatabaseDescriptor.getLocalComparator());
+            liveReplicas = liveReplicas.sorted(DatabaseDescriptor.getLocalComparator());
 
-        return liveEndpoints.subList(0, Math.min(liveEndpoints.size(), blockFor(keyspace)));
+        return liveReplicas.subList(0, Math.min(liveReplicas.size(), blockFor(keyspace) + (alwaysSpeculate ? 1 : 0)));
     }
 
-    private List<InetAddressAndPort> filterForEachQuorum(Keyspace keyspace, List<InetAddressAndPort> liveEndpoints)
+    private <E extends Endpoints<E>> E filterForEachQuorum(Keyspace keyspace, E liveReplicas)
     {
         NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
-
-        Map<String, List<InetAddressAndPort>> dcsEndpoints = new HashMap<>();
-        for (String dc: strategy.getDatacenters())
-            dcsEndpoints.put(dc, new ArrayList<>());
-
-        for (InetAddressAndPort add : liveEndpoints)
+        Map<String, Integer> dcsReplicas = new HashMap<>();
+        for (String dc : strategy.getDatacenters())
         {
-            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(add);
-            dcsEndpoints.get(dc).add(add);
+            // we put _up to_ dc replicas only
+            dcsReplicas.put(dc, localQuorumFor(keyspace, dc));
         }
 
-        List<InetAddressAndPort> waitSet = new ArrayList<>();
-        for (Map.Entry<String, List<InetAddressAndPort>> dcEndpoints : dcsEndpoints.entrySet())
-        {
-            List<InetAddressAndPort> dcEndpoint = dcEndpoints.getValue();
-            waitSet.addAll(dcEndpoint.subList(0, Math.min(localQuorumFor(keyspace, dcEndpoints.getKey()), dcEndpoint.size())));
-        }
-
-        return waitSet;
+        return liveReplicas.filter((replica) -> {
+            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica);
+            int replicas = dcsReplicas.get(dc);
+            if (replicas > 0)
+            {
+                dcsReplicas.put(dc, --replicas);
+                return true;
+            }
+            return false;
+        });
     }
 
-    public boolean isSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints)
+    public boolean isSufficientLiveNodesForRead(Keyspace keyspace, Endpoints<?> liveReplicas)
     {
         switch (this)
         {
@@ -235,75 +293,92 @@ public enum ConsistencyLevel
                 // local hint is acceptable, and local node is always live
                 return true;
             case LOCAL_ONE:
-                return countLocalEndpoints(liveEndpoints) >= 1;
+                return countDCLocalReplicas(liveReplicas).isSufficient(1, 1);
             case LOCAL_QUORUM:
-                return countLocalEndpoints(liveEndpoints) >= blockFor(keyspace);
+                return countDCLocalReplicas(liveReplicas).isSufficient(blockFor(keyspace), 1);
             case EACH_QUORUM:
                 if (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
                 {
-                    for (Map.Entry<String, Integer> entry : countPerDCEndpoints(keyspace, liveEndpoints).entrySet())
+                    int fullCount = 0;
+                    for (Map.Entry<String, ReplicaCount> entry : countPerDCEndpoints(keyspace, liveReplicas).entrySet())
                     {
-                        if (entry.getValue() < localQuorumFor(keyspace, entry.getKey()))
+                        ReplicaCount count = entry.getValue();
+                        if (!count.isSufficient(localQuorumFor(keyspace, entry.getKey()), 0))
                             return false;
+                        fullCount += count.fullReplicas;
                     }
-                    return true;
+                    return fullCount > 0;
                 }
                 // Fallthough on purpose for SimpleStrategy
             default:
-                return Iterables.size(liveEndpoints) >= blockFor(keyspace);
+                return liveReplicas.size() >= blockFor(keyspace)
+                        && Replicas.countFull(liveReplicas) > 0;
         }
     }
 
-    public void assureSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddressAndPort> liveEndpoints) throws UnavailableException
+    public void assureSufficientLiveNodesForRead(Keyspace keyspace, Endpoints<?> liveReplicas) throws UnavailableException
+    {
+        assureSufficientLiveNodes(keyspace, liveReplicas, blockFor(keyspace), 1);
+    }
+    public void assureSufficientLiveNodesForWrite(Keyspace keyspace, Endpoints<?> allLive, Endpoints<?> pendingWithDown) throws UnavailableException
+    {
+        assureSufficientLiveNodes(keyspace, allLive, blockForWrite(keyspace, pendingWithDown), 0);
+    }
+    public void assureSufficientLiveNodes(Keyspace keyspace, Endpoints<?> allLive, int blockFor, int blockForFullReplicas) throws UnavailableException
     {
-        int blockFor = blockFor(keyspace);
         switch (this)
         {
             case ANY:
                 // local hint is acceptable, and local node is always live
                 break;
             case LOCAL_ONE:
-                if (countLocalEndpoints(liveEndpoints) == 0)
-                    throw new UnavailableException(this, 1, 0);
+            {
+                ReplicaCount localLive = countDCLocalReplicas(allLive);
+                if (!localLive.isSufficient(blockFor, blockForFullReplicas))
+                    throw UnavailableException.create(this, 1, blockForFullReplicas, localLive.allReplicas(), localLive.fullReplicas);
                 break;
+            }
             case LOCAL_QUORUM:
-                int localLive = countLocalEndpoints(liveEndpoints);
-                if (localLive < blockFor)
+            {
+                ReplicaCount localLive = countDCLocalReplicas(allLive);
+                if (!localLive.isSufficient(blockFor, blockForFullReplicas))
                 {
                     if (logger.isTraceEnabled())
                     {
-                        StringBuilder builder = new StringBuilder("Local replicas [");
-                        for (InetAddressAndPort endpoint : liveEndpoints)
-                        {
-                            if (isLocal(endpoint))
-                                builder.append(endpoint).append(",");
-                        }
-                        builder.append("] are insufficient to satisfy LOCAL_QUORUM requirement of ").append(blockFor).append(" live nodes in '").append(DatabaseDescriptor.getLocalDataCenter()).append("'");
-                        logger.trace(builder.toString());
+                        logger.trace(String.format("Local replicas %s are insufficient to satisfy LOCAL_QUORUM requirement of %d live replicas and %d full replicas in '%s'",
+                                allLive.filter(ConsistencyLevel::isLocal), blockFor, blockForFullReplicas, DatabaseDescriptor.getLocalDataCenter()));
                     }
-                    throw new UnavailableException(this, blockFor, localLive);
+                    throw UnavailableException.create(this, blockFor, blockForFullReplicas, localLive.allReplicas(), localLive.fullReplicas);
                 }
                 break;
+            }
             case EACH_QUORUM:
                 if (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
                 {
-                    for (Map.Entry<String, Integer> entry : countPerDCEndpoints(keyspace, liveEndpoints).entrySet())
+                    int total = 0;
+                    int totalFull = 0;
+                    for (Map.Entry<String, ReplicaCount> entry : countPerDCEndpoints(keyspace, allLive).entrySet())
                     {
                         int dcBlockFor = localQuorumFor(keyspace, entry.getKey());
-                        int dcLive = entry.getValue();
-                        if (dcLive < dcBlockFor)
-                            throw new UnavailableException(this, entry.getKey(), dcBlockFor, dcLive);
+                        ReplicaCount dcCount = entry.getValue();
+                        if (!dcCount.isSufficient(dcBlockFor, 0))
+                            throw UnavailableException.create(this, entry.getKey(), dcBlockFor, dcCount.allReplicas(), 0, dcCount.fullReplicas);
+                        totalFull += dcCount.fullReplicas;
+                        total += dcCount.allReplicas();
                     }
+                    if (totalFull < blockForFullReplicas)
+                        throw UnavailableException.create(this, blockFor, total, blockForFullReplicas, totalFull);
                     break;
                 }
                 // Fallthough on purpose for SimpleStrategy
             default:
-                int live = Iterables.size(liveEndpoints);
-                if (live < blockFor)
+                int live = allLive.size();
+                int full = Replicas.countFull(allLive);
+                if (live < blockFor || full < blockForFullReplicas)
                 {
                     if (logger.isTraceEnabled())
-                        logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(liveEndpoints), blockFor);
-                    throw new UnavailableException(this, blockFor, live);
+                        logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(allLive), blockFor);
+                    throw UnavailableException.create(this, blockFor, blockForFullReplicas, live, full);
                 }
                 break;
         }


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


[14/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index db73b4f..8eb8603 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -21,6 +21,9 @@ import java.util.Collection;
 import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.RowIndexEntry;
@@ -85,13 +88,15 @@ public class BigFormat implements SSTableFormat
                                   long keyCount,
                                   long repairedAt,
                                   UUID pendingRepair,
+                                  boolean isTransient,
                                   TableMetadataRef metadata,
                                   MetadataCollector metadataCollector,
                                   SerializationHeader header,
                                   Collection<SSTableFlushObserver> observers,
                                   LifecycleTransaction txn)
         {
-            return new BigTableWriter(descriptor, keyCount, repairedAt, pendingRepair, metadata, metadataCollector, header, observers, txn);
+            SSTable.validateRepairedMetadata(repairedAt, pendingRepair, isTransient);
+            return new BigTableWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers, txn);
         }
     }
 
@@ -120,7 +125,7 @@ public class BigFormat implements SSTableFormat
         // mb (3.0.7, 3.7): commit log lower bound included
         // mc (3.0.8, 3.9): commit log intervals included
 
-        // na (4.0.0): uncompressed chunks, pending repair session, checksummed sstable metadata file, new Bloomfilter format
+        // na (4.0.0): uncompressed chunks, pending repair session, isTransient, checksummed sstable metadata file, new Bloomfilter format
         //
         // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
 
@@ -131,6 +136,7 @@ public class BigFormat implements SSTableFormat
         public final boolean hasMaxCompressedLength;
         private final boolean hasPendingRepair;
         private final boolean hasMetadataChecksum;
+        private final boolean hasIsTransient;
         /**
          * CASSANDRA-9067: 4.0 bloom filter representation changed (two longs just swapped)
          * have no 'static' bits caused by using the same upper bits for both bloom filter and token distribution.
@@ -148,6 +154,7 @@ public class BigFormat implements SSTableFormat
             hasCommitLogIntervals = version.compareTo("mc") >= 0;
             hasMaxCompressedLength = version.compareTo("na") >= 0;
             hasPendingRepair = version.compareTo("na") >= 0;
+            hasIsTransient = version.compareTo("na") >= 0;
             hasMetadataChecksum = version.compareTo("na") >= 0;
             hasOldBfFormat = version.compareTo("na") < 0;
         }
@@ -176,6 +183,12 @@ public class BigFormat implements SSTableFormat
         }
 
         @Override
+        public boolean hasIsTransient()
+        {
+            return hasIsTransient;
+        }
+
+        @Override
         public int correspondingMessagingVersion()
         {
             return correspondingMessagingVersion;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index b5488ed..7513e95 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -68,13 +68,14 @@ public class BigTableWriter extends SSTableWriter
                           long keyCount,
                           long repairedAt,
                           UUID pendingRepair,
+                          boolean isTransient,
                           TableMetadataRef metadata,
                           MetadataCollector metadataCollector, 
                           SerializationHeader header,
                           Collection<SSTableFlushObserver> observers,
                           LifecycleTransaction txn)
     {
-        super(descriptor, keyCount, repairedAt, pendingRepair, metadata, metadataCollector, header, observers);
+        super(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers);
         txn.trackNew(this); // must track before any files are created
 
         if (compression)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
index 6a40d94..eb7b2c7 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
@@ -71,7 +71,7 @@ public interface IMetadataSerializer
     void mutateLevel(Descriptor descriptor, int newLevel) throws IOException;
 
     /**
-     * Mutate the repairedAt time and pendingRepair ID
+     * Mutate the repairedAt time, pendingRepair ID, and transient status
      */
-    void mutateRepaired(Descriptor descriptor, long newRepairedAt, UUID newPendingRepair) throws IOException;
+    public void mutateRepairMetadata(Descriptor descriptor, long newRepairedAt, UUID newPendingRepair, boolean isTransient) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 9d9c1a8..36c218b 100755
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -83,7 +83,8 @@ public class MetadataCollector implements PartitionStatisticsCollector
                                  ActiveRepairService.UNREPAIRED_SSTABLE,
                                  -1,
                                  -1,
-                                 null);
+                                 null,
+                                 false);
     }
 
     protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
@@ -272,7 +273,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
         this.hasLegacyCounterShards = this.hasLegacyCounterShards || hasLegacyCounterShards;
     }
 
-    public Map<MetadataType, MetadataComponent> finalizeMetadata(String partitioner, double bloomFilterFPChance, long repairedAt, UUID pendingRepair, SerializationHeader header)
+    public Map<MetadataType, MetadataComponent> finalizeMetadata(String partitioner, double bloomFilterFPChance, long repairedAt, UUID pendingRepair, boolean isTransient, SerializationHeader header)
     {
         Map<MetadataType, MetadataComponent> components = new EnumMap<>(MetadataType.class);
         components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
@@ -294,7 +295,8 @@ public class MetadataCollector implements PartitionStatisticsCollector
                                                              repairedAt,
                                                              totalColumnsSet,
                                                              totalRows,
-                                                             pendingRepair));
+                                                             pendingRepair,
+                                                             isTransient));
         components.put(MetadataType.COMPACTION, new CompactionMetadata(cardinality));
         components.put(MetadataType.HEADER, header.toComponent());
         return components;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 74923a0..f76db2d 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -230,7 +230,7 @@ public class MetadataSerializer implements IMetadataSerializer
         rewriteSSTableMetadata(descriptor, currentComponents);
     }
 
-    public void mutateRepaired(Descriptor descriptor, long newRepairedAt, UUID newPendingRepair) throws IOException
+    public void mutateRepairMetadata(Descriptor descriptor, long newRepairedAt, UUID newPendingRepair, boolean isTransient) throws IOException
     {
         if (logger.isTraceEnabled())
             logger.trace("Mutating {} to repairedAt time {} and pendingRepair {}",
@@ -238,7 +238,7 @@ public class MetadataSerializer implements IMetadataSerializer
         Map<MetadataType, MetadataComponent> currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class));
         StatsMetadata stats = (StatsMetadata) currentComponents.remove(MetadataType.STATS);
         // mutate time & id
-        currentComponents.put(MetadataType.STATS, stats.mutateRepairedAt(newRepairedAt).mutatePendingRepair(newPendingRepair));
+        currentComponents.put(MetadataType.STATS, stats.mutateRepairedMetadata(newRepairedAt, newPendingRepair, isTransient));
         rewriteSSTableMetadata(descriptor, currentComponents);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index 2b8ebef..f14fb5d 100755
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@ -64,6 +64,7 @@ public class StatsMetadata extends MetadataComponent
     public final long totalColumnsSet;
     public final long totalRows;
     public final UUID pendingRepair;
+    public final boolean isTransient;
 
     public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                          EstimatedHistogram estimatedColumnCount,
@@ -83,7 +84,8 @@ public class StatsMetadata extends MetadataComponent
                          long repairedAt,
                          long totalColumnsSet,
                          long totalRows,
-                         UUID pendingRepair)
+                         UUID pendingRepair,
+                         boolean isTransient)
     {
         this.estimatedPartitionSize = estimatedPartitionSize;
         this.estimatedColumnCount = estimatedColumnCount;
@@ -104,6 +106,7 @@ public class StatsMetadata extends MetadataComponent
         this.totalColumnsSet = totalColumnsSet;
         this.totalRows = totalRows;
         this.pendingRepair = pendingRepair;
+        this.isTransient = isTransient;
     }
 
     public MetadataType getType()
@@ -155,10 +158,11 @@ public class StatsMetadata extends MetadataComponent
                                  repairedAt,
                                  totalColumnsSet,
                                  totalRows,
-                                 pendingRepair);
+                                 pendingRepair,
+                                 isTransient);
     }
 
-    public StatsMetadata mutateRepairedAt(long newRepairedAt)
+    public StatsMetadata mutateRepairedMetadata(long newRepairedAt, UUID newPendingRepair, boolean newIsTransient)
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
@@ -178,30 +182,8 @@ public class StatsMetadata extends MetadataComponent
                                  newRepairedAt,
                                  totalColumnsSet,
                                  totalRows,
-                                 pendingRepair);
-    }
-
-    public StatsMetadata mutatePendingRepair(UUID newPendingRepair)
-    {
-        return new StatsMetadata(estimatedPartitionSize,
-                                 estimatedColumnCount,
-                                 commitLogIntervals,
-                                 minTimestamp,
-                                 maxTimestamp,
-                                 minLocalDeletionTime,
-                                 maxLocalDeletionTime,
-                                 minTTL,
-                                 maxTTL,
-                                 compressionRatio,
-                                 estimatedTombstoneDropTime,
-                                 sstableLevel,
-                                 minClusteringValues,
-                                 maxClusteringValues,
-                                 hasLegacyCounterShards,
-                                 repairedAt,
-                                 totalColumnsSet,
-                                 totalRows,
-                                 newPendingRepair);
+                                 newPendingRepair,
+                                 newIsTransient);
     }
 
     @Override
@@ -292,6 +274,12 @@ public class StatsMetadata extends MetadataComponent
                 if (component.pendingRepair != null)
                     size += UUIDSerializer.serializer.serializedSize(component.pendingRepair, 0);
             }
+
+            if (version.hasIsTransient())
+            {
+                size += TypeSizes.sizeof(component.isTransient);
+            }
+
             return size;
         }
 
@@ -338,6 +326,11 @@ public class StatsMetadata extends MetadataComponent
                     out.writeByte(0);
                 }
             }
+
+            if (version.hasIsTransient())
+            {
+                out.writeBoolean(component.isTransient);
+            }
         }
 
         public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@ -386,6 +379,8 @@ public class StatsMetadata extends MetadataComponent
                 pendingRepair = UUIDSerializer.serializer.deserialize(in, 0);
             }
 
+            boolean isTransient = version.hasIsTransient() && in.readBoolean();
+
             return new StatsMetadata(partitionSizes,
                                      columnCounts,
                                      commitLogIntervals,
@@ -404,7 +399,8 @@ public class StatsMetadata extends MetadataComponent
                                      repairedAt,
                                      totalColumnsSet,
                                      totalRows,
-                                     pendingRepair);
+                                     pendingRepair,
+                                     isTransient);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
index 2ee8eea..2e7408b 100644
--- a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java
@@ -17,13 +17,12 @@
  */
 package org.apache.cassandra.locator;
 
-import java.util.*;
-
+import com.google.common.collect.Iterables;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 public abstract class AbstractEndpointSnitch implements IEndpointSnitch
 {
-    public abstract int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2);
+    public abstract int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2);
 
     /**
      * Sorts the <tt>Collection</tt> of node addresses by proximity to the given address
@@ -31,27 +30,9 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
      * @param unsortedAddress the nodes to sort
      * @return a new sorted <tt>List</tt>
      */
-    public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress)
-    {
-        List<InetAddressAndPort> preferred = new ArrayList<>(unsortedAddress);
-        sortByProximity(address, preferred);
-        return preferred;
-    }
-
-    /**
-     * Sorts the <tt>List</tt> of node addresses, in-place, by proximity to the given address
-     * @param address the address to sort the proximity by
-     * @param addresses the nodes to sort
-     */
-    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
+    public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddress)
     {
-        Collections.sort(addresses, new Comparator<InetAddressAndPort>()
-        {
-            public int compare(InetAddressAndPort a1, InetAddressAndPort a2)
-            {
-                return compareEndpoints(address, a1, a2);
-            }
-        });
+        return unsortedAddress.sorted((r1, r2) -> compareEndpoints(address, r1, r2));
     }
 
     public void gossiperStarting()
@@ -59,7 +40,7 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
         // noop by default
     }
 
-    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
+    public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2)
     {
         // Querying remote DC is likely to be an order of magnitude slower than
         // querying locally, so 2 queries to local nodes is likely to still be
@@ -70,14 +51,9 @@ public abstract class AbstractEndpointSnitch implements IEndpointSnitch
              : true;
     }
 
-    private boolean hasRemoteNode(List<InetAddressAndPort> l)
+    private boolean hasRemoteNode(ReplicaCollection<?> l)
     {
         String localDc = DatabaseDescriptor.getLocalDataCenter();
-        for (InetAddressAndPort ep : l)
-        {
-            if (!localDc.equals(getDatacenter(ep)))
-                return true;
-        }
-        return false;
+        return Iterables.any(l, replica -> !localDc.equals(getDatacenter(replica)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
index e91f6ac..08c41f0 100644
--- a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
+++ b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java
@@ -37,8 +37,11 @@ public abstract class AbstractNetworkTopologySnitch extends AbstractEndpointSnit
      */
     abstract public String getDatacenter(InetAddressAndPort endpoint);
 
-    public int compareEndpoints(InetAddressAndPort address, InetAddressAndPort a1, InetAddressAndPort a2)
+    @Override
+    public int compareEndpoints(InetAddressAndPort address, Replica r1, Replica r2)
     {
+        InetAddressAndPort a1 = r1.endpoint();
+        InetAddressAndPort a2 = r2.endpoint();
         if (address.equals(a1) && !address.equals(a2))
             return -1;
         if (address.equals(a2) && !address.equals(a1))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/AbstractReplicaCollection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicaCollection.java b/src/java/org/apache/cassandra/locator/AbstractReplicaCollection.java
new file mode 100644
index 0000000..ecf1296
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicaCollection.java
@@ -0,0 +1,264 @@
+/*
+ * 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 com.google.common.collect.Iterables;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.BinaryOperator;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collector;
+import java.util.stream.Stream;
+
+/**
+ * A collection like class for Replica objects. Since the Replica class contains inetaddress, range, and
+ * transient replication status, basic contains and remove methods can be ambiguous. Replicas forces you
+ * to be explicit about what you're checking the container for, or removing from it.
+ */
+public abstract class AbstractReplicaCollection<C extends AbstractReplicaCollection<C>> implements ReplicaCollection<C>
+{
+    protected static final List<Replica> EMPTY_LIST = new ArrayList<>(); // since immutable, can safely return this to avoid megamorphic callsites
+
+    public static <C extends ReplicaCollection<C>, B extends Builder<C, ?, B>> Collector<Replica, B, C> collector(Set<Collector.Characteristics> characteristics, Supplier<B> supplier)
+    {
+        return new Collector<Replica, B, C>()
+        {
+            private final BiConsumer<B, Replica> accumulator = Builder::add;
+            private final BinaryOperator<B> combiner = (a, b) -> { a.addAll(b.mutable); return a; };
+            private final Function<B, C> finisher = Builder::build;
+            public Supplier<B> supplier() { return supplier; }
+            public BiConsumer<B, Replica> accumulator() { return accumulator; }
+            public BinaryOperator<B> combiner() { return combiner; }
+            public Function<B, C> finisher() { return finisher; }
+            public Set<Characteristics> characteristics() { return characteristics; }
+        };
+    }
+
+    protected final List<Replica> list;
+    protected final boolean isSnapshot;
+    protected AbstractReplicaCollection(List<Replica> list, boolean isSnapshot)
+    {
+        this.list = list;
+        this.isSnapshot = isSnapshot;
+    }
+
+    // if subList == null, should return self (or a clone thereof)
+    protected abstract C snapshot(List<Replica> subList);
+    protected abstract C self();
+    /**
+     * construct a new Mutable of our own type, so that we can concatenate
+     * TODO: this isn't terribly pretty, but we need sometimes to select / merge two Endpoints of unknown type;
+     */
+    public abstract Mutable<C> newMutable(int initialCapacity);
+
+
+    public C snapshot()
+    {
+        return isSnapshot ? self()
+                          : snapshot(list.isEmpty() ? EMPTY_LIST
+                                                    : new ArrayList<>(list));
+    }
+
+    public final C subList(int start, int end)
+    {
+        List<Replica> subList;
+        if (isSnapshot)
+        {
+            if (start == 0 && end == size()) return self();
+            else if (start == end) subList = EMPTY_LIST;
+            else subList = list.subList(start, end);
+        }
+        else
+        {
+            if (start == end) subList = EMPTY_LIST;
+            else subList = new ArrayList<>(list.subList(start, end)); // TODO: we could take a subList here, but comodification checks stop us
+        }
+        return snapshot(subList);
+    }
+
+    public final C filter(Predicate<Replica> predicate)
+    {
+        return filter(predicate, Integer.MAX_VALUE);
+    }
+
+    public final C filter(Predicate<Replica> predicate, int limit)
+    {
+        if (isEmpty())
+            return snapshot();
+
+        List<Replica> copy = null;
+        int beginRun = -1, endRun = -1;
+        int i = 0;
+        for (; i < list.size() ; ++i)
+        {
+            Replica replica = list.get(i);
+            if (predicate.test(replica))
+            {
+                if (copy != null)
+                    copy.add(replica);
+                else if (beginRun < 0)
+                    beginRun = i;
+                else if (endRun > 0)
+                {
+                    copy = new ArrayList<>(Math.min(limit, (list.size() - i) + (endRun - beginRun)));
+                    for (int j = beginRun ; j < endRun ; ++j)
+                        copy.add(list.get(j));
+                    copy.add(list.get(i));
+                }
+                if (--limit == 0)
+                {
+                    ++i;
+                    break;
+                }
+            }
+            else if (beginRun >= 0 && endRun < 0)
+                endRun = i;
+        }
+
+        if (beginRun < 0)
+            beginRun = endRun = 0;
+        if (endRun < 0)
+            endRun = i;
+        if (copy == null)
+            return subList(beginRun, endRun);
+        return snapshot(copy);
+    }
+
+    public final class Select
+    {
+        private final List<Replica> result;
+        public Select(int expectedSize)
+        {
+            this.result = new ArrayList<>(expectedSize);
+        }
+
+        /**
+         * Add matching replica to the result; this predicate should be mutually exclusive with all prior predicates.
+         * Stop once we have targetSize replicas in total, including preceding calls
+         */
+        public Select add(Predicate<Replica> predicate, int targetSize)
+        {
+            assert !Iterables.any(result, predicate::test);
+            for (int i = 0 ; result.size() < targetSize && i < list.size() ; ++i)
+                if (predicate.test(list.get(i)))
+                    result.add(list.get(i));
+            return this;
+        }
+        public Select add(Predicate<Replica> predicate)
+        {
+            return add(predicate, Integer.MAX_VALUE);
+        }
+        public C get()
+        {
+            return snapshot(result);
+        }
+    }
+
+    /**
+     * An efficient method for selecting a subset of replica via a sequence of filters.
+     *
+     * Example: select().add(filter1).add(filter2, 3).get();
+     *
+     * @return a Select object
+     */
+    public final Select select()
+    {
+        return select(list.size());
+    }
+    public final Select select(int expectedSize)
+    {
+        return new Select(expectedSize);
+    }
+
+    public final C sorted(Comparator<Replica> comparator)
+    {
+        List<Replica> copy = new ArrayList<>(list);
+        copy.sort(comparator);
+        return snapshot(copy);
+    }
+
+    public final Replica get(int i)
+    {
+        return list.get(i);
+    }
+
+    public final int size()
+    {
+        return list.size();
+    }
+
+    public final boolean isEmpty()
+    {
+        return list.isEmpty();
+    }
+
+    public final Iterator<Replica> iterator()
+    {
+        return list.iterator();
+    }
+
+    public final Stream<Replica> stream() { return list.stream(); }
+
+    public final boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (!(o instanceof AbstractReplicaCollection<?>))
+        {
+            if (!(o instanceof ReplicaCollection<?>))
+                return false;
+
+            ReplicaCollection<?> that = (ReplicaCollection<?>) o;
+            return Iterables.elementsEqual(this, that);
+        }
+        AbstractReplicaCollection<?> that = (AbstractReplicaCollection<?>) o;
+        return Objects.equals(list, that.list);
+    }
+
+    public final int hashCode()
+    {
+        return list.hashCode();
+    }
+
+    @Override
+    public final String toString()
+    {
+        return list.toString();
+    }
+
+    static <C extends AbstractReplicaCollection<C>> C concat(C replicas, C extraReplicas, Mutable.Conflict ignoreConflicts)
+    {
+        if (extraReplicas.isEmpty())
+            return replicas;
+        if (replicas.isEmpty())
+            return extraReplicas;
+        Mutable<C> mutable = replicas.newMutable(replicas.size() + extraReplicas.size());
+        mutable.addAll(replicas);
+        mutable.addAll(extraReplicas, ignoreConflicts);
+        return mutable.asImmutableView();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 3e9b5bb..0ddc0a4 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -22,8 +22,8 @@ import java.lang.reflect.InvocationTargetException;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
+import com.google.common.base.Preconditions;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,9 +73,9 @@ public abstract class AbstractReplicationStrategy
         // lazy-initialize keyspace itself since we don't create them until after the replication strategies
     }
 
-    private final Map<Token, ArrayList<InetAddressAndPort>> cachedEndpoints = new NonBlockingHashMap<Token, ArrayList<InetAddressAndPort>>();
+    private final Map<Token, EndpointsForRange> cachedReplicas = new NonBlockingHashMap<>();
 
-    public ArrayList<InetAddressAndPort> getCachedEndpoints(Token t)
+    public EndpointsForRange getCachedReplicas(Token t)
     {
         long lastVersion = tokenMetadata.getRingVersion();
 
@@ -86,13 +86,13 @@ public abstract class AbstractReplicationStrategy
                 if (lastVersion > lastInvalidatedVersion)
                 {
                     logger.trace("clearing cached endpoints");
-                    cachedEndpoints.clear();
+                    cachedReplicas.clear();
                     lastInvalidatedVersion = lastVersion;
                 }
             }
         }
 
-        return cachedEndpoints.get(t);
+        return cachedReplicas.get(t);
     }
 
     /**
@@ -102,64 +102,65 @@ public abstract class AbstractReplicationStrategy
      * @param searchPosition the position the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
      */
-    public ArrayList<InetAddressAndPort> getNaturalEndpoints(RingPosition searchPosition)
+    public EndpointsForToken getNaturalReplicasForToken(RingPosition searchPosition)
+    {
+        return getNaturalReplicas(searchPosition).forToken(searchPosition.getToken());
+    }
+
+    public EndpointsForRange getNaturalReplicas(RingPosition searchPosition)
     {
         Token searchToken = searchPosition.getToken();
         Token keyToken = TokenMetadata.firstToken(tokenMetadata.sortedTokens(), searchToken);
-        ArrayList<InetAddressAndPort> endpoints = getCachedEndpoints(keyToken);
+        EndpointsForRange endpoints = getCachedReplicas(keyToken);
         if (endpoints == null)
         {
             TokenMetadata tm = tokenMetadata.cachedOnlyTokenMap();
             // if our cache got invalidated, it's possible there is a new token to account for too
             keyToken = TokenMetadata.firstToken(tm.sortedTokens(), searchToken);
-            endpoints = new ArrayList<InetAddressAndPort>(calculateNaturalEndpoints(searchToken, tm));
-            cachedEndpoints.put(keyToken, endpoints);
+            endpoints = calculateNaturalReplicas(searchToken, tm);
+            cachedReplicas.put(keyToken, endpoints);
         }
 
-        return new ArrayList<InetAddressAndPort>(endpoints);
+        return endpoints;
     }
 
     /**
      * calculate the natural endpoints for the given token
      *
-     * @see #getNaturalEndpoints(org.apache.cassandra.dht.RingPosition)
+     * @see #getNaturalReplicasForToken(org.apache.cassandra.dht.RingPosition)
      *
      * @param searchToken the token the natural endpoints are requested for
      * @return a copy of the natural endpoints for the given token
      */
-    public abstract List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata);
+    public abstract EndpointsForRange calculateNaturalReplicas(Token searchToken, TokenMetadata tokenMetadata);
 
-    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
-                                                                       Collection<InetAddressAndPort> pendingEndpoints,
-                                                                       ConsistencyLevel consistency_level,
+    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                                                        Runnable callback,
                                                                        WriteType writeType,
                                                                        long queryStartNanoTime)
     {
-        return getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType, queryStartNanoTime, DatabaseDescriptor.getIdealConsistencyLevel());
+        return getWriteResponseHandler(replicaLayout, callback, writeType, queryStartNanoTime, DatabaseDescriptor.getIdealConsistencyLevel());
     }
 
-    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
-                                                                       Collection<InetAddressAndPort> pendingEndpoints,
-                                                                       ConsistencyLevel consistency_level,
+    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                                                        Runnable callback,
                                                                        WriteType writeType,
                                                                        long queryStartNanoTime,
                                                                        ConsistencyLevel idealConsistencyLevel)
     {
         AbstractWriteResponseHandler resultResponseHandler;
-        if (consistency_level.isDatacenterLocal())
+        if (replicaLayout.consistencyLevel.isDatacenterLocal())
         {
             // block for in this context will be localnodes block.
-            resultResponseHandler = new DatacenterWriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType, queryStartNanoTime);
+            resultResponseHandler = new DatacenterWriteResponseHandler<T>(replicaLayout, callback, writeType, queryStartNanoTime);
         }
-        else if (consistency_level == ConsistencyLevel.EACH_QUORUM && (this instanceof NetworkTopologyStrategy))
+        else if (replicaLayout.consistencyLevel == ConsistencyLevel.EACH_QUORUM && (this instanceof NetworkTopologyStrategy))
         {
-            resultResponseHandler = new DatacenterSyncWriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType, queryStartNanoTime);
+            resultResponseHandler = new DatacenterSyncWriteResponseHandler<T>(replicaLayout, callback, writeType, queryStartNanoTime);
         }
         else
         {
-            resultResponseHandler = new WriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType, queryStartNanoTime);
+            resultResponseHandler = new WriteResponseHandler<T>(replicaLayout, callback, writeType, queryStartNanoTime);
         }
 
         //Check if tracking the ideal consistency level is configured
@@ -168,16 +169,14 @@ public abstract class AbstractReplicationStrategy
             //If ideal and requested are the same just use this handler to track the ideal consistency level
             //This is also used so that the ideal consistency level handler when constructed knows it is the ideal
             //one for tracking purposes
-            if (idealConsistencyLevel == consistency_level)
+            if (idealConsistencyLevel == replicaLayout.consistencyLevel)
             {
                 resultResponseHandler.setIdealCLResponseHandler(resultResponseHandler);
             }
             else
             {
                 //Construct a delegate response handler to use to track the ideal consistency level
-                AbstractWriteResponseHandler idealHandler = getWriteResponseHandler(naturalEndpoints,
-                                                                                    pendingEndpoints,
-                                                                                    idealConsistencyLevel,
+                AbstractWriteResponseHandler idealHandler = getWriteResponseHandler(replicaLayout.withConsistencyLevel(idealConsistencyLevel),
                                                                                     callback,
                                                                                     writeType,
                                                                                     queryStartNanoTime,
@@ -202,7 +201,12 @@ public abstract class AbstractReplicationStrategy
      *
      * @return the replication factor
      */
-    public abstract int getReplicationFactor();
+    public abstract ReplicationFactor getReplicationFactor();
+
+    public boolean hasTransientReplicas()
+    {
+        return getReplicationFactor().hasTransientReplicas();
+    }
 
     /*
      * NOTE: this is pretty inefficient. also the inverse (getRangeAddresses) below.
@@ -210,53 +214,81 @@ public abstract class AbstractReplicationStrategy
      * (fixing this would probably require merging tokenmetadata into replicationstrategy,
      * so we could cache/invalidate cleanly.)
      */
-    public Multimap<InetAddressAndPort, Range<Token>> getAddressRanges(TokenMetadata metadata)
+    public RangesByEndpoint getAddressReplicas(TokenMetadata metadata)
     {
-        Multimap<InetAddressAndPort, Range<Token>> map = HashMultimap.create();
+        RangesByEndpoint.Mutable map = new RangesByEndpoint.Mutable();
 
         for (Token token : metadata.sortedTokens())
         {
             Range<Token> range = metadata.getPrimaryRangeFor(token);
-            for (InetAddressAndPort ep : calculateNaturalEndpoints(token, metadata))
+            for (Replica replica : calculateNaturalReplicas(token, metadata))
             {
-                map.put(ep, range);
+                // LocalStrategy always returns (min, min] ranges for it's replicas, so we skip the check here
+                Preconditions.checkState(range.equals(replica.range()) || this instanceof LocalStrategy);
+                map.put(replica.endpoint(), replica);
             }
         }
 
-        return map;
+        return map.asImmutableView();
     }
 
-    public Multimap<Range<Token>, InetAddressAndPort> getRangeAddresses(TokenMetadata metadata)
+    public RangesAtEndpoint getAddressReplicas(TokenMetadata metadata, InetAddressAndPort endpoint)
     {
-        Multimap<Range<Token>, InetAddressAndPort> map = HashMultimap.create();
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(endpoint);
+        for (Token token : metadata.sortedTokens())
+        {
+            Range<Token> range = metadata.getPrimaryRangeFor(token);
+            Replica replica = calculateNaturalReplicas(token, metadata)
+                    .byEndpoint().get(endpoint);
+            if (replica != null)
+            {
+                // LocalStrategy always returns (min, min] ranges for it's replicas, so we skip the check here
+                Preconditions.checkState(range.equals(replica.range()) || this instanceof LocalStrategy);
+                builder.add(replica, Conflict.DUPLICATE);
+            }
+        }
+        return builder.build();
+    }
+
+
+    public EndpointsByRange getRangeAddresses(TokenMetadata metadata)
+    {
+        EndpointsByRange.Mutable map = new EndpointsByRange.Mutable();
 
         for (Token token : metadata.sortedTokens())
         {
             Range<Token> range = metadata.getPrimaryRangeFor(token);
-            for (InetAddressAndPort ep : calculateNaturalEndpoints(token, metadata))
+            for (Replica replica : calculateNaturalReplicas(token, metadata))
             {
-                map.put(range, ep);
+                // LocalStrategy always returns (min, min] ranges for it's replicas, so we skip the check here
+                Preconditions.checkState(range.equals(replica.range()) || this instanceof LocalStrategy);
+                map.put(range, replica);
             }
         }
 
-        return map;
+        return map.asImmutableView();
     }
 
-    public Multimap<InetAddressAndPort, Range<Token>> getAddressRanges()
+    public RangesByEndpoint getAddressReplicas()
     {
-        return getAddressRanges(tokenMetadata.cloneOnlyTokenMap());
+        return getAddressReplicas(tokenMetadata.cloneOnlyTokenMap());
     }
 
-    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Token pendingToken, InetAddressAndPort pendingAddress)
+    public RangesAtEndpoint getAddressReplicas(InetAddressAndPort endpoint)
     {
-        return getPendingAddressRanges(metadata, Arrays.asList(pendingToken), pendingAddress);
+        return getAddressReplicas(tokenMetadata.cloneOnlyTokenMap(), endpoint);
     }
 
-    public Collection<Range<Token>> getPendingAddressRanges(TokenMetadata metadata, Collection<Token> pendingTokens, InetAddressAndPort pendingAddress)
+    public RangesAtEndpoint getPendingAddressRanges(TokenMetadata metadata, Token pendingToken, InetAddressAndPort pendingAddress)
+    {
+        return getPendingAddressRanges(metadata, Collections.singleton(pendingToken), pendingAddress);
+    }
+
+    public RangesAtEndpoint getPendingAddressRanges(TokenMetadata metadata, Collection<Token> pendingTokens, InetAddressAndPort pendingAddress)
     {
         TokenMetadata temp = metadata.cloneOnlyTokenMap();
         temp.updateNormalTokens(pendingTokens, pendingAddress);
-        return getAddressRanges(temp).get(pendingAddress);
+        return getAddressReplicas(temp, pendingAddress);
     }
 
     public abstract void validateOptions() throws ConfigurationException;
@@ -329,6 +361,10 @@ public abstract class AbstractReplicationStrategy
         AbstractReplicationStrategy strategy = createInternal(keyspaceName, strategyClass, tokenMetadata, snitch, strategyOptions);
         strategy.validateExpectedOptions();
         strategy.validateOptions();
+        if (strategy.hasTransientReplicas() && !DatabaseDescriptor.isTransientReplicationEnabled())
+        {
+            throw new ConfigurationException("Transient replication is disabled. Enable in cassandra.yaml to use.");
+        }
     }
 
     public static Class<AbstractReplicationStrategy> getClass(String cls) throws ConfigurationException
@@ -344,21 +380,23 @@ public abstract class AbstractReplicationStrategy
 
     public boolean hasSameSettings(AbstractReplicationStrategy other)
     {
-        return getClass().equals(other.getClass()) && getReplicationFactor() == other.getReplicationFactor();
+        return getClass().equals(other.getClass()) && getReplicationFactor().equals(other.getReplicationFactor());
     }
 
-    protected void validateReplicationFactor(String rf) throws ConfigurationException
+    protected void validateReplicationFactor(String s) throws ConfigurationException
     {
         try
         {
-            if (Integer.parseInt(rf) < 0)
+            ReplicationFactor rf = ReplicationFactor.fromString(s);
+            if (rf.hasTransientReplicas())
             {
-                throw new ConfigurationException("Replication factor must be non-negative; found " + rf);
+                if (DatabaseDescriptor.getNumTokens() > 1)
+                    throw new ConfigurationException(String.format("Transient replication is not supported with vnodes yet"));
             }
         }
-        catch (NumberFormatException e2)
+        catch (IllegalArgumentException e)
         {
-            throw new ConfigurationException("Replication factor must be numeric; found " + rf);
+            throw new ConfigurationException(e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index 010c892..d35f1fb 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -42,7 +42,6 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
-
 /**
  * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
  */
@@ -185,55 +184,38 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         return subsnitch.getDatacenter(endpoint);
     }
 
-    public List<InetAddressAndPort> getSortedListByProximity(final InetAddressAndPort address, Collection<InetAddressAndPort> addresses)
-    {
-        List<InetAddressAndPort> list = new ArrayList<>(addresses);
-        sortByProximity(address, list);
-        return list;
-    }
-
     @Override
-    public void sortByProximity(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
+    public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses)
     {
         assert address.equals(FBUtilities.getBroadcastAddressAndPort()); // we only know about ourself
-        if (dynamicBadnessThreshold == 0)
-        {
-            sortByProximityWithScore(address, addresses);
-        }
-        else
-        {
-            sortByProximityWithBadness(address, addresses);
-        }
+        return dynamicBadnessThreshold == 0
+                ? sortedByProximityWithScore(address, unsortedAddresses)
+                : sortedByProximityWithBadness(address, unsortedAddresses);
     }
 
-    private void sortByProximityWithScore(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
+    private <C extends ReplicaCollection<? extends C>> C sortedByProximityWithScore(final InetAddressAndPort address, C unsortedAddresses)
     {
         // Scores can change concurrently from a call to this method. But Collections.sort() expects
         // its comparator to be "stable", that is 2 endpoint should compare the same way for the duration
         // of the sort() call. As we copy the scores map on write, it is thus enough to alias the current
         // version of it during this call.
         final HashMap<InetAddressAndPort, Double> scores = this.scores;
-        Collections.sort(addresses, new Comparator<InetAddressAndPort>()
-        {
-            public int compare(InetAddressAndPort a1, InetAddressAndPort a2)
-            {
-                return compareEndpoints(address, a1, a2, scores);
-            }
-        });
+        return unsortedAddresses.sorted((r1, r2) -> compareEndpoints(address, r1, r2, scores));
     }
 
-    private void sortByProximityWithBadness(final InetAddressAndPort address, List<InetAddressAndPort> addresses)
+    private <C extends ReplicaCollection<? extends C>> C sortedByProximityWithBadness(final InetAddressAndPort address, C replicas)
     {
-        if (addresses.size() < 2)
-            return;
+        if (replicas.size() < 2)
+            return replicas;
 
-        subsnitch.sortByProximity(address, addresses);
+        // TODO: avoid copy
+        replicas = subsnitch.sortedByProximity(address, replicas);
         HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below
                                                            // (which wouldn't really matter here but its cleaner that way).
-        ArrayList<Double> subsnitchOrderedScores = new ArrayList<>(addresses.size());
-        for (InetAddressAndPort inet : addresses)
+        ArrayList<Double> subsnitchOrderedScores = new ArrayList<>(replicas.size());
+        for (Replica replica : replicas)
         {
-            Double score = scores.get(inet);
+            Double score = scores.get(replica.endpoint());
             if (score == null)
                 score = 0.0;
             subsnitchOrderedScores.add(score);
@@ -250,17 +232,18 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         {
             if (subsnitchScore > (sortedScoreIterator.next() * (1.0 + dynamicBadnessThreshold)))
             {
-                sortByProximityWithScore(address, addresses);
-                return;
+                return sortedByProximityWithScore(address, replicas);
             }
         }
+
+        return replicas;
     }
 
     // Compare endpoints given an immutable snapshot of the scores
-    private int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2, Map<InetAddressAndPort, Double> scores)
+    private int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2, Map<InetAddressAndPort, Double> scores)
     {
-        Double scored1 = scores.get(a1);
-        Double scored2 = scores.get(a2);
+        Double scored1 = scores.get(a1.endpoint());
+        Double scored2 = scores.get(a2.endpoint());
         
         if (scored1 == null)
         {
@@ -280,7 +263,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
             return 1;
     }
 
-    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+    public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
     {
         // That function is fundamentally unsafe because the scores can change at any time and so the result of that
         // method is not stable for identical arguments. This is why we don't rely on super.sortByProximity() in
@@ -414,7 +397,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
         return getSeverity(FBUtilities.getBroadcastAddressAndPort());
     }
 
-    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
+    public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2)
     {
         if (!subsnitch.isWorthMergingForRangeQuery(merged, l1, l2))
             return false;
@@ -434,12 +417,12 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILa
     }
 
     // Return the max score for the endpoint in the provided list, or -1.0 if no node have a score.
-    private double maxScore(List<InetAddressAndPort> endpoints)
+    private double maxScore(ReplicaCollection<?> endpoints)
     {
         double maxScore = -1.0;
-        for (InetAddressAndPort endpoint : endpoints)
+        for (Replica replica : endpoints)
         {
-            Double score = scores.get(endpoint);
+            Double score = scores.get(replica.endpoint());
             if (score == null)
                 continue;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/Ec2Snitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Ec2Snitch.java b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
index b6aafd3..d0474e4 100644
--- a/src/java/org/apache/cassandra/locator/Ec2Snitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2Snitch.java
@@ -68,7 +68,7 @@ public class Ec2Snitch extends AbstractNetworkTopologySnitch
     {
         String az = awsApiCall(ZONE_NAME_QUERY_URL);
 
-        // if using the full naming scheme, region name is created by removing letters from the 
+        // if using the full naming scheme, region name is created by removing letters from the
         // end of the availability zone and zone is the full zone name
         usingLegacyNaming = isUsingLegacyNaming(props);
         String region;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/Endpoints.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/Endpoints.java b/src/java/org/apache/cassandra/locator/Endpoints.java
new file mode 100644
index 0000000..3d5faa4
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/Endpoints.java
@@ -0,0 +1,157 @@
+/*
+ * 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 org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
+import org.apache.cassandra.utils.FBUtilities;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public abstract class Endpoints<E extends Endpoints<E>> extends AbstractReplicaCollection<E>
+{
+    static final Map<InetAddressAndPort, Replica> EMPTY_MAP = Collections.unmodifiableMap(new LinkedHashMap<>());
+
+    volatile Map<InetAddressAndPort, Replica> byEndpoint;
+
+    Endpoints(List<Replica> list, boolean isSnapshot, Map<InetAddressAndPort, Replica> byEndpoint)
+    {
+        super(list, isSnapshot);
+        this.byEndpoint = byEndpoint;
+    }
+
+    @Override
+    public Set<InetAddressAndPort> endpoints()
+    {
+        return byEndpoint().keySet();
+    }
+
+    public Map<InetAddressAndPort, Replica> byEndpoint()
+    {
+        Map<InetAddressAndPort, Replica> map = byEndpoint;
+        if (map == null)
+            byEndpoint = map = buildByEndpoint(list);
+        return map;
+    }
+
+    public boolean contains(InetAddressAndPort endpoint, boolean isFull)
+    {
+        Replica replica = byEndpoint().get(endpoint);
+        return replica != null && replica.isFull() == isFull;
+    }
+
+    @Override
+    public boolean contains(Replica replica)
+    {
+        return replica != null
+                && Objects.equals(
+                        byEndpoint().get(replica.endpoint()),
+                        replica);
+    }
+
+    private static Map<InetAddressAndPort, Replica> buildByEndpoint(List<Replica> list)
+    {
+        // TODO: implement a delegating map that uses our superclass' list, and is immutable
+        Map<InetAddressAndPort, Replica> byEndpoint = new LinkedHashMap<>(list.size());
+        for (Replica replica : list)
+        {
+            Replica prev = byEndpoint.put(replica.endpoint(), replica);
+            assert prev == null : "duplicate endpoint in EndpointsForRange: " + prev + " and " + replica;
+        }
+
+        return Collections.unmodifiableMap(byEndpoint);
+    }
+
+    public E withoutSelf()
+    {
+        InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
+        return filter(r -> !self.equals(r.endpoint()));
+    }
+
+    public E without(Set<InetAddressAndPort> remove)
+    {
+        return filter(r -> !remove.contains(r.endpoint()));
+    }
+
+    public E keep(Set<InetAddressAndPort> keep)
+    {
+        return filter(r -> keep.contains(r.endpoint()));
+    }
+
+    public E keep(Iterable<InetAddressAndPort> endpoints)
+    {
+        ReplicaCollection.Mutable<E> copy = newMutable(
+                endpoints instanceof Collection<?>
+                        ? ((Collection<InetAddressAndPort>) endpoints).size()
+                        : size()
+        );
+        Map<InetAddressAndPort, Replica> byEndpoint = byEndpoint();
+        for (InetAddressAndPort endpoint : endpoints)
+        {
+            Replica keep = byEndpoint.get(endpoint);
+            if (keep == null)
+                continue;
+            copy.add(keep, ReplicaCollection.Mutable.Conflict.DUPLICATE);
+        }
+        return copy.asSnapshot();
+    }
+
+    /**
+     * Care must be taken to ensure no conflicting ranges occur in pending and natural.
+     * Conflicts can occur for two reasons:
+     *   1) due to lack of isolation when reading pending/natural
+     *   2) because a movement that changes the type of replication from transient to full must be handled
+     *      differently for reads and writes (with the reader treating it as transient, and writer as full)
+     *
+     * The method haveConflicts() below, and resolveConflictsInX, are used to detect and resolve any issues
+     */
+    public static <E extends Endpoints<E>> E concat(E natural, E pending)
+    {
+        return AbstractReplicaCollection.concat(natural, pending, Conflict.NONE);
+    }
+
+    public static <E extends Endpoints<E>> boolean haveConflicts(E natural, E pending)
+    {
+        Set<InetAddressAndPort> naturalEndpoints = natural.endpoints();
+        for (InetAddressAndPort pendingEndpoint : pending.endpoints())
+        {
+            if (naturalEndpoints.contains(pendingEndpoint))
+                return true;
+        }
+        return false;
+    }
+
+    // must apply first
+    public static <E extends Endpoints<E>> E resolveConflictsInNatural(E natural, E pending)
+    {
+        return natural.filter(r -> !r.isTransient() || !pending.contains(r.endpoint(), true));
+    }
+
+    // must apply second
+    public static <E extends Endpoints<E>> E resolveConflictsInPending(E natural, E pending)
+    {
+        return pending.without(natural.endpoints());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/EndpointsByRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/EndpointsByRange.java b/src/java/org/apache/cassandra/locator/EndpointsByRange.java
new file mode 100644
index 0000000..cdc8a68
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/EndpointsByRange.java
@@ -0,0 +1,63 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class EndpointsByRange extends ReplicaMultimap<Range<Token>, EndpointsForRange>
+{
+    public EndpointsByRange(Map<Range<Token>, EndpointsForRange> map)
+    {
+        super(map);
+    }
+
+    public EndpointsForRange get(Range<Token> range)
+    {
+        Preconditions.checkNotNull(range);
+        return map.getOrDefault(range, EndpointsForRange.empty(range));
+    }
+
+    public static class Mutable extends ReplicaMultimap.Mutable<Range<Token>, EndpointsForRange.Mutable>
+    {
+        @Override
+        protected EndpointsForRange.Mutable newMutable(Range<Token> range)
+        {
+            return new EndpointsForRange.Mutable(range);
+        }
+
+        // TODO: consider all ignoreDuplicates cases
+        public void putAll(Range<Token> range, EndpointsForRange replicas, Conflict ignoreConflicts)
+        {
+            get(range).addAll(replicas, ignoreConflicts);
+        }
+
+        public EndpointsByRange asImmutableView()
+        {
+            return new EndpointsByRange(Collections.unmodifiableMap(Maps.transformValues(map, EndpointsForRange.Mutable::asImmutableView)));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/EndpointsByReplica.java b/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
new file mode 100644
index 0000000..ceea2d1
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/EndpointsByReplica.java
@@ -0,0 +1,61 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class EndpointsByReplica extends ReplicaMultimap<Replica, EndpointsForRange>
+{
+    public EndpointsByReplica(Map<Replica, EndpointsForRange> map)
+    {
+        super(map);
+    }
+
+    public EndpointsForRange get(Replica range)
+    {
+        Preconditions.checkNotNull(range);
+        return map.getOrDefault(range, EndpointsForRange.empty(range.range()));
+    }
+
+    public static class Mutable extends ReplicaMultimap.Mutable<Replica, EndpointsForRange.Mutable>
+    {
+        @Override
+        protected EndpointsForRange.Mutable newMutable(Replica replica)
+        {
+            return new EndpointsForRange.Mutable(replica.range());
+        }
+
+        // TODO: consider all ignoreDuplicates cases
+        public void putAll(Replica range, EndpointsForRange replicas, Conflict ignoreConflicts)
+        {
+            map.computeIfAbsent(range, r -> newMutable(r)).addAll(replicas, ignoreConflicts);
+        }
+
+        public EndpointsByReplica asImmutableView()
+        {
+            return new EndpointsByReplica(Collections.unmodifiableMap(Maps.transformValues(map, EndpointsForRange.Mutable::asImmutableView)));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/EndpointsForRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/EndpointsForRange.java b/src/java/org/apache/cassandra/locator/EndpointsForRange.java
new file mode 100644
index 0000000..c2d8232
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/EndpointsForRange.java
@@ -0,0 +1,188 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.collect.Iterables.all;
+
+/**
+ * A ReplicaCollection where all Replica are required to cover a range that fully contains the range() defined in the builder().
+ * Endpoints are guaranteed to be unique; on construction, this is enforced unless optionally silenced (in which case
+ * only the first occurrence makes the cut).
+ */
+public class EndpointsForRange extends Endpoints<EndpointsForRange>
+{
+    private final Range<Token> range;
+    private EndpointsForRange(Range<Token> range, List<Replica> list, boolean isSnapshot)
+    {
+        this(range, list, isSnapshot, null);
+    }
+    private EndpointsForRange(Range<Token> range, List<Replica> list, boolean isSnapshot, Map<InetAddressAndPort, Replica> byEndpoint)
+    {
+        super(list, isSnapshot, byEndpoint);
+        this.range = range;
+        assert range != null;
+    }
+
+    public Range<Token> range()
+    {
+        return range;
+    }
+
+    @Override
+    public Mutable newMutable(int initialCapacity)
+    {
+        return new Mutable(range, initialCapacity);
+    }
+
+    public EndpointsForToken forToken(Token token)
+    {
+        if (!range.contains(token))
+            throw new IllegalArgumentException(token + " is not contained within " + range);
+        return new EndpointsForToken(token, list, isSnapshot, byEndpoint);
+    }
+
+    @Override
+    public EndpointsForRange self()
+    {
+        return this;
+    }
+
+    @Override
+    protected EndpointsForRange snapshot(List<Replica> snapshot)
+    {
+        if (snapshot.isEmpty()) return empty(range);
+        return new EndpointsForRange(range, snapshot, true);
+    }
+
+    public static class Mutable extends EndpointsForRange implements ReplicaCollection.Mutable<EndpointsForRange>
+    {
+        boolean hasSnapshot;
+        public Mutable(Range<Token> range) { this(range, 0); }
+        public Mutable(Range<Token> range, int capacity) { super(range, new ArrayList<>(capacity), false, new LinkedHashMap<>()); }
+
+        public void add(Replica replica, Conflict ignoreConflict)
+        {
+            if (hasSnapshot) throw new IllegalStateException();
+            Preconditions.checkNotNull(replica);
+            if (!replica.range().contains(super.range))
+                throw new IllegalArgumentException("Replica " + replica + " does not contain " + super.range);
+
+            Replica prev = super.byEndpoint.put(replica.endpoint(), replica);
+            if (prev != null)
+            {
+                super.byEndpoint.put(replica.endpoint(), prev); // restore prev
+                switch (ignoreConflict)
+                {
+                    case DUPLICATE:
+                        if (prev.equals(replica))
+                            break;
+                    case NONE:
+                        throw new IllegalArgumentException("Conflicting replica added (expected unique endpoints): " + replica + "; existing: " + prev);
+                    case ALL:
+                }
+                return;
+            }
+
+            list.add(replica);
+        }
+
+        @Override
+        public Map<InetAddressAndPort, Replica> byEndpoint()
+        {
+            // our internal map is modifiable, but it is unsafe to modify the map externally
+            // it would be possible to implement a safe modifiable map, but it is probably not valuable
+            return Collections.unmodifiableMap(super.byEndpoint());
+        }
+
+        private EndpointsForRange get(boolean isSnapshot)
+        {
+            return new EndpointsForRange(super.range, super.list, isSnapshot, Collections.unmodifiableMap(super.byEndpoint));
+        }
+
+        public EndpointsForRange asImmutableView()
+        {
+            return get(false);
+        }
+
+        public EndpointsForRange asSnapshot()
+        {
+            hasSnapshot = true;
+            return get(true);
+        }
+    }
+
+    public static class Builder extends ReplicaCollection.Builder<EndpointsForRange, Mutable, EndpointsForRange.Builder>
+    {
+        public Builder(Range<Token> range) { this(range, 0); }
+        public Builder(Range<Token> range, int capacity) { super (new Mutable(range, capacity)); }
+        public boolean containsEndpoint(InetAddressAndPort endpoint)
+        {
+            return mutable.asImmutableView().byEndpoint.containsKey(endpoint);
+        }
+    }
+
+    public static Builder builder(Range<Token> range)
+    {
+        return new Builder(range);
+    }
+    public static Builder builder(Range<Token> range, int capacity)
+    {
+        return new Builder(range, capacity);
+    }
+
+    public static EndpointsForRange empty(Range<Token> range)
+    {
+        return new EndpointsForRange(range, EMPTY_LIST, true, EMPTY_MAP);
+    }
+
+    public static EndpointsForRange of(Replica replica)
+    {
+        // we only use ArrayList or ArrayList.SubList, to ensure callsites are bimorphic
+        ArrayList<Replica> one = new ArrayList<>(1);
+        one.add(replica);
+        // we can safely use singletonMap, as we only otherwise use LinkedHashMap
+        return new EndpointsForRange(replica.range(), one, true, Collections.unmodifiableMap(Collections.singletonMap(replica.endpoint(), replica)));
+    }
+
+    public static EndpointsForRange of(Replica ... replicas)
+    {
+        return copyOf(Arrays.asList(replicas));
+    }
+
+    public static EndpointsForRange copyOf(Collection<Replica> replicas)
+    {
+        if (replicas.isEmpty())
+            throw new IllegalArgumentException("Collection must be non-empty to copy");
+        Range<Token> range = replicas.iterator().next().range();
+        assert all(replicas, r -> range.equals(r.range()));
+        return builder(range, replicas.size()).addAll(replicas).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/EndpointsForToken.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/EndpointsForToken.java b/src/java/org/apache/cassandra/locator/EndpointsForToken.java
new file mode 100644
index 0000000..f24c615
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/EndpointsForToken.java
@@ -0,0 +1,172 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import org.apache.cassandra.dht.Token;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A ReplicaCollection where all Replica are required to cover a range that fully contains the token() defined in the builder().
+ * Endpoints are guaranteed to be unique; on construction, this is enforced unless optionally silenced (in which case
+ * only the first occurrence makes the cut).
+ */
+public class EndpointsForToken extends Endpoints<EndpointsForToken>
+{
+    private final Token token;
+    private EndpointsForToken(Token token, List<Replica> list, boolean isSnapshot)
+    {
+        this(token, list, isSnapshot, null);
+    }
+
+    EndpointsForToken(Token token, List<Replica> list, boolean isSnapshot, Map<InetAddressAndPort, Replica> byEndpoint)
+    {
+        super(list, isSnapshot, byEndpoint);
+        this.token = token;
+        assert token != null;
+    }
+
+    public Token token()
+    {
+        return token;
+    }
+
+    @Override
+    public Mutable newMutable(int initialCapacity)
+    {
+        return new Mutable(token, initialCapacity);
+    }
+
+    @Override
+    public EndpointsForToken self()
+    {
+        return this;
+    }
+
+    @Override
+    protected EndpointsForToken snapshot(List<Replica> subList)
+    {
+        if (subList.isEmpty()) return empty(token);
+        return new EndpointsForToken(token, subList, true);
+    }
+
+    public static class Mutable extends EndpointsForToken implements ReplicaCollection.Mutable<EndpointsForToken>
+    {
+        boolean hasSnapshot;
+        public Mutable(Token token) { this(token, 0); }
+        public Mutable(Token token, int capacity) { super(token, new ArrayList<>(capacity), false, new LinkedHashMap<>()); }
+
+        public void add(Replica replica, Conflict ignoreConflict)
+        {
+            if (hasSnapshot) throw new IllegalStateException();
+            Preconditions.checkNotNull(replica);
+            if (!replica.range().contains(super.token))
+                throw new IllegalArgumentException("Replica " + replica + " does not contain " + super.token);
+
+            Replica prev = super.byEndpoint.put(replica.endpoint(), replica);
+            if (prev != null)
+            {
+                super.byEndpoint.put(replica.endpoint(), prev); // restore prev
+                switch (ignoreConflict)
+                {
+                    case DUPLICATE:
+                        if (prev.equals(replica))
+                            break;
+                    case NONE:
+                        throw new IllegalArgumentException("Conflicting replica added (expected unique endpoints): " + replica + "; existing: " + prev);
+                    case ALL:
+                }
+                return;
+            }
+
+            list.add(replica);
+        }
+
+        @Override
+        public Map<InetAddressAndPort, Replica> byEndpoint()
+        {
+            // our internal map is modifiable, but it is unsafe to modify the map externally
+            // it would be possible to implement a safe modifiable map, but it is probably not valuable
+            return Collections.unmodifiableMap(super.byEndpoint());
+        }
+
+        private EndpointsForToken get(boolean isSnapshot)
+        {
+            return new EndpointsForToken(super.token, super.list, isSnapshot, Collections.unmodifiableMap(super.byEndpoint));
+        }
+
+        public EndpointsForToken asImmutableView()
+        {
+            return get(false);
+        }
+
+        public EndpointsForToken asSnapshot()
+        {
+            hasSnapshot = true;
+            return get(true);
+        }
+    }
+
+    public static class Builder extends ReplicaCollection.Builder<EndpointsForToken, Mutable, EndpointsForToken.Builder>
+    {
+        public Builder(Token token) { this(token, 0); }
+        public Builder(Token token, int capacity) { super (new Mutable(token, capacity)); }
+    }
+
+    public static Builder builder(Token token)
+    {
+        return new Builder(token);
+    }
+    public static Builder builder(Token token, int capacity)
+    {
+        return new Builder(token, capacity);
+    }
+
+    public static EndpointsForToken empty(Token token)
+    {
+        return new EndpointsForToken(token, EMPTY_LIST, true, EMPTY_MAP);
+    }
+
+    public static EndpointsForToken of(Token token, Replica replica)
+    {
+        // we only use ArrayList or ArrayList.SubList, to ensure callsites are bimorphic
+        ArrayList<Replica> one = new ArrayList<>(1);
+        one.add(replica);
+        // we can safely use singletonMap, as we only otherwise use LinkedHashMap
+        return new EndpointsForToken(token, one, true, Collections.unmodifiableMap(Collections.singletonMap(replica.endpoint(), replica)));
+    }
+
+    public static EndpointsForToken of(Token token, Replica ... replicas)
+    {
+        return copyOf(token, Arrays.asList(replicas));
+    }
+
+    public static EndpointsForToken copyOf(Token token, Collection<Replica> replicas)
+    {
+        if (replicas.isEmpty()) return empty(token);
+        return builder(token, replicas.size()).addAll(replicas).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
index 63d333b..b7797b0 100644
--- a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.locator;
 
-import java.util.Collection;
-import java.util.List;
 import java.util.Set;
 
 /**
@@ -39,20 +37,20 @@ public interface IEndpointSnitch
      */
     public String getDatacenter(InetAddressAndPort endpoint);
 
-    /**
-     * returns a new <tt>List</tt> sorted by proximity to the given endpoint
-     */
-    public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress);
+    default public String getDatacenter(Replica replica)
+    {
+        return getDatacenter(replica.endpoint());
+    }
 
     /**
-     * This method will sort the <tt>List</tt> by proximity to the given address.
+     * returns a new <tt>List</tt> sorted by proximity to the given endpoint
      */
-    public void sortByProximity(InetAddressAndPort address, List<InetAddressAndPort> addresses);
+    public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C addresses);
 
     /**
      * compares two endpoints in relation to the target endpoint, returning as Comparator.compare would
      */
-    public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2);
+    public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2);
 
     /**
      * called after Gossiper instance exists immediately before it starts gossiping
@@ -63,7 +61,7 @@ public interface IEndpointSnitch
      * Returns whether for a range query doing a query against merged is likely
      * to be faster than 2 sequential queries, one against l1 followed by one against l2.
      */
-    public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2);
+    public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2);
 
     /**
      * Determine if the datacenter or rack values in the current node's snitch conflict with those passed in parameters.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
index 38a1a49..a47c72a 100644
--- a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
+++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java
@@ -25,6 +25,7 @@ import java.net.UnknownHostException;
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
 
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.FastByteOperations;
 
 /**
@@ -191,9 +192,9 @@ public final class InetAddressAndPort implements Comparable<InetAddressAndPort>,
         return InetAddressAndPort.getByAddress(InetAddress.getLoopbackAddress());
     }
 
-    public static InetAddressAndPort getLocalHost() throws UnknownHostException
+    public static InetAddressAndPort getLocalHost()
     {
-        return InetAddressAndPort.getByAddress(InetAddress.getLocalHost());
+        return FBUtilities.getLocalAddressAndPort();
     }
 
     public static void initializeDefaultPort(int port)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/LocalStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/LocalStrategy.java b/src/java/org/apache/cassandra/locator/LocalStrategy.java
index a76fe96..41cc9b0 100644
--- a/src/java/org/apache/cassandra/locator/LocalStrategy.java
+++ b/src/java/org/apache/cassandra/locator/LocalStrategy.java
@@ -17,12 +17,11 @@
  */
 package org.apache.cassandra.locator;
 
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.RingPosition;
 import org.apache.cassandra.dht.Token;
@@ -30,32 +29,40 @@ import org.apache.cassandra.utils.FBUtilities;
 
 public class LocalStrategy extends AbstractReplicationStrategy
 {
+    private static final ReplicationFactor RF = ReplicationFactor.fullOnly(1);
+    private final EndpointsForRange replicas;
+
     public LocalStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
         super(keyspaceName, tokenMetadata, snitch, configOptions);
+        replicas = EndpointsForRange.of(
+                new Replica(FBUtilities.getBroadcastAddressAndPort(),
+                        DatabaseDescriptor.getPartitioner().getMinimumToken(),
+                        DatabaseDescriptor.getPartitioner().getMinimumToken(),
+                        true
+                )
+        );
     }
 
     /**
-     * We need to override this even if we override calculateNaturalEndpoints,
+     * We need to override this even if we override calculateNaturalReplicas,
      * because the default implementation depends on token calculations but
      * LocalStrategy may be used before tokens are set up.
      */
     @Override
-    public ArrayList<InetAddressAndPort> getNaturalEndpoints(RingPosition searchPosition)
+    public EndpointsForRange getNaturalReplicas(RingPosition searchPosition)
     {
-        ArrayList<InetAddressAndPort> l = new ArrayList<InetAddressAndPort>(1);
-        l.add(FBUtilities.getBroadcastAddressAndPort());
-        return l;
+        return replicas;
     }
 
-    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
     {
-        return Collections.singletonList(FBUtilities.getBroadcastAddressAndPort());
+        return replicas;
     }
 
-    public int getReplicationFactor()
+    public ReplicationFactor getReplicationFactor()
     {
-        return 1;
+        return RF;
     }
 
     public void validateOptions() throws ConfigurationException


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


[05/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
index 447d504..374a760 100644
--- a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.db.repair;
 
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -42,6 +43,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
@@ -64,6 +67,9 @@ public class PendingAntiCompactionTest
 {
     private static final Logger logger = LoggerFactory.getLogger(PendingAntiCompactionTest.class);
     private static final Collection<Range<Token>> FULL_RANGE;
+    private static final Collection<Range<Token>> NO_RANGES = Collections.emptyList();
+    private static InetAddressAndPort local;
+
     static
     {
         DatabaseDescriptor.daemonInitialization();
@@ -77,9 +83,10 @@ public class PendingAntiCompactionTest
     private ColumnFamilyStore cfs;
 
     @BeforeClass
-    public static void setupClass()
+    public static void setupClass() throws Throwable
     {
         SchemaLoader.prepareServer();
+        local = InetAddressAndPort.getByName("127.0.0.1");
     }
 
     @Before
@@ -89,6 +96,7 @@ public class PendingAntiCompactionTest
         cfm = CreateTableStatement.parse(String.format("CREATE TABLE %s.%s (k INT PRIMARY KEY, v INT)", ks, tbl), ks).build();
         SchemaLoader.createKeyspace(ks, KeyspaceParams.simple(1), cfm);
         cfs = Schema.instance.getColumnFamilyStoreInstance(cfm.id);
+
     }
 
     private void makeSSTables(int num)
@@ -105,7 +113,7 @@ public class PendingAntiCompactionTest
 
     private static class InstrumentedAcquisitionCallback extends PendingAntiCompaction.AcquisitionCallback
     {
-        public InstrumentedAcquisitionCallback(UUID parentRepairSession, Collection<Range<Token>> ranges)
+        public InstrumentedAcquisitionCallback(UUID parentRepairSession, RangesAtEndpoint ranges)
         {
             super(parentRepairSession, ranges);
         }
@@ -155,7 +163,7 @@ public class PendingAntiCompactionTest
         ExecutorService executor = Executors.newSingleThreadExecutor();
         try
         {
-            pac = new PendingAntiCompaction(sessionID, tables, ranges, executor);
+            pac = new PendingAntiCompaction(sessionID, tables, atEndpoint(ranges, NO_RANGES), executor);
             pac.run().get();
         }
         finally
@@ -217,7 +225,7 @@ public class PendingAntiCompactionTest
         Assert.assertTrue(repaired.intersects(FULL_RANGE));
         Assert.assertTrue(unrepaired.intersects(FULL_RANGE));
 
-        repaired.descriptor.getMetadataSerializer().mutateRepaired(repaired.descriptor, 1, null);
+        repaired.descriptor.getMetadataSerializer().mutateRepairMetadata(repaired.descriptor, 1, null, false);
         repaired.reloadSSTableMetadata();
 
         PendingAntiCompaction.AcquisitionCallable acquisitionCallable = new PendingAntiCompaction.AcquisitionCallable(cfs, FULL_RANGE, UUIDGen.getTimeUUID());
@@ -243,7 +251,7 @@ public class PendingAntiCompactionTest
         Assert.assertTrue(repaired.intersects(FULL_RANGE));
         Assert.assertTrue(unrepaired.intersects(FULL_RANGE));
 
-        repaired.descriptor.getMetadataSerializer().mutateRepaired(repaired.descriptor, 0, UUIDGen.getTimeUUID());
+        repaired.descriptor.getMetadataSerializer().mutateRepairMetadata(repaired.descriptor, 0, UUIDGen.getTimeUUID(), false);
         repaired.reloadSSTableMetadata();
         Assert.assertTrue(repaired.isPendingRepair());
 
@@ -284,7 +292,7 @@ public class PendingAntiCompactionTest
         PendingAntiCompaction.AcquireResult result = acquisitionCallable.call();
         Assert.assertNotNull(result);
 
-        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), FULL_RANGE);
+        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), atEndpoint(FULL_RANGE, NO_RANGES));
         Assert.assertTrue(cb.submittedCompactions.isEmpty());
         cb.apply(Lists.newArrayList(result));
 
@@ -308,7 +316,7 @@ public class PendingAntiCompactionTest
         Assert.assertNotNull(result);
         Assert.assertEquals(Transactional.AbstractTransactional.State.IN_PROGRESS, result.txn.state());
 
-        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), FULL_RANGE);
+        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), atEndpoint(FULL_RANGE, Collections.emptyList()));
         Assert.assertTrue(cb.submittedCompactions.isEmpty());
         cb.apply(Lists.newArrayList(result, null));
 
@@ -333,7 +341,7 @@ public class PendingAntiCompactionTest
         ColumnFamilyStore cfs2 = Schema.instance.getColumnFamilyStoreInstance(Schema.instance.getTableMetadata("system", "peers").id);
         PendingAntiCompaction.AcquireResult fakeResult = new PendingAntiCompaction.AcquireResult(cfs2, null, null);
 
-        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), FULL_RANGE);
+        InstrumentedAcquisitionCallback cb = new InstrumentedAcquisitionCallback(UUIDGen.getTimeUUID(), atEndpoint(FULL_RANGE, NO_RANGES));
         Assert.assertTrue(cb.submittedCompactions.isEmpty());
         cb.apply(Lists.newArrayList(result, fakeResult));
 
@@ -359,8 +367,19 @@ public class PendingAntiCompactionTest
                                                                  true,0,
                                                                  true,
                                                                  PreviewKind.NONE);
-        CompactionManager.instance.performAnticompaction(result.cfs, FULL_RANGE, result.refs, result.txn,
-                                                         ActiveRepairService.UNREPAIRED_SSTABLE, sessionID, sessionID);
+        CompactionManager.instance.performAnticompaction(result.cfs, atEndpoint(FULL_RANGE, NO_RANGES), result.refs, result.txn, sessionID);
+
+    }
+
+    private static RangesAtEndpoint atEndpoint(Collection<Range<Token>> full, Collection<Range<Token>> trans)
+    {
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(local);
+        for (Range<Token> range : full)
+            builder.add(new Replica(local, range, true));
+
+        for (Range<Token> range : trans)
+            builder.add(new Replica(local, range, false));
 
+        return builder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java
index 8256ac6..5e44346 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java
@@ -114,6 +114,7 @@ public class CassandraOutgoingFileTest
         List<Range<Token>> requestedRanges = Arrays.asList(new Range<>(store.getPartitioner().getMinimumToken(), getTokenAtIndex(4)),
                                                          new Range<>(getTokenAtIndex(2), getTokenAtIndex(6)),
                                                          new Range<>(getTokenAtIndex(5), sstable.last.getToken()));
+        requestedRanges = Range.normalize(requestedRanges);
 
         CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(),
                                                               sstable.getPositionsForRanges(requestedRanges),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
index 86018af..b597bfe 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
@@ -33,6 +33,8 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -118,10 +120,10 @@ public class CassandraStreamManagerTest
         return Iterables.getOnlyElement(diff);
     }
 
-    private static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair) throws IOException
+    private static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair, boolean isTransient) throws IOException
     {
         Descriptor descriptor = sstable.descriptor;
-        descriptor.getMetadataSerializer().mutateRepaired(descriptor, repairedAt, pendingRepair);
+        descriptor.getMetadataSerializer().mutateRepairMetadata(descriptor, repairedAt, pendingRepair, isTransient);
         sstable.reloadSSTableMetadata();
 
     }
@@ -141,7 +143,7 @@ public class CassandraStreamManagerTest
     private Set<SSTableReader> getReadersForRange(Range<Token> range)
     {
         Collection<OutgoingStream> streams = cfs.getStreamManager().createOutgoingStreams(session(NO_PENDING_REPAIR),
-                                                                                          Collections.singleton(range),
+                                                                                          RangesAtEndpoint.toDummyList(Collections.singleton(range)),
                                                                                           NO_PENDING_REPAIR,
                                                                                           PreviewKind.NONE);
         return sstablesFromStreams(streams);
@@ -151,7 +153,7 @@ public class CassandraStreamManagerTest
     {
         IPartitioner partitioner = DatabaseDescriptor.getPartitioner();
         Collection<Range<Token>> ranges = Lists.newArrayList(new Range<Token>(partitioner.getMinimumToken(), partitioner.getMinimumToken()));
-        Collection<OutgoingStream> streams = cfs.getStreamManager().createOutgoingStreams(session(pendingRepair), ranges, pendingRepair, PreviewKind.NONE);
+        Collection<OutgoingStream> streams = cfs.getStreamManager().createOutgoingStreams(session(pendingRepair), RangesAtEndpoint.toDummyList(ranges), pendingRepair, PreviewKind.NONE);
         return sstablesFromStreams(streams);
     }
 
@@ -167,9 +169,9 @@ public class CassandraStreamManagerTest
 
         UUID pendingRepair = UUIDGen.getTimeUUID();
         long repairedAt = System.currentTimeMillis();
-        mutateRepaired(sstable2, ActiveRepairService.UNREPAIRED_SSTABLE, pendingRepair);
-        mutateRepaired(sstable3, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
-        mutateRepaired(sstable4, repairedAt, NO_PENDING_REPAIR);
+        mutateRepaired(sstable2, ActiveRepairService.UNREPAIRED_SSTABLE, pendingRepair, false);
+        mutateRepaired(sstable3, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID(), false);
+        mutateRepaired(sstable4, repairedAt, NO_PENDING_REPAIR, false);
 
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/streaming/StreamRequestTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/StreamRequestTest.java b/test/unit/org/apache/cassandra/db/streaming/StreamRequestTest.java
new file mode 100644
index 0000000..2a6cb65
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/streaming/StreamRequestTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.db.streaming;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.StreamRequest;
+
+public class StreamRequestTest
+{
+    private static InetAddressAndPort local;
+    private final String ks = "keyspace";
+    private final int version = MessagingService.current_version;
+
+    @BeforeClass
+    public static void setUp() throws Throwable
+    {
+        DatabaseDescriptor.daemonInitialization();
+        local = InetAddressAndPort.getByName("127.0.0.1");
+    }
+
+    @Test
+    public void serializationRoundTrip() throws Throwable
+    {
+        StreamRequest orig = new StreamRequest(ks,
+                                               atEndpoint(Arrays.asList(range(1, 2), range(3, 4), range(5, 6)),
+                                                          Collections.emptyList()),
+                                               atEndpoint(Collections.emptyList(),
+                                                          Arrays.asList(range(5, 6), range(7, 8))),
+                                               Arrays.asList("a", "b", "c"));
+
+        int expectedSize = (int) StreamRequest.serializer.serializedSize(orig, version);
+        try (DataOutputBuffer out = new DataOutputBuffer(expectedSize))
+        {
+            StreamRequest.serializer.serialize(orig, out, version);
+            Assert.assertEquals(expectedSize, out.buffer().limit());
+            try (DataInputBuffer in = new DataInputBuffer(out.buffer(), false))
+            {
+                StreamRequest decoded = StreamRequest.serializer.deserialize(in, version);
+
+                Assert.assertEquals(orig.keyspace, decoded.keyspace);
+                Assert.assertEquals(orig.full, decoded.full);
+                Assert.assertEquals(orig.transientReplicas, decoded.transientReplicas);
+                Assert.assertEquals(orig.columnFamilies, decoded.columnFamilies);
+            }
+        }
+    }
+
+    private static RangesAtEndpoint atEndpoint(Collection<Range<Token>> full, Collection<Range<Token>> trans)
+    {
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(local);
+        for (Range<Token> range : full)
+            builder.add(new Replica(local, range, true));
+
+        for (Range<Token> range : trans)
+            builder.add(new Replica(local, range, false));
+
+        return builder.build();
+    }
+
+    private static Range<Token> range(int l, int r)
+    {
+        return new Range<>(new ByteOrderedPartitioner.BytesToken(Integer.toString(l).getBytes()),
+                           new ByteOrderedPartitioner.BytesToken(Integer.toString(r).getBytes()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
index ec0f6b1..7eebef7 100644
--- a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
+++ b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.junit.Assert;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
@@ -76,12 +77,12 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                                        new StringToken("CA"),
-                                                                                        new StringToken("BB"));
+        Optional<Replica> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                             new StringToken("CA"),
+                                                                             new StringToken("BB"));
 
         Assert.assertTrue(naturalEndpoint.isPresent());
-        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.2"), naturalEndpoint.get());
+        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.2"), naturalEndpoint.get().endpoint());
     }
 
 
@@ -109,12 +110,12 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                                        new StringToken("CA"),
-                                                                                        new StringToken("BB"));
+        Optional<Replica> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                             new StringToken("CA"),
+                                                                             new StringToken("BB"));
 
         Assert.assertTrue(naturalEndpoint.isPresent());
-        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.1"), naturalEndpoint.get());
+        Assert.assertEquals(InetAddressAndPort.getByName("127.0.0.1"), naturalEndpoint.get().endpoint());
     }
 
     @Test
@@ -141,9 +142,9 @@ public class ViewUtilsTest
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
         Schema.instance.load(meta);
 
-        Optional<InetAddressAndPort> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                                        new StringToken("AB"),
-                                                                                        new StringToken("BB"));
+        Optional<Replica> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                             new StringToken("AB"),
+                                                                             new StringToken("BB"));
 
         Assert.assertFalse(naturalEndpoint.isPresent());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index f11cb62..8ae6853 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -18,18 +18,20 @@
 package org.apache.cassandra.dht;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.net.UnknownHostException;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
 
+import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
 
 import org.junit.AfterClass;
@@ -41,6 +43,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
@@ -60,6 +63,7 @@ public class BootStrapperTest
 {
     static IPartitioner oldPartitioner;
 
+    static Predicate<Replica> originalAlivePredicate = RangeStreamer.ALIVE_PREDICATE;
     @BeforeClass
     public static void setup() throws ConfigurationException
     {
@@ -68,12 +72,14 @@ public class BootStrapperTest
         SchemaLoader.startGossiper();
         SchemaLoader.prepareServer();
         SchemaLoader.schemaDefinition("BootStrapperTest");
+        RangeStreamer.ALIVE_PREDICATE = Predicates.alwaysTrue();
     }
 
     @AfterClass
     public static void tearDown()
     {
         DatabaseDescriptor.setPartitionerUnsafe(oldPartitioner);
+        RangeStreamer.ALIVE_PREDICATE = originalAlivePredicate;
     }
 
     @Test
@@ -82,7 +88,7 @@ public class BootStrapperTest
         final int[] clusterSizes = new int[] { 1, 3, 5, 10, 100};
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
-            int replicationFactor = Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor();
+            int replicationFactor = Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor().allReplicas;
             for (int clusterSize : clusterSizes)
                 if (clusterSize >= replicationFactor)
                     testSourceTargetComputation(keyspaceName, clusterSize, replicationFactor);
@@ -115,21 +121,25 @@ public class BootStrapperTest
             public void forceConviction(InetAddressAndPort ep) { throw new UnsupportedOperationException(); }
         };
         s.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(mockFailureDetector));
+        assertNotNull(Keyspace.open(keyspaceName));
         s.addRanges(keyspaceName, Keyspace.open(keyspaceName).getReplicationStrategy().getPendingAddressRanges(tmd, myToken, myEndpoint));
 
-        Collection<Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch = s.toFetch().get(keyspaceName);
 
-        // Check we get get RF new ranges in total
-        Set<Range<Token>> ranges = new HashSet<>();
-        for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> e : toFetch)
-            ranges.addAll(e.getValue());
+        Collection<Multimap<InetAddressAndPort, FetchReplica>> toFetch = s.toFetch().get(keyspaceName);
 
-        assertEquals(replicationFactor, ranges.size());
+        // Check we get get RF new ranges in total
+        long rangesCount = toFetch.stream()
+               .map(Multimap::values)
+               .flatMap(Collection::stream)
+               .map(f -> f.remote)
+               .map(Replica::range)
+               .count();
+        assertEquals(replicationFactor, rangesCount);
 
         // there isn't any point in testing the size of these collections for any specific size.  When a random partitioner
         // is used, they will vary.
-        assert toFetch.iterator().next().getValue().size() > 0;
-        assert !toFetch.iterator().next().getKey().equals(myEndpoint);
+        assert toFetch.stream().map(Multimap::values).flatMap(Collection::stream).count() > 0;
+        assert toFetch.stream().map(Multimap::keySet).map(Collection::stream).noneMatch(myEndpoint::equals);
         return s;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
index 78e87c1..07d6377 100644
--- a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java
@@ -25,8 +25,9 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 
-import com.google.common.collect.HashMultimap;
+import com.google.common.base.Predicate;
 import com.google.common.collect.Multimap;
+import org.apache.cassandra.locator.EndpointsByRange;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -34,6 +35,8 @@ import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.AbstractNetworkTopologySnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaUtils;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -78,14 +81,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithSingleSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.4");
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.5");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
@@ -95,14 +98,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithNonOverlappingSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.4");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.5", "127.0.0.6");
         addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.7", "127.0.0.8");
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.9", "127.0.0.10");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
@@ -112,12 +115,12 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithRFThreeReplacement() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2", "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3", "127.0.0.4");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
@@ -128,14 +131,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForMultipleRoundsComputation() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.3");
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.3", "127.0.0.2");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
@@ -150,14 +153,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForMultipleRoundsComputationWithLocalHost() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1", "127.0.0.2");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
 
@@ -170,14 +173,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForEmptyGraph() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.1");
         addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<RangeStreamer.ISourceFilter>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         //All ranges map to local host so we will not stream anything.
         assertTrue(map.isEmpty());
@@ -186,31 +189,28 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testWithNoSourceWithLocal() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.5");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
 
         //Return false for all except 127.0.0.5
-        final RangeStreamer.ISourceFilter filter = new RangeStreamer.ISourceFilter()
+        final Predicate<Replica> filter = replica ->
         {
-            public boolean shouldInclude(InetAddressAndPort endpoint)
+            try
             {
-                try
-                {
-                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.5")))
-                        return false;
-                    else
-                        return true;
-                }
-                catch (UnknownHostException e)
-                {
+                if (replica.endpoint().equals(InetAddressAndPort.getByName("127.0.0.5")))
+                    return false;
+                else
                     return true;
-                }
+            }
+            catch (UnknownHostException e)
+            {
+                return true;
             }
         };
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Arrays.asList(filter), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Arrays.asList(filter), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
 
         validateRange(rangesWithSources, map);
@@ -225,32 +225,26 @@ public class RangeFetchMapCalculatorTest
     @Test (expected = IllegalStateException.class)
     public void testWithNoLiveSource() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.5");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3");
 
-        final RangeStreamer.ISourceFilter allDeadFilter = new RangeStreamer.ISourceFilter()
-        {
-            public boolean shouldInclude(InetAddressAndPort endpoint)
-            {
-                return false;
-            }
-        };
+        final Predicate<Replica> allDeadFilter = replica -> false;
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Arrays.asList(allDeadFilter), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Arrays.asList(allDeadFilter), "Test");
         calculator.getRangeFetchMap();
     }
 
     @Test
     public void testForLocalDC() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.1", "127.0.0.3", "127.0.0.53");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1", "127.0.0.3", "127.0.0.57");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59", "127.0.0.61");
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, new ArrayList<>(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), new ArrayList<>(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
         Assert.assertEquals(2, map.asMap().size());
@@ -263,31 +257,28 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testForRemoteDC() throws Exception
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59");
 
         //Reject only 127.0.0.3 and accept everyone else
-        final RangeStreamer.ISourceFilter localHostFilter = new RangeStreamer.ISourceFilter()
+        final Predicate<Replica> localHostFilter = replica ->
         {
-            public boolean shouldInclude(InetAddressAndPort endpoint)
+            try
             {
-                try
-                {
-                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.3")))
-                        return false;
-                    else
-                        return true;
-                }
-                catch (UnknownHostException e)
-                {
+                if (replica.endpoint().equals(InetAddressAndPort.getByName("127.0.0.3")))
+                    return false;
+                else
                     return true;
-                }
+            }
+            catch (UnknownHostException e)
+            {
+                return true;
             }
         };
 
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Arrays.asList(localHostFilter), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Arrays.asList(localHostFilter), "Test");
         Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap();
         validateRange(rangesWithSources, map);
         Assert.assertEquals(3, map.asMap().size());
@@ -301,14 +292,14 @@ public class RangeFetchMapCalculatorTest
     @Test
     public void testTrivialRanges() throws UnknownHostException
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         // add non-trivial ranges
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
         addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59");
         // and a trivial one:
         addTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Collections.emptyList(), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.emptyList(), "Test");
         Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
         Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
         assertTrue(trivialMap.get(InetAddressAndPort.getByName("127.0.0.3")).contains(generateTrivialRange(1,10)) ^
@@ -319,7 +310,7 @@ public class RangeFetchMapCalculatorTest
     @Test(expected = IllegalStateException.class)
     public void testNotEnoughEndpointsForTrivialRange() throws UnknownHostException
     {
-        Multimap<Range<Token>, InetAddressAndPort> rangesWithSources = HashMultimap.create();
+        EndpointsByRange.Mutable rangesWithSources = new EndpointsByRange.Mutable();
         // add non-trivial ranges
         addNonTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51");
         addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.3", "127.0.0.55");
@@ -327,23 +318,20 @@ public class RangeFetchMapCalculatorTest
         // and a trivial one:
         addTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3");
 
-        RangeStreamer.ISourceFilter filter = new RangeStreamer.ISourceFilter()
+        Predicate<Replica> filter = replica ->
         {
-            public boolean shouldInclude(InetAddressAndPort endpoint)
+            try
             {
-                try
-                {
-                    if (endpoint.equals(InetAddressAndPort.getByName("127.0.0.3")))
-                        return false;
-                }
-                catch (UnknownHostException e)
-                {
-                    throw new RuntimeException(e);
-                }
-                return true;
+                if (replica.endpoint().equals(InetAddressAndPort.getByName("127.0.0.3")))
+                    return false;
+            }
+            catch (UnknownHostException e)
+            {
+                throw new RuntimeException(e);
             }
+            return true;
         };
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, Collections.singleton(filter), "Test");
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.asImmutableView(), Collections.singleton(filter), "Test");
         Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges();
         Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap);
 
@@ -355,27 +343,29 @@ public class RangeFetchMapCalculatorTest
         assertTrue(result.containsAll(expected));
     }
 
-    private void validateRange(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> result)
+    private void validateRange(EndpointsByRange.Mutable rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> result)
     {
         for (Map.Entry<InetAddressAndPort, Range<Token>> entry : result.entries())
         {
-            assertTrue(rangesWithSources.get(entry.getValue()).contains(entry.getKey()));
+            assertTrue(rangesWithSources.get(entry.getValue()).endpoints().contains(entry.getKey()));
         }
     }
 
-    private void addNonTrivialRangeAndSources(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
+    private void addNonTrivialRangeAndSources(EndpointsByRange.Mutable rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
     {
         for (InetAddressAndPort endpoint : makeAddrs(hosts))
         {
-            rangesWithSources.put(generateNonTrivialRange(left, right), endpoint);
+            Range<Token> range = generateNonTrivialRange(left, right);
+            rangesWithSources.put(range, Replica.fullReplica(endpoint, range));
         }
     }
 
-    private void addTrivialRangeAndSources(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
+    private void addTrivialRangeAndSources(EndpointsByRange.Mutable rangesWithSources, int left, int right, String... hosts) throws UnknownHostException
     {
         for (InetAddressAndPort endpoint : makeAddrs(hosts))
         {
-            rangesWithSources.put(generateTrivialRange(left, right), endpoint);
+            Range<Token> range = generateTrivialRange(left, right);
+            rangesWithSources.put(range, Replica.fullReplica(endpoint, range));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/dht/RangeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/RangeTest.java b/test/unit/org/apache/cassandra/dht/RangeTest.java
index 495979e..36a8da1 100644
--- a/test/unit/org/apache/cassandra/dht/RangeTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeTest.java
@@ -642,7 +642,7 @@ public class RangeTest
             Range.OrderedRangeContainmentChecker checker = new Range.OrderedRangeContainmentChecker(ranges);
             for (Token t : tokensToTest)
             {
-                if (checker.contains(t) != Range.isInRanges(t, ranges)) // avoid running Joiner.on(..) every iteration
+                if (checker.test(t) != Range.isInRanges(t, ranges)) // avoid running Joiner.on(..) every iteration
                     fail(String.format("This should never flap! If it does, it is a bug (ranges = %s, token = %s)", Joiner.on(",").join(ranges), t));
             }
         }
@@ -653,11 +653,11 @@ public class RangeTest
     {
         List<Range<Token>> ranges = asList(r(Long.MIN_VALUE, Long.MIN_VALUE + 1), r(Long.MAX_VALUE - 1, Long.MAX_VALUE));
         Range.OrderedRangeContainmentChecker checker = new Range.OrderedRangeContainmentChecker(ranges);
-        assertFalse(checker.contains(t(Long.MIN_VALUE)));
-        assertTrue(checker.contains(t(Long.MIN_VALUE + 1)));
-        assertFalse(checker.contains(t(0)));
-        assertFalse(checker.contains(t(Long.MAX_VALUE - 1)));
-        assertTrue(checker.contains(t(Long.MAX_VALUE)));
+        assertFalse(checker.test(t(Long.MIN_VALUE)));
+        assertTrue(checker.test(t(Long.MIN_VALUE + 1)));
+        assertFalse(checker.test(t(0)));
+        assertFalse(checker.test(t(Long.MAX_VALUE - 1)));
+        assertTrue(checker.test(t(Long.MAX_VALUE)));
     }
 
     private static Range<Token> r(long left, long right)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/dht/SplitterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/SplitterTest.java b/test/unit/org/apache/cassandra/dht/SplitterTest.java
index 322a57c..c591499 100644
--- a/test/unit/org/apache/cassandra/dht/SplitterTest.java
+++ b/test/unit/org/apache/cassandra/dht/SplitterTest.java
@@ -62,13 +62,54 @@ public class SplitterTest
         randomSplitTestVNodes(new Murmur3Partitioner());
     }
 
+    @Test
+    public void testWithWeight()
+    {
+        List<Splitter.WeightedRange> ranges = new ArrayList<>();
+        ranges.add(new Splitter.WeightedRange(1.0, t(0, 10)));
+        ranges.add(new Splitter.WeightedRange(1.0, t(20, 30)));
+        ranges.add(new Splitter.WeightedRange(0.5, t(40, 60)));
+
+        List<Splitter.WeightedRange> ranges2 = new ArrayList<>();
+        ranges2.add(new Splitter.WeightedRange(1.0, t(0, 10)));
+        ranges2.add(new Splitter.WeightedRange(1.0, t(20, 30)));
+        ranges2.add(new Splitter.WeightedRange(1.0, t(40, 50)));
+        IPartitioner partitioner = Murmur3Partitioner.instance;
+        Splitter splitter = partitioner.splitter().get();
+
+        assertEquals(splitter.splitOwnedRanges(2, ranges, false), splitter.splitOwnedRanges(2, ranges2, false));
+    }
+
+    @Test
+    public void testWithWeight2()
+    {
+        List<Splitter.WeightedRange> ranges = new ArrayList<>();
+        ranges.add(new Splitter.WeightedRange(0.2, t(0, 10)));
+        ranges.add(new Splitter.WeightedRange(1.0, t(20, 30)));
+        ranges.add(new Splitter.WeightedRange(1.0, t(40, 50)));
+
+        List<Splitter.WeightedRange> ranges2 = new ArrayList<>();
+        ranges2.add(new Splitter.WeightedRange(1.0, t(0, 2)));
+        ranges2.add(new Splitter.WeightedRange(1.0, t(20, 30)));
+        ranges2.add(new Splitter.WeightedRange(1.0, t(40, 50)));
+        IPartitioner partitioner = Murmur3Partitioner.instance;
+        Splitter splitter = partitioner.splitter().get();
+
+        assertEquals(splitter.splitOwnedRanges(2, ranges, false), splitter.splitOwnedRanges(2, ranges2, false));
+    }
+
+    private Range<Token> t(long left, long right)
+    {
+        return new Range<>(new Murmur3Partitioner.LongToken(left), new Murmur3Partitioner.LongToken(right));
+    }
+
     private static void randomSplitTestNoVNodes(IPartitioner partitioner)
     {
         Splitter splitter = getSplitter(partitioner);
         Random r = new Random();
         for (int i = 0; i < 10000; i++)
         {
-            List<Range<Token>> localRanges = generateLocalRanges(1, r.nextInt(4) + 1, splitter, r, partitioner instanceof RandomPartitioner);
+            List<Splitter.WeightedRange> localRanges = generateLocalRanges(1, r.nextInt(4) + 1, splitter, r, partitioner instanceof RandomPartitioner);
             List<Token> boundaries = splitter.splitOwnedRanges(r.nextInt(9) + 1, localRanges, false);
             assertTrue("boundaries = " + boundaries + " ranges = " + localRanges, assertRangeSizeEqual(localRanges, boundaries, partitioner, splitter, true));
         }
@@ -84,14 +125,14 @@ public class SplitterTest
             int numTokens = 172 + r.nextInt(128);
             int rf = r.nextInt(4) + 2;
             int parts = r.nextInt(5) + 1;
-            List<Range<Token>> localRanges = generateLocalRanges(numTokens, rf, splitter, r, partitioner instanceof RandomPartitioner);
+            List<Splitter.WeightedRange> localRanges = generateLocalRanges(numTokens, rf, splitter, r, partitioner instanceof RandomPartitioner);
             List<Token> boundaries = splitter.splitOwnedRanges(parts, localRanges, true);
             if (!assertRangeSizeEqual(localRanges, boundaries, partitioner, splitter, false))
                 fail(String.format("Could not split %d tokens with rf=%d into %d parts (localRanges=%s, boundaries=%s)", numTokens, rf, parts, localRanges, boundaries));
         }
     }
 
-    private static boolean assertRangeSizeEqual(List<Range<Token>> localRanges, List<Token> tokens, IPartitioner partitioner, Splitter splitter, boolean splitIndividualRanges)
+    private static boolean assertRangeSizeEqual(List<Splitter.WeightedRange> localRanges, List<Token> tokens, IPartitioner partitioner, Splitter splitter, boolean splitIndividualRanges)
     {
         Token start = partitioner.getMinimumToken();
         List<BigInteger> splits = new ArrayList<>();
@@ -119,27 +160,27 @@ public class SplitterTest
         return allBalanced;
     }
 
-    private static BigInteger sumOwnedBetween(List<Range<Token>> localRanges, Token start, Token end, Splitter splitter, boolean splitIndividualRanges)
+    private static BigInteger sumOwnedBetween(List<Splitter.WeightedRange> localRanges, Token start, Token end, Splitter splitter, boolean splitIndividualRanges)
     {
         BigInteger sum = BigInteger.ZERO;
-        for (Range<Token> range : localRanges)
+        for (Splitter.WeightedRange range : localRanges)
         {
             if (splitIndividualRanges)
             {
-                Set<Range<Token>> intersections = new Range<>(start, end).intersectionWith(range);
+                Set<Range<Token>> intersections = new Range<>(start, end).intersectionWith(range.range());
                 for (Range<Token> intersection : intersections)
                     sum = sum.add(splitter.valueForToken(intersection.right).subtract(splitter.valueForToken(intersection.left)));
             }
             else
             {
-                if (new Range<>(start, end).contains(range.left))
-                    sum = sum.add(splitter.valueForToken(range.right).subtract(splitter.valueForToken(range.left)));
+                if (new Range<>(start, end).contains(range.left()))
+                    sum = sum.add(splitter.valueForToken(range.right()).subtract(splitter.valueForToken(range.left())));
             }
         }
         return sum;
     }
 
-    private static List<Range<Token>> generateLocalRanges(int numTokens, int rf, Splitter splitter, Random r, boolean randomPartitioner)
+    private static List<Splitter.WeightedRange> generateLocalRanges(int numTokens, int rf, Splitter splitter, Random r, boolean randomPartitioner)
     {
         int localTokens = numTokens * rf;
         List<Token> randomTokens = new ArrayList<>();
@@ -152,11 +193,11 @@ public class SplitterTest
 
         Collections.sort(randomTokens);
 
-        List<Range<Token>> localRanges = new ArrayList<>(localTokens);
+        List<Splitter.WeightedRange> localRanges = new ArrayList<>(localTokens);
         for (int i = 0; i < randomTokens.size() - 1; i++)
         {
             assert randomTokens.get(i).compareTo(randomTokens.get(i + 1)) < 0;
-            localRanges.add(new Range<>(randomTokens.get(i), randomTokens.get(i + 1)));
+            localRanges.add(new Splitter.WeightedRange(1.0, new Range<>(randomTokens.get(i), randomTokens.get(i + 1))));
             i++;
         }
         return localRanges;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index bf71c09..34096a7 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.dht;
 
 import java.util.Collections;
 
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -53,7 +54,7 @@ public class StreamStateStoreTest
 
         InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
         StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, local, new DefaultConnectionFactory(), 0, null, PreviewKind.NONE);
-        session.addStreamRequest("keyspace1", Collections.singleton(range), Collections.singleton("cf"));
+        session.addStreamRequest("keyspace1", RangesAtEndpoint.toDummyList(Collections.singleton(range)), RangesAtEndpoint.toDummyList(Collections.emptyList()), Collections.singleton("cf"));
 
         StreamStateStore store = new StreamStateStore();
         // session complete event that is not completed makes data not available for keyspace/ranges
@@ -74,7 +75,7 @@ public class StreamStateStoreTest
         // add different range within the same keyspace
         Range<Token> range2 = new Range<>(factory.fromString("100"), factory.fromString("200"));
         session = new StreamSession(StreamOperation.BOOTSTRAP, local, new DefaultConnectionFactory(), 0, null, PreviewKind.NONE);
-        session.addStreamRequest("keyspace1", Collections.singleton(range2), Collections.singleton("cf"));
+        session.addStreamRequest("keyspace1", RangesAtEndpoint.toDummyList(Collections.singleton(range2)), RangesAtEndpoint.toDummyList(Collections.emptyList()), Collections.singleton("cf"));
         session.state(StreamSession.State.COMPLETE);
         store.handleStreamEvent(new StreamEvent.SessionCompleteEvent(session));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
index 833ee8b..0710945 100644
--- a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
+++ b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
@@ -62,7 +62,7 @@ public class PendingRangeCalculatorServiceTest
     @BMRule(name = "Block pending range calculation",
             targetClass = "TokenMetadata",
             targetMethod = "calculatePendingRanges",
-            targetLocation = "AT INVOKE org.apache.cassandra.locator.AbstractReplicationStrategy.getAddressRanges",
+            targetLocation = "AT INVOKE org.apache.cassandra.locator.AbstractReplicationStrategy.getAddressReplicas",
             action = "org.apache.cassandra.gms.PendingRangeCalculatorServiceTest.calculationLock.lock()")
     public void testDelayedResponse() throws UnknownHostException, InterruptedException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
index fccb344..9e3594b 100644
--- a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -69,7 +69,7 @@ public class BigTableWriterTest extends AbstractTransactionalTest
 
         private TestableBTW(Descriptor desc)
         {
-            this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, null,
+            this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, null, false,
                                                new SerializationHeader(true, cfs.metadata(),
                                                                        cfs.metadata().regularAndStaticColumns(),
                                                                        EncodingStats.NO_STATS)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index dbb929d..faf46bc 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -649,7 +649,7 @@ public class CQLSSTableWriterTest
             public void init(String keyspace)
             {
                 this.keyspace = keyspace;
-                for (Range<Token> range : StorageService.instance.getLocalRanges(ks))
+                for (Range<Token> range : StorageService.instance.getLocalReplicas(ks).ranges())
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 044cd9f..c4ccf48 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -176,21 +176,26 @@ public class LegacySSTableTest
             {
                 for (SSTableReader sstable : cfs.getLiveSSTables())
                 {
-                    sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, 1234, NO_PENDING_REPAIR);
+                    sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, 1234, NO_PENDING_REPAIR, false);
                     sstable.reloadSSTableMetadata();
                     assertEquals(1234, sstable.getRepairedAt());
                     if (sstable.descriptor.version.hasPendingRepair())
                         assertEquals(NO_PENDING_REPAIR, sstable.getPendingRepair());
                 }
 
+                boolean isTransient = false;
                 for (SSTableReader sstable : cfs.getLiveSSTables())
                 {
                     UUID random = UUID.randomUUID();
-                    sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, UNREPAIRED_SSTABLE, random);
+                    sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, UNREPAIRED_SSTABLE, random, isTransient);
                     sstable.reloadSSTableMetadata();
                     assertEquals(UNREPAIRED_SSTABLE, sstable.getRepairedAt());
                     if (sstable.descriptor.version.hasPendingRepair())
                         assertEquals(random, sstable.getPendingRepair());
+                    if (sstable.descriptor.version.hasIsTransient())
+                        assertEquals(isTransient, sstable.isTransient());
+
+                    isTransient = !isTransient;
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 8509115..5d40f8c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -31,14 +31,13 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -109,8 +108,8 @@ public class SSTableLoaderTest
         public void init(String keyspace)
         {
             this.keyspace = keyspace;
-            for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
-                addRangeForEndpoint(range, FBUtilities.getBroadcastAddressAndPort());
+            for (Replica replica : StorageService.instance.getLocalReplicas(KEYSPACE1))
+                addRangeForEndpoint(replica.range(), FBUtilities.getBroadcastAddressAndPort());
         }
 
         public TableMetadataRef getTableMetadata(String tableName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 6412ef4..7c47c8b 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -877,7 +877,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
             File dir = cfs.getDirectories().getDirectoryForNewSSTables();
             Descriptor desc = cfs.newSSTableDescriptor(dir);
 
-            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, null, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
+            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, null, false, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
             {
                 int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
                 for ( ; i < end ; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index 441a3b9..731cee2 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -219,7 +219,7 @@ public class SSTableUtils
             TableMetadata metadata = Schema.instance.getTableMetadata(ksname, cfname);
             ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.id);
             SerializationHeader header = appender.header();
-            SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, 0, header);
+            SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, false, 0, header);
             while (appender.append(writer)) { /* pass */ }
             Collection<SSTableReader> readers = writer.finish(true);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
index 5d62cdb..31d0b89 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.File;
 import java.nio.ByteBuffer;
+import java.util.UUID;
 
 import org.junit.Test;
 
@@ -32,9 +33,12 @@ import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static junit.framework.Assert.fail;
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -245,4 +249,60 @@ public class SSTableWriterTest extends SSTableWriterTestBase
         }
     }
 
+    private static void assertValidRepairMetadata(long repairedAt, UUID pendingRepair, boolean isTransient)
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_SMALL_MAX_VALUE);
+        File dir = cfs.getDirectories().getDirectoryForNewSSTables();
+        LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.STREAM);
+
+        try (SSTableWriter writer = getWriter(cfs, dir, txn, repairedAt, pendingRepair, isTransient))
+        {
+            // expected
+        }
+        catch (IllegalArgumentException e)
+        {
+            throw new AssertionError("Unexpected IllegalArgumentException", e);
+        }
+
+        txn.abort();
+        LifecycleTransaction.waitForDeletions();
+    }
+
+    private static void assertInvalidRepairMetadata(long repairedAt, UUID pendingRepair, boolean isTransient)
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_SMALL_MAX_VALUE);
+        File dir = cfs.getDirectories().getDirectoryForNewSSTables();
+        LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.STREAM);
+
+        try (SSTableWriter writer = getWriter(cfs, dir, txn, repairedAt, pendingRepair, isTransient))
+        {
+            fail("Expected IllegalArgumentException");
+        }
+        catch (IllegalArgumentException e)
+        {
+            // expected
+        }
+
+        txn.abort();
+        LifecycleTransaction.waitForDeletions();
+    }
+
+    /**
+     * It should only be possible to create sstables marked transient that also have a pending repair
+     */
+    @Test
+    public void testRepairMetadataValidation()
+    {
+        assertValidRepairMetadata(UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, false);
+        assertValidRepairMetadata(1, NO_PENDING_REPAIR, false);
+        assertValidRepairMetadata(UNREPAIRED_SSTABLE, UUID.randomUUID(), false);
+        assertValidRepairMetadata(UNREPAIRED_SSTABLE, UUID.randomUUID(), true);
+
+        assertInvalidRepairMetadata(UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, true);
+        assertInvalidRepairMetadata(1, UUID.randomUUID(), false);
+        assertInvalidRepairMetadata(1, NO_PENDING_REPAIR, true);
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
index d42c49b..962e1a1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 
@@ -48,6 +49,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class SSTableWriterTestBase extends SchemaLoader
 {
@@ -161,10 +163,15 @@ public class SSTableWriterTestBase extends SchemaLoader
             assertFalse(CompactionManager.instance.submitMaximal(cfs, cfs.gcBefore((int) (System.currentTimeMillis() / 1000)), false).isEmpty());
     }
 
-    public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn)
+    public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn, long repairedAt, UUID pendingRepair, boolean isTransient)
     {
         Descriptor desc = cfs.newSSTableDescriptor(directory);
-        return SSTableWriter.create(desc, 0, 0, null, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
+        return SSTableWriter.create(desc, 0, repairedAt, pendingRepair, isTransient, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
+    }
+
+    public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn)
+    {
+        return getWriter(cfs, directory, txn, 0, null, false);
     }
 
     public static ByteBuffer random(int i, int size)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
index 9aa4e28..aea3b4a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
@@ -96,7 +96,7 @@ public class SSTableFlushObserverTest
                                                                   KS_NAME, CF_NAME,
                                                                   0,
                                                                   sstableFormat),
-                                                   10L, 0L, null, TableMetadataRef.forOfflineTools(cfm),
+                                                   10L, 0L, null, false, TableMetadataRef.forOfflineTools(cfm),
                                                    new MetadataCollector(cfm.comparator).sstableLevel(0),
                                                    new SerializationHeader(true, cfm, cfm.regularAndStaticColumns(), EncodingStats.NO_STATS),
                                                    Collections.singletonList(observer),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 8ab1511..f109d8f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -99,7 +99,7 @@ public class MetadataSerializerTest
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
-        return collector.finalizeMetadata(partitioner, bfFpChance, 0, null, SerializationHeader.make(cfm, Collections.emptyList()));
+        return collector.finalizeMetadata(partitioner, bfFpChance, 0, null, false, SerializationHeader.make(cfm, Collections.emptyList()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
index 202d7f1..bf1d940 100644
--- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
@@ -50,6 +50,16 @@ public class DynamicEndpointSnitchTest
         Thread.sleep(150);
     }
 
+    private static EndpointsForRange full(InetAddressAndPort... endpoints)
+    {
+        EndpointsForRange.Builder rlist = EndpointsForRange.builder(ReplicaUtils.FULL_RANGE, endpoints.length);
+        for (InetAddressAndPort endpoint: endpoints)
+        {
+            rlist.add(ReplicaUtils.full(endpoint));
+        }
+        return rlist.build();
+    }
+
     @Test
     public void testSnitch() throws InterruptedException, IOException, ConfigurationException
     {
@@ -66,41 +76,41 @@ public class DynamicEndpointSnitchTest
 
         // first, make all hosts equal
         setScores(dsnitch, 1, hosts, 10, 10, 10);
-        List<InetAddressAndPort> order = Arrays.asList(host1, host2, host3);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        EndpointsForRange order = full(host1, host2, host3);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
         // make host1 a little worse
         setScores(dsnitch, 1, hosts, 20, 10, 10);
-        order = Arrays.asList(host2, host3, host1);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        order = full(host2, host3, host1);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
         // make host2 as bad as host1
         setScores(dsnitch, 2, hosts, 15, 20, 10);
-        order = Arrays.asList(host3, host1, host2);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        order = full(host3, host1, host2);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
         // make host3 the worst
         setScores(dsnitch, 3, hosts, 10, 10, 30);
-        order = Arrays.asList(host1, host2, host3);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        order = full(host1, host2, host3);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
         // make host3 equal to the others
         setScores(dsnitch, 5, hosts, 10, 10, 10);
-        order = Arrays.asList(host1, host2, host3);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        order = full(host1, host2, host3);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
         /// Tests CASSANDRA-6683 improvements
         // make the scores differ enough from the ideal order that we sort by score; under the old
         // dynamic snitch behavior (where we only compared neighbors), these wouldn't get sorted
         setScores(dsnitch, 20, hosts, 10, 70, 20);
-        order = Arrays.asList(host1, host3, host2);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+        order = full(host1, host3, host2);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3)));
 
-        order = Arrays.asList(host4, host1, host3, host2);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3, host4)));
+        order = full(host4, host1, host3, host2);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3, host4)));
 
         setScores(dsnitch, 20, hosts, 10, 10, 10);
-        order = Arrays.asList(host4, host1, host2, host3);
-        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3, host4)));
+        order = full(host4, host1, host2, host3);
+        assertEquals(order, dsnitch.sortedByProximity(self, full(host1, host2, host3, host4)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
index ab6c6cd..5f6e26f 100644
--- a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
@@ -25,6 +25,7 @@ import java.util.stream.Collectors;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 
 import org.junit.Assert;
@@ -36,12 +37,17 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
 import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.TokenMetadata.Topology;
 import org.apache.cassandra.service.StorageService;
 
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+
 public class NetworkTopologyStrategyTest
 {
     private String keyspaceName = "Keyspace1";
@@ -51,6 +57,7 @@ public class NetworkTopologyStrategyTest
     public static void setupDD()
     {
         DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
     }
 
     @Test
@@ -68,13 +75,14 @@ public class NetworkTopologyStrategyTest
 
         // Set the localhost to the tokenmetadata. Embedded cassandra way?
         NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
-        assert strategy.getReplicationFactor("DC1") == 3;
-        assert strategy.getReplicationFactor("DC2") == 2;
-        assert strategy.getReplicationFactor("DC3") == 1;
+        assert strategy.getReplicationFactor("DC1").allReplicas == 3;
+        assert strategy.getReplicationFactor("DC2").allReplicas == 2;
+        assert strategy.getReplicationFactor("DC3").allReplicas == 1;
         // Query for the natural hosts
-        ArrayList<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
-        assert 6 == endpoints.size();
-        assert 6 == new HashSet<>(endpoints).size(); // ensure uniqueness
+        EndpointsForToken replicas = strategy.getNaturalReplicasForToken(new StringToken("123"));
+        assert 6 == replicas.size();
+        assert 6 == replicas.endpoints().size(); // ensure uniqueness
+        assert 6 == new HashSet<>(replicas.byEndpoint().values()).size(); // ensure uniqueness
     }
 
     @Test
@@ -92,13 +100,14 @@ public class NetworkTopologyStrategyTest
 
         // Set the localhost to the tokenmetadata. Embedded cassandra way?
         NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
-        assert strategy.getReplicationFactor("DC1") == 3;
-        assert strategy.getReplicationFactor("DC2") == 3;
-        assert strategy.getReplicationFactor("DC3") == 0;
+        assert strategy.getReplicationFactor("DC1").allReplicas == 3;
+        assert strategy.getReplicationFactor("DC2").allReplicas == 3;
+        assert strategy.getReplicationFactor("DC3").allReplicas == 0;
         // Query for the natural hosts
-        ArrayList<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
-        assert 6 == endpoints.size();
-        assert 6 == new HashSet<>(endpoints).size(); // ensure uniqueness
+        EndpointsForToken replicas = strategy.getNaturalReplicasForToken(new StringToken("123"));
+        assert 6 == replicas.size();
+        assert 6 == replicas.endpoints().size(); // ensure uniqueness
+        assert 6 == new HashSet<>(replicas.byEndpoint().values()).size(); // ensure uniqueness
     }
 
     @Test
@@ -137,12 +146,13 @@ public class NetworkTopologyStrategyTest
 
         for (String testToken : new String[]{"123456", "200000", "000402", "ffffff", "400200"})
         {
-            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(new StringToken(testToken), metadata);
-            Set<InetAddressAndPort> epSet = new HashSet<>(endpoints);
+            EndpointsForRange replicas = strategy.calculateNaturalReplicas(new StringToken(testToken), metadata);
+            Set<InetAddressAndPort> endpointSet = replicas.endpoints();
 
-            Assert.assertEquals(totalRF, endpoints.size());
-            Assert.assertEquals(totalRF, epSet.size());
-            logger.debug("{}: {}", testToken, endpoints);
+            Assert.assertEquals(totalRF, replicas.size());
+            Assert.assertEquals(totalRF, new HashSet<>(replicas.byEndpoint().values()).size());
+            Assert.assertEquals(totalRF, endpointSet.size());
+            logger.debug("{}: {}", testToken, replicas);
         }
     }
 
@@ -209,7 +219,7 @@ public class NetworkTopologyStrategyTest
         {
             Token token = Murmur3Partitioner.instance.getRandomToken(rand);
             List<InetAddressAndPort> expected = calculateNaturalEndpoints(token, tokenMetadata, datacenters, snitch);
-            List<InetAddressAndPort> actual = nts.calculateNaturalEndpoints(token, tokenMetadata);
+            List<InetAddressAndPort> actual = new ArrayList<>(nts.calculateNaturalReplicas(token, tokenMetadata).endpoints());
             if (endpointsDiffer(expected, actual))
             {
                 System.err.println("Endpoints mismatch for token " + token);
@@ -373,4 +383,50 @@ public class NetworkTopologyStrategyTest
         Integer replicas = datacenters.get(dc);
         return replicas == null ? 0 : replicas;
     }
+
+    private static Token tk(long t)
+    {
+        return new LongToken(t);
+    }
+
+    private static Range<Token> range(long l, long r)
+    {
+        return new Range<>(tk(l), tk(r));
+    }
+
+    @Test
+    public void testTransientReplica() throws Exception
+    {
+        IEndpointSnitch snitch = new SimpleSnitch();
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+
+        List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"),
+                                                                InetAddressAndPort.getByName("127.0.0.2"),
+                                                                InetAddressAndPort.getByName("127.0.0.3"),
+                                                                InetAddressAndPort.getByName("127.0.0.4"));
+
+        Multimap<InetAddressAndPort, Token> tokens = HashMultimap.create();
+        tokens.put(endpoints.get(0), tk(100));
+        tokens.put(endpoints.get(1), tk(200));
+        tokens.put(endpoints.get(2), tk(300));
+        tokens.put(endpoints.get(3), tk(400));
+        TokenMetadata metadata = new TokenMetadata();
+        metadata.updateNormalTokens(tokens);
+
+        Map<String, String> configOptions = new HashMap<String, String>();
+        configOptions.put(snitch.getDatacenter((InetAddressAndPort) null), "3/1");
+
+        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
+
+        Assert.assertEquals(EndpointsForRange.of(fullReplica(endpoints.get(0), range(400, 100)),
+                                           fullReplica(endpoints.get(1), range(400, 100)),
+                                           transientReplica(endpoints.get(2), range(400, 100))),
+                            strategy.getNaturalReplicasForToken(tk(99)));
+
+
+        Assert.assertEquals(EndpointsForRange.of(fullReplica(endpoints.get(1), range(100, 200)),
+                                           fullReplica(endpoints.get(2), range(100, 200)),
+                                           transientReplica(endpoints.get(3), range(100, 200))),
+                            strategy.getNaturalReplicasForToken(tk(101)));
+    }
 }


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


[15/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index 110fed6..e8aa5d3 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -18,27 +18,40 @@
 package org.apache.cassandra.dht;
 
 import java.util.*;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ArrayListMultimap;
+import com.google.common.base.Predicate;
 import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
 
+import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.EndpointsByReplica;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.LocalStrategy;
 
+import org.apache.cassandra.locator.EndpointsByRange;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.Replicas;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.PreviewKind;
@@ -47,13 +60,25 @@ import org.apache.cassandra.streaming.StreamResultFuture;
 import org.apache.cassandra.streaming.StreamOperation;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static com.google.common.base.Predicates.and;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.any;
+import static org.apache.cassandra.locator.Replica.fullReplica;
+
 /**
- * Assists in streaming ranges to a node.
+ * Assists in streaming ranges to this node.
  */
 public class RangeStreamer
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeStreamer.class);
 
+    public static Predicate<Replica> ALIVE_PREDICATE = replica ->
+                                                             (!Gossiper.instance.isEnabled() ||
+                                                              (Gossiper.instance.getEndpointStateForEndpoint(replica.endpoint()) == null ||
+                                                               Gossiper.instance.getEndpointStateForEndpoint(replica.endpoint()).isAlive())) &&
+                                                             FailureDetector.instance.isAlive(replica.endpoint());
+
     /* bootstrap tokens. can be null if replacing the node. */
     private final Collection<Token> tokens;
     /* current token ring */
@@ -62,26 +87,59 @@ public class RangeStreamer
     private final InetAddressAndPort address;
     /* streaming description */
     private final String description;
-    private final Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch = HashMultimap.create();
-    private final Set<ISourceFilter> sourceFilters = new HashSet<>();
+    private final Multimap<String, Multimap<InetAddressAndPort, FetchReplica>> toFetch = HashMultimap.create();
+    private final Set<Predicate<Replica>> sourceFilters = new HashSet<>();
     private final StreamPlan streamPlan;
     private final boolean useStrictConsistency;
     private final IEndpointSnitch snitch;
     private final StreamStateStore stateStore;
 
-    /**
-     * A filter applied to sources to stream from when constructing a fetch map.
-     */
-    public static interface ISourceFilter
+    public static class FetchReplica
     {
-        public boolean shouldInclude(InetAddressAndPort endpoint);
+        public final Replica local;
+        public final Replica remote;
+
+        public FetchReplica(Replica local, Replica remote)
+        {
+            Preconditions.checkNotNull(local);
+            Preconditions.checkNotNull(remote);
+            assert local.isLocal() && !remote.isLocal();
+            this.local = local;
+            this.remote = remote;
+        }
+
+        public String toString()
+        {
+            return "FetchReplica{" +
+                   "local=" + local +
+                   ", remote=" + remote +
+                   '}';
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            FetchReplica that = (FetchReplica) o;
+
+            if (!local.equals(that.local)) return false;
+            return remote.equals(that.remote);
+        }
+
+        public int hashCode()
+        {
+            int result = local.hashCode();
+            result = 31 * result + remote.hashCode();
+            return result;
+        }
     }
 
     /**
      * Source filter which excludes any endpoints that are not alive according to a
      * failure detector.
      */
-    public static class FailureDetectorSourceFilter implements ISourceFilter
+    public static class FailureDetectorSourceFilter implements Predicate<Replica>
     {
         private final IFailureDetector fd;
 
@@ -90,16 +148,16 @@ public class RangeStreamer
             this.fd = fd;
         }
 
-        public boolean shouldInclude(InetAddressAndPort endpoint)
+        public boolean apply(Replica replica)
         {
-            return fd.isAlive(endpoint);
+            return fd.isAlive(replica.endpoint());
         }
     }
 
     /**
      * Source filter which excludes any endpoints that are not in a specific data center.
      */
-    public static class SingleDatacenterFilter implements ISourceFilter
+    public static class SingleDatacenterFilter implements Predicate<Replica>
     {
         private final String sourceDc;
         private final IEndpointSnitch snitch;
@@ -110,27 +168,27 @@ public class RangeStreamer
             this.snitch = snitch;
         }
 
-        public boolean shouldInclude(InetAddressAndPort endpoint)
+        public boolean apply(Replica replica)
         {
-            return snitch.getDatacenter(endpoint).equals(sourceDc);
+            return snitch.getDatacenter(replica).equals(sourceDc);
         }
     }
 
     /**
      * Source filter which excludes the current node from source calculations
      */
-    public static class ExcludeLocalNodeFilter implements ISourceFilter
+    public static class ExcludeLocalNodeFilter implements Predicate<Replica>
     {
-        public boolean shouldInclude(InetAddressAndPort endpoint)
+        public boolean apply(Replica replica)
         {
-            return !FBUtilities.getBroadcastAddressAndPort().equals(endpoint);
+            return !replica.isLocal();
         }
     }
 
     /**
      * Source filter which only includes endpoints contained within a provided set.
      */
-    public static class WhitelistedSourcesFilter implements ISourceFilter
+    public static class WhitelistedSourcesFilter implements Predicate<Replica>
     {
         private final Set<InetAddressAndPort> whitelistedSources;
 
@@ -139,9 +197,9 @@ public class RangeStreamer
             this.whitelistedSources = whitelistedSources;
         }
 
-        public boolean shouldInclude(InetAddressAndPort endpoint)
+        public boolean apply(Replica replica)
         {
-            return whitelistedSources.contains(endpoint);
+            return whitelistedSources.contains(replica.endpoint());
         }
     }
 
@@ -167,7 +225,7 @@ public class RangeStreamer
         streamPlan.listeners(this.stateStore);
     }
 
-    public void addSourceFilter(ISourceFilter filter)
+    public void addSourceFilter(Predicate<Replica> filter)
     {
         sourceFilters.add(filter);
     }
@@ -176,80 +234,95 @@ public class RangeStreamer
      * Add ranges to be streamed for given keyspace.
      *
      * @param keyspaceName keyspace name
-     * @param ranges ranges to be streamed
+     * @param replicas ranges to be streamed
      */
-    public void addRanges(String keyspaceName, Collection<Range<Token>> ranges)
+    public void addRanges(String keyspaceName, ReplicaCollection<?> replicas)
     {
-        if(Keyspace.open(keyspaceName).getReplicationStrategy() instanceof LocalStrategy)
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        AbstractReplicationStrategy strat = keyspace.getReplicationStrategy();
+        if(strat instanceof LocalStrategy)
         {
             logger.info("Not adding ranges for Local Strategy keyspace={}", keyspaceName);
             return;
         }
 
-        boolean useStrictSource = useStrictSourcesForRanges(keyspaceName);
-        Multimap<Range<Token>, InetAddressAndPort> rangesForKeyspace = useStrictSource
-                ? getAllRangesWithStrictSourcesFor(keyspaceName, ranges) : getAllRangesWithSourcesFor(keyspaceName, ranges);
+        boolean useStrictSource = useStrictSourcesForRanges(strat);
+        EndpointsByReplica fetchMap = calculateRangesToFetchWithPreferredEndpoints(replicas, keyspace, useStrictSource);
 
-        for (Map.Entry<Range<Token>, InetAddressAndPort> entry : rangesForKeyspace.entries())
+        for (Map.Entry<Replica, Replica> entry : fetchMap.flattenEntries())
             logger.info("{}: range {} exists on {} for keyspace {}", description, entry.getKey(), entry.getValue(), keyspaceName);
 
-        AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMap = useStrictSource || strat == null || strat.getReplicationFactor() == 1
-                                                            ? getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName, useStrictConsistency)
-                                                            : getOptimizedRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName);
 
-        for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : rangeFetchMap.asMap().entrySet())
+        Multimap<InetAddressAndPort, FetchReplica> workMap;
+        //Only use the optimized strategy if we don't care about strict sources, have a replication factor > 1, and no
+        //transient replicas.
+        if (useStrictSource || strat == null || strat.getReplicationFactor().allReplicas == 1 || strat.getReplicationFactor().hasTransientReplicas())
+        {
+            workMap = convertPreferredEndpointsToWorkMap(fetchMap);
+        }
+        else
+        {
+            workMap = getOptimizedWorkMap(fetchMap, sourceFilters, keyspaceName);
+        }
+
+        toFetch.put(keyspaceName, workMap);
+        for (Map.Entry<InetAddressAndPort, Collection<FetchReplica>> entry : workMap.asMap().entrySet())
         {
             if (logger.isTraceEnabled())
             {
-                for (Range<Token> r : entry.getValue())
-                    logger.trace("{}: range {} from source {} for keyspace {}", description, r, entry.getKey(), keyspaceName);
+                for (FetchReplica r : entry.getValue())
+                    logger.trace("{}: range source {} local range {} for keyspace {}", description, r.remote, r.local, keyspaceName);
             }
-            toFetch.put(keyspaceName, entry);
         }
     }
 
     /**
-     * @param keyspaceName keyspace name to check
+     * @param strat AbstractReplicationStrategy of keyspace to check
      * @return true when the node is bootstrapping, useStrictConsistency is true and # of nodes in the cluster is more than # of replica
      */
-    private boolean useStrictSourcesForRanges(String keyspaceName)
+    private boolean useStrictSourcesForRanges(AbstractReplicationStrategy strat)
     {
-        AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
         return useStrictConsistency
                 && tokens != null
-                && metadata.getSizeOfAllEndpoints() != strat.getReplicationFactor();
+                && metadata.getSizeOfAllEndpoints() != strat.getReplicationFactor().allReplicas;
     }
 
     /**
-     * Get a map of all ranges and their respective sources that are candidates for streaming the given ranges
-     * to us. For each range, the list of sources is sorted by proximity relative to the given destAddress.
-     *
-     * @throws java.lang.IllegalStateException when there is no source to get data streamed
+     * Wrapper method to assemble the arguments for invoking the implementation with RangeStreamer's parameters
+     * @param fetchRanges
+     * @param keyspace
+     * @param useStrictConsistency
+     * @return
      */
-    private Multimap<Range<Token>, InetAddressAndPort> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
+    private EndpointsByReplica calculateRangesToFetchWithPreferredEndpoints(ReplicaCollection<?> fetchRanges, Keyspace keyspace, boolean useStrictConsistency)
     {
-        AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Multimap<Range<Token>, InetAddressAndPort> rangeAddresses = strat.getRangeAddresses(metadata.cloneOnlyTokenMap());
+        AbstractReplicationStrategy strat = keyspace.getReplicationStrategy();
 
-        Multimap<Range<Token>, InetAddressAndPort> rangeSources = ArrayListMultimap.create();
-        for (Range<Token> desiredRange : desiredRanges)
-        {
-            for (Range<Token> range : rangeAddresses.keySet())
-            {
-                if (range.contains(desiredRange))
-                {
-                    List<InetAddressAndPort> preferred = snitch.getSortedListByProximity(address, rangeAddresses.get(range));
-                    rangeSources.putAll(desiredRange, preferred);
-                    break;
-                }
-            }
+        TokenMetadata tmd = metadata.cloneOnlyTokenMap();
+
+        TokenMetadata tmdAfter = null;
 
-            if (!rangeSources.keySet().contains(desiredRange))
-                throw new IllegalStateException("No sources found for " + desiredRange);
+        if (tokens != null)
+        {
+            // Pending ranges
+            tmdAfter =  tmd.cloneOnlyTokenMap();
+            tmdAfter.updateNormalTokens(tokens, address);
         }
+        else if (useStrictConsistency)
+        {
+            throw new IllegalArgumentException("Can't ask for strict consistency and not supply tokens");
+        }
+
+        return RangeStreamer.calculateRangesToFetchWithPreferredEndpoints(snitch::sortedByProximity,
+                                                                           strat,
+                                                                           fetchRanges,
+                                                                           useStrictConsistency,
+                                                                           tmd,
+                                                                           tmdAfter,
+                                                                           ALIVE_PREDICATE,
+                                                                           keyspace.getName(),
+                                                                           sourceFilters);
 
-        return rangeSources;
     }
 
     /**
@@ -257,129 +330,234 @@ public class RangeStreamer
      * For each range, the list should only contain a single source. This allows us to consistently migrate data without violating
      * consistency.
      *
-     * @throws java.lang.IllegalStateException when there is no source to get data streamed, or more than 1 source found.
-     */
-    private Multimap<Range<Token>, InetAddressAndPort> getAllRangesWithStrictSourcesFor(String keyspace, Collection<Range<Token>> desiredRanges)
+     **/
+     public static EndpointsByReplica
+     calculateRangesToFetchWithPreferredEndpoints(BiFunction<InetAddressAndPort, EndpointsForRange, EndpointsForRange> snitchGetSortedListByProximity,
+                                                  AbstractReplicationStrategy strat,
+                                                  ReplicaCollection<?> fetchRanges,
+                                                  boolean useStrictConsistency,
+                                                  TokenMetadata tmdBefore,
+                                                  TokenMetadata tmdAfter,
+                                                  Predicate<Replica> isAlive,
+                                                  String keyspace,
+                                                  Collection<Predicate<Replica>> sourceFilters)
     {
-        assert tokens != null;
-        AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
-
-        // Active ranges
-        TokenMetadata metadataClone = metadata.cloneOnlyTokenMap();
-        Multimap<Range<Token>, InetAddressAndPort> addressRanges = strat.getRangeAddresses(metadataClone);
+        EndpointsByRange rangeAddresses = strat.getRangeAddresses(tmdBefore);
 
-        // Pending ranges
-        metadataClone.updateNormalTokens(tokens, address);
-        Multimap<Range<Token>, InetAddressAndPort> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
+        InetAddressAndPort localAddress = FBUtilities.getBroadcastAddressAndPort();
+        logger.debug ("Keyspace: {}", keyspace);
+        logger.debug("To fetch RN: {}", fetchRanges);
+        logger.debug("Fetch ranges: {}", rangeAddresses);
 
-        // Collects the source that will have its range moved to the new node
-        Multimap<Range<Token>, InetAddressAndPort> rangeSources = ArrayListMultimap.create();
+        Predicate<Replica> testSourceFilters = and(sourceFilters);
+        Function<EndpointsForRange, EndpointsForRange> sorted =
+                endpoints -> snitchGetSortedListByProximity.apply(localAddress, endpoints);
 
-        for (Range<Token> desiredRange : desiredRanges)
+        //This list of replicas is just candidates. With strict consistency it's going to be a narrow list.
+        EndpointsByReplica.Mutable rangesToFetchWithPreferredEndpoints = new EndpointsByReplica.Mutable();
+        for (Replica toFetch : fetchRanges)
         {
-            for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> preEntry : addressRanges.asMap().entrySet())
+            //Replica that is sufficient to provide the data we need
+            //With strict consistency and transient replication we may end up with multiple types
+            //so this isn't used with strict consistency
+            Predicate<Replica> isSufficient = r -> (toFetch.isTransient() || r.isFull());
+            Predicate<Replica> accept = r ->
+                       isSufficient.test(r)                 // is sufficient
+                    && !r.endpoint().equals(localAddress)   // is not self
+                    && isAlive.test(r);                     // is alive
+
+            logger.debug("To fetch {}", toFetch);
+            for (Range<Token> range : rangeAddresses.keySet())
             {
-                if (preEntry.getKey().contains(desiredRange))
+                if (range.contains(toFetch.range()))
                 {
-                    Set<InetAddressAndPort> oldEndpoints = Sets.newHashSet(preEntry.getValue());
-                    Set<InetAddressAndPort> newEndpoints = Sets.newHashSet(pendingRangeAddresses.get(desiredRange));
+                    EndpointsForRange oldEndpoints = rangeAddresses.get(range);
 
-                    // Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
-                    // So we need to be careful to only be strict when endpoints == RF
-                    if (oldEndpoints.size() == strat.getReplicationFactor())
+                    //Ultimately we populate this with whatever is going to be fetched from to satisfy toFetch
+                    //It could be multiple endpoints and we must fetch from all of them if they are there
+                    //With transient replication and strict consistency this is to get the full data from a full replica and
+                    //transient data from the transient replica losing data
+                    EndpointsForRange sources;
+                    if (useStrictConsistency)
+                    {
+                        //Start with two sets of who replicates the range before and who replicates it after
+                        EndpointsForRange newEndpoints = strat.calculateNaturalReplicas(toFetch.range().right, tmdAfter);
+                        logger.debug("Old endpoints {}", oldEndpoints);
+                        logger.debug("New endpoints {}", newEndpoints);
+
+                        //Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
+                        //So we need to be careful to only be strict when endpoints == RF
+                        if (oldEndpoints.size() == strat.getReplicationFactor().allReplicas)
+                        {
+                            Set<InetAddressAndPort> endpointsStillReplicated = newEndpoints.endpoints();
+                            // Remove new endpoints from old endpoints based on address
+                            oldEndpoints = oldEndpoints.filter(r -> !endpointsStillReplicated.contains(r.endpoint()));
+
+                            if (!all(oldEndpoints, isAlive))
+                                throw new IllegalStateException("A node required to move the data consistently is down: "
+                                                                + oldEndpoints.filter(not(isAlive)));
+
+                            if (oldEndpoints.size() > 1)
+                                throw new AssertionError("Expected <= 1 endpoint but found " + oldEndpoints);
+
+                            //If we are transitioning from transient to full and and the set of replicas for the range is not changing
+                            //we might end up with no endpoints to fetch from by address. In that case we can pick any full replica safely
+                            //since we are already a transient replica and the existing replica remains.
+                            //The old behavior where we might be asked to fetch ranges we don't need shouldn't occur anymore.
+                            //So it's an error if we don't find what we need.
+                            if (oldEndpoints.isEmpty() && toFetch.isTransient())
+                            {
+                                throw new AssertionError("If there are no endpoints to fetch from then we must be transitioning from transient to full for range " + toFetch);
+                            }
+
+                            if (!any(oldEndpoints, isSufficient))
+                            {
+                                // need an additional replica
+                                EndpointsForRange endpointsForRange = sorted.apply(rangeAddresses.get(range));
+                                // include all our filters, to ensure we include a matching node
+                                Optional<Replica> fullReplica = Iterables.<Replica>tryFind(endpointsForRange, and(accept, testSourceFilters)).toJavaUtil();
+                                if (fullReplica.isPresent())
+                                    oldEndpoints = Endpoints.concat(oldEndpoints, EndpointsForRange.of(fullReplica.get()));
+                                else
+                                    throw new IllegalStateException("Couldn't find any matching sufficient replica out of " + endpointsForRange);
+                            }
+
+                            //We have to check the source filters here to see if they will remove any replicas
+                            //required for strict consistency
+                            if (!all(oldEndpoints, testSourceFilters))
+                                throw new IllegalStateException("Necessary replicas for strict consistency were removed by source filters: " + oldEndpoints.filter(not(testSourceFilters)));
+                        }
+                        else
+                        {
+                            oldEndpoints = sorted.apply(oldEndpoints.filter(accept));
+                        }
+
+                        //Apply testSourceFilters that were given to us, and establish everything remaining is alive for the strict case
+                        sources = oldEndpoints.filter(testSourceFilters);
+                    }
+                    else
                     {
-                        oldEndpoints.removeAll(newEndpoints);
-                        assert oldEndpoints.size() == 1 : "Expected 1 endpoint but found " + oldEndpoints.size();
+                        //Without strict consistency we have given up on correctness so no point in fetching from
+                        //a random full + transient replica since it's also likely to lose data
+                        //Also apply testSourceFilters that were given to us so we can safely select a single source
+                        sources = sorted.apply(rangeAddresses.get(range).filter(and(accept, testSourceFilters)));
+                        //Limit it to just the first possible source, we don't need more than one and downstream
+                        //will fetch from every source we supply
+                        sources = sources.size() > 0 ? sources.subList(0, 1) : sources;
                     }
 
-                    rangeSources.put(desiredRange, oldEndpoints.iterator().next());
+                    // storing range and preferred endpoint set
+                    rangesToFetchWithPreferredEndpoints.putAll(toFetch, sources, Conflict.NONE);
+                    logger.debug("Endpoints to fetch for {} are {}", toFetch, sources);
                 }
             }
 
-            // Validate
-            Collection<InetAddressAndPort> addressList = rangeSources.get(desiredRange);
-            if (addressList == null || addressList.isEmpty())
-                throw new IllegalStateException("No sources found for " + desiredRange);
-
-            if (addressList.size() > 1)
-                throw new IllegalStateException("Multiple endpoints found for " + desiredRange);
+            EndpointsForRange addressList = rangesToFetchWithPreferredEndpoints.getIfPresent(toFetch);
+            if (addressList == null)
+                throw new IllegalStateException("Failed to find endpoints to fetch " + toFetch);
+
+            /*
+             * When we move forwards (shrink our bucket) we are the one losing a range and no one else loses
+             * from that action (we also don't gain). When we move backwards there are two people losing a range. One is a full replica
+             * and the other is a transient replica. So we must need fetch from two places in that case for the full range we gain.
+             * For a transient range we only need to fetch from one.
+             */
+            if (useStrictConsistency && addressList.size() > 1 && (addressList.filter(Replica::isFull).size() > 1 || addressList.filter(Replica::isTransient).size() > 1))
+                throw new IllegalStateException(String.format("Multiple strict sources found for %s, sources: %s", toFetch, addressList));
+
+            //We must have enough stuff to fetch from
+            if ((toFetch.isFull() && !any(addressList, Replica::isFull)) || addressList.isEmpty())
+            {
+                if (strat.getReplicationFactor().allReplicas == 1)
+                {
+                    if (useStrictConsistency)
+                    {
+                        logger.warn("A node required to move the data consistently is down");
+                        throw new IllegalStateException("Unable to find sufficient sources for streaming range " + toFetch + " in keyspace " + keyspace + " with RF=1. " +
+                                                        "Ensure this keyspace contains replicas in the source datacenter.");
+                    }
+                    else
+                        logger.warn("Unable to find sufficient sources for streaming range {} in keyspace {} with RF=1. " +
+                                    "Keyspace might be missing data.", toFetch, keyspace);
 
-            InetAddressAndPort sourceIp = addressList.iterator().next();
-            EndpointState sourceState = Gossiper.instance.getEndpointStateForEndpoint(sourceIp);
-            if (Gossiper.instance.isEnabled() && (sourceState == null || !sourceState.isAlive()))
-                throw new RuntimeException("A node required to move the data consistently is down (" + sourceIp + "). " +
-                                           "If you wish to move the data from a potentially inconsistent replica, restart the node with -Dcassandra.consistent.rangemovement=false");
+                }
+                else
+                {
+                    if (useStrictConsistency)
+                        logger.warn("A node required to move the data consistently is down");
+                    throw new IllegalStateException("Unable to find sufficient sources for streaming range " + toFetch + " in keyspace " + keyspace);
+                }
+            }
         }
-
-        return rangeSources;
+        return rangesToFetchWithPreferredEndpoints.asImmutableView();
     }
 
     /**
-     * @param rangesWithSources The ranges we want to fetch (key) and their potential sources (value)
-     * @param sourceFilters A (possibly empty) collection of source filters to apply. In addition to any filters given
-     *                      here, we always exclude ourselves.
-     * @param keyspace keyspace name
-     * @return Map of source endpoint to collection of ranges
+     * The preferred endpoint list is the wrong format because it is keyed by Replica (this node) rather than the source
+     * endpoint we will fetch from which streaming wants.
+     * @param preferredEndpoints
+     * @return
      */
-    private static Multimap<InetAddressAndPort, Range<Token>> getRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
-                                                                               Collection<ISourceFilter> sourceFilters, String keyspace,
-                                                                               boolean useStrictConsistency)
+    public static Multimap<InetAddressAndPort, FetchReplica> convertPreferredEndpointsToWorkMap(EndpointsByReplica preferredEndpoints)
     {
-        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = HashMultimap.create();
-        for (Range<Token> range : rangesWithSources.keySet())
+        Multimap<InetAddressAndPort, FetchReplica> workMap = HashMultimap.create();
+        for (Map.Entry<Replica, EndpointsForRange> e : preferredEndpoints.entrySet())
         {
-            boolean foundSource = false;
-
-            outer:
-            for (InetAddressAndPort address : rangesWithSources.get(range))
+            for (Replica source : e.getValue())
             {
-                for (ISourceFilter filter : sourceFilters)
-                {
-                    if (!filter.shouldInclude(address))
-                        continue outer;
-                }
+                assert (e.getKey()).isLocal();
+                assert !source.isLocal();
+                workMap.put(source.endpoint(), new FetchReplica(e.getKey(), source));
+            }
+        }
+        logger.debug("Work map {}", workMap);
+        return workMap;
+    }
 
-                if (address.equals(FBUtilities.getBroadcastAddressAndPort()))
-                {
-                    // If localhost is a source, we have found one, but we don't add it to the map to avoid streaming locally
-                    foundSource = true;
-                    continue;
-                }
+    /**
+     * Optimized version that also outputs the final work map
+     */
+    private static Multimap<InetAddressAndPort, FetchReplica> getOptimizedWorkMap(EndpointsByReplica rangesWithSources,
+                                                                                  Collection<Predicate<Replica>> sourceFilters, String keyspace)
+    {
+        //For now we just aren't going to use the optimized range fetch map with transient replication to shrink
+        //the surface area to test and introduce bugs.
+        //In the future it's possible we could run it twice once for full ranges with only full replicas
+        //and once with transient ranges and all replicas. Then merge the result.
+        EndpointsByRange.Mutable unwrapped = new EndpointsByRange.Mutable();
+        for (Map.Entry<Replica, Replica> entry : rangesWithSources.flattenEntries())
+        {
+            Replicas.temporaryAssertFull(entry.getValue());
+            unwrapped.put(entry.getKey().range(), entry.getValue());
+        }
 
-                rangeFetchMapMap.put(address, range);
-                foundSource = true;
-                break; // ensure we only stream from one other node for each range
-            }
+        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(unwrapped.asImmutableView(), sourceFilters, keyspace);
+        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = calculator.getRangeFetchMap();
+        logger.info("Output from RangeFetchMapCalculator for keyspace {}", keyspace);
+        validateRangeFetchMap(unwrapped.asImmutableView(), rangeFetchMapMap, keyspace);
 
-            if (!foundSource)
+        //Need to rewrap as Replicas
+        Multimap<InetAddressAndPort, FetchReplica> wrapped = HashMultimap.create();
+        for (Map.Entry<InetAddressAndPort, Range<Token>> entry : rangeFetchMapMap.entries())
+        {
+            Replica toFetch = null;
+            for (Replica r : rangesWithSources.keySet())
             {
-                AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
-                if (strat != null && strat.getReplicationFactor() == 1)
+                if (r.range().equals(entry.getValue()))
                 {
-                    if (useStrictConsistency)
-                        throw new IllegalStateException("Unable to find sufficient sources for streaming range " + range + " in keyspace " + keyspace + " with RF=1. " +
-                                                        "Ensure this keyspace contains replicas in the source datacenter.");
-                    else
-                        logger.warn("Unable to find sufficient sources for streaming range {} in keyspace {} with RF=1. " +
-                                    "Keyspace might be missing data.", range, keyspace);
+                    if (toFetch != null)
+                        throw new AssertionError(String.format("There shouldn't be multiple replicas for range %s, replica %s and %s here", r.range(), r, toFetch));
+                    toFetch = r;
                 }
-                else
-                    throw new IllegalStateException("Unable to find sufficient sources for streaming range " + range + " in keyspace " + keyspace);
             }
+            if (toFetch == null)
+                throw new AssertionError("Shouldn't be possible for the Replica we fetch to be null here");
+            //Committing the cardinal sin of synthesizing a Replica, but it's ok because we assert earlier all of them
+            //are full and optimized range fetch map doesn't support transient replication yet.
+            wrapped.put(entry.getKey(), new FetchReplica(toFetch, fullReplica(entry.getKey(), entry.getValue())));
         }
 
-        return rangeFetchMapMap;
-    }
-
-
-    private static Multimap<InetAddressAndPort, Range<Token>> getOptimizedRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
-                                                                                        Collection<ISourceFilter> sourceFilters, String keyspace)
-    {
-        RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources, sourceFilters, keyspace);
-        Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = calculator.getRangeFetchMap();
-        logger.info("Output from RangeFetchMapCalculator for keyspace {}", keyspace);
-        validateRangeFetchMap(rangesWithSources, rangeFetchMapMap, keyspace);
-        return rangeFetchMapMap;
+        return wrapped;
     }
 
     /**
@@ -388,7 +566,7 @@ public class RangeStreamer
      * @param rangeFetchMapMap
      * @param keyspace
      */
-    private static void validateRangeFetchMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap, String keyspace)
+    private static void validateRangeFetchMap(EndpointsByRange rangesWithSources, Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap, String keyspace)
     {
         for (Map.Entry<InetAddressAndPort, Range<Token>> entry : rangeFetchMapMap.entries())
         {
@@ -398,7 +576,7 @@ public class RangeStreamer
                                         + " in keyspace " + keyspace);
             }
 
-            if (!rangesWithSources.get(entry.getValue()).contains(entry.getKey()))
+            if (!rangesWithSources.get(entry.getValue()).endpoints().contains(entry.getKey()))
             {
                 throw new IllegalStateException("Trying to stream from wrong endpoint. Range: " + entry.getValue()
                                                 + " in keyspace " + keyspace + " from endpoint: " + entry.getKey());
@@ -408,39 +586,70 @@ public class RangeStreamer
         }
     }
 
-    public static Multimap<InetAddressAndPort, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddressAndPort> rangesWithSourceTarget, String keyspace,
-                                                                        IFailureDetector fd, boolean useStrictConsistency)
-    {
-        return getRangeFetchMap(rangesWithSourceTarget, Collections.<ISourceFilter>singleton(new FailureDetectorSourceFilter(fd)), keyspace, useStrictConsistency);
-    }
-
     // For testing purposes
     @VisibleForTesting
-    Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> toFetch()
+    Multimap<String, Multimap<InetAddressAndPort, FetchReplica>> toFetch()
     {
         return toFetch;
     }
 
     public StreamResultFuture fetchAsync()
     {
-        for (Map.Entry<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> entry : toFetch.entries())
-        {
-            String keyspace = entry.getKey();
-            InetAddressAndPort source = entry.getValue().getKey();
-            Collection<Range<Token>> ranges = entry.getValue().getValue();
+        toFetch.forEach((keyspace, sources) -> {
+            logger.debug("Keyspace {} Sources {}", keyspace, sources);
+            sources.asMap().forEach((source, fetchReplicas) -> {
 
-            // filter out already streamed ranges
-            Set<Range<Token>> availableRanges = stateStore.getAvailableRanges(keyspace, StorageService.instance.getTokenMetadata().partitioner);
-            if (ranges.removeAll(availableRanges))
-            {
-                logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges);
-            }
+                // filter out already streamed ranges
+                RangesAtEndpoint available = stateStore.getAvailableRanges(keyspace, StorageService.instance.getTokenMetadata().partitioner);
 
-            if (logger.isTraceEnabled())
-                logger.trace("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
-            /* Send messages to respective folks to stream data over to me */
-            streamPlan.requestRanges(source, keyspace, ranges);
-        }
+                Predicate<FetchReplica> isAvailable = fetch -> {
+                    Replica availableRange =  available.byRange().get(fetch.local.range());
+                    if (availableRange == null)
+                        //Range is unavailable
+                        return false;
+                    if (fetch.local.isFull())
+                        //For full, pick only replicas with matching transientness
+                        return availableRange.isFull() == fetch.remote.isFull();
+
+                    // Any transient or full will do
+                    return true;
+                };
+
+                List<FetchReplica> remaining = fetchReplicas.stream().filter(not(isAvailable)).collect(Collectors.toList());
+
+                if (remaining.size() < available.size())
+                {
+                    List<FetchReplica> skipped = fetchReplicas.stream().filter(isAvailable).collect(Collectors.toList());
+                    logger.info("Some ranges of {} are already available. Skipping streaming those ranges. Skipping {}. Fully available {} Transiently available {}",
+                                fetchReplicas, skipped, available.filter(Replica::isFull).ranges(), available.filter(Replica::isTransient).ranges());
+                }
+
+                if (logger.isTraceEnabled())
+                    logger.trace("{}ing from {} ranges {}", description, source, StringUtils.join(remaining, ", "));
+
+                //At the other end the distinction between full and transient is ignored it just used the transient status
+                //of the Replica objects we send to determine what to send. The real reason we have this split down to
+                //StreamRequest is that on completion StreamRequest is used to write to the system table tracking
+                //what has already been streamed. At that point since we only have the local Replica instances so we don't
+                //know what we got from the remote. We preserve that here by splitting based on the remotes transient
+                //status.
+                InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort();
+                RangesAtEndpoint full = remaining.stream()
+                        .filter(pair -> pair.remote.isFull())
+                        .map(pair -> pair.local)
+                        .collect(RangesAtEndpoint.collector(self));
+                RangesAtEndpoint transientReplicas = remaining.stream()
+                        .filter(pair -> pair.remote.isTransient())
+                        .map(pair -> pair.local)
+                        .collect(RangesAtEndpoint.collector(self));
+
+                logger.debug("Source and our replicas {}", fetchReplicas);
+                logger.debug("Source {} Keyspace {}  streaming full {} transient {}", source, keyspace, full, transientReplicas);
+
+                /* Send messages to respective folks to stream data over to me */
+                streamPlan.requestRanges(source, keyspace, full, transientReplicas);
+            });
+        });
 
         return streamPlan.execute();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/Splitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Splitter.java b/src/java/org/apache/cassandra/dht/Splitter.java
index c63fe91..8578448 100644
--- a/src/java/org/apache/cassandra/dht/Splitter.java
+++ b/src/java/org/apache/cassandra/dht/Splitter.java
@@ -25,6 +25,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -117,32 +118,31 @@ public abstract class Splitter
         return new BigDecimal(elapsedTokens(token, range)).divide(new BigDecimal(tokensInRange(range)), 3, BigDecimal.ROUND_HALF_EVEN).doubleValue();
     }
 
-    public List<Token> splitOwnedRanges(int parts, List<Range<Token>> localRanges, boolean dontSplitRanges)
+    public List<Token> splitOwnedRanges(int parts, List<WeightedRange> weightedRanges, boolean dontSplitRanges)
     {
-        if (localRanges.isEmpty() || parts == 1)
+        if (weightedRanges.isEmpty() || parts == 1)
             return Collections.singletonList(partitioner.getMaximumToken());
 
         BigInteger totalTokens = BigInteger.ZERO;
-        for (Range<Token> r : localRanges)
+        for (WeightedRange weightedRange : weightedRanges)
         {
-            BigInteger right = valueForToken(token(r.right));
-            totalTokens = totalTokens.add(right.subtract(valueForToken(r.left)));
+            totalTokens = totalTokens.add(weightedRange.totalTokens(this));
         }
+
         BigInteger perPart = totalTokens.divide(BigInteger.valueOf(parts));
         // the range owned is so tiny we can't split it:
         if (perPart.equals(BigInteger.ZERO))
             return Collections.singletonList(partitioner.getMaximumToken());
 
         if (dontSplitRanges)
-            return splitOwnedRangesNoPartialRanges(localRanges, perPart, parts);
+            return splitOwnedRangesNoPartialRanges(weightedRanges, perPart, parts);
 
         List<Token> boundaries = new ArrayList<>();
         BigInteger sum = BigInteger.ZERO;
-        for (Range<Token> r : localRanges)
+        for (WeightedRange weightedRange : weightedRanges)
         {
-            Token right = token(r.right);
-            BigInteger currentRangeWidth = valueForToken(right).subtract(valueForToken(r.left)).abs();
-            BigInteger left = valueForToken(r.left);
+            BigInteger currentRangeWidth = weightedRange.totalTokens(this);
+            BigInteger left = valueForToken(weightedRange.left());
             while (sum.add(currentRangeWidth).compareTo(perPart) >= 0)
             {
                 BigInteger withinRangeBoundary = perPart.subtract(sum);
@@ -155,26 +155,24 @@ public abstract class Splitter
         }
         boundaries.set(boundaries.size() - 1, partitioner.getMaximumToken());
 
-        assert boundaries.size() == parts : boundaries.size() + "!=" + parts + " " + boundaries + ":" + localRanges;
+        assert boundaries.size() == parts : boundaries.size() + "!=" + parts + " " + boundaries + ":" + weightedRanges;
         return boundaries;
     }
 
-    private List<Token> splitOwnedRangesNoPartialRanges(List<Range<Token>> localRanges, BigInteger perPart, int parts)
+    private List<Token> splitOwnedRangesNoPartialRanges(List<WeightedRange> weightedRanges, BigInteger perPart, int parts)
     {
         List<Token> boundaries = new ArrayList<>(parts);
         BigInteger sum = BigInteger.ZERO;
 
         int i = 0;
-        final int rangesCount = localRanges.size();
+        final int rangesCount = weightedRanges.size();
         while (boundaries.size() < parts - 1 && i < rangesCount - 1)
         {
-            Range<Token> r = localRanges.get(i);
-            Range<Token> nextRange = localRanges.get(i + 1);
-            Token right = token(r.right);
-            Token nextRight = token(nextRange.right);
+            WeightedRange r = weightedRanges.get(i);
+            WeightedRange nextRange = weightedRanges.get(i + 1);
 
-            BigInteger currentRangeWidth = valueForToken(right).subtract(valueForToken(r.left));
-            BigInteger nextRangeWidth = valueForToken(nextRight).subtract(valueForToken(nextRange.left));
+            BigInteger currentRangeWidth = r.totalTokens(this);
+            BigInteger nextRangeWidth = nextRange.totalTokens(this);
             sum = sum.add(currentRangeWidth);
 
             // does this or next range take us beyond the per part limit?
@@ -187,7 +185,7 @@ public abstract class Splitter
                 if (diffNext.compareTo(diffCurrent) >= 0)
                 {
                     sum = BigInteger.ZERO;
-                    boundaries.add(right);
+                    boundaries.add(token(r.right()));
                 }
             }
             i++;
@@ -256,4 +254,61 @@ public abstract class Splitter
         }
         return subranges;
     }
+
+    public static class WeightedRange
+    {
+        private final double weight;
+        private final Range<Token> range;
+
+        public WeightedRange(double weight, Range<Token> range)
+        {
+            this.weight = weight;
+            this.range = range;
+        }
+
+        public BigInteger totalTokens(Splitter splitter)
+        {
+            BigInteger right = splitter.valueForToken(splitter.token(range.right));
+            BigInteger left = splitter.valueForToken(range.left);
+            BigInteger factor = BigInteger.valueOf(Math.max(1, (long) (1 / weight)));
+            BigInteger size = right.subtract(left);
+            return size.abs().divide(factor);
+        }
+
+        public Token left()
+        {
+            return range.left;
+        }
+
+        public Token right()
+        {
+            return range.right;
+        }
+
+        public Range<Token> range()
+        {
+            return range;
+        }
+
+        public String toString()
+        {
+            return "WeightedRange{" +
+                   "weight=" + weight +
+                   ", range=" + range +
+                   '}';
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (!(o instanceof WeightedRange)) return false;
+            WeightedRange that = (WeightedRange) o;
+            return Objects.equals(range, that.range);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(weight, range);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/StreamStateStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/StreamStateStore.java b/src/java/org/apache/cassandra/dht/StreamStateStore.java
index e3ea838..3144e81 100644
--- a/src/java/org/apache/cassandra/dht/StreamStateStore.java
+++ b/src/java/org/apache/cassandra/dht/StreamStateStore.java
@@ -19,38 +19,43 @@ package org.apache.cassandra.dht;
 
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamEventHandler;
 import org.apache.cassandra.streaming.StreamRequest;
 import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Store and update available ranges (data already received) to system keyspace.
  */
 public class StreamStateStore implements StreamEventHandler
 {
-    public Set<Range<Token>> getAvailableRanges(String keyspace, IPartitioner partitioner)
+    private static final Logger logger = LoggerFactory.getLogger(StreamStateStore.class);
+
+    public RangesAtEndpoint getAvailableRanges(String keyspace, IPartitioner partitioner)
     {
         return SystemKeyspace.getAvailableRanges(keyspace, partitioner);
     }
 
     /**
-     * Check if given token's data is available in this node.
+     * Check if given token's data is available in this node. This doesn't handle transientness in a useful way
+     * so it's only used by a legacy test
      *
      * @param keyspace keyspace name
      * @param token token to check
      * @return true if given token in the keyspace is already streamed and ready to be served.
      */
+    @VisibleForTesting
     public boolean isDataAvailable(String keyspace, Token token)
     {
-        Set<Range<Token>> availableRanges = getAvailableRanges(keyspace, token.getPartitioner());
-        for (Range<Token> range : availableRanges)
-        {
-            if (range.contains(token))
-                return true;
-        }
-        return false;
+        RangesAtEndpoint availableRanges = getAvailableRanges(keyspace, token.getPartitioner());
+        return availableRanges.ranges().stream().anyMatch(range -> range.contains(token));
     }
 
     /**
@@ -73,7 +78,7 @@ public class StreamStateStore implements StreamEventHandler
                 }
                 for (StreamRequest request : se.requests)
                 {
-                    SystemKeyspace.updateAvailableRanges(request.keyspace, request.ranges);
+                    SystemKeyspace.updateAvailableRanges(request.keyspace, request.full.ranges(), request.transientReplicas.ranges());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
index efd2766..36fc8c2 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
@@ -73,7 +73,7 @@ class ReplicationAwareTokenAllocator<Unit> extends TokenAllocatorBase<Unit>
         Map<Unit, UnitInfo<Unit>> unitInfos = createUnitInfos(groups);
         if (groups.size() < replicas)
         {
-            // We need at least replicas groups to do allocation correctly. If there aren't enough, 
+            // We need at least replicas groups to do allocation correctly. If there aren't enough,
             // use random allocation.
             // This part of the code should only be reached via the RATATest. StrategyAdapter should disallow
             // token allocation in this case as the algorithm is not able to cover the behavior of NetworkTopologyStrategy.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
index 61082df..ef91fbb 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
@@ -113,7 +113,7 @@ public class TokenAllocation
     {
         double size = current.size(next);
         Token representative = current.getPartitioner().midpoint(current, next);
-        for (InetAddressAndPort n : rs.calculateNaturalEndpoints(representative, tokenMetadata))
+        for (InetAddressAndPort n : rs.calculateNaturalReplicas(representative, tokenMetadata).endpoints())
         {
             Double v = ownership.get(n);
             ownership.put(n, v != null ? v + size : size);
@@ -169,7 +169,7 @@ public class TokenAllocation
 
     static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final SimpleStrategy rs, final InetAddressAndPort endpoint)
     {
-        final int replicas = rs.getReplicationFactor();
+        final int replicas = rs.getReplicationFactor().allReplicas;
 
         return new StrategyAdapter()
         {
@@ -196,7 +196,7 @@ public class TokenAllocation
     static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final NetworkTopologyStrategy rs, final IEndpointSnitch snitch, final InetAddressAndPort endpoint)
     {
         final String dc = snitch.getDatacenter(endpoint);
-        final int replicas = rs.getReplicationFactor(dc);
+        final int replicas = rs.getReplicationFactor(dc).allReplicas;
 
         if (replicas == 0 || replicas == 1)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/exceptions/UnavailableException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnavailableException.java b/src/java/org/apache/cassandra/exceptions/UnavailableException.java
index 7b4edd8..d6e8488 100644
--- a/src/java/org/apache/cassandra/exceptions/UnavailableException.java
+++ b/src/java/org/apache/cassandra/exceptions/UnavailableException.java
@@ -25,14 +25,26 @@ public class UnavailableException extends RequestExecutionException
     public final int required;
     public final int alive;
 
-    public UnavailableException(ConsistencyLevel consistency, int required, int alive)
+    public static UnavailableException create(ConsistencyLevel consistency, int required, int alive)
     {
-        this("Cannot achieve consistency level " + consistency, consistency, required, alive);
+        assert alive < required;
+        return create(consistency, required, 0, alive, 0);
     }
 
-    public UnavailableException(ConsistencyLevel consistency, String dc, int required, int alive)
+    public static UnavailableException create(ConsistencyLevel consistency, int required, int requiredFull, int alive, int aliveFull)
     {
-        this("Cannot achieve consistency level " + consistency + " in DC " + dc, consistency, required, alive);
+        if (required > alive)
+            return new UnavailableException("Cannot achieve consistency level " + consistency, consistency, required, alive);
+        assert requiredFull < aliveFull;
+        return new UnavailableException("Insufficient full replicas", consistency, required, alive);
+    }
+
+    public static UnavailableException create(ConsistencyLevel consistency, String dc, int required, int requiredFull, int alive, int aliveFull)
+    {
+        if (required > alive)
+            return new UnavailableException("Cannot achieve consistency level " + consistency + " in DC " + dc, consistency, required, alive);
+        assert requiredFull < aliveFull;
+        return new UnavailableException("Insufficient full replicas in DC " + dc, consistency, required, alive);
     }
 
     public UnavailableException(String msg, ConsistencyLevel consistency, int required, int alive)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/gms/EndpointState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index 5646bf6..8546a70 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -144,6 +144,11 @@ public class EndpointState
         return rpcState != null && Boolean.parseBoolean(rpcState.value);
     }
 
+    public boolean isNormalState()
+    {
+        return getStatus().equals(VersionedValue.STATUS_NORMAL);
+    }
+
     public String getStatus()
     {
         VersionedValue status = getApplicationState(ApplicationState.STATUS_WITH_PORT);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/hints/HintsService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsService.java b/src/java/org/apache/cassandra/hints/HintsService.java
index 0cd1278..c6ad3d9 100644
--- a/src/java/org/apache/cassandra/hints/HintsService.java
+++ b/src/java/org/apache/cassandra/hints/HintsService.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.hints;
 import java.io.File;
 import java.lang.management.ManagementFactory;
 import java.net.UnknownHostException;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -39,6 +42,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.IFailureDetector;
+import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.metrics.HintedHandoffMetrics;
 import org.apache.cassandra.metrics.StorageMetrics;
@@ -46,9 +50,7 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
 
-import static com.google.common.collect.Iterables.filter;
 import static com.google.common.collect.Iterables.transform;
-import static com.google.common.collect.Iterables.size;
 
 /**
  * A singleton-ish wrapper over various hints components:
@@ -151,7 +153,7 @@ public final class HintsService implements HintsServiceMBean
      * @param hostIds host ids of the hint's target nodes
      * @param hint the hint to store
      */
-    public void write(Iterable<UUID> hostIds, Hint hint)
+    public void write(Collection<UUID> hostIds, Hint hint)
     {
         if (isShutDown)
             throw new IllegalStateException("HintsService is shut down and can't accept new hints");
@@ -161,7 +163,7 @@ public final class HintsService implements HintsServiceMBean
 
         bufferPool.write(hostIds, hint);
 
-        StorageMetrics.totalHints.inc(size(hostIds));
+        StorageMetrics.totalHints.inc(hostIds.size());
     }
 
     /**
@@ -183,9 +185,14 @@ public final class HintsService implements HintsServiceMBean
         String keyspaceName = hint.mutation.getKeyspaceName();
         Token token = hint.mutation.key().getToken();
 
-        Iterable<UUID> hostIds =
-        transform(filter(StorageService.instance.getNaturalAndPendingEndpoints(keyspaceName, token), StorageProxy::shouldHint),
-                  StorageService.instance::getHostIdForEndpoint);
+        EndpointsForToken replicas = StorageService.instance.getNaturalAndPendingReplicasForToken(keyspaceName, token);
+
+        // judicious use of streams: eagerly materializing probably cheaper
+        // than performing filters / translations 2x extra via Iterables.filter/transform
+        List<UUID> hostIds = replicas.stream()
+                .filter(StorageProxy::shouldHint)
+                .map(replica -> StorageService.instance.getHostIdForEndpoint(replica.endpoint()))
+                .collect(Collectors.toList());
 
         write(hostIds, hint);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 044a00b..4eaf1fe 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -69,13 +69,14 @@ abstract class AbstractSSTableSimpleWriter implements Closeable
         SerializationHeader header = new SerializationHeader(true, metadata.get(), columns, EncodingStats.NO_STATS);
 
         if (makeRangeAware)
-            return SSTableTxnWriter.createRangeAware(metadata, 0,  ActiveRepairService.UNREPAIRED_SSTABLE, ActiveRepairService.NO_PENDING_REPAIR, formatType, 0, header);
+            return SSTableTxnWriter.createRangeAware(metadata, 0,  ActiveRepairService.UNREPAIRED_SSTABLE, ActiveRepairService.NO_PENDING_REPAIR, false, formatType, 0, header);
 
         return SSTableTxnWriter.create(metadata,
                                        createDescriptor(directory, metadata.keyspace, metadata.name, formatType),
                                        0,
                                        ActiveRepairService.UNREPAIRED_SSTABLE,
                                        ActiveRepairService.NO_PENDING_REPAIR,
+                                       false,
                                        0,
                                        header,
                                        Collections.emptySet());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index f2605fb..055bf24 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -23,6 +23,7 @@ import java.nio.charset.Charset;
 import java.util.*;
 import java.util.concurrent.CopyOnWriteArraySet;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Sets;
@@ -46,6 +47,9 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
+
 /**
  * This class is built on top of the SequenceFile. It stores
  * data on disk in sorted fashion. However the sorting is upto
@@ -350,4 +354,13 @@ public abstract class SSTable
     {
         return AbstractBounds.bounds(first.getToken(), true, last.getToken(), true);
     }
+
+    public static void validateRepairedMetadata(long repairedAt, UUID pendingRepair, boolean isTransient)
+    {
+        Preconditions.checkArgument((pendingRepair == NO_PENDING_REPAIR) || (repairedAt == UNREPAIRED_SSTABLE),
+                                    "pendingRepair cannot be set on a repaired sstable");
+        Preconditions.checkArgument(!isTransient || (pendingRepair != NO_PENDING_REPAIR),
+                                    "isTransient can only be true for sstables pending repair");
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 4ba0533..ec2a700 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -126,7 +126,7 @@ public class SSTableLoader implements StreamEventHandler
                                               for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : ranges.entrySet())
                                               {
                                                   InetAddressAndPort endpoint = entry.getKey();
-                                                  Collection<Range<Token>> tokenRanges = entry.getValue();
+                                                  List<Range<Token>> tokenRanges = Range.normalize(entry.getValue());
 
                                                   List<SSTableReader.PartitionPositionBounds> sstableSections = sstable.getPositionsForRanges(tokenRanges);
                                                   long estimatedKeys = sstable.estimatedKeysForRanges(tokenRanges);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
index 60b8962..cfb1365 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
@@ -99,10 +99,10 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
     }
 
     @SuppressWarnings("resource") // log and writer closed during doPostCleanup
-    public static SSTableTxnWriter create(ColumnFamilyStore cfs, Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, int sstableLevel, SerializationHeader header)
+    public static SSTableTxnWriter create(ColumnFamilyStore cfs, Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, int sstableLevel, SerializationHeader header)
     {
         LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE);
-        SSTableMultiWriter writer = cfs.createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, sstableLevel, header, txn);
+        SSTableMultiWriter writer = cfs.createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, sstableLevel, header, txn);
         return new SSTableTxnWriter(txn, writer);
     }
 
@@ -112,6 +112,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
                                                     long keyCount,
                                                     long repairedAt,
                                                     UUID pendingRepair,
+                                                    boolean isTransient,
                                                     SSTableFormat.Type type,
                                                     int sstableLevel,
                                                     SerializationHeader header)
@@ -122,7 +123,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
         SSTableMultiWriter writer;
         try
         {
-            writer = new RangeAwareSSTableWriter(cfs, keyCount, repairedAt, pendingRepair, type, sstableLevel, 0, txn, header);
+            writer = new RangeAwareSSTableWriter(cfs, keyCount, repairedAt, pendingRepair, isTransient, type, sstableLevel, 0, txn, header);
         }
         catch (IOException e)
         {
@@ -140,6 +141,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
                                           long keyCount,
                                           long repairedAt,
                                           UUID pendingRepair,
+                                          boolean isTransient,
                                           int sstableLevel,
                                           SerializationHeader header,
                                           Collection<Index> indexes)
@@ -147,12 +149,12 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
         // if the column family store does not exist, we create a new default SSTableMultiWriter to use:
         LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE);
         MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel);
-        SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, metadata, collector, header, indexes, txn);
+        SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexes, txn);
         return new SSTableTxnWriter(txn, writer);
     }
 
-    public static SSTableTxnWriter create(ColumnFamilyStore cfs, Descriptor desc, long keyCount, long repairedAt, UUID pendingRepair, SerializationHeader header)
+    public static SSTableTxnWriter create(ColumnFamilyStore cfs, Descriptor desc, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, SerializationHeader header)
     {
-        return create(cfs, desc, keyCount, repairedAt, pendingRepair, 0, header);
+        return create(cfs, desc, keyCount, repairedAt, pendingRepair, isTransient, 0, header);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java b/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
index a40ec18..eb5c5fe 100644
--- a/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
@@ -111,13 +111,14 @@ public class SimpleSSTableMultiWriter implements SSTableMultiWriter
                                             long keyCount,
                                             long repairedAt,
                                             UUID pendingRepair,
+                                            boolean isTransient,
                                             TableMetadataRef metadata,
                                             MetadataCollector metadataCollector,
                                             SerializationHeader header,
                                             Collection<Index> indexes,
                                             LifecycleTransaction txn)
     {
-        SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, pendingRepair, metadata, metadataCollector, header, indexes, txn);
+        SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, indexes, txn);
         return new SimpleSSTableMultiWriter(writer, txn);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
index f289fe3..29fa573 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
@@ -44,6 +44,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
     private final long estimatedKeys;
     private final long repairedAt;
     private final UUID pendingRepair;
+    private final boolean isTransient;
     private final SSTableFormat.Type format;
     private final SerializationHeader header;
     private final LifecycleTransaction txn;
@@ -53,7 +54,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
     private final List<SSTableReader> finishedReaders = new ArrayList<>();
     private SSTableMultiWriter currentWriter = null;
 
-    public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, UUID pendingRepair, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader header) throws IOException
+    public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, UUID pendingRepair, boolean isTransient, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader header) throws IOException
     {
         DiskBoundaries db = cfs.getDiskBoundaries();
         directories = db.directories;
@@ -62,6 +63,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
         this.estimatedKeys = estimatedKeys / directories.size();
         this.repairedAt = repairedAt;
         this.pendingRepair = pendingRepair;
+        this.isTransient = isTransient;
         this.format = format;
         this.txn = txn;
         this.header = header;
@@ -73,7 +75,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
                 throw new IOException(String.format("Insufficient disk space to store %s",
                                                     FBUtilities.prettyPrintMemory(totalSize)));
             Descriptor desc = cfs.newSSTableDescriptor(cfs.getDirectories().getLocationForDisk(localDir), format);
-            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, pendingRepair, sstableLevel, header, txn);
+            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, pendingRepair, isTransient, sstableLevel, header, txn);
         }
     }
 
@@ -95,7 +97,7 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
                 finishedWriters.add(currentWriter);
 
             Descriptor desc = cfs.newSSTableDescriptor(cfs.getDirectories().getLocationForDisk(directories.get(currentIndex)), format);
-            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, pendingRepair, sstableLevel, header, txn);
+            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, pendingRepair, isTransient, sstableLevel, header, txn);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 2fade21..edb3afa 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -1852,6 +1852,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         return sstableMetadata.repairedAt;
     }
 
+    public boolean isTransient()
+    {
+        return sstableMetadata.isTransient;
+    }
+
     public boolean intersects(Collection<Range<Token>> ranges)
     {
         Bounds<Token> range = new Bounds<>(first.getToken(), last.getToken());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 1e183e2..cca59cf 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -55,6 +55,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
 {
     protected long repairedAt;
     protected UUID pendingRepair;
+    protected boolean isTransient;
     protected long maxDataAge = -1;
     protected final long keyCount;
     protected final MetadataCollector metadataCollector;
@@ -77,6 +78,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                             long keyCount,
                             long repairedAt,
                             UUID pendingRepair,
+                            boolean isTransient,
                             TableMetadataRef metadata,
                             MetadataCollector metadataCollector,
                             SerializationHeader header,
@@ -86,6 +88,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         this.keyCount = keyCount;
         this.repairedAt = repairedAt;
         this.pendingRepair = pendingRepair;
+        this.isTransient = isTransient;
         this.metadataCollector = metadataCollector;
         this.header = header;
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), descriptor.version, header);
@@ -96,6 +99,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        Long keyCount,
                                        Long repairedAt,
                                        UUID pendingRepair,
+                                       boolean isTransient,
                                        TableMetadataRef metadata,
                                        MetadataCollector metadataCollector,
                                        SerializationHeader header,
@@ -103,20 +107,21 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        LifecycleTransaction txn)
     {
         Factory writerFactory = descriptor.getFormat().getWriterFactory();
-        return writerFactory.open(descriptor, keyCount, repairedAt, pendingRepair, metadata, metadataCollector, header, observers(descriptor, indexes, txn.opType()), txn);
+        return writerFactory.open(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers(descriptor, indexes, txn.opType()), txn);
     }
 
     public static SSTableWriter create(Descriptor descriptor,
                                        long keyCount,
                                        long repairedAt,
                                        UUID pendingRepair,
+                                       boolean isTransient,
                                        int sstableLevel,
                                        SerializationHeader header,
                                        Collection<Index> indexes,
                                        LifecycleTransaction txn)
     {
         TableMetadataRef metadata = Schema.instance.getTableMetadataRef(descriptor);
-        return create(metadata, descriptor, keyCount, repairedAt, pendingRepair, sstableLevel, header, indexes, txn);
+        return create(metadata, descriptor, keyCount, repairedAt, pendingRepair, isTransient, sstableLevel, header, indexes, txn);
     }
 
     public static SSTableWriter create(TableMetadataRef metadata,
@@ -124,13 +129,14 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        long keyCount,
                                        long repairedAt,
                                        UUID pendingRepair,
+                                       boolean isTransient,
                                        int sstableLevel,
                                        SerializationHeader header,
                                        Collection<Index> indexes,
                                        LifecycleTransaction txn)
     {
         MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel);
-        return create(descriptor, keyCount, repairedAt, pendingRepair, metadata, collector, header, indexes, txn);
+        return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexes, txn);
     }
 
     @VisibleForTesting
@@ -138,11 +144,12 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        long keyCount,
                                        long repairedAt,
                                        UUID pendingRepair,
+                                       boolean isTransient,
                                        SerializationHeader header,
                                        Collection<Index> indexes,
                                        LifecycleTransaction txn)
     {
-        return create(descriptor, keyCount, repairedAt, pendingRepair, 0, header, indexes, txn);
+        return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, 0, header, indexes, txn);
     }
 
     private static Set<Component> components(TableMetadata metadata)
@@ -309,6 +316,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                                   metadata().params.bloomFilterFpChance,
                                                   repairedAt,
                                                   pendingRepair,
+                                                  isTransient,
                                                   header);
     }
 
@@ -338,6 +346,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                            long keyCount,
                                            long repairedAt,
                                            UUID pendingRepair,
+                                           boolean isTransient,
                                            TableMetadataRef metadata,
                                            MetadataCollector metadataCollector,
                                            SerializationHeader header,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/io/sstable/format/Version.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
index 1d965ce..9b82c14 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/Version.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -55,6 +55,8 @@ public abstract class Version
 
     public abstract boolean hasPendingRepair();
 
+    public abstract boolean hasIsTransient();
+
     public abstract boolean hasMetadataChecksum();
 
     /**


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


[10/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index ed0cafc..c23eb88 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -29,7 +29,6 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.cache.CacheLoader;
 import com.google.common.collect.*;
 import com.google.common.primitives.Ints;
@@ -133,18 +132,10 @@ public class StorageProxy implements StorageProxyMBean
         HintsService.instance.registerMBean();
         HintedHandOffManager.instance.registerMBean();
 
-        standardWritePerformer = new WritePerformer()
+        standardWritePerformer = (mutation, targets, responseHandler, localDataCenter) ->
         {
-            public void apply(IMutation mutation,
-                              Iterable<InetAddressAndPort> targets,
-                              AbstractWriteResponseHandler<IMutation> responseHandler,
-                              String localDataCenter,
-                              ConsistencyLevel consistency_level)
-            throws OverloadedException
-            {
-                assert mutation instanceof Mutation;
-                sendToHintedEndpoints((Mutation) mutation, targets, responseHandler, localDataCenter, Stage.MUTATION);
-            }
+            assert mutation instanceof Mutation;
+            sendToHintedReplicas((Mutation) mutation, targets.selected(), responseHandler, localDataCenter, Stage.MUTATION);
         };
 
         /*
@@ -153,29 +144,19 @@ public class StorageProxy implements StorageProxyMBean
          * but on the latter case, the verb handler already run on the COUNTER_MUTATION stage, so we must not execute the
          * underlying on the stage otherwise we risk a deadlock. Hence two different performer.
          */
-        counterWritePerformer = new WritePerformer()
+        counterWritePerformer = (mutation, targets, responseHandler, localDataCenter) ->
         {
-            public void apply(IMutation mutation,
-                              Iterable<InetAddressAndPort> targets,
-                              AbstractWriteResponseHandler<IMutation> responseHandler,
-                              String localDataCenter,
-                              ConsistencyLevel consistencyLevel)
-            {
-                counterWriteTask(mutation, targets, responseHandler, localDataCenter).run();
-            }
+            EndpointsForToken selected = targets.selected().withoutSelf();
+            Replicas.temporaryAssertFull(selected); // TODO CASSANDRA-14548
+            counterWriteTask(mutation, selected, responseHandler, localDataCenter).run();
         };
 
-        counterWriteOnCoordinatorPerformer = new WritePerformer()
+        counterWriteOnCoordinatorPerformer = (mutation, targets, responseHandler, localDataCenter) ->
         {
-            public void apply(IMutation mutation,
-                              Iterable<InetAddressAndPort> targets,
-                              AbstractWriteResponseHandler<IMutation> responseHandler,
-                              String localDataCenter,
-                              ConsistencyLevel consistencyLevel)
-            {
-                StageManager.getStage(Stage.COUNTER_MUTATION)
-                            .execute(counterWriteTask(mutation, targets, responseHandler, localDataCenter));
-            }
+            EndpointsForToken selected = targets.selected().withoutSelf();
+            Replicas.temporaryAssertFull(selected); // TODO CASSANDRA-14548
+            StageManager.getStage(Stage.COUNTER_MUTATION)
+                        .execute(counterWriteTask(mutation, selected, responseHandler, localDataCenter));
         };
 
         for(ConsistencyLevel level : ConsistencyLevel.values())
@@ -251,11 +232,9 @@ public class StorageProxy implements StorageProxyMBean
             while (System.nanoTime() - queryStartNanoTime < timeout)
             {
                 // for simplicity, we'll do a single liveness check at the start of each attempt
-                PaxosParticipants p = getPaxosParticipants(metadata, key, consistencyForPaxos);
-                List<InetAddressAndPort> liveEndpoints = p.liveEndpoints;
-                int requiredParticipants = p.participants;
+                ReplicaLayout.ForPaxos replicaLayout = ReplicaLayout.forPaxos(Keyspace.open(keyspaceName), key, consistencyForPaxos);
 
-                final PaxosBallotAndContention pair = beginAndRepairPaxos(queryStartNanoTime, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit, true, state);
+                final PaxosBallotAndContention pair = beginAndRepairPaxos(queryStartNanoTime, key, metadata, replicaLayout, consistencyForPaxos, consistencyForCommit, true, state);
                 final UUID ballot = pair.ballot;
                 contentions += pair.contentions;
 
@@ -297,7 +276,7 @@ public class StorageProxy implements StorageProxyMBean
 
                 Commit proposal = Commit.newProposal(ballot, updates);
                 Tracing.trace("CAS precondition is met; proposing client-requested updates for {}", ballot);
-                if (proposePaxos(proposal, liveEndpoints, requiredParticipants, true, consistencyForPaxos, queryStartNanoTime))
+                if (proposePaxos(proposal, replicaLayout, true, queryStartNanoTime))
                 {
                     commitPaxos(proposal, consistencyForCommit, true, queryStartNanoTime);
                     Tracing.trace("CAS successful");
@@ -346,49 +325,6 @@ public class StorageProxy implements StorageProxyMBean
             casWriteMetrics.contention.update(contentions);
     }
 
-    private static Predicate<InetAddressAndPort> sameDCPredicateFor(final String dc)
-    {
-        final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        return new Predicate<InetAddressAndPort>()
-        {
-            public boolean apply(InetAddressAndPort host)
-            {
-                return dc.equals(snitch.getDatacenter(host));
-            }
-        };
-    }
-
-    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);
-        if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
-        {
-            // Restrict naturalEndpoints and pendingEndpoints 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));
-        }
-        int participants = pendingEndpoints.size() + naturalEndpoints.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());
-
-        // We cannot allow CAS operations with 2 or more pending endpoints, see #8346.
-        // Note that we fake an impossible number of required nodes in the unavailable exception
-        // to nail home the point that it's an impossible operation no matter how many nodes are live.
-        if (pendingEndpoints.size() > 1)
-            throw new UnavailableException(String.format("Cannot perform LWT operation as there is more than one (%d) pending range movement", pendingEndpoints.size()),
-                                           consistencyForPaxos,
-                                           participants + 1,
-                                           liveEndpoints.size());
-
-        return new PaxosParticipants(liveEndpoints, requiredParticipants);
-    }
-
     /**
      * begin a Paxos session by sending a prepare request and completing any in-progress requests seen in the replies
      *
@@ -396,14 +332,13 @@ public class StorageProxy implements StorageProxyMBean
      * nodes have seen the mostRecentCommit.  Otherwise, return null.
      */
     private static PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoTime,
-                                                           DecoratedKey key,
-                                                           TableMetadata metadata,
-                                                           List<InetAddressAndPort> liveEndpoints,
-                                                           int requiredParticipants,
-                                                           ConsistencyLevel consistencyForPaxos,
-                                                           ConsistencyLevel consistencyForCommit,
-                                                           final boolean isWrite,
-                                                           ClientState state)
+                                                                DecoratedKey key,
+                                                                TableMetadata metadata,
+                                                                ReplicaLayout.ForPaxos replicaLayout,
+                                                                ConsistencyLevel consistencyForPaxos,
+                                                                ConsistencyLevel consistencyForCommit,
+                                                                final boolean isWrite,
+                                                                ClientState state)
     throws WriteTimeoutException, WriteFailureException
     {
         long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
@@ -425,7 +360,7 @@ public class StorageProxy implements StorageProxyMBean
             // prepare
             Tracing.trace("Preparing {}", ballot);
             Commit toPrepare = Commit.newPrepare(key, metadata, ballot);
-            summary = preparePaxos(toPrepare, liveEndpoints, requiredParticipants, consistencyForPaxos, queryStartNanoTime);
+            summary = preparePaxos(toPrepare, replicaLayout, queryStartNanoTime);
             if (!summary.promised)
             {
                 Tracing.trace("Some replicas have already promised a higher ballot than ours; aborting");
@@ -448,7 +383,7 @@ public class StorageProxy implements StorageProxyMBean
                 else
                     casReadMetrics.unfinishedCommit.inc();
                 Commit refreshedInProgress = Commit.newProposal(ballot, inProgress.update);
-                if (proposePaxos(refreshedInProgress, liveEndpoints, requiredParticipants, false, consistencyForPaxos, queryStartNanoTime))
+                if (proposePaxos(refreshedInProgress, replicaLayout, false, queryStartNanoTime))
                 {
                     try
                     {
@@ -505,14 +440,14 @@ public class StorageProxy implements StorageProxyMBean
             MessagingService.instance().sendOneWay(message, target);
     }
 
-    private static PrepareCallback preparePaxos(Commit toPrepare, List<InetAddressAndPort> endpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, long queryStartNanoTime)
+    private static PrepareCallback preparePaxos(Commit toPrepare, ReplicaLayout.ForPaxos replicaLayout, long queryStartNanoTime)
     throws WriteTimeoutException
     {
-        PrepareCallback callback = new PrepareCallback(toPrepare.update.partitionKey(), toPrepare.update.metadata(), requiredParticipants, consistencyForPaxos, queryStartNanoTime);
+        PrepareCallback callback = new PrepareCallback(toPrepare.update.partitionKey(), toPrepare.update.metadata(), replicaLayout.getRequiredParticipants(), replicaLayout.consistencyLevel(), queryStartNanoTime);
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PREPARE, toPrepare, Commit.serializer);
-        for (InetAddressAndPort target : endpoints)
+        for (Replica replica: replicaLayout.selected())
         {
-            if (canDoLocalRequest(target))
+            if (replica.isLocal())
             {
                 StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_PREPARE)).execute(new Runnable()
                 {
@@ -536,21 +471,21 @@ public class StorageProxy implements StorageProxyMBean
             }
             else
             {
-                MessagingService.instance().sendRR(message, target, callback);
+                MessagingService.instance().sendRR(message, replica.endpoint(), callback);
             }
         }
         callback.await();
         return callback;
     }
 
-    private static boolean proposePaxos(Commit proposal, List<InetAddressAndPort> endpoints, int requiredParticipants, boolean timeoutIfPartial, ConsistencyLevel consistencyLevel, long queryStartNanoTime)
+    private static boolean proposePaxos(Commit proposal, ReplicaLayout.ForPaxos replicaLayout, boolean timeoutIfPartial, long queryStartNanoTime)
     throws WriteTimeoutException
     {
-        ProposeCallback callback = new ProposeCallback(endpoints.size(), requiredParticipants, !timeoutIfPartial, consistencyLevel, queryStartNanoTime);
+        ProposeCallback callback = new ProposeCallback(replicaLayout.selected().size(), replicaLayout.getRequiredParticipants(), !timeoutIfPartial, replicaLayout.consistencyLevel(), queryStartNanoTime);
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PROPOSE, proposal, Commit.serializer);
-        for (InetAddressAndPort target : endpoints)
+        for (Replica replica : replicaLayout.selected())
         {
-            if (canDoLocalRequest(target))
+            if (replica.isLocal())
             {
                 StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_PROPOSE)).execute(new Runnable()
                 {
@@ -574,7 +509,7 @@ public class StorageProxy implements StorageProxyMBean
             }
             else
             {
-                MessagingService.instance().sendRR(message, target, callback);
+                MessagingService.instance().sendRR(message, replica.endpoint(), callback);
             }
         }
         callback.await();
@@ -583,7 +518,7 @@ public class StorageProxy implements StorageProxyMBean
             return true;
 
         if (timeoutIfPartial && !callback.isFullyRefused())
-            throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, callback.getAcceptCount(), requiredParticipants);
+            throw new WriteTimeoutException(WriteType.CAS, replicaLayout.consistencyLevel(), callback.getAcceptCount(), replicaLayout.getRequiredParticipants());
 
         return false;
     }
@@ -594,30 +529,30 @@ public class StorageProxy implements StorageProxyMBean
         Keyspace keyspace = Keyspace.open(proposal.update.metadata().keyspace);
 
         Token tk = proposal.update.partitionKey().getToken();
-        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
-        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
 
         AbstractWriteResponseHandler<Commit> responseHandler = null;
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWrite(keyspace, consistencyLevel, tk, FailureDetector.instance::isAlive);
         if (shouldBlock)
         {
             AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
-            responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistencyLevel, null, WriteType.SIMPLE, queryStartNanoTime);
+            responseHandler = rs.getWriteResponseHandler(replicaLayout, null, WriteType.SIMPLE, queryStartNanoTime);
             responseHandler.setSupportsBackPressure(false);
         }
 
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, proposal, Commit.serializer);
-        for (InetAddressAndPort destination : Iterables.concat(naturalEndpoints, pendingEndpoints))
+        for (Replica replica : replicaLayout.all())
         {
-            checkHintOverload(destination);
+            InetAddressAndPort destination = replica.endpoint();
+            checkHintOverload(replica);
 
             if (FailureDetector.instance.isAlive(destination))
             {
                 if (shouldBlock)
                 {
-                    if (canDoLocalRequest(destination))
-                        commitPaxosLocal(message, responseHandler);
+                    if (replica.isLocal())
+                        commitPaxosLocal(replica, message, responseHandler);
                     else
-                        MessagingService.instance().sendRR(message, destination, responseHandler, allowHints && shouldHint(destination));
+                        MessagingService.instance().sendWriteRR(message, replica, responseHandler, allowHints && shouldHint(replica));
                 }
                 else
                 {
@@ -630,9 +565,9 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     responseHandler.expired();
                 }
-                if (allowHints && shouldHint(destination))
+                if (allowHints && shouldHint(replica))
                 {
-                    submitHint(proposal.makeMutation(), destination, null);
+                    submitHint(proposal.makeMutation(), replica, null);
                 }
             }
         }
@@ -646,9 +581,9 @@ public class StorageProxy implements StorageProxyMBean
      * submit a fake one that executes immediately on the mutation stage, but generates the necessary backpressure
      * signal for hints
      */
-    private static void commitPaxosLocal(final MessageOut<Commit> message, final AbstractWriteResponseHandler<?> responseHandler)
+    private static void commitPaxosLocal(Replica localReplica, final MessageOut<Commit> message, final AbstractWriteResponseHandler<?> responseHandler)
     {
-        StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_COMMIT)).maybeExecuteImmediately(new LocalMutationRunnable()
+        StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_COMMIT)).maybeExecuteImmediately(new LocalMutationRunnable(localReplica)
         {
             public void runMayThrow()
             {
@@ -684,35 +619,37 @@ public class StorageProxy implements StorageProxyMBean
      * @param consistency_level the consistency level for the operation
      * @param queryStartNanoTime the value of System.nanoTime() when the query started to be processed
      */
-    public static void mutate(Collection<? extends IMutation> mutations, ConsistencyLevel consistency_level, long queryStartNanoTime)
+    public static void mutate(List<? extends IMutation> mutations, ConsistencyLevel consistency_level, long queryStartNanoTime)
     throws UnavailableException, OverloadedException, WriteTimeoutException, WriteFailureException
     {
         Tracing.trace("Determining replicas for mutation");
         final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
         long startTime = System.nanoTime();
+
         List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size());
+        WriteType plainWriteType = mutations.size() <= 1 ? WriteType.SIMPLE : WriteType.UNLOGGED_BATCH;
 
         try
         {
             for (IMutation mutation : mutations)
             {
                 if (mutation instanceof CounterMutation)
-                {
                     responseHandlers.add(mutateCounter((CounterMutation)mutation, localDataCenter, queryStartNanoTime));
-                }
                 else
-                {
-                    WriteType wt = mutations.size() <= 1 ? WriteType.SIMPLE : WriteType.UNLOGGED_BATCH;
-                    responseHandlers.add(performWrite(mutation, consistency_level, localDataCenter, standardWritePerformer, null, wt, queryStartNanoTime));
-                }
+                    responseHandlers.add(performWrite(mutation, consistency_level, localDataCenter, standardWritePerformer, null, plainWriteType, queryStartNanoTime));
+            }
+
+            // upgrade to full quorum any failed cheap quorums
+            for (int i = 0 ; i < mutations.size() ; ++i)
+            {
+                if (!(mutations.get(i) instanceof CounterMutation)) // at the moment, only non-counter writes support cheap quorums
+                    responseHandlers.get(i).maybeTryAdditionalReplicas(mutations.get(i), standardWritePerformer, localDataCenter);
             }
 
             // wait for writes.  throws TimeoutException if necessary
             for (AbstractWriteResponseHandler<IMutation> responseHandler : responseHandlers)
-            {
                 responseHandler.get();
-            }
         }
         catch (WriteTimeoutException|WriteFailureException ex)
         {
@@ -786,16 +723,12 @@ public class StorageProxy implements StorageProxyMBean
         String keyspaceName = mutation.getKeyspaceName();
         Token token = mutation.key().getToken();
 
-        Iterable<InetAddressAndPort> endpoints = StorageService.instance.getNaturalAndPendingEndpoints(keyspaceName, token);
-        ArrayList<InetAddressAndPort> endpointsToHint = new ArrayList<>(Iterables.size(endpoints));
-
         // local writes can timeout, but cannot be dropped (see LocalMutationRunnable and CASSANDRA-6510),
         // so there is no need to hint or retry.
-        for (InetAddressAndPort target : endpoints)
-            if (!target.equals(FBUtilities.getBroadcastAddressAndPort()) && shouldHint(target))
-                endpointsToHint.add(target);
+        EndpointsForToken replicasToHint = StorageService.instance.getNaturalAndPendingReplicasForToken(keyspaceName, token)
+                .filter(StorageProxy::shouldHint);
 
-        submitHint(mutation, endpointsToHint, null);
+        submitHint(mutation, replicasToHint, null);
     }
 
     public boolean appliesLocally(Mutation mutation)
@@ -804,8 +737,8 @@ public class StorageProxy implements StorageProxyMBean
         Token token = mutation.key().getToken();
         InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
 
-        return StorageService.instance.getNaturalEndpoints(keyspaceName, token).contains(local)
-               || StorageService.instance.getTokenMetadata().pendingEndpointsFor(token, keyspaceName).contains(local);
+        return StorageService.instance.getNaturalReplicasForToken(keyspaceName, token).endpoints().contains(local)
+               || StorageService.instance.getTokenMetadata().pendingEndpointsForToken(token, keyspaceName).endpoints().contains(local);
     }
 
     /**
@@ -854,13 +787,13 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     String keyspaceName = mutation.getKeyspaceName();
                     Token tk = mutation.key().getToken();
-                    Optional<InetAddressAndPort> pairedEndpoint = ViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
-                    Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
+                    Optional<Replica> pairedEndpoint = ViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
+                    EndpointsForToken pendingReplicas = StorageService.instance.getTokenMetadata().pendingEndpointsForToken(tk, keyspaceName);
 
                     // if there are no paired endpoints there are probably range movements going on, so we write to the local batchlog to replay later
                     if (!pairedEndpoint.isPresent())
                     {
-                        if (pendingEndpoints.isEmpty())
+                        if (pendingReplicas.isEmpty())
                             logger.warn("Received base materialized view mutation for key {} that does not belong " +
                                         "to this node. There is probably a range movement happening (move or decommission)," +
                                         "but this node hasn't updated its ring metadata yet. Adding mutation to " +
@@ -872,8 +805,8 @@ public class StorageProxy implements StorageProxyMBean
                     // When local node is the endpoint we can just apply the mutation locally,
                     // unless there are pending endpoints, in which case we want to do an ordinary
                     // write so the view mutation is sent to the pending endpoint
-                    if (pairedEndpoint.get().equals(FBUtilities.getBroadcastAddressAndPort()) && StorageService.instance.isJoined()
-                        && pendingEndpoints.isEmpty())
+                    if (pairedEndpoint.get().isLocal() && StorageService.instance.isJoined()
+                        && pendingReplicas.isEmpty())
                     {
                         try
                         {
@@ -892,7 +825,8 @@ public class StorageProxy implements StorageProxyMBean
                         wrappers.add(wrapViewBatchResponseHandler(mutation,
                                                                   consistencyLevel,
                                                                   consistencyLevel,
-                                                                  Collections.singletonList(pairedEndpoint.get()),
+                                                                  EndpointsForToken.of(tk, pairedEndpoint.get()),
+                                                                  pendingReplicas,
                                                                   baseComplete,
                                                                   WriteType.BATCH,
                                                                   cleanup,
@@ -916,7 +850,7 @@ public class StorageProxy implements StorageProxyMBean
     }
 
     @SuppressWarnings("unchecked")
-    public static void mutateWithTriggers(Collection<? extends IMutation> mutations,
+    public static void mutateWithTriggers(List<? extends IMutation> mutations,
                                           ConsistencyLevel consistencyLevel,
                                           boolean mutateAtomically,
                                           long queryStartNanoTime)
@@ -966,6 +900,9 @@ public class StorageProxy implements StorageProxyMBean
         List<WriteResponseHandlerWrapper> wrappers = new ArrayList<WriteResponseHandlerWrapper>(mutations.size());
         String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
 
+        if (mutations.stream().anyMatch(mutation -> Keyspace.open(mutation.getKeyspaceName()).getReplicationStrategy().hasTransientReplicas()))
+            throw new AssertionError("Logged batches are unsupported with transient replication");
+
         try
         {
 
@@ -982,7 +919,7 @@ public class StorageProxy implements StorageProxyMBean
                     batchConsistencyLevel = consistency_level;
             }
 
-            final Collection<InetAddressAndPort> batchlogEndpoints = getBatchlogEndpoints(localDataCenter, batchConsistencyLevel);
+            final Collection<InetAddressAndPort> batchlogEndpoints = getBatchlogReplicas(localDataCenter, batchConsistencyLevel);
             final UUID batchUUID = UUIDGen.getTimeUUID();
             BatchlogResponseHandler.BatchlogCleanup cleanup = new BatchlogResponseHandler.BatchlogCleanup(mutations.size(),
                                                                                                           () -> asyncRemoveFromBatchlog(batchlogEndpoints, batchUUID));
@@ -1037,11 +974,6 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static boolean canDoLocalRequest(InetAddressAndPort replica)
-    {
-        return replica.equals(FBUtilities.getBroadcastAddressAndPort());
-    }
-
     private static void updateCoordinatorWriteLatencyTableMetric(Collection<? extends IMutation> mutations, long latency)
     {
         if (null == mutations)
@@ -1069,24 +1001,22 @@ public class StorageProxy implements StorageProxyMBean
     private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddressAndPort> endpoints, UUID uuid, long queryStartNanoTime)
     throws WriteTimeoutException, WriteFailureException
     {
-        WriteResponseHandler<?> handler = new WriteResponseHandler<>(endpoints,
-                                                                     Collections.emptyList(),
-                                                                     endpoints.size() == 1 ? ConsistencyLevel.ONE : ConsistencyLevel.TWO,
-                                                                     Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME),
-                                                                     null,
-                                                                     WriteType.BATCH_LOG,
-                                                                     queryStartNanoTime);
+        Keyspace systemKeypsace = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME);
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forBatchlogWrite(systemKeypsace, endpoints);
+        WriteResponseHandler<?> handler = new WriteResponseHandler(replicaLayout,
+                                                                   WriteType.BATCH_LOG,
+                                                                   queryStartNanoTime);
 
         Batch batch = Batch.createLocal(uuid, FBUtilities.timestampMicros(), mutations);
         MessageOut<Batch> message = new MessageOut<>(MessagingService.Verb.BATCH_STORE, batch, Batch.serializer);
-        for (InetAddressAndPort target : endpoints)
+        for (Replica replica : replicaLayout.all())
         {
-            logger.trace("Sending batchlog store request {} to {} for {} mutations", batch.id, target, batch.size());
+            logger.trace("Sending batchlog store request {} to {} for {} mutations", batch.id, replica, batch.size());
 
-            if (canDoLocalRequest(target))
-                performLocally(Stage.MUTATION, Optional.empty(), () -> BatchlogManager.store(batch), handler);
+            if (replica.isLocal())
+                performLocally(Stage.MUTATION, replica, Optional.empty(), () -> BatchlogManager.store(batch), handler);
             else
-                MessagingService.instance().sendRR(message, target, handler);
+                MessagingService.instance().sendRR(message, replica.endpoint(), handler);
         }
         handler.get();
     }
@@ -1099,8 +1029,8 @@ public class StorageProxy implements StorageProxyMBean
             if (logger.isTraceEnabled())
                 logger.trace("Sending batchlog remove request {} to {}", uuid, target);
 
-            if (canDoLocalRequest(target))
-                performLocally(Stage.MUTATION, () -> BatchlogManager.remove(uuid));
+            if (target.equals(FBUtilities.getBroadcastAddressAndPort()))
+                performLocally(Stage.MUTATION, SystemReplicas.getSystemReplica(target), () -> BatchlogManager.remove(uuid));
             else
                 MessagingService.instance().sendOneWay(message, target);
         }
@@ -1110,11 +1040,12 @@ public class StorageProxy implements StorageProxyMBean
     {
         for (WriteResponseHandlerWrapper wrapper : wrappers)
         {
-            Iterable<InetAddressAndPort> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
+            Replicas.temporaryAssertFull(wrapper.handler.replicaLayout.all());  // TODO: CASSANDRA-14549
+            ReplicaLayout.ForToken replicas = wrapper.handler.replicaLayout.withSelected(wrapper.handler.replicaLayout.all());
 
             try
             {
-                sendToHintedEndpoints(wrapper.mutation, endpoints, wrapper.handler, localDataCenter, stage);
+                sendToHintedReplicas(wrapper.mutation, replicas.selected(), wrapper.handler, localDataCenter, stage);
             }
             catch (OverloadedException | WriteTimeoutException e)
             {
@@ -1128,8 +1059,8 @@ public class StorageProxy implements StorageProxyMBean
     {
         for (WriteResponseHandlerWrapper wrapper : wrappers)
         {
-            Iterable<InetAddressAndPort> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
-            sendToHintedEndpoints(wrapper.mutation, endpoints, wrapper.handler, localDataCenter, stage);
+            Replicas.temporaryAssertFull(wrapper.handler.replicaLayout.all()); // TODO: CASSANDRA-14549
+            sendToHintedReplicas(wrapper.mutation, wrapper.handler.replicaLayout.all(), wrapper.handler, localDataCenter, stage);
         }
 
 
@@ -1144,7 +1075,7 @@ public class StorageProxy implements StorageProxyMBean
      * responses based on consistency level.
      *
      * @param mutation the mutation to be applied
-     * @param consistency_level the consistency level for the write operation
+     * @param consistencyLevel the consistency level for the write operation
      * @param performer the WritePerformer in charge of appliying the mutation
      * given the list of write endpoints (either standardWritePerformer for
      * standard writes or counterWritePerformer for counter writes).
@@ -1152,33 +1083,32 @@ public class StorageProxy implements StorageProxyMBean
      * @param queryStartNanoTime the value of System.nanoTime() when the query started to be processed
      */
     public static AbstractWriteResponseHandler<IMutation> performWrite(IMutation mutation,
-                                                                       ConsistencyLevel consistency_level,
+                                                                       ConsistencyLevel consistencyLevel,
                                                                        String localDataCenter,
                                                                        WritePerformer performer,
                                                                        Runnable callback,
                                                                        WriteType writeType,
                                                                        long queryStartNanoTime)
-    throws UnavailableException, OverloadedException
     {
         String keyspaceName = mutation.getKeyspaceName();
-        AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
 
         Token tk = mutation.key().getToken();
-        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
-        AbstractWriteResponseHandler<IMutation> responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType, queryStartNanoTime);
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWriteWithDownNodes(keyspace, consistencyLevel, tk);
+        AbstractWriteResponseHandler<IMutation> responseHandler = rs.getWriteResponseHandler(replicaLayout, callback, writeType, queryStartNanoTime);
 
         // exit early if we can't fulfill the CL at this time
         responseHandler.assureSufficientLiveNodes();
 
-        performer.apply(mutation, Iterables.concat(naturalEndpoints, pendingEndpoints), responseHandler, localDataCenter, consistency_level);
+        performer.apply(mutation, replicaLayout, responseHandler, localDataCenter);
         return responseHandler;
     }
 
     // same as performWrites except does not initiate writes (but does perform availability checks).
     private static WriteResponseHandlerWrapper wrapBatchResponseHandler(Mutation mutation,
-                                                                        ConsistencyLevel consistency_level,
+                                                                        ConsistencyLevel consistencyLevel,
                                                                         ConsistencyLevel batchConsistencyLevel,
                                                                         WriteType writeType,
                                                                         BatchlogResponseHandler.BatchlogCleanup cleanup,
@@ -1186,11 +1116,10 @@ public class StorageProxy implements StorageProxyMBean
     {
         Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
         AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
-        String keyspaceName = mutation.getKeyspaceName();
         Token tk = mutation.key().getToken();
-        List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
-        AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType, queryStartNanoTime);
+
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWrite(keyspace, consistencyLevel, tk, FailureDetector.instance::isAlive);
+        AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(replicaLayout,null, writeType, queryStartNanoTime);
         BatchlogResponseHandler<IMutation> batchHandler = new BatchlogResponseHandler<>(writeHandler, batchConsistencyLevel.blockFor(keyspace), cleanup, queryStartNanoTime);
         return new WriteResponseHandlerWrapper(batchHandler, mutation);
     }
@@ -1200,9 +1129,10 @@ public class StorageProxy implements StorageProxyMBean
      * Keeps track of ViewWriteMetrics
      */
     private static WriteResponseHandlerWrapper wrapViewBatchResponseHandler(Mutation mutation,
-                                                                            ConsistencyLevel consistency_level,
+                                                                            ConsistencyLevel consistencyLevel,
                                                                             ConsistencyLevel batchConsistencyLevel,
-                                                                            List<InetAddressAndPort> naturalEndpoints,
+                                                                            EndpointsForToken naturalEndpoints,
+                                                                            EndpointsForToken pendingEndpoints,
                                                                             AtomicLong baseComplete,
                                                                             WriteType writeType,
                                                                             BatchlogResponseHandler.BatchlogCleanup cleanup,
@@ -1210,10 +1140,10 @@ public class StorageProxy implements StorageProxyMBean
     {
         Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
         AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
-        String keyspaceName = mutation.getKeyspaceName();
         Token tk = mutation.key().getToken();
-        Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
-        AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, () -> {
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWriteWithDownNodes(keyspace, consistencyLevel, tk, naturalEndpoints, pendingEndpoints);
+
+        AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(replicaLayout, () -> {
             long delay = Math.max(0, System.currentTimeMillis() - baseComplete.get());
             viewWriteMetrics.viewWriteLatency.update(delay, TimeUnit.MILLISECONDS);
         }, writeType, queryStartNanoTime);
@@ -1241,7 +1171,7 @@ public class StorageProxy implements StorageProxyMBean
      * - 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
      */
-    private static Collection<InetAddressAndPort> getBatchlogEndpoints(String localDataCenter, ConsistencyLevel consistencyLevel)
+    private static Collection<InetAddressAndPort> getBatchlogReplicas(String localDataCenter, ConsistencyLevel consistencyLevel)
     throws UnavailableException
     {
         TokenMetadata.Topology topology = StorageService.instance.getTokenMetadata().cachedOnlyTokenMap().getTopology();
@@ -1254,7 +1184,7 @@ public class StorageProxy implements StorageProxyMBean
             if (consistencyLevel == ConsistencyLevel.ANY)
                 return Collections.singleton(FBUtilities.getBroadcastAddressAndPort());
 
-            throw new UnavailableException(ConsistencyLevel.ONE, 1, 0);
+            throw UnavailableException.create(ConsistencyLevel.ONE, 1, 0);
         }
 
         return chosenEndpoints;
@@ -1277,36 +1207,36 @@ public class StorageProxy implements StorageProxyMBean
      *
      * @throws OverloadedException if the hints cannot be written/enqueued
      */
-    public static void sendToHintedEndpoints(final Mutation mutation,
-                                             Iterable<InetAddressAndPort> targets,
-                                             AbstractWriteResponseHandler<IMutation> responseHandler,
-                                             String localDataCenter,
-                                             Stage stage)
+    public static void sendToHintedReplicas(final Mutation mutation,
+                                            EndpointsForToken targets,
+                                            AbstractWriteResponseHandler<IMutation> responseHandler,
+                                            String localDataCenter,
+                                            Stage stage)
     throws OverloadedException
     {
-        int targetsSize = Iterables.size(targets);
-
         // this dc replicas:
-        Collection<InetAddressAndPort> localDc = null;
+        Collection<Replica> localDc = null;
         // extra-datacenter replicas, grouped by dc
-        Map<String, Collection<InetAddressAndPort>> dcGroups = null;
+        Map<String, Collection<Replica>> dcGroups = null;
         // only need to create a Message for non-local writes
         MessageOut<Mutation> message = null;
 
         boolean insertLocal = false;
-        ArrayList<InetAddressAndPort> endpointsToHint = null;
+        Replica localReplica = null;
+        Collection<Replica> endpointsToHint = null;
 
         List<InetAddressAndPort> backPressureHosts = null;
 
-        for (InetAddressAndPort destination : targets)
+        for (Replica destination : targets)
         {
             checkHintOverload(destination);
 
-            if (FailureDetector.instance.isAlive(destination))
+            if (FailureDetector.instance.isAlive(destination.endpoint()))
             {
-                if (canDoLocalRequest(destination))
+                if (destination.isLocal())
                 {
                     insertLocal = true;
+                    localReplica = destination;
                 }
                 else
                 {
@@ -1321,28 +1251,26 @@ public class StorageProxy implements StorageProxyMBean
                     if (localDataCenter.equals(dc))
                     {
                         if (localDc == null)
-                            localDc = new ArrayList<>(targetsSize);
+                            localDc = new ArrayList<>(targets.size());
 
                         localDc.add(destination);
                     }
                     else
                     {
-                        Collection<InetAddressAndPort> messages = (dcGroups != null) ? dcGroups.get(dc) : null;
+                        if (dcGroups == null)
+                            dcGroups = new HashMap<>();
+
+                        Collection<Replica> messages = dcGroups.get(dc);
                         if (messages == null)
-                        {
-                            messages = new ArrayList<>(3); // most DCs will have <= 3 replicas
-                            if (dcGroups == null)
-                                dcGroups = new HashMap<>();
-                            dcGroups.put(dc, messages);
-                        }
+                            messages = dcGroups.computeIfAbsent(dc, (v) -> new ArrayList<>(3)); // most DCs will have <= 3 replicas
 
                         messages.add(destination);
                     }
 
                     if (backPressureHosts == null)
-                        backPressureHosts = new ArrayList<>(targetsSize);
+                        backPressureHosts = new ArrayList<>(targets.size());
 
-                    backPressureHosts.add(destination);
+                    backPressureHosts.add(destination.endpoint());
                 }
             }
             else
@@ -1352,7 +1280,7 @@ public class StorageProxy implements StorageProxyMBean
                 if (shouldHint(destination))
                 {
                     if (endpointsToHint == null)
-                        endpointsToHint = new ArrayList<>(targetsSize);
+                        endpointsToHint = new ArrayList<>();
 
                     endpointsToHint.add(destination);
                 }
@@ -1363,25 +1291,28 @@ public class StorageProxy implements StorageProxyMBean
             MessagingService.instance().applyBackPressure(backPressureHosts, responseHandler.currentTimeout());
 
         if (endpointsToHint != null)
-            submitHint(mutation, endpointsToHint, responseHandler);
+            submitHint(mutation, EndpointsForToken.copyOf(mutation.key().getToken(), endpointsToHint), responseHandler);
 
         if (insertLocal)
-            performLocally(stage, Optional.of(mutation), mutation::apply, responseHandler);
+        {
+            Preconditions.checkNotNull(localReplica);
+            performLocally(stage, localReplica, Optional.of(mutation), mutation::apply, responseHandler);
+        }
 
         if (localDc != null)
         {
-            for (InetAddressAndPort destination : localDc)
-                MessagingService.instance().sendRR(message, destination, responseHandler, true);
+            for (Replica destination : localDc)
+                MessagingService.instance().sendWriteRR(message, destination, responseHandler, true);
         }
         if (dcGroups != null)
         {
             // for each datacenter, send the message to one node to relay the write to other replicas
-            for (Collection<InetAddressAndPort> dcTargets : dcGroups.values())
-                sendMessagesToNonlocalDC(message, dcTargets, responseHandler);
+            for (Collection<Replica> dcTargets : dcGroups.values())
+                sendMessagesToNonlocalDC(message, EndpointsForToken.copyOf(mutation.key().getToken(), dcTargets), responseHandler);
         }
     }
 
-    private static void checkHintOverload(InetAddressAndPort destination)
+    private static void checkHintOverload(Replica destination)
     {
         // avoid OOMing due to excess hints.  we need to do this check even for "live" nodes, since we can
         // still generate hints for those if it's overloaded or simply dead but not yet known-to-be-dead.
@@ -1389,45 +1320,46 @@ public class StorageProxy implements StorageProxyMBean
         // a small number of nodes causing problems, so we should avoid shutting down writes completely to
         // healthy nodes.  Any node with no hintsInProgress is considered healthy.
         if (StorageMetrics.totalHintsInProgress.getCount() > maxHintsInProgress
-                && (getHintsInProgressFor(destination).get() > 0 && shouldHint(destination)))
+                && (getHintsInProgressFor(destination.endpoint()).get() > 0 && shouldHint(destination)))
         {
             throw new OverloadedException("Too many in flight hints: " + StorageMetrics.totalHintsInProgress.getCount() +
                                           " destination: " + destination +
-                                          " destination hints: " + getHintsInProgressFor(destination).get());
+                                          " destination hints: " + getHintsInProgressFor(destination.endpoint()).get());
         }
     }
 
     private static void sendMessagesToNonlocalDC(MessageOut<? extends IMutation> message,
-                                                 Collection<InetAddressAndPort> targets,
+                                                 EndpointsForToken targets,
                                                  AbstractWriteResponseHandler<IMutation> handler)
     {
-        Iterator<InetAddressAndPort> iter = targets.iterator();
+        Iterator<Replica> iter = targets.iterator();
         int[] messageIds = new int[targets.size()];
-        InetAddressAndPort target = iter.next();
+        Replica target = iter.next();
 
         int idIdx = 0;
         // Add the other destinations of the same message as a FORWARD_HEADER entry
         while (iter.hasNext())
         {
-            InetAddressAndPort destination = iter.next();
-            int id = MessagingService.instance().addCallback(handler,
-                                                             message,
-                                                             destination,
-                                                             message.getTimeout(),
-                                                             handler.consistencyLevel,
-                                                             true);
+            Replica destination = iter.next();
+            int id = MessagingService.instance().addWriteCallback(handler,
+                                                                  message,
+                                                                  destination,
+                                                                  message.getTimeout(),
+                                                                  handler.replicaLayout.consistencyLevel(),
+                                                                  true);
             messageIds[idIdx++] = id;
             logger.trace("Adding FWD message to {}@{}", id, destination);
         }
-        message = message.withParameter(ParameterType.FORWARD_TO.FORWARD_TO, new ForwardToContainer(targets, messageIds));
+
+        message = message.withParameter(ParameterType.FORWARD_TO, new ForwardToContainer(targets.endpoints(), messageIds));
         // send the combined message + forward headers
-        int id = MessagingService.instance().sendRR(message, target, handler, true);
+        int id = MessagingService.instance().sendWriteRR(message, target, handler, true);
         logger.trace("Sending message to {}@{}", id, target);
     }
 
-    private static void performLocally(Stage stage, final Runnable runnable)
+    private static void performLocally(Stage stage, Replica localReplica, final Runnable runnable)
     {
-        StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable()
+        StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable(localReplica)
         {
             public void runMayThrow()
             {
@@ -1449,9 +1381,9 @@ public class StorageProxy implements StorageProxyMBean
         });
     }
 
-    private static void performLocally(Stage stage, Optional<IMutation> mutation, final Runnable runnable, final IAsyncCallbackWithFailure<?> handler)
+    private static void performLocally(Stage stage, Replica localReplica, Optional<IMutation> mutation, final Runnable runnable, final IAsyncCallbackWithFailure<?> handler)
     {
-        StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable(mutation)
+        StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable(localReplica, mutation)
         {
             public void runMayThrow()
             {
@@ -1492,9 +1424,9 @@ public class StorageProxy implements StorageProxyMBean
      */
     public static AbstractWriteResponseHandler<IMutation> mutateCounter(CounterMutation cm, String localDataCenter, long queryStartNanoTime) throws UnavailableException, OverloadedException
     {
-        InetAddressAndPort endpoint = findSuitableEndpoint(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
+        Replica replica = findSuitableReplica(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
 
-        if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
+        if (replica.isLocal())
         {
             return applyCounterMutationOnCoordinator(cm, localDataCenter, queryStartNanoTime);
         }
@@ -1502,18 +1434,19 @@ public class StorageProxy implements StorageProxyMBean
         {
             // Exit now if we can't fulfill the CL here instead of forwarding to the leader replica
             String keyspaceName = cm.getKeyspaceName();
+            Keyspace keyspace = Keyspace.open(keyspaceName);
             AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
             Token tk = cm.key().getToken();
-            List<InetAddressAndPort> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
-            Collection<InetAddressAndPort> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
-            rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, cm.consistency(), null, WriteType.COUNTER, queryStartNanoTime).assureSufficientLiveNodes();
+            ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWriteWithDownNodes(keyspace, cm.consistency(), tk);
+            rs.getWriteResponseHandler(replicaLayout, null, WriteType.COUNTER, queryStartNanoTime).assureSufficientLiveNodes();
 
             // Forward the actual update to the chosen leader replica
-            AbstractWriteResponseHandler<IMutation> responseHandler = new WriteResponseHandler<>(endpoint, WriteType.COUNTER, queryStartNanoTime);
+            AbstractWriteResponseHandler<IMutation> responseHandler = new WriteResponseHandler<>(ReplicaLayout.forCounterWrite(keyspace, tk, replica),
+                                                                                                 WriteType.COUNTER, queryStartNanoTime);
 
-            Tracing.trace("Enqueuing counter update to {}", endpoint);
-            MessagingService.instance().sendRR(cm.makeMutationMessage(), endpoint, responseHandler, false);
+            Tracing.trace("Enqueuing counter update to {}", replica);
+            MessagingService.instance().sendWriteRR(cm.makeMutationMessage(), replica, responseHandler, false);
             return responseHandler;
         }
     }
@@ -1528,38 +1461,37 @@ public class StorageProxy implements StorageProxyMBean
      * is unclear we want to mix those latencies with read latencies, so this
      * may be a bit involved.
      */
-    private static InetAddressAndPort findSuitableEndpoint(String keyspaceName, DecoratedKey key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
+    private static Replica findSuitableReplica(String keyspaceName, DecoratedKey key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
     {
         Keyspace keyspace = Keyspace.open(keyspaceName);
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        List<InetAddressAndPort> endpoints = new ArrayList<>();
-        StorageService.instance.getLiveNaturalEndpoints(keyspace, key, endpoints);
+        EndpointsForToken replicas = StorageService.instance.getLiveNaturalReplicasForToken(keyspace, key);
 
         // CASSANDRA-13043: filter out those endpoints not accepting clients yet, maybe because still bootstrapping
-        endpoints.removeIf(endpoint -> !StorageService.instance.isRpcReady(endpoint));
+        replicas = replicas.filter(replica -> StorageService.instance.isRpcReady(replica.endpoint()));
 
         // TODO have a way to compute the consistency level
-        if (endpoints.isEmpty())
-            throw new UnavailableException(cl, cl.blockFor(keyspace), 0);
+        if (replicas.isEmpty())
+            throw UnavailableException.create(cl, cl.blockFor(keyspace), 0);
 
-        List<InetAddressAndPort> localEndpoints = new ArrayList<>(endpoints.size());
+        List<Replica> localReplicas = new ArrayList<>(replicas.size());
 
-        for (InetAddressAndPort endpoint : endpoints)
-            if (snitch.getDatacenter(endpoint).equals(localDataCenter))
-                localEndpoints.add(endpoint);
+        for (Replica replica : replicas)
+            if (snitch.getDatacenter(replica).equals(localDataCenter))
+                localReplicas.add(replica);
 
-        if (localEndpoints.isEmpty())
+        if (localReplicas.isEmpty())
         {
             // If the consistency required is local then we should not involve other DCs
             if (cl.isDatacenterLocal())
-                throw new UnavailableException(cl, cl.blockFor(keyspace), 0);
+                throw UnavailableException.create(cl, cl.blockFor(keyspace), 0);
 
             // No endpoint in local DC, pick the closest endpoint according to the snitch
-            snitch.sortByProximity(FBUtilities.getBroadcastAddressAndPort(), endpoints);
-            return endpoints.get(0);
+            replicas = snitch.sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas);
+            return replicas.get(0);
         }
 
-        return localEndpoints.get(ThreadLocalRandom.current().nextInt(localEndpoints.size()));
+        return localReplicas.get(ThreadLocalRandom.current().nextInt(localReplicas.size()));
     }
 
     // Must be called on a replica of the mutation. This replica becomes the
@@ -1579,7 +1511,7 @@ public class StorageProxy implements StorageProxyMBean
     }
 
     private static Runnable counterWriteTask(final IMutation mutation,
-                                             final Iterable<InetAddressAndPort> targets,
+                                             final EndpointsForToken targets,
                                              final AbstractWriteResponseHandler<IMutation> responseHandler,
                                              final String localDataCenter)
     {
@@ -1592,11 +1524,7 @@ public class StorageProxy implements StorageProxyMBean
 
                 Mutation result = ((CounterMutation) mutation).applyCounterMutation();
                 responseHandler.response(null);
-
-                Set<InetAddressAndPort> remotes = Sets.difference(ImmutableSet.copyOf(targets),
-                                                                  ImmutableSet.of(FBUtilities.getBroadcastAddressAndPort()));
-                if (!remotes.isEmpty())
-                    sendToHintedEndpoints(result, remotes, responseHandler, localDataCenter, Stage.COUNTER_MUTATION);
+                sendToHintedReplicas(result, targets, responseHandler, localDataCenter, Stage.COUNTER_MUTATION);
             }
         };
     }
@@ -1664,9 +1592,7 @@ public class StorageProxy implements StorageProxyMBean
         try
         {
             // make sure any in-progress paxos writes are done (i.e., committed to a majority of replicas), before performing a quorum read
-            PaxosParticipants p = getPaxosParticipants(metadata, key, consistencyLevel);
-            List<InetAddressAndPort> liveEndpoints = p.liveEndpoints;
-            int requiredParticipants = p.participants;
+            ReplicaLayout.ForPaxos replicaLayout = ReplicaLayout.forPaxos(Keyspace.open(metadata.keyspace), key, consistencyLevel);
 
             // does the work of applying in-progress writes; throws UAE or timeout if it can't
             final ConsistencyLevel consistencyForCommitOrFetch = consistencyLevel == ConsistencyLevel.LOCAL_SERIAL
@@ -1675,7 +1601,7 @@ public class StorageProxy implements StorageProxyMBean
 
             try
             {
-                final PaxosBallotAndContention pair = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyLevel, consistencyForCommitOrFetch, false, state);
+                final PaxosBallotAndContention pair = beginAndRepairPaxos(start, key, metadata, replicaLayout, consistencyLevel, consistencyForCommitOrFetch, false, state);
                 if (pair.contentions > 0)
                     casReadMetrics.contention.update(pair.contentions);
             }
@@ -1924,28 +1850,19 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static List<InetAddressAndPort> getLiveSortedEndpoints(Keyspace keyspace, ByteBuffer key)
+    public static EndpointsForToken getLiveSortedReplicasForToken(Keyspace keyspace, RingPosition pos)
     {
-        return getLiveSortedEndpoints(keyspace, StorageService.instance.getTokenMetadata().decorateKey(key));
+        return getLiveSortedReplicas(keyspace, pos).forToken(pos.getToken());
     }
 
-    public static List<InetAddressAndPort> getLiveSortedEndpoints(Keyspace keyspace, RingPosition pos)
+    public static EndpointsForRange getLiveSortedReplicas(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddressAndPort> liveEndpoints = StorageService.instance.getLiveNaturalEndpoints(keyspace, pos);
-        DatabaseDescriptor.getEndpointSnitch().sortByProximity(FBUtilities.getBroadcastAddressAndPort(), liveEndpoints);
-        return liveEndpoints;
-    }
+        EndpointsForRange liveReplicas = StorageService.instance.getLiveNaturalReplicas(keyspace, pos);
+        // Replica availability is considered by the query path
+        Preconditions.checkState(liveReplicas.isEmpty() || liveReplicas.stream().anyMatch(Replica::isFull),
+                                 "At least one full replica required for reads: " + liveReplicas);
 
-    private static List<InetAddressAndPort> intersection(List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
-    {
-        // Note: we don't use Guava Sets.intersection() for 3 reasons:
-        //   1) retainAll would be inefficient if l1 and l2 are large but in practice both are the replicas for a range and
-        //   so will be very small (< RF). In that case, retainAll is in fact more efficient.
-        //   2) we do ultimately need a list so converting everything to sets don't make sense
-        //   3) l1 and l2 are sorted by proximity. The use of retainAll  maintain that sorting in the result, while using sets wouldn't.
-        List<InetAddressAndPort> inter = new ArrayList<>(l1);
-        inter.retainAll(l2);
-        return inter;
+        return DatabaseDescriptor.getEndpointSnitch().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), liveReplicas);
     }
 
     /**
@@ -1963,24 +1880,10 @@ public class StorageProxy implements StorageProxyMBean
                                  : index.getEstimatedResultRows();
 
         // adjust maxExpectedResults by the number of tokens this node has and the replication factor for this ks
-        return (maxExpectedResults / DatabaseDescriptor.getNumTokens()) / keyspace.getReplicationStrategy().getReplicationFactor();
+        return (maxExpectedResults / DatabaseDescriptor.getNumTokens()) / keyspace.getReplicationStrategy().getReplicationFactor().allReplicas;
     }
 
-    private static class RangeForQuery
-    {
-        public final AbstractBounds<PartitionPosition> range;
-        public final List<InetAddressAndPort> liveEndpoints;
-        public final List<InetAddressAndPort> filteredEndpoints;
-
-        public RangeForQuery(AbstractBounds<PartitionPosition> range, List<InetAddressAndPort> liveEndpoints, List<InetAddressAndPort> filteredEndpoints)
-        {
-            this.range = range;
-            this.liveEndpoints = liveEndpoints;
-            this.filteredEndpoints = filteredEndpoints;
-        }
-    }
-
-    private static class RangeIterator extends AbstractIterator<RangeForQuery>
+    private static class RangeIterator extends AbstractIterator<ReplicaLayout.ForRange>
     {
         private final Keyspace keyspace;
         private final ConsistencyLevel consistency;
@@ -2004,38 +1907,43 @@ public class StorageProxy implements StorageProxyMBean
             return rangeCount;
         }
 
-        protected RangeForQuery computeNext()
+        protected ReplicaLayout.ForRange computeNext()
         {
             if (!ranges.hasNext())
                 return endOfData();
 
             AbstractBounds<PartitionPosition> range = ranges.next();
-            List<InetAddressAndPort> liveEndpoints = getLiveSortedEndpoints(keyspace, range.right);
-            return new RangeForQuery(range,
-                                     liveEndpoints,
-                                     consistency.filterForQuery(keyspace, liveEndpoints));
+            EndpointsForRange liveReplicas = getLiveSortedReplicas(keyspace, range.right);
+
+            int blockFor = consistency.blockFor(keyspace);
+            EndpointsForRange targetReplicas = consistency.filterForQuery(keyspace, liveReplicas);
+            int minResponses = Math.min(targetReplicas.size(), blockFor);
+
+            // Endpoitns for range here as well
+            return ReplicaLayout.forRangeRead(keyspace, consistency, range,
+                                              liveReplicas, targetReplicas.subList(0, minResponses));
         }
     }
 
-    private static class RangeMerger extends AbstractIterator<RangeForQuery>
+    private static class RangeMerger extends AbstractIterator<ReplicaLayout.ForRange>
     {
         private final Keyspace keyspace;
         private final ConsistencyLevel consistency;
-        private final PeekingIterator<RangeForQuery> ranges;
+        private final PeekingIterator<ReplicaLayout.ForRange> ranges;
 
-        private RangeMerger(Iterator<RangeForQuery> iterator, Keyspace keyspace, ConsistencyLevel consistency)
+        private RangeMerger(Iterator<ReplicaLayout.ForRange> iterator, Keyspace keyspace, ConsistencyLevel consistency)
         {
             this.keyspace = keyspace;
             this.consistency = consistency;
             this.ranges = Iterators.peekingIterator(iterator);
         }
 
-        protected RangeForQuery computeNext()
+        protected ReplicaLayout.ForRange computeNext()
         {
             if (!ranges.hasNext())
                 return endOfData();
 
-            RangeForQuery current = ranges.next();
+            ReplicaLayout.ForRange current = ranges.next();
 
             // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take
             // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges
@@ -2050,22 +1958,22 @@ public class StorageProxy implements StorageProxyMBean
                 if (current.range.right.isMinimum())
                     break;
 
-                RangeForQuery next = ranges.peek();
+                ReplicaLayout.ForRange next = ranges.peek();
 
-                List<InetAddressAndPort> merged = intersection(current.liveEndpoints, next.liveEndpoints);
+                EndpointsForRange merged = current.all().keep(next.all().endpoints());
 
                 // Check if there is enough endpoint for the merge to be possible.
-                if (!consistency.isSufficientLiveNodes(keyspace, merged))
+                if (!consistency.isSufficientLiveNodesForRead(keyspace, merged))
                     break;
 
-                List<InetAddressAndPort> filteredMerged = consistency.filterForQuery(keyspace, merged);
+                EndpointsForRange filteredMerged = consistency.filterForQuery(keyspace, merged);
 
                 // Estimate whether merging will be a win or not
-                if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, current.filteredEndpoints, next.filteredEndpoints))
+                if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, current.selected(), next.selected()))
                     break;
 
                 // If we get there, merge this range and the next one
-                current = new RangeForQuery(current.range.withNewRight(next.range.right), merged, filteredMerged);
+                current = ReplicaLayout.forRangeRead(keyspace, consistency, current.range.withNewRight(next.range.right), merged, filteredMerged);
                 ranges.next(); // consume the range we just merged since we've only peeked so far
             }
             return current;
@@ -2110,11 +2018,9 @@ public class StorageProxy implements StorageProxyMBean
 
     private static class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
     {
-        private final Iterator<RangeForQuery> ranges;
+        private final Iterator<ReplicaLayout.ForRange> ranges;
         private final int totalRangeCount;
         private final PartitionRangeReadCommand command;
-        private final Keyspace keyspace;
-        private final ConsistencyLevel consistency;
         private final boolean enforceStrictLiveness;
 
         private final long startTime;
@@ -2135,8 +2041,6 @@ public class StorageProxy implements StorageProxyMBean
             this.startTime = System.nanoTime();
             this.ranges = new RangeMerger(ranges, keyspace, consistency);
             this.totalRangeCount = ranges.rangeCount();
-            this.consistency = consistency;
-            this.keyspace = keyspace;
             this.queryStartNanoTime = queryStartNanoTime;
             this.enforceStrictLiveness = command.metadata().enforceStrictLiveness();
         }
@@ -2204,36 +2108,36 @@ public class StorageProxy implements StorageProxyMBean
         /**
          * Queries the provided sub-range.
          *
-         * @param toQuery the subRange to query.
+         * @param replicaLayout the subRange to query.
          * @param isFirst in the case where multiple queries are sent in parallel, whether that's the first query on
          * that batch or not. The reason it matters is that whe paging queries, the command (more specifically the
          * {@code DataLimits}) may have "state" information and that state may only be valid for the first query (in
          * that it's the query that "continues" whatever we're previously queried).
          */
-        private SingleRangeResponse query(RangeForQuery toQuery, boolean isFirst)
+        private SingleRangeResponse query(ReplicaLayout.ForRange replicaLayout, boolean isFirst)
         {
-            PartitionRangeReadCommand rangeCommand = command.forSubRange(toQuery.range, isFirst);
+            PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaLayout.range, isFirst);
+            ReadRepair<EndpointsForRange, ReplicaLayout.ForRange> readRepair = ReadRepair.create(command, replicaLayout, queryStartNanoTime);
+            DataResolver<EndpointsForRange, ReplicaLayout.ForRange> resolver = new DataResolver<>(rangeCommand, replicaLayout, readRepair, queryStartNanoTime);
+            Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
 
-            ReadRepair readRepair = ReadRepair.create(command, queryStartNanoTime, consistency);
-            DataResolver resolver = new DataResolver(keyspace, rangeCommand, consistency, toQuery.filteredEndpoints.size(), queryStartNanoTime, readRepair);
-
-            int blockFor = consistency.blockFor(keyspace);
-            int minResponses = Math.min(toQuery.filteredEndpoints.size(), blockFor);
-            List<InetAddressAndPort> minimalEndpoints = toQuery.filteredEndpoints.subList(0, minResponses);
-            ReadCallback handler = new ReadCallback(resolver, consistency, rangeCommand, minimalEndpoints, queryStartNanoTime);
+            ReadCallback<EndpointsForRange, ReplicaLayout.ForRange> handler = new ReadCallback<>(resolver,
+                                                                              replicaLayout.consistencyLevel().blockFor(keyspace),
+                                                                              rangeCommand,
+                                                                              replicaLayout,
+                                                                              queryStartNanoTime);
 
             handler.assureSufficientLiveNodes();
-
-            if (toQuery.filteredEndpoints.size() == 1 && canDoLocalRequest(toQuery.filteredEndpoints.get(0)))
+            if (replicaLayout.selected().size() == 1 && replicaLayout.selected().get(0).isLocal())
             {
                 StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(rangeCommand, handler));
             }
             else
             {
-                for (InetAddressAndPort endpoint : toQuery.filteredEndpoints)
+                for (Replica replica : replicaLayout.selected())
                 {
-                    Tracing.trace("Enqueuing request to {}", endpoint);
-                    MessagingService.instance().sendRRWithFailure(rangeCommand.createMessage(), endpoint, handler);
+                    Tracing.trace("Enqueuing request to {}", replica);
+                    MessagingService.instance().sendRRWithFailure(rangeCommand.createMessage(), replica.endpoint(), handler);
                 }
             }
 
@@ -2486,32 +2390,30 @@ public class StorageProxy implements StorageProxyMBean
         DatabaseDescriptor.setMaxHintWindow(ms);
     }
 
-    public static boolean shouldHint(InetAddressAndPort ep)
+    public static boolean shouldHint(Replica replica)
     {
-        if (DatabaseDescriptor.hintedHandoffEnabled())
+        if (!DatabaseDescriptor.hintedHandoffEnabled())
+            return false;
+        if (replica.isTransient() || replica.isLocal())
+            return false;
+
+        Set<String> disabledDCs = DatabaseDescriptor.hintedHandoffDisabledDCs();
+        if (!disabledDCs.isEmpty())
         {
-            Set<String> disabledDCs = DatabaseDescriptor.hintedHandoffDisabledDCs();
-            if (!disabledDCs.isEmpty())
-            {
-                final String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(ep);
-                if (disabledDCs.contains(dc))
-                {
-                    Tracing.trace("Not hinting {} since its data center {} has been disabled {}", ep, dc, disabledDCs);
-                    return false;
-                }
-            }
-            boolean hintWindowExpired = Gossiper.instance.getEndpointDowntime(ep) > DatabaseDescriptor.getMaxHintWindow();
-            if (hintWindowExpired)
+            final String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica);
+            if (disabledDCs.contains(dc))
             {
-                HintsService.instance.metrics.incrPastWindow(ep);
-                Tracing.trace("Not hinting {} which has been down {} ms", ep, Gossiper.instance.getEndpointDowntime(ep));
+                Tracing.trace("Not hinting {} since its data center {} has been disabled {}", replica, dc, disabledDCs);
+                return false;
             }
-            return !hintWindowExpired;
         }
-        else
+        boolean hintWindowExpired = Gossiper.instance.getEndpointDowntime(replica.endpoint()) > DatabaseDescriptor.getMaxHintWindow();
+        if (hintWindowExpired)
         {
-            return false;
+            HintsService.instance.metrics.incrPastWindow(replica.endpoint());
+            Tracing.trace("Not hinting {} which has been down {} ms", replica, Gossiper.instance.getEndpointDowntime(replica.endpoint()));
         }
+        return !hintWindowExpired;
     }
 
     /**
@@ -2532,7 +2434,7 @@ public class StorageProxy implements StorageProxyMBean
             // invoked by an admin, for simplicity we require that all nodes are up
             // to perform the operation.
             int liveMembers = Gossiper.instance.getLiveMembers().size();
-            throw new UnavailableException(ConsistencyLevel.ALL, liveMembers + Gossiper.instance.getUnreachableMembers().size(), liveMembers);
+            throw UnavailableException.create(ConsistencyLevel.ALL, liveMembers + Gossiper.instance.getUnreachableMembers().size(), liveMembers);
         }
 
         Set<InetAddressAndPort> allEndpoints = StorageService.instance.getLiveRingMembers(true);
@@ -2571,10 +2473,9 @@ public class StorageProxy implements StorageProxyMBean
     public interface WritePerformer
     {
         public void apply(IMutation mutation,
-                          Iterable<InetAddressAndPort> targets,
+                          ReplicaLayout.ForToken targets,
                           AbstractWriteResponseHandler<IMutation> responseHandler,
-                          String localDataCenter,
-                          ConsistencyLevel consistencyLevel) throws OverloadedException;
+                          String localDataCenter) throws OverloadedException;
     }
 
     /**
@@ -2638,15 +2539,18 @@ public class StorageProxy implements StorageProxyMBean
     {
         private final long constructionTime = System.currentTimeMillis();
 
+        private final Replica localReplica;
         private final Optional<IMutation> mutationOpt;
 
-        public LocalMutationRunnable(Optional<IMutation> mutationOpt)
+        public LocalMutationRunnable(Replica localReplica, Optional<IMutation> mutationOpt)
         {
+            this.localReplica = localReplica;
             this.mutationOpt = mutationOpt;
         }
 
-        public LocalMutationRunnable()
+        public LocalMutationRunnable(Replica localReplica)
         {
+            this.localReplica = localReplica;
             this.mutationOpt = Optional.empty();
         }
 
@@ -2659,7 +2563,8 @@ public class StorageProxy implements StorageProxyMBean
             {
                 if (MessagingService.DROPPABLE_VERBS.contains(verb))
                     MessagingService.instance().incrementDroppedMutations(mutationOpt, timeTaken);
-                HintRunnable runnable = new HintRunnable(Collections.singleton(FBUtilities.getBroadcastAddressAndPort()))
+
+                HintRunnable runnable = new HintRunnable(EndpointsForToken.of(localReplica.range().right, localReplica))
                 {
                     protected void runMayThrow() throws Exception
                     {
@@ -2690,9 +2595,9 @@ public class StorageProxy implements StorageProxyMBean
      */
     private abstract static class HintRunnable implements Runnable
     {
-        public final Collection<InetAddressAndPort> targets;
+        public final EndpointsForToken targets;
 
-        protected HintRunnable(Collection<InetAddressAndPort> targets)
+        protected HintRunnable(EndpointsForToken targets)
         {
             this.targets = targets;
         }
@@ -2710,7 +2615,7 @@ public class StorageProxy implements StorageProxyMBean
             finally
             {
                 StorageMetrics.totalHintsInProgress.dec(targets.size());
-                for (InetAddressAndPort target : targets)
+                for (InetAddressAndPort target : targets.endpoints())
                     getHintsInProgressFor(target).decrementAndGet();
             }
         }
@@ -2756,22 +2661,23 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static Future<Void> submitHint(Mutation mutation, InetAddressAndPort target, AbstractWriteResponseHandler<IMutation> responseHandler)
+    public static Future<Void> submitHint(Mutation mutation, Replica target, AbstractWriteResponseHandler<IMutation> responseHandler)
     {
-        return submitHint(mutation, Collections.singleton(target), responseHandler);
+        return submitHint(mutation, EndpointsForToken.of(target.range().right, target), responseHandler);
     }
 
     public static Future<Void> submitHint(Mutation mutation,
-                                          Collection<InetAddressAndPort> targets,
+                                          EndpointsForToken targets,
                                           AbstractWriteResponseHandler<IMutation> responseHandler)
     {
+        Replicas.assertFull(targets); // hints should not be written for transient replicas
         HintRunnable runnable = new HintRunnable(targets)
         {
             public void runMayThrow()
             {
                 Set<InetAddressAndPort> validTargets = new HashSet<>(targets.size());
                 Set<UUID> hostIds = new HashSet<>(targets.size());
-                for (InetAddressAndPort target : targets)
+                for (InetAddressAndPort target : targets.endpoints())
                 {
                     UUID hostId = StorageService.instance.getHostIdForEndpoint(target);
                     if (hostId != null)
@@ -2786,7 +2692,7 @@ public class StorageProxy implements StorageProxyMBean
                 HintsService.instance.write(hostIds, Hint.create(mutation, System.currentTimeMillis()));
                 validTargets.forEach(HintsService.instance.metrics::incrCreatedHints);
                 // Notify the handler only for CL == ANY
-                if (responseHandler != null && responseHandler.consistencyLevel == ConsistencyLevel.ANY)
+                if (responseHandler != null && responseHandler.replicaLayout.consistencyLevel() == ConsistencyLevel.ANY)
                     responseHandler.response(null);
             }
         };
@@ -2797,8 +2703,8 @@ public class StorageProxy implements StorageProxyMBean
     private static Future<Void> submitHint(HintRunnable runnable)
     {
         StorageMetrics.totalHintsInProgress.inc(runnable.targets.size());
-        for (InetAddressAndPort target : runnable.targets)
-            getHintsInProgressFor(target).incrementAndGet();
+        for (Replica target : runnable.targets)
+            getHintsInProgressFor(target.endpoint()).incrementAndGet();
         return (Future<Void>) StageManager.getStage(Stage.MUTATION).submit(runnable);
     }
 
@@ -2892,36 +2798,6 @@ public class StorageProxy implements StorageProxyMBean
         DatabaseDescriptor.setOtcBacklogExpirationInterval(intervalInMillis);
     }
 
-
-    static class PaxosParticipants
-    {
-        final List<InetAddressAndPort> liveEndpoints;
-        final int participants;
-
-        PaxosParticipants(List<InetAddressAndPort> liveEndpoints, int participants)
-        {
-            this.liveEndpoints = liveEndpoints;
-            this.participants = participants;
-        }
-
-        @Override
-        public final int hashCode()
-        {
-            int hashCode = 31 + (liveEndpoints == null ? 0 : liveEndpoints.hashCode());
-            return 31 * hashCode * this.participants;
-        }
-
-        @Override
-        public final boolean equals(Object o)
-        {
-            if(!(o instanceof PaxosParticipants))
-                return false;
-            PaxosParticipants that = (PaxosParticipants)o;
-            // handles nulls properly
-            return Objects.equals(liveEndpoints, that.liveEndpoints) && participants == that.participants;
-        }
-    }
-
     static class PaxosBallotAndContention
     {
         final UUID ballot;


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


[16/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 9766454..afe628b 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -112,6 +113,7 @@ public class CompactionStrategyManager implements INotificationConsumer
     /**
      * Variables guarded by read and write lock above
      */
+    private final PendingRepairHolder transientRepairs;
     private final PendingRepairHolder pendingRepairs;
     private final CompactionStrategyHolder repaired;
     private final CompactionStrategyHolder unrepaired;
@@ -156,10 +158,11 @@ public class CompactionStrategyManager implements INotificationConsumer
                 return compactionStrategyIndexForDirectory(descriptor);
             }
         };
-        pendingRepairs = new PendingRepairHolder(cfs, router);
+        transientRepairs = new PendingRepairHolder(cfs, router, true);
+        pendingRepairs = new PendingRepairHolder(cfs, router, false);
         repaired = new CompactionStrategyHolder(cfs, router, true);
         unrepaired = new CompactionStrategyHolder(cfs, router, false);
-        holders = ImmutableList.of(pendingRepairs, repaired, unrepaired);
+        holders = ImmutableList.of(transientRepairs, pendingRepairs, repaired, unrepaired);
 
         cfs.getTracker().subscribe(this);
         logger.trace("{} subscribed to the data tracker.", this);
@@ -176,7 +179,6 @@ public class CompactionStrategyManager implements INotificationConsumer
      * Return the next background task
      *
      * Returns a task for the compaction strategy that needs it the most (most estimated remaining tasks)
-     *
      */
     public AbstractCompactionTask getNextBackgroundTask(int gcBefore)
     {
@@ -188,18 +190,16 @@ public class CompactionStrategyManager implements INotificationConsumer
                 return null;
 
             int numPartitions = getNumTokenPartitions();
+
             // first try to promote/demote sstables from completed repairs
-            List<TaskSupplier> repairFinishedSuppliers = pendingRepairs.getRepairFinishedTaskSuppliers();
-            if (!repairFinishedSuppliers.isEmpty())
-            {
-                Collections.sort(repairFinishedSuppliers);
-                for (TaskSupplier supplier : repairFinishedSuppliers)
-                {
-                    AbstractCompactionTask task = supplier.getTask();
-                    if (task != null)
-                        return task;
-                }
-            }
+            AbstractCompactionTask repairFinishedTask;
+            repairFinishedTask = pendingRepairs.getNextRepairFinishedTask();
+            if (repairFinishedTask != null)
+                return repairFinishedTask;
+
+            repairFinishedTask = transientRepairs.getNextRepairFinishedTask();
+            if (repairFinishedTask != null)
+                return repairFinishedTask;
 
             // sort compaction task suppliers by remaining tasks descending
             List<TaskSupplier> suppliers = new ArrayList<>(numPartitions * holders.size());
@@ -393,64 +393,28 @@ public class CompactionStrategyManager implements INotificationConsumer
         }
     }
 
-
-
     @VisibleForTesting
-    List<AbstractCompactionStrategy> getRepaired()
+    CompactionStrategyHolder getRepairedUnsafe()
     {
-        readLock.lock();
-        try
-        {
-            return Lists.newArrayList(repaired.allStrategies());
-        }
-        finally
-        {
-            readLock.unlock();
-        }
+        return repaired;
     }
 
     @VisibleForTesting
-    List<AbstractCompactionStrategy> getUnrepaired()
+    CompactionStrategyHolder getUnrepairedUnsafe()
     {
-        readLock.lock();
-        try
-        {
-            return Lists.newArrayList(unrepaired.allStrategies());
-        }
-        finally
-        {
-            readLock.unlock();
-        }
+        return unrepaired;
     }
 
     @VisibleForTesting
-    Iterable<AbstractCompactionStrategy> getForPendingRepair(UUID sessionID)
+    PendingRepairHolder getPendingRepairsUnsafe()
     {
-        readLock.lock();
-        try
-        {
-            return pendingRepairs.getStrategiesFor(sessionID);
-        }
-        finally
-        {
-            readLock.unlock();
-        }
+        return pendingRepairs;
     }
 
     @VisibleForTesting
-    Set<UUID> pendingRepairs()
+    PendingRepairHolder getTransientRepairsUnsafe()
     {
-        readLock.lock();
-        try
-        {
-            Set<UUID> ids = new HashSet<>();
-            pendingRepairs.getManagers().forEach(p -> ids.addAll(p.getSessions()));
-            return ids;
-        }
-        finally
-        {
-            readLock.unlock();
-        }
+        return transientRepairs;
     }
 
     public boolean hasDataForPendingRepair(UUID sessionID)
@@ -458,8 +422,7 @@ public class CompactionStrategyManager implements INotificationConsumer
         readLock.lock();
         try
         {
-            return Iterables.any(pendingRepairs.getManagers(),
-                                 prm -> prm.hasDataForSession(sessionID));
+            return pendingRepairs.hasDataForSession(sessionID) || transientRepairs.hasDataForSession(sessionID);
         }
         finally
         {
@@ -682,18 +645,19 @@ public class CompactionStrategyManager implements INotificationConsumer
         throw new IllegalStateException("No holder claimed " + sstable);
     }
 
-    private AbstractStrategyHolder getHolder(long repairedAt, UUID pendingRepair)
+    private AbstractStrategyHolder getHolder(long repairedAt, UUID pendingRepair, boolean isTransient)
     {
         return getHolder(repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE,
-                         pendingRepair != ActiveRepairService.NO_PENDING_REPAIR);
+                         pendingRepair != ActiveRepairService.NO_PENDING_REPAIR,
+                         isTransient);
     }
 
     @VisibleForTesting
-    AbstractStrategyHolder getHolder(boolean isRepaired, boolean isPendingRepair)
+    AbstractStrategyHolder getHolder(boolean isRepaired, boolean isPendingRepair, boolean isTransient)
     {
         for (AbstractStrategyHolder holder : holders)
         {
-            if (holder.managesRepairedGroup(isRepaired, isPendingRepair))
+            if (holder.managesRepairedGroup(isRepaired, isPendingRepair, isTransient))
                 return holder;
         }
 
@@ -1146,16 +1110,26 @@ public class CompactionStrategyManager implements INotificationConsumer
                                                        long keyCount,
                                                        long repairedAt,
                                                        UUID pendingRepair,
+                                                       boolean isTransient,
                                                        MetadataCollector collector,
                                                        SerializationHeader header,
                                                        Collection<Index> indexes,
                                                        LifecycleTransaction txn)
     {
+        SSTable.validateRepairedMetadata(repairedAt, pendingRepair, isTransient);
         maybeReloadDiskBoundaries();
         readLock.lock();
         try
         {
-            return getHolder(repairedAt, pendingRepair).createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, collector, header, indexes, txn);
+            return getHolder(repairedAt, pendingRepair, isTransient).createSSTableMultiWriter(descriptor,
+                                                                                              keyCount,
+                                                                                              repairedAt,
+                                                                                              pendingRepair,
+                                                                                              isTransient,
+                                                                                              collector,
+                                                                                              header,
+                                                                                              indexes,
+                                                                                              txn);
         }
         finally
         {
@@ -1220,7 +1194,7 @@ public class CompactionStrategyManager implements INotificationConsumer
      * Mutates sstable repairedAt times and notifies listeners of the change with the writeLock held. Prevents races
      * with other processes between when the metadata is changed and when sstables are moved between strategies.
       */
-    public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt, UUID pendingRepair) throws IOException
+    public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt, UUID pendingRepair, boolean isTransient) throws IOException
     {
         Set<SSTableReader> changed = new HashSet<>();
 
@@ -1229,7 +1203,7 @@ public class CompactionStrategyManager implements INotificationConsumer
         {
             for (SSTableReader sstable: sstables)
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, repairedAt, pendingRepair);
+                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
                 sstable.reloadSSTableMetadata();
                 changed.add(sstable);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 662384c..591b7c4 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -29,20 +29,19 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.RateLimiter;
-
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
-import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -339,6 +338,23 @@ public class CompactionTask extends AbstractCompactionTask
         return ids.iterator().next();
     }
 
+    public static boolean getIsTransient(Set<SSTableReader> sstables)
+    {
+        if (sstables.isEmpty())
+        {
+            return false;
+        }
+
+        boolean isTransient = sstables.iterator().next().isTransient();
+
+        if (!Iterables.all(sstables, sstable -> sstable.isTransient() == isTransient))
+        {
+            throw new RuntimeException("Attempting to compact transient sstables with non transient sstables");
+        }
+
+        return isTransient;
+    }
+
 
     /*
      * Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
index 7b9123f..92e44a7 100644
--- a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
+++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.compaction;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 
@@ -43,10 +44,12 @@ import org.apache.cassandra.service.ActiveRepairService;
 public class PendingRepairHolder extends AbstractStrategyHolder
 {
     private final List<PendingRepairManager> managers = new ArrayList<>();
+    private final boolean isTransient;
 
-    public PendingRepairHolder(ColumnFamilyStore cfs, DestinationRouter router)
+    public PendingRepairHolder(ColumnFamilyStore cfs, DestinationRouter router, boolean isTransient)
     {
         super(cfs, router);
+        this.isTransient = isTransient;
     }
 
     @Override
@@ -66,15 +69,15 @@ public class PendingRepairHolder extends AbstractStrategyHolder
     {
         managers.clear();
         for (int i = 0; i < numTokenPartitions; i++)
-            managers.add(new PendingRepairManager(cfs, params));
+            managers.add(new PendingRepairManager(cfs, params, isTransient));
     }
 
     @Override
-    public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair)
+    public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair, boolean isTransient)
     {
         Preconditions.checkArgument(!isPendingRepair || !isRepaired,
                                     "SSTables cannot be both repaired and pending repair");
-        return isPendingRepair;
+        return isPendingRepair && (this.isTransient == isTransient);
     }
 
     @Override
@@ -145,7 +148,23 @@ public class PendingRepairHolder extends AbstractStrategyHolder
         return tasks;
     }
 
-    public ArrayList<TaskSupplier> getRepairFinishedTaskSuppliers()
+    AbstractCompactionTask getNextRepairFinishedTask()
+    {
+        List<TaskSupplier> repairFinishedSuppliers = getRepairFinishedTaskSuppliers();
+        if (!repairFinishedSuppliers.isEmpty())
+        {
+            Collections.sort(repairFinishedSuppliers);
+            for (TaskSupplier supplier : repairFinishedSuppliers)
+            {
+                AbstractCompactionTask task = supplier.getTask();
+                if (task != null)
+                    return task;
+            }
+        }
+        return null;
+    }
+
+    private ArrayList<TaskSupplier> getRepairFinishedTaskSuppliers()
     {
         ArrayList<TaskSupplier> suppliers = new ArrayList<>(managers.size());
         for (PendingRepairManager manager : managers)
@@ -218,6 +237,7 @@ public class PendingRepairHolder extends AbstractStrategyHolder
                                                        long keyCount,
                                                        long repairedAt,
                                                        UUID pendingRepair,
+                                                       boolean isTransient,
                                                        MetadataCollector collector,
                                                        SerializationHeader header,
                                                        Collection<Index> indexes,
@@ -233,6 +253,7 @@ public class PendingRepairHolder extends AbstractStrategyHolder
                                                  keyCount,
                                                  repairedAt,
                                                  pendingRepair,
+                                                 isTransient,
                                                  collector,
                                                  header,
                                                  indexes,
@@ -249,4 +270,15 @@ public class PendingRepairHolder extends AbstractStrategyHolder
         }
         return -1;
     }
+
+    public boolean hasDataForSession(UUID sessionID)
+    {
+        return Iterables.any(managers, prm -> prm.hasDataForSession(sessionID));
+    }
+
+    @Override
+    public boolean containsSSTable(SSTableReader sstable)
+    {
+        return Iterables.any(managers, prm -> prm.containsSSTable(sstable));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java
index edc9a2f..6763abf 100644
--- a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java
@@ -30,7 +30,9 @@ import java.util.UUID;
 import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
 import org.slf4j.Logger;
@@ -62,6 +64,7 @@ class PendingRepairManager
 
     private final ColumnFamilyStore cfs;
     private final CompactionParams params;
+    private final boolean isTransient;
     private volatile ImmutableMap<UUID, AbstractCompactionStrategy> strategies = ImmutableMap.of();
 
     /**
@@ -75,10 +78,11 @@ class PendingRepairManager
         }
     }
 
-    PendingRepairManager(ColumnFamilyStore cfs, CompactionParams params)
+    PendingRepairManager(ColumnFamilyStore cfs, CompactionParams params, boolean isTransient)
     {
         this.cfs = cfs;
         this.params = params;
+        this.isTransient = isTransient;
     }
 
     private ImmutableMap.Builder<UUID, AbstractCompactionStrategy> mapBuilder()
@@ -156,6 +160,7 @@ class PendingRepairManager
 
     synchronized void addSSTable(SSTableReader sstable)
     {
+        Preconditions.checkArgument(sstable.isTransient() == isTransient);
         getOrCreate(sstable).addSSTable(sstable);
     }
 
@@ -389,6 +394,15 @@ class PendingRepairManager
         return strategies.keySet().contains(sessionID);
     }
 
+    boolean containsSSTable(SSTableReader sstable)
+    {
+        if (!sstable.isPendingRepair())
+            return false;
+
+        AbstractCompactionStrategy strategy = strategies.get(sstable.getPendingRepair());
+        return strategy != null && strategy.getSSTables().contains(sstable);
+    }
+
     public Collection<AbstractCompactionTask> createUserDefinedTasks(Collection<SSTableReader> sstables, int gcBefore)
     {
         Map<UUID, List<SSTableReader>> group = sstables.stream().collect(Collectors.groupingBy(s -> s.getSSTableMetadata().pendingRepair));
@@ -419,18 +433,35 @@ class PendingRepairManager
         protected void runMayThrow() throws Exception
         {
             boolean completed = false;
+            boolean obsoleteSSTables = isTransient && repairedAt > 0;
             try
             {
-                logger.debug("Setting repairedAt to {} on {} for {}", repairedAt, transaction.originals(), sessionID);
-                cfs.getCompactionStrategyManager().mutateRepaired(transaction.originals(), repairedAt, ActiveRepairService.NO_PENDING_REPAIR);
+                if (obsoleteSSTables)
+                {
+                    logger.info("Obsoleting transient repaired ssatbles");
+                    Preconditions.checkState(Iterables.all(transaction.originals(), SSTableReader::isTransient));
+                    transaction.obsoleteOriginals();
+                }
+                else
+                {
+                    logger.debug("Setting repairedAt to {} on {} for {}", repairedAt, transaction.originals(), sessionID);
+                    cfs.getCompactionStrategyManager().mutateRepaired(transaction.originals(), repairedAt, ActiveRepairService.NO_PENDING_REPAIR, false);
+                }
                 completed = true;
             }
             finally
             {
-                // we always abort because mutating metadata isn't guarded by LifecycleTransaction, so this won't roll
-                // anything back. Also, we don't want to obsolete the originals. We're only using it to prevent other
-                // compactions from marking these sstables compacting, and unmarking them when we're done
-                transaction.abort();
+                if (obsoleteSSTables)
+                {
+                    transaction.finish();
+                }
+                else
+                {
+                    // we abort here because mutating metadata isn't guarded by LifecycleTransaction, so this won't roll
+                    // anything back. Also, we don't want to obsolete the originals. We're only using it to prevent other
+                    // compactions from marking these sstables compacting, and unmarking them when we're done
+                    transaction.abort();
+                }
                 if (completed)
                 {
                     removeSession(sessionID);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index f97b693..aa41051 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -170,7 +170,7 @@ public class Scrubber implements Closeable
             }
 
             StatsMetadata metadata = sstable.getSSTableMetadata();
-            writer.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, sstable, transaction));
+            writer.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, metadata.isTransient, sstable, transaction));
 
             DecoratedKey prevKey = null;
 
@@ -277,7 +277,7 @@ public class Scrubber implements Closeable
                 // out of order rows, but no bad rows found - we can keep our repairedAt time
                 long repairedAt = badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : metadata.repairedAt;
                 SSTableReader newInOrderSstable;
-                try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, metadata.pendingRepair, sstable, transaction))
+                try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, metadata.pendingRepair, metadata.isTransient, sstable, transaction))
                 {
                     for (Partition partition : outOfOrder)
                         inOrderWriter.append(partition.unfilteredIterator());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/Upgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 80453ef..e1406aa 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -68,14 +68,15 @@ public class Upgrader
         this.estimatedRows = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables);
     }
 
-    private SSTableWriter createCompactionWriter(long repairedAt, UUID parentRepair)
+    private SSTableWriter createCompactionWriter(StatsMetadata metadata)
     {
         MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.getComparator());
         sstableMetadataCollector.sstableLevel(sstable.getSSTableLevel());
         return SSTableWriter.create(cfs.newSSTableDescriptor(directory),
                                     estimatedRows,
-                                    repairedAt,
-                                    parentRepair,
+                                    metadata.repairedAt,
+                                    metadata.pendingRepair,
+                                    metadata.isTransient,
                                     cfs.metadata,
                                     sstableMetadataCollector,
                                     SerializationHeader.make(cfs.metadata(), Sets.newHashSet(sstable)),
@@ -91,8 +92,7 @@ public class Upgrader
              AbstractCompactionStrategy.ScannerList scanners = strategyManager.getScanners(transaction.originals());
              CompactionIterator iter = new CompactionIterator(transaction.opType(), scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
         {
-            StatsMetadata metadata = sstable.getSSTableMetadata();
-            writer.switchWriter(createCompactionWriter(metadata.repairedAt, metadata.pendingRepair));
+            writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata()));
             while (iter.hasNext())
                 writer.append(iter.next());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/Verifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java
index db49369..446d527 100644
--- a/src/java/org/apache/cassandra/db/compaction/Verifier.java
+++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java
@@ -350,6 +350,7 @@ public class Verifier implements Closeable
         public RangeOwnHelper(List<Range<Token>> normalizedRanges)
         {
             this.normalizedRanges = normalizedRanges;
+            Range.assertNormalized(normalizedRanges);
         }
 
         /**
@@ -457,7 +458,7 @@ public class Verifier implements Closeable
         {
             try
             {
-                sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getSSTableMetadata().pendingRepair);
+                sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getPendingRepair(), sstable.isTransient());
                 sstable.reloadSSTableMetadata();
                 cfs.getTracker().notifySSTableRepairedStatusChanged(Collections.singleton(sstable));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index 5ddd99c..d72b236 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -57,6 +57,7 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
     protected final long maxAge;
     protected final long minRepairedAt;
     protected final UUID pendingRepair;
+    protected final boolean isTransient;
 
     protected final SSTableRewriter sstableWriter;
     protected final LifecycleTransaction txn;
@@ -91,6 +92,7 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
         sstableWriter = SSTableRewriter.construct(cfs, txn, keepOriginals, maxAge);
         minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
         pendingRepair = CompactionTask.getPendingRepair(nonExpiredSSTables);
+        isTransient = CompactionTask.getIsTransient(nonExpiredSSTables);
         DiskBoundaries db = cfs.getDiskBoundaries();
         diskBoundaries = db.positions;
         locations = db.directories;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
index cda7e38..6180f96 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -72,6 +72,7 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
                                                     estimatedTotalKeys,
                                                     minRepairedAt,
                                                     pendingRepair,
+                                                    isTransient,
                                                     cfs.metadata,
                                                     new MetadataCollector(txn.originals(), cfs.metadata().comparator, sstableLevel),
                                                     SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
index 3959b4b..2b93eb4 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -108,6 +108,7 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
                 keysPerSSTable,
                 minRepairedAt,
                 pendingRepair,
+                isTransient,
                 cfs.metadata,
                 new MetadataCollector(txn.originals(), cfs.metadata().comparator, currentLevel),
                 SerializationHeader.make(cfs.metadata(), txn.originals()),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
index c4f84e8..df7eeaf 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -111,6 +111,7 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
                                                     estimatedTotalKeys / estimatedSSTables,
                                                     minRepairedAt,
                                                     pendingRepair,
+                                                    isTransient,
                                                     cfs.metadata,
                                                     new MetadataCollector(allSSTables, cfs.metadata().comparator, level),
                                                     SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index a4af783..7533f1d 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -107,6 +107,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
                                                     currentPartitionsToWrite,
                                                     minRepairedAt,
                                                     pendingRepair,
+                                                    isTransient,
                                                     cfs.metadata,
                                                     new MetadataCollector(allSSTables, cfs.metadata().comparator, 0),
                                                     SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
index 9064b0f..bed0958 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
@@ -82,18 +82,6 @@ public abstract class PartitionIterators
         return new SingletonPartitionIterator(iterator);
     }
 
-    public static void consume(PartitionIterator iterator)
-    {
-        while (iterator.hasNext())
-        {
-            try (RowIterator partition = iterator.next())
-            {
-                while (partition.hasNext())
-                    partition.next();
-            }
-        }
-    }
-
     /**
      * Wraps the provided iterator so it logs the returned rows for debugging purposes.
      * <p>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java b/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
index 5f2e5a0..fa2e653 100644
--- a/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
+++ b/src/java/org/apache/cassandra/db/repair/CassandraKeyspaceRepairManager.java
@@ -26,8 +26,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.repair.KeyspaceRepairManager;
 
 public class CassandraKeyspaceRepairManager implements KeyspaceRepairManager
@@ -40,9 +39,12 @@ public class CassandraKeyspaceRepairManager implements KeyspaceRepairManager
     }
 
     @Override
-    public ListenableFuture prepareIncrementalRepair(UUID sessionID, Collection<ColumnFamilyStore> tables, Collection<Range<Token>> ranges, ExecutorService executor)
+    public ListenableFuture prepareIncrementalRepair(UUID sessionID,
+                                                     Collection<ColumnFamilyStore> tables,
+                                                     RangesAtEndpoint tokenRanges,
+                                                     ExecutorService executor)
     {
-        PendingAntiCompaction pac = new PendingAntiCompaction(sessionID, tables, ranges, executor);
+        PendingAntiCompaction pac = new PendingAntiCompaction(sessionID, tables, tokenRanges, executor);
         return pac.run();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
index 4e0f13d..a205c3c 100644
--- a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
+++ b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.utils.concurrent.Refs;
 
 /**
@@ -126,17 +127,17 @@ public class PendingAntiCompaction
     static class AcquisitionCallback implements AsyncFunction<List<AcquireResult>, Object>
     {
         private final UUID parentRepairSession;
-        private final Collection<Range<Token>> ranges;
+        private final RangesAtEndpoint tokenRanges;
 
-        public AcquisitionCallback(UUID parentRepairSession, Collection<Range<Token>> ranges)
+        public AcquisitionCallback(UUID parentRepairSession, RangesAtEndpoint tokenRanges)
         {
             this.parentRepairSession = parentRepairSession;
-            this.ranges = ranges;
+            this.tokenRanges = tokenRanges;
         }
 
         ListenableFuture<?> submitPendingAntiCompaction(AcquireResult result)
         {
-            return CompactionManager.instance.submitPendingAntiCompaction(result.cfs, ranges, result.refs, result.txn, parentRepairSession);
+            return CompactionManager.instance.submitPendingAntiCompaction(result.cfs, tokenRanges, result.refs, result.txn, parentRepairSession);
         }
 
         public ListenableFuture apply(List<AcquireResult> results) throws Exception
@@ -177,14 +178,17 @@ public class PendingAntiCompaction
 
     private final UUID prsId;
     private final Collection<ColumnFamilyStore> tables;
-    private final Collection<Range<Token>> ranges;
+    private final RangesAtEndpoint tokenRanges;
     private final ExecutorService executor;
 
-    public PendingAntiCompaction(UUID prsId, Collection<ColumnFamilyStore> tables, Collection<Range<Token>> ranges, ExecutorService executor)
+    public PendingAntiCompaction(UUID prsId,
+                                 Collection<ColumnFamilyStore> tables,
+                                 RangesAtEndpoint tokenRanges,
+                                 ExecutorService executor)
     {
         this.prsId = prsId;
         this.tables = tables;
-        this.ranges = ranges;
+        this.tokenRanges = tokenRanges;
         this.executor = executor;
     }
 
@@ -194,12 +198,12 @@ public class PendingAntiCompaction
         for (ColumnFamilyStore cfs : tables)
         {
             cfs.forceBlockingFlush();
-            ListenableFutureTask<AcquireResult> task = ListenableFutureTask.create(new AcquisitionCallable(cfs, ranges, prsId));
+            ListenableFutureTask<AcquireResult> task = ListenableFutureTask.create(new AcquisitionCallable(cfs, tokenRanges.ranges(), prsId));
             executor.submit(task);
             tasks.add(task);
         }
         ListenableFuture<List<AcquireResult>> acquisitionResults = Futures.successfulAsList(tasks);
-        ListenableFuture compactionResult = Futures.transformAsync(acquisitionResults, new AcquisitionCallback(prsId, ranges), MoreExecutors.directExecutor());
+        ListenableFuture compactionResult = Futures.transformAsync(acquisitionResults, new AcquisitionCallback(prsId, tokenRanges), MoreExecutors.directExecutor());
         return compactionResult;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
index 5252187..c688fdf 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
@@ -68,17 +68,18 @@ public class CassandraOutgoingFile implements OutgoingStream
     private final ComponentManifest manifest;
     private Boolean isFullyContained;
 
-    private final List<Range<Token>> ranges;
+    private final List<Range<Token>> normalizedRanges;
 
     public CassandraOutgoingFile(StreamOperation operation, Ref<SSTableReader> ref,
-                                 List<SSTableReader.PartitionPositionBounds> sections, Collection<Range<Token>> ranges,
+                                 List<SSTableReader.PartitionPositionBounds> sections, List<Range<Token>> normalizedRanges,
                                  long estimatedKeys)
     {
         Preconditions.checkNotNull(ref.get());
+        Range.assertNormalized(normalizedRanges);
         this.ref = ref;
         this.estimatedKeys = estimatedKeys;
         this.sections = sections;
-        this.ranges = ImmutableList.copyOf(ranges);
+        this.normalizedRanges = ImmutableList.copyOf(normalizedRanges);
         this.filename = ref.get().getFilename();
         this.manifest = getComponentManifest(ref.get());
 
@@ -194,7 +195,7 @@ public class CassandraOutgoingFile implements OutgoingStream
                                                            .getCompactionStrategyFor(ref.get());
 
         if (compactionStrategy instanceof LeveledCompactionStrategy)
-            return contained(ranges, ref.get());
+            return contained(normalizedRanges, ref.get());
 
         return false;
     }
@@ -251,6 +252,6 @@ public class CassandraOutgoingFile implements OutgoingStream
     @Override
     public String toString()
     {
-        return "CassandraOutgoingFile{" + ref.get().getFilename() + '}';
+        return "CassandraOutgoingFile{" + filename + '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java
index 43667d0..6c2631c 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java
@@ -18,19 +18,10 @@
 
 package org.apache.cassandra.db.streaming;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
@@ -39,6 +30,8 @@ import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.IncomingStream;
 import org.apache.cassandra.streaming.OutgoingStream;
@@ -49,6 +42,14 @@ import org.apache.cassandra.streaming.TableStreamManager;
 import org.apache.cassandra.streaming.messages.StreamMessageHeader;
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.concurrent.Refs;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 
 /**
  * Implements the streaming interface for the native cassandra storage engine.
@@ -96,14 +97,14 @@ public class CassandraStreamManager implements TableStreamManager
     }
 
     @Override
-    public Collection<OutgoingStream> createOutgoingStreams(StreamSession session, Collection<Range<Token>> ranges, UUID pendingRepair, PreviewKind previewKind)
+    public Collection<OutgoingStream> createOutgoingStreams(StreamSession session, RangesAtEndpoint replicas, UUID pendingRepair, PreviewKind previewKind)
     {
         Refs<SSTableReader> refs = new Refs<>();
         try
         {
-            final List<Range<PartitionPosition>> keyRanges = new ArrayList<>(ranges.size());
-            for (Range<Token> range : ranges)
-                keyRanges.add(Range.makeRowRange(range));
+            final List<Range<PartitionPosition>> keyRanges = new ArrayList<>(replicas.size());
+            for (Replica replica : replicas)
+                keyRanges.add(Range.makeRowRange(replica.range()));
             refs.addAll(cfs.selectAndReference(view -> {
                 Set<SSTableReader> sstables = Sets.newHashSet();
                 SSTableIntervalTree intervalTree = SSTableIntervalTree.build(view.select(SSTableSet.CANONICAL));
@@ -141,11 +142,16 @@ public class CassandraStreamManager implements TableStreamManager
             }).refs);
 
 
+            List<Range<Token>> normalizedFullRanges = Range.normalize(replicas.filter(Replica::isFull).ranges());
+            List<Range<Token>> normalizedAllRanges = Range.normalize(replicas.ranges());
+            //Create outgoing file streams for ranges possibly skipping repaired ranges in sstables
             List<OutgoingStream> streams = new ArrayList<>(refs.size());
-            for (SSTableReader sstable: refs)
+            for (SSTableReader sstable : refs)
             {
-                Ref<SSTableReader> ref = refs.get(sstable);
+                List<Range<Token>> ranges = sstable.isRepaired() ? normalizedFullRanges : normalizedAllRanges;
                 List<SSTableReader.PartitionPositionBounds> sections = sstable.getPositionsForRanges(ranges);
+
+                Ref<SSTableReader> ref = refs.get(sstable);
                 if (sections.isEmpty())
                 {
                     ref.release();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
index fccabfe..572c648 100644
--- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
+++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java
@@ -156,7 +156,7 @@ public class CassandraStreamReader implements IStreamReader
         Preconditions.checkState(streamReceiver instanceof CassandraStreamReceiver);
         LifecycleTransaction txn = CassandraStreamReceiver.fromReceiver(session.getAggregator(tableId)).getTransaction();
 
-        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata()));
+        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, false, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata()));
         return writer;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java
index d35457e..09490e8 100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -60,6 +60,11 @@ public class TableViews extends AbstractCollection<View>
         baseTableMetadata = Schema.instance.getTableMetadataRef(id);
     }
 
+    public boolean hasViews()
+    {
+        return !views.isEmpty();
+    }
+
     public int size()
     {
         return views.size();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index c727f63..6717297 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -43,6 +43,8 @@ import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replicas;
 import org.apache.cassandra.repair.SystemDistributedKeyspace;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -135,14 +137,15 @@ class ViewBuilder
         }
 
         // Get the local ranges for which the view hasn't already been built nor it's building
-        Set<Range<Token>> newRanges = StorageService.instance.getLocalRanges(ksName)
-                                                             .stream()
-                                                             .map(r -> r.subtractAll(builtRanges))
-                                                             .flatMap(Set::stream)
-                                                             .map(r -> r.subtractAll(pendingRanges.keySet()))
-                                                             .flatMap(Set::stream)
-                                                             .collect(Collectors.toSet());
-
+        RangesAtEndpoint replicatedRanges = StorageService.instance.getLocalReplicas(ksName);
+        Replicas.temporaryAssertFull(replicatedRanges);
+        Set<Range<Token>> newRanges = replicatedRanges.ranges()
+                                                      .stream()
+                                                      .map(r -> r.subtractAll(builtRanges))
+                                                      .flatMap(Set::stream)
+                                                      .map(r -> r.subtractAll(pendingRanges.keySet()))
+                                                      .flatMap(Set::stream)
+                                                      .collect(Collectors.toSet());
         // If there are no new nor pending ranges we should finish the build
         if (newRanges.isEmpty() && pendingRanges.isEmpty())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java
index 000477d..7e3ea1b 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -79,7 +79,7 @@ public class ViewManager
             {
                 assert keyspace.getName().equals(update.metadata().keyspace);
 
-                if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
+                if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor().allReplicas == 1)
                     continue;
 
                 if (!forTable(update.metadata().id).updatedViews(update).isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/db/view/ViewUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewUtils.java b/src/java/org/apache/cassandra/db/view/ViewUtils.java
index df16943..ad10d9d 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUtils.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUtils.java
@@ -18,16 +18,17 @@
 
 package org.apache.cassandra.db.view;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Optional;
+import java.util.function.Predicate;
 
+import com.google.common.collect.Iterables;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.utils.FBUtilities;
 
 public final class ViewUtils
@@ -58,46 +59,51 @@ public final class ViewUtils
      *
      * @return Optional.empty() if this method is called using a base token which does not belong to this replica
      */
-    public static Optional<InetAddressAndPort> getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
+    public static Optional<Replica> getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
     {
         AbstractReplicationStrategy replicationStrategy = Keyspace.open(keyspaceName).getReplicationStrategy();
 
         String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort());
-        List<InetAddressAndPort> baseEndpoints = new ArrayList<>();
-        List<InetAddressAndPort> viewEndpoints = new ArrayList<>();
-        for (InetAddressAndPort baseEndpoint : replicationStrategy.getNaturalEndpoints(baseToken))
-        {
-            // An endpoint is local if we're not using Net
-            if (!(replicationStrategy instanceof NetworkTopologyStrategy) ||
-                DatabaseDescriptor.getEndpointSnitch().getDatacenter(baseEndpoint).equals(localDataCenter))
-                baseEndpoints.add(baseEndpoint);
-        }
+        EndpointsForToken naturalBaseReplicas = replicationStrategy.getNaturalReplicasForToken(baseToken);
+        EndpointsForToken naturalViewReplicas = replicationStrategy.getNaturalReplicasForToken(viewToken);
 
-        for (InetAddressAndPort viewEndpoint : replicationStrategy.getNaturalEndpoints(viewToken))
-        {
-            // If we are a base endpoint which is also a view replica, we use ourselves as our view replica
-            if (viewEndpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
-                return Optional.of(viewEndpoint);
+        Optional<Replica> localReplica = Iterables.tryFind(naturalViewReplicas, Replica::isLocal).toJavaUtil();
+        if (localReplica.isPresent())
+            return localReplica;
 
-            // We have to remove any endpoint which is shared between the base and the view, as it will select itself
-            // and throw off the counts otherwise.
-            if (baseEndpoints.contains(viewEndpoint))
-                baseEndpoints.remove(viewEndpoint);
-            else if (!(replicationStrategy instanceof NetworkTopologyStrategy) ||
-                     DatabaseDescriptor.getEndpointSnitch().getDatacenter(viewEndpoint).equals(localDataCenter))
-                viewEndpoints.add(viewEndpoint);
-        }
+        // We only select replicas from our own DC
+        // TODO: this is poor encapsulation, leaking implementation details of replication strategy
+        Predicate<Replica> isLocalDC = r -> !(replicationStrategy instanceof NetworkTopologyStrategy)
+                || DatabaseDescriptor.getEndpointSnitch().getDatacenter(r).equals(localDataCenter);
+
+        // We have to remove any endpoint which is shared between the base and the view, as it will select itself
+        // and throw off the counts otherwise.
+        EndpointsForToken baseReplicas = naturalBaseReplicas.filter(
+                r -> !naturalViewReplicas.endpoints().contains(r.endpoint()) && isLocalDC.test(r)
+        );
+        EndpointsForToken viewReplicas = naturalViewReplicas.filter(
+                r -> !naturalBaseReplicas.endpoints().contains(r.endpoint()) && isLocalDC.test(r)
+        );
 
         // The replication strategy will be the same for the base and the view, as they must belong to the same keyspace.
         // Since the same replication strategy is used, the same placement should be used and we should get the same
         // number of replicas for all of the tokens in the ring.
-        assert baseEndpoints.size() == viewEndpoints.size() : "Replication strategy should have the same number of endpoints for the base and the view";
-        int baseIdx = baseEndpoints.indexOf(FBUtilities.getBroadcastAddressAndPort());
+        assert baseReplicas.size() == viewReplicas.size() : "Replication strategy should have the same number of endpoints for the base and the view";
+
+        int baseIdx = -1;
+        for (int i=0; i<baseReplicas.size(); i++)
+        {
+            if (baseReplicas.get(i).isLocal())
+            {
+                baseIdx = i;
+                break;
+            }
+        }
 
         if (baseIdx < 0)
             //This node is not a base replica of this key, so we return empty
             return Optional.empty();
 
-        return Optional.of(viewEndpoints.get(baseIdx));
+        return Optional.of(viewReplicas.get(baseIdx));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/Range.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Range.java b/src/java/org/apache/cassandra/dht/Range.java
index 974d08e..e03c5ec 100644
--- a/src/java/org/apache/cassandra/dht/Range.java
+++ b/src/java/org/apache/cassandra/dht/Range.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.dht;
 
 import java.io.Serializable;
 import java.util.*;
+import java.util.function.Predicate;
 
 import org.apache.commons.lang3.ObjectUtils;
 
@@ -529,7 +530,7 @@ public class Range<T extends RingPosition<T>> extends AbstractBounds<T> implemen
     /**
      * Helper class to check if a token is contained within a given collection of ranges
      */
-    public static class OrderedRangeContainmentChecker
+    public static class OrderedRangeContainmentChecker implements Predicate<Token>
     {
         private final Iterator<Range<Token>> normalizedRangesIterator;
         private Token lastToken = null;
@@ -550,7 +551,8 @@ public class Range<T extends RingPosition<T>> extends AbstractBounds<T> implemen
          * @param t token to check, must be larger than or equal to the last token passed
          * @return true if the token is contained within the ranges given to the constructor.
          */
-        public boolean contains(Token t)
+        @Override
+        public boolean test(Token t)
         {
             assert lastToken == null || lastToken.compareTo(t) <= 0;
             lastToken = t;
@@ -567,4 +569,25 @@ public class Range<T extends RingPosition<T>> extends AbstractBounds<T> implemen
             }
         }
     }
+
+    public static <T extends RingPosition<T>> void assertNormalized(List<Range<T>> ranges)
+    {
+        Range<T> lastRange = null;
+        for (Range<T> range : ranges)
+        {
+            if (lastRange == null)
+            {
+                lastRange = range;
+            }
+            else if (lastRange.left.compareTo(range.left) >= 0 || lastRange.intersects(range))
+            {
+                throw new AssertionError(String.format("Ranges aren't properly normalized. lastRange %s, range %s, compareTo %d, intersects %b, all ranges %s%n",
+                                                       lastRange,
+                                                       range,
+                                                       lastRange.compareTo(range),
+                                                       lastRange.intersects(range),
+                                                       ranges));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
index b90bc96..4b98b97 100644
--- a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
+++ b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java
@@ -19,25 +19,27 @@
 package org.apache.cassandra.dht;
 
 import java.math.BigInteger;
-import java.net.InetAddress;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
-import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
+import org.apache.cassandra.locator.EndpointsByRange;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.locator.Replica;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.Replicas;
 import org.psjava.algo.graph.flownetwork.FordFulkersonAlgorithm;
 import org.psjava.algo.graph.flownetwork.MaximumFlowAlgorithm;
 import org.psjava.algo.graph.flownetwork.MaximumFlowAlgorithmResult;
@@ -73,20 +75,20 @@ public class RangeFetchMapCalculator
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeFetchMapCalculator.class);
     private static final long TRIVIAL_RANGE_LIMIT = 1000;
-    private final Multimap<Range<Token>, InetAddressAndPort> rangesWithSources;
-    private final Collection<RangeStreamer.ISourceFilter> sourceFilters;
+    private final EndpointsByRange rangesWithSources;
+    private final Predicate<Replica> sourceFilters;
     private final String keyspace;
     //We need two Vertices to act as source and destination in the algorithm
     private final Vertex sourceVertex = OuterVertex.getSourceVertex();
     private final Vertex destinationVertex = OuterVertex.getDestinationVertex();
     private final Set<Range<Token>> trivialRanges;
 
-    public RangeFetchMapCalculator(Multimap<Range<Token>, InetAddressAndPort> rangesWithSources,
-                                   Collection<RangeStreamer.ISourceFilter> sourceFilters,
+    public RangeFetchMapCalculator(EndpointsByRange rangesWithSources,
+                                   Collection<Predicate<Replica>> sourceFilters,
                                    String keyspace)
     {
         this.rangesWithSources = rangesWithSources;
-        this.sourceFilters = sourceFilters;
+        this.sourceFilters = Predicates.and(sourceFilters);
         this.keyspace = keyspace;
         this.trivialRanges = rangesWithSources.keySet()
                                               .stream()
@@ -158,14 +160,15 @@ public class RangeFetchMapCalculator
             boolean localDCCheck = true;
             while (!added)
             {
-                List<InetAddressAndPort> srcs = new ArrayList<>(rangesWithSources.get(trivialRange));
                 // sort with the endpoint having the least number of streams first:
-                srcs.sort(Comparator.comparingInt(o -> optimisedMap.get(o).size()));
-                for (InetAddressAndPort src : srcs)
+                EndpointsForRange replicas = rangesWithSources.get(trivialRange)
+                        .sorted(Comparator.comparingInt(o -> optimisedMap.get(o.endpoint()).size()));
+                Replicas.temporaryAssertFull(replicas);
+                for (Replica replica : replicas)
                 {
-                    if (passFilters(src, localDCCheck))
+                    if (passFilters(replica, localDCCheck))
                     {
-                        fetchMap.put(src, trivialRange);
+                        fetchMap.put(replica.endpoint(), trivialRange);
                         added = true;
                         break;
                     }
@@ -347,15 +350,16 @@ public class RangeFetchMapCalculator
     private boolean addEndpoints(MutableCapacityGraph<Vertex, Integer> capacityGraph, RangeVertex rangeVertex, boolean localDCCheck)
     {
         boolean sourceFound = false;
-        for (InetAddressAndPort endpoint : rangesWithSources.get(rangeVertex.getRange()))
+        Replicas.temporaryAssertFull(rangesWithSources.get(rangeVertex.getRange()));
+        for (Replica replica : rangesWithSources.get(rangeVertex.getRange()))
         {
-            if (passFilters(endpoint, localDCCheck))
+            if (passFilters(replica, localDCCheck))
             {
                 sourceFound = true;
                 // if we pass filters, it means that we don't filter away localhost and we can count it as a source:
-                if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()))
+                if (replica.isLocal())
                     continue; // but don't add localhost to the graph to avoid streaming locally
-                final Vertex endpointVertex = new EndpointVertex(endpoint);
+                final Vertex endpointVertex = new EndpointVertex(replica.endpoint());
                 capacityGraph.insertVertex(rangeVertex);
                 capacityGraph.insertVertex(endpointVertex);
                 capacityGraph.addEdge(rangeVertex, endpointVertex, Integer.MAX_VALUE);
@@ -364,26 +368,20 @@ public class RangeFetchMapCalculator
         return sourceFound;
     }
 
-    private boolean isInLocalDC(InetAddressAndPort endpoint)
+    private boolean isInLocalDC(Replica replica)
     {
-        return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint));
+        return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica));
     }
 
     /**
      *
-     * @param endpoint   Endpoint to check
+     * @param replica   Replica to check
      * @param localDCCheck Allow endpoints with local DC
      * @return   True if filters pass this endpoint
      */
-    private boolean passFilters(final InetAddressAndPort endpoint, boolean localDCCheck)
+    private boolean passFilters(final Replica replica, boolean localDCCheck)
     {
-        for (RangeStreamer.ISourceFilter filter : sourceFilters)
-        {
-            if (!filter.shouldInclude(endpoint))
-                return false;
-        }
-
-        return !localDCCheck || isInLocalDC(endpoint);
+        return sourceFilters.apply(replica) && (!localDCCheck || isInLocalDC(replica));
     }
 
     private static abstract class Vertex


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


[08/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java
index 61b9948..031326e 100644
--- a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.service.reads;
 
-import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.locator.ReplicaLayout;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,15 +38,20 @@ import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.EndpointsForToken;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.reads.repair.ReadRepair;
 import org.apache.cassandra.service.StorageProxy.LocalReadRunnable;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.tryFind;
+
 /**
  * Sends a read request to the replicas needed to satisfy a given ConsistencyLevel.
  *
@@ -59,32 +65,27 @@ public abstract class AbstractReadExecutor
     private static final Logger logger = LoggerFactory.getLogger(AbstractReadExecutor.class);
 
     protected final ReadCommand command;
-    protected final ConsistencyLevel consistency;
-    protected final List<InetAddressAndPort> targetReplicas;
-    protected final ReadRepair readRepair;
-    protected final DigestResolver digestResolver;
-    protected final ReadCallback handler;
+    private   final ReplicaLayout.ForToken replicaLayout;
+    protected final ReadRepair<EndpointsForToken, ReplicaLayout.ForToken> readRepair;
+    protected final DigestResolver<EndpointsForToken, ReplicaLayout.ForToken> digestResolver;
+    protected final ReadCallback<EndpointsForToken, ReplicaLayout.ForToken> handler;
     protected final TraceState traceState;
     protected final ColumnFamilyStore cfs;
     protected final long queryStartNanoTime;
+    private   final int initialDataRequestCount;
     protected volatile PartitionIterator result = null;
 
-    protected final Keyspace keyspace;
-    protected final int blockFor;
-
-    AbstractReadExecutor(Keyspace keyspace, ColumnFamilyStore cfs, ReadCommand command, ConsistencyLevel consistency, List<InetAddressAndPort> targetReplicas, long queryStartNanoTime)
+    AbstractReadExecutor(ColumnFamilyStore cfs, ReadCommand command, ReplicaLayout.ForToken replicaLayout, int initialDataRequestCount, long queryStartNanoTime)
     {
         this.command = command;
-        this.consistency = consistency;
-        this.targetReplicas = targetReplicas;
-        this.readRepair = ReadRepair.create(command, queryStartNanoTime, consistency);
-        this.digestResolver = new DigestResolver(keyspace, command, consistency, readRepair, targetReplicas.size());
-        this.handler = new ReadCallback(digestResolver, consistency, command, targetReplicas, queryStartNanoTime);
+        this.replicaLayout = replicaLayout;
+        this.initialDataRequestCount = initialDataRequestCount;
+        this.readRepair = ReadRepair.create(command, replicaLayout, queryStartNanoTime);
+        this.digestResolver = new DigestResolver<>(command, replicaLayout, readRepair, queryStartNanoTime);
+        this.handler = new ReadCallback<>(digestResolver, replicaLayout.consistencyLevel().blockFor(replicaLayout.keyspace()), command, replicaLayout, queryStartNanoTime);
         this.cfs = cfs;
         this.traceState = Tracing.instance.get();
         this.queryStartNanoTime = queryStartNanoTime;
-        this.keyspace = keyspace;
-        this.blockFor = consistency.blockFor(keyspace);
 
 
         // Set the digest version (if we request some digests). This is the smallest version amongst all our target replicas since new nodes
@@ -92,8 +93,8 @@ public abstract class AbstractReadExecutor
         // TODO: we need this when talking with pre-3.0 nodes. So if we preserve the digest format moving forward, we can get rid of this once
         // we stop being compatible with pre-3.0 nodes.
         int digestVersion = MessagingService.current_version;
-        for (InetAddressAndPort replica : targetReplicas)
-            digestVersion = Math.min(digestVersion, MessagingService.instance().getVersion(replica));
+        for (Replica replica : replicaLayout.selected())
+            digestVersion = Math.min(digestVersion, MessagingService.instance().getVersion(replica.endpoint()));
         command.setDigestVersion(digestVersion);
     }
 
@@ -109,24 +110,34 @@ public abstract class AbstractReadExecutor
         return readRepair;
     }
 
-    protected void makeDataRequests(Iterable<InetAddressAndPort> endpoints)
+    protected void makeFullDataRequests(ReplicaCollection<?> replicas)
     {
-        makeRequests(command, endpoints);
+        assert all(replicas, Replica::isFull);
+        makeRequests(command, replicas.filter(Replica::isFull));
+    }
 
+    protected void makeTransientDataRequests(ReplicaCollection<?> replicas)
+    {
+        makeRequests(command.copyAsTransientQuery(), replicas);
     }
 
-    protected void makeDigestRequests(Iterable<InetAddressAndPort> endpoints)
+    protected void makeDigestRequests(ReplicaCollection<?> replicas)
     {
-        makeRequests(command.copyAsDigestQuery(), endpoints);
+        assert all(replicas, Replica::isFull);
+        // only send digest requests to full replicas, send data requests instead to the transient replicas
+        makeRequests(command.copyAsDigestQuery(), replicas);
     }
 
-    private void makeRequests(ReadCommand readCommand, Iterable<InetAddressAndPort> endpoints)
+    private void makeRequests(ReadCommand readCommand, ReplicaCollection<?> replicas)
     {
         boolean hasLocalEndpoint = false;
 
-        for (InetAddressAndPort endpoint : endpoints)
+        Preconditions.checkArgument(replicas.stream().allMatch(replica -> replica.isFull() || !readCommand.isDigestQuery()),
+                                    "Can not send digest requests to transient replicas");
+        for (Replica replica: replicas)
         {
-            if (StorageProxy.canDoLocalRequest(endpoint))
+            InetAddressAndPort endpoint = replica.endpoint();
+            if (replica.isLocal())
             {
                 hasLocalEndpoint = true;
                 continue;
@@ -134,7 +145,6 @@ public abstract class AbstractReadExecutor
 
             if (traceState != null)
                 traceState.trace("reading {} from {}", readCommand.isDigestQuery() ? "digest" : "data", endpoint);
-            logger.trace("reading {} from {}", readCommand.isDigestQuery() ? "digest" : "data", endpoint);
             MessageOut<ReadCommand> message = readCommand.createMessage();
             MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
         }
@@ -154,16 +164,16 @@ public abstract class AbstractReadExecutor
     public abstract void maybeTryAdditionalReplicas();
 
     /**
-     * Get the replicas involved in the [finished] request.
-     *
-     * @return target replicas + the extra replica, *IF* we speculated.
-     */
-    public abstract List<InetAddressAndPort> getContactedReplicas();
-
-    /**
      * send the initial set of requests
      */
-    public abstract void executeAsync();
+    public void executeAsync()
+    {
+        EndpointsForToken selected = replicaLayout().selected();
+        EndpointsForToken fullDataRequests = selected.filter(Replica::isFull, initialDataRequestCount);
+        makeFullDataRequests(fullDataRequests);
+        makeTransientDataRequests(selected.filter(Replica::isTransient));
+        makeDigestRequests(selected.filter(r -> r.isFull() && !fullDataRequests.contains(r)));
+    }
 
     /**
      * @return an executor appropriate for the configured speculative read policy
@@ -171,34 +181,33 @@ public abstract class AbstractReadExecutor
     public static AbstractReadExecutor getReadExecutor(SinglePartitionReadCommand command, ConsistencyLevel consistencyLevel, long queryStartNanoTime) throws UnavailableException
     {
         Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
-
-        List<InetAddressAndPort> allLiveReplicas = StorageProxy.getLiveSortedEndpoints(keyspace, command.partitionKey());
-        List<InetAddressAndPort> selectedReplicas = consistencyLevel.filterForQuery(keyspace, allLiveReplicas);
-
-        // Throw UAE early if we don't have enough replicas.
-        consistencyLevel.assureSufficientLiveNodes(keyspace, selectedReplicas);
-
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().id);
         SpeculativeRetryPolicy retry = cfs.metadata().params.speculativeRetry;
 
+        // Endpoints for Token
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forRead(keyspace, command.partitionKey().getToken(), consistencyLevel, retry);
+
         // Speculative retry is disabled *OR*
         // 11980: Disable speculative retry if using EACH_QUORUM in order to prevent miscounting DC responses
         if (retry.equals(NeverSpeculativeRetryPolicy.INSTANCE) || consistencyLevel == ConsistencyLevel.EACH_QUORUM)
-            return new NeverSpeculatingReadExecutor(keyspace, cfs, command, consistencyLevel, selectedReplicas, queryStartNanoTime, false);
+            // TODO Looks like we might want to move speculation into the replica layout, but that might be a story for post-4.0
+            return new NeverSpeculatingReadExecutor(cfs, command, replicaLayout, queryStartNanoTime, false);
 
         // There are simply no extra replicas to speculate.
         // Handle this separately so it can record failed attempts to speculate due to lack of replicas
-        if (selectedReplicas.size() == allLiveReplicas.size())
+        if (replicaLayout.selected().size() == replicaLayout.all().size())
         {
             boolean recordFailedSpeculation = consistencyLevel != ConsistencyLevel.ALL;
-            return new NeverSpeculatingReadExecutor(keyspace, cfs, command, consistencyLevel, selectedReplicas, queryStartNanoTime, recordFailedSpeculation);
+            return new NeverSpeculatingReadExecutor(cfs, command, replicaLayout, queryStartNanoTime, recordFailedSpeculation);
         }
 
-        selectedReplicas.add(allLiveReplicas.get(selectedReplicas.size()));
+        // If CL.ALL, upgrade to AlwaysSpeculating;
+        // If We are going to contact every node anyway, ask for 2 full data requests instead of 1, for redundancy
+        // (same amount of requests in total, but we turn 1 digest request into a full blown data request)
         if (retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE))
-            return new AlwaysSpeculatingReadExecutor(keyspace, cfs, command, consistencyLevel, selectedReplicas, queryStartNanoTime);
+            return new AlwaysSpeculatingReadExecutor(cfs, command, replicaLayout, queryStartNanoTime);
         else // PERCENTILE or CUSTOM.
-            return new SpeculatingReadExecutor(keyspace, cfs, command, consistencyLevel, selectedReplicas, queryStartNanoTime);
+            return new SpeculatingReadExecutor(cfs, command, replicaLayout, queryStartNanoTime);
     }
 
     /**
@@ -208,10 +217,15 @@ public abstract class AbstractReadExecutor
     boolean shouldSpeculateAndMaybeWait()
     {
         // no latency information, or we're overloaded
-        if (cfs.sampleLatencyNanos > TimeUnit.MILLISECONDS.toNanos(command.getTimeout()))
+        if (cfs.sampleReadLatencyNanos > TimeUnit.MILLISECONDS.toNanos(command.getTimeout()))
             return false;
 
-        return !handler.await(cfs.sampleLatencyNanos, TimeUnit.NANOSECONDS);
+        return !handler.await(cfs.sampleReadLatencyNanos, TimeUnit.NANOSECONDS);
+    }
+
+    ReplicaLayout.ForToken replicaLayout()
+    {
+        return replicaLayout;
     }
 
     void onReadTimeout() {}
@@ -223,78 +237,36 @@ public abstract class AbstractReadExecutor
          * log it is as a failure if it should have happened
          * but couldn't due to lack of replicas
          */
-        private final boolean recordFailedSpeculation;
-
-        NeverSpeculatingReadExecutor(Keyspace keyspace,
-                                     ColumnFamilyStore cfs,
-                                     ReadCommand command,
-                                     ConsistencyLevel consistencyLevel,
-                                     List<InetAddressAndPort> targetReplicas,
-                                     long queryStartNanoTime,
-                                     boolean recordFailedSpeculation)
-        {
-            super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
-            this.recordFailedSpeculation = recordFailedSpeculation;
-        }
+        private final boolean logFailedSpeculation;
 
-        public void executeAsync()
+        public NeverSpeculatingReadExecutor(ColumnFamilyStore cfs, ReadCommand command, ReplicaLayout.ForToken replicaLayout, long queryStartNanoTime, boolean logFailedSpeculation)
         {
-            makeDataRequests(targetReplicas.subList(0, 1));
-            if (targetReplicas.size() > 1)
-                makeDigestRequests(targetReplicas.subList(1, targetReplicas.size()));
+            super(cfs, command, replicaLayout, 1, queryStartNanoTime);
+            this.logFailedSpeculation = logFailedSpeculation;
         }
 
         public void maybeTryAdditionalReplicas()
         {
-            if (shouldSpeculateAndMaybeWait() && recordFailedSpeculation)
+            if (shouldSpeculateAndMaybeWait() && logFailedSpeculation)
             {
                 cfs.metric.speculativeInsufficientReplicas.inc();
             }
         }
-
-        public List<InetAddressAndPort> getContactedReplicas()
-        {
-            return targetReplicas;
-        }
     }
 
     static class SpeculatingReadExecutor extends AbstractReadExecutor
     {
         private volatile boolean speculated = false;
 
-        public SpeculatingReadExecutor(Keyspace keyspace,
-                                       ColumnFamilyStore cfs,
+        public SpeculatingReadExecutor(ColumnFamilyStore cfs,
                                        ReadCommand command,
-                                       ConsistencyLevel consistencyLevel,
-                                       List<InetAddressAndPort> targetReplicas,
+                                       ReplicaLayout.ForToken replicaLayout,
                                        long queryStartNanoTime)
         {
-            super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
-        }
-
-        public void executeAsync()
-        {
-            // if CL + RR result in covering all replicas, getReadExecutor forces AlwaysSpeculating.  So we know
-            // that the last replica in our list is "extra."
-            List<InetAddressAndPort> initialReplicas = targetReplicas.subList(0, targetReplicas.size() - 1);
-
-            if (handler.blockfor < initialReplicas.size())
-            {
-                // We're hitting additional targets for read repair.  Since our "extra" replica is the least-
-                // preferred by the snitch, we do an extra data read to start with against a replica more
-                // likely to reply; better to let RR fail than the entire query.
-                makeDataRequests(initialReplicas.subList(0, 2));
-                if (initialReplicas.size() > 2)
-                    makeDigestRequests(initialReplicas.subList(2, initialReplicas.size()));
-            }
-            else
-            {
-                // not doing read repair; all replies are important, so it doesn't matter which nodes we
-                // perform data reads against vs digest.
-                makeDataRequests(initialReplicas.subList(0, 1));
-                if (initialReplicas.size() > 1)
-                    makeDigestRequests(initialReplicas.subList(1, initialReplicas.size()));
-            }
+            // We're hitting additional targets for read repair (??).  Since our "extra" replica is the least-
+            // preferred by the snitch, we do an extra data read to start with against a replica more
+            // likely to reply; better to let RR fail than the entire query.
+            super(cfs, command, replicaLayout, replicaLayout.blockFor() < replicaLayout.selected().size() ? 2 : 1, queryStartNanoTime);
         }
 
         public void maybeTryAdditionalReplicas()
@@ -302,28 +274,43 @@ public abstract class AbstractReadExecutor
             if (shouldSpeculateAndMaybeWait())
             {
                 //Handle speculation stats first in case the callback fires immediately
-                speculated = true;
                 cfs.metric.speculativeRetries.inc();
-                // Could be waiting on the data, or on enough digests.
+                speculated = true;
+
                 ReadCommand retryCommand = command;
+                Replica extraReplica;
                 if (handler.resolver.isDataPresent())
-                    retryCommand = command.copyAsDigestQuery();
+                {
+                    extraReplica = tryFind(replicaLayout().all(),
+                            r -> !replicaLayout().selected().contains(r)).orNull();
+
+                    // we should only use a SpeculatingReadExecutor if we have an extra replica to speculate against
+                    assert extraReplica != null;
+
+                    retryCommand = extraReplica.isTransient()
+                            ? command.copyAsTransientQuery()
+                            : command.copyAsDigestQuery();
+                }
+                else
+                {
+                    extraReplica = tryFind(replicaLayout().all(),
+                            r -> r.isFull() && !replicaLayout().selected().contains(r)).orNull();
+                    if (extraReplica == null)
+                    {
+                        cfs.metric.speculativeInsufficientReplicas.inc();
+                        // cannot safely speculate a new data request, without more work - requests assumed to be
+                        // unique per endpoint, and we have no full nodes left to speculate against
+                        return;
+                    }
+                }
 
-                InetAddressAndPort extraReplica = Iterables.getLast(targetReplicas);
                 if (traceState != null)
                     traceState.trace("speculating read retry on {}", extraReplica);
                 logger.trace("speculating read retry on {}", extraReplica);
-                MessagingService.instance().sendRRWithFailure(retryCommand.createMessage(), extraReplica, handler);
+                MessagingService.instance().sendRRWithFailure(retryCommand.createMessage(), extraReplica.endpoint(), handler);
             }
         }
 
-        public List<InetAddressAndPort> getContactedReplicas()
-        {
-            return speculated
-                 ? targetReplicas
-                 : targetReplicas.subList(0, targetReplicas.size() - 1);
-        }
-
         @Override
         void onReadTimeout()
         {
@@ -336,14 +323,12 @@ public abstract class AbstractReadExecutor
 
     private static class AlwaysSpeculatingReadExecutor extends AbstractReadExecutor
     {
-        public AlwaysSpeculatingReadExecutor(Keyspace keyspace,
-                                             ColumnFamilyStore cfs,
+        public AlwaysSpeculatingReadExecutor(ColumnFamilyStore cfs,
                                              ReadCommand command,
-                                             ConsistencyLevel consistencyLevel,
-                                             List<InetAddressAndPort> targetReplicas,
+                                             ReplicaLayout.ForToken replicaLayout,
                                              long queryStartNanoTime)
         {
-            super(keyspace, cfs, command, consistencyLevel, targetReplicas, queryStartNanoTime);
+            super(cfs, command, replicaLayout, replicaLayout.selected().size() > 1 ? 2 : 1, queryStartNanoTime);
         }
 
         public void maybeTryAdditionalReplicas()
@@ -351,17 +336,10 @@ public abstract class AbstractReadExecutor
             // no-op
         }
 
-        public List<InetAddressAndPort> getContactedReplicas()
-        {
-            return targetReplicas;
-        }
-
         @Override
         public void executeAsync()
         {
-            makeDataRequests(targetReplicas.subList(0, targetReplicas.size() > 1 ? 2 : 1));
-            if (targetReplicas.size() > 2)
-                makeDigestRequests(targetReplicas.subList(2, targetReplicas.size()));
+            super.executeAsync();
             cfs.metric.speculativeRetries.inc();
         }
 
@@ -407,7 +385,7 @@ public abstract class AbstractReadExecutor
         else
         {
             Tracing.trace("Digest mismatch: Mismatch for key {}", getKey());
-            readRepair.startRepair(digestResolver, handler.endpoints, getContactedReplicas(), this::setResult);
+            readRepair.startRepair(digestResolver, this::setResult);
         }
     }
 
@@ -425,8 +403,7 @@ public abstract class AbstractReadExecutor
                 logger.trace("Timed out waiting on digest mismatch repair requests");
             // the caught exception here will have CL.ALL from the repair command,
             // not whatever CL the initial command was at (CASSANDRA-7947)
-            int blockFor = consistency.blockFor(Keyspace.open(command.metadata().keyspace));
-            throw new ReadTimeoutException(consistency, blockFor-1, blockFor, true);
+            throw new ReadTimeoutException(replicaLayout().consistencyLevel(), handler.blockfor - 1, handler.blockfor, true);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/DataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/DataResolver.java b/src/java/org/apache/cassandra/service/reads/DataResolver.java
index c0bff7a..9043e87 100644
--- a/src/java/org/apache/cassandra/service/reads/DataResolver.java
+++ b/src/java/org/apache/cassandra/service/reads/DataResolver.java
@@ -17,39 +17,55 @@
  */
 package org.apache.cassandra.service.reads;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
 
 import com.google.common.base.Joiner;
+import com.google.common.collect.Collections2;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.db.rows.RangeTombstoneMarker;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
-import org.apache.cassandra.db.transform.*;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.*;
+import org.apache.cassandra.db.transform.EmptyPartitionsDiscarder;
+import org.apache.cassandra.db.transform.Filter;
+import org.apache.cassandra.db.transform.FilteredPartitions;
+import org.apache.cassandra.db.transform.Transformation;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.Replicas;
+import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.reads.repair.ReadRepair;
 
-public class DataResolver extends ResponseResolver
+import static com.google.common.collect.Iterables.*;
+
+public class DataResolver<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends ResponseResolver<E, L>
 {
-    private final long queryStartNanoTime;
     private final boolean enforceStrictLiveness;
 
-    public DataResolver(Keyspace keyspace, ReadCommand command, ConsistencyLevel consistency, int maxResponseCount, long queryStartNanoTime, ReadRepair readRepair)
+    public DataResolver(ReadCommand command, L replicaLayout, ReadRepair<E, L> readRepair, long queryStartNanoTime)
     {
-        super(keyspace, command, consistency, readRepair, maxResponseCount);
-        this.queryStartNanoTime = queryStartNanoTime;
+        super(command, replicaLayout, readRepair, queryStartNanoTime);
         this.enforceStrictLiveness = command.metadata().enforceStrictLiveness();
     }
 
     public PartitionIterator getData()
     {
-        ReadResponse response = responses.iterator().next().payload;
+        ReadResponse response = responses.get(0).payload;
         return UnfilteredPartitionIterators.filter(response.makeIterator(command), command.nowInSec());
     }
 
@@ -63,15 +79,13 @@ public class DataResolver extends ResponseResolver
     {
         // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here
         // at the beginning of this method), so grab the response count once and use that through the method.
-        int count = responses.size();
-        List<UnfilteredPartitionIterator> iters = new ArrayList<>(count);
-        InetAddressAndPort[] sources = new InetAddressAndPort[count];
-        for (int i = 0; i < count; i++)
-        {
-            MessageIn<ReadResponse> msg = responses.get(i);
-            iters.add(msg.payload.makeIterator(command));
-            sources[i] = msg.from;
-        }
+        Collection<MessageIn<ReadResponse>> messages = responses.snapshot();
+        assert !any(messages, msg -> msg.payload.isDigestResponse());
+
+        E replicas = replicaLayout.all().keep(transform(messages, msg -> msg.from));
+        List<UnfilteredPartitionIterator> iters = new ArrayList<>(
+                Collections2.transform(messages, msg -> msg.payload.makeIterator(command)));
+        assert replicas.size() == iters.size();
 
         /*
          * Even though every response, individually, will honor the limit, it is possible that we will, after the merge,
@@ -86,18 +100,19 @@ public class DataResolver extends ResponseResolver
          *
          * See CASSANDRA-13747 for more details.
          */
-
         DataLimits.Counter mergedResultCounter =
             command.limits().newCounter(command.nowInSec(), true, command.selectsFullPartition(), enforceStrictLiveness);
 
-        UnfilteredPartitionIterator merged = mergeWithShortReadProtection(iters, sources, mergedResultCounter);
+        UnfilteredPartitionIterator merged = mergeWithShortReadProtection(iters,
+                                                                          replicaLayout.withSelected(replicas),
+                                                                          mergedResultCounter);
         FilteredPartitions filtered = FilteredPartitions.filter(merged, new Filter(command.nowInSec(), command.metadata().enforceStrictLiveness()));
         PartitionIterator counted = Transformation.apply(filtered, mergedResultCounter);
         return Transformation.apply(counted, new EmptyPartitionsDiscarder());
     }
 
     private UnfilteredPartitionIterator mergeWithShortReadProtection(List<UnfilteredPartitionIterator> results,
-                                                                     InetAddressAndPort[] sources,
+                                                                     L sources,
                                                                      DataLimits.Counter mergedResultCounter)
     {
         // If we have only one results, there is no read repair to do and we can't get short reads
@@ -110,17 +125,17 @@ public class DataResolver extends ResponseResolver
          */
         if (!command.limits().isUnlimited())
             for (int i = 0; i < results.size(); i++)
-                results.set(i, ShortReadProtection.extend(sources[i], results.get(i), command, mergedResultCounter, queryStartNanoTime, enforceStrictLiveness));
+                results.set(i, ShortReadProtection.extend(sources.selected().get(i), results.get(i), command, mergedResultCounter, queryStartNanoTime, enforceStrictLiveness));
 
         return UnfilteredPartitionIterators.merge(results, wrapMergeListener(readRepair.getMergeListener(sources), sources));
     }
 
     private String makeResponsesDebugString(DecoratedKey partitionKey)
     {
-        return Joiner.on(",\n").join(Iterables.transform(getMessages(), m -> m.from + " => " + m.payload.toDebugString(command, partitionKey)));
+        return Joiner.on(",\n").join(transform(getMessages().snapshot(), m -> m.from + " => " + m.payload.toDebugString(command, partitionKey)));
     }
 
-    private UnfilteredPartitionIterators.MergeListener wrapMergeListener(UnfilteredPartitionIterators.MergeListener partitionListener, InetAddressAndPort[] sources)
+    private UnfilteredPartitionIterators.MergeListener wrapMergeListener(UnfilteredPartitionIterators.MergeListener partitionListener, L sources)
     {
         return new UnfilteredPartitionIterators.MergeListener()
         {
@@ -144,8 +159,8 @@ public class DataResolver extends ResponseResolver
                             String details = String.format("Error merging partition level deletion on %s: merged=%s, versions=%s, sources={%s}, debug info:%n %s",
                                                            table,
                                                            mergedDeletion == null ? "null" : mergedDeletion.toString(),
-                                                           '[' + Joiner.on(", ").join(Iterables.transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString())) + ']',
-                                                           Arrays.toString(sources),
+                                                           '[' + Joiner.on(", ").join(transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString())) + ']',
+                                                           sources.selected(),
                                                            makeResponsesDebugString(partitionKey));
                             throw new AssertionError(details, e);
                         }
@@ -165,8 +180,8 @@ public class DataResolver extends ResponseResolver
                             String details = String.format("Error merging rows on %s: merged=%s, versions=%s, sources={%s}, debug info:%n %s",
                                                            table,
                                                            merged == null ? "null" : merged.toString(table),
-                                                           '[' + Joiner.on(", ").join(Iterables.transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString(table))) + ']',
-                                                           Arrays.toString(sources),
+                                                           '[' + Joiner.on(", ").join(transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString(table))) + ']',
+                                                           sources.selected(),
                                                            makeResponsesDebugString(partitionKey));
                             throw new AssertionError(details, e);
                         }
@@ -191,8 +206,8 @@ public class DataResolver extends ResponseResolver
                             String details = String.format("Error merging RTs on %s: merged=%s, versions=%s, sources={%s}, debug info:%n %s",
                                                            table,
                                                            merged == null ? "null" : merged.toString(table),
-                                                           '[' + Joiner.on(", ").join(Iterables.transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString(table))) + ']',
-                                                           Arrays.toString(sources),
+                                                           '[' + Joiner.on(", ").join(transform(Arrays.asList(versions), rt -> rt == null ? "null" : rt.toString(table))) + ']',
+                                                           sources.selected(),
                                                            makeResponsesDebugString(partitionKey));
                             throw new AssertionError(details, e);
                         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/DigestResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/DigestResolver.java b/src/java/org/apache/cassandra/service/reads/DigestResolver.java
index 897892f..c3eee43 100644
--- a/src/java/org/apache/cassandra/service/reads/DigestResolver.java
+++ b/src/java/org/apache/cassandra/service/reads/DigestResolver.java
@@ -18,25 +18,35 @@
 package org.apache.cassandra.service.reads;
 
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.service.reads.repair.NoopReadRepair;
 import org.apache.cassandra.service.reads.repair.ReadRepair;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class DigestResolver extends ResponseResolver
+import static com.google.common.collect.Iterables.any;
+
+public class DigestResolver<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends ResponseResolver<E, L>
 {
-    private volatile ReadResponse dataResponse;
+    private volatile MessageIn<ReadResponse> dataResponse;
 
-    public DigestResolver(Keyspace keyspace, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair, int maxResponseCount)
+    public DigestResolver(ReadCommand command, L replicas, ReadRepair<E, L> readRepair, long queryStartNanoTime)
     {
-        super(keyspace, command, consistency, readRepair, maxResponseCount);
+        super(command, replicas, readRepair, queryStartNanoTime);
         Preconditions.checkArgument(command instanceof SinglePartitionReadCommand,
                                     "DigestResolver can only be used with SinglePartitionReadCommand commands");
     }
@@ -45,14 +55,60 @@ public class DigestResolver extends ResponseResolver
     public void preprocess(MessageIn<ReadResponse> message)
     {
         super.preprocess(message);
-        if (dataResponse == null && !message.payload.isDigestResponse())
-            dataResponse = message.payload;
+        Replica replica = replicaLayout.getReplicaFor(message.from);
+        if (dataResponse == null && !message.payload.isDigestResponse() && replica.isFull())
+        {
+            dataResponse = message;
+        }
+        else if (replica.isTransient() && message.payload.isDigestResponse())
+        {
+            throw new IllegalStateException("digest response received from transient replica");
+        }
+    }
+
+    @VisibleForTesting
+    public boolean hasTransientResponse()
+    {
+        return hasTransientResponse(responses.snapshot());
+    }
+
+    private boolean hasTransientResponse(Collection<MessageIn<ReadResponse>> responses)
+    {
+        return any(responses,
+                msg -> !msg.payload.isDigestResponse()
+                        && replicaLayout.getReplicaFor(msg.from).isTransient());
     }
 
     public PartitionIterator getData()
     {
         assert isDataPresent();
-        return UnfilteredPartitionIterators.filter(dataResponse.makeIterator(command), command.nowInSec());
+
+        Collection<MessageIn<ReadResponse>> responses = this.responses.snapshot();
+
+        if (!hasTransientResponse(responses))
+        {
+            return UnfilteredPartitionIterators.filter(dataResponse.payload.makeIterator(command), command.nowInSec());
+        }
+        else
+        {
+            // This path can be triggered only if we've got responses from full replicas and they match, but
+            // transient replica response still contains data, which needs to be reconciled.
+            DataResolver<E, L> dataResolver = new DataResolver<>(command,
+                                                                 replicaLayout,
+                                                                 (ReadRepair<E, L>) NoopReadRepair.instance,
+                                                                 queryStartNanoTime);
+
+            dataResolver.preprocess(dataResponse);
+            // Forward differences to all full nodes
+            for (MessageIn<ReadResponse> response : responses)
+            {
+                Replica replica = replicaLayout.getReplicaFor(response.from);
+                if (replica.isTransient())
+                    dataResolver.preprocess(response);
+            }
+
+            return dataResolver.resolve();
+        }
     }
 
     public boolean responsesMatch()
@@ -61,11 +117,12 @@ public class DigestResolver extends ResponseResolver
 
         // validate digests against each other; return false immediately on mismatch.
         ByteBuffer digest = null;
-        for (MessageIn<ReadResponse> message : responses)
+        for (MessageIn<ReadResponse> message : responses.snapshot())
         {
-            ReadResponse response = message.payload;
+            if (replicaLayout.getReplicaFor(message.from).isTransient())
+                continue;
 
-            ByteBuffer newDigest = response.digest(command);
+            ByteBuffer newDigest = message.payload.digest(command);
             if (digest == null)
                 digest = newDigest;
             else if (!digest.equals(newDigest))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/ReadCallback.java b/src/java/org/apache/cassandra/service/reads/ReadCallback.java
index 537e684..3d39377 100644
--- a/src/java/org/apache/cassandra/service/reads/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/reads/ReadCallback.java
@@ -18,42 +18,43 @@
 package org.apache.cassandra.service.reads;
 
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.RequestFailureReason;
 import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
-public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
+public class ReadCallback<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> implements IAsyncCallbackWithFailure<ReadResponse>
 {
     protected static final Logger logger = LoggerFactory.getLogger( ReadCallback.class );
 
     public final ResponseResolver resolver;
     final SimpleCondition condition = new SimpleCondition();
     private final long queryStartNanoTime;
+    // TODO: move to replica layout as well?
     final int blockfor;
-    final List<InetAddressAndPort> endpoints;
+    final L replicaLayout;
     private final ReadCommand command;
-    private final ConsistencyLevel consistencyLevel;
     private static final AtomicIntegerFieldUpdater<ReadCallback> recievedUpdater
             = AtomicIntegerFieldUpdater.newUpdater(ReadCallback.class, "received");
     private volatile int received = 0;
@@ -62,37 +63,19 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     private volatile int failures = 0;
     private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint;
 
-    private final Keyspace keyspace; // TODO push this into ConsistencyLevel?
-
-    /**
-     * Constructor when response count has to be calculated and blocked for.
-     */
-    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, ReadCommand command, List<InetAddressAndPort> filteredEndpoints, long queryStartNanoTime)
-    {
-        this(resolver,
-             consistencyLevel,
-             consistencyLevel.blockFor(Keyspace.open(command.metadata().keyspace)),
-             command,
-             Keyspace.open(command.metadata().keyspace),
-             filteredEndpoints,
-             queryStartNanoTime);
-    }
-
-    public ReadCallback(ResponseResolver resolver, ConsistencyLevel consistencyLevel, int blockfor, ReadCommand command, Keyspace keyspace, List<InetAddressAndPort> endpoints, long queryStartNanoTime)
+    public ReadCallback(ResponseResolver resolver, int blockfor, ReadCommand command, L replicaLayout, long queryStartNanoTime)
     {
         this.command = command;
-        this.keyspace = keyspace;
         this.blockfor = blockfor;
-        this.consistencyLevel = consistencyLevel;
         this.resolver = resolver;
         this.queryStartNanoTime = queryStartNanoTime;
-        this.endpoints = endpoints;
+        this.replicaLayout = replicaLayout;
         this.failureReasonByEndpoint = new ConcurrentHashMap<>();
         // we don't support read repair (or rapid read protection) for range scans yet (CASSANDRA-6897)
-        assert !(command instanceof PartitionRangeReadCommand) || blockfor >= endpoints.size();
+        assert !(command instanceof PartitionRangeReadCommand) || blockfor >= replicaLayout.selected().size();
 
         if (logger.isTraceEnabled())
-            logger.trace("Blockfor is {}; setting up requests to {}", blockfor, StringUtils.join(this.endpoints, ","));
+            logger.trace("Blockfor is {}; setting up requests to {}", blockfor, this.replicaLayout);
     }
 
     public boolean await(long timePastStart, TimeUnit unit)
@@ -111,7 +94,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     public void awaitResults() throws ReadFailureException, ReadTimeoutException
     {
         boolean signaled = await(command.getTimeout(), TimeUnit.MILLISECONDS);
-        boolean failed = blockfor + failures > endpoints.size();
+        boolean failed = failures > 0 && blockfor + failures > replicaLayout.selected().size();
         if (signaled && !failed)
             return;
 
@@ -128,8 +111,8 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
 
         // Same as for writes, see AbstractWriteResponseHandler
         throw failed
-            ? new ReadFailureException(consistencyLevel, received, blockfor, resolver.isDataPresent(), failureReasonByEndpoint)
-            : new ReadTimeoutException(consistencyLevel, received, blockfor, resolver.isDataPresent());
+            ? new ReadFailureException(replicaLayout.consistencyLevel(), received, blockfor, resolver.isDataPresent(), failureReasonByEndpoint)
+            : new ReadTimeoutException(replicaLayout.consistencyLevel(), received, blockfor, resolver.isDataPresent());
     }
 
     public int blockFor()
@@ -153,9 +136,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
      */
     private boolean waitingFor(InetAddressAndPort from)
     {
-        return consistencyLevel.isDatacenterLocal()
-             ? DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(from))
-             : true;
+        return !replicaLayout.consistencyLevel().isDatacenterLocal() || DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(from));
     }
 
     /**
@@ -178,7 +159,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
 
     public void assureSufficientLiveNodes() throws UnavailableException
     {
-        consistencyLevel.assureSufficientLiveNodes(keyspace, endpoints);
+        replicaLayout.consistencyLevel().assureSufficientLiveNodesForRead(replicaLayout.keyspace(), replicaLayout.selected());
     }
 
     public boolean isLatencyForSnitch()
@@ -195,7 +176,7 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
 
         failureReasonByEndpoint.put(from, failureReason);
 
-        if (blockfor + n > endpoints.size())
+        if (blockfor + n > replicaLayout.selected().size())
             condition.signalAll();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/ResponseResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/ResponseResolver.java b/src/java/org/apache/cassandra/service/reads/ResponseResolver.java
index f4f00a2..e306b4d 100644
--- a/src/java/org/apache/cassandra/service/reads/ResponseResolver.java
+++ b/src/java/org/apache/cassandra/service/reads/ResponseResolver.java
@@ -20,37 +20,49 @@ package org.apache.cassandra.service.reads;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.service.reads.repair.ReadRepair;
 import org.apache.cassandra.utils.concurrent.Accumulator;
 
-public abstract class ResponseResolver
+public abstract class ResponseResolver<E extends Endpoints<E>, L extends ReplicaLayout<E, L>>
 {
     protected static final Logger logger = LoggerFactory.getLogger(ResponseResolver.class);
 
-    protected final Keyspace keyspace;
     protected final ReadCommand command;
-    protected final ConsistencyLevel consistency;
-    protected final ReadRepair readRepair;
+    protected final L replicaLayout;
+    protected final ReadRepair<E, L> readRepair;
 
     // Accumulator gives us non-blocking thread-safety with optimal algorithmic constraints
     protected final Accumulator<MessageIn<ReadResponse>> responses;
+    protected final long queryStartNanoTime;
 
-    public ResponseResolver(Keyspace keyspace, ReadCommand command, ConsistencyLevel consistency, ReadRepair readRepair, int maxResponseCount)
+    public ResponseResolver(ReadCommand command, L replicaLayout, ReadRepair<E, L> readRepair, long queryStartNanoTime)
     {
-        this.keyspace = keyspace;
         this.command = command;
-        this.consistency = consistency;
+        this.replicaLayout = replicaLayout;
         this.readRepair = readRepair;
-        this.responses = new Accumulator<>(maxResponseCount);
+        // TODO: calculate max possible replicas for the query (e.g. local dc queries won't contact remotes)
+        this.responses = new Accumulator<>(replicaLayout.all().size());
+        this.queryStartNanoTime = queryStartNanoTime;
     }
 
     public abstract boolean isDataPresent();
 
     public void preprocess(MessageIn<ReadResponse> message)
     {
-        responses.add(message);
+        try
+        {
+            responses.add(message);
+        }
+        catch (IllegalStateException e)
+        {
+            logger.error("Encountered error while trying to preprocess the message {}: %s in command {}, replicas: {}", message, command, readRepair, replicaLayout.consistencyLevel(), replicaLayout.selected());
+            throw e;
+        }
     }
 
     public Accumulator<MessageIn<ReadResponse>> getMessages()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java b/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java
index d4e8957..580b790 100644
--- a/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java
+++ b/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java
@@ -18,12 +18,13 @@
 
 package org.apache.cassandra.service.reads;
 
-import java.util.Collections;
+import org.apache.cassandra.locator.Endpoints;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DataRange;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
@@ -39,7 +40,8 @@ import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.ExcludingBounds;
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.reads.repair.NoopReadRepair;
 import org.apache.cassandra.service.StorageProxy;
@@ -47,8 +49,9 @@ import org.apache.cassandra.tracing.Tracing;
 
 public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowIterator> implements MorePartitions<UnfilteredPartitionIterator>
 {
+    private static final Logger logger = LoggerFactory.getLogger(ShortReadPartitionsProtection.class);
     private final ReadCommand command;
-    private final InetAddressAndPort source;
+    private final Replica source;
 
     private final DataLimits.Counter singleResultCounter; // unmerged per-source counter
     private final DataLimits.Counter mergedResultCounter; // merged end-result counter
@@ -59,7 +62,7 @@ public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowI
 
     private final long queryStartNanoTime;
 
-    public ShortReadPartitionsProtection(ReadCommand command, InetAddressAndPort source,
+    public ShortReadPartitionsProtection(ReadCommand command, Replica source,
                                          DataLimits.Counter singleResultCounter,
                                          DataLimits.Counter mergedResultCounter,
                                          long queryStartNanoTime)
@@ -84,9 +87,10 @@ public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowI
          * If we don't apply the transformation *after* extending the partition with MoreRows,
          * applyToRow() method of protection will not be called on the first row of the new extension iterator.
          */
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forSingleReplica(Keyspace.open(command.metadata().keyspace), partition.partitionKey().getToken(), source);
         ShortReadRowsProtection protection = new ShortReadRowsProtection(partition.partitionKey(),
                                                                          command, source,
-                                                                         this::executeReadCommand,
+                                                                         (cmd) -> executeReadCommand(cmd, replicaLayout),
                                                                          singleResultCounter,
                                                                          mergedResultCounter);
         return Transformation.apply(MoreRows.extend(partition, protection), protection);
@@ -140,9 +144,9 @@ public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowI
 
         ColumnFamilyStore.metricsFor(command.metadata().id).shortReadProtectionRequests.mark();
         Tracing.trace("Requesting {} extra rows from {} for short read protection", toQuery, source);
+        logger.info("Requesting {} extra rows from {} for short read protection", toQuery, source);
 
-        PartitionRangeReadCommand cmd = makeFetchAdditionalPartitionReadCommand(toQuery);
-        return executeReadCommand(cmd);
+        return makeAndExecuteFetchAdditionalPartitionReadCommand(toQuery);
     }
 
     // Counts the number of rows for regular queries and the number of groups for GROUP BY queries
@@ -153,7 +157,7 @@ public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowI
                : counter.counted();
     }
 
-    private PartitionRangeReadCommand makeFetchAdditionalPartitionReadCommand(int toQuery)
+    private UnfilteredPartitionIterator makeAndExecuteFetchAdditionalPartitionReadCommand(int toQuery)
     {
         PartitionRangeReadCommand cmd = (PartitionRangeReadCommand) command;
 
@@ -165,19 +169,19 @@ public class ShortReadPartitionsProtection extends Transformation<UnfilteredRowI
                                                       : new ExcludingBounds<>(lastPartitionKey, bounds.right);
         DataRange newDataRange = cmd.dataRange().forSubRange(newBounds);
 
-        return cmd.withUpdatedLimitsAndDataRange(newLimits, newDataRange);
+        ReplicaLayout.ForRange replicaLayout = ReplicaLayout.forSingleReplica(Keyspace.open(command.metadata().keyspace), cmd.dataRange().keyRange(), source);
+        return executeReadCommand(cmd.withUpdatedLimitsAndDataRange(newLimits, newDataRange), replicaLayout);
     }
 
-    private UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd)
+    private <E extends Endpoints<E>, L extends ReplicaLayout<E, L>> UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, L replicaLayout)
     {
-        Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
-        DataResolver resolver = new DataResolver(keyspace, cmd, ConsistencyLevel.ONE, 1, queryStartNanoTime, NoopReadRepair.instance);
-        ReadCallback handler = new ReadCallback(resolver, ConsistencyLevel.ONE, cmd, Collections.singletonList(source), queryStartNanoTime);
+        DataResolver<E, L> resolver = new DataResolver<>(cmd, replicaLayout, (NoopReadRepair<E, L>)NoopReadRepair.instance, queryStartNanoTime);
+        ReadCallback<E, L> handler = new ReadCallback<>(resolver, replicaLayout.consistencyLevel().blockFor(replicaLayout.keyspace()), cmd, replicaLayout, queryStartNanoTime);
 
-        if (StorageProxy.canDoLocalRequest(source))
+        if (source.isLocal())
             StageManager.getStage(Stage.READ).maybeExecuteImmediately(new StorageProxy.LocalReadRunnable(cmd, handler));
         else
-            MessagingService.instance().sendRRWithFailure(cmd.createMessage(), source, handler);
+            MessagingService.instance().sendRRWithFailure(cmd.createMessage(), source.endpoint(), handler);
 
         // We don't call handler.get() because we want to preserve tombstones since we're still in the middle of merging node results.
         handler.awaitResults();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java b/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
index f603e9b..ef1d45b 100644
--- a/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
+++ b/src/java/org/apache/cassandra/service/reads/ShortReadProtection.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.transform.MorePartitions;
 import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 
 /**
  * We have a potential short read if the result from a given node contains the requested number of rows
@@ -40,7 +41,7 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 public class ShortReadProtection
 {
     @SuppressWarnings("resource")
-    public static UnfilteredPartitionIterator extend(InetAddressAndPort source, UnfilteredPartitionIterator partitions,
+    public static UnfilteredPartitionIterator extend(Replica source, UnfilteredPartitionIterator partitions,
                                                      ReadCommand command, DataLimits.Counter mergedResultCounter,
                                                      long queryStartNanoTime, boolean enforceStrictLiveness)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/ShortReadRowsProtection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/ShortReadRowsProtection.java b/src/java/org/apache/cassandra/service/reads/ShortReadRowsProtection.java
index 6b1da0b..8dc7fc7 100644
--- a/src/java/org/apache/cassandra/service/reads/ShortReadRowsProtection.java
+++ b/src/java/org/apache/cassandra/service/reads/ShortReadRowsProtection.java
@@ -33,14 +33,14 @@ import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.transform.MoreRows;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.tracing.Tracing;
 
 class ShortReadRowsProtection extends Transformation implements MoreRows<UnfilteredRowIterator>
 {
     private final ReadCommand command;
-    private final InetAddressAndPort source;
+    private final Replica source;
     private final DataLimits.Counter singleResultCounter; // unmerged per-source counter
     private final DataLimits.Counter mergedResultCounter; // merged end-result counter
     private final Function<ReadCommand, UnfilteredPartitionIterator> commandExecutor;
@@ -53,7 +53,7 @@ class ShortReadRowsProtection extends Transformation implements MoreRows<Unfilte
     private int lastFetched = 0; // # rows returned by last attempt to get more (or by the original read command)
     private int lastQueried = 0; // # extra rows requested from the replica last time
 
-    ShortReadRowsProtection(DecoratedKey partitionKey, ReadCommand command, InetAddressAndPort source,
+    ShortReadRowsProtection(DecoratedKey partitionKey, ReadCommand command, Replica source,
                             Function<ReadCommand, UnfilteredPartitionIterator> commandExecutor,
                             DataLimits.Counter singleResultCounter, DataLimits.Counter mergedResultCounter)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java
index 7e3f0ae..30dea74 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java
@@ -18,29 +18,23 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import java.util.List;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
-import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
 
 import com.codahale.metrics.Meter;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.Endpoints;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.reads.DataResolver;
@@ -48,11 +42,11 @@ import org.apache.cassandra.service.reads.DigestResolver;
 import org.apache.cassandra.service.reads.ReadCallback;
 import org.apache.cassandra.tracing.Tracing;
 
-public abstract class AbstractReadRepair implements ReadRepair
+public abstract class AbstractReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> implements ReadRepair<E, L>
 {
     protected final ReadCommand command;
     protected final long queryStartNanoTime;
-    protected final ConsistencyLevel consistency;
+    protected final L replicaLayout;
     protected final ColumnFamilyStore cfs;
 
     private volatile DigestRepair digestRepair = null;
@@ -62,41 +56,25 @@ public abstract class AbstractReadRepair implements ReadRepair
         private final DataResolver dataResolver;
         private final ReadCallback readCallback;
         private final Consumer<PartitionIterator> resultConsumer;
-        private final List<InetAddressAndPort> initialContacts;
 
-        public DigestRepair(DataResolver dataResolver, ReadCallback readCallback, Consumer<PartitionIterator> resultConsumer, List<InetAddressAndPort> initialContacts)
+        public DigestRepair(DataResolver dataResolver, ReadCallback readCallback, Consumer<PartitionIterator> resultConsumer)
         {
             this.dataResolver = dataResolver;
             this.readCallback = readCallback;
             this.resultConsumer = resultConsumer;
-            this.initialContacts = initialContacts;
         }
     }
 
     public AbstractReadRepair(ReadCommand command,
-                              long queryStartNanoTime,
-                              ConsistencyLevel consistency)
+                              L replicaLayout,
+                              long queryStartNanoTime)
     {
         this.command = command;
         this.queryStartNanoTime = queryStartNanoTime;
-        this.consistency = consistency;
+        this.replicaLayout = replicaLayout;
         this.cfs = Keyspace.openAndGetStore(command.metadata());
     }
 
-    private int getMaxResponses()
-    {
-        AbstractReplicationStrategy strategy = cfs.keyspace.getReplicationStrategy();
-        if (consistency.isDatacenterLocal() && strategy instanceof NetworkTopologyStrategy)
-        {
-            NetworkTopologyStrategy nts = (NetworkTopologyStrategy) strategy;
-            return nts.getReplicationFactor(DatabaseDescriptor.getLocalDataCenter());
-        }
-        else
-        {
-            return strategy.getReplicationFactor();
-        }
-    }
-
     void sendReadCommand(InetAddressAndPort to, ReadCallback readCallback)
     {
         MessagingService.instance().sendRRWithFailure(command.createMessage(), to, readCallback);
@@ -105,24 +83,23 @@ public abstract class AbstractReadRepair implements ReadRepair
     abstract Meter getRepairMeter();
 
     // digestResolver isn't used here because we resend read requests to all participants
-    public void startRepair(DigestResolver digestResolver, List<InetAddressAndPort> allEndpoints, List<InetAddressAndPort> contactedEndpoints, Consumer<PartitionIterator> resultConsumer)
+    public void startRepair(DigestResolver<E, L> digestResolver, Consumer<PartitionIterator> resultConsumer)
     {
         getRepairMeter().mark();
 
         // Do a full data read to resolve the correct response (and repair node that need be)
-        Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
-        DataResolver resolver = new DataResolver(keyspace, command, ConsistencyLevel.ALL, getMaxResponses(), queryStartNanoTime, this);
-        ReadCallback readCallback = new ReadCallback(resolver, ConsistencyLevel.ALL, consistency.blockFor(cfs.keyspace), command,
-                                                     keyspace, allEndpoints, queryStartNanoTime);
+        DataResolver<E, L> resolver = new DataResolver<>(command, replicaLayout, this, queryStartNanoTime);
+        ReadCallback<E, L> readCallback = new ReadCallback<>(resolver, replicaLayout.consistencyLevel().blockFor(cfs.keyspace),
+                                                             command, replicaLayout, queryStartNanoTime);
 
-        digestRepair = new DigestRepair(resolver, readCallback, resultConsumer, contactedEndpoints);
+        digestRepair = new DigestRepair(resolver, readCallback, resultConsumer);
 
-        for (InetAddressAndPort endpoint : contactedEndpoints)
+        for (Replica replica : replicaLayout.selected())
         {
-            Tracing.trace("Enqueuing full data read to {}", endpoint);
-            sendReadCommand(endpoint, readCallback);
+            Tracing.trace("Enqueuing full data read to {}", replica);
+            sendReadCommand(replica.endpoint(), readCallback);
         }
-        ReadRepairDiagnostics.startRepair(this, contactedEndpoints, digestResolver, allEndpoints);
+        ReadRepairDiagnostics.startRepair(this, replicaLayout.selected().endpoints(), digestResolver, replicaLayout.all().endpoints());
     }
 
     public void awaitReads() throws ReadTimeoutException
@@ -137,15 +114,11 @@ public abstract class AbstractReadRepair implements ReadRepair
 
     private boolean shouldSpeculate()
     {
+        ConsistencyLevel consistency = replicaLayout.consistencyLevel();
         ConsistencyLevel speculativeCL = consistency.isDatacenterLocal() ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM;
         return  consistency != ConsistencyLevel.EACH_QUORUM
                 && consistency.satisfies(speculativeCL, cfs.keyspace)
-                && cfs.sampleLatencyNanos <= TimeUnit.MILLISECONDS.toNanos(command.getTimeout());
-    }
-
-    Iterable<InetAddressAndPort> getCandidatesForToken(Token token)
-    {
-        return BlockingReadRepairs.getCandidateEndpoints(cfs.keyspace, token, consistency);
+                && cfs.sampleReadLatencyNanos <= TimeUnit.MILLISECONDS.toNanos(command.getTimeout());
     }
 
     public void maybeSendAdditionalReads()
@@ -156,20 +129,17 @@ public abstract class AbstractReadRepair implements ReadRepair
         if (repair == null)
             return;
 
-        if (shouldSpeculate() && !repair.readCallback.await(cfs.sampleLatencyNanos, TimeUnit.NANOSECONDS))
+        if (shouldSpeculate() && !repair.readCallback.await(cfs.sampleReadLatencyNanos, TimeUnit.NANOSECONDS))
         {
-            Set<InetAddressAndPort> contacted = Sets.newHashSet(repair.initialContacts);
-            Token replicaToken = ((SinglePartitionReadCommand) command).partitionKey().getToken();
-            Iterable<InetAddressAndPort> candidates = getCandidatesForToken(replicaToken);
-
-            Optional<InetAddressAndPort> endpoint = Iterables.tryFind(candidates, e -> !contacted.contains(e));
-            if (endpoint.isPresent())
-            {
-                Tracing.trace("Enqueuing speculative full data read to {}", endpoint);
-                sendReadCommand(endpoint.get(), repair.readCallback);
-                ReadRepairMetrics.speculatedRead.mark();
-                ReadRepairDiagnostics.speculatedRead(this, endpoint.get(), candidates);
-            }
+            L uncontacted = replicaLayout.forNaturalUncontacted();
+            if (uncontacted.selected().isEmpty())
+                return;
+
+            Replica replica = uncontacted.selected().iterator().next();
+            Tracing.trace("Enqueuing speculative full data read to {}", replica);
+            sendReadCommand(replica.endpoint(), repair.readCallback);
+            ReadRepairMetrics.speculatedRead.mark();
+            ReadRepairDiagnostics.speculatedRead(this, replica.endpoint(), uncontacted.all().endpoints());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
index 8d69bef..54af2cf 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java
@@ -20,16 +20,14 @@ package org.apache.cassandra.service.reads.repair;
 
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
+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 com.google.common.util.concurrent.AbstractFuture;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -38,7 +36,11 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.locator.Endpoints;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.Replicas;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.net.IAsyncCallback;
 import org.apache.cassandra.net.MessageIn;
@@ -47,33 +49,29 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.tracing.Tracing;
 
-public class BlockingPartitionRepair extends AbstractFuture<Object> implements IAsyncCallback<Object>
+public class BlockingPartitionRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends AbstractFuture<Object> implements IAsyncCallback<Object>
 {
-    private final Keyspace keyspace;
     private final DecoratedKey key;
-    private final ConsistencyLevel consistency;
-    private final InetAddressAndPort[] participants;
-    private final ConcurrentMap<InetAddressAndPort, Mutation> pendingRepairs;
+    private final L replicaLayout;
+    private final Map<Replica, Mutation> pendingRepairs;
     private final CountDownLatch latch;
 
     private volatile long mutationsSentTime;
 
-    public BlockingPartitionRepair(Keyspace keyspace, DecoratedKey key, ConsistencyLevel consistency, Map<InetAddressAndPort, Mutation> repairs, int maxBlockFor, InetAddressAndPort[] participants)
+    public BlockingPartitionRepair(DecoratedKey key, Map<Replica, Mutation> repairs, int maxBlockFor, L replicaLayout)
     {
-        this.keyspace = keyspace;
         this.key = key;
-        this.consistency = consistency;
         this.pendingRepairs = new ConcurrentHashMap<>(repairs);
-        this.participants = participants;
+        this.replicaLayout = replicaLayout;
 
         // here we remove empty repair mutations from the block for total, since
         // we're not sending them mutations
         int blockFor = maxBlockFor;
-        for (InetAddressAndPort participant: participants)
+        for (Replica participant: replicaLayout.selected())
         {
             // remote dcs can sometimes get involved in dc-local reads. We want to repair
             // them if they do, but they shouldn't interfere with blocking the client read.
-            if (!repairs.containsKey(participant) && shouldBlockOn(participant))
+            if (!repairs.containsKey(participant) && shouldBlockOn(participant.endpoint()))
                 blockFor--;
         }
 
@@ -99,7 +97,7 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
 
     private boolean shouldBlockOn(InetAddressAndPort endpoint)
     {
-        return !consistency.isDatacenterLocal() || isLocal(endpoint);
+        return !replicaLayout.consistencyLevel().isDatacenterLocal() || isLocal(endpoint);
     }
 
     @VisibleForTesting
@@ -107,7 +105,7 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
     {
         if (shouldBlockOn(from))
         {
-            pendingRepairs.remove(from);
+            pendingRepairs.remove(replicaLayout.getReplicaFor(from));
             latch.countDown();
         }
     }
@@ -148,20 +146,23 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
     public void sendInitialRepairs()
     {
         mutationsSentTime = System.nanoTime();
-        for (Map.Entry<InetAddressAndPort, Mutation> entry: pendingRepairs.entrySet())
+        Replicas.assertFull(pendingRepairs.keySet());
+
+        for (Map.Entry<Replica, Mutation> entry: pendingRepairs.entrySet())
         {
-            InetAddressAndPort destination = entry.getKey();
+            Replica destination = entry.getKey();
+            Preconditions.checkArgument(destination.isFull(), "Can't send repairs to transient replicas: %s", destination);
             Mutation mutation = entry.getValue();
             TableId tableId = extractUpdate(mutation).metadata().id;
 
             Tracing.trace("Sending read-repair-mutation to {}", destination);
             // use a separate verb here to avoid writing hints on timeouts
-            sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), destination);
+            sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), destination.endpoint());
             ColumnFamilyStore.metricsFor(tableId).readRepairRequests.mark();
 
-            if (!shouldBlockOn(destination))
+            if (!shouldBlockOn(destination.endpoint()))
                 pendingRepairs.remove(destination);
-            ReadRepairDiagnostics.sendInitialRepair(this, destination, mutation);
+            ReadRepairDiagnostics.sendInitialRepair(this, destination.endpoint(), mutation);
         }
     }
 
@@ -197,9 +198,8 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
         if (awaitRepairs(timeout, timeoutUnit))
             return;
 
-        Set<InetAddressAndPort> exclude = Sets.newHashSet(participants);
-        Iterable<InetAddressAndPort> candidates = Iterables.filter(getCandidateEndpoints(), e -> !exclude.contains(e));
-        if (Iterables.isEmpty(candidates))
+        L newCandidates = replicaLayout.forNaturalUncontacted();
+        if (newCandidates.selected().isEmpty())
             return;
 
         PartitionUpdate update = mergeUnackedUpdates();
@@ -212,34 +212,34 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
 
         Mutation[] versionedMutations = new Mutation[msgVersionIdx(MessagingService.current_version) + 1];
 
-        for (InetAddressAndPort endpoint: candidates)
+        for (Replica replica : newCandidates.selected())
         {
-            int versionIdx = msgVersionIdx(MessagingService.instance().getVersion(endpoint));
+            int versionIdx = msgVersionIdx(MessagingService.instance().getVersion(replica.endpoint()));
 
             Mutation mutation = versionedMutations[versionIdx];
 
             if (mutation == null)
             {
-                mutation = BlockingReadRepairs.createRepairMutation(update, consistency, endpoint, true);
+                mutation = BlockingReadRepairs.createRepairMutation(update, replicaLayout.consistencyLevel(), replica.endpoint(), true);
                 versionedMutations[versionIdx] = mutation;
             }
 
             if (mutation == null)
             {
                 // the mutation is too large to send.
-                ReadRepairDiagnostics.speculatedWriteOversized(this, endpoint);
+                ReadRepairDiagnostics.speculatedWriteOversized(this, replica.endpoint());
                 continue;
             }
 
-            Tracing.trace("Sending speculative read-repair-mutation to {}", endpoint);
-            sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), endpoint);
-            ReadRepairDiagnostics.speculatedWrite(this, endpoint, mutation);
+            Tracing.trace("Sending speculative read-repair-mutation to {}", replica);
+            sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), replica.endpoint());
+            ReadRepairDiagnostics.speculatedWrite(this, replica.endpoint(), mutation);
         }
     }
 
     Keyspace getKeyspace()
     {
-        return keyspace;
+        return replicaLayout.keyspace();
     }
 
     DecoratedKey getKey()
@@ -249,13 +249,6 @@ public class BlockingPartitionRepair extends AbstractFuture<Object> implements I
 
     ConsistencyLevel getConsistency()
     {
-        return consistency;
-    }
-
-    @VisibleForTesting
-    protected Iterable<InetAddressAndPort> getCandidateEndpoints()
-    {
-        return BlockingReadRepairs.getCandidateEndpoints(keyspace, key.getToken(), consistency);
+        return replicaLayout.consistencyLevel();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java
index e46372e..402aed0 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java
@@ -23,18 +23,19 @@ import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.locator.Endpoints;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.Meter;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.tracing.Tracing;
 
@@ -43,20 +44,22 @@ import org.apache.cassandra.tracing.Tracing;
  *  updates have been written to nodes needing correction. Breaks write
  *  atomicity in some situations
  */
-public class BlockingReadRepair extends AbstractReadRepair
+public class BlockingReadRepair<E extends Endpoints<E>, L extends ReplicaLayout<E, L>> extends AbstractReadRepair<E, L>
 {
     private static final Logger logger = LoggerFactory.getLogger(BlockingReadRepair.class);
 
     protected final Queue<BlockingPartitionRepair> repairs = new ConcurrentLinkedQueue<>();
+    private final int blockFor;
 
-    public BlockingReadRepair(ReadCommand command, long queryStartNanoTime, ConsistencyLevel consistency)
+    BlockingReadRepair(ReadCommand command, L replicaLayout, long queryStartNanoTime)
     {
-        super(command, queryStartNanoTime, consistency);
+        super(command, replicaLayout, queryStartNanoTime);
+        this.blockFor = replicaLayout.consistencyLevel().blockFor(cfs.keyspace);
     }
 
-    public UnfilteredPartitionIterators.MergeListener getMergeListener(InetAddressAndPort[] endpoints)
+    public UnfilteredPartitionIterators.MergeListener getMergeListener(L replicaLayout)
     {
-        return new PartitionIteratorMergeListener(endpoints, command, consistency, this);
+        return new PartitionIteratorMergeListener(replicaLayout, command, this.replicaLayout.consistencyLevel(), this);
     }
 
     @Override
@@ -70,7 +73,7 @@ public class BlockingReadRepair extends AbstractReadRepair
     {
         for (BlockingPartitionRepair repair: repairs)
         {
-            repair.maybeSendAdditionalWrites(cfs.sampleLatencyNanos, TimeUnit.NANOSECONDS);
+            repair.maybeSendAdditionalWrites(cfs.transientWriteLatencyNanos, TimeUnit.NANOSECONDS);
         }
     }
 
@@ -88,20 +91,20 @@ public class BlockingReadRepair extends AbstractReadRepair
         if (timedOut)
         {
             // We got all responses, but timed out while repairing
-            int blockFor = consistency.blockFor(cfs.keyspace);
+            int blockFor = replicaLayout.consistencyLevel().blockFor(cfs.keyspace);
             if (Tracing.isTracing())
                 Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor);
             else
                 logger.debug("Timeout while read-repairing after receiving all {} data and digest responses", blockFor);
 
-            throw new ReadTimeoutException(consistency, blockFor-1, blockFor, true);
+            throw new ReadTimeoutException(replicaLayout.consistencyLevel(), blockFor - 1, blockFor, true);
         }
     }
 
     @Override
-    public void repairPartition(DecoratedKey key, Map<InetAddressAndPort, Mutation> mutations, InetAddressAndPort[] destinations)
+    public void repairPartition(DecoratedKey partitionKey, Map<Replica, Mutation> mutations, L replicaLayout)
     {
-        BlockingPartitionRepair blockingRepair = new BlockingPartitionRepair(cfs.keyspace, key, consistency, mutations, consistency.blockFor(cfs.keyspace), destinations);
+        BlockingPartitionRepair<E, L> blockingRepair = new BlockingPartitionRepair<>(partitionKey, mutations, blockFor, replicaLayout);
         blockingRepair.sendInitialRepairs();
         repairs.add(blockingRepair);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java
index e5f7179..ceb1765 100644
--- a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java
+++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java
@@ -18,10 +18,6 @@
 
 package org.apache.cassandra.service.reads.repair;
 
-import java.util.List;
-
-import com.google.common.collect.Iterables;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,13 +27,10 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.tracing.Tracing;
 
 public class BlockingReadRepairs
@@ -48,18 +41,6 @@ public class BlockingReadRepairs
         Boolean.getBoolean("cassandra.drop_oversized_readrepair_mutations");
 
     /**
-     * Returns all of the endpoints that are replicas for the given key. If the consistency level is datacenter
-     * local, only the endpoints in the local dc will be returned.
-     */
-    static Iterable<InetAddressAndPort> getCandidateEndpoints(Keyspace keyspace, Token token, ConsistencyLevel consistency)
-    {
-        List<InetAddressAndPort> endpoints = StorageProxy.getLiveSortedEndpoints(keyspace, token);
-        return consistency.isDatacenterLocal() && keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy
-               ? Iterables.filter(endpoints, ConsistencyLevel::isLocal)
-               : endpoints;
-    }
-
-    /**
      * Create a read repair mutation from the given update, if the mutation is not larger than the maximum
      * mutation size, otherwise return null. Or, if we're configured to be strict, throw an exception.
      */


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


[04/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
index 9c90d57..4afeb5a 100644
--- a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.locator;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -39,9 +38,11 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class OldNetworkTopologyStrategyTest
 {
+
     private List<Token> keyTokens;
     private TokenMetadata tmd;
     private Map<String, ArrayList<InetAddressAndPort>> expectedResults;
@@ -53,7 +54,7 @@ public class OldNetworkTopologyStrategyTest
     }
 
     @Before
-    public void init()
+    public void init() throws Exception
     {
         keyTokens = new ArrayList<Token>();
         tmd = new TokenMetadata();
@@ -160,11 +161,11 @@ public class OldNetworkTopologyStrategyTest
     {
         for (Token keyToken : keyTokens)
         {
-            List<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(keyToken);
-            for (int j = 0; j < endpoints.size(); j++)
+            int j = 0;
+            for (InetAddressAndPort endpoint : strategy.getNaturalReplicasForToken(keyToken).endpoints())
             {
                 ArrayList<InetAddressAndPort> hostsExpected = expectedResults.get(keyToken.toString());
-                assertEquals(endpoints.get(j), hostsExpected.get(j));
+                assertEquals(endpoint, hostsExpected.get(j++));
             }
         }
     }
@@ -188,7 +189,6 @@ public class OldNetworkTopologyStrategyTest
         assertEquals(ranges.left.iterator().next().left, tokensAfterMove[movingNodeIdx]);
         assertEquals(ranges.left.iterator().next().right, tokens[movingNodeIdx]);
         assertEquals("No data should be fetched", ranges.right.size(), 0);
-
     }
 
     @Test
@@ -205,7 +205,6 @@ public class OldNetworkTopologyStrategyTest
         assertEquals("No data should be streamed", ranges.left.size(), 0);
         assertEquals(ranges.right.iterator().next().left, tokens[movingNodeIdx]);
         assertEquals(ranges.right.iterator().next().right, tokensAfterMove[movingNodeIdx]);
-
     }
 
     @SuppressWarnings("unchecked")
@@ -366,16 +365,21 @@ public class OldNetworkTopologyStrategyTest
         TokenMetadata tokenMetadataAfterMove = initTokenMetadata(tokensAfterMove);
         AbstractReplicationStrategy strategy = new OldNetworkTopologyStrategy("Keyspace1", tokenMetadataCurrent, endpointSnitch, optsWithRF(2));
 
-        Collection<Range<Token>> currentRanges = strategy.getAddressRanges().get(movingNode);
-        Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetadataAfterMove, tokensAfterMove[movingNodeIdx], movingNode);
-
-        Pair<Set<Range<Token>>, Set<Range<Token>>> ranges = StorageService.instance.calculateStreamAndFetchRanges(currentRanges, updatedRanges);
+        RangesAtEndpoint currentRanges = strategy.getAddressReplicas().get(movingNode);
+        RangesAtEndpoint updatedRanges = strategy.getPendingAddressRanges(tokenMetadataAfterMove, tokensAfterMove[movingNodeIdx], movingNode);
 
-        return ranges;
+        return asRanges(StorageService.calculateStreamAndFetchRanges(currentRanges, updatedRanges));
     }
 
     private static Map<String, String> optsWithRF(int rf)
     {
         return Collections.singletonMap("replication_factor", Integer.toString(rf));
     }
+
+    public static Pair<Set<Range<Token>>, Set<Range<Token>>> asRanges(Pair<RangesAtEndpoint, RangesAtEndpoint> replicas)
+    {
+        Set<Range<Token>> leftRanges = replicas.left.ranges();
+        Set<Range<Token>> rightRanges = replicas.right.ranges();
+        return Pair.create(leftRanges, rightRanges);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
index 56fd181..8e0bc00 100644
--- a/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
+++ b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.dht.Token;
 import org.junit.Test;
 
 import java.net.UnknownHostException;
-import java.util.Collection;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -38,17 +37,29 @@ public class PendingRangeMapsTest {
         return new Range<Token>(new BigIntegerToken(left), new BigIntegerToken(right));
     }
 
+    private static void addPendingRange(PendingRangeMaps pendingRangeMaps, Range<Token> range, String endpoint)
+    {
+        try
+        {
+            pendingRangeMaps.addPendingRange(range, Replica.fullReplica(InetAddressAndPort.getByName(endpoint), range));
+        }
+        catch (UnknownHostException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
     @Test
     public void testPendingEndpoints() throws UnknownHostException
     {
         PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
 
-        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddressAndPort.getByName("127.0.0.1"));
-        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddressAndPort.getByName("127.0.0.2"));
-        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddressAndPort.getByName("127.0.0.3"));
-        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddressAndPort.getByName("127.0.0.4"));
-        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddressAndPort.getByName("127.0.0.5"));
-        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddressAndPort.getByName("127.0.0.6"));
+        addPendingRange(pendingRangeMaps, genRange("5", "15"), "127.0.0.1");
+        addPendingRange(pendingRangeMaps, genRange("15", "25"), "127.0.0.2");
+        addPendingRange(pendingRangeMaps, genRange("25", "35"), "127.0.0.3");
+        addPendingRange(pendingRangeMaps, genRange("35", "45"), "127.0.0.4");
+        addPendingRange(pendingRangeMaps, genRange("45", "55"), "127.0.0.5");
+        addPendingRange(pendingRangeMaps, genRange("45", "65"), "127.0.0.6");
 
         assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
         assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
@@ -61,8 +72,8 @@ public class PendingRangeMapsTest {
         assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
 
-        Collection<InetAddressAndPort> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15"));
-        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.1")));
+        EndpointsForToken replicas = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15"));
+        assertTrue(replicas.endpoints().contains(InetAddressAndPort.getByName("127.0.0.1")));
     }
 
     @Test
@@ -70,13 +81,13 @@ public class PendingRangeMapsTest {
     {
         PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
 
-        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddressAndPort.getByName("127.0.0.1"));
-        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddressAndPort.getByName("127.0.0.2"));
-        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddressAndPort.getByName("127.0.0.3"));
-        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddressAndPort.getByName("127.0.0.4"));
-        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddressAndPort.getByName("127.0.0.5"));
-        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddressAndPort.getByName("127.0.0.6"));
-        pendingRangeMaps.addPendingRange(genRange("65", "7"), InetAddressAndPort.getByName("127.0.0.7"));
+        addPendingRange(pendingRangeMaps, genRange("5", "15"), "127.0.0.1");
+        addPendingRange(pendingRangeMaps, genRange("15", "25"), "127.0.0.2");
+        addPendingRange(pendingRangeMaps, genRange("25", "35"), "127.0.0.3");
+        addPendingRange(pendingRangeMaps, genRange("35", "45"), "127.0.0.4");
+        addPendingRange(pendingRangeMaps, genRange("45", "55"), "127.0.0.5");
+        addPendingRange(pendingRangeMaps, genRange("45", "65"), "127.0.0.6");
+        addPendingRange(pendingRangeMaps, genRange("65", "7"), "127.0.0.7");
 
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
@@ -90,8 +101,8 @@ public class PendingRangeMapsTest {
         assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
         assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
 
-        Collection<InetAddressAndPort> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("6"));
-        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.1")));
-        assertTrue(endpoints.contains(InetAddressAndPort.getByName("127.0.0.7")));
+        EndpointsForToken replicas = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("6"));
+        assertTrue(replicas.endpoints().contains(InetAddressAndPort.getByName("127.0.0.1")));
+        assertTrue(replicas.endpoints().contains(InetAddressAndPort.getByName("127.0.0.7")));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/ReplicaCollectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicaCollectionTest.java b/test/unit/org/apache/cassandra/locator/ReplicaCollectionTest.java
new file mode 100644
index 0000000..66eff23
--- /dev/null
+++ b/test/unit/org/apache/cassandra/locator/ReplicaCollectionTest.java
@@ -0,0 +1,468 @@
+/*
+ * 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 com.google.common.base.Predicates;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
+import org.apache.cassandra.utils.FBUtilities;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+
+public class ReplicaCollectionTest
+{
+
+    static final InetAddressAndPort EP1, EP2, EP3, EP4, EP5, BROADCAST_EP, NULL_EP;
+    static final Range<Token> R1, R2, R3, R4, R5, BROADCAST_RANGE, NULL_RANGE;
+
+    static
+    {
+        try
+        {
+            EP1 = InetAddressAndPort.getByName("127.0.0.1");
+            EP2 = InetAddressAndPort.getByName("127.0.0.2");
+            EP3 = InetAddressAndPort.getByName("127.0.0.3");
+            EP4 = InetAddressAndPort.getByName("127.0.0.4");
+            EP5 = InetAddressAndPort.getByName("127.0.0.5");
+            BROADCAST_EP = FBUtilities.getBroadcastAddressAndPort();
+            NULL_EP = InetAddressAndPort.getByName("127.255.255.255");
+            R1 = range(0, 1);
+            R2 = range(1, 2);
+            R3 = range(2, 3);
+            R4 = range(3, 4);
+            R5 = range(4, 5);
+            BROADCAST_RANGE = range(10, 11);
+            NULL_RANGE = range(10000, 10001);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    static Token tk(long t)
+    {
+        return new Murmur3Partitioner.LongToken(t);
+    }
+
+    static Range<Token> range(long left, long right)
+    {
+        return new Range<>(tk(left), tk(right));
+    }
+
+    static class TestCase<C extends AbstractReplicaCollection<C>>
+    {
+        final C test;
+        final List<Replica> canonicalList;
+        final Multimap<InetAddressAndPort, Replica> canonicalByEndpoint;
+        final Multimap<Range<Token>, Replica> canonicalByRange;
+
+        TestCase(C test, List<Replica> canonicalList)
+        {
+            this.test = test;
+            this.canonicalList = canonicalList;
+            this.canonicalByEndpoint = HashMultimap.create();
+            this.canonicalByRange = HashMultimap.create();
+            for (Replica replica : canonicalList)
+                canonicalByEndpoint.put(replica.endpoint(), replica);
+            for (Replica replica : canonicalList)
+                canonicalByRange.put(replica.range(), replica);
+        }
+
+        void testSize()
+        {
+            Assert.assertEquals(canonicalList.size(), test.size());
+        }
+
+        void testEquals()
+        {
+            Assert.assertTrue(Iterables.elementsEqual(canonicalList, test));
+        }
+
+        void testEndpoints()
+        {
+            // TODO: we should do more exhaustive tests of the collection
+            Assert.assertEquals(ImmutableSet.copyOf(canonicalByEndpoint.keySet()), ImmutableSet.copyOf(test.endpoints()));
+            try
+            {
+                test.endpoints().add(EP5);
+                Assert.fail();
+            } catch (UnsupportedOperationException e) {}
+            try
+            {
+                test.endpoints().remove(EP5);
+                Assert.fail();
+            } catch (UnsupportedOperationException e) {}
+
+            Assert.assertTrue(test.endpoints().containsAll(canonicalByEndpoint.keySet()));
+            for (InetAddressAndPort ep : canonicalByEndpoint.keySet())
+                Assert.assertTrue(test.endpoints().contains(ep));
+            for (InetAddressAndPort ep : ImmutableList.of(EP1, EP2, EP3, EP4, EP5, BROADCAST_EP))
+                if (!canonicalByEndpoint.containsKey(ep))
+                    Assert.assertFalse(test.endpoints().contains(ep));
+        }
+
+        public void testOrderOfIteration()
+        {
+            Assert.assertEquals(canonicalList, ImmutableList.copyOf(test));
+            Assert.assertEquals(canonicalList, test.stream().collect(Collectors.toList()));
+            Assert.assertEquals(new LinkedHashSet<>(Lists.transform(canonicalList, Replica::endpoint)), test.endpoints());
+        }
+
+        void testSelect(int subListDepth, int filterDepth, int sortDepth, int selectDepth)
+        {
+            TestCase<C> allMatchZeroCapacity = new TestCase<>(test.select().add(Predicates.alwaysTrue(), 0).get(), Collections.emptyList());
+            allMatchZeroCapacity.testAll(subListDepth, filterDepth, sortDepth, selectDepth - 1);
+
+            TestCase<C> noMatchFullCapacity = new TestCase<>(test.select().add(Predicates.alwaysFalse(), canonicalList.size()).get(), Collections.emptyList());
+            noMatchFullCapacity.testAll(subListDepth, filterDepth, sortDepth,selectDepth - 1);
+
+            if (canonicalList.size() <= 2)
+                return;
+
+            List<Replica> newOrderList = ImmutableList.of(canonicalList.get(2), canonicalList.get(1), canonicalList.get(0));
+            TestCase<C> newOrder = new TestCase<>(
+                    test.select()
+                            .add(r -> r == newOrderList.get(0), 3)
+                            .add(r -> r == newOrderList.get(1), 3)
+                            .add(r -> r == newOrderList.get(2), 3)
+                            .get(), newOrderList
+            );
+            newOrder.testAll(subListDepth, filterDepth, sortDepth,selectDepth - 1);
+        }
+
+        private void assertSubList(C subCollection, int from, int to)
+        {
+            Assert.assertTrue(subCollection.isSnapshot);
+            if (from == to)
+            {
+                Assert.assertTrue(subCollection.isEmpty());
+            }
+            else
+            {
+                List<Replica> subList = this.test.list.subList(from, to);
+                if (test.isSnapshot)
+                    Assert.assertSame(subList.getClass(), subCollection.list.getClass());
+                Assert.assertEquals(subList, subCollection.list);
+            }
+        }
+
+        void testSubList(int subListDepth, int filterDepth, int sortDepth, int selectDepth)
+        {
+            if (test.isSnapshot)
+                Assert.assertSame(test, test.subList(0, test.size()));
+
+            if (test.isEmpty())
+                return;
+
+            TestCase<C> skipFront = new TestCase<>(test.subList(1, test.size()), canonicalList.subList(1, canonicalList.size()));
+            assertSubList(skipFront.test, 1, canonicalList.size());
+            skipFront.testAll(subListDepth - 1, filterDepth, sortDepth, selectDepth);
+            TestCase<C> skipBack = new TestCase<>(test.subList(0, test.size() - 1), canonicalList.subList(0, canonicalList.size() - 1));
+            assertSubList(skipBack.test, 0, canonicalList.size() - 1);
+            skipBack.testAll(subListDepth - 1, filterDepth, sortDepth, selectDepth);
+        }
+
+        void testFilter(int subListDepth, int filterDepth, int sortDepth, int selectDepth)
+        {
+            if (test.isSnapshot)
+                Assert.assertSame(test, test.filter(Predicates.alwaysTrue()));
+
+            if (test.isEmpty())
+                return;
+            // remove start
+            // we recurse on the same subset in testSubList, so just corroborate we have the correct list here
+            assertSubList(test.filter(r -> r != canonicalList.get(0)), 1, canonicalList.size());
+
+            if (test.size() <= 1)
+                return;
+            // remove end
+            // we recurse on the same subset in testSubList, so just corroborate we have the correct list here
+            assertSubList(test.filter(r -> r != canonicalList.get(canonicalList.size() - 1)), 0, canonicalList.size() - 1);
+
+            if (test.size() <= 2)
+                return;
+            Predicate<Replica> removeMiddle = r -> r != canonicalList.get(canonicalList.size() / 2);
+            TestCase<C> filtered = new TestCase<>(test.filter(removeMiddle), ImmutableList.copyOf(Iterables.filter(canonicalList, removeMiddle::test)));
+            filtered.testAll(subListDepth, filterDepth - 1, sortDepth, selectDepth);
+        }
+
+        void testContains()
+        {
+            for (Replica replica : canonicalList)
+                Assert.assertTrue(test.contains(replica));
+            Assert.assertFalse(test.contains(fullReplica(NULL_EP, NULL_RANGE)));
+        }
+
+        void testGet()
+        {
+            for (int i = 0 ; i < canonicalList.size() ; ++i)
+                Assert.assertEquals(canonicalList.get(i), test.get(i));
+        }
+
+        void testSort(int subListDepth, int filterDepth, int sortDepth, int selectDepth)
+        {
+            final Comparator<Replica> comparator = (o1, o2) ->
+            {
+                boolean f1 = o1 == canonicalList.get(0);
+                boolean f2 = o2 == canonicalList.get(0);
+                return f1 == f2 ? 0 : f1 ? 1 : -1;
+            };
+            TestCase<C> sorted = new TestCase<>(test.sorted(comparator), ImmutableList.sortedCopyOf(comparator, canonicalList));
+            sorted.testAll(subListDepth, filterDepth, sortDepth - 1, selectDepth);
+        }
+
+        private void testAll(int subListDepth, int filterDepth, int sortDepth, int selectDepth)
+        {
+            testEndpoints();
+            testOrderOfIteration();
+            testContains();
+            testGet();
+            testEquals();
+            testSize();
+            if (subListDepth > 0)
+                testSubList(subListDepth, filterDepth, sortDepth, selectDepth);
+            if (filterDepth > 0)
+                testFilter(subListDepth, filterDepth, sortDepth, selectDepth);
+            if (sortDepth > 0)
+                testSort(subListDepth, filterDepth, sortDepth, selectDepth);
+            if (selectDepth > 0)
+                testSelect(subListDepth, filterDepth, sortDepth, selectDepth);
+        }
+
+        public void testAll()
+        {
+            testAll(2, 2, 2, 2);
+        }
+    }
+
+    static class RangesAtEndpointTestCase extends TestCase<RangesAtEndpoint>
+    {
+        RangesAtEndpointTestCase(RangesAtEndpoint test, List<Replica> canonicalList)
+        {
+            super(test, canonicalList);
+        }
+
+        void testRanges()
+        {
+            Assert.assertEquals(ImmutableSet.copyOf(canonicalByRange.keySet()), ImmutableSet.copyOf(test.ranges()));
+            try
+            {
+                test.ranges().add(R5);
+                Assert.fail();
+            } catch (UnsupportedOperationException e) {}
+            try
+            {
+                test.ranges().remove(R5);
+                Assert.fail();
+            } catch (UnsupportedOperationException e) {}
+
+            Assert.assertTrue(test.ranges().containsAll(canonicalByRange.keySet()));
+            for (Range<Token> range : canonicalByRange.keySet())
+                Assert.assertTrue(test.ranges().contains(range));
+            for (Range<Token> range : ImmutableList.of(R1, R2, R3, R4, R5, BROADCAST_RANGE))
+                if (!canonicalByRange.containsKey(range))
+                    Assert.assertFalse(test.ranges().contains(range));
+        }
+
+        @Override
+        public void testOrderOfIteration()
+        {
+            super.testOrderOfIteration();
+            Assert.assertEquals(new LinkedHashSet<>(Lists.transform(canonicalList, Replica::range)), test.ranges());
+        }
+
+        @Override
+        public void testAll()
+        {
+            super.testAll();
+            testRanges();
+        }
+    }
+
+    private static final ImmutableList<Replica> RANGES_AT_ENDPOINT = ImmutableList.of(
+            fullReplica(EP1, R1),
+            fullReplica(EP1, R2),
+            transientReplica(EP1, R3),
+            fullReplica(EP1, R4),
+            transientReplica(EP1, R5)
+    );
+
+    @Test
+    public void testRangesAtEndpoint()
+    {
+        ImmutableList<Replica> canonical = RANGES_AT_ENDPOINT;
+        new RangesAtEndpointTestCase(
+                RangesAtEndpoint.copyOf(canonical), canonical
+        ).testAll();
+    }
+
+    @Test
+    public void testMutableRangesAtEndpoint()
+    {
+        ImmutableList<Replica> canonical1 = RANGES_AT_ENDPOINT.subList(0, RANGES_AT_ENDPOINT.size());
+        RangesAtEndpoint.Mutable test = new RangesAtEndpoint.Mutable(RANGES_AT_ENDPOINT.get(0).endpoint(), canonical1.size());
+        test.addAll(canonical1, Conflict.NONE);
+        try
+        {   // incorrect range
+            test.addAll(canonical1, Conflict.NONE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.addAll(canonical1, Conflict.DUPLICATE); // we ignore exact duplicates
+        try
+        {   // invalid endpoint; always error
+            test.add(fullReplica(EP2, BROADCAST_RANGE), Conflict.ALL);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        try
+        {   // conflict on isFull/isTransient
+            test.add(fullReplica(EP1, R3), Conflict.DUPLICATE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.add(fullReplica(EP1, R3), Conflict.ALL);
+
+        new RangesAtEndpointTestCase(test, canonical1).testAll();
+
+        RangesAtEndpoint view = test.asImmutableView();
+        RangesAtEndpoint snapshot = view.subList(0, view.size());
+
+        ImmutableList<Replica> canonical2 = RANGES_AT_ENDPOINT;
+        test.addAll(canonical2.reverse(), Conflict.DUPLICATE);
+        new TestCase<>(snapshot, canonical1).testAll();
+        new TestCase<>(view, canonical2).testAll();
+        new TestCase<>(test, canonical2).testAll();
+    }
+
+    private static final ImmutableList<Replica> ENDPOINTS_FOR_X = ImmutableList.of(
+            fullReplica(EP1, R1),
+            fullReplica(EP2, R1),
+            transientReplica(EP3, R1),
+            fullReplica(EP4, R1),
+            transientReplica(EP5, R1)
+    );
+
+    @Test
+    public void testEndpointsForRange()
+    {
+        ImmutableList<Replica> canonical = ENDPOINTS_FOR_X;
+        new TestCase<>(
+                EndpointsForRange.copyOf(canonical), canonical
+        ).testAll();
+    }
+
+    @Test
+    public void testMutableEndpointsForRange()
+    {
+        ImmutableList<Replica> canonical1 = ENDPOINTS_FOR_X.subList(0, ENDPOINTS_FOR_X.size() - 1);
+        EndpointsForRange.Mutable test = new EndpointsForRange.Mutable(R1, canonical1.size());
+        test.addAll(canonical1, Conflict.NONE);
+        try
+        {   // incorrect range
+            test.addAll(canonical1, Conflict.NONE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.addAll(canonical1, Conflict.DUPLICATE); // we ignore exact duplicates
+        try
+        {   // incorrect range
+            test.add(fullReplica(BROADCAST_EP, R2), Conflict.ALL);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        try
+        {   // conflict on isFull/isTransient
+            test.add(transientReplica(EP1, R1), Conflict.DUPLICATE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.add(transientReplica(EP1, R1), Conflict.ALL);
+
+        new TestCase<>(test, canonical1).testAll();
+
+        EndpointsForRange view = test.asImmutableView();
+        EndpointsForRange snapshot = view.subList(0, view.size());
+
+        ImmutableList<Replica> canonical2 = ENDPOINTS_FOR_X;
+        test.addAll(canonical2.reverse(), Conflict.DUPLICATE);
+        new TestCase<>(snapshot, canonical1).testAll();
+        new TestCase<>(view, canonical2).testAll();
+        new TestCase<>(test, canonical2).testAll();
+    }
+
+    @Test
+    public void testEndpointsForToken()
+    {
+        ImmutableList<Replica> canonical = ENDPOINTS_FOR_X;
+        new TestCase<>(
+                EndpointsForToken.copyOf(tk(1), canonical), canonical
+        ).testAll();
+    }
+
+    @Test
+    public void testMutableEndpointsForToken()
+    {
+        ImmutableList<Replica> canonical1 = ENDPOINTS_FOR_X.subList(0, ENDPOINTS_FOR_X.size() - 1);
+        EndpointsForToken.Mutable test = new EndpointsForToken.Mutable(tk(1), canonical1.size());
+        test.addAll(canonical1, Conflict.NONE);
+        try
+        {   // incorrect range
+            test.addAll(canonical1, Conflict.NONE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.addAll(canonical1, Conflict.DUPLICATE); // we ignore exact duplicates
+        try
+        {   // incorrect range
+            test.add(fullReplica(BROADCAST_EP, R2), Conflict.ALL);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        try
+        {   // conflict on isFull/isTransient
+            test.add(transientReplica(EP1, R1), Conflict.DUPLICATE);
+            Assert.fail();
+        } catch (IllegalArgumentException e) { }
+        test.add(transientReplica(EP1, R1), Conflict.ALL);
+
+        new TestCase<>(test, canonical1).testAll();
+
+        EndpointsForToken view = test.asImmutableView();
+        EndpointsForToken snapshot = view.subList(0, view.size());
+
+        ImmutableList<Replica> canonical2 = ENDPOINTS_FOR_X;
+        test.addAll(canonical2.reverse(), Conflict.DUPLICATE);
+        new TestCase<>(snapshot, canonical1).testAll();
+        new TestCase<>(view, canonical2).testAll();
+        new TestCase<>(test, canonical2).testAll();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/ReplicaUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicaUtils.java b/test/unit/org/apache/cassandra/locator/ReplicaUtils.java
new file mode 100644
index 0000000..66f538f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/locator/ReplicaUtils.java
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+
+public class ReplicaUtils
+{
+    public static final Range<Token> FULL_RANGE = new Range<>(Murmur3Partitioner.MINIMUM, Murmur3Partitioner.MINIMUM);
+    public static final AbstractBounds<PartitionPosition> FULL_BOUNDS = new Range<>(Murmur3Partitioner.MINIMUM.minKeyBound(), Murmur3Partitioner.MINIMUM.maxKeyBound());
+
+    public static Replica full(InetAddressAndPort endpoint)
+    {
+        return fullReplica(endpoint, FULL_RANGE);
+    }
+
+    public static Replica trans(InetAddressAndPort endpoint)
+    {
+        return transientReplica(endpoint, FULL_RANGE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/ReplicationFactorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationFactorTest.java b/test/unit/org/apache/cassandra/locator/ReplicationFactorTest.java
new file mode 100644
index 0000000..a0427db
--- /dev/null
+++ b/test/unit/org/apache/cassandra/locator/ReplicationFactorTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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 org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+public class ReplicationFactorTest
+{
+
+    @BeforeClass
+    public static void setupClass()
+    {
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        Gossiper.instance.start(1);
+    }
+
+    private static void assertRfParseFailure(String s)
+    {
+        try
+        {
+            ReplicationFactor.fromString(s);
+            Assert.fail("Expected IllegalArgumentException");
+        }
+        catch (IllegalArgumentException e)
+        {
+            // expected
+        }
+    }
+
+    private static void assertRfParse(String s, int expectedReplicas, int expectedTrans)
+    {
+        ReplicationFactor rf = ReplicationFactor.fromString(s);
+        Assert.assertEquals(expectedReplicas, rf.allReplicas);
+        Assert.assertEquals(expectedTrans, rf.transientReplicas());
+        Assert.assertEquals(expectedReplicas - expectedTrans, rf.fullReplicas);
+    }
+
+    @Test
+    public void parseTest()
+    {
+        assertRfParse("3", 3, 0);
+        assertRfParse("3/1", 3, 1);
+
+        assertRfParse("5", 5, 0);
+        assertRfParse("5/2", 5, 2);
+
+        assertRfParseFailure("-1");
+        assertRfParseFailure("3/3");
+        assertRfParseFailure("3/4");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
index a8caa72..e6a9365 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.locator;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang3.StringUtils;
@@ -75,7 +73,7 @@ public class ReplicationStrategyEndpointCacheTest
     public void runEndpointsWereCachedTest(Class stratClass, Map<String, String> configOptions) throws Exception
     {
         setup(stratClass, configOptions);
-        assert strategy.getNaturalEndpoints(searchToken).equals(strategy.getNaturalEndpoints(searchToken));
+        assert strategy.getNaturalReplicasForToken(searchToken).equals(strategy.getNaturalReplicasForToken(searchToken));
     }
 
     @Test
@@ -89,34 +87,34 @@ public class ReplicationStrategyEndpointCacheTest
     public void runCacheRespectsTokenChangesTest(Class stratClass, Map<String, String> configOptions) throws Exception
     {
         setup(stratClass, configOptions);
-        ArrayList<InetAddressAndPort> initial;
-        ArrayList<InetAddressAndPort> endpoints;
+        EndpointsForToken initial;
+        EndpointsForToken replicas;
 
-        endpoints = strategy.getNaturalEndpoints(searchToken);
-        assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
+        replicas = strategy.getNaturalReplicasForToken(searchToken);
+        assert replicas.size() == 5 : StringUtils.join(replicas, ",");
 
         // test token addition, in DC2 before existing token
-        initial = strategy.getNaturalEndpoints(searchToken);
+        initial = strategy.getNaturalReplicasForToken(searchToken);
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(35)), InetAddressAndPort.getByName("127.0.0.5"));
-        endpoints = strategy.getNaturalEndpoints(searchToken);
-        assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
-        assert !endpoints.equals(initial);
+        replicas = strategy.getNaturalReplicasForToken(searchToken);
+        assert replicas.size() == 5 : StringUtils.join(replicas, ",");
+        assert !replicas.equals(initial);
 
         // test token removal, newly created token
-        initial = strategy.getNaturalEndpoints(searchToken);
+        initial = strategy.getNaturalReplicasForToken(searchToken);
         tmd.removeEndpoint(InetAddressAndPort.getByName("127.0.0.5"));
-        endpoints = strategy.getNaturalEndpoints(searchToken);
-        assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
-        assert !endpoints.contains(InetAddressAndPort.getByName("127.0.0.5"));
-        assert !endpoints.equals(initial);
+        replicas = strategy.getNaturalReplicasForToken(searchToken);
+        assert replicas.size() == 5 : StringUtils.join(replicas, ",");
+        assert !replicas.endpoints().contains(InetAddressAndPort.getByName("127.0.0.5"));
+        assert !replicas.equals(initial);
 
         // test token change
-        initial = strategy.getNaturalEndpoints(searchToken);
+        initial = strategy.getNaturalReplicasForToken(searchToken);
         //move .8 after search token but before other DC3
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(25)), InetAddressAndPort.getByName("127.0.0.8"));
-        endpoints = strategy.getNaturalEndpoints(searchToken);
-        assert endpoints.size() == 5 : StringUtils.join(endpoints, ",");
-        assert !endpoints.equals(initial);
+        replicas = strategy.getNaturalReplicasForToken(searchToken);
+        assert replicas.size() == 5 : StringUtils.join(replicas, ",");
+        assert !replicas.equals(initial);
     }
 
     protected static class FakeSimpleStrategy extends SimpleStrategy
@@ -128,11 +126,11 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
         {
-            assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
+            assert !called : "calculateNaturalReplicas was already called, result should have been cached";
             called = true;
-            return super.calculateNaturalEndpoints(token, metadata);
+            return super.calculateNaturalReplicas(token, metadata);
         }
     }
 
@@ -145,11 +143,11 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
         {
-            assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
+            assert !called : "calculateNaturalReplicas was already called, result should have been cached";
             called = true;
-            return super.calculateNaturalEndpoints(token, metadata);
+            return super.calculateNaturalReplicas(token, metadata);
         }
     }
 
@@ -162,11 +160,11 @@ public class ReplicationStrategyEndpointCacheTest
             super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
-        public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+        public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
         {
-            assert !called : "calculateNaturalEndpoints was already called, result should have been cached";
+            assert !called : "calculateNaturalReplicas was already called, result should have been cached";
             called = true;
-            return super.calculateNaturalEndpoints(token, metadata);
+            return super.calculateNaturalReplicas(token, metadata);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index fe77b0e..1e0c152 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -19,14 +19,22 @@ package org.apache.cassandra.locator;
 
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.IPartitioner;
@@ -53,6 +61,7 @@ public class SimpleStrategyTest
     {
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1));
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
     }
 
     @Test
@@ -107,12 +116,12 @@ public class SimpleStrategyTest
 
             for (int i = 0; i < keyTokens.length; i++)
             {
-                List<InetAddressAndPort> endpoints = strategy.getNaturalEndpoints(keyTokens[i]);
-                assertEquals(strategy.getReplicationFactor(), endpoints.size());
+                EndpointsForToken replicas = strategy.getNaturalReplicasForToken(keyTokens[i]);
+                assertEquals(strategy.getReplicationFactor().allReplicas, replicas.size());
                 List<InetAddressAndPort> correctEndpoints = new ArrayList<>();
-                for (int j = 0; j < endpoints.size(); j++)
+                for (int j = 0; j < replicas.size(); j++)
                     correctEndpoints.add(hosts.get((i + j + 1) % hosts.size()));
-                assertEquals(new HashSet<>(correctEndpoints), new HashSet<>(endpoints));
+                assertEquals(new HashSet<>(correctEndpoints), replicas.endpoints());
             }
         }
     }
@@ -154,30 +163,80 @@ public class SimpleStrategyTest
 
             PendingRangeCalculatorService.calculatePendingRanges(strategy, keyspaceName);
 
-            int replicationFactor = strategy.getReplicationFactor();
+            int replicationFactor = strategy.getReplicationFactor().allReplicas;
 
             for (int i = 0; i < keyTokens.length; i++)
             {
-                Collection<InetAddressAndPort> endpoints = tmd.getWriteEndpoints(keyTokens[i], keyspaceName, strategy.getNaturalEndpoints(keyTokens[i]));
-                assertTrue(endpoints.size() >= replicationFactor);
+                EndpointsForToken replicas = tmd.getWriteEndpoints(keyTokens[i], keyspaceName, strategy.getNaturalReplicasForToken(keyTokens[i]));
+                assertTrue(replicas.size() >= replicationFactor);
 
                 for (int j = 0; j < replicationFactor; j++)
                 {
                     //Check that the old nodes are definitely included
-                    assertTrue(endpoints.contains(hosts.get((i + j + 1) % hosts.size())));
+                   assertTrue(replicas.endpoints().contains(hosts.get((i + j + 1) % hosts.size())));
                 }
 
                 // bootstrapEndpoint should be in the endpoints for i in MAX-RF to MAX, but not in any earlier ep.
                 if (i < RING_SIZE - replicationFactor)
-                    assertFalse(endpoints.contains(bootstrapEndpoint));
+                    assertFalse(replicas.endpoints().contains(bootstrapEndpoint));
                 else
-                    assertTrue(endpoints.contains(bootstrapEndpoint));
+                    assertTrue(replicas.endpoints().contains(bootstrapEndpoint));
             }
         }
 
         StorageServiceAccessor.setTokenMetadata(oldTmd);
     }
 
+    private static Token tk(long t)
+    {
+        return new Murmur3Partitioner.LongToken(t);
+    }
+
+    private static Range<Token> range(long l, long r)
+    {
+        return new Range<>(tk(l), tk(r));
+    }
+
+    @Test
+    public void transientReplica() throws Exception
+    {
+        IEndpointSnitch snitch = new SimpleSnitch();
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+
+        List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"),
+                                                                InetAddressAndPort.getByName("127.0.0.2"),
+                                                                InetAddressAndPort.getByName("127.0.0.3"),
+                                                                InetAddressAndPort.getByName("127.0.0.4"));
+
+        Multimap<InetAddressAndPort, Token> tokens = HashMultimap.create();
+        tokens.put(endpoints.get(0), tk(100));
+        tokens.put(endpoints.get(1), tk(200));
+        tokens.put(endpoints.get(2), tk(300));
+        tokens.put(endpoints.get(3), tk(400));
+        TokenMetadata metadata = new TokenMetadata();
+        metadata.updateNormalTokens(tokens);
+
+        Map<String, String> configOptions = new HashMap<String, String>();
+        configOptions.put("replication_factor", "3/1");
+
+        SimpleStrategy strategy = new SimpleStrategy("ks", metadata, snitch, configOptions);
+
+        Range<Token> range1 = range(400, 100);
+        Assert.assertEquals(EndpointsForToken.of(range1.right,
+                                                 Replica.fullReplica(endpoints.get(0), range1),
+                                                 Replica.fullReplica(endpoints.get(1), range1),
+                                                 Replica.transientReplica(endpoints.get(2), range1)),
+                            strategy.getNaturalReplicasForToken(tk(99)));
+
+
+        Range<Token> range2 = range(100, 200);
+        Assert.assertEquals(EndpointsForToken.of(range2.right,
+                                                 Replica.fullReplica(endpoints.get(1), range2),
+                                                 Replica.fullReplica(endpoints.get(2), range2),
+                                                 Replica.transientReplica(endpoints.get(3), range2)),
+                            strategy.getNaturalReplicasForToken(tk(101)));
+    }
+
     private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd)
     {
         KeyspaceMetadata ksmd = Schema.instance.getKeyspaceMetadata(keyspaceName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
index b589d2d..ae8c011 100644
--- a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
+++ b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
@@ -118,7 +118,7 @@ public class TokenMetadataTest
             }
 
             @Override
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
             {
                 return 0;
             }
@@ -165,7 +165,7 @@ public class TokenMetadataTest
             }
 
             @Override
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
             {
                 return 0;
             }
@@ -216,7 +216,7 @@ public class TokenMetadataTest
             }
 
             @Override
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
             {
                 return 0;
             }
@@ -263,7 +263,7 @@ public class TokenMetadataTest
             }
 
             @Override
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
             {
                 return 0;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
index d97cdb8..e226d32 100644
--- a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
+++ b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
@@ -25,19 +25,20 @@ import org.junit.Test;
 
 import org.junit.Assert;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.net.MessagingService.Verb;
 import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.locator.ReplicaUtils.full;
+
 public class WriteCallbackInfoTest
 {
     @BeforeClass
@@ -65,7 +66,7 @@ public class WriteCallbackInfoTest
                          ? new Commit(UUID.randomUUID(), new PartitionUpdate.Builder(metadata, ByteBufferUtil.EMPTY_BYTE_BUFFER, RegularAndStaticColumns.NONE, 1).build())
                          : new Mutation(PartitionUpdate.simpleBuilder(metadata, "").build());
 
-        WriteCallbackInfo wcbi = new WriteCallbackInfo(InetAddressAndPort.getByName("192.168.1.1"), null, new MessageOut(verb, payload, null), null, cl, allowHints);
+        WriteCallbackInfo wcbi = new WriteCallbackInfo(full(InetAddressAndPort.getByName("192.168.1.1")), null, new MessageOut(verb, payload, null), null, cl, allowHints);
         Assert.assertEquals(expectHint, wcbi.shouldHint());
         if (expectHint)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
index 6a8dc83..379031c 100644
--- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
+++ b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java
@@ -46,6 +46,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceTest;
@@ -172,7 +173,7 @@ public class OutboundMessagingConnectionTest
             return nodeToDc.get(endpoint);
         }
 
-        public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+        public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
         {
             return 0;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
deleted file mode 100644
index 802a673..0000000
--- a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import com.google.common.collect.Lists;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableId;
-import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.streaming.StreamPlan;
-import org.apache.cassandra.streaming.PreviewKind;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTree;
-import org.apache.cassandra.utils.MerkleTrees;
-import org.apache.cassandra.utils.UUIDGen;
-
-import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class LocalSyncTaskTest extends AbstractRepairTest
-{
-    private static final IPartitioner partitioner = Murmur3Partitioner.instance;
-    public static final String KEYSPACE1 = "DifferencerTest";
-    public static final String CF_STANDARD = "Standard1";
-    public static ColumnFamilyStore cfs;
-
-    @BeforeClass
-    public static void defineSchema()
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
-
-        TableId tid = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD).id;
-        cfs = Schema.instance.getColumnFamilyStoreInstance(tid);
-    }
-
-    /**
-     * When there is no difference between two, LocalSyncTask should return stats with 0 difference.
-     */
-    @Test
-    public void testNoDifference() throws Throwable
-    {
-        final InetAddressAndPort ep1 = InetAddressAndPort.getByName("127.0.0.1");
-        final InetAddressAndPort ep2 = InetAddressAndPort.getByName("127.0.0.1");
-
-        Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
-        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
-
-        MerkleTrees tree1 = createInitialTree(desc);
-
-        MerkleTrees tree2 = createInitialTree(desc);
-
-        // difference the trees
-        // note: we reuse the same endpoint which is bogus in theory but fine here
-        TreeResponse r1 = new TreeResponse(ep1, tree1);
-        TreeResponse r2 = new TreeResponse(ep2, tree2);
-        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, NO_PENDING_REPAIR, false, PreviewKind.NONE);
-        task.run();
-
-        assertEquals(0, task.get().numberOfDifferences);
-    }
-
-    @Test
-    public void testDifference() throws Throwable
-    {
-        Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
-        UUID parentRepairSession = UUID.randomUUID();
-        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
-
-        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddressAndPort(),
-                                                                 Arrays.asList(cfs), Arrays.asList(range), false,
-                                                                 ActiveRepairService.UNREPAIRED_SSTABLE, false,
-                                                                 PreviewKind.NONE);
-
-        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
-
-        MerkleTrees tree1 = createInitialTree(desc);
-
-        MerkleTrees tree2 = createInitialTree(desc);
-
-        // change a range in one of the trees
-        Token token = partitioner.midpoint(range.left, range.right);
-        tree1.invalidate(token);
-        MerkleTree.TreeRange changed = tree1.get(token);
-        changed.hash("non-empty hash!".getBytes());
-
-        Set<Range<Token>> interesting = new HashSet<>();
-        interesting.add(changed);
-
-        // difference the trees
-        // note: we reuse the same endpoint which is bogus in theory but fine here
-        TreeResponse r1 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.1"), tree1);
-        TreeResponse r2 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.2"), tree2);
-        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, NO_PENDING_REPAIR, false, PreviewKind.NONE);
-        task.run();
-
-        // ensure that the changed range was recorded
-        assertEquals("Wrong differing ranges", interesting.size(), task.getCurrentStat().numberOfDifferences);
-    }
-
-    @Test
-    public void fullRepairStreamPlan() throws Exception
-    {
-        UUID sessionID = registerSession(cfs, true, true);
-        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
-        RepairJobDesc desc = new RepairJobDesc(sessionID, UUIDGen.getTimeUUID(), KEYSPACE1, CF_STANDARD, prs.getRanges());
-
-        TreeResponse r1 = new TreeResponse(PARTICIPANT1, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
-        TreeResponse r2 = new TreeResponse(PARTICIPANT2, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
-
-        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, NO_PENDING_REPAIR, false, PreviewKind.NONE);
-        StreamPlan plan = task.createStreamPlan(PARTICIPANT1, Lists.newArrayList(RANGE1));
-
-        assertEquals(NO_PENDING_REPAIR, plan.getPendingRepair());
-        assertTrue(plan.getFlushBeforeTransfer());
-    }
-
-    @Test
-    public void incrementalRepairStreamPlan() throws Exception
-    {
-        UUID sessionID = registerSession(cfs, true, true);
-        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
-        RepairJobDesc desc = new RepairJobDesc(sessionID, UUIDGen.getTimeUUID(), KEYSPACE1, CF_STANDARD, prs.getRanges());
-
-        TreeResponse r1 = new TreeResponse(PARTICIPANT1, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
-        TreeResponse r2 = new TreeResponse(PARTICIPANT2, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
-
-        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, desc.parentSessionId, false, PreviewKind.NONE);
-        StreamPlan plan = task.createStreamPlan(PARTICIPANT1, Lists.newArrayList(RANGE1));
-
-        assertEquals(desc.parentSessionId, plan.getPendingRepair());
-        assertFalse(plan.getFlushBeforeTransfer());
-    }
-
-    private MerkleTrees createInitialTree(RepairJobDesc desc, IPartitioner partitioner)
-    {
-        MerkleTrees tree = new MerkleTrees(partitioner);
-        tree.addMerkleTrees((int) Math.pow(2, 15), desc.ranges);
-        tree.init();
-        for (MerkleTree.TreeRange r : tree.invalids())
-        {
-            r.ensureHashInitialised();
-        }
-        return tree;
-    }
-
-    private MerkleTrees createInitialTree(RepairJobDesc desc)
-    {
-        return createInitialTree(desc, partitioner);
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java b/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
index 2044106..418d7de 100644
--- a/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairRunnableTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.repair;
 
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -29,7 +28,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.repair.RepairRunnable.CommonRange;
 
 import static org.apache.cassandra.repair.RepairRunnable.filterCommonRanges;
 
@@ -41,7 +39,7 @@ public class RepairRunnableTest extends AbstractRepairTest
     @Test
     public void filterCommonIncrementalRangesNotForced() throws Exception
     {
-        CommonRange cr = new CommonRange(PARTICIPANTS, ALL_RANGES);
+        CommonRange cr = new CommonRange(PARTICIPANTS, Collections.emptySet(), ALL_RANGES);
 
         List<CommonRange> expected = Lists.newArrayList(cr);
         List<CommonRange> actual = filterCommonRanges(expected, Collections.emptySet(), false);
@@ -52,13 +50,13 @@ public class RepairRunnableTest extends AbstractRepairTest
     @Test
     public void forceFilterCommonIncrementalRanges() throws Exception
     {
-        CommonRange cr1 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2), Sets.newHashSet(RANGE1, RANGE2));
-        CommonRange cr2 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3), Sets.newHashSet(RANGE3));
+        CommonRange cr1 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2), Collections.emptySet(), Sets.newHashSet(RANGE1, RANGE2));
+        CommonRange cr2 = new CommonRange(Sets.newHashSet(PARTICIPANT1, PARTICIPANT2, PARTICIPANT3), Collections.emptySet(), Sets.newHashSet(RANGE3));
         Set<InetAddressAndPort> liveEndpoints = Sets.newHashSet(PARTICIPANT2, PARTICIPANT3); // PARTICIPANT1 is excluded
 
         List<CommonRange> initial = Lists.newArrayList(cr1, cr2);
-        List<CommonRange> expected = Lists.newArrayList(new CommonRange(Sets.newHashSet(PARTICIPANT2), Sets.newHashSet(RANGE1, RANGE2)),
-                                                        new CommonRange(Sets.newHashSet(PARTICIPANT2, PARTICIPANT3), Sets.newHashSet(RANGE3)));
+        List<CommonRange> expected = Lists.newArrayList(new CommonRange(Sets.newHashSet(PARTICIPANT2), Collections.emptySet(), Sets.newHashSet(RANGE1, RANGE2)),
+                                                        new CommonRange(Sets.newHashSet(PARTICIPANT2, PARTICIPANT3), Collections.emptySet(), Sets.newHashSet(RANGE3)));
         List<CommonRange> actual = filterCommonRanges(initial, liveEndpoints, true);
 
         Assert.assertEquals(expected, actual);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
index 54f0511..e77d657 100644
--- a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.repair;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
@@ -62,9 +63,10 @@ public class RepairSessionTest
         IPartitioner p = Murmur3Partitioner.instance;
         Range<Token> repairRange = new Range<>(p.getToken(ByteBufferUtil.bytes(0)), p.getToken(ByteBufferUtil.bytes(100)));
         Set<InetAddressAndPort> endpoints = Sets.newHashSet(remote);
-        RepairSession session = new RepairSession(parentSessionId, sessionId, Arrays.asList(repairRange),
+        RepairSession session = new RepairSession(parentSessionId, sessionId,
+                                                  new CommonRange(endpoints, Collections.emptySet(), Arrays.asList(repairRange)),
                                                   "Keyspace1", RepairParallelism.SEQUENTIAL,
-                                                  endpoints, false, false, false,
+                                                  false, false, false,
                                                   PreviewKind.NONE, false, "Standard1");
 
         // perform convict

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/SymmetricLocalSyncTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/SymmetricLocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/SymmetricLocalSyncTaskTest.java
new file mode 100644
index 0000000..92ae172
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/SymmetricLocalSyncTaskTest.java
@@ -0,0 +1,232 @@
+/*
+ * 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.repair;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTree;
+import org.apache.cassandra.utils.MerkleTrees;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class SymmetricLocalSyncTaskTest extends AbstractRepairTest
+{
+    private static final IPartitioner partitioner = Murmur3Partitioner.instance;
+    public static final String KEYSPACE1 = "DifferencerTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static ColumnFamilyStore cfs;
+
+    @BeforeClass
+    public static void defineSchema()
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+
+        TableId tid = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD).id;
+        cfs = Schema.instance.getColumnFamilyStoreInstance(tid);
+    }
+
+    /**
+     * When there is no difference between two, SymmetricLocalSyncTask should return stats with 0 difference.
+     */
+    @Test
+    public void testNoDifference() throws Throwable
+    {
+        final InetAddressAndPort ep1 = InetAddressAndPort.getByName("127.0.0.1");
+        final InetAddressAndPort ep2 = InetAddressAndPort.getByName("127.0.0.1");
+
+        Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
+        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
+
+        MerkleTrees tree1 = createInitialTree(desc);
+
+        MerkleTrees tree2 = createInitialTree(desc);
+
+        // difference the trees
+        // note: we reuse the same endpoint which is bogus in theory but fine here
+        TreeResponse r1 = new TreeResponse(ep1, tree1);
+        TreeResponse r2 = new TreeResponse(ep2, tree2);
+        SymmetricLocalSyncTask task = new SymmetricLocalSyncTask(desc, r1, r2, false, NO_PENDING_REPAIR, false, PreviewKind.NONE);
+        task.run();
+
+        assertEquals(0, task.get().numberOfDifferences);
+    }
+
+    @Test
+    public void testDifference() throws Throwable
+    {
+        Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
+        UUID parentRepairSession = UUID.randomUUID();
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+
+        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddressAndPort(),
+                                                                 Arrays.asList(cfs), Arrays.asList(range), false,
+                                                                 ActiveRepairService.UNREPAIRED_SSTABLE, false,
+                                                                 PreviewKind.NONE);
+
+        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
+
+        MerkleTrees tree1 = createInitialTree(desc);
+
+        MerkleTrees tree2 = createInitialTree(desc);
+
+        // change a range in one of the trees
+        Token token = partitioner.midpoint(range.left, range.right);
+        tree1.invalidate(token);
+        MerkleTree.TreeRange changed = tree1.get(token);
+        changed.hash("non-empty hash!".getBytes());
+
+        Set<Range<Token>> interesting = new HashSet<>();
+        interesting.add(changed);
+
+        // difference the trees
+        // note: we reuse the same endpoint which is bogus in theory but fine here
+        TreeResponse r1 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.1"), tree1);
+        TreeResponse r2 = new TreeResponse(InetAddressAndPort.getByName("127.0.0.2"), tree2);
+        SymmetricLocalSyncTask task = new SymmetricLocalSyncTask(desc, r1, r2, false, NO_PENDING_REPAIR, false, PreviewKind.NONE);
+        DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 1;
+        DefaultConnectionFactory.MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(2);
+        try
+        {
+            task.run();
+        }
+        finally
+        {
+            DefaultConnectionFactory.MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30);
+            DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 3;
+        }
+
+        // ensure that the changed range was recorded
+        assertEquals("Wrong differing ranges", interesting.size(), task.getCurrentStat().numberOfDifferences);
+    }
+
+    @Test
+    public void fullRepairStreamPlan() throws Exception
+    {
+        UUID sessionID = registerSession(cfs, true, true);
+        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
+        RepairJobDesc desc = new RepairJobDesc(sessionID, UUIDGen.getTimeUUID(), KEYSPACE1, CF_STANDARD, prs.getRanges());
+
+        TreeResponse r1 = new TreeResponse(PARTICIPANT1, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+        TreeResponse r2 = new TreeResponse(PARTICIPANT2, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+
+        SymmetricLocalSyncTask task = new SymmetricLocalSyncTask(desc, r1, r2, false, NO_PENDING_REPAIR, false, PreviewKind.NONE);
+        StreamPlan plan = task.createStreamPlan(PARTICIPANT1, Lists.newArrayList(RANGE1));
+
+        assertEquals(NO_PENDING_REPAIR, plan.getPendingRepair());
+        assertTrue(plan.getFlushBeforeTransfer());
+    }
+
+    private static void assertNumInOut(StreamPlan plan, int expectedIncoming, int expectedOutgoing)
+    {
+        StreamCoordinator coordinator = plan.getCoordinator();
+        StreamSession session = Iterables.getOnlyElement(coordinator.getAllStreamSessions());
+        assertEquals(expectedIncoming, session.getNumRequests());
+        assertEquals(expectedOutgoing, session.getNumTransfers());
+    }
+
+    @Test
+    public void incrementalRepairStreamPlan() throws Exception
+    {
+        UUID sessionID = registerSession(cfs, true, true);
+        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
+        RepairJobDesc desc = new RepairJobDesc(sessionID, UUIDGen.getTimeUUID(), KEYSPACE1, CF_STANDARD, prs.getRanges());
+
+        TreeResponse r1 = new TreeResponse(PARTICIPANT1, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+        TreeResponse r2 = new TreeResponse(PARTICIPANT2, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+
+        SymmetricLocalSyncTask task = new SymmetricLocalSyncTask(desc, r1, r2, false, desc.parentSessionId, false, PreviewKind.NONE);
+        StreamPlan plan = task.createStreamPlan(PARTICIPANT1, Lists.newArrayList(RANGE1));
+
+        assertEquals(desc.parentSessionId, plan.getPendingRepair());
+        assertFalse(plan.getFlushBeforeTransfer());
+        assertNumInOut(plan, 1, 1);
+    }
+
+    /**
+     * Don't reciprocate streams if the other endpoint is a transient replica
+     */
+    @Test
+    public void transientStreamPlan()
+    {
+        UUID sessionID = registerSession(cfs, true, true);
+        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(sessionID);
+        RepairJobDesc desc = new RepairJobDesc(sessionID, UUIDGen.getTimeUUID(), KEYSPACE1, CF_STANDARD, prs.getRanges());
+
+        TreeResponse r1 = new TreeResponse(PARTICIPANT1, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+        TreeResponse r2 = new TreeResponse(PARTICIPANT2, createInitialTree(desc, DatabaseDescriptor.getPartitioner()));
+
+        SymmetricLocalSyncTask task = new SymmetricLocalSyncTask(desc, r1, r2, true, desc.parentSessionId, false, PreviewKind.NONE);
+        StreamPlan plan = task.createStreamPlan(PARTICIPANT2, Lists.newArrayList(RANGE1));
+        assertNumInOut(plan, 1, 0);
+    }
+
+    private MerkleTrees createInitialTree(RepairJobDesc desc, IPartitioner partitioner)
+    {
+        MerkleTrees tree = new MerkleTrees(partitioner);
+        tree.addMerkleTrees((int) Math.pow(2, 15), desc.ranges);
+        tree.init();
+        for (MerkleTree.TreeRange r : tree.invalids())
+        {
+            r.ensureHashInitialised();
+        }
+        return tree;
+    }
+
+    private MerkleTrees createInitialTree(RepairJobDesc desc)
+    {
+        return createInitialTree(desc, partitioner);
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java
new file mode 100644
index 0000000..06f968f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.repair;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.repair.messages.RepairMessage;
+import org.apache.cassandra.repair.messages.SyncRequest;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class SymmetricRemoteSyncTaskTest extends AbstractRepairTest
+{
+    private static final RepairJobDesc DESC = new RepairJobDesc(UUIDGen.getTimeUUID(), UUIDGen.getTimeUUID(), "ks", "tbl", ALL_RANGES);
+    private static final List<Range<Token>> RANGE_LIST = ImmutableList.of(RANGE1);
+
+    private static class InstrumentedSymmetricRemoteSyncTask extends SymmetricRemoteSyncTask
+    {
+        public InstrumentedSymmetricRemoteSyncTask(InetAddressAndPort e1, InetAddressAndPort e2)
+        {
+            super(DESC, new TreeResponse(e1, null), new TreeResponse(e2, null), PreviewKind.NONE);
+        }
+
+        RepairMessage sentMessage = null;
+        InetAddressAndPort sentTo = null;
+
+        @Override
+        void sendRequest(RepairMessage request, InetAddressAndPort to)
+        {
+            Assert.assertNull(sentMessage);
+            Assert.assertNotNull(request);
+            Assert.assertNotNull(to);
+            sentMessage = request;
+            sentTo = to;
+        }
+    }
+
+    @Test
+    public void normalSync()
+    {
+        InstrumentedSymmetricRemoteSyncTask syncTask = new InstrumentedSymmetricRemoteSyncTask(PARTICIPANT1, PARTICIPANT2);
+        syncTask.startSync(RANGE_LIST);
+
+        Assert.assertNotNull(syncTask.sentMessage);
+        Assert.assertSame(SyncRequest.class, syncTask.sentMessage.getClass());
+        Assert.assertEquals(PARTICIPANT1, syncTask.sentTo);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
index 3ea888d..a7e8272 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionAccessor.java
@@ -44,12 +44,13 @@ public class LocalSessionAccessor
         ARS.consistent.local.putSessionUnsafe(session);
     }
 
-    public static void finalizeUnsafe(UUID sessionID)
+    public static long finalizeUnsafe(UUID sessionID)
     {
         LocalSession session = ARS.consistent.local.getSession(sessionID);
         assert session != null;
         session.setState(ConsistentSession.State.FINALIZED);
         ARS.consistent.local.save(session);
+        return session.repairedAt;
     }
 
     public static void failUnsafe(UUID sessionID)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
index d368510..e387c41 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
@@ -40,10 +40,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.KeyspaceRepairManager;
@@ -136,7 +135,11 @@ public class LocalSessionTest extends AbstractRepairTest
         boolean prepareSessionCalled = false;
 
         @Override
-        ListenableFuture prepareSession(KeyspaceRepairManager repairManager, UUID sessionID, Collection<ColumnFamilyStore> tables, Collection<Range<Token>> ranges, ExecutorService executor)
+        ListenableFuture prepareSession(KeyspaceRepairManager repairManager,
+                                        UUID sessionID,
+                                        Collection<ColumnFamilyStore> tables,
+                                        RangesAtEndpoint ranges,
+                                        ExecutorService executor)
         {
             prepareSessionCalled = true;
             if (prepareSessionFuture != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/schema/MockSchema.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/MockSchema.java b/test/unit/org/apache/cassandra/schema/MockSchema.java
index 98bf9ca..7a6b011 100644
--- a/test/unit/org/apache/cassandra/schema/MockSchema.java
+++ b/test/unit/org/apache/cassandra/schema/MockSchema.java
@@ -127,7 +127,7 @@ public class MockSchema
         }
         SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList());
         StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator)
-                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, null, header)
+                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, null, false, header)
                                                  .get(MetadataType.STATS);
         SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata,
                                                           RANDOM_ACCESS_READER_FACTORY.sharedCopy(), RANDOM_ACCESS_READER_FACTORY.sharedCopy(), indexSummary.sharedCopy(),


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


[09/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 9467c9a..7f4ae14 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.MatchResult;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
 import javax.annotation.Nullable;
@@ -41,9 +42,12 @@ import javax.management.openmbean.TabularDataSupport;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 
+import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 import org.apache.commons.lang3.StringUtils;
 
 import org.slf4j.Logger;
@@ -110,6 +114,8 @@ import org.apache.cassandra.utils.progress.ProgressEventType;
 import org.apache.cassandra.utils.progress.jmx.JMXBroadcastExecutor;
 import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport;
 
+import static com.google.common.collect.Iterables.transform;
+import static com.google.common.collect.Iterables.tryFind;
 import static java.util.Arrays.asList;
 import static java.util.stream.Collectors.toList;
 import static org.apache.cassandra.index.SecondaryIndexManager.getIndexName;
@@ -164,9 +170,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return isShutdown;
     }
 
-    public Collection<Range<Token>> getLocalRanges(String keyspaceName)
+    public RangesAtEndpoint getLocalReplicas(String keyspaceName)
     {
-        return getRangesForEndpoint(keyspaceName, FBUtilities.getBroadcastAddressAndPort());
+        return getReplicasForEndpoint(keyspaceName, FBUtilities.getBroadcastAddressAndPort());
     }
 
     public List<Range<Token>> getLocalAndPendingRanges(String ks)
@@ -174,9 +180,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         InetAddressAndPort broadcastAddress = FBUtilities.getBroadcastAddressAndPort();
         Keyspace keyspace = Keyspace.open(ks);
         List<Range<Token>> ranges = new ArrayList<>();
-        ranges.addAll(keyspace.getReplicationStrategy().getAddressRanges().get(broadcastAddress));
-        ranges.addAll(getTokenMetadata().getPendingRanges(ks, broadcastAddress));
-        return Range.normalize(ranges);
+        for (Replica r : keyspace.getReplicationStrategy().getAddressReplicas(broadcastAddress))
+            ranges.add(r.range());
+        for (Replica r : getTokenMetadata().getPendingRanges(ks, broadcastAddress))
+            ranges.add(r.range());
+        return ranges;
     }
 
     public Collection<Range<Token>> getPrimaryRanges(String keyspace)
@@ -1225,11 +1233,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             if (keyspace == null)
             {
                 for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
-                    streamer.addRanges(keyspaceName, getLocalRanges(keyspaceName));
+                    streamer.addRanges(keyspaceName, getLocalReplicas(keyspaceName));
             }
             else if (tokens == null)
             {
-                streamer.addRanges(keyspace, getLocalRanges(keyspace));
+                streamer.addRanges(keyspace, getLocalReplicas(keyspace));
             }
             else
             {
@@ -1251,14 +1259,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 }
 
                 // Ensure all specified ranges are actually ranges owned by this host
-                Collection<Range<Token>> localRanges = getLocalRanges(keyspace);
+                RangesAtEndpoint localReplicas = getLocalReplicas(keyspace);
+                RangesAtEndpoint.Builder streamRanges = new RangesAtEndpoint.Builder(FBUtilities.getBroadcastAddressAndPort(), ranges.size());
                 for (Range<Token> specifiedRange : ranges)
                 {
                     boolean foundParentRange = false;
-                    for (Range<Token> localRange : localRanges)
+                    for (Replica localReplica : localReplicas)
                     {
-                        if (localRange.contains(specifiedRange))
+                        if (localReplica.contains(specifiedRange))
                         {
+                            streamRanges.add(localReplica.decorateSubrange(specifiedRange));
                             foundParentRange = true;
                             break;
                         }
@@ -1292,7 +1302,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     streamer.addSourceFilter(new RangeStreamer.WhitelistedSourcesFilter(sources));
                 }
 
-                streamer.addRanges(keyspace, ranges);
+                streamer.addRanges(keyspace, streamRanges.build());
             }
 
             StreamResultFuture resultFuture = streamer.fetchAsync();
@@ -1700,9 +1710,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         /* All the ranges for the tokens */
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>,List<InetAddressAndPort>> entry : getRangeToAddressMap(keyspace).entrySet())
+        for (Map.Entry<Range<Token>, EndpointsForRange> entry : getRangeToAddressMap(keyspace).entrySet())
         {
-            map.put(entry.getKey().asList(), stringify(entry.getValue(), withPort));
+            map.put(entry.getKey().asList(), Replicas.stringify(entry.getValue(), withPort));
         }
         return map;
     }
@@ -1753,12 +1763,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         /* All the ranges for the tokens */
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : getRangeToAddressMap(keyspace).entrySet())
+        for (Map.Entry<Range<Token>, EndpointsForRange> entry : getRangeToAddressMap(keyspace).entrySet())
         {
             List<String> rpcaddrs = new ArrayList<>(entry.getValue().size());
-            for (InetAddressAndPort endpoint: entry.getValue())
+            for (Replica replicas: entry.getValue())
             {
-                rpcaddrs.add(getNativeaddress(endpoint, withPort));
+                rpcaddrs.add(getNativeaddress(replicas.endpoint(), withPort));
             }
             map.put(entry.getKey().asList(), rpcaddrs);
         }
@@ -1783,38 +1793,31 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             keyspace = Schema.instance.getNonLocalStrategyKeyspaces().get(0);
 
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddressAndPort>> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet())
+        for (Map.Entry<Range<Token>, EndpointsForRange> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet())
         {
-            List<InetAddressAndPort> l = new ArrayList<>(entry.getValue());
-            map.put(entry.getKey().asList(), stringify(l, withPort));
+            map.put(entry.getKey().asList(), Replicas.stringify(entry.getValue(), withPort));
         }
         return map;
     }
 
-    public Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMap(String keyspace)
+    public EndpointsByRange getRangeToAddressMap(String keyspace)
     {
         return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
     }
 
-    public Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMapInLocalDC(String keyspace)
+    public EndpointsByRange getRangeToAddressMapInLocalDC(String keyspace)
     {
-        Predicate<InetAddressAndPort> isLocalDC = new Predicate<InetAddressAndPort>()
-        {
-            public boolean apply(InetAddressAndPort address)
-            {
-                return isLocalDC(address);
-            }
-        };
+        Predicate<Replica> isLocalDC = replica -> isLocalDC(replica.endpoint());
 
-        Map<Range<Token>, List<InetAddressAndPort>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
-        Map<Range<Token>, List<InetAddressAndPort>> filteredMap = Maps.newHashMap();
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : origMap.entrySet())
+        EndpointsByRange origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+        Map<Range<Token>, EndpointsForRange> filteredMap = Maps.newHashMap();
+        for (Map.Entry<Range<Token>, EndpointsForRange> entry : origMap.entrySet())
         {
-            List<InetAddressAndPort> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+            EndpointsForRange endpointsInLocalDC = entry.getValue().filter(isLocalDC);
             filteredMap.put(entry.getKey(), endpointsInLocalDC);
         }
 
-        return filteredMap;
+        return new EndpointsByRange(filteredMap);
     }
 
     private List<Token> getTokensInLocalDC()
@@ -1836,7 +1839,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return remoteDC.equals(localDC);
     }
 
-    private Map<Range<Token>, List<InetAddressAndPort>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+    private EndpointsByRange getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
     {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system keyspace.
@@ -1917,13 +1920,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         List<TokenRange> ranges = new ArrayList<>();
         Token.TokenFactory tf = getTokenFactory();
 
-        Map<Range<Token>, List<InetAddressAndPort>> rangeToAddressMap =
+        EndpointsByRange rangeToAddressMap =
                 includeOnlyLocalDC
                         ? getRangeToAddressMapInLocalDC(keyspace)
                         : getRangeToAddressMap(keyspace);
 
-        for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : rangeToAddressMap.entrySet())
-            ranges.add(TokenRange.create(tf, entry.getKey(), entry.getValue(), withPort));
+        for (Map.Entry<Range<Token>, EndpointsForRange> entry : rangeToAddressMap.entrySet())
+            ranges.add(TokenRange.create(tf, entry.getKey(), ImmutableList.copyOf(entry.getValue().endpoints()), withPort));
 
         return ranges;
     }
@@ -2010,14 +2013,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ranges
      * @return mapping of ranges to the replicas responsible for them.
     */
-    private Map<Range<Token>, List<InetAddressAndPort>> constructRangeToEndpointMap(String keyspace, List<Range<Token>> ranges)
+    private EndpointsByRange constructRangeToEndpointMap(String keyspace, List<Range<Token>> ranges)
     {
-        Map<Range<Token>, List<InetAddressAndPort>> rangeToEndpointMap = new HashMap<>(ranges.size());
+        Map<Range<Token>, EndpointsForRange> rangeToEndpointMap = new HashMap<>(ranges.size());
         for (Range<Token> range : ranges)
         {
-            rangeToEndpointMap.put(range, Keyspace.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.right));
+            rangeToEndpointMap.put(range, Keyspace.open(keyspace).getReplicationStrategy().getNaturalReplicas(range.right));
         }
-        return rangeToEndpointMap;
+        return new EndpointsByRange(rangeToEndpointMap);
     }
 
     public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue)
@@ -2735,32 +2738,56 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * Finds living endpoints responsible for the given ranges
      *
      * @param keyspaceName the keyspace ranges belong to
-     * @param ranges the ranges to find sources for
+     * @param leavingReplicas the ranges to find sources for
      * @return multimap of addresses to ranges the address is responsible for
      */
-    private Multimap<InetAddressAndPort, Range<Token>> getNewSourceRanges(String keyspaceName, Set<Range<Token>> ranges)
+    private Multimap<InetAddressAndPort, FetchReplica> getNewSourceReplicas(String keyspaceName, Set<LeavingReplica> leavingReplicas)
     {
         InetAddressAndPort myAddress = FBUtilities.getBroadcastAddressAndPort();
-        Multimap<Range<Token>, InetAddressAndPort> rangeAddresses = Keyspace.open(keyspaceName).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
-        Multimap<InetAddressAndPort, Range<Token>> sourceRanges = HashMultimap.create();
+        EndpointsByRange rangeReplicas = Keyspace.open(keyspaceName).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
+        Multimap<InetAddressAndPort, FetchReplica> sourceRanges = HashMultimap.create();
         IFailureDetector failureDetector = FailureDetector.instance;
 
+        logger.debug("Getting new source replicas for {}", leavingReplicas);
+
         // find alive sources for our new ranges
-        for (Range<Token> range : ranges)
-        {
-            Collection<InetAddressAndPort> possibleRanges = rangeAddresses.get(range);
+        for (LeavingReplica leaver : leavingReplicas)
+        {
+            //We need this to find the replicas from before leaving to supply the data
+            Replica leavingReplica = leaver.leavingReplica;
+            //We need this to know what to fetch and what the transient status is
+            Replica ourReplica = leaver.ourReplica;
+            //If we are going to be a full replica only consider full replicas
+            Predicate<Replica> replicaFilter = ourReplica.isFull() ? Replica::isFull : Predicates.alwaysTrue();
+            Predicate<Replica> notSelf = replica -> !replica.endpoint().equals(myAddress);
+            EndpointsForRange possibleReplicas = rangeReplicas.get(leavingReplica.range());
+            logger.info("Possible replicas for newReplica {} are {}", ourReplica, possibleReplicas);
             IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-            List<InetAddressAndPort> sources = snitch.getSortedListByProximity(myAddress, possibleRanges);
+            EndpointsForRange sortedPossibleReplicas = snitch.sortedByProximity(myAddress, possibleReplicas);
+            logger.info("Sorted possible replicas starts as {}", sortedPossibleReplicas);
+            Optional<Replica> myCurrentReplica = tryFind(possibleReplicas, replica -> replica.endpoint().equals(myAddress)).toJavaUtil();
 
-            assert (!sources.contains(myAddress));
+            boolean transientToFull = myCurrentReplica.isPresent() && myCurrentReplica.get().isTransient() && ourReplica.isFull();
+            assert !sortedPossibleReplicas.endpoints().contains(myAddress) || transientToFull : String.format("My address %s, sortedPossibleReplicas %s, myCurrentReplica %s, myNewReplica %s", myAddress, sortedPossibleReplicas, myCurrentReplica, ourReplica);
 
-            for (InetAddressAndPort source : sources)
+            //Originally this didn't log if it couldn't restore replication and that seems wrong
+            boolean foundLiveReplica = false;
+            for (Replica possibleReplica : sortedPossibleReplicas.filter(Predicates.and(replicaFilter, notSelf)))
             {
-                if (failureDetector.isAlive(source))
+                if (failureDetector.isAlive(possibleReplica.endpoint()))
                 {
-                    sourceRanges.put(source, range);
+                    foundLiveReplica = true;
+                    sourceRanges.put(possibleReplica.endpoint(), new FetchReplica(ourReplica, possibleReplica));
                     break;
                 }
+                else
+                {
+                    logger.debug("Skipping down replica {}", possibleReplica);
+                }
+            }
+            if (!foundLiveReplica)
+            {
+                logger.warn("Didn't find live replica to restore replication for " + ourReplica);
             }
         }
         return sourceRanges;
@@ -2793,6 +2820,49 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
     }
 
+    private static class LeavingReplica
+    {
+        //The node that is leaving
+        private final Replica leavingReplica;
+
+        //Our range and transient status
+        private final Replica ourReplica;
+
+        public LeavingReplica(Replica leavingReplica, Replica ourReplica)
+        {
+            Preconditions.checkNotNull(leavingReplica);
+            Preconditions.checkNotNull(ourReplica);
+            this.leavingReplica = leavingReplica;
+            this.ourReplica = ourReplica;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            LeavingReplica that = (LeavingReplica) o;
+
+            if (!leavingReplica.equals(that.leavingReplica)) return false;
+            return ourReplica.equals(that.ourReplica);
+        }
+
+        public int hashCode()
+        {
+            int result = leavingReplica.hashCode();
+            result = 31 * result + ourReplica.hashCode();
+            return result;
+        }
+
+        public String toString()
+        {
+            return "LeavingReplica{" +
+                   "leavingReplica=" + leavingReplica +
+                   ", ourReplica=" + ourReplica +
+                   '}';
+        }
+    }
+
     /**
      * Called when an endpoint is removed from the ring. This function checks
      * whether this node becomes responsible for new ranges as a
@@ -2805,38 +2875,52 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     private void restoreReplicaCount(InetAddressAndPort endpoint, final InetAddressAndPort notifyEndpoint)
     {
-        Multimap<String, Map.Entry<InetAddressAndPort, Collection<Range<Token>>>> rangesToFetch = HashMultimap.create();
+        Map<String, Multimap<InetAddressAndPort, FetchReplica>> replicasToFetch = new HashMap<>();
 
         InetAddressAndPort myAddress = FBUtilities.getBroadcastAddressAndPort();
 
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
-            Multimap<Range<Token>, InetAddressAndPort> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
-            Set<Range<Token>> myNewRanges = new HashSet<>();
-            for (Map.Entry<Range<Token>, InetAddressAndPort> entry : changedRanges.entries())
-            {
-                if (entry.getValue().equals(myAddress))
-                    myNewRanges.add(entry.getKey());
-            }
-            Multimap<InetAddressAndPort, Range<Token>> sourceRanges = getNewSourceRanges(keyspaceName, myNewRanges);
-            for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
+            logger.debug("Restoring replica count for keyspace {}", keyspaceName);
+            EndpointsByReplica changedReplicas = getChangedReplicasForLeaving(keyspaceName, endpoint, tokenMetadata, Keyspace.open(keyspaceName).getReplicationStrategy());
+            Set<LeavingReplica> myNewReplicas = new HashSet<>();
+            for (Map.Entry<Replica, Replica> entry : changedReplicas.flattenEntries())
             {
-                rangesToFetch.put(keyspaceName, entry);
+                Replica replica = entry.getValue();
+                if (replica.endpoint().equals(myAddress))
+                {
+                    //Maybe we don't technically need to fetch transient data from somewhere
+                    //but it's probably not a lot and it probably makes things a hair more resilient to people
+                    //not running repair when they should.
+                    myNewReplicas.add(new LeavingReplica(entry.getKey(), entry.getValue()));
+                }
             }
+            logger.debug("Changed replicas for leaving {}, myNewReplicas {}", changedReplicas, myNewReplicas);
+            replicasToFetch.put(keyspaceName, getNewSourceReplicas(keyspaceName, myNewReplicas));
         }
 
         StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT);
-        for (String keyspaceName : rangesToFetch.keySet())
-        {
-            for (Map.Entry<InetAddressAndPort, Collection<Range<Token>>> entry : rangesToFetch.get(keyspaceName))
-            {
-                InetAddressAndPort source = entry.getKey();
-                Collection<Range<Token>> ranges = entry.getValue();
+        replicasToFetch.forEach((keyspaceName, sources) -> {
+            logger.debug("Requesting keyspace {} sources", keyspaceName);
+            sources.asMap().forEach((sourceAddress, fetchReplicas) -> {
+                logger.debug("Source and our replicas are {}", fetchReplicas);
+                //Remember whether this node is providing the full or transient replicas for this range. We are going
+                //to pass streaming the local instance of Replica for the range which doesn't tell us anything about the source
+                //By encoding it as two separate sets we retain this information about the source.
+                RangesAtEndpoint full = fetchReplicas.stream()
+                                                             .filter(f -> f.remote.isFull())
+                                                             .map(f -> f.local)
+                                                             .collect(RangesAtEndpoint.collector(myAddress));
+                RangesAtEndpoint transientReplicas = fetchReplicas.stream()
+                                                                  .filter(f -> f.remote.isTransient())
+                                                                  .map(f -> f.local)
+                                                                  .collect(RangesAtEndpoint.collector(myAddress));
                 if (logger.isDebugEnabled())
-                    logger.debug("Requesting from {} ranges {}", source, StringUtils.join(ranges, ", "));
-                stream.requestRanges(source, keyspaceName, ranges);
-            }
-        }
+                    logger.debug("Requesting from {} full replicas {} transient replicas {}", sourceAddress, StringUtils.join(full, ", "), StringUtils.join(transientReplicas, ", "));
+
+                stream.requestRanges(sourceAddress, keyspaceName, full, transientReplicas);
+            });
+        });
         StreamResultFuture future = stream.execute();
         Futures.addCallback(future, new FutureCallback<StreamState>()
         {
@@ -2854,21 +2938,36 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         });
     }
 
+    /**
+     * This is used in three contexts, graceful decomission, and restoreReplicaCount/removeNode.
+     * Graceful decomission should never lose data and it's going to be important that transient data
+     * is streamed to at least one other node from this one for each range.
+     *
+     * For ranges this node replicates its removal should cause a new replica to be selected either as transient or full
+     * for every range. So I believe the current code doesn't have to do anything special because it will engage in streaming
+     * for every range it replicates to at least one other node and that should propagate the transient data that was here.
+     * When I graphed this out on paper the result of removal looked correct and there are no issues such as
+     * this node needing to create a full replica for a range it transiently replicates because what is created is just another
+     * transient replica to replace this node.
+     * @param keyspaceName
+     * @param endpoint
+     * @return
+     */
     // needs to be modified to accept either a keyspace or ARS.
-    private Multimap<Range<Token>, InetAddressAndPort> getChangedRangesForLeaving(String keyspaceName, InetAddressAndPort endpoint)
+    static EndpointsByReplica getChangedReplicasForLeaving(String keyspaceName, InetAddressAndPort endpoint, TokenMetadata tokenMetadata, AbstractReplicationStrategy strat)
     {
         // First get all ranges the leaving endpoint is responsible for
-        Collection<Range<Token>> ranges = getRangesForEndpoint(keyspaceName, endpoint);
+        RangesAtEndpoint replicas = strat.getAddressReplicas(endpoint);
 
         if (logger.isDebugEnabled())
-            logger.debug("Node {} ranges [{}]", endpoint, StringUtils.join(ranges, ", "));
+            logger.debug("Node {} replicas [{}]", endpoint, StringUtils.join(replicas, ", "));
 
-        Map<Range<Token>, List<InetAddressAndPort>> currentReplicaEndpoints = new HashMap<>(ranges.size());
+        Map<Replica, EndpointsForRange> currentReplicaEndpoints = Maps.newHashMapWithExpectedSize(replicas.size());
 
         // Find (for each range) all nodes that store replicas for these ranges as well
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap(); // don't do this in the loop! #7758
-        for (Range<Token> range : ranges)
-            currentReplicaEndpoints.put(range, Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, metadata));
+        for (Replica replica : replicas)
+            currentReplicaEndpoints.put(replica, strat.calculateNaturalReplicas(replica.range().right, metadata));
 
         TokenMetadata temp = tokenMetadata.cloneAfterAllLeft();
 
@@ -2877,26 +2976,43 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (temp.isMember(endpoint))
             temp.removeEndpoint(endpoint);
 
-        Multimap<Range<Token>, InetAddressAndPort> changedRanges = HashMultimap.create();
+        EndpointsByReplica.Mutable changedRanges = new EndpointsByReplica.Mutable();
 
         // Go through the ranges and for each range check who will be
         // storing replicas for these ranges when the leaving endpoint
         // is gone. Whoever is present in newReplicaEndpoints list, but
         // not in the currentReplicaEndpoints list, will be needing the
         // range.
-        for (Range<Token> range : ranges)
-        {
-            Collection<InetAddressAndPort> newReplicaEndpoints = Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
-            newReplicaEndpoints.removeAll(currentReplicaEndpoints.get(range));
+        for (Replica replica : replicas)
+        {
+            EndpointsForRange newReplicaEndpoints = strat.calculateNaturalReplicas(replica.range().right, temp);
+            newReplicaEndpoints = newReplicaEndpoints.filter(newReplica -> {
+                Optional<Replica> currentReplicaOptional =
+                    tryFind(currentReplicaEndpoints.get(replica),
+                            currentReplica -> newReplica.endpoint().equals(currentReplica.endpoint())
+                    ).toJavaUtil();
+                //If it is newly replicating then yes we must do something to get the data there
+                if (!currentReplicaOptional.isPresent())
+                    return true;
+
+                Replica currentReplica = currentReplicaOptional.get();
+                //This transition requires streaming to occur
+                //Full -> transient is handled by nodetool cleanup
+                //transient -> transient and full -> full don't require any action
+                if (currentReplica.isTransient() && newReplica.isFull())
+                    return true;
+                return false;
+            });
+
             if (logger.isDebugEnabled())
                 if (newReplicaEndpoints.isEmpty())
-                    logger.debug("Range {} already in all replicas", range);
+                    logger.debug("Replica {} already in all replicas", replica);
                 else
-                    logger.debug("Range {} will be responsibility of {}", range, StringUtils.join(newReplicaEndpoints, ", "));
-            changedRanges.putAll(range, newReplicaEndpoints);
+                    logger.debug("Replica {} will be responsibility of {}", replica, StringUtils.join(newReplicaEndpoints, ", "));
+            changedRanges.putAll(replica, newReplicaEndpoints, Conflict.NONE);
         }
 
-        return changedRanges;
+        return changedRanges.asImmutableView();
     }
 
     public void onJoin(InetAddressAndPort endpoint, EndpointState epState)
@@ -3602,10 +3718,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             }
             else
             {
-                option.getRanges().addAll(getLocalRanges(keyspace));
+                Iterables.addAll(option.getRanges(), getLocalReplicas(keyspace).filter(Replica::isFull).ranges());
             }
         }
-        if (option.getRanges().isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
+        if (option.getRanges().isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor().allReplicas < 2)
             return 0;
 
         int cmd = nextRepairCommand.incrementAndGet();
@@ -3703,7 +3819,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * Get the "primary ranges" for the specified keyspace and endpoint.
      * "Primary ranges" are the ranges that the node is responsible for storing replica primarily.
      * The node that stores replica primarily is defined as the first node returned
-     * by {@link AbstractReplicationStrategy#calculateNaturalEndpoints}.
+     * by {@link AbstractReplicationStrategy#calculateNaturalReplicas}.
      *
      * @param keyspace Keyspace name to check primary ranges
      * @param ep endpoint we are interested in.
@@ -3716,9 +3832,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
         for (Token token : metadata.sortedTokens())
         {
-            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
-            if (endpoints.size() > 0 && endpoints.get(0).equals(ep))
+            EndpointsForRange replicas = strategy.calculateNaturalReplicas(token, metadata);
+            if (replicas.size() > 0 && replicas.get(0).endpoint().equals(ep))
+            {
+                Preconditions.checkState(replicas.get(0).isFull());
                 primaryRanges.add(new Range<>(metadata.getPredecessor(token), token));
+            }
         }
         return primaryRanges;
     }
@@ -3741,12 +3860,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Collection<Range<Token>> localDCPrimaryRanges = new HashSet<>();
         for (Token token : metadata.sortedTokens())
         {
-            List<InetAddressAndPort> endpoints = strategy.calculateNaturalEndpoints(token, metadata);
-            for (InetAddressAndPort endpoint : endpoints)
+            EndpointsForRange replicas = strategy.calculateNaturalReplicas(token, metadata);
+            for (Replica replica : replicas)
             {
-                if (localDcNodes.contains(endpoint))
+                if (localDcNodes.contains(replica.endpoint()))
                 {
-                    if (endpoint.equals(referenceEndpoint))
+                    if (replica.endpoint().equals(referenceEndpoint))
                     {
                         localDCPrimaryRanges.add(new Range<>(metadata.getPredecessor(token), token));
                     }
@@ -3763,9 +3882,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param ep endpoint we are interested in.
      * @return ranges for the specified endpoint.
      */
-    Collection<Range<Token>> getRangesForEndpoint(String keyspaceName, InetAddressAndPort ep)
+    RangesAtEndpoint getReplicasForEndpoint(String keyspaceName, InetAddressAndPort ep)
     {
-        return Keyspace.open(keyspaceName).getReplicationStrategy().getAddressRanges().get(ep);
+        return Keyspace.open(keyspaceName).getReplicationStrategy().getAddressReplicas(ep);
     }
 
     /**
@@ -3806,40 +3925,53 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     @Deprecated
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key)
     {
-        KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
-        if (ksMetaData == null)
-            throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
-
-        TableMetadata metadata = ksMetaData.getTableOrViewNullable(cf);
-        if (metadata == null)
-            throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
-
-        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key))).stream().map(i -> i.address).collect(toList());
+        EndpointsForToken replicas = getNaturalReplicasForToken(keyspaceName, cf, key);
+        List<InetAddress> inetList = new ArrayList<>(replicas.size());
+        replicas.forEach(r -> inetList.add(r.endpoint().address));
+        return inetList;
     }
 
     public List<String> getNaturalEndpointsWithPort(String keyspaceName, String cf, String key)
     {
-        KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
-        if (ksMetaData == null)
-            throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
-
-        TableMetadata metadata = ksMetaData.getTableOrViewNullable(cf);
-        if (metadata == null)
-            throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
-
-        return stringify(getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key))), true);
+        return Replicas.stringify(getNaturalReplicasForToken(keyspaceName, cf, key), true);
     }
 
 
     @Deprecated
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key)
     {
-        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(key)).stream().map(i -> i.address).collect(toList());
+        EndpointsForToken replicas = getNaturalReplicasForToken(keyspaceName, tokenMetadata.partitioner.getToken(key));
+        List<InetAddress> inetList = new ArrayList<>(replicas.size());
+        replicas.forEach(r -> inetList.add(r.endpoint().address));
+        return inetList;
     }
 
     public List<String> getNaturalEndpointsWithPort(String keyspaceName, ByteBuffer key)
     {
-        return stringify(getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(key)), true);
+        return Replicas.stringify(getNaturalReplicasForToken(keyspaceName, tokenMetadata.partitioner.getToken(key)), true);
+    }
+
+    public List<String> getReplicas(String keyspaceName, String cf, String key)
+    {
+        List<String> res = new ArrayList<>();
+        for (Replica replica : getNaturalReplicasForToken(keyspaceName, cf, key))
+        {
+            res.add(replica.toString());
+        }
+        return res;
+    }
+
+    public EndpointsForToken getNaturalReplicasForToken(String keyspaceName, String cf, String key)
+    {
+        KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (ksMetaData == null)
+            throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
+
+        TableMetadata metadata = ksMetaData.getTableOrViewNullable(cf);
+        if (metadata == null)
+            throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
+
+        return getNaturalReplicasForToken(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key)));
     }
 
     /**
@@ -3850,17 +3982,25 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @param pos position for which we need to find the endpoint
      * @return the endpoint responsible for this token
      */
-    public List<InetAddressAndPort> getNaturalEndpoints(String keyspaceName, RingPosition pos)
+    public static EndpointsForToken getNaturalReplicasForToken(String keyspaceName, RingPosition pos)
     {
-        return Keyspace.open(keyspaceName).getReplicationStrategy().getNaturalEndpoints(pos);
+        return Keyspace.open(keyspaceName).getReplicationStrategy().getNaturalReplicasForToken(pos);
     }
 
     /**
      * Returns the endpoints currently responsible for storing the token plus pending ones
      */
-    public Iterable<InetAddressAndPort> getNaturalAndPendingEndpoints(String keyspaceName, Token token)
+    public EndpointsForToken getNaturalAndPendingReplicasForToken(String keyspaceName, Token token)
     {
-        return Iterables.concat(getNaturalEndpoints(keyspaceName, token), tokenMetadata.pendingEndpointsFor(token, keyspaceName));
+        // TODO: race condition to fetch these. impliciations??
+        EndpointsForToken natural = getNaturalReplicasForToken(keyspaceName, token);
+        EndpointsForToken pending = tokenMetadata.pendingEndpointsForToken(token, keyspaceName);
+        if (Endpoints.haveConflicts(natural, pending))
+        {
+            natural = Endpoints.resolveConflictsInNatural(natural, pending);
+            pending = Endpoints.resolveConflictsInPending(natural, pending);
+        }
+        return Endpoints.concat(natural, pending);
     }
 
     /**
@@ -3868,19 +4008,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * specified key i.e for replication.
      *
      * @param keyspace keyspace name also known as keyspace
-     * @param key key for which we need to find the endpoint
-     * @return the endpoint responsible for this key
+     * @param pos position for which we need to find the endpoint
      */
-    public List<InetAddressAndPort> getLiveNaturalEndpoints(Keyspace keyspace, ByteBuffer key)
-    {
-        return getLiveNaturalEndpoints(keyspace, tokenMetadata.decorateKey(key));
-    }
-
-    public List<InetAddressAndPort> getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos)
+    public EndpointsForToken getLiveNaturalReplicasForToken(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddressAndPort> liveEps = new ArrayList<>();
-        getLiveNaturalEndpoints(keyspace, pos, liveEps);
-        return liveEps;
+        return getLiveNaturalReplicas(keyspace, pos).forToken(pos.getToken());
     }
 
     /**
@@ -3889,17 +4021,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      *
      * @param keyspace keyspace name also known as keyspace
      * @param pos position for which we need to find the endpoint
-     * @param liveEps the list of endpoints to mutate
      */
-    public void getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos, List<InetAddressAndPort> liveEps)
+    public EndpointsForRange getLiveNaturalReplicas(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddressAndPort> endpoints = keyspace.getReplicationStrategy().getNaturalEndpoints(pos);
-
-        for (InetAddressAndPort endpoint : endpoints)
-        {
-            if (FailureDetector.instance.isAlive(endpoint))
-                liveEps.add(endpoint);
-        }
+        EndpointsForRange replicas = keyspace.getReplicationStrategy().getNaturalReplicas(pos);
+        return replicas.filter(r -> FailureDetector.instance.isAlive(r.endpoint()));
     }
 
     public void setLoggingLevel(String classQualifier, String rawLevel) throws Exception
@@ -4019,13 +4145,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                         if (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy)
                         {
                             NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
-                            rf = strategy.getReplicationFactor(dc);
+                            rf = strategy.getReplicationFactor(dc).allReplicas;
                             numNodes = metadata.getTopology().getDatacenterEndpoints().get(dc).size();
                         }
                         else
                         {
                             numNodes = metadata.getAllEndpoints().size();
-                            rf = keyspace.getReplicationStrategy().getReplicationFactor();
+                            rf = keyspace.getReplicationStrategy().getReplicationFactor().allReplicas;
                         }
 
                         if (numNodes <= rf)
@@ -4033,6 +4159,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                                                     + keyspaceName + " (RF = " + rf + ", N = " + numNodes + ")."
                                                                     + " Perform a forceful decommission to ignore.");
                     }
+                    // TODO: do we care about fixing transient/full self-movements here? probably
                     if (tokenMetadata.getPendingRanges(keyspaceName, FBUtilities.getBroadcastAddressAndPort()).size() > 0)
                         throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
                 }
@@ -4095,11 +4222,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     private void unbootstrap(Runnable onFinish) throws ExecutionException, InterruptedException
     {
-        Map<String, Multimap<Range<Token>, InetAddressAndPort>> rangesToStream = new HashMap<>();
+        Map<String, EndpointsByReplica> rangesToStream = new HashMap<>();
 
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
-            Multimap<Range<Token>, InetAddressAndPort> rangesMM = getChangedRangesForLeaving(keyspaceName, FBUtilities.getBroadcastAddressAndPort());
+            EndpointsByReplica rangesMM = getChangedReplicasForLeaving(keyspaceName, FBUtilities.getBroadcastAddressAndPort(), tokenMetadata, Keyspace.open(keyspaceName).getReplicationStrategy());
 
             if (logger.isDebugEnabled())
                 logger.debug("Ranges needing transfer are [{}]", StringUtils.join(rangesMM.keySet(), ","));
@@ -4135,20 +4262,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return HintsService.instance.transferHints(this::getPreferredHintsStreamTarget);
     }
 
+    private static EndpointsForRange getStreamCandidates(Collection<InetAddressAndPort> endpoints)
+    {
+        endpoints = endpoints.stream()
+                             .filter(endpoint -> FailureDetector.instance.isAlive(endpoint) && !FBUtilities.getBroadcastAddressAndPort().equals(endpoint))
+                             .collect(Collectors.toList());
+
+        return EndpointsForRange.copyOf(SystemReplicas.getSystemReplicas(endpoints));
+    }
     /**
      * Find the best target to stream hints to. Currently the closest peer according to the snitch
      */
     private UUID getPreferredHintsStreamTarget()
     {
-        List<InetAddressAndPort> candidates = new ArrayList<>(StorageService.instance.getTokenMetadata().cloneAfterAllLeft().getAllEndpoints());
-        candidates.remove(FBUtilities.getBroadcastAddressAndPort());
-        for (Iterator<InetAddressAndPort> iter = candidates.iterator(); iter.hasNext(); )
-        {
-            InetAddressAndPort address = iter.next();
-            if (!FailureDetector.instance.isAlive(address))
-                iter.remove();
-        }
+        Set<InetAddressAndPort> endpoints = StorageService.instance.getTokenMetadata().cloneAfterAllLeft().getAllEndpoints();
 
+        EndpointsForRange candidates = getStreamCandidates(endpoints);
         if (candidates.isEmpty())
         {
             logger.warn("Unable to stream hints since no live endpoints seen");
@@ -4157,8 +4286,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         else
         {
             // stream to the closest peer as chosen by the snitch
-            DatabaseDescriptor.getEndpointSnitch().sortByProximity(FBUtilities.getBroadcastAddressAndPort(), candidates);
-            InetAddressAndPort hintsDestinationHost = candidates.get(0);
+            candidates = DatabaseDescriptor.getEndpointSnitch().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), candidates);
+            InetAddressAndPort hintsDestinationHost = candidates.get(0).endpoint();
             return tokenMetadata.getHostId(hintsDestinationHost);
         }
     }
@@ -4207,6 +4336,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // checking if data is moving to this node
         for (String keyspaceName : keyspacesToProcess)
         {
+            // TODO: do we care about fixing transient/full self-movements here?
             if (tokenMetadata.getPendingRanges(keyspaceName, localAddress).size() > 0)
                 throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
         }
@@ -4218,7 +4348,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         setMode(Mode.MOVING, String.format("Sleeping %s ms before start streaming/fetching ranges", RING_DELAY), true);
         Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS);
 
-        RangeRelocator relocator = new RangeRelocator(Collections.singleton(newToken), keyspacesToProcess);
+        RangeRelocator relocator = new RangeRelocator(Collections.singleton(newToken), keyspacesToProcess, tokenMetadata);
+        relocator.calculateToFromStreams();
 
         if (relocator.streamsNeeded())
         {
@@ -4243,131 +4374,191 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             logger.debug("Successfully moved to new token {}", getLocalTokens().iterator().next());
     }
 
-    private class RangeRelocator
+    @VisibleForTesting
+    public static class RangeRelocator
     {
         private final StreamPlan streamPlan = new StreamPlan(StreamOperation.RELOCATION);
-
-        private RangeRelocator(Collection<Token> tokens, List<String> keyspaceNames)
-        {
-            calculateToFromStreams(tokens, keyspaceNames);
-        }
-
-        private void calculateToFromStreams(Collection<Token> newTokens, List<String> keyspaceNames)
-        {
-            InetAddressAndPort localAddress = FBUtilities.getBroadcastAddressAndPort();
-            IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-            TokenMetadata tokenMetaCloneAllSettled = tokenMetadata.cloneAfterAllSettled();
-            // clone to avoid concurrent modification in calculateNaturalEndpoints
-            TokenMetadata tokenMetaClone = tokenMetadata.cloneOnlyTokenMap();
-
-            for (String keyspace : keyspaceNames)
+        private final InetAddressAndPort localAddress = FBUtilities.getBroadcastAddressAndPort();
+        private final TokenMetadata tokenMetaCloneAllSettled;
+        // clone to avoid concurrent modification in calculateNaturalReplicas
+        private final TokenMetadata tokenMetaClone;
+        private final Collection<Token> tokens;
+        private final List<String> keyspaceNames;
+
+
+        private RangeRelocator(Collection<Token> tokens, List<String> keyspaceNames, TokenMetadata tmd)
+        {
+            this.tokens = tokens;
+            this.keyspaceNames = keyspaceNames;
+            this.tokenMetaCloneAllSettled = tmd.cloneAfterAllSettled();
+            // clone to avoid concurrent modification in calculateNaturalReplicas
+            this.tokenMetaClone = tmd.cloneOnlyTokenMap();
+        }
+
+        @VisibleForTesting
+        public RangeRelocator()
+        {
+            this.tokens = null;
+            this.keyspaceNames = null;
+            this.tokenMetaCloneAllSettled = null;
+            this.tokenMetaClone = null;
+        }
+
+        /**
+         * Wrapper that supplies accessors to the real implementations of the various dependencies for this method
+         */
+        private Multimap<InetAddressAndPort, FetchReplica> calculateRangesToFetchWithPreferredEndpoints(AbstractReplicationStrategy strategy, RangesAtEndpoint fetchRanges, String keyspace)
+        {
+            EndpointsByReplica preferredEndpoints =
+            RangeStreamer.calculateRangesToFetchWithPreferredEndpoints((address, replicas) -> DatabaseDescriptor.getEndpointSnitch().sortedByProximity(address, replicas),
+                                                                       strategy,
+                                                                       fetchRanges,
+                                                                       useStrictConsistency,
+                                                                       tokenMetaClone,
+                                                                       tokenMetaCloneAllSettled,
+                                                                       RangeStreamer.ALIVE_PREDICATE,
+                                                                       keyspace,
+                                                                       Collections.emptyList());
+            return RangeStreamer.convertPreferredEndpointsToWorkMap(preferredEndpoints);
+        }
+
+        /**
+         * calculating endpoints to stream current ranges to if needed
+         * in some situations node will handle current ranges as part of the new ranges
+         **/
+        public RangesByEndpoint calculateRangesToStreamWithEndpoints(RangesAtEndpoint streamRanges,
+                                                                     AbstractReplicationStrategy strat,
+                                                                     TokenMetadata tmdBefore,
+                                                                     TokenMetadata tmdAfter)
+        {
+            RangesByEndpoint.Mutable endpointRanges = new RangesByEndpoint.Mutable();
+            for (Replica toStream : streamRanges)
             {
-                // replication strategy of the current keyspace
-                AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
-                Multimap<InetAddressAndPort, Range<Token>> endpointToRanges = strategy.getAddressRanges();
-
-                logger.debug("Calculating ranges to stream and request for keyspace {}", keyspace);
-                for (Token newToken : newTokens)
+                //If the range we are sending is full only send it to the new full replica
+                //There will also be a new transient replica we need to send the data to, but not
+                //the repaired data
+                EndpointsForRange currentEndpoints = strat.calculateNaturalReplicas(toStream.range().right, tmdBefore);
+                EndpointsForRange newEndpoints = strat.calculateNaturalReplicas(toStream.range().right, tmdAfter);
+                logger.debug("Need to stream {}, current endpoints {}, new endpoints {}", toStream, currentEndpoints, newEndpoints);
+
+                for (Replica current : currentEndpoints)
                 {
-                    // getting collection of the currently used ranges by this keyspace
-                    Collection<Range<Token>> currentRanges = endpointToRanges.get(localAddress);
-                    // collection of ranges which this node will serve after move to the new token
-                    Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetaClone, newToken, localAddress);
-
-                    // ring ranges and endpoints associated with them
-                    // this used to determine what nodes should we ping about range data
-                    Multimap<Range<Token>, InetAddressAndPort> rangeAddresses = strategy.getRangeAddresses(tokenMetaClone);
-
-                    // calculated parts of the ranges to request/stream from/to nodes in the ring
-                    Pair<Set<Range<Token>>, Set<Range<Token>>> rangesPerKeyspace = calculateStreamAndFetchRanges(currentRanges, updatedRanges);
-
-                    /**
-                     * In this loop we are going through all ranges "to fetch" and determining
-                     * nodes in the ring responsible for data we are interested in
-                     */
-                    Multimap<Range<Token>, InetAddressAndPort> rangesToFetchWithPreferredEndpoints = ArrayListMultimap.create();
-                    for (Range<Token> toFetch : rangesPerKeyspace.right)
+                    for (Replica updated : newEndpoints)
                     {
-                        for (Range<Token> range : rangeAddresses.keySet())
+                        if (current.endpoint().equals(updated.endpoint()))
                         {
-                            if (range.contains(toFetch))
-                            {
-                                List<InetAddressAndPort> endpoints = null;
+                            //Nothing to do
+                            if (current.equals(updated))
+                                break;
 
-                                if (useStrictConsistency)
+                            //In these two (really three) cases the existing data is sufficient and we should subtract whatever is already replicated
+                            if (current.isFull() == updated.isFull() || current.isFull())
+                            {
+                                //First subtract what we already have
+                                Set<Range<Token>> subsToStream = toStream.range().subtract(current.range());
+                                //Now we only stream what is still replicated
+                                subsToStream = subsToStream.stream().flatMap(range -> range.intersectionWith(updated.range()).stream()).collect(Collectors.toSet());
+                                for (Range<Token> subrange : subsToStream)
                                 {
-                                    Set<InetAddressAndPort> oldEndpoints = Sets.newHashSet(rangeAddresses.get(range));
-                                    Set<InetAddressAndPort> newEndpoints = Sets.newHashSet(strategy.calculateNaturalEndpoints(toFetch.right, tokenMetaCloneAllSettled));
-
-                                    //Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
-                                    //So we need to be careful to only be strict when endpoints == RF
-                                    if (oldEndpoints.size() == strategy.getReplicationFactor())
+                                    //Only stream what intersects with what is in the new world
+                                    Set<Range<Token>> intersections = subrange.intersectionWith(updated.range());
+                                    for (Range<Token> intersection : intersections)
                                     {
-                                        oldEndpoints.removeAll(newEndpoints);
-
-                                        //No relocation required
-                                        if (oldEndpoints.isEmpty())
-                                            continue;
-
-                                        assert oldEndpoints.size() == 1 : "Expected 1 endpoint but found " + oldEndpoints.size();
+                                        endpointRanges.put(updated.endpoint(), updated.decorateSubrange(intersection));
                                     }
-
-                                    endpoints = Lists.newArrayList(oldEndpoints.iterator().next());
                                 }
-                                else
+                            }
+                            else
+                            {
+                                for (Range<Token> intersection : toStream.range().intersectionWith(updated.range()))
                                 {
-                                    endpoints = snitch.getSortedListByProximity(localAddress, rangeAddresses.get(range));
+                                    endpointRanges.put(updated.endpoint(), updated.decorateSubrange(intersection));
                                 }
-
-                                // storing range and preferred endpoint set
-                                rangesToFetchWithPreferredEndpoints.putAll(toFetch, endpoints);
                             }
                         }
+                    }
+                }
 
-                        Collection<InetAddressAndPort> addressList = rangesToFetchWithPreferredEndpoints.get(toFetch);
-                        if (addressList == null || addressList.isEmpty())
-                            continue;
-
-                        if (useStrictConsistency)
+                for (Replica updated : newEndpoints)
+                {
+                    if (!currentEndpoints.byEndpoint().containsKey(updated.endpoint()))
+                    {
+                        // Completely new range for this endpoint
+                        if (toStream.isTransient() && updated.isFull())
                         {
-                            if (addressList.size() > 1)
-                                throw new IllegalStateException("Multiple strict sources found for " + toFetch);
-
-                            InetAddressAndPort sourceIp = addressList.iterator().next();
-                            if (Gossiper.instance.isEnabled() && !Gossiper.instance.getEndpointStateForEndpoint(sourceIp).isAlive())
-                                throw new RuntimeException("A node required to move the data consistently is down ("+sourceIp+").  If you wish to move the data from a potentially inconsistent replica, restart the node with -Dcassandra.consistent.rangemovement=false");
+                            throw new AssertionError(String.format("Need to stream %s, but only have %s which is transient and not full", updated, toStream));
+                        }
+                        for (Range<Token> intersection : updated.range().intersectionWith(toStream.range()))
+                        {
+                            endpointRanges.put(updated.endpoint(), updated.decorateSubrange(intersection));
                         }
                     }
+                }
+            }
+            return endpointRanges.asImmutableView();
+        }
 
-                    // calculating endpoints to stream current ranges to if needed
-                    // in some situations node will handle current ranges as part of the new ranges
-                    Multimap<InetAddressAndPort, Range<Token>> endpointRanges = HashMultimap.create();
-                    for (Range<Token> toStream : rangesPerKeyspace.left)
+        private void calculateToFromStreams()
+        {
+            logger.debug("Current tmd " + tokenMetaClone);
+            logger.debug("Updated tmd " + tokenMetaCloneAllSettled);
+            for (String keyspace : keyspaceNames)
+            {
+                // replication strategy of the current keyspace
+                AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
+                // getting collection of the currently used ranges by this keyspace
+                RangesAtEndpoint currentReplicas = strategy.getAddressReplicas(localAddress);
+
+                logger.info("Calculating ranges to stream and request for keyspace {}", keyspace);
+                //From what I have seen we only ever call this with a single token from StorageService.move(Token)
+                for (Token newToken : tokens)
+                {
+                    Collection<Token> currentTokens = tokenMetaClone.getTokens(localAddress);
+                    if (currentTokens.size() > 1 || currentTokens.isEmpty())
                     {
-                        Set<InetAddressAndPort> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaClone));
-                        Set<InetAddressAndPort> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaCloneAllSettled));
-                        logger.debug("Range: {} Current endpoints: {} New endpoints: {}", toStream, currentEndpoints, newEndpoints);
-                        for (InetAddressAndPort address : Sets.difference(newEndpoints, currentEndpoints))
-                        {
-                            logger.debug("Range {} has new owner {}", toStream, address);
-                            endpointRanges.put(address, toStream);
-                        }
+                        throw new AssertionError("Unexpected current tokens: " + currentTokens);
                     }
 
+                    // collection of ranges which this node will serve after move to the new token
+                    RangesAtEndpoint updatedReplicas = strategy.getPendingAddressRanges(tokenMetaClone, newToken, localAddress);
+
+                    // calculated parts of the ranges to request/stream from/to nodes in the ring
+                    Pair<RangesAtEndpoint, RangesAtEndpoint> streamAndFetchOwnRanges = Pair.create(RangesAtEndpoint.empty(localAddress), RangesAtEndpoint.empty(localAddress));
+                    //In the single node token move there is nothing to do and Range subtraction is broken
+                    //so it's easier to just identify this case up front.
+                    if (tokenMetaClone.getTopology().getDatacenterEndpoints().get(DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort()
+)).size() > 1)
+                    {
+                        streamAndFetchOwnRanges = calculateStreamAndFetchRanges(currentReplicas, updatedReplicas);
+                    }
+
+                    Multimap<InetAddressAndPort, FetchReplica> workMap = calculateRangesToFetchWithPreferredEndpoints(strategy, streamAndFetchOwnRanges.right, keyspace);
+
+                    RangesByEndpoint endpointRanges = calculateRangesToStreamWithEndpoints(streamAndFetchOwnRanges.left, strategy, tokenMetaClone, tokenMetaCloneAllSettled);
+
+                    logger.info("Endpoint ranges to stream to " + endpointRanges);
+
                     // stream ranges
                     for (InetAddressAndPort address : endpointRanges.keySet())
                     {
                         logger.debug("Will stream range {} of keyspace {} to endpoint {}", endpointRanges.get(address), keyspace, address);
-                        streamPlan.transferRanges(address, keyspace, endpointRanges.get(address));
+                        RangesAtEndpoint ranges = endpointRanges.get(address);
+                        streamPlan.transferRanges(address, keyspace, ranges);
                     }
 
                     // stream requests
-                    Multimap<InetAddressAndPort, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints, keyspace, FailureDetector.instance, useStrictConsistency);
-                    for (InetAddressAndPort address : workMap.keySet())
-                    {
+                    workMap.asMap().forEach((address, sourceAndOurReplicas) -> {
+                        RangesAtEndpoint full = sourceAndOurReplicas.stream()
+                                .filter(pair -> pair.remote.isFull())
+                                .map(pair -> pair.local)
+                                .collect(RangesAtEndpoint.collector(localAddress));
+                        RangesAtEndpoint transientReplicas = sourceAndOurReplicas.stream()
+                                .filter(pair -> pair.remote.isTransient())
+                                .map(pair -> pair.local)
+                                .collect(RangesAtEndpoint.collector(localAddress));
                         logger.debug("Will request range {} of keyspace {} from endpoint {}", workMap.get(address), keyspace, address);
-                        streamPlan.requestRanges(address, keyspace, workMap.get(address));
-                    }
+                        streamPlan.requestRanges(address, keyspace, full, transientReplicas);
+                    });
 
                     logger.debug("Keyspace {}: work map {}.", keyspace, workMap);
                 }
@@ -4486,14 +4677,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         for (String keyspaceName : Schema.instance.getNonLocalStrategyKeyspaces())
         {
             // if the replication factor is 1 the data is lost so we shouldn't wait for confirmation
-            if (Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() == 1)
+            if (Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor().allReplicas == 1)
                 continue;
 
             // get all ranges that change ownership (that is, a node needs
             // to take responsibility for new range)
-            Multimap<Range<Token>, InetAddressAndPort> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
+            EndpointsByReplica changedRanges = getChangedReplicasForLeaving(keyspaceName, endpoint, tokenMetadata, Keyspace.open(keyspaceName).getReplicationStrategy());
             IFailureDetector failureDetector = FailureDetector.instance;
-            for (InetAddressAndPort ep : changedRanges.values())
+            for (InetAddressAndPort ep : transform(changedRanges.flattenValues(), Replica::endpoint))
             {
                 if (failureDetector.isAlive(ep))
                     replicatingNodes.add(ep);
@@ -4903,15 +5094,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         Collection<Collection<InetAddressAndPort>> endpointsGroupedByDc = new ArrayList<>();
         // mapping of dc's to nodes, use sorted map so that we get dcs sorted
-        SortedMap<String, Collection<InetAddressAndPort>> sortedDcsToEndpoints = new TreeMap<>();
-        sortedDcsToEndpoints.putAll(metadata.getTopology().getDatacenterEndpoints().asMap());
+        SortedMap<String, Collection<InetAddressAndPort>> sortedDcsToEndpoints = new TreeMap<>(metadata.getTopology().getDatacenterEndpoints().asMap());
         for (Collection<InetAddressAndPort> endpoints : sortedDcsToEndpoints.values())
             endpointsGroupedByDc.add(endpoints);
 
         Map<Token, Float> tokenOwnership = tokenMetadata.partitioner.describeOwnership(tokenMetadata.sortedTokens());
         LinkedHashMap<InetAddressAndPort, Float> finalOwnership = Maps.newLinkedHashMap();
 
-        Multimap<InetAddressAndPort, Range<Token>> endpointToRanges = strategy.getAddressRanges();
+        RangesByEndpoint endpointToRanges = strategy.getAddressReplicas();
         // calculate ownership per dc
         for (Collection<InetAddressAndPort> endpoints : endpointsGroupedByDc)
         {
@@ -4919,10 +5109,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             for (InetAddressAndPort endpoint : endpoints)
             {
                 float ownership = 0.0f;
-                for (Range<Token> range : endpointToRanges.get(endpoint))
+                for (Replica replica : endpointToRanges.get(endpoint))
                 {
-                    if (tokenOwnership.containsKey(range.right))
-                        ownership += tokenOwnership.get(range.right);
+                    if (tokenOwnership.containsKey(replica.range().right))
+                        ownership += tokenOwnership.get(replica.range().right);
                 }
                 finalOwnership.put(endpoint, ownership);
             }
@@ -4974,9 +5164,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             UUID hostId = entry.getValue();
             InetAddressAndPort endpoint = entry.getKey();
             result.put(endpoint.toString(withPort),
-                       coreViewStatus.containsKey(hostId)
-                       ? coreViewStatus.get(hostId)
-                       : "UNKNOWN");
+                       coreViewStatus.getOrDefault(hostId, "UNKNOWN"));
         }
 
         return Collections.unmodifiableMap(result);
@@ -5079,69 +5267,63 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     /**
-     * Seed data to the endpoints that will be responsible for it at the future
+     * Send data to the endpoints that will be responsible for it in the future
      *
      * @param rangesToStreamByKeyspace keyspaces and data ranges with endpoints included for each
      * @return async Future for whether stream was success
      */
-    private Future<StreamState> streamRanges(Map<String, Multimap<Range<Token>, InetAddressAndPort>> rangesToStreamByKeyspace)
+    private Future<StreamState> streamRanges(Map<String, EndpointsByReplica> rangesToStreamByKeyspace)
     {
         // First, we build a list of ranges to stream to each host, per table
-        Map<String, Map<InetAddressAndPort, List<Range<Token>>>> sessionsToStreamByKeyspace = new HashMap<>();
+        Map<String, RangesByEndpoint> sessionsToStreamByKeyspace = new HashMap<>();
 
-        for (Map.Entry<String, Multimap<Range<Token>, InetAddressAndPort>> entry : rangesToStreamByKeyspace.entrySet())
+        for (Map.Entry<String, EndpointsByReplica> entry : rangesToStreamByKeyspace.entrySet())
         {
             String keyspace = entry.getKey();
-            Multimap<Range<Token>, InetAddressAndPort> rangesWithEndpoints = entry.getValue();
+            EndpointsByReplica rangesWithEndpoints = entry.getValue();
 
             if (rangesWithEndpoints.isEmpty())
                 continue;
 
+            //Description is always Unbootstrap? Is that right?
             Map<InetAddressAndPort, Set<Range<Token>>> transferredRangePerKeyspace = SystemKeyspace.getTransferredRanges("Unbootstrap",
                                                                                                                          keyspace,
                                                                                                                          StorageService.instance.getTokenMetadata().partitioner);
-            Map<InetAddressAndPort, List<Range<Token>>> rangesPerEndpoint = new HashMap<>();
-            for (Map.Entry<Range<Token>, InetAddressAndPort> endPointEntry : rangesWithEndpoints.entries())
+            RangesByEndpoint.Mutable replicasPerEndpoint = new RangesByEndpoint.Mutable();
+            for (Map.Entry<Replica, Replica> endPointEntry : rangesWithEndpoints.flattenEntries())
             {
-                Range<Token> range = endPointEntry.getKey();
-                InetAddressAndPort endpoint = endPointEntry.getValue();
-
-                Set<Range<Token>> transferredRanges = transferredRangePerKeyspace.get(endpoint);
-                if (transferredRanges != null && transferredRanges.contains(range))
+                Replica local = endPointEntry.getKey();
+                Replica remote = endPointEntry.getValue();
+                Set<Range<Token>> transferredRanges = transferredRangePerKeyspace.get(remote.endpoint());
+                if (transferredRanges != null && transferredRanges.contains(local.range()))
                 {
-                    logger.debug("Skipping transferred range {} of keyspace {}, endpoint {}", range, keyspace, endpoint);
+                    logger.debug("Skipping transferred range {} of keyspace {}, endpoint {}", local, keyspace, remote);
                     continue;
                 }
 
-                List<Range<Token>> curRanges = rangesPerEndpoint.get(endpoint);
-                if (curRanges == null)
-                {
-                    curRanges = new LinkedList<>();
-                    rangesPerEndpoint.put(endpoint, curRanges);
-                }
-                curRanges.add(range);
+                replicasPerEndpoint.put(remote.endpoint(), remote.decorateSubrange(local.range()));
             }
 
-            sessionsToStreamByKeyspace.put(keyspace, rangesPerEndpoint);
+            sessionsToStreamByKeyspace.put(keyspace, replicasPerEndpoint.asImmutableView());
         }
 
         StreamPlan streamPlan = new StreamPlan(StreamOperation.DECOMMISSION);
 
-        // Vinculate StreamStateStore to current StreamPlan to update transferred ranges per StreamSession
+        // Vinculate StreamStateStore to current StreamPlan to update transferred rangeas per StreamSession
         streamPlan.listeners(streamStateStore);
 
-        for (Map.Entry<String, Map<InetAddressAndPort, List<Range<Token>>>> entry : sessionsToStreamByKeyspace.entrySet())
+        for (Map.Entry<String, RangesByEndpoint> entry : sessionsToStreamByKeyspace.entrySet())
         {
             String keyspaceName = entry.getKey();
-            Map<InetAddressAndPort, List<Range<Token>>> rangesPerEndpoint = entry.getValue();
+            RangesByEndpoint replicasPerEndpoint = entry.getValue();
 
-            for (Map.Entry<InetAddressAndPort, List<Range<Token>>> rangesEntry : rangesPerEndpoint.entrySet())
+            for (Map.Entry<InetAddressAndPort, RangesAtEndpoint> rangesEntry : replicasPerEndpoint.asMap().entrySet())
             {
-                List<Range<Token>> ranges = rangesEntry.getValue();
+                RangesAtEndpoint replicas = rangesEntry.getValue();
                 InetAddressAndPort newEndpoint = rangesEntry.getKey();
 
                 // TODO each call to transferRanges re-flushes, this is potentially a lot of waste
-                streamPlan.transferRanges(newEndpoint, keyspaceName, ranges);
+                streamPlan.transferRanges(newEndpoint, keyspaceName, replicas);
             }
         }
         return streamPlan.execute();
@@ -5151,53 +5333,109 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * Calculate pair of ranges to stream/fetch for given two range collections
      * (current ranges for keyspace and ranges after move to new token)
      *
+     * With transient replication the added wrinkle is that if a range transitions from full to transient then
+     * we need to stream the range despite the fact that we are retaining it as transient. Some replica
+     * somewhere needs to transition from transient to full and we wll be the source.
+     *
+     * If the range is transient and is transitioning to full then always fetch even if the range was already transient
+     * since a transiently replicated obviously needs to fetch data to become full.
+     *
+     * This why there is a continue after checking for instersection because intersection is not sufficient reason
+     * to do the subtraction since we might need to stream/fetch data anyways.
+     *
      * @param current collection of the ranges by current token
      * @param updated collection of the ranges after token is changed
      * @return pair of ranges to stream/fetch for given current and updated range collections
      */
-    public Pair<Set<Range<Token>>, Set<Range<Token>>> calculateStreamAndFetchRanges(Collection<Range<Token>> current, Collection<Range<Token>> updated)
+    public static Pair<RangesAtEndpoint, RangesAtEndpoint> calculateStreamAndFetchRanges(RangesAtEndpoint current, RangesAtEndpoint updated)
     {
-        Set<Range<Token>> toStream = new HashSet<>();
-        Set<Range<Token>> toFetch  = new HashSet<>();
-
+        // FIXME: transient replication
+        // this should always be the local node, except for tests TODO: assert this
+        RangesAtEndpoint.Builder toStream = RangesAtEndpoint.builder(current.endpoint());
+        RangesAtEndpoint.Builder toFetch  = RangesAtEndpoint.builder(current.endpoint());
 
-        for (Range<Token> r1 : current)
+        logger.debug("Calculating toStream");
+        for (Replica r1 : current)
         {
             boolean intersect = false;
-            for (Range<Token> r2 : updated)
+            RangesAtEndpoint.Mutable remainder = null;
+            for (Replica r2 : updated)
             {
-                if (r1.intersects(r2))
+                logger.debug("Comparing {} and {}", r1, r2);
+                //If we will end up transiently replicating send the entire thing and don't subtract
+                if (r1.intersectsOnRange(r2) && !(r1.isFull() && r2.isTransient()))
                 {
-                    // adding difference ranges to fetch from a ring
-                    toStream.addAll(r1.subtract(r2));
+                    RangesAtEndpoint.Mutable oldRemainder = remainder;
+                    remainder = new RangesAtEndpoint.Mutable(current.endpoint());
+                    if (oldRemainder != null)
+                    {
+                        for (Replica replica : oldRemainder)
+                        {
+                            remainder.addAll(replica.subtractIgnoreTransientStatus(r2.range()));
+                        }
+                    }
+                    else
+                    {
+                        remainder.addAll(r1.subtractIgnoreTransientStatus(r2.range()));
+                    }
+                    logger.debug("    Intersects adding {}", remainder);
                     intersect = true;
                 }
             }
             if (!intersect)
             {
-                toStream.add(r1); // should seed whole old range
+                logger.debug("    Doesn't intersect adding {}", r1);
+                toStream.add(r1); // should stream whole old range
+            }
+            else
+            {
+                toStream.addAll(remainder);
             }
         }
 
-        for (Range<Token> r2 : updated)
+        logger.debug("Calculating toFetch");
+        for (Replica r2 : updated)
         {
             boolean intersect = false;
-            for (Range<Token> r1 : current)
+            RangesAtEndpoint.Mutable remainder = null;
+            for (Replica r1 : current)
             {
-                if (r2.intersects(r1))
+                logger.info("Comparing {} and {}", r2, r1);
+                //Transitioning from transient to full means fetch everything so intersection doesn't matter.
+                if (r2.intersectsOnRange(r1) && !(r1.isTransient() && r2.isFull()))
                 {
-                    // adding difference ranges to fetch from a ring
-                    toFetch.addAll(r2.subtract(r1));
+                    RangesAtEndpoint.Mutable oldRemainder = remainder;
+                    remainder = new RangesAtEndpoint.Mutable(current.endpoint());
+                    if (oldRemainder != null)
+                    {
+                        for (Replica replica : oldRemainder)
+                        {
+                            remainder.addAll(replica.subtractIgnoreTransientStatus(r1.range()));
+                        }
+                    }
+                    else
+                    {
+                        remainder.addAll(r2.subtractIgnoreTransientStatus(r1.range()));
+                    }
+                    logger.debug("    Intersects adding {}", remainder);
                     intersect = true;
                 }
             }
             if (!intersect)
             {
+                logger.debug("    Doesn't intersect adding {}", r2);
                 toFetch.add(r2); // should fetch whole old range
             }
+            else
+            {
+                toFetch.addAll(remainder);
+            }
         }
 
-        return Pair.create(toStream, toFetch);
+        logger.debug("To stream {}", toStream);
+        logger.debug("To fetch {}", toFetch);
+
+        return Pair.create(toStream.build(), toFetch.build());
     }
 
     public void bulkLoad(String directory)
@@ -5233,10 +5471,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 this.keyspace = keyspace;
                 try
                 {
-                    for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : StorageService.instance.getRangeToAddressMap(keyspace).entrySet())
+                    for (Map.Entry<Range<Token>, EndpointsForRange> entry : StorageService.instance.getRangeToAddressMap(keyspace).entrySet())
                     {
                         Range<Token> range = entry.getKey();
-                        for (InetAddressAndPort endpoint : entry.getValue())
+                        EndpointsForRange replicas = entry.getValue();
+                        Replicas.temporaryAssertFull(replicas);
+                        for (InetAddressAndPort endpoint : replicas.endpoints())
                             addRangeForEndpoint(range, endpoint);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 0f4c7dd..4e6295a 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -217,6 +217,8 @@ public interface StorageServiceMBean extends NotificationEmitter
     @Deprecated public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key);
     public List<String> getNaturalEndpointsWithPort(String keysapceName, ByteBuffer key);
 
+    public List<String> getReplicas(String keyspaceName, String cf, String key);
+
     /**
      * @deprecated use {@link #takeSnapshot(String tag, Map options, String... entities)} instead.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/WriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/WriteResponseHandler.java b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
index 65efeff..a07aae6 100644
--- a/src/java/org/apache/cassandra/service/WriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
@@ -17,18 +17,15 @@
  */
 package org.apache.cassandra.service;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.ReplicaLayout;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
 
 /**
@@ -42,26 +39,18 @@ public class WriteResponseHandler<T> extends AbstractWriteResponseHandler<T>
     private static final AtomicIntegerFieldUpdater<WriteResponseHandler> responsesUpdater
             = AtomicIntegerFieldUpdater.newUpdater(WriteResponseHandler.class, "responses");
 
-    public WriteResponseHandler(Collection<InetAddressAndPort> writeEndpoints,
-                                Collection<InetAddressAndPort> pendingEndpoints,
-                                ConsistencyLevel consistencyLevel,
-                                Keyspace keyspace,
+    public WriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                 Runnable callback,
                                 WriteType writeType,
                                 long queryStartNanoTime)
     {
-        super(keyspace, writeEndpoints, pendingEndpoints, consistencyLevel, callback, writeType, queryStartNanoTime);
+        super(replicaLayout, callback, writeType, queryStartNanoTime);
         responses = totalBlockFor();
     }
 
-    public WriteResponseHandler(InetAddressAndPort endpoint, WriteType writeType, Runnable callback, long queryStartNanoTime)
-    {
-        this(Arrays.asList(endpoint), Collections.<InetAddressAndPort>emptyList(), ConsistencyLevel.ONE, null, callback, writeType, queryStartNanoTime);
-    }
-
-    public WriteResponseHandler(InetAddressAndPort endpoint, WriteType writeType, long queryStartNanoTime)
+    public WriteResponseHandler(ReplicaLayout.ForToken replicaLayout, WriteType writeType, long queryStartNanoTime)
     {
-        this(endpoint, writeType, null, queryStartNanoTime);
+        this(replicaLayout, null, writeType, queryStartNanoTime);
     }
 
     public void response(MessageIn<T> m)


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


[02/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/StorageServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceTest.java b/test/unit/org/apache/cassandra/service/StorageServiceTest.java
new file mode 100644
index 0000000..9d5c324
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/StorageServiceTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.service;
+
+import org.apache.cassandra.locator.EndpointsByReplica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaMultimap;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.locator.TokenMetadata;
+
+import static org.junit.Assert.assertEquals;
+
+public class StorageServiceTest
+{
+    static InetAddressAndPort aAddress;
+    static InetAddressAndPort bAddress;
+    static InetAddressAndPort cAddress;
+    static InetAddressAndPort dAddress;
+    static InetAddressAndPort eAddress;
+
+    @BeforeClass
+    public static void setUpClass() throws Exception
+    {
+        aAddress = InetAddressAndPort.getByName("127.0.0.1");
+        bAddress = InetAddressAndPort.getByName("127.0.0.2");
+        cAddress = InetAddressAndPort.getByName("127.0.0.3");
+        dAddress = InetAddressAndPort.getByName("127.0.0.4");
+        eAddress = InetAddressAndPort.getByName("127.0.0.5");
+    }
+
+    private static final Token threeToken = new RandomPartitioner.BigIntegerToken("3");
+    private static final Token sixToken = new RandomPartitioner.BigIntegerToken("6");
+    private static final Token nineToken = new RandomPartitioner.BigIntegerToken("9");
+    private static final Token elevenToken = new RandomPartitioner.BigIntegerToken("11");
+    private static final Token oneToken = new RandomPartitioner.BigIntegerToken("1");
+
+    Range<Token> aRange = new Range<>(oneToken, threeToken);
+    Range<Token> bRange = new Range<>(threeToken, sixToken);
+    Range<Token> cRange = new Range<>(sixToken, nineToken);
+    Range<Token> dRange = new Range<>(nineToken, elevenToken);
+    Range<Token> eRange = new Range<>(elevenToken, oneToken);
+
+    @Before
+    public void setUp()
+    {
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        IEndpointSnitch snitch = new AbstractEndpointSnitch()
+        {
+            public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2)
+            {
+                return 0;
+            }
+
+            public String getRack(InetAddressAndPort endpoint)
+            {
+                return "R1";
+            }
+
+            public String getDatacenter(InetAddressAndPort endpoint)
+            {
+                return "DC1";
+            }
+        };
+
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+    }
+
+    private AbstractReplicationStrategy simpleStrategy(TokenMetadata tmd)
+    {
+        return new SimpleStrategy("MoveTransientTest",
+                                  tmd,
+                                  DatabaseDescriptor.getEndpointSnitch(),
+                                  com.google.common.collect.ImmutableMap.of("replication_factor", "3/1"));
+    }
+
+    public static <K, C extends ReplicaCollection<? extends C>>  void assertMultimapEqualsIgnoreOrder(ReplicaMultimap<K, C> a, ReplicaMultimap<K, C> b)
+    {
+        if (!a.keySet().equals(b.keySet()))
+            assertEquals(a, b);
+        for (K key : a.keySet())
+        {
+            C ac = a.get(key);
+            C bc = b.get(key);
+            if (ac.size() != bc.size())
+                assertEquals(a, b);
+            for (Replica r : ac)
+            {
+                if (!bc.contains(r))
+                    assertEquals(a, b);
+            }
+        }
+    }
+
+    @Test
+    public void testGetChangedReplicasForLeaving() throws Exception
+    {
+        TokenMetadata tmd = new TokenMetadata();
+        tmd.updateNormalToken(threeToken, aAddress);
+        tmd.updateNormalToken(sixToken, bAddress);
+        tmd.updateNormalToken(nineToken, cAddress);
+        tmd.updateNormalToken(elevenToken, dAddress);
+        tmd.updateNormalToken(oneToken, eAddress);
+
+        tmd.addLeavingEndpoint(aAddress);
+
+        AbstractReplicationStrategy strat = simpleStrategy(tmd);
+
+        EndpointsByReplica result = StorageService.getChangedReplicasForLeaving("StorageServiceTest", aAddress, tmd, strat);
+        System.out.println(result);
+        EndpointsByReplica.Mutable expectedResult = new EndpointsByReplica.Mutable();
+        expectedResult.put(new Replica(aAddress, aRange, true), new Replica(cAddress, new Range<>(oneToken, sixToken), true));
+        expectedResult.put(new Replica(aAddress, aRange, true), new Replica(dAddress, new Range<>(oneToken, sixToken), false));
+        expectedResult.put(new Replica(aAddress, eRange, true), new Replica(bAddress, eRange, true));
+        expectedResult.put(new Replica(aAddress, eRange, true), new Replica(cAddress, eRange, false));
+        expectedResult.put(new Replica(aAddress, dRange, false), new Replica(bAddress, dRange, false));
+        assertMultimapEqualsIgnoreOrder(result, expectedResult.asImmutableView());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
index f8567e8..cf1e06a 100644
--- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
+++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java
@@ -20,12 +20,14 @@ package org.apache.cassandra.service;
 
 
 import java.net.InetAddress;
-import java.util.Collection;
-import java.util.List;
+import java.net.UnknownHostException;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.ImmutableList;
+import com.google.common.base.Predicates;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -38,9 +40,13 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -49,12 +55,26 @@ public class WriteResponseHandlerTest
 {
     static Keyspace ks;
     static ColumnFamilyStore cfs;
-    static List<InetAddressAndPort> targets;
+    static EndpointsForToken targets;
+    static EndpointsForToken pending;
+
+    private static Replica full(String name)
+    {
+        try
+        {
+            return ReplicaUtils.full(InetAddressAndPort.getByName(name));
+        }
+        catch (UnknownHostException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
 
     @BeforeClass
     public static void setUpClass() throws Throwable
     {
         SchemaLoader.loadSchema();
+        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
         // Register peers with expected DC for NetworkTopologyStrategy.
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
@@ -77,17 +97,12 @@ public class WriteResponseHandlerTest
                     return "datacenter2";
             }
 
-            public List<InetAddressAndPort> getSortedListByProximity(InetAddressAndPort address, Collection<InetAddressAndPort> unsortedAddress)
-            {
-                return null;
-            }
-
-            public void sortByProximity(InetAddressAndPort address, List<InetAddressAndPort> addresses)
+            public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C replicas)
             {
-
+                return replicas;
             }
 
-            public int compareEndpoints(InetAddressAndPort target, InetAddressAndPort a1, InetAddressAndPort a2)
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
             {
                 return 0;
             }
@@ -97,7 +112,7 @@ public class WriteResponseHandlerTest
 
             }
 
-            public boolean isWorthMergingForRangeQuery(List<InetAddressAndPort> merged, List<InetAddressAndPort> l1, List<InetAddressAndPort> l2)
+            public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2)
             {
                 return false;
             }
@@ -106,8 +121,10 @@ public class WriteResponseHandlerTest
         SchemaLoader.createKeyspace("Foo", KeyspaceParams.nts("datacenter1", 3, "datacenter2", 3), SchemaLoader.standardCFMD("Foo", "Bar"));
         ks = Keyspace.open("Foo");
         cfs = ks.getColumnFamilyStore("Bar");
-        targets = ImmutableList.of(InetAddressAndPort.getByName("127.1.0.255"), InetAddressAndPort.getByName("127.1.0.254"), InetAddressAndPort.getByName("127.1.0.253"),
-                                   InetAddressAndPort.getByName("127.2.0.255"), InetAddressAndPort.getByName("127.2.0.254"), InetAddressAndPort.getByName("127.2.0.253"));
+        targets = EndpointsForToken.of(DatabaseDescriptor.getPartitioner().getToken(ByteBufferUtil.bytes(0)),
+                                       full("127.1.0.255"), full("127.1.0.254"), full("127.1.0.253"),
+                                       full("127.2.0.255"), full("127.2.0.254"), full("127.2.0.253"));
+        pending = EndpointsForToken.empty(DatabaseDescriptor.getPartitioner().getToken(ByteBufferUtil.bytes(0)));
     }
 
     @Before
@@ -197,7 +214,6 @@ public class WriteResponseHandlerTest
     @Test
     public void failedIdealCLIncrementsStat() throws Throwable
     {
-
         AbstractWriteResponseHandler awr = createWriteResponseHandler(ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.EACH_QUORUM);
 
         //Succeed in local DC
@@ -220,16 +236,12 @@ public class WriteResponseHandlerTest
 
     private static AbstractWriteResponseHandler createWriteResponseHandler(ConsistencyLevel cl, ConsistencyLevel ideal, long queryStartTime)
     {
-        return ks.getReplicationStrategy().getWriteResponseHandler(targets, ImmutableList.of(), cl, new Runnable() {
-            public void run()
-            {
-
-            }
-        }, WriteType.SIMPLE, queryStartTime, ideal);
+        return ks.getReplicationStrategy().getWriteResponseHandler(ReplicaLayout.forWriteWithDownNodes(ks, cl, targets.token(), targets, pending),
+                                                                   null, WriteType.SIMPLE, queryStartTime, ideal);
     }
 
     private static MessageIn createDummyMessage(int target)
     {
-        return MessageIn.create(targets.get(target), null, null,  null, 0, 0L);
+        return MessageIn.create(targets.get(target).endpoint(), null, null,  null, 0, 0L);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
new file mode 100644
index 0000000..c19e65e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.service;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import com.google.common.base.Predicates;
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.EndpointsForToken;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.ReplicaUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaCollection;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.locator.Replica.fullReplica;
+import static org.apache.cassandra.locator.Replica.transientReplica;
+import static org.apache.cassandra.locator.ReplicaUtils.FULL_RANGE;
+import static org.apache.cassandra.locator.ReplicaUtils.full;
+import static org.apache.cassandra.locator.ReplicaUtils.trans;
+
+public class WriteResponseHandlerTransientTest
+{
+    static Keyspace ks;
+    static ColumnFamilyStore cfs;
+
+    static final InetAddressAndPort EP1;
+    static final InetAddressAndPort EP2;
+    static final InetAddressAndPort EP3;
+    static final InetAddressAndPort EP4;
+    static final InetAddressAndPort EP5;
+    static final InetAddressAndPort EP6;
+
+    static final String DC1 = "datacenter1";
+    static final String DC2 = "datacenter2";
+    static Token dummy;
+    static
+    {
+        try
+        {
+            EP1 = InetAddressAndPort.getByName("127.1.0.1");
+            EP2 = InetAddressAndPort.getByName("127.1.0.2");
+            EP3 = InetAddressAndPort.getByName("127.1.0.3");
+            EP4 = InetAddressAndPort.getByName("127.2.0.4");
+            EP5 = InetAddressAndPort.getByName("127.2.0.5");
+            EP6 = InetAddressAndPort.getByName("127.2.0.6");
+        }
+        catch (UnknownHostException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
+    @BeforeClass
+    public static void setupClass() throws Throwable
+    {
+        SchemaLoader.loadSchema();
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+
+        // Register peers with expected DC for NetworkTopologyStrategy.
+        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
+        metadata.clearUnsafe();
+        metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.1.0.1"));
+        metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.2.0.1"));
+
+        DatabaseDescriptor.setEndpointSnitch(new IEndpointSnitch()
+        {
+            public String getRack(InetAddressAndPort endpoint)
+            {
+                return null;
+            }
+
+            public String getDatacenter(InetAddressAndPort endpoint)
+            {
+                byte[] address = endpoint.address.getAddress();
+                if (address[1] == 1)
+                    return DC1;
+                else
+                    return DC2;
+            }
+
+            public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C unsortedAddress)
+            {
+                return unsortedAddress;
+            }
+
+            public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2)
+            {
+                return 0;
+            }
+
+            public void gossiperStarting()
+            {
+
+            }
+
+            public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2)
+            {
+                return false;
+            }
+        });
+
+        DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.1.0.1"));
+        SchemaLoader.createKeyspace("ks", KeyspaceParams.nts(DC1, "3/1", DC2, "3/1"), SchemaLoader.standardCFMD("ks", "tbl"));
+        ks = Keyspace.open("ks");
+        cfs = ks.getColumnFamilyStore("tbl");
+        dummy = DatabaseDescriptor.getPartitioner().getToken(ByteBufferUtil.bytes(0));
+    }
+
+    @Ignore("Throws unavailable for quorum as written")
+    @Test
+    public void checkPendingReplicasAreNotFiltered()
+    {
+        EndpointsForToken natural = EndpointsForToken.of(dummy.getToken(), full(EP1), full(EP2), trans(EP3));
+        EndpointsForToken pending = EndpointsForToken.of(dummy.getToken(), full(EP4), full(EP5), trans(EP6));
+        ReplicaLayout.ForToken replicaLayout = ReplicaLayout.forWrite(ks, ConsistencyLevel.QUORUM, dummy.getToken(), 2, natural, pending, Predicates.alwaysTrue());
+
+        Assert.assertEquals(EndpointsForRange.of(full(EP4), full(EP5), trans(EP6)), replicaLayout.pending());
+    }
+
+    private static ReplicaLayout.ForToken expected(EndpointsForToken all, EndpointsForToken selected)
+    {
+        return new ReplicaLayout.ForToken(ks, ConsistencyLevel.QUORUM, dummy.getToken(), all, EndpointsForToken.empty(dummy.getToken()), selected);
+    }
+
+    private static ReplicaLayout.ForToken getSpeculationContext(EndpointsForToken replicas, int blockFor, Predicate<InetAddressAndPort> livePredicate)
+    {
+        return ReplicaLayout.forWrite(ks, ConsistencyLevel.QUORUM, dummy.getToken(), blockFor, replicas, EndpointsForToken.empty(dummy.getToken()), livePredicate);
+    }
+
+    private static void assertSpeculationReplicas(ReplicaLayout.ForToken expected, EndpointsForToken replicas, int blockFor, Predicate<InetAddressAndPort> livePredicate)
+    {
+        ReplicaLayout.ForToken actual = getSpeculationContext(replicas, blockFor, livePredicate);
+        Assert.assertEquals(expected.natural(), actual.natural());
+        Assert.assertEquals(expected.selected(), actual.selected());
+    }
+
+    private static Predicate<InetAddressAndPort> dead(InetAddressAndPort... endpoints)
+    {
+        Set<InetAddressAndPort> deadSet = Sets.newHashSet(endpoints);
+        return ep -> !deadSet.contains(ep);
+    }
+
+    private static EndpointsForToken replicas(Replica... rr)
+    {
+        return EndpointsForToken.of(dummy.getToken(), rr);
+    }
+
+    @Ignore("Throws unavailable for quorum as written")
+    @Test
+    public void checkSpeculationContext()
+    {
+        EndpointsForToken all = replicas(full(EP1), full(EP2), trans(EP3));
+        // in happy path, transient replica should be classified as a backup
+        assertSpeculationReplicas(expected(all,
+                                           replicas(full(EP1), full(EP2))),
+                                  replicas(full(EP1), full(EP2), trans(EP3)),
+                                  2, dead());
+
+        // if one of the full replicas is dead, they should all be in the initial contacts
+        assertSpeculationReplicas(expected(all,
+                                           replicas(full(EP1), trans(EP3))),
+                                  replicas(full(EP1), full(EP2), trans(EP3)),
+                                  2, dead(EP2));
+
+        // block only for 1 full replica, use transient as backups
+        assertSpeculationReplicas(expected(all,
+                                           replicas(full(EP1))),
+                                  replicas(full(EP1), full(EP2), trans(EP3)),
+                                  1, dead(EP2));
+    }
+
+    @Test (expected = UnavailableException.class)
+    public void noFullReplicas()
+    {
+        getSpeculationContext(replicas(full(EP1), trans(EP2), trans(EP3)), 2, dead(EP1));
+    }
+
+    @Test (expected = UnavailableException.class)
+    public void notEnoughTransientReplicas()
+    {
+        getSpeculationContext(replicas(full(EP1), trans(EP2), trans(EP3)), 2, dead(EP2, EP3));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java b/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java
new file mode 100644
index 0000000..c6f2232
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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.service.reads;
+
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringBoundary;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadResponse;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
+import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * Base class for testing various components which deal with read responses
+ */
+@Ignore
+public abstract class AbstractReadResponseTest
+{
+    public static final String KEYSPACE1 = "DataResolverTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COLLECTION = "Collection1";
+
+    public static Keyspace ks;
+    public static ColumnFamilyStore cfs;
+    public static ColumnFamilyStore cfs2;
+    public static TableMetadata cfm;
+    public static TableMetadata cfm2;
+    public static ColumnMetadata m;
+
+    public static DecoratedKey dk;
+    static int nowInSec;
+
+    static final InetAddressAndPort EP1;
+    static final InetAddressAndPort EP2;
+    static final InetAddressAndPort EP3;
+
+    static
+    {
+        try
+        {
+            EP1 = InetAddressAndPort.getByName("127.0.0.1");
+            EP2 = InetAddressAndPort.getByName("127.0.0.2");
+            EP3 = InetAddressAndPort.getByName("127.0.0.3");
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @BeforeClass
+    public static void setupClass() throws Throwable
+    {
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+
+        TableMetadata.Builder builder1 =
+        TableMetadata.builder(KEYSPACE1, CF_STANDARD)
+                     .addPartitionKeyColumn("key", BytesType.instance)
+                     .addClusteringColumn("col1", AsciiType.instance)
+                     .addRegularColumn("c1", AsciiType.instance)
+                     .addRegularColumn("c2", AsciiType.instance)
+                     .addRegularColumn("one", AsciiType.instance)
+                     .addRegularColumn("two", AsciiType.instance);
+
+        TableMetadata.Builder builder2 =
+        TableMetadata.builder(KEYSPACE1, CF_COLLECTION)
+                     .addPartitionKeyColumn("k", ByteType.instance)
+                     .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true));
+
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), builder1, builder2);
+
+        ks = Keyspace.open(KEYSPACE1);
+        cfs = ks.getColumnFamilyStore(CF_STANDARD);
+        cfm = cfs.metadata();
+        cfs2 = ks.getColumnFamilyStore(CF_COLLECTION);
+        cfm2 = cfs2.metadata();
+        m = cfm2.getColumn(new ColumnIdentifier("m", false));
+    }
+
+    @Before
+    public void setUp() throws Exception
+    {
+        dk = Util.dk("key1");
+        nowInSec = FBUtilities.nowInSeconds();
+    }
+
+    static void assertPartitionsEqual(RowIterator l, RowIterator r)
+    {
+        try (RowIterator left = l; RowIterator right = r)
+        {
+            Assert.assertTrue(Util.sameContent(left, right));
+        }
+    }
+
+    static void assertPartitionsEqual(UnfilteredRowIterator left, UnfilteredRowIterator right)
+    {
+        Assert.assertTrue(Util.sameContent(left, right));
+    }
+
+    static void assertPartitionsEqual(UnfilteredPartitionIterator left, UnfilteredPartitionIterator right)
+    {
+        while (left.hasNext())
+        {
+            Assert.assertTrue(right.hasNext());
+            assertPartitionsEqual(left.next(), right.next());
+        }
+        Assert.assertFalse(right.hasNext());
+    }
+
+    static void assertPartitionsEqual(PartitionIterator l, PartitionIterator r)
+    {
+        try (PartitionIterator left = l; PartitionIterator right = r)
+        {
+            while (left.hasNext())
+            {
+                Assert.assertTrue(right.hasNext());
+                assertPartitionsEqual(left.next(), right.next());
+            }
+            Assert.assertFalse(right.hasNext());
+        }
+    }
+
+    static void consume(PartitionIterator i)
+    {
+        try (PartitionIterator iterator = i)
+        {
+            while (iterator.hasNext())
+            {
+                try (RowIterator rows = iterator.next())
+                {
+                    while (rows.hasNext())
+                        rows.next();
+                }
+            }
+        }
+    }
+
+    static PartitionIterator filter(UnfilteredPartitionIterator iter)
+    {
+        return UnfilteredPartitionIterators.filter(iter, nowInSec);
+    }
+
+    static DecoratedKey dk(String k)
+    {
+        return cfs.decorateKey(ByteBufferUtil.bytes(k));
+    }
+
+    static DecoratedKey dk(int k)
+    {
+        return dk(Integer.toString(k));
+    }
+
+    static MessageIn<ReadResponse> response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data, boolean digest)
+    {
+        ReadResponse response = digest ? ReadResponse.createDigestResponse(data, command) : ReadResponse.createDataResponse(data, command);
+        return MessageIn.create(from, response, Collections.emptyMap(), MessagingService.Verb.READ, MessagingService.current_version);
+    }
+
+    static MessageIn<ReadResponse> response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data)
+    {
+        return response(command, from, data, false);
+    }
+
+    public RangeTombstone tombstone(Object start, Object end, long markedForDeleteAt, int localDeletionTime)
+    {
+        return tombstone(start, true, end, true, markedForDeleteAt, localDeletionTime);
+    }
+
+    public RangeTombstone tombstone(Object start, boolean inclusiveStart, Object end, boolean inclusiveEnd, long markedForDeleteAt, int localDeletionTime)
+    {
+        ClusteringBound startBound = rtBound(start, true, inclusiveStart);
+        ClusteringBound endBound = rtBound(end, false, inclusiveEnd);
+        return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(markedForDeleteAt, localDeletionTime));
+    }
+
+    public ClusteringBound rtBound(Object value, boolean isStart, boolean inclusive)
+    {
+        ClusteringBound.Kind kind = isStart
+                                    ? (inclusive ? ClusteringPrefix.Kind.INCL_START_BOUND : ClusteringPrefix.Kind.EXCL_START_BOUND)
+                                    : (inclusive ? ClusteringPrefix.Kind.INCL_END_BOUND : ClusteringPrefix.Kind.EXCL_END_BOUND);
+
+        return ClusteringBound.create(kind, cfm.comparator.make(value).getRawValues());
+    }
+
+    public ClusteringBoundary rtBoundary(Object value, boolean inclusiveOnEnd)
+    {
+        ClusteringBound.Kind kind = inclusiveOnEnd
+                                    ? ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY
+                                    : ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY;
+        return ClusteringBoundary.create(kind, cfm.comparator.make(value).getRawValues());
+    }
+
+    public RangeTombstoneBoundMarker marker(Object value, boolean isStart, boolean inclusive, long markedForDeleteAt, int localDeletionTime)
+    {
+        return new RangeTombstoneBoundMarker(rtBound(value, isStart, inclusive), new DeletionTime(markedForDeleteAt, localDeletionTime));
+    }
+
+    public RangeTombstoneBoundaryMarker boundary(Object value, boolean inclusiveOnEnd, long markedForDeleteAt1, int localDeletionTime1, long markedForDeleteAt2, int localDeletionTime2)
+    {
+        return new RangeTombstoneBoundaryMarker(rtBoundary(value, inclusiveOnEnd),
+                                                new DeletionTime(markedForDeleteAt1, localDeletionTime1),
+                                                new DeletionTime(markedForDeleteAt2, localDeletionTime2));
+    }
+
+    public UnfilteredPartitionIterator fullPartitionDelete(TableMetadata table, DecoratedKey dk, long timestamp, int nowInSec)
+    {
+        return new SingletonUnfilteredPartitionIterator(PartitionUpdate.fullPartitionDelete(table, dk, timestamp, nowInSec).unfilteredIterator());
+    }
+
+    public UnfilteredPartitionIterator iter(PartitionUpdate update)
+    {
+        return new SingletonUnfilteredPartitionIterator(update.unfilteredIterator());
+    }
+
+    public UnfilteredPartitionIterator iter(DecoratedKey key, Unfiltered... unfiltereds)
+    {
+        SortedSet<Unfiltered> s = new TreeSet<>(cfm.comparator);
+        Collections.addAll(s, unfiltereds);
+        final Iterator<Unfiltered> iterator = s.iterator();
+
+        UnfilteredRowIterator rowIter = new AbstractUnfilteredRowIterator(cfm,
+                                                                          key,
+                                                                          DeletionTime.LIVE,
+                                                                          cfm.regularAndStaticColumns(),
+                                                                          Rows.EMPTY_STATIC_ROW,
+                                                                          false,
+                                                                          EncodingStats.NO_STATS)
+        {
+            protected Unfiltered computeNext()
+            {
+                return iterator.hasNext() ? iterator.next() : endOfData();
+            }
+        };
+        return new SingletonUnfilteredPartitionIterator(rowIter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java b/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java
index ac8ed0b..abec25d 100644
--- a/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/reads/DataResolverTest.java
@@ -19,114 +19,101 @@ package org.apache.cassandra.service.reads;
 
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.Collections;
+import java.util.Iterator;
 
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Sets;
-import org.junit.*;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.EmptyIterators;
+import org.apache.cassandra.db.MutableDeletionInfo;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundaryMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.ByteType;
-import org.apache.cassandra.db.marshal.IntegerType;
-import org.apache.cassandra.db.marshal.MapType;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.net.*;
-import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.locator.ReplicaLayout;
+import org.apache.cassandra.locator.ReplicaUtils;
 import org.apache.cassandra.service.reads.repair.TestableReadRepair;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.Util.assertClustering;
 import static org.apache.cassandra.Util.assertColumn;
 import static org.apache.cassandra.Util.assertColumns;
+import static org.apache.cassandra.db.ClusteringBound.Kind;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.apache.cassandra.db.ClusteringBound.Kind;
 
-public class DataResolverTest
+public class DataResolverTest extends AbstractReadResponseTest
 {
     public static final String KEYSPACE1 = "DataResolverTest";
     public static final String CF_STANDARD = "Standard1";
-    public static final String CF_COLLECTION = "Collection1";
-
-    // counter to generate the last byte of the respondent's address in a ReadResponse message
-    private int addressSuffix = 10;
-
-    private DecoratedKey dk;
-    private Keyspace ks;
-    private ColumnFamilyStore cfs;
-    private ColumnFamilyStore cfs2;
-    private TableMetadata cfm;
-    private TableMetadata cfm2;
-    private ColumnMetadata m;
-    private int nowInSec;
+
     private ReadCommand command;
     private TestableReadRepair readRepair;
 
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        DatabaseDescriptor.daemonInitialization();
-
-        TableMetadata.Builder builder1 =
-            TableMetadata.builder(KEYSPACE1, CF_STANDARD)
-                         .addPartitionKeyColumn("key", BytesType.instance)
-                         .addClusteringColumn("col1", AsciiType.instance)
-                         .addRegularColumn("c1", AsciiType.instance)
-                         .addRegularColumn("c2", AsciiType.instance)
-                         .addRegularColumn("one", AsciiType.instance)
-                         .addRegularColumn("two", AsciiType.instance);
-
-        TableMetadata.Builder builder2 =
-            TableMetadata.builder(KEYSPACE1, CF_COLLECTION)
-                         .addPartitionKeyColumn("k", ByteType.instance)
-                         .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true));
-
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), builder1, builder2);
-    }
-
     @Before
     public void setup()
     {
-        dk = Util.dk("key1");
-        ks = Keyspace.open(KEYSPACE1);
-        cfs = ks.getColumnFamilyStore(CF_STANDARD);
-        cfm = cfs.metadata();
-        cfs2 = ks.getColumnFamilyStore(CF_COLLECTION);
-        cfm2 = cfs2.metadata();
-        m = cfm2.getColumn(new ColumnIdentifier("m", false));
-
-        nowInSec = FBUtilities.nowInSeconds();
         command = Util.cmd(cfs, dk).withNowInSeconds(nowInSec).build();
         readRepair = new TestableReadRepair(command, ConsistencyLevel.QUORUM);
     }
 
-    @Test
-    public void testResolveNewerSingleRow() throws UnknownHostException
+    private static EndpointsForRange makeReplicas(int num)
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
-                                                                                                       .add("c1", "v1")
-                                                                                                       .buildUpdate())));
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
-                                                                                                       .add("c1", "v2")
-                                                                                                       .buildUpdate())));
+        EndpointsForRange.Builder replicas = EndpointsForRange.builder(ReplicaUtils.FULL_RANGE, num);
+        for (int i = 0; i < num; i++)
+        {
+            try
+            {
+                replicas.add(ReplicaUtils.full(InetAddressAndPort.getByAddress(new byte[]{ 127, 0, 0, (byte) (i + 1) })));
+            }
+            catch (UnknownHostException e)
+            {
+                throw new AssertionError(e);
+            }
+        }
+        return replicas.build();
+    }
+
+    @Test
+    public void testResolveNewerSingleRow()
+    {
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
+                                                                                                     .add("c1", "v1")
+                                                                                                     .buildUpdate()), false));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
+                                                                                                     .add("c1", "v2")
+                                                                                                     .buildUpdate()), false));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -149,16 +136,17 @@ public class DataResolverTest
     @Test
     public void testResolveDisjointSingleRow()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
-                                                                                                       .add("c1", "v1")
-                                                                                                       .buildUpdate())));
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
+                                                                                                     .add("c1", "v1")
+                                                                                                     .buildUpdate())));
 
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
-                                                                                                       .add("c2", "v2")
-                                                                                                       .buildUpdate())));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
+                                                                                                     .add("c2", "v2")
+                                                                                                     .buildUpdate())));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -185,15 +173,16 @@ public class DataResolverTest
     @Test
     public void testResolveDisjointMultipleRows() throws UnknownHostException
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
-                                                                                                       .add("c1", "v1")
-                                                                                                       .buildUpdate())));
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("2")
-                                                                                                       .add("c2", "v2")
-                                                                                                       .buildUpdate())));
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
+                                                                                                     .add("c1", "v1")
+                                                                                                     .buildUpdate())));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("2")
+                                                                                                     .add("c2", "v2")
+                                                                                                     .buildUpdate())));
 
         try (PartitionIterator data = resolver.resolve())
         {
@@ -231,37 +220,38 @@ public class DataResolverTest
     @Test
     public void testResolveDisjointMultipleRowsWithRangeTombstones()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 4, System.nanoTime(), readRepair);
+        EndpointsForRange replicas = makeReplicas(4);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
 
         RangeTombstone tombstone1 = tombstone("1", "11", 1, nowInSec);
         RangeTombstone tombstone2 = tombstone("3", "31", 1, nowInSec);
         PartitionUpdate update = new RowUpdateBuilder(cfm, nowInSec, 1L, dk).addRangeTombstone(tombstone1)
-                                                                                  .addRangeTombstone(tombstone2)
-                                                                                  .buildUpdate();
+                                                                            .addRangeTombstone(tombstone2)
+                                                                            .buildUpdate();
 
-        InetAddressAndPort peer1 = peer();
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
         UnfilteredPartitionIterator iter1 = iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).addRangeTombstone(tombstone1)
-                                                                                  .addRangeTombstone(tombstone2)
-                                                                                  .buildUpdate());
-        resolver.preprocess(readResponseMessage(peer1, iter1));
+                                                                                            .addRangeTombstone(tombstone2)
+                                                                                            .buildUpdate());
+        resolver.preprocess(response(command, peer1, iter1));
         // not covered by any range tombstone
-        InetAddressAndPort peer2 = peer();
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
         UnfilteredPartitionIterator iter2 = iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("0")
-                                                                                  .add("c1", "v0")
-                                                                                  .buildUpdate());
-        resolver.preprocess(readResponseMessage(peer2, iter2));
+                                                                                            .add("c1", "v0")
+                                                                                            .buildUpdate());
+        resolver.preprocess(response(command, peer2, iter2));
         // covered by a range tombstone
-        InetAddressAndPort peer3 = peer();
+        InetAddressAndPort peer3 = replicas.get(2).endpoint();
         UnfilteredPartitionIterator iter3 = iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("10")
-                                                                                  .add("c2", "v1")
-                                                                                  .buildUpdate());
-        resolver.preprocess(readResponseMessage(peer3, iter3));
+                                                                                            .add("c2", "v1")
+                                                                                            .buildUpdate());
+        resolver.preprocess(response(command, peer3, iter3));
         // range covered by rt, but newer
-        InetAddressAndPort peer4 = peer();
+        InetAddressAndPort peer4 = replicas.get(3).endpoint();
         UnfilteredPartitionIterator iter4 = iter(new RowUpdateBuilder(cfm, nowInSec, 2L, dk).clustering("3")
-                                                                                  .add("one", "A")
-                                                                                  .buildUpdate());
-        resolver.preprocess(readResponseMessage(peer4, iter4));
+                                                                                            .add("one", "A")
+                                                                                            .buildUpdate());
+        resolver.preprocess(response(command, peer4, iter4));
         try (PartitionIterator data = resolver.resolve())
         {
             try (RowIterator rows = data.next())
@@ -311,13 +301,14 @@ public class DataResolverTest
     @Test
     public void testResolveWithOneEmpty()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
-                                                                                                       .add("c2", "v2")
-                                                                                                       .buildUpdate())));
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, EmptyIterators.unfilteredPartition(cfm)));
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
+                                                                                                     .add("c2", "v2")
+                                                                                                     .buildUpdate())));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, EmptyIterators.unfilteredPartition(cfm)));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -340,10 +331,11 @@ public class DataResolverTest
     @Test
     public void testResolveWithBothEmpty()
     {
+        EndpointsForRange replicas = makeReplicas(2);
         TestableReadRepair readRepair = new TestableReadRepair(command, ConsistencyLevel.QUORUM);
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        resolver.preprocess(readResponseMessage(peer(), EmptyIterators.unfilteredPartition(cfm)));
-        resolver.preprocess(readResponseMessage(peer(), EmptyIterators.unfilteredPartition(cfm)));
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        resolver.preprocess(response(command, replicas.get(0).endpoint(), EmptyIterators.unfilteredPartition(cfm)));
+        resolver.preprocess(response(command, replicas.get(1).endpoint(), EmptyIterators.unfilteredPartition(cfm)));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -356,14 +348,15 @@ public class DataResolverTest
     @Test
     public void testResolveDeleted()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
         // one response with columns timestamped before a delete in another response
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
-                                                                                                       .add("one", "A")
-                                                                                                       .buildUpdate())));
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, fullPartitionDelete(cfm, dk, 1, nowInSec)));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, iter(new RowUpdateBuilder(cfm, nowInSec, 0L, dk).clustering("1")
+                                                                                                     .add("one", "A")
+                                                                                                     .buildUpdate())));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, fullPartitionDelete(cfm, dk, 1, nowInSec)));
 
         try (PartitionIterator data = resolver.resolve())
         {
@@ -381,23 +374,24 @@ public class DataResolverTest
     @Test
     public void testResolveMultipleDeleted()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 4, System.nanoTime(), readRepair);
+        EndpointsForRange replicas = makeReplicas(4);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
         // deletes and columns with interleaved timestamp, with out of order return sequence
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, fullPartitionDelete(cfm, dk, 0, nowInSec)));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(command, peer1, fullPartitionDelete(cfm, dk, 0, nowInSec)));
         // these columns created after the previous deletion
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
-                                                                                                       .add("one", "A")
-                                                                                                       .add("two", "A")
-                                                                                                       .buildUpdate())));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(command, peer2, iter(new RowUpdateBuilder(cfm, nowInSec, 1L, dk).clustering("1")
+                                                                                                     .add("one", "A")
+                                                                                                     .add("two", "A")
+                                                                                                     .buildUpdate())));
         //this column created after the next delete
-        InetAddressAndPort peer3 = peer();
-        resolver.preprocess(readResponseMessage(peer3, iter(new RowUpdateBuilder(cfm, nowInSec, 3L, dk).clustering("1")
-                                                                                                       .add("two", "B")
-                                                                                                       .buildUpdate())));
-        InetAddressAndPort peer4 = peer();
-        resolver.preprocess(readResponseMessage(peer4, fullPartitionDelete(cfm, dk, 2, nowInSec)));
+        InetAddressAndPort peer3 = replicas.get(2).endpoint();
+        resolver.preprocess(response(command, peer3, iter(new RowUpdateBuilder(cfm, nowInSec, 3L, dk).clustering("1")
+                                                                                                     .add("two", "B")
+                                                                                                     .buildUpdate())));
+        InetAddressAndPort peer4 = replicas.get(3).endpoint();
+        resolver.preprocess(response(command, peer4, fullPartitionDelete(cfm, dk, 2, nowInSec)));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -465,9 +459,10 @@ public class DataResolverTest
      */
     private void resolveRangeTombstonesOnBoundary(long timestamp1, long timestamp2)
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        InetAddressAndPort peer2 = peer();
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
 
         // 1st "stream"
         RangeTombstone one_two    = tombstone("1", true , "2", false, timestamp1, nowInSec);
@@ -485,8 +480,8 @@ public class DataResolverTest
                                                                                             .addRangeTombstone(four_five)
                                                                                             .buildUpdate());
 
-        resolver.preprocess(readResponseMessage(peer1, iter1));
-        resolver.preprocess(readResponseMessage(peer2, iter2));
+        resolver.preprocess(response(command, peer1, iter1));
+        resolver.preprocess(response(command, peer2, iter2));
 
         // No results, we've only reconciled tombstones.
         try (PartitionIterator data = resolver.resolve())
@@ -538,9 +533,10 @@ public class DataResolverTest
      */
     private void testRepairRangeTombstoneBoundary(int timestamp1, int timestamp2, int timestamp3) throws UnknownHostException
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        InetAddressAndPort peer2 = peer();
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
 
         // 1st "stream"
         RangeTombstone one_nine = tombstone("0", true , "9", true, timestamp1, nowInSec);
@@ -554,8 +550,8 @@ public class DataResolverTest
         RangeTombstoneBoundMarker close_nine = marker("9", false, true, timestamp3, nowInSec);
         UnfilteredPartitionIterator iter2 = iter(dk, open_one, boundary_five, close_nine);
 
-        resolver.preprocess(readResponseMessage(peer1, iter1));
-        resolver.preprocess(readResponseMessage(peer2, iter2));
+        resolver.preprocess(response(command, peer1, iter1));
+        resolver.preprocess(response(command, peer2, iter2));
 
         boolean shouldHaveRepair = timestamp1 != timestamp2 || timestamp1 != timestamp3;
 
@@ -589,9 +585,10 @@ public class DataResolverTest
     @Test
     public void testRepairRangeTombstoneWithPartitionDeletion()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        InetAddressAndPort peer2 = peer();
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
 
         // 1st "stream": just a partition deletion
         UnfilteredPartitionIterator iter1 = iter(PartitionUpdate.fullPartitionDelete(cfm, dk, 10, nowInSec));
@@ -602,8 +599,8 @@ public class DataResolverTest
                                                  .addRangeTombstone(rt)
                                                  .buildUpdate());
 
-        resolver.preprocess(readResponseMessage(peer1, iter1));
-        resolver.preprocess(readResponseMessage(peer2, iter2));
+        resolver.preprocess(response(command, peer1, iter1));
+        resolver.preprocess(response(command, peer2, iter2));
 
         // No results, we've only reconciled tombstones.
         try (PartitionIterator data = resolver.resolve())
@@ -627,15 +624,16 @@ public class DataResolverTest
     @Test
     public void testRepairRangeTombstoneWithPartitionDeletion2()
     {
-        DataResolver resolver = new DataResolver(ks, command, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
-        InetAddressAndPort peer1 = peer();
-        InetAddressAndPort peer2 = peer();
+        EndpointsForRange replicas = makeReplicas(2);
+        DataResolver resolver = new DataResolver(command, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
 
         // 1st "stream": a partition deletion and a range tombstone
         RangeTombstone rt1 = tombstone("0", true , "9", true, 11, nowInSec);
         PartitionUpdate upd1 = new RowUpdateBuilder(cfm, nowInSec, 1L, dk)
-                                                 .addRangeTombstone(rt1)
-                                                 .buildUpdate();
+                               .addRangeTombstone(rt1)
+                               .buildUpdate();
         ((MutableDeletionInfo)upd1.deletionInfo()).add(new DeletionTime(10, nowInSec));
         UnfilteredPartitionIterator iter1 = iter(upd1);
 
@@ -647,8 +645,8 @@ public class DataResolverTest
                                                  .addRangeTombstone(rt3)
                                                  .buildUpdate());
 
-        resolver.preprocess(readResponseMessage(peer1, iter1));
-        resolver.preprocess(readResponseMessage(peer2, iter2));
+        resolver.preprocess(response(command, peer1, iter1));
+        resolver.preprocess(response(command, peer2, iter2));
 
         // No results, we've only reconciled tombstones.
         try (PartitionIterator data = resolver.resolve())
@@ -678,8 +676,8 @@ public class DataResolverTest
         Slice slice = rt.deletedSlice();
         ClusteringBound newStart = ClusteringBound.create(Kind.EXCL_START_BOUND, slice.start().getRawValues());
         return condition
-             ? new RangeTombstone(Slice.make(newStart, slice.end()), rt.deletionTime())
-             : rt;
+               ? new RangeTombstone(Slice.make(newStart, slice.end()), rt.deletionTime())
+               : rt;
     }
 
     // Forces the end to be exclusive if the condition holds
@@ -691,8 +689,8 @@ public class DataResolverTest
         Slice slice = rt.deletedSlice();
         ClusteringBound newEnd = ClusteringBound.create(Kind.EXCL_END_BOUND, slice.end().getRawValues());
         return condition
-             ? new RangeTombstone(Slice.make(slice.start(), newEnd), rt.deletionTime())
-             : rt;
+               ? new RangeTombstone(Slice.make(slice.start(), newEnd), rt.deletionTime())
+               : rt;
     }
 
     private static ByteBuffer bb(int b)
@@ -708,9 +706,10 @@ public class DataResolverTest
     @Test
     public void testResolveComplexDelete()
     {
+        EndpointsForRange replicas = makeReplicas(2);
         ReadCommand cmd = Util.cmd(cfs2, dk).withNowInSeconds(nowInSec).build();
         TestableReadRepair readRepair = new TestableReadRepair(cmd, ConsistencyLevel.QUORUM);
-        DataResolver resolver = new DataResolver(ks, cmd, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
+        DataResolver resolver = new DataResolver(cmd, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
 
         long[] ts = {100, 200};
 
@@ -719,8 +718,8 @@ public class DataResolverTest
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
         builder.addCell(mapCell(0, 0, ts[0]));
 
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(cmd, peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         builder.newRow(Clustering.EMPTY);
         DeletionTime expectedCmplxDelete = new DeletionTime(ts[1] - 1, nowInSec);
@@ -728,8 +727,8 @@ public class DataResolverTest
         Cell expectedCell = mapCell(1, 1, ts[1]);
         builder.addCell(expectedCell);
 
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(cmd, peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -742,7 +741,6 @@ public class DataResolverTest
             }
         }
 
-
         Mutation mutation = readRepair.getForEndpoint(peer1);
         Iterator<Row> rowIter = mutation.getPartitionUpdate(cfm2).iterator();
         assertTrue(rowIter.hasNext());
@@ -760,9 +758,10 @@ public class DataResolverTest
     @Test
     public void testResolveDeletedCollection()
     {
+        EndpointsForRange replicas = makeReplicas(2);
         ReadCommand cmd = Util.cmd(cfs2, dk).withNowInSeconds(nowInSec).build();
         TestableReadRepair readRepair = new TestableReadRepair(cmd, ConsistencyLevel.QUORUM);
-        DataResolver resolver = new DataResolver(ks, cmd, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
+        DataResolver resolver = new DataResolver(cmd, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
 
         long[] ts = {100, 200};
 
@@ -771,15 +770,15 @@ public class DataResolverTest
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
         builder.addCell(mapCell(0, 0, ts[0]));
 
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(cmd, peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         builder.newRow(Clustering.EMPTY);
         DeletionTime expectedCmplxDelete = new DeletionTime(ts[1] - 1, nowInSec);
         builder.addComplexDeletion(m, expectedCmplxDelete);
 
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(cmd, peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -803,9 +802,10 @@ public class DataResolverTest
     @Test
     public void testResolveNewCollection()
     {
+        EndpointsForRange replicas = makeReplicas(2);
         ReadCommand cmd = Util.cmd(cfs2, dk).withNowInSeconds(nowInSec).build();
         TestableReadRepair readRepair = new TestableReadRepair(cmd, ConsistencyLevel.QUORUM);
-        DataResolver resolver = new DataResolver(ks, cmd, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
+        DataResolver resolver = new DataResolver(cmd, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
 
         long[] ts = {100, 200};
 
@@ -818,11 +818,11 @@ public class DataResolverTest
         builder.addCell(expectedCell);
 
         // empty map column
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(cmd, peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.emptyUpdate(cfm2, dk))));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(cmd, peer2, iter(PartitionUpdate.emptyUpdate(cfm2, dk))));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -852,9 +852,10 @@ public class DataResolverTest
     @Test
     public void testResolveNewCollectionOverwritingDeleted()
     {
+        EndpointsForRange replicas = makeReplicas(2);
         ReadCommand cmd = Util.cmd(cfs2, dk).withNowInSeconds(nowInSec).build();
         TestableReadRepair readRepair = new TestableReadRepair(cmd, ConsistencyLevel.QUORUM);
-        DataResolver resolver = new DataResolver(ks, cmd, ConsistencyLevel.ALL, 2, System.nanoTime(), readRepair);
+        DataResolver resolver = new DataResolver(cmd, plan(replicas, ConsistencyLevel.ALL), readRepair, System.nanoTime());
 
         long[] ts = {100, 200};
 
@@ -863,8 +864,8 @@ public class DataResolverTest
         builder.newRow(Clustering.EMPTY);
         builder.addComplexDeletion(m, new DeletionTime(ts[0] - 1, nowInSec));
 
-        InetAddressAndPort peer1 = peer();
-        resolver.preprocess(readResponseMessage(peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer1 = replicas.get(0).endpoint();
+        resolver.preprocess(response(cmd, peer1, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         // newer, overwritten map column
         builder.newRow(Clustering.EMPTY);
@@ -873,8 +874,8 @@ public class DataResolverTest
         Cell expectedCell = mapCell(1, 1, ts[1]);
         builder.addCell(expectedCell);
 
-        InetAddressAndPort peer2 = peer();
-        resolver.preprocess(readResponseMessage(peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build())), cmd));
+        InetAddressAndPort peer2 = replicas.get(1).endpoint();
+        resolver.preprocess(response(cmd, peer2, iter(PartitionUpdate.singleRowUpdate(cfm2, dk, builder.build()))));
 
         try(PartitionIterator data = resolver.resolve())
         {
@@ -897,18 +898,6 @@ public class DataResolverTest
         Assert.assertNull(readRepair.sent.get(peer2));
     }
 
-    private InetAddressAndPort peer()
-    {
-        try
-        {
-            return InetAddressAndPort.getByAddress(new byte[]{ 127, 0, 0, (byte) addressSuffix++ });
-        }
-        catch (UnknownHostException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     private void assertRepairContainsDeletions(Mutation mutation,
                                                DeletionTime deletionTime,
                                                RangeTombstone...rangeTombstones)
@@ -961,91 +950,8 @@ public class DataResolverTest
         assertEquals(update.metadata().name, cfm.name);
     }
 
-
-    public MessageIn<ReadResponse> readResponseMessage(InetAddressAndPort from, UnfilteredPartitionIterator partitionIterator)
-    {
-        return readResponseMessage(from, partitionIterator, command);
-
-    }
-    public MessageIn<ReadResponse> readResponseMessage(InetAddressAndPort from, UnfilteredPartitionIterator partitionIterator, ReadCommand cmd)
+    private ReplicaLayout.ForRange plan(EndpointsForRange replicas, ConsistencyLevel consistencyLevel)
     {
-        return MessageIn.create(from,
-                                ReadResponse.createRemoteDataResponse(partitionIterator, cmd),
-                                Collections.EMPTY_MAP,
-                                MessagingService.Verb.REQUEST_RESPONSE,
-                                MessagingService.current_version);
-    }
-
-    private RangeTombstone tombstone(Object start, Object end, long markedForDeleteAt, int localDeletionTime)
-    {
-        return tombstone(start, true, end, true, markedForDeleteAt, localDeletionTime);
-    }
-
-    private RangeTombstone tombstone(Object start, boolean inclusiveStart, Object end, boolean inclusiveEnd, long markedForDeleteAt, int localDeletionTime)
-    {
-        ClusteringBound startBound = rtBound(start, true, inclusiveStart);
-        ClusteringBound endBound = rtBound(end, false, inclusiveEnd);
-        return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(markedForDeleteAt, localDeletionTime));
-    }
-
-    private ClusteringBound rtBound(Object value, boolean isStart, boolean inclusive)
-    {
-        ClusteringBound.Kind kind = isStart
-                                  ? (inclusive ? Kind.INCL_START_BOUND : Kind.EXCL_START_BOUND)
-                                  : (inclusive ? Kind.INCL_END_BOUND : Kind.EXCL_END_BOUND);
-
-        return ClusteringBound.create(kind, cfm.comparator.make(value).getRawValues());
-    }
-
-    private ClusteringBoundary rtBoundary(Object value, boolean inclusiveOnEnd)
-    {
-        ClusteringBound.Kind kind = inclusiveOnEnd
-                                  ? Kind.INCL_END_EXCL_START_BOUNDARY
-                                  : Kind.EXCL_END_INCL_START_BOUNDARY;
-        return ClusteringBoundary.create(kind, cfm.comparator.make(value).getRawValues());
-    }
-
-    private RangeTombstoneBoundMarker marker(Object value, boolean isStart, boolean inclusive, long markedForDeleteAt, int localDeletionTime)
-    {
-        return new RangeTombstoneBoundMarker(rtBound(value, isStart, inclusive), new DeletionTime(markedForDeleteAt, localDeletionTime));
-    }
-
-    private RangeTombstoneBoundaryMarker boundary(Object value, boolean inclusiveOnEnd, long markedForDeleteAt1, int localDeletionTime1, long markedForDeleteAt2, int localDeletionTime2)
-    {
-        return new RangeTombstoneBoundaryMarker(rtBoundary(value, inclusiveOnEnd),
-                                                new DeletionTime(markedForDeleteAt1, localDeletionTime1),
-                                                new DeletionTime(markedForDeleteAt2, localDeletionTime2));
-    }
-
-    private UnfilteredPartitionIterator fullPartitionDelete(TableMetadata table, DecoratedKey dk, long timestamp, int nowInSec)
-    {
-        return new SingletonUnfilteredPartitionIterator(PartitionUpdate.fullPartitionDelete(table, dk, timestamp, nowInSec).unfilteredIterator());
-    }
-
-    private UnfilteredPartitionIterator iter(PartitionUpdate update)
-    {
-        return new SingletonUnfilteredPartitionIterator(update.unfilteredIterator());
-    }
-
-    private UnfilteredPartitionIterator iter(DecoratedKey key, Unfiltered... unfiltereds)
-    {
-        SortedSet<Unfiltered> s = new TreeSet<>(cfm.comparator);
-        Collections.addAll(s, unfiltereds);
-        final Iterator<Unfiltered> iterator = s.iterator();
-
-        UnfilteredRowIterator rowIter = new AbstractUnfilteredRowIterator(cfm,
-                                                                          key,
-                                                                          DeletionTime.LIVE,
-                                                                          cfm.regularAndStaticColumns(),
-                                                                          Rows.EMPTY_STATIC_ROW,
-                                                                          false,
-                                                                          EncodingStats.NO_STATS)
-        {
-            protected Unfiltered computeNext()
-            {
-                return iterator.hasNext() ? iterator.next() : endOfData();
-            }
-        };
-        return new SingletonUnfilteredPartitionIterator(rowIter);
+        return new ReplicaLayout.ForRange(ks, consistencyLevel, ReplicaUtils.FULL_BOUNDS, replicas, replicas);
     }
-}
+}
\ No newline at end of file


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


[12/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/SimpleStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index 545ad28..7a000b7 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -21,9 +21,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.Token;
 
@@ -36,34 +36,41 @@ import org.apache.cassandra.dht.Token;
  */
 public class SimpleStrategy extends AbstractReplicationStrategy
 {
+    private final ReplicationFactor rf;
+
     public SimpleStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
         super(keyspaceName, tokenMetadata, snitch, configOptions);
+        this.rf = ReplicationFactor.fromString(this.configOptions.get("replication_factor"));
     }
 
-    public List<InetAddressAndPort> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
+    public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata metadata)
     {
-        int replicas = getReplicationFactor();
-        ArrayList<Token> tokens = metadata.sortedTokens();
-        List<InetAddressAndPort> endpoints = new ArrayList<InetAddressAndPort>(replicas);
+        ArrayList<Token> ring = metadata.sortedTokens();
+        if (ring.isEmpty())
+            return EndpointsForRange.empty(new Range<>(metadata.partitioner.getMinimumToken(), metadata.partitioner.getMinimumToken()));
+
+        Token replicaEnd = TokenMetadata.firstToken(ring, token);
+        Token replicaStart = metadata.getPredecessor(replicaEnd);
+        Range<Token> replicaRange = new Range<>(replicaStart, replicaEnd);
+        Iterator<Token> iter = TokenMetadata.ringIterator(ring, token, false);
 
-        if (tokens.isEmpty())
-            return endpoints;
+        EndpointsForRange.Builder replicas = EndpointsForRange.builder(replicaRange, rf.allReplicas);
 
         // Add the token at the index by default
-        Iterator<Token> iter = TokenMetadata.ringIterator(tokens, token, false);
-        while (endpoints.size() < replicas && iter.hasNext())
+        while (replicas.size() < rf.allReplicas && iter.hasNext())
         {
-            InetAddressAndPort ep = metadata.getEndpoint(iter.next());
-            if (!endpoints.contains(ep))
-                endpoints.add(ep);
+            Token tk = iter.next();
+            InetAddressAndPort ep = metadata.getEndpoint(tk);
+            if (!replicas.containsEndpoint(ep))
+                replicas.add(new Replica(ep, replicaRange, replicas.size() < rf.fullReplicas));
         }
-        return endpoints;
+        return replicas.build();
     }
 
-    public int getReplicationFactor()
+    public ReplicationFactor getReplicationFactor()
     {
-        return Integer.parseInt(this.configOptions.get("replication_factor"));
+        return rf;
     }
 
     public void validateOptions() throws ConfigurationException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/SystemReplicas.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SystemReplicas.java b/src/java/org/apache/cassandra/locator/SystemReplicas.java
new file mode 100644
index 0000000..13a9d74
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/SystemReplicas.java
@@ -0,0 +1,62 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+public class SystemReplicas
+{
+    private static final Map<InetAddressAndPort, Replica> systemReplicas = new ConcurrentHashMap<>();
+    public static final Range<Token> FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(),
+                                                              DatabaseDescriptor.getPartitioner().getMinimumToken());
+
+    private static Replica createSystemReplica(InetAddressAndPort endpoint)
+    {
+        return new Replica(endpoint, FULL_RANGE, true);
+    }
+
+    /**
+     * There are a few places where a system function borrows write path functionality, but doesn't otherwise
+     * fit into normal replication strategies (ie: hints and batchlog). So here we provide a replica instance
+     * @param endpoint
+     * @return
+     */
+    public static Replica getSystemReplica(InetAddressAndPort endpoint)
+    {
+        return systemReplicas.computeIfAbsent(endpoint, SystemReplicas::createSystemReplica);
+    }
+
+    public static Collection<Replica> getSystemReplicas(Collection<InetAddressAndPort> endpoints)
+    {
+        List<Replica> replicas = new ArrayList<>(endpoints.size());
+        for (InetAddressAndPort endpoint: endpoints)
+        {
+            replicas.add(getSystemReplica(endpoint));
+        }
+        return replicas;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/locator/TokenMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index 46c191f..4ab34db 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
+import org.apache.cassandra.locator.ReplicaCollection.Mutable.Conflict;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -87,6 +88,7 @@ public class TokenMetadata
     // (don't need to record Token here since it's still part of tokenToEndpointMap until it's done leaving)
     private final Set<InetAddressAndPort> leavingEndpoints = new HashSet<>();
     // this is a cache of the calculation from {tokenToEndpointMap, bootstrapTokens, leavingEndpoints}
+    // NOTE: this may contain ranges that conflict with the those implied by sortedTokens when a range is changing its transient status
     private final ConcurrentMap<String, PendingRangeMaps> pendingRanges = new ConcurrentHashMap<String, PendingRangeMaps>();
 
     // nodes which are migrating to the new tokens in the ring
@@ -733,24 +735,20 @@ public class TokenMetadata
         return sortedTokens;
     }
 
-    public Multimap<Range<Token>, InetAddressAndPort> getPendingRangesMM(String keyspaceName)
+    public EndpointsByRange getPendingRangesMM(String keyspaceName)
     {
-        Multimap<Range<Token>, InetAddressAndPort> map = HashMultimap.create();
+        EndpointsByRange.Mutable byRange = new EndpointsByRange.Mutable();
         PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
 
         if (pendingRangeMaps != null)
         {
-            for (Map.Entry<Range<Token>, List<InetAddressAndPort>> entry : pendingRangeMaps)
+            for (Map.Entry<Range<Token>, EndpointsForRange.Mutable> entry : pendingRangeMaps)
             {
-                Range<Token> range = entry.getKey();
-                for (InetAddressAndPort address : entry.getValue())
-                {
-                    map.put(range, address);
-                }
+                byRange.putAll(entry.getKey(), entry.getValue(), Conflict.ALL);
             }
         }
 
-        return map;
+        return byRange.asImmutableView();
     }
 
     /** a mutable map may be returned but caller should not modify it */
@@ -759,17 +757,18 @@ public class TokenMetadata
         return this.pendingRanges.get(keyspaceName);
     }
 
-    public List<Range<Token>> getPendingRanges(String keyspaceName, InetAddressAndPort endpoint)
+    public RangesAtEndpoint getPendingRanges(String keyspaceName, InetAddressAndPort endpoint)
     {
-        List<Range<Token>> ranges = new ArrayList<>();
-        for (Map.Entry<Range<Token>, InetAddressAndPort> entry : getPendingRangesMM(keyspaceName).entries())
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(endpoint);
+        for (Map.Entry<Range<Token>, Replica> entry : getPendingRangesMM(keyspaceName).flattenEntries())
         {
-            if (entry.getValue().equals(endpoint))
+            Replica replica = entry.getValue();
+            if (replica.endpoint().equals(endpoint))
             {
-                ranges.add(entry.getKey());
+                builder.add(replica);
             }
         }
-        return ranges;
+        return builder.build();
     }
 
      /**
@@ -858,25 +857,27 @@ public class TokenMetadata
     {
         PendingRangeMaps newPendingRanges = new PendingRangeMaps();
 
-        Multimap<InetAddressAndPort, Range<Token>> addressRanges = strategy.getAddressRanges(metadata);
+        RangesByEndpoint addressRanges = strategy.getAddressReplicas(metadata);
 
         // Copy of metadata reflecting the situation after all leave operations are finished.
         TokenMetadata allLeftMetadata = removeEndpoints(metadata.cloneOnlyTokenMap(), leavingEndpoints);
 
         // get all ranges that will be affected by leaving nodes
-        Set<Range<Token>> affectedRanges = new HashSet<Range<Token>>();
+        Set<Range<Token>> removeAffectedRanges = new HashSet<>();
         for (InetAddressAndPort endpoint : leavingEndpoints)
-            affectedRanges.addAll(addressRanges.get(endpoint));
+            removeAffectedRanges.addAll(addressRanges.get(endpoint).ranges());
 
         // for each of those ranges, find what new nodes will be responsible for the range when
         // all leaving nodes are gone.
-        for (Range<Token> range : affectedRanges)
+        for (Range<Token> range : removeAffectedRanges)
         {
-            Set<InetAddressAndPort> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-            Set<InetAddressAndPort> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-            for (InetAddressAndPort address : Sets.difference(newEndpoints, currentEndpoints))
+            EndpointsForRange currentReplicas = strategy.calculateNaturalReplicas(range.right, metadata);
+            EndpointsForRange newReplicas = strategy.calculateNaturalReplicas(range.right, allLeftMetadata);
+            for (Replica replica : newReplicas)
             {
-                newPendingRanges.addPendingRange(range, address);
+                if (currentReplicas.endpoints().contains(replica.endpoint()))
+                    continue;
+                newPendingRanges.addPendingRange(range, replica);
             }
         }
 
@@ -891,9 +892,9 @@ public class TokenMetadata
             Collection<Token> tokens = bootstrapAddresses.get(endpoint);
 
             allLeftMetadata.updateNormalTokens(tokens, endpoint);
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            for (Replica replica : strategy.getAddressReplicas(allLeftMetadata, endpoint))
             {
-                newPendingRanges.addPendingRange(range, endpoint);
+                newPendingRanges.addPendingRange(replica.range(), replica);
             }
             allLeftMetadata.removeEndpoint(endpoint);
         }
@@ -906,38 +907,43 @@ public class TokenMetadata
         for (Pair<Token, InetAddressAndPort> moving : movingEndpoints)
         {
             //Calculate all the ranges which will could be affected. This will include the ranges before and after the move.
-            Set<Range<Token>> moveAffectedRanges = new HashSet<>();
+            Set<Replica> moveAffectedReplicas = new HashSet<>();
             InetAddressAndPort endpoint = moving.right; // address of the moving node
             //Add ranges before the move
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            for (Replica replica : strategy.getAddressReplicas(allLeftMetadata, endpoint))
             {
-                moveAffectedRanges.add(range);
+                moveAffectedReplicas.add(replica);
             }
 
             allLeftMetadata.updateNormalToken(moving.left, endpoint);
             //Add ranges after the move
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            for (Replica replica : strategy.getAddressReplicas(allLeftMetadata, endpoint))
             {
-                moveAffectedRanges.add(range);
+                moveAffectedReplicas.add(replica);
             }
 
-            for(Range<Token> range : moveAffectedRanges)
+            for (Replica replica : moveAffectedReplicas)
             {
-                Set<InetAddressAndPort> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-                Set<InetAddressAndPort> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
+                Set<InetAddressAndPort> currentEndpoints = strategy.calculateNaturalReplicas(replica.range().right, metadata).endpoints();
+                Set<InetAddressAndPort> newEndpoints = strategy.calculateNaturalReplicas(replica.range().right, allLeftMetadata).endpoints();
                 Set<InetAddressAndPort> difference = Sets.difference(newEndpoints, currentEndpoints);
-                for(final InetAddressAndPort address : difference)
+                for (final InetAddressAndPort address : difference)
                 {
-                    Collection<Range<Token>> newRanges = strategy.getAddressRanges(allLeftMetadata).get(address);
-                    Collection<Range<Token>> oldRanges = strategy.getAddressRanges(metadata).get(address);
-                    //We want to get rid of any ranges which the node is currently getting.
-                    newRanges.removeAll(oldRanges);
+                    RangesAtEndpoint newReplicas = strategy.getAddressReplicas(allLeftMetadata, address);
+                    RangesAtEndpoint oldReplicas = strategy.getAddressReplicas(metadata, address);
 
-                    for(Range<Token> newRange : newRanges)
+                    // Filter out the things that are already replicated
+                    newReplicas = newReplicas.filter(r -> !oldReplicas.contains(r));
+                    for (Replica newReplica : newReplicas)
                     {
-                        for(Range<Token> pendingRange : newRange.subtractAll(oldRanges))
+                        // for correctness on write, we need to treat ranges that are becoming full differently
+                        // to those that are presently transient; however reads must continue to use the current view
+                        // for ranges that are becoming transient. We could choose to ignore them here, but it's probably
+                        // cleaner to ensure this is dealt with at point of use, where we can make a conscious decision
+                        // about which to use
+                        for (Replica pendingReplica : newReplica.subtractSameReplication(oldReplicas))
                         {
-                            newPendingRanges.addPendingRange(pendingRange, address);
+                            newPendingRanges.addPendingRange(pendingReplica.range(), pendingReplica);
                         }
                     }
                 }
@@ -1206,11 +1212,11 @@ public class TokenMetadata
         return sb.toString();
     }
 
-    public Collection<InetAddressAndPort> pendingEndpointsFor(Token token, String keyspaceName)
+    public EndpointsForToken pendingEndpointsForToken(Token token, String keyspaceName)
     {
         PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
         if (pendingRangeMaps == null)
-            return Collections.emptyList();
+            return EndpointsForToken.empty(token);
 
         return pendingRangeMaps.pendingEndpointsFor(token);
     }
@@ -1218,9 +1224,15 @@ public class TokenMetadata
     /**
      * @deprecated retained for benefit of old tests
      */
-    public Collection<InetAddressAndPort> getWriteEndpoints(Token token, String keyspaceName, Collection<InetAddressAndPort> naturalEndpoints)
+    public EndpointsForToken getWriteEndpoints(Token token, String keyspaceName, EndpointsForToken natural)
     {
-        return ImmutableList.copyOf(Iterables.concat(naturalEndpoints, pendingEndpointsFor(token, keyspaceName)));
+        EndpointsForToken pending = pendingEndpointsForToken(token, keyspaceName);
+        if (Endpoints.haveConflicts(natural, pending))
+        {
+            natural = Endpoints.resolveConflictsInNatural(natural, pending);
+            pending = Endpoints.resolveConflictsInPending(natural, pending);
+        }
+        return Endpoints.concat(natural, pending);
     }
 
     /** @return an endpoint to token multimap representation of tokenToEndpointMap (a copy) */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 9e8d542..5a90804 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -102,6 +102,8 @@ public class KeyspaceMetrics
     public final Counter speculativeFailedRetries;
     /** Needed to speculate, but didn't have enough replicas **/
     public final Counter speculativeInsufficientReplicas;
+    /** Needed to write to a transient replica to satisfy quorum **/
+    public final Counter speculativeWrites;
     /** Number of started repairs as coordinator on this keyspace */
     public final Counter repairsStarted;
     /** Number of completed repairs as coordinator on this keyspace */
@@ -268,41 +270,12 @@ public class KeyspaceMetrics
         writeFailedIdealCL = Metrics.counter(factory.createMetricName("WriteFailedIdealCL"));
         idealCLWriteLatency = new LatencyMetrics(factory, "IdealCLWrite");
 
-        speculativeRetries = createKeyspaceCounter("SpeculativeRetries", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.speculativeRetries.getCount();
-            }
-        });
-        speculativeFailedRetries = createKeyspaceCounter("SpeculativeFailedRetries", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.speculativeFailedRetries.getCount();
-            }
-        });
-        speculativeInsufficientReplicas = createKeyspaceCounter("SpeculativeInsufficientReplicas", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.speculativeInsufficientReplicas.getCount();
-            }
-        });
-        repairsStarted = createKeyspaceCounter("RepairJobsStarted", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.repairsStarted.getCount();
-            }
-        });
-        repairsCompleted = createKeyspaceCounter("RepairJobsCompleted", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.repairsCompleted.getCount();
-            }
-        });
+        speculativeRetries = createKeyspaceCounter("SpeculativeRetries", metric -> metric.speculativeRetries.getCount());
+        speculativeFailedRetries = createKeyspaceCounter("SpeculativeFailedRetries", metric -> metric.speculativeFailedRetries.getCount());
+        speculativeInsufficientReplicas = createKeyspaceCounter("SpeculativeInsufficientReplicas", metric -> metric.speculativeInsufficientReplicas.getCount());
+        speculativeWrites = createKeyspaceCounter("SpeculativeWrites", metric -> metric.speculativeWrites.getCount());
+        repairsStarted = createKeyspaceCounter("RepairJobsStarted", metric -> metric.repairsStarted.getCount());
+        repairsCompleted = createKeyspaceCounter("RepairJobsCompleted", metric -> metric.repairsCompleted.getCount());
         repairTime = Metrics.timer(factory.createMetricName("RepairTime"));
         repairPrepareTime = Metrics.timer(factory.createMetricName("RepairPrepareTime"));
         anticompactionTime = Metrics.timer(factory.createMetricName("AntiCompactionTime"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
index fe7673d..3d00b12 100644
--- a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
@@ -37,6 +37,7 @@ public class ReadRepairMetrics
     @Deprecated
     public static final Meter attempted = Metrics.meter(factory.createMetricName("Attempted"));
 
+    // Incremented when additional requests were sent during blocking read repair due to unavailable or slow nodes
     public static final Meter speculatedRead = Metrics.meter(factory.createMetricName("SpeculatedRead"));
     public static final Meter speculatedWrite = Metrics.meter(factory.createMetricName("SpeculatedWrite"));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/metrics/TableMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index 49603ba..53ebcb0 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -214,6 +214,9 @@ public class TableMetrics
     public final Counter speculativeInsufficientReplicas;
     public final Gauge<Long> speculativeSampleLatencyNanos;
 
+    public final Counter speculativeWrites;
+    public final Gauge<Long> speculativeWriteLatencyNanos;
+
     public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Read");
     public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Write");
     public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Range");
@@ -239,7 +242,7 @@ public class TableMetrics
             Keyspace k = Schema.instance.getKeyspaceInstance(keyspace);
             if (SchemaConstants.DISTRIBUTED_KEYSPACE_NAME.equals(k.getName()))
                 continue;
-            if (k.getReplicationStrategy().getReplicationFactor() < 2)
+            if (k.getReplicationStrategy().getReplicationFactor().allReplicas < 2)
                 continue;
 
             for (ColumnFamilyStore cf : k.getColumnFamilyStores())
@@ -825,13 +828,11 @@ public class TableMetrics
         speculativeRetries = createTableCounter("SpeculativeRetries");
         speculativeFailedRetries = createTableCounter("SpeculativeFailedRetries");
         speculativeInsufficientReplicas = createTableCounter("SpeculativeInsufficientReplicas");
-        speculativeSampleLatencyNanos = createTableGauge("SpeculativeSampleLatencyNanos", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.sampleLatencyNanos;
-            }
-        });
+        speculativeSampleLatencyNanos = createTableGauge("SpeculativeSampleLatencyNanos", () -> cfs.sampleReadLatencyNanos);
+
+        speculativeWrites = createTableCounter("SpeculativeWrites");
+        speculativeWriteLatencyNanos = createTableGauge("SpeculativeWriteLatencyNanos", () -> cfs.transientWriteLatencyNanos);
+
         keyCacheHitRate = Metrics.register(factory.createMetricName("KeyCacheHitRate"),
                                            aliasFactory.createMetricName("KeyCacheHitRate"),
                                            new RatioGauge()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/net/IAsyncCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IAsyncCallback.java b/src/java/org/apache/cassandra/net/IAsyncCallback.java
index 251d263..253b412 100644
--- a/src/java/org/apache/cassandra/net/IAsyncCallback.java
+++ b/src/java/org/apache/cassandra/net/IAsyncCallback.java
@@ -21,6 +21,7 @@ import com.google.common.base.Predicate;
 
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 
 /**
  * implementors of IAsyncCallback need to make sure that any public methods
@@ -30,13 +31,9 @@ import org.apache.cassandra.locator.InetAddressAndPort;
  */
 public interface IAsyncCallback<T>
 {
-    Predicate<InetAddressAndPort> isAlive = new Predicate<InetAddressAndPort>()
-    {
-        public boolean apply(InetAddressAndPort endpoint)
-        {
-            return FailureDetector.instance.isAlive(endpoint);
-        }
-    };
+    final Predicate<InetAddressAndPort> isAlive = FailureDetector.instance::isAlive;
+
+    final Predicate<Replica> isReplicaAlive = replica -> isAlive.apply(replica.endpoint());
 
     /**
      * @param msg response received.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index c8fe3b7..bd290a1 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -90,6 +90,7 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.ILatencySubscriber;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
@@ -604,8 +605,9 @@ public final class MessagingService implements MessagingServiceMBean
 
                 if (expiredCallbackInfo.shouldHint())
                 {
-                    Mutation mutation = ((WriteCallbackInfo) expiredCallbackInfo).mutation();
-                    return StorageProxy.submitHint(mutation, expiredCallbackInfo.target, null);
+                    WriteCallbackInfo writeCallbackInfo = ((WriteCallbackInfo) expiredCallbackInfo);
+                    Mutation mutation = writeCallbackInfo.mutation();
+                    return StorageProxy.submitHint(mutation, writeCallbackInfo.getReplica(), null);
                 }
 
                 return null;
@@ -961,7 +963,7 @@ public final class MessagingService implements MessagingServiceMBean
         return verbHandlers.get(type);
     }
 
-    public int addCallback(IAsyncCallback cb, MessageOut message, InetAddressAndPort to, long timeout, boolean failureCallback)
+    public int addWriteCallback(IAsyncCallback cb, MessageOut message, InetAddressAndPort to, long timeout, boolean failureCallback)
     {
         assert message.verb != Verb.MUTATION; // mutations need to call the overload with a ConsistencyLevel
         int messageId = nextId();
@@ -970,12 +972,12 @@ public final class MessagingService implements MessagingServiceMBean
         return messageId;
     }
 
-    public int addCallback(IAsyncCallback cb,
-                           MessageOut<?> message,
-                           InetAddressAndPort to,
-                           long timeout,
-                           ConsistencyLevel consistencyLevel,
-                           boolean allowHints)
+    public int addWriteCallback(IAsyncCallback cb,
+                                MessageOut<?> message,
+                                Replica to,
+                                long timeout,
+                                ConsistencyLevel consistencyLevel,
+                                boolean allowHints)
     {
         assert message.verb == Verb.MUTATION
             || message.verb == Verb.COUNTER_MUTATION
@@ -1024,7 +1026,7 @@ public final class MessagingService implements MessagingServiceMBean
      */
     public int sendRR(MessageOut message, InetAddressAndPort to, IAsyncCallback cb, long timeout, boolean failureCallback)
     {
-        int id = addCallback(cb, message, to, timeout, failureCallback);
+        int id = addWriteCallback(cb, message, to, timeout, failureCallback);
         updateBackPressureOnSend(to, cb, message);
         sendOneWay(failureCallback ? message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE) : message, id, to);
         return id;
@@ -1042,14 +1044,14 @@ public final class MessagingService implements MessagingServiceMBean
      *                suggest that a timeout occurred to the invoker of the send().
      * @return an reference to message id used to match with the result
      */
-    public int sendRR(MessageOut<?> message,
-                      InetAddressAndPort to,
-                      AbstractWriteResponseHandler<?> handler,
-                      boolean allowHints)
-    {
-        int id = addCallback(handler, message, to, message.getTimeout(), handler.consistencyLevel, allowHints);
-        updateBackPressureOnSend(to, handler, message);
-        sendOneWay(message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE), id, to);
+    public int sendWriteRR(MessageOut<?> message,
+                           Replica to,
+                           AbstractWriteResponseHandler<?> handler,
+                           boolean allowHints)
+    {
+        int id = addWriteCallback(handler, message, to, message.getTimeout(), handler.consistencyLevel(), allowHints);
+        updateBackPressureOnSend(to.endpoint(), handler, message);
+        sendOneWay(message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE), id, to.endpoint());
         return id;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
index 41ac31b..c54e7dc 100644
--- a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.net;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.FBUtilities;
@@ -30,24 +30,31 @@ public class WriteCallbackInfo extends CallbackInfo
 {
     // either a Mutation, or a Paxos Commit (MessageOut)
     private final Object mutation;
+    private final Replica replica;
 
-    public WriteCallbackInfo(InetAddressAndPort target,
+    public WriteCallbackInfo(Replica replica,
                              IAsyncCallback callback,
                              MessageOut message,
                              IVersionedSerializer<?> serializer,
                              ConsistencyLevel consistencyLevel,
                              boolean allowHints)
     {
-        super(target, callback, serializer, true);
+        super(replica.endpoint(), callback, serializer, true);
         assert message != null;
         this.mutation = shouldHint(allowHints, message, consistencyLevel);
         //Local writes shouldn't go through messaging service (https://issues.apache.org/jira/browse/CASSANDRA-10477)
         assert (!target.equals(FBUtilities.getBroadcastAddressAndPort()));
+        this.replica = replica;
     }
 
     public boolean shouldHint()
     {
-        return mutation != null && StorageProxy.shouldHint(target);
+        return mutation != null && StorageProxy.shouldHint(replica);
+    }
+
+    public Replica getReplica()
+    {
+        return replica;
     }
 
     public Mutation mutation()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/AbstractSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AbstractSyncTask.java b/src/java/org/apache/cassandra/repair/AbstractSyncTask.java
new file mode 100644
index 0000000..124baa1
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/AbstractSyncTask.java
@@ -0,0 +1,31 @@
+/*
+ * 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.repair;
+
+import java.util.List;
+
+import com.google.common.util.concurrent.AbstractFuture;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+public abstract class AbstractSyncTask extends AbstractFuture<SyncStat> implements Runnable
+{
+    protected abstract void startSync(List<Range<Token>> rangesToStream);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
index 2ca524f..eaf890a 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricLocalSyncTask.java
@@ -18,12 +18,14 @@
 
 package org.apache.cassandra.repair;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamEvent;
@@ -54,8 +56,9 @@ public class AsymmetricLocalSyncTask extends AsymmetricSyncTask implements Strea
                                          previewKind)
                           .listeners(this)
                           .flushBeforeTransfer(pendingRepair == null)
-                          // request ranges from the remote node
-                          .requestRanges(fetchFrom, desc.keyspace, rangesToFetch, desc.columnFamily);
+                          // request ranges from the remote node, see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
+                          .requestRanges(fetchFrom, desc.keyspace, RangesAtEndpoint.toDummyList(rangesToFetch),
+                                  RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);
         plan.execute();
 
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
index e24d854..2b171c9 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.streaming.SessionSummary;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTrees;
 
 public class AsymmetricRemoteSyncTask extends AsymmetricSyncTask implements CompletableRemoteSyncTask
 {
@@ -37,6 +38,10 @@ public class AsymmetricRemoteSyncTask extends AsymmetricSyncTask implements Comp
     {
         super(desc, fetchNode, fetchFrom, rangesToFetch, previewKind);
     }
+    public AsymmetricRemoteSyncTask(RepairJobDesc desc, TreeResponse to, TreeResponse from, PreviewKind previewKind)
+    {
+        this(desc, to.endpoint, from.endpoint, MerkleTrees.difference(to.trees, from.trees), previewKind);
+    }
 
     public void startSync(List<Range<Token>> rangesToFetch)
     {
@@ -46,6 +51,7 @@ public class AsymmetricRemoteSyncTask extends AsymmetricSyncTask implements Comp
         Tracing.traceRepair(message);
         MessagingService.instance().sendOneWay(request.createMessage(), request.fetchingNode);
     }
+
     public void syncComplete(boolean success, List<SessionSummary> summaries)
     {
         if (success)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
index 4d38e8a..35474af 100644
--- a/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
+++ b/src/java/org/apache/cassandra/repair/AsymmetricSyncTask.java
@@ -21,8 +21,6 @@ package org.apache.cassandra.repair;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.util.concurrent.AbstractFuture;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,7 +31,7 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.tracing.Tracing;
 
-public abstract class AsymmetricSyncTask extends AbstractFuture<SyncStat> implements Runnable
+public abstract class AsymmetricSyncTask extends AbstractSyncTask
 {
     private static Logger logger = LoggerFactory.getLogger(AsymmetricSyncTask.class);
     protected final RepairJobDesc desc;
@@ -44,9 +42,9 @@ public abstract class AsymmetricSyncTask extends AbstractFuture<SyncStat> implem
     private long startTime = Long.MIN_VALUE;
     protected volatile SyncStat stat;
 
-
     public AsymmetricSyncTask(RepairJobDesc desc, InetAddressAndPort fetchingNode, InetAddressAndPort fetchFrom, List<Range<Token>> rangesToFetch, PreviewKind previewKind)
     {
+        assert !fetchFrom.equals(fetchingNode) : "Fetching from self " + fetchFrom;
         this.desc = desc;
         this.fetchFrom = fetchFrom;
         this.fetchingNode = fetchingNode;
@@ -55,6 +53,7 @@ public abstract class AsymmetricSyncTask extends AbstractFuture<SyncStat> implem
         stat = new SyncStat(new NodePair(fetchingNode, fetchFrom), rangesToFetch.size());
         this.previewKind = previewKind;
     }
+
     public void run()
     {
         startTime = System.currentTimeMillis();
@@ -79,7 +78,4 @@ public abstract class AsymmetricSyncTask extends AbstractFuture<SyncStat> implem
         if (startTime != Long.MIN_VALUE)
             Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).metric.syncTime.update(System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
     }
-
-
-    public abstract void startSync(List<Range<Token>> rangesToFetch);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/CommonRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/CommonRange.java b/src/java/org/apache/cassandra/repair/CommonRange.java
new file mode 100644
index 0000000..928e570
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/CommonRange.java
@@ -0,0 +1,82 @@
+/*
+ * 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.repair;
+
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+/**
+ * Groups ranges with identical endpoints/transient endpoints
+ */
+public class CommonRange
+{
+    public final ImmutableSet<InetAddressAndPort> endpoints;
+    public final ImmutableSet<InetAddressAndPort> transEndpoints;
+    public final Collection<Range<Token>> ranges;
+
+    public CommonRange(Set<InetAddressAndPort> endpoints, Set<InetAddressAndPort> transEndpoints, Collection<Range<Token>> ranges)
+    {
+        Preconditions.checkArgument(endpoints != null && !endpoints.isEmpty());
+        Preconditions.checkArgument(transEndpoints != null);
+        Preconditions.checkArgument(endpoints.containsAll(transEndpoints), "transEndpoints must be a subset of endpoints");
+        Preconditions.checkArgument(ranges != null && !ranges.isEmpty());
+
+        this.endpoints = ImmutableSet.copyOf(endpoints);
+        this.transEndpoints = ImmutableSet.copyOf(transEndpoints);
+        this.ranges = new ArrayList(ranges);
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        CommonRange that = (CommonRange) o;
+
+        if (!endpoints.equals(that.endpoints)) return false;
+        if (!transEndpoints.equals(that.transEndpoints)) return false;
+        return ranges.equals(that.ranges);
+    }
+
+    public int hashCode()
+    {
+        int result = endpoints.hashCode();
+        result = 31 * result + transEndpoints.hashCode();
+        result = 31 * result + ranges.hashCode();
+        return result;
+    }
+
+    public String toString()
+    {
+        return "CommonRange{" +
+               "endpoints=" + endpoints +
+               ", transEndpoints=" + transEndpoints +
+               ", ranges=" + ranges +
+               '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java b/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
index 8aa4381..bc614dc 100644
--- a/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
+++ b/src/java/org/apache/cassandra/repair/KeyspaceRepairManager.java
@@ -25,8 +25,7 @@ import java.util.concurrent.ExecutorService;
 import com.google.common.util.concurrent.ListenableFuture;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 
 /**
  * Keyspace level hook for repair.
@@ -38,5 +37,8 @@ public interface KeyspaceRepairManager
      * been notified that the repair session has been completed, the data associated with the given session id must
      * not be combined with repaired or unrepaired data, or data from other repair sessions.
      */
-    ListenableFuture prepareIncrementalRepair(UUID sessionID, Collection<ColumnFamilyStore> tables, Collection<Range<Token>> ranges, ExecutorService executor);
+    ListenableFuture prepareIncrementalRepair(UUID sessionID,
+                                              Collection<ColumnFamilyStore> tables,
+                                              RangesAtEndpoint tokenRanges,
+                                              ExecutorService executor);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/LocalSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
deleted file mode 100644
index d7e0387..0000000
--- a/src/java/org/apache/cassandra/repair/LocalSyncTask.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.List;
-import java.util.UUID;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.streaming.PreviewKind;
-import org.apache.cassandra.streaming.ProgressInfo;
-import org.apache.cassandra.streaming.StreamEvent;
-import org.apache.cassandra.streaming.StreamEventHandler;
-import org.apache.cassandra.streaming.StreamPlan;
-import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.streaming.StreamOperation;
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
- */
-public class LocalSyncTask extends SyncTask implements StreamEventHandler
-{
-    private final TraceState state = Tracing.instance.get();
-
-    private static final Logger logger = LoggerFactory.getLogger(LocalSyncTask.class);
-
-    private final UUID pendingRepair;
-    private final boolean pullRepair;
-
-    public LocalSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, UUID pendingRepair, boolean pullRepair, PreviewKind previewKind)
-    {
-        super(desc, r1, r2, previewKind);
-        this.pendingRepair = pendingRepair;
-        this.pullRepair = pullRepair;
-    }
-
-    @VisibleForTesting
-    StreamPlan createStreamPlan(InetAddressAndPort dst, List<Range<Token>> differences)
-    {
-        StreamPlan plan = new StreamPlan(StreamOperation.REPAIR, 1, false, pendingRepair, previewKind)
-                          .listeners(this)
-                          .flushBeforeTransfer(pendingRepair == null)
-                          .requestRanges(dst, desc.keyspace, differences, desc.columnFamily);  // request ranges from the remote node
-        if (!pullRepair)
-        {
-            // send ranges to the remote node if we are not performing a pull repair
-            plan.transferRanges(dst, desc.keyspace, differences, desc.columnFamily);
-        }
-
-        return plan;
-    }
-
-    /**
-     * Starts sending/receiving our list of differences to/from the remote endpoint: creates a callback
-     * that will be called out of band once the streams complete.
-     */
-    @Override
-    protected void startSync(List<Range<Token>> differences)
-    {
-        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
-        // We can take anyone of the node as source or destination, however if one is localhost, we put at source to avoid a forwarding
-        InetAddressAndPort dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
-
-        String message = String.format("Performing streaming repair of %d ranges with %s", differences.size(), dst);
-        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
-        Tracing.traceRepair(message);
-
-        createStreamPlan(dst, differences).execute();
-    }
-
-    public void handleStreamEvent(StreamEvent event)
-    {
-        if (state == null)
-            return;
-        switch (event.eventType)
-        {
-            case STREAM_PREPARED:
-                StreamEvent.SessionPreparedEvent spe = (StreamEvent.SessionPreparedEvent) event;
-                state.trace("Streaming session with {} prepared", spe.session.peer);
-                break;
-            case STREAM_COMPLETE:
-                StreamEvent.SessionCompleteEvent sce = (StreamEvent.SessionCompleteEvent) event;
-                state.trace("Streaming session with {} {}", sce.peer, sce.success ? "completed successfully" : "failed");
-                break;
-            case FILE_PROGRESS:
-                ProgressInfo pi = ((StreamEvent.ProgressEvent) event).progress;
-                state.trace("{}/{} ({}%) {} idx:{}{}",
-                            new Object[] { FBUtilities.prettyPrintMemory(pi.currentBytes),
-                                           FBUtilities.prettyPrintMemory(pi.totalBytes),
-                                           pi.currentBytes * 100 / pi.totalBytes,
-                                           pi.direction == ProgressInfo.Direction.OUT ? "sent to" : "received from",
-                                           pi.sessionIndex,
-                                           pi.peer });
-        }
-    }
-
-    public void onSuccess(StreamState result)
-    {
-        String message = String.format("Sync complete using session %s between %s and %s on %s", desc.sessionId, r1.endpoint, r2.endpoint, desc.columnFamily);
-        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
-        Tracing.traceRepair(message);
-        set(stat.withSummaries(result.createSummaries()));
-        finished();
-    }
-
-    public void onFailure(Throwable t)
-    {
-        setException(t);
-        finished();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RemoteSyncTask.java b/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
deleted file mode 100644
index 0a47f73..0000000
--- a/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.RepairException;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.repair.messages.SyncRequest;
-import org.apache.cassandra.streaming.PreviewKind;
-import org.apache.cassandra.streaming.SessionSummary;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * RemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node
- * to repair(stream) data with other replica.
- *
- * When RemoteSyncTask receives SyncComplete from remote node, task completes.
- */
-public class RemoteSyncTask extends SyncTask implements CompletableRemoteSyncTask
-{
-    private static final Logger logger = LoggerFactory.getLogger(RemoteSyncTask.class);
-
-    public RemoteSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, PreviewKind previewKind)
-    {
-        super(desc, r1, r2, previewKind);
-    }
-
-    @Override
-    protected void startSync(List<Range<Token>> differences)
-    {
-        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
-        SyncRequest request = new SyncRequest(desc, local, r1.endpoint, r2.endpoint, differences, previewKind);
-        String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.src, request.dst);
-        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
-        Tracing.traceRepair(message);
-        MessagingService.instance().sendOneWay(request.createMessage(), request.src);
-    }
-
-    public void syncComplete(boolean success, List<SessionSummary> summaries)
-    {
-        if (success)
-        {
-            set(stat.withSummaries(summaries));
-        }
-        else
-        {
-            setException(new RepairException(desc, previewKind, String.format("Sync failed between %s and %s", r1.endpoint, r2.endpoint)));
-        }
-        finished();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 48973d2..d38435b 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -64,7 +64,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
     public RepairJob(RepairSession session, String columnFamily, boolean isIncremental, PreviewKind previewKind, boolean optimiseStreams)
     {
         this.session = session;
-        this.desc = new RepairJobDesc(session.parentRepairSession, session.getId(), session.keyspace, columnFamily, session.getRanges());
+        this.desc = new RepairJobDesc(session.parentRepairSession, session.getId(), session.keyspace, columnFamily, session.commonRange.ranges);
         this.taskExecutor = session.taskExecutor;
         this.parallelismDegree = session.parallelismDegree;
         this.isIncremental = isIncremental;
@@ -83,7 +83,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         Keyspace ks = Keyspace.open(desc.keyspace);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(desc.columnFamily);
         cfs.metric.repairsStarted.inc();
-        List<InetAddressAndPort> allEndpoints = new ArrayList<>(session.endpoints);
+        List<InetAddressAndPort> allEndpoints = new ArrayList<>(session.commonRange.endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddressAndPort());
 
         ListenableFuture<List<TreeResponse>> validations;
@@ -160,13 +160,18 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         }, taskExecutor);
     }
 
+    private boolean isTransient(InetAddressAndPort ep)
+    {
+        return session.commonRange.transEndpoints.contains(ep);
+    }
+
     private AsyncFunction<List<TreeResponse>, List<SyncStat>> standardSyncing()
     {
         return trees ->
         {
             InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
 
-            List<SyncTask> syncTasks = new ArrayList<>();
+            List<AbstractSyncTask> syncTasks = new ArrayList<>();
             // We need to difference all trees one against another
             for (int i = 0; i < trees.size() - 1; ++i)
             {
@@ -174,17 +179,29 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
                 for (int j = i + 1; j < trees.size(); ++j)
                 {
                     TreeResponse r2 = trees.get(j);
-                    SyncTask task;
+
+                    if (isTransient(r1.endpoint) && isTransient(r2.endpoint))
+                        continue;
+
+                    AbstractSyncTask task;
                     if (r1.endpoint.equals(local) || r2.endpoint.equals(local))
                     {
-                        task = new LocalSyncTask(desc, r1, r2, isIncremental ? desc.parentSessionId : null, session.pullRepair, session.previewKind);
+                        InetAddressAndPort remote = r1.endpoint.equals(local) ? r2.endpoint : r1.endpoint;
+                        task = new SymmetricLocalSyncTask(desc, r1, r2, isTransient(remote), isIncremental ? desc.parentSessionId : null, session.pullRepair, session.previewKind);
+                    }
+                    else if (isTransient(r1.endpoint) || isTransient(r2.endpoint))
+                    {
+                        TreeResponse streamFrom = isTransient(r1.endpoint) ? r1 : r2;
+                        TreeResponse streamTo = isTransient(r1.endpoint) ? r2: r1;
+                        task = new AsymmetricRemoteSyncTask(desc, streamTo, streamFrom, previewKind);
+                        session.waitForSync(Pair.create(desc, new NodePair(streamTo.endpoint, streamFrom.endpoint)), (AsymmetricRemoteSyncTask) task);
                     }
                     else
                     {
-                        task = new RemoteSyncTask(desc, r1, r2, session.previewKind);
-                        // RemoteSyncTask expects SyncComplete message sent back.
+                        task = new SymmetricRemoteSyncTask(desc, r1, r2, session.previewKind);
+                        // SymmetricRemoteSyncTask expects SyncComplete message sent back.
                         // Register task to RepairSession to receive response.
-                        session.waitForSync(Pair.create(desc, new NodePair(r1.endpoint, r2.endpoint)), (RemoteSyncTask) task);
+                        session.waitForSync(Pair.create(desc, new NodePair(r1.endpoint, r2.endpoint)), (SymmetricRemoteSyncTask) task);
                     }
                     syncTasks.add(task);
                     taskExecutor.submit(task);
@@ -200,7 +217,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         {
             InetAddressAndPort local = FBUtilities.getLocalAddressAndPort();
 
-            List<AsymmetricSyncTask> syncTasks = new ArrayList<>();
+            List<AbstractSyncTask> syncTasks = new ArrayList<>();
             // We need to difference all trees one against another
             DifferenceHolder diffHolder = new DifferenceHolder(trees);
 
@@ -215,6 +232,11 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
             for (int i = 0; i < trees.size(); i++)
             {
                 InetAddressAndPort address = trees.get(i).endpoint;
+
+                // we don't stream to transient replicas
+                if (isTransient(address))
+                    continue;
+
                 HostDifferences streamsFor = reducedDifferences.get(address);
                 if (streamsFor != null)
                 {
@@ -373,4 +395,4 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         }
         return Futures.allAsList(tasks);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 90c0146..8d3cd54 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.repair;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ExecutorService;
@@ -29,8 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableCollection;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -38,8 +36,9 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.*;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.Replica;
 import org.apache.commons.lang3.time.DurationFormatUtils;
-import org.junit.internal.runners.statements.Fail;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,7 +73,6 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.WrappedRunnable;
 import org.apache.cassandra.utils.progress.ProgressEvent;
@@ -141,46 +139,6 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
         recordFailure(message, completionMessage);
     }
 
-    @VisibleForTesting
-    static class CommonRange
-    {
-        public final Set<InetAddressAndPort> endpoints;
-        public final Collection<Range<Token>> ranges;
-
-        public CommonRange(Set<InetAddressAndPort> endpoints, Collection<Range<Token>> ranges)
-        {
-            Preconditions.checkArgument(endpoints != null && !endpoints.isEmpty());
-            Preconditions.checkArgument(ranges != null && !ranges.isEmpty());
-            this.endpoints = endpoints;
-            this.ranges = ranges;
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            CommonRange that = (CommonRange) o;
-
-            if (!endpoints.equals(that.endpoints)) return false;
-            return ranges.equals(that.ranges);
-        }
-
-        public int hashCode()
-        {
-            int result = endpoints.hashCode();
-            result = 31 * result + ranges.hashCode();
-            return result;
-        }
-
-        public String toString()
-        {
-            return "CommonRange{" +
-                   "endpoints=" + endpoints +
-                   ", ranges=" + ranges +
-                   '}';
-        }
-    }
 
     protected void runMayThrow() throws Exception
     {
@@ -238,20 +196,24 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
         Set<InetAddressAndPort> allNeighbors = new HashSet<>();
         List<CommonRange> commonRanges = new ArrayList<>();
 
-        //pre-calculate output of getLocalRanges and pass it to getNeighbors to increase performance and prevent
+        //pre-calculate output of getLocalReplicas and pass it to getNeighbors to increase performance and prevent
         //calculation multiple times
-        Collection<Range<Token>> keyspaceLocalRanges = storageService.getLocalRanges(keyspace);
+        // we arbitrarily limit ourselves to only full replicas, in lieu of ensuring it is safe to coordinate from a transient replica
+        Iterable<Range<Token>> keyspaceLocalRanges = storageService
+                .getLocalReplicas(keyspace)
+                .filter(Replica::isFull)
+                .ranges();
 
         try
         {
             for (Range<Token> range : options.getRanges())
             {
-                Set<InetAddressAndPort> neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
-                                                                                     options.getDataCenters(),
-                                                                                     options.getHosts());
+                EndpointsForRange neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
+                                                                        options.getDataCenters(),
+                                                                        options.getHosts());
 
                 addRangeToNeighbors(commonRanges, range, neighbors);
-                allNeighbors.addAll(neighbors);
+                allNeighbors.addAll(neighbors.endpoints());
             }
 
             progress.incrementAndGet();
@@ -387,11 +349,13 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
             for (CommonRange commonRange: commonRanges)
             {
                 Set<InetAddressAndPort> endpoints = ImmutableSet.copyOf(Iterables.filter(commonRange.endpoints, liveEndpoints::contains));
+                Set<InetAddressAndPort> transEndpoints = ImmutableSet.copyOf(Iterables.filter(commonRange.transEndpoints, liveEndpoints::contains));
+                Preconditions.checkState(endpoints.containsAll(transEndpoints), "transEndpoints must be a subset of endpoints");
 
                 // this node is implicitly a participant in this repair, so a single endpoint is ok here
                 if (!endpoints.isEmpty())
                 {
-                    filtered.add(new CommonRange(endpoints, commonRange.ranges));
+                    filtered.add(new CommonRange(endpoints, transEndpoints, commonRange.ranges));
                 }
             }
             Preconditions.checkState(!filtered.isEmpty(), "Not enough live endpoints for a repair");
@@ -514,14 +478,13 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
         // we do endpoint filtering at the start of an incremental repair,
         // so repair sessions shouldn't also be checking liveness
         boolean force = options.isForcedRepair() && !isIncremental;
-        for (CommonRange cr : commonRanges)
+        for (CommonRange commonRange : commonRanges)
         {
-            logger.info("Starting RepairSession for {}", cr);
+            logger.info("Starting RepairSession for {}", commonRange);
             RepairSession session = ActiveRepairService.instance.submitRepairSession(parentSession,
-                                                                                     cr.ranges,
+                                                                                     commonRange,
                                                                                      keyspace,
                                                                                      options.getParallelism(),
-                                                                                     cr.endpoints,
                                                                                      isIncremental,
                                                                                      options.isPullRepair(),
                                                                                      force,
@@ -559,7 +522,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
         public void onSuccess(RepairSessionResult result)
         {
             String message = String.format("Repair session %s for range %s finished", session.getId(),
-                                           session.getRanges().toString());
+                                           session.ranges().toString());
             logger.info(message);
             fireProgressEvent(new ProgressEvent(ProgressEventType.PROGRESS,
                                                 progress.incrementAndGet(),
@@ -572,7 +535,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
             StorageMetrics.repairExceptions.inc();
 
             String message = String.format("Repair session %s for range %s failed with error %s",
-                                           session.getId(), session.getRanges().toString(), t.getMessage());
+                                           session.getId(), session.ranges().toString(), t.getMessage());
             logger.error(message, t);
             fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR,
                                                 progress.incrementAndGet(),
@@ -684,13 +647,15 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
                                                ImmutableList.of(failureMessage, completionMessage));
     }
 
-    private void addRangeToNeighbors(List<CommonRange> neighborRangeList, Range<Token> range, Set<InetAddressAndPort> neighbors)
+    private void addRangeToNeighbors(List<CommonRange> neighborRangeList, Range<Token> range, EndpointsForRange neighbors)
     {
+        Set<InetAddressAndPort> endpoints = neighbors.endpoints();
+        Set<InetAddressAndPort> transEndpoints = neighbors.filter(Replica::isTransient).endpoints();
         for (int i = 0; i < neighborRangeList.size(); i++)
         {
             CommonRange cr = neighborRangeList.get(i);
 
-            if (cr.endpoints.containsAll(neighbors))
+            if (cr.endpoints.containsAll(endpoints) && cr.transEndpoints.containsAll(transEndpoints))
             {
                 cr.ranges.add(range);
                 return;
@@ -699,7 +664,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
         List<Range<Token>> ranges = new ArrayList<>();
         ranges.add(range);
-        neighborRangeList.add(new CommonRange(neighbors, ranges));
+        neighborRangeList.add(new CommonRange(endpoints, transEndpoints, ranges));
     }
 
     private Thread createQueryThread(final int cmd, final UUID sessionId)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index ec06f37..2ff60ec 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -55,8 +55,8 @@ import org.apache.cassandra.utils.Pair;
  *      validationComplete()).
  *   </li>
  *   <li>Synchronization phase: once all trees are received, the job compares each tree with
- *      all the other using a so-called {@link SyncTask}. If there is difference between 2 trees, the
- *      concerned SyncTask will start a streaming of the difference between the 2 endpoint concerned.
+ *      all the other using a so-called {@link SymmetricSyncTask}. If there is difference between 2 trees, the
+ *      concerned SymmetricSyncTask will start a streaming of the difference between the 2 endpoint concerned.
  *   </li>
  * </ol>
  * The job is done once all its SyncTasks are done (i.e. have either computed no differences
@@ -74,7 +74,7 @@ import org.apache.cassandra.utils.Pair;
  * we still first send a message to each node to flush and snapshot data so each merkle tree
  * creation is still done on similar data, even if the actual creation is not
  * done simulatneously). If not sequential, all merkle tree are requested in parallel.
- * Similarly, if a job is sequential, it will handle one SyncTask at a time, but will handle
+ * Similarly, if a job is sequential, it will handle one SymmetricSyncTask at a time, but will handle
  * all of them in parallel otherwise.
  */
 public class RepairSession extends AbstractFuture<RepairSessionResult> implements IEndpointStateChangeSubscriber,
@@ -94,8 +94,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
     public final boolean skippedReplicas;
 
     /** Range to repair */
-    public final Collection<Range<Token>> ranges;
-    public final Set<InetAddressAndPort> endpoints;
+    public final CommonRange commonRange;
     public final boolean isIncremental;
     public final PreviewKind previewKind;
 
@@ -114,23 +113,20 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
 
     /**
      * Create new repair session.
-     *
      * @param parentRepairSession the parent sessions id
      * @param id this sessions id
-     * @param ranges ranges to repair
+     * @param commonRange ranges to repair
      * @param keyspace name of keyspace
      * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
-     * @param endpoints the data centers that should be part of the repair; null for all DCs
      * @param pullRepair true if the repair should be one way (from remote host to this host and only applicable between two hosts--see RepairOption)
      * @param force true if the repair should ignore dead endpoints (instead of failing)
      * @param cfnames names of columnfamilies
      */
     public RepairSession(UUID parentRepairSession,
                          UUID id,
-                         Collection<Range<Token>> ranges,
+                         CommonRange commonRange,
                          String keyspace,
                          RepairParallelism parallelismDegree,
-                         Set<InetAddressAndPort> endpoints,
                          boolean isIncremental,
                          boolean pullRepair,
                          boolean force,
@@ -145,7 +141,6 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         this.parallelismDegree = parallelismDegree;
         this.keyspace = keyspace;
         this.cfnames = cfnames;
-        this.ranges = ranges;
 
         //If force then filter out dead endpoints
         boolean forceSkippedReplicas = false;
@@ -153,7 +148,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         {
             logger.debug("force flag set, removing dead endpoints");
             final Set<InetAddressAndPort> removeCandidates = new HashSet<>();
-            for (final InetAddressAndPort endpoint : endpoints)
+            for (final InetAddressAndPort endpoint : commonRange.endpoints)
             {
                 if (!FailureDetector.instance.isAlive(endpoint))
                 {
@@ -166,12 +161,13 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
                 // we shouldn't be recording a successful repair if
                 // any replicas are excluded from the repair
                 forceSkippedReplicas = true;
-                endpoints = new HashSet<>(endpoints);
-                endpoints.removeAll(removeCandidates);
+                Set<InetAddressAndPort> filteredEndpoints = new HashSet<>(commonRange.endpoints);
+                filteredEndpoints.removeAll(removeCandidates);
+                commonRange = new CommonRange(filteredEndpoints, commonRange.transEndpoints, commonRange.ranges);
             }
         }
 
-        this.endpoints = endpoints;
+        this.commonRange = commonRange;
         this.isIncremental = isIncremental;
         this.previewKind = previewKind;
         this.pullRepair = pullRepair;
@@ -184,9 +180,14 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         return id;
     }
 
-    public Collection<Range<Token>> getRanges()
+    public Collection<Range<Token>> ranges()
+    {
+        return commonRange.ranges;
+    }
+
+    public Collection<InetAddressAndPort> endpoints()
     {
-        return ranges;
+        return commonRange.endpoints;
     }
 
     public void waitForValidation(Pair<RepairJobDesc, InetAddressAndPort> key, ValidationTask task)
@@ -247,7 +248,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
     {
         StringBuilder sb = new StringBuilder();
         sb.append(FBUtilities.getBroadcastAddressAndPort());
-        for (InetAddressAndPort ep : endpoints)
+        for (InetAddressAndPort ep : commonRange.endpoints)
             sb.append(", ").append(ep);
         return sb.toString();
     }
@@ -266,18 +267,18 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         if (terminated)
             return;
 
-        logger.info("{} new session: will sync {} on range {} for {}.{}", previewKind.logPrefix(getId()), repairedNodes(), ranges, keyspace, Arrays.toString(cfnames));
-        Tracing.traceRepair("Syncing range {}", ranges);
+        logger.info("{} new session: will sync {} on range {} for {}.{}", previewKind.logPrefix(getId()), repairedNodes(), commonRange, keyspace, Arrays.toString(cfnames));
+        Tracing.traceRepair("Syncing range {}", commonRange);
         if (!previewKind.isPreview())
         {
-            SystemDistributedKeyspace.startRepairs(getId(), parentRepairSession, keyspace, cfnames, ranges, endpoints);
+            SystemDistributedKeyspace.startRepairs(getId(), parentRepairSession, keyspace, cfnames, commonRange);
         }
 
-        if (endpoints.isEmpty())
+        if (commonRange.endpoints.isEmpty())
         {
-            logger.info("{} {}", previewKind.logPrefix(getId()), message = String.format("No neighbors to repair with on range %s: session completed", ranges));
+            logger.info("{} {}", previewKind.logPrefix(getId()), message = String.format("No neighbors to repair with on range %s: session completed", commonRange));
             Tracing.traceRepair(message);
-            set(new RepairSessionResult(id, keyspace, ranges, Lists.<RepairResult>newArrayList(), skippedReplicas));
+            set(new RepairSessionResult(id, keyspace, commonRange.ranges, Lists.<RepairResult>newArrayList(), skippedReplicas));
             if (!previewKind.isPreview())
             {
                 SystemDistributedKeyspace.failRepairs(getId(), keyspace, cfnames, new RuntimeException(message));
@@ -286,7 +287,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
         }
 
         // Checking all nodes are live
-        for (InetAddressAndPort endpoint : endpoints)
+        for (InetAddressAndPort endpoint : commonRange.endpoints)
         {
             if (!FailureDetector.instance.isAlive(endpoint) && !skippedReplicas)
             {
@@ -318,8 +319,8 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
             {
                 // this repair session is completed
                 logger.info("{} {}", previewKind.logPrefix(getId()), "Session completed successfully");
-                Tracing.traceRepair("Completed sync of range {}", ranges);
-                set(new RepairSessionResult(id, keyspace, ranges, results, skippedReplicas));
+                Tracing.traceRepair("Completed sync of range {}", commonRange);
+                set(new RepairSessionResult(id, keyspace, commonRange.ranges, results, skippedReplicas));
 
                 taskExecutor.shutdown();
                 // mark this session as terminated
@@ -372,7 +373,7 @@ public class RepairSession extends AbstractFuture<RepairSessionResult> implement
 
     public void convict(InetAddressAndPort endpoint, double phi)
     {
-        if (!endpoints.contains(endpoint))
+        if (!commonRange.endpoints.contains(endpoint))
             return;
 
         // We want a higher confidence in the failure detection than usual because failing a repair wrongly has a high cost.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
index 5d2b2ec..e9cba89 100644
--- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.Collection;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,9 +79,12 @@ public class StreamingRepairTask implements Runnable, StreamEventHandler
         StreamPlan sp = new StreamPlan(StreamOperation.REPAIR, 1, false, pendingRepair, previewKind)
                .listeners(this)
                .flushBeforeTransfer(pendingRepair == null) // sstables are isolated at the beginning of an incremental repair session, so flushing isn't neccessary
-               .requestRanges(dest, desc.keyspace, ranges, desc.columnFamily); // request ranges from the remote node
+               // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
+               .requestRanges(dest, desc.keyspace, RangesAtEndpoint.toDummyList(ranges),
+                       RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily); // request ranges from the remote node
         if (!asymmetric)
-            sp.transferRanges(dest, desc.keyspace, ranges, desc.columnFamily); // send ranges to the remote node
+            // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
+            sp.transferRanges(dest, desc.keyspace, RangesAtEndpoint.toDummyList(ranges), desc.columnFamily); // send ranges to the remote node
         return sp;
     }
 


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


[11/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/SymmetricLocalSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SymmetricLocalSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricLocalSyncTask.java
new file mode 100644
index 0000000..7eedab7
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SymmetricLocalSyncTask.java
@@ -0,0 +1,142 @@
+/*
+ * 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.repair;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamEvent;
+import org.apache.cassandra.streaming.StreamEventHandler;
+import org.apache.cassandra.streaming.StreamOperation;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * SymmetricLocalSyncTask performs streaming between local(coordinator) node and remote replica.
+ */
+public class SymmetricLocalSyncTask extends SymmetricSyncTask implements StreamEventHandler
+{
+    private final TraceState state = Tracing.instance.get();
+
+    private static final Logger logger = LoggerFactory.getLogger(SymmetricLocalSyncTask.class);
+
+    private final boolean remoteIsTransient;
+    private final UUID pendingRepair;
+    private final boolean pullRepair;
+
+    public SymmetricLocalSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, boolean remoteIsTransient, UUID pendingRepair, boolean pullRepair, PreviewKind previewKind)
+    {
+        super(desc, r1, r2, previewKind);
+        this.remoteIsTransient = remoteIsTransient;
+        this.pendingRepair = pendingRepair;
+        this.pullRepair = pullRepair;
+    }
+
+    @VisibleForTesting
+    StreamPlan createStreamPlan(InetAddressAndPort dst, List<Range<Token>> differences)
+    {
+        StreamPlan plan = new StreamPlan(StreamOperation.REPAIR, 1, false, pendingRepair, previewKind)
+                          .listeners(this)
+                          .flushBeforeTransfer(pendingRepair == null)
+                          // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
+                          .requestRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences),
+                                  RangesAtEndpoint.toDummyList(Collections.emptyList()), desc.columnFamily);  // request ranges from the remote node
+
+        if (!pullRepair && !remoteIsTransient)
+        {
+            // send ranges to the remote node if we are not performing a pull repair
+            // see comment on RangesAtEndpoint.toDummyList for why we synthesize replicas here
+            plan.transferRanges(dst, desc.keyspace, RangesAtEndpoint.toDummyList(differences), desc.columnFamily);
+        }
+
+        return plan;
+    }
+
+    /**
+     * Starts sending/receiving our list of differences to/from the remote endpoint: creates a callback
+     * that will be called out of band once the streams complete.
+     */
+    @Override
+    protected void startSync(List<Range<Token>> differences)
+    {
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
+        // We can take anyone of the node as source or destination, however if one is localhost, we put at source to avoid a forwarding
+        InetAddressAndPort dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
+
+        String message = String.format("Performing streaming repair of %d ranges with %s", differences.size(), dst);
+        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
+        Tracing.traceRepair(message);
+
+        createStreamPlan(dst, differences).execute();
+    }
+
+    public void handleStreamEvent(StreamEvent event)
+    {
+        if (state == null)
+            return;
+        switch (event.eventType)
+        {
+            case STREAM_PREPARED:
+                StreamEvent.SessionPreparedEvent spe = (StreamEvent.SessionPreparedEvent) event;
+                state.trace("Streaming session with {} prepared", spe.session.peer);
+                break;
+            case STREAM_COMPLETE:
+                StreamEvent.SessionCompleteEvent sce = (StreamEvent.SessionCompleteEvent) event;
+                state.trace("Streaming session with {} {}", sce.peer, sce.success ? "completed successfully" : "failed");
+                break;
+            case FILE_PROGRESS:
+                ProgressInfo pi = ((StreamEvent.ProgressEvent) event).progress;
+                state.trace("{}/{} ({}%) {} idx:{}{}",
+                            new Object[] { FBUtilities.prettyPrintMemory(pi.currentBytes),
+                                           FBUtilities.prettyPrintMemory(pi.totalBytes),
+                                           pi.currentBytes * 100 / pi.totalBytes,
+                                           pi.direction == ProgressInfo.Direction.OUT ? "sent to" : "received from",
+                                           pi.sessionIndex,
+                                           pi.peer });
+        }
+    }
+
+    public void onSuccess(StreamState result)
+    {
+        String message = String.format("Sync complete using session %s between %s and %s on %s", desc.sessionId, r1.endpoint, r2.endpoint, desc.columnFamily);
+        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
+        Tracing.traceRepair(message);
+        set(stat.withSummaries(result.createSummaries()));
+        finished();
+    }
+
+    public void onFailure(Throwable t)
+    {
+        setException(t);
+        finished();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
new file mode 100644
index 0000000..1f2740f
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java
@@ -0,0 +1,84 @@
+/*
+ * 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.repair;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.repair.messages.AsymmetricSyncRequest;
+import org.apache.cassandra.repair.messages.RepairMessage;
+import org.apache.cassandra.repair.messages.SyncRequest;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.SessionSummary;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * SymmetricRemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node
+ * to repair(stream) data with other replica.
+ *
+ * When SymmetricRemoteSyncTask receives SyncComplete from remote node, task completes.
+ */
+public class SymmetricRemoteSyncTask extends SymmetricSyncTask implements CompletableRemoteSyncTask
+{
+    private static final Logger logger = LoggerFactory.getLogger(SymmetricRemoteSyncTask.class);
+
+    public SymmetricRemoteSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, PreviewKind previewKind)
+    {
+        super(desc, r1, r2, previewKind);
+    }
+
+    void sendRequest(RepairMessage request, InetAddressAndPort to)
+    {
+        MessagingService.instance().sendOneWay(request.createMessage(), to);
+    }
+
+    @Override
+    protected void startSync(List<Range<Token>> differences)
+    {
+        InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
+
+        SyncRequest request = new SyncRequest(desc, local, r1.endpoint, r2.endpoint, differences, previewKind);
+        String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.src, request.dst);
+        logger.info("{} {}", previewKind.logPrefix(desc.sessionId), message);
+        Tracing.traceRepair(message);
+        sendRequest(request, request.src);
+    }
+
+    public void syncComplete(boolean success, List<SessionSummary> summaries)
+    {
+        if (success)
+        {
+            set(stat.withSummaries(summaries));
+        }
+        else
+        {
+            setException(new RepairException(desc, previewKind, String.format("Sync failed between %s and %s", r1.endpoint, r2.endpoint)));
+        }
+        finished();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/SymmetricSyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SymmetricSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricSyncTask.java
new file mode 100644
index 0000000..3da2293
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SymmetricSyncTask.java
@@ -0,0 +1,94 @@
+/*
+ * 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.repair;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.MerkleTrees;
+
+/**
+ * SymmetricSyncTask will calculate the difference of MerkleTree between two nodes
+ * and perform necessary operation to repair replica.
+ */
+public abstract class SymmetricSyncTask extends AbstractSyncTask
+{
+    private static Logger logger = LoggerFactory.getLogger(SymmetricSyncTask.class);
+
+    protected final RepairJobDesc desc;
+    protected final TreeResponse r1;
+    protected final TreeResponse r2;
+    protected final PreviewKind previewKind;
+
+    protected volatile SyncStat stat;
+    protected long startTime = Long.MIN_VALUE;
+
+    public SymmetricSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, PreviewKind previewKind)
+    {
+        this.desc = desc;
+        this.r1 = r1;
+        this.r2 = r2;
+        this.previewKind = previewKind;
+    }
+
+    /**
+     * Compares trees, and triggers repairs for any ranges that mismatch.
+     */
+    public void run()
+    {
+        startTime = System.currentTimeMillis();
+        // compare trees, and collect differences
+        List<Range<Token>> differences = MerkleTrees.difference(r1.trees, r2.trees);
+
+        stat = new SyncStat(new NodePair(r1.endpoint, r2.endpoint), differences.size());
+
+        // choose a repair method based on the significance of the difference
+        String format = String.format("%s Endpoints %s and %s %%s for %s", previewKind.logPrefix(desc.sessionId), r1.endpoint, r2.endpoint, desc.columnFamily);
+        if (differences.isEmpty())
+        {
+            logger.info(String.format(format, "are consistent"));
+            Tracing.traceRepair("Endpoint {} is consistent with {} for {}", r1.endpoint, r2.endpoint, desc.columnFamily);
+            set(stat);
+            return;
+        }
+
+        // non-0 difference: perform streaming repair
+        logger.info(String.format(format, "have " + differences.size() + " range(s) out of sync"));
+        Tracing.traceRepair("Endpoint {} has {} range(s) out of sync with {} for {}", r1.endpoint, differences.size(), r2.endpoint, desc.columnFamily);
+        startSync(differences);
+    }
+
+    public SyncStat getCurrentStat()
+    {
+        return stat;
+    }
+
+    protected void finished()
+    {
+        if (startTime != Long.MIN_VALUE)
+            Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).metric.syncTime.update(System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/SyncTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java
deleted file mode 100644
index f7cf5f1..0000000
--- a/src/java/org/apache/cassandra/repair/SyncTask.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.util.concurrent.AbstractFuture;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.streaming.PreviewKind;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.MerkleTrees;
-
-/**
- * SyncTask will calculate the difference of MerkleTree between two nodes
- * and perform necessary operation to repair replica.
- */
-public abstract class SyncTask extends AbstractFuture<SyncStat> implements Runnable
-{
-    private static Logger logger = LoggerFactory.getLogger(SyncTask.class);
-
-    protected final RepairJobDesc desc;
-    protected final TreeResponse r1;
-    protected final TreeResponse r2;
-    protected final PreviewKind previewKind;
-
-    protected volatile SyncStat stat;
-    protected long startTime = Long.MIN_VALUE;
-
-    public SyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, PreviewKind previewKind)
-    {
-        this.desc = desc;
-        this.r1 = r1;
-        this.r2 = r2;
-        this.previewKind = previewKind;
-    }
-
-    /**
-     * Compares trees, and triggers repairs for any ranges that mismatch.
-     */
-    public void run()
-    {
-        startTime = System.currentTimeMillis();
-        // compare trees, and collect differences
-        List<Range<Token>> differences = MerkleTrees.difference(r1.trees, r2.trees);
-
-        stat = new SyncStat(new NodePair(r1.endpoint, r2.endpoint), differences.size());
-
-        // choose a repair method based on the significance of the difference
-        String format = String.format("%s Endpoints %s and %s %%s for %s", previewKind.logPrefix(desc.sessionId), r1.endpoint, r2.endpoint, desc.columnFamily);
-        if (differences.isEmpty())
-        {
-            logger.info(String.format(format, "are consistent"));
-            Tracing.traceRepair("Endpoint {} is consistent with {} for {}", r1.endpoint, r2.endpoint, desc.columnFamily);
-            set(stat);
-            return;
-        }
-
-        // non-0 difference: perform streaming repair
-        logger.info(String.format(format, "have " + differences.size() + " range(s) out of sync"));
-        Tracing.traceRepair("Endpoint {} has {} range(s) out of sync with {} for {}", r1.endpoint, differences.size(), r2.endpoint, desc.columnFamily);
-        startSync(differences);
-    }
-
-    public SyncStat getCurrentStat()
-    {
-        return stat;
-    }
-
-    protected void finished()
-    {
-        if (startTime != Long.MIN_VALUE)
-            Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).metric.syncTime.update(System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
-    }
-
-    protected abstract void startSync(List<Range<Token>> differences);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index a85a1e5..fc09e71 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -185,13 +185,13 @@ public final class SystemDistributedKeyspace
         processSilent(fmtQuery);
     }
 
-    public static void startRepairs(UUID id, UUID parent_id, String keyspaceName, String[] cfnames, Collection<Range<Token>> ranges, Iterable<InetAddressAndPort> endpoints)
+    public static void startRepairs(UUID id, UUID parent_id, String keyspaceName, String[] cfnames, CommonRange commonRange)
     {
         InetAddressAndPort coordinator = FBUtilities.getBroadcastAddressAndPort();
         Set<String> participants = Sets.newHashSet();
         Set<String> participants_v2 = Sets.newHashSet();
 
-        for (InetAddressAndPort endpoint : endpoints)
+        for (InetAddressAndPort endpoint : commonRange.endpoints)
         {
             participants.add(endpoint.getHostAddress(false));
             participants_v2.add(endpoint.toString());
@@ -203,7 +203,7 @@ public final class SystemDistributedKeyspace
 
         for (String cfname : cfnames)
         {
-            for (Range<Token> range : ranges)
+            for (Range<Token> range : commonRange.ranges)
             {
                 String fmtQry = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
                                               keyspaceName,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
index ed25166..4089e77 100644
--- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
+++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java
@@ -47,11 +47,14 @@ import com.google.common.primitives.Ints;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+
+import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.repair.KeyspaceRepairManager;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -543,9 +546,35 @@ public class LocalSessions
     }
 
     @VisibleForTesting
-    ListenableFuture prepareSession(KeyspaceRepairManager repairManager, UUID sessionID, Collection<ColumnFamilyStore> tables, Collection<Range<Token>> ranges, ExecutorService executor)
+    ListenableFuture prepareSession(KeyspaceRepairManager repairManager,
+                                    UUID sessionID,
+                                    Collection<ColumnFamilyStore> tables,
+                                    RangesAtEndpoint tokenRanges,
+                                    ExecutorService executor)
+    {
+        return repairManager.prepareIncrementalRepair(sessionID, tables, tokenRanges, executor);
+    }
+
+    RangesAtEndpoint filterLocalRanges(String keyspace, Set<Range<Token>> ranges)
     {
-        return repairManager.prepareIncrementalRepair(sessionID, tables, ranges, executor);
+        RangesAtEndpoint localRanges = StorageService.instance.getLocalReplicas(keyspace);
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(localRanges.endpoint());
+        for (Range<Token> range : ranges)
+        {
+            for (Replica replica : localRanges)
+            {
+                if (replica.range().equals(range))
+                {
+                    builder.add(replica);
+                }
+                else if (replica.contains(range))
+                {
+                    builder.add(replica.decorateSubrange(range));
+                }
+            }
+
+        }
+        return builder.build();
     }
 
     /**
@@ -582,7 +611,8 @@ public class LocalSessions
         ExecutorService executor = Executors.newFixedThreadPool(parentSession.getColumnFamilyStores().size());
 
         KeyspaceRepairManager repairManager = parentSession.getKeyspace().getRepairManager();
-        ListenableFuture repairPreparation = prepareSession(repairManager, sessionID, parentSession.getColumnFamilyStores(), parentSession.getRanges(), executor);
+        RangesAtEndpoint tokenRanges = filterLocalRanges(parentSession.getKeyspace().getName(), parentSession.getRanges());
+        ListenableFuture repairPreparation = prepareSession(repairManager, sessionID, parentSession.getColumnFamilyStores(), tokenRanges, executor);
 
         Futures.addCallback(repairPreparation, new FutureCallback<Object>()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
index 68ac5e4..cc46474 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@ -74,6 +74,11 @@ public final class KeyspaceParams
         return new KeyspaceParams(true, ReplicationParams.simple(replicationFactor));
     }
 
+    public static KeyspaceParams simple(String replicationFactor)
+    {
+        return new KeyspaceParams(true, ReplicationParams.simple(replicationFactor));
+    }
+
     public static KeyspaceParams simpleTransient(int replicationFactor)
     {
         return new KeyspaceParams(false, ReplicationParams.simple(replicationFactor));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/schema/ReplicationParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/ReplicationParams.java b/src/java/org/apache/cassandra/schema/ReplicationParams.java
index 21c029e..21e19d6 100644
--- a/src/java/org/apache/cassandra/schema/ReplicationParams.java
+++ b/src/java/org/apache/cassandra/schema/ReplicationParams.java
@@ -51,6 +51,11 @@ public final class ReplicationParams
         return new ReplicationParams(SimpleStrategy.class, ImmutableMap.of("replication_factor", Integer.toString(replicationFactor)));
     }
 
+    static ReplicationParams simple(String replicationFactor)
+    {
+        return new ReplicationParams(SimpleStrategy.class, ImmutableMap.of("replication_factor", replicationFactor));
+    }
+
     static ReplicationParams nts(Object... args)
     {
         assert args.length % 2 == 0;
@@ -58,9 +63,7 @@ public final class ReplicationParams
         Map<String, String> options = new HashMap<>();
         for (int i = 0; i < args.length; i += 2)
         {
-            String dc = (String) args[i];
-            Integer rf = (Integer) args[i + 1];
-            options.put(dc, rf.toString());
+            options.put((String) args[i], args[i + 1].toString());
         }
 
         return new ReplicationParams(NetworkTopologyStrategy.class, options);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index a88aebb..22a8c39 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -137,6 +137,7 @@ public final class SchemaKeyspace
               + "min_index_interval int,"
               + "read_repair_chance double," // no longer used, left for drivers' sake
               + "speculative_retry text,"
+              + "speculative_write_threshold text,"
               + "cdc boolean,"
               + "read_repair text,"
               + "PRIMARY KEY ((keyspace_name), table_name))");
@@ -203,6 +204,7 @@ public final class SchemaKeyspace
               + "min_index_interval int,"
               + "read_repair_chance double," // no longer used, left for drivers' sake
               + "speculative_retry text,"
+              + "speculative_write_threshold text,"
               + "cdc boolean,"
               + "read_repair text,"
               + "PRIMARY KEY ((keyspace_name), view_name))");
@@ -563,6 +565,7 @@ public final class SchemaKeyspace
                .add("min_index_interval", params.minIndexInterval)
                .add("read_repair_chance", 0.0) // no longer used, left for drivers' sake
                .add("speculative_retry", params.speculativeRetry.toString())
+               .add("speculative_write_threshold", params.speculativeWriteThreshold.toString())
                .add("crc_check_chance", params.crcCheckChance)
                .add("caching", params.caching.asMap())
                .add("compaction", params.compaction.asMap())
@@ -991,6 +994,7 @@ public final class SchemaKeyspace
                           .minIndexInterval(row.getInt("min_index_interval"))
                           .crcCheckChance(row.getDouble("crc_check_chance"))
                           .speculativeRetry(SpeculativeRetryPolicy.fromString(row.getString("speculative_retry")))
+                          .speculativeWriteThreshold(SpeculativeRetryPolicy.fromString(row.getString("speculative_write_threshold")))
                           .cdc(row.has("cdc") && row.getBoolean("cdc"))
                           .readRepair(getReadRepairStrategy(row))
                           .build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/schema/TableMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java
index 6466e2e..10edf4d 100644
--- a/src/java/org/apache/cassandra/schema/TableMetadata.java
+++ b/src/java/org/apache/cassandra/schema/TableMetadata.java
@@ -834,6 +834,12 @@ public final class TableMetadata
             return this;
         }
 
+        public Builder speculativeWriteThreshold(SpeculativeRetryPolicy val)
+        {
+            params.speculativeWriteThreshold(val);
+            return this;
+        }
+
         public Builder extensions(Map<String, ByteBuffer> val)
         {
             params.extensions(val);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/schema/TableParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java
index afbf26c..0bba5e1 100644
--- a/src/java/org/apache/cassandra/schema/TableParams.java
+++ b/src/java/org/apache/cassandra/schema/TableParams.java
@@ -51,6 +51,7 @@ public final class TableParams
         MEMTABLE_FLUSH_PERIOD_IN_MS,
         MIN_INDEX_INTERVAL,
         SPECULATIVE_RETRY,
+        SPECULATIVE_WRITE_THRESHOLD,
         CRC_CHECK_CHANCE,
         CDC,
         READ_REPAIR;
@@ -71,6 +72,7 @@ public final class TableParams
     public final int minIndexInterval;
     public final int maxIndexInterval;
     public final SpeculativeRetryPolicy speculativeRetry;
+    public final SpeculativeRetryPolicy speculativeWriteThreshold;
     public final CachingParams caching;
     public final CompactionParams compaction;
     public final CompressionParams compression;
@@ -91,6 +93,7 @@ public final class TableParams
         minIndexInterval = builder.minIndexInterval;
         maxIndexInterval = builder.maxIndexInterval;
         speculativeRetry = builder.speculativeRetry;
+        speculativeWriteThreshold = builder.speculativeWriteThreshold;
         caching = builder.caching;
         compaction = builder.compaction;
         compression = builder.compression;
@@ -118,6 +121,7 @@ public final class TableParams
                             .memtableFlushPeriodInMs(params.memtableFlushPeriodInMs)
                             .minIndexInterval(params.minIndexInterval)
                             .speculativeRetry(params.speculativeRetry)
+                            .speculativeWriteThreshold(params.speculativeWriteThreshold)
                             .extensions(params.extensions)
                             .cdc(params.cdc)
                             .readRepair(params.readRepair);
@@ -260,6 +264,7 @@ public final class TableParams
         private int minIndexInterval = 128;
         private int maxIndexInterval = 2048;
         private SpeculativeRetryPolicy speculativeRetry = PercentileSpeculativeRetryPolicy.NINETY_NINE_P;
+        private SpeculativeRetryPolicy speculativeWriteThreshold = PercentileSpeculativeRetryPolicy.NINETY_NINE_P;
         private CachingParams caching = CachingParams.DEFAULT;
         private CompactionParams compaction = CompactionParams.DEFAULT;
         private CompressionParams compression = CompressionParams.DEFAULT;
@@ -330,6 +335,12 @@ public final class TableParams
             return this;
         }
 
+        public Builder speculativeWriteThreshold(SpeculativeRetryPolicy val)
+        {
+            speculativeWriteThreshold = val;
+            return this;
+        }
+
         public Builder caching(CachingParams val)
         {
             caching = val;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index 9d800a0..e817cc8 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -17,28 +17,35 @@
  */
 package org.apache.cassandra.service;
 
-import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.stream.Collectors;
 
-import com.google.common.collect.Iterables;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.locator.ReplicaLayout;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.WriteType;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.exceptions.RequestFailureReason;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.exceptions.WriteFailureException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
+
 public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackWithFailure<T>
 {
     protected static final Logger logger = LoggerFactory.getLogger(AbstractWriteResponseHandler.class);
@@ -46,11 +53,9 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     //Count down until all responses and expirations have occured before deciding whether the ideal CL was reached.
     private AtomicInteger responsesAndExpirations;
     private final SimpleCondition condition = new SimpleCondition();
-    protected final Keyspace keyspace;
-    protected final Collection<InetAddressAndPort> naturalEndpoints;
-    public final ConsistencyLevel consistencyLevel;
+    protected final ReplicaLayout.ForToken replicaLayout;
+
     protected final Runnable callback;
-    protected final Collection<InetAddressAndPort> pendingEndpoints;
     protected final WriteType writeType;
     private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater
     = AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures");
@@ -60,7 +65,7 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     private volatile boolean supportsBackPressure = true;
 
     /**
-      * Delegate to another WriteReponseHandler or possibly this one to track if the ideal consistency level was reached.
+      * Delegate to another WriteResponseHandler or possibly this one to track if the ideal consistency level was reached.
       * Will be set to null if ideal CL was not configured
       * Will be set to an AWRH delegate if ideal CL was configured
       * Will be same as "this" if this AWRH is the ideal consistency level
@@ -71,18 +76,12 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
      * @param callback           A callback to be called when the write is successful.
      * @param queryStartNanoTime
      */
-    protected AbstractWriteResponseHandler(Keyspace keyspace,
-                                           Collection<InetAddressAndPort> naturalEndpoints,
-                                           Collection<InetAddressAndPort> pendingEndpoints,
-                                           ConsistencyLevel consistencyLevel,
+    protected AbstractWriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                            Runnable callback,
                                            WriteType writeType,
                                            long queryStartNanoTime)
     {
-        this.keyspace = keyspace;
-        this.pendingEndpoints = pendingEndpoints;
-        this.consistencyLevel = consistencyLevel;
-        this.naturalEndpoints = naturalEndpoints;
+        this.replicaLayout = replicaLayout;
         this.callback = callback;
         this.writeType = writeType;
         this.failureReasonByEndpoint = new ConcurrentHashMap<>();
@@ -112,12 +111,12 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
             // avoid sending confusing info to the user (see CASSANDRA-6491).
             if (acks >= blockedFor)
                 acks = blockedFor - 1;
-            throw new WriteTimeoutException(writeType, consistencyLevel, acks, blockedFor);
+            throw new WriteTimeoutException(writeType, replicaLayout.consistencyLevel(), acks, blockedFor);
         }
 
         if (totalBlockFor() + failures > totalEndpoints())
         {
-            throw new WriteFailureException(consistencyLevel, ackCount(), totalBlockFor(), writeType, failureReasonByEndpoint);
+            throw new WriteFailureException(replicaLayout.consistencyLevel(), ackCount(), totalBlockFor(), writeType, failureReasonByEndpoint);
         }
     }
 
@@ -136,7 +135,7 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     public void setIdealCLResponseHandler(AbstractWriteResponseHandler handler)
     {
         this.idealCLDelegate = handler;
-        idealCLDelegate.responsesAndExpirations = new AtomicInteger(naturalEndpoints.size() + pendingEndpoints.size());
+        idealCLDelegate.responsesAndExpirations = new AtomicInteger(replicaLayout.selected().size());
     }
 
     /**
@@ -194,15 +193,20 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
     {
         // During bootstrap, we have to include the pending endpoints or we may fail the consistency level
         // guarantees (see #833)
-        return consistencyLevel.blockFor(keyspace) + pendingEndpoints.size();
+        return replicaLayout.consistencyLevel().blockForWrite(replicaLayout.keyspace(), replicaLayout.pending());
     }
 
     /**
-     * @return the total number of endpoints the request has been sent to.
+     * @return the total number of endpoints the request can been sent to.
      */
     protected int totalEndpoints()
     {
-        return naturalEndpoints.size() + pendingEndpoints.size();
+        return replicaLayout.all().size();
+    }
+
+    public ConsistencyLevel consistencyLevel()
+    {
+        return replicaLayout.consistencyLevel();
     }
 
     /**
@@ -225,7 +229,7 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
 
     public void assureSufficientLiveNodes() throws UnavailableException
     {
-        consistencyLevel.assureSufficientLiveNodes(keyspace, Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), isAlive));
+        replicaLayout.consistencyLevel().assureSufficientLiveNodesForWrite(replicaLayout.keyspace(), replicaLayout.all().filter(isReplicaAlive), replicaLayout.pending());
     }
 
     protected void signal()
@@ -274,12 +278,49 @@ public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackW
             //The condition being signaled is a valid proxy for the CL being achieved
             if (!condition.isSignaled())
             {
-                keyspace.metric.writeFailedIdealCL.inc();
+                replicaLayout.keyspace().metric.writeFailedIdealCL.inc();
             }
             else
             {
-                keyspace.metric.idealCLWriteLatency.addNano(System.nanoTime() - queryStartNanoTime);
+                replicaLayout.keyspace().metric.idealCLWriteLatency.addNano(System.nanoTime() - queryStartNanoTime);
             }
         }
     }
+
+    /**
+     * Cheap Quorum backup.  If we failed to reach quorum with our initial (full) nodes, reach out to other nodes.
+     */
+    public void maybeTryAdditionalReplicas(IMutation mutation, StorageProxy.WritePerformer writePerformer, String localDC)
+    {
+        if (replicaLayout.all().size() == replicaLayout.selected().size())
+            return;
+
+        long timeout = Long.MAX_VALUE;
+        List<ColumnFamilyStore> cfs = mutation.getTableIds().stream()
+                                              .map(Schema.instance::getColumnFamilyStoreInstance)
+                                              .collect(Collectors.toList());
+        for (ColumnFamilyStore cf : cfs)
+            timeout = Math.min(timeout, cf.transientWriteLatencyNanos);
+
+        // no latency information, or we're overloaded
+        if (timeout > TimeUnit.MILLISECONDS.toNanos(mutation.getTimeout()))
+            return;
+
+        try
+        {
+            if (!condition.await(timeout, TimeUnit.NANOSECONDS))
+            {
+                for (ColumnFamilyStore cf : cfs)
+                    cf.metric.speculativeWrites.inc();
+
+                writePerformer.apply(mutation, replicaLayout.forNaturalUncontacted(),
+                                     (AbstractWriteResponseHandler<IMutation>) this,
+                                     localDC);
+            }
+        }
+        catch (InterruptedException ex)
+        {
+            throw new AssertionError(ex);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index b60088c..9f37095 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -38,6 +38,8 @@ import com.google.common.util.concurrent.AbstractFuture;
 
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.cassandra.locator.EndpointsByRange;
+import org.apache.cassandra.locator.EndpointsForRange;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,11 +62,14 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replicas;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.repair.CommonRange;
+import org.apache.cassandra.repair.RepairRunnable;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.repair.RepairParallelism;
@@ -79,7 +84,8 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
 
-import static org.apache.cassandra.config.Config.RepairCommandPoolFullStrategy.queue;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.transform;
 
 /**
  * ActiveRepairService is the starting point for manual "active" repairs.
@@ -204,10 +210,9 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
      * @return Future for asynchronous call or null if there is no need to repair
      */
     public RepairSession submitRepairSession(UUID parentRepairSession,
-                                             Collection<Range<Token>> range,
+                                             CommonRange range,
                                              String keyspace,
                                              RepairParallelism parallelismDegree,
-                                             Set<InetAddressAndPort> endpoints,
                                              boolean isIncremental,
                                              boolean pullRepair,
                                              boolean force,
@@ -216,14 +221,15 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                                              ListeningExecutorService executor,
                                              String... cfnames)
     {
-        if (endpoints.isEmpty())
+        if (range.endpoints.isEmpty())
             return null;
 
         if (cfnames.length == 0)
             return null;
 
-
-        final RepairSession session = new RepairSession(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, isIncremental, pullRepair, force, previewKind, optimiseStreams, cfnames);
+        final RepairSession session = new RepairSession(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace,
+                                                        parallelismDegree, isIncremental, pullRepair, force,
+                                                        previewKind, optimiseStreams, cfnames);
 
         sessions.put(session.getId(), session);
         // register listeners
@@ -296,12 +302,12 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
      *
      * @return neighbors with whom we share the provided range
      */
-    public static Set<InetAddressAndPort> getNeighbors(String keyspaceName, Collection<Range<Token>> keyspaceLocalRanges,
-                                                       Range<Token> toRepair, Collection<String> dataCenters,
-                                                       Collection<String> hosts)
+    public static EndpointsForRange getNeighbors(String keyspaceName, Iterable<Range<Token>> keyspaceLocalRanges,
+                                          Range<Token> toRepair, Collection<String> dataCenters,
+                                          Collection<String> hosts)
     {
         StorageService ss = StorageService.instance;
-        Map<Range<Token>, List<InetAddressAndPort>> replicaSets = ss.getRangeToAddressMap(keyspaceName);
+        EndpointsByRange replicaSets = ss.getRangeToAddressMap(keyspaceName);
         Range<Token> rangeSuperSet = null;
         for (Range<Token> range : keyspaceLocalRanges)
         {
@@ -319,23 +325,16 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             }
         }
         if (rangeSuperSet == null || !replicaSets.containsKey(rangeSuperSet))
-            return Collections.emptySet();
+            return EndpointsForRange.empty(toRepair);
 
-        Set<InetAddressAndPort> neighbors = new HashSet<>(replicaSets.get(rangeSuperSet));
-        neighbors.remove(FBUtilities.getBroadcastAddressAndPort());
+        EndpointsForRange neighbors = replicaSets.get(rangeSuperSet).withoutSelf();
 
         if (dataCenters != null && !dataCenters.isEmpty())
         {
             TokenMetadata.Topology topology = ss.getTokenMetadata().cloneOnlyTokenMap().getTopology();
-            Set<InetAddressAndPort> dcEndpoints = Sets.newHashSet();
-            Multimap<String,InetAddressAndPort> dcEndpointsMap = topology.getDatacenterEndpoints();
-            for (String dc : dataCenters)
-            {
-                Collection<InetAddressAndPort> c = dcEndpointsMap.get(dc);
-                if (c != null)
-                   dcEndpoints.addAll(c);
-            }
-            return Sets.intersection(neighbors, dcEndpoints);
+            Multimap<String, InetAddressAndPort> dcEndpointsMap = topology.getDatacenterEndpoints();
+            Iterable<InetAddressAndPort> dcEndpoints = concat(transform(dataCenters, dcEndpointsMap::get));
+            return neighbors.keep(dcEndpoints);
         }
         else if (hosts != null && !hosts.isEmpty())
         {
@@ -345,7 +344,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                 try
                 {
                     final InetAddressAndPort endpoint = InetAddressAndPort.getByName(host.trim());
-                    if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) || neighbors.contains(endpoint))
+                    if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) || neighbors.endpoints().contains(endpoint))
                         specifiedHost.add(endpoint);
                 }
                 catch (UnknownHostException e)
@@ -366,8 +365,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             }
 
             specifiedHost.remove(FBUtilities.getBroadcastAddressAndPort());
-            return specifiedHost;
-
+            return neighbors.keep(specifiedHost);
         }
 
         return neighbors;
@@ -594,10 +592,10 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
 
         public Set<TableId> getTableIds()
         {
-            return ImmutableSet.copyOf(Iterables.transform(getColumnFamilyStores(), cfs -> cfs.metadata.id));
+            return ImmutableSet.copyOf(transform(getColumnFamilyStores(), cfs -> cfs.metadata.id));
         }
 
-        public Collection<Range<Token>> getRanges()
+        public Set<Range<Token>> getRanges()
         {
             return ImmutableSet.copyOf(ranges);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
index e373fb6..ee74df5 100644
--- a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java
@@ -36,7 +36,7 @@ public class BatchlogResponseHandler<T> extends AbstractWriteResponseHandler<T>
 
     public BatchlogResponseHandler(AbstractWriteResponseHandler<T> wrapped, int requiredBeforeFinish, BatchlogCleanup cleanup, long queryStartNanoTime)
     {
-        super(wrapped.keyspace, wrapped.naturalEndpoints, wrapped.pendingEndpoints, wrapped.consistencyLevel, wrapped.callback, wrapped.writeType, queryStartNanoTime);
+        super(wrapped.replicaLayout, wrapped.callback, wrapped.writeType, queryStartNanoTime);
         this.wrapped = wrapped;
         this.requiredBeforeFinish = requiredBeforeFinish;
         this.cleanup = cleanup;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index dbd3667..d4cdcc6 100644
--- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -17,16 +17,15 @@
  */
 package org.apache.cassandra.service;
 
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
@@ -41,29 +40,26 @@ public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponse
     private final Map<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>();
     private final AtomicInteger acks = new AtomicInteger(0);
 
-    public DatacenterSyncWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
-                                              Collection<InetAddressAndPort> pendingEndpoints,
-                                              ConsistencyLevel consistencyLevel,
-                                              Keyspace keyspace,
+    public DatacenterSyncWriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                               Runnable callback,
                                               WriteType writeType,
                                               long queryStartNanoTime)
     {
         // Response is been managed by the map so make it 1 for the superclass.
-        super(keyspace, naturalEndpoints, pendingEndpoints, consistencyLevel, callback, writeType, queryStartNanoTime);
-        assert consistencyLevel == ConsistencyLevel.EACH_QUORUM;
+        super(replicaLayout, callback, writeType, queryStartNanoTime);
+        assert replicaLayout.consistencyLevel() == ConsistencyLevel.EACH_QUORUM;
 
-        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
+        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) replicaLayout.keyspace().getReplicationStrategy();
 
         for (String dc : strategy.getDatacenters())
         {
-            int rf = strategy.getReplicationFactor(dc);
+            int rf = strategy.getReplicationFactor(dc).allReplicas;
             responses.put(dc, new AtomicInteger((rf / 2) + 1));
         }
 
         // During bootstrap, we have to include the pending endpoints or we may fail the consistency level
         // guarantees (see #833)
-        for (InetAddressAndPort pending : pendingEndpoints)
+        for (Replica pending : replicaLayout.pending())
         {
             responses.get(snitch.getDatacenter(pending)).incrementAndGet();
         }
@@ -105,4 +101,5 @@ public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponse
     {
         return false;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
index a8d7b28..b458a71 100644
--- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
@@ -17,29 +17,23 @@
  */
 package org.apache.cassandra.service;
 
-import java.util.Collection;
-
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.ReplicaLayout;
 import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.WriteType;
 
 /**
  * This class blocks for a quorum of responses _in the local datacenter only_ (CL.LOCAL_QUORUM).
  */
 public class DatacenterWriteResponseHandler<T> extends WriteResponseHandler<T>
 {
-    public DatacenterWriteResponseHandler(Collection<InetAddressAndPort> naturalEndpoints,
-                                          Collection<InetAddressAndPort> pendingEndpoints,
-                                          ConsistencyLevel consistencyLevel,
-                                          Keyspace keyspace,
+    public DatacenterWriteResponseHandler(ReplicaLayout.ForToken replicaLayout,
                                           Runnable callback,
                                           WriteType writeType,
                                           long queryStartNanoTime)
     {
-        super(naturalEndpoints, pendingEndpoints, consistencyLevel, keyspace, callback, writeType, queryStartNanoTime);
-        assert consistencyLevel.isDatacenterLocal();
+        super(replicaLayout, callback, writeType, queryStartNanoTime);
+        assert replicaLayout.consistencyLevel().isDatacenterLocal();
     }
 
     @Override
@@ -58,16 +52,8 @@ public class DatacenterWriteResponseHandler<T> extends WriteResponseHandler<T>
     }
 
     @Override
-    protected int totalBlockFor()
-    {
-        // during bootstrap, include pending endpoints (only local here) in the count
-        // or we may fail the consistency level guarantees (see #833, #8058)
-        return consistencyLevel.blockFor(keyspace) + consistencyLevel.countLocalEndpoints(pendingEndpoints);
-    }
-
-    @Override
     protected boolean waitingFor(InetAddressAndPort from)
     {
-        return consistencyLevel.isLocal(from);
+        return replicaLayout.consistencyLevel().isLocal(from);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
index e1c0f55..7b6bd58 100644
--- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
+++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
@@ -92,7 +92,7 @@ public class PendingRangeCalculatorService
     {
         int jobs = updateJobs.incrementAndGet();
         PendingRangeCalculatorServiceDiagnostics.taskCountChanged(instance, jobs);
-        executor.submit(new PendingRangeTask(updateJobs));
+        executor.execute(new PendingRangeTask(updateJobs));
     }
 
     public void blockUntilFinished()


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


[06/18] cassandra git commit: Transient Replication and Cheap Quorums

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index a096c78..46c0afd 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -107,7 +107,6 @@ public class CleanupTest
                                     SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEXED2, true));
     }
 
-    /*
     @Test
     public void testCleanup() throws ExecutionException, InterruptedException
     {
@@ -116,7 +115,6 @@ public class CleanupTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
-        UnfilteredPartitionIterator iter;
 
         // insert data and verify we get it back w/ range query
         fillCF(cfs, "val", LOOPS);
@@ -124,8 +122,7 @@ public class CleanupTest
         // record max timestamps of the sstables pre-cleanup
         List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
 
-        iter = Util.getRangeSlice(cfs);
-        assertEquals(LOOPS, Iterators.size(iter));
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
         CompactionManager.instance.performCleanup(cfs, 2);
@@ -134,10 +131,8 @@ public class CleanupTest
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
 
         // check data is still there
-        iter = Util.getRangeSlice(cfs);
-        assertEquals(LOOPS, Iterators.size(iter));
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
     }
-    */
 
     @Test
     public void testCleanupWithIndexes() throws IOException, ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/CleanupTransientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTransientTest.java b/test/unit/org/apache/cassandra/db/CleanupTransientTest.java
new file mode 100644
index 0000000..9789183
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/CleanupTransientTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.db;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.AbstractNetworkTopologySnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.PendingRangeCalculatorService;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class CleanupTransientTest
+{
+    private static final IPartitioner partitioner = RandomPartitioner.instance;
+    private static IPartitioner oldPartitioner;
+
+    public static final int LOOPS = 200;
+    public static final String KEYSPACE1 = "CleanupTest1";
+    public static final String CF_INDEXED1 = "Indexed1";
+    public static final String CF_STANDARD1 = "Standard1";
+
+    public static final String KEYSPACE2 = "CleanupTestMultiDc";
+    public static final String CF_INDEXED2 = "Indexed2";
+    public static final String CF_STANDARD2 = "Standard2";
+
+    public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate");
+    public static final ByteBuffer VALUE = ByteBuffer.allocate(8);
+    static
+    {
+        VALUE.putLong(20101229);
+        VALUE.flip();
+    }
+
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true);
+        oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple("2/1"),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEXED1, true));
+
+        StorageService ss = StorageService.instance;
+        final int RING_SIZE = 2;
+
+        TokenMetadata tmd = ss.getTokenMetadata();
+        tmd.clearUnsafe();
+        ArrayList<Token> endpointTokens = new ArrayList<>();
+        ArrayList<Token> keyTokens = new ArrayList<>();
+        List<InetAddressAndPort> hosts = new ArrayList<>();
+        List<UUID> hostIds = new ArrayList<>();
+
+        endpointTokens.add(RandomPartitioner.MINIMUM);
+        endpointTokens.add(RandomPartitioner.instance.midpoint(RandomPartitioner.MINIMUM, new RandomPartitioner.BigIntegerToken(RandomPartitioner.MAXIMUM)));
+
+        Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
+        PendingRangeCalculatorService.instance.blockUntilFinished();
+
+
+        DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch()
+        {
+            @Override
+            public String getRack(InetAddressAndPort endpoint)
+            {
+                return "RC1";
+            }
+
+            @Override
+            public String getDatacenter(InetAddressAndPort endpoint)
+            {
+                return "DC1";
+            }
+        });
+    }
+
+    @Test
+    public void testCleanup() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
+
+
+        // insert data and verify we get it back w/ range query
+        fillCF(cfs, "val", LOOPS);
+
+        // record max timestamps of the sstables pre-cleanup
+        List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
+
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
+
+        // with two tokens RF=2/1 and the sstable not repaired this should do nothing
+        CompactionManager.instance.performCleanup(cfs, 2);
+
+        // ensure max timestamp of the sstables are retained post-cleanup
+        assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
+
+        // check data is still there
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
+
+        //Get an exact count of how many partitions are in the fully replicated range and should
+        //be retained
+        int fullCount = 0;
+        RangesAtEndpoint localRanges = StorageService.instance.getLocalReplicas(keyspace.getName()).filter(Replica::isFull);
+        for (FilteredPartition partition : Util.getAll(Util.cmd(cfs).build()))
+        {
+            Token token = partition.partitionKey().getToken();
+            for (Replica r : localRanges)
+            {
+                if (r.range().contains(token))
+                    fullCount++;
+            }
+        }
+
+        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+        sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, 1, null, false);
+        sstable.reloadSSTableMetadata();
+
+        // This should remove approximately 50% of the data, specifically whatever was transiently replicated
+        CompactionManager.instance.performCleanup(cfs, 2);
+
+        // ensure max timestamp of the sstables are retained post-cleanup
+        assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
+
+        // check less data is there, all transient data should be gone since the table was repaired
+        assertEquals(fullCount, Util.getAll(Util.cmd(cfs).build()).size());
+    }
+
+    protected void fillCF(ColumnFamilyStore cfs, String colName, int rowsPerSSTable)
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        for (int i = 0; i < rowsPerSSTable; i++)
+        {
+            String key = String.valueOf(i);
+            // create a row and update the birthdate value, test that the index query fetches the new version
+            new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), ByteBufferUtil.bytes(key))
+                    .clustering(COLUMN)
+                    .add(colName, VALUE)
+                    .build()
+                    .applyUnsafe();
+        }
+
+        cfs.forceBlockingFlush();
+    }
+
+    protected List<Long> getMaxTimestampList(ColumnFamilyStore cfs)
+    {
+        List<Long> list = new LinkedList<Long>();
+        for (SSTableReader sstable : cfs.getLiveSSTables())
+            list.add(sstable.getMaxTimestamp());
+        return list;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/ImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ImportTest.java b/test/unit/org/apache/cassandra/db/ImportTest.java
index 66bbff3..5ceb233 100644
--- a/test/unit/org/apache/cassandra/db/ImportTest.java
+++ b/test/unit/org/apache/cassandra/db/ImportTest.java
@@ -174,7 +174,7 @@ public class ImportTest extends CQLTester
         Set<SSTableReader> sstables = getCurrentColumnFamilyStore().getLiveSSTables();
         getCurrentColumnFamilyStore().clearUnsafe();
         for (SSTableReader sstable : sstables)
-            sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, 111, null);
+            sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, 111, null, false);
 
         File backupdir = moveToBackupDir(sstables);
         assertEquals(0, execute("select * from %s").size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java b/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
index e01088d..a864786 100644
--- a/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
+++ b/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
@@ -308,7 +308,7 @@ public class RepairedDataTombstonesTest extends CQLTester
 
     public static void repair(ColumnFamilyStore cfs, SSTableReader sstable) throws IOException
     {
-        sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, 1, null);
+        sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, 1, null, false);
         sstable.reloadSSTableMetadata();
         cfs.getTracker().notifySSTableRepairedStatusChanged(Collections.singleton(sstable));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
index 706b274..1ac5440 100644
--- a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
+++ b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
@@ -70,6 +70,12 @@ public class RowUpdateBuilder
         this.updateBuilder.nowInSec(localDeletionTime);
     }
 
+    public RowUpdateBuilder timestamp(long ts)
+    {
+        updateBuilder.timestamp(ts);
+        return this;
+    }
+
     private Row.SimpleBuilder rowBuilder()
     {
         // Normally, rowBuilder is created by the call to clustering(), but we allow skipping that call for an empty

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 36af54f..b58909b 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -615,7 +615,7 @@ public class ScrubTest
     {
         SerializationHeader header = new SerializationHeader(true, metadata.get(), metadata.get().regularAndStaticColumns(), EncodingStats.NO_STATS);
         MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(0);
-        return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, null, metadata, collector, header, txn), txn);
+        return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, null, false, metadata, collector, header, txn), txn);
     }
 
     private static class TestMultiWriter extends SimpleSSTableMultiWriter
@@ -631,10 +631,10 @@ public class ScrubTest
      */
     private static class TestWriter extends BigTableWriter
     {
-        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, TableMetadataRef metadata,
+        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, TableMetadataRef metadata,
                    MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
         {
-            super(descriptor, keyCount, repairedAt, pendingRepair, metadata, collector, header, Collections.emptySet(), txn);
+            super(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, Collections.emptySet(), txn);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
index 1c051f5..a14db00 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceMigrator40Test.java
@@ -34,6 +34,8 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.UUIDGen;
 
 import static org.junit.Assert.assertEquals;
@@ -189,4 +191,28 @@ public class SystemKeyspaceMigrator40Test extends CQLTester
         }
         assertEquals(1, rowCount);
     }
+
+    @Test
+    public void testMigrateAvailableRanges() throws Throwable
+    {
+        Range<Token> testRange = new Range<>(DatabaseDescriptor.getPartitioner().getRandomToken(), DatabaseDescriptor.getPartitioner().getRandomToken());
+        String insert = String.format("INSERT INTO %s ("
+                                      + "keyspace_name, "
+                                      + "ranges) "
+                                      + " values ( ?, ? )",
+                                      SystemKeyspaceMigrator40.legacyAvailableRangesName);
+        execute(insert,
+                "foo",
+                ImmutableSet.of(SystemKeyspace.rangeToBytes(testRange)));
+        SystemKeyspaceMigrator40.migrate();
+
+        int rowCount = 0;
+        for (UntypedResultSet.Row row : execute(String.format("SELECT * FROM %s", SystemKeyspaceMigrator40.availableRangesName)))
+        {
+            rowCount++;
+            assertEquals("foo", row.getString("keyspace_name"));
+            assertEquals(ImmutableSet.of(testRange), SystemKeyspace.rawRangesToRangeSet(row.getSet("full_ranges", BytesType.instance), DatabaseDescriptor.getPartitioner()));
+        }
+        assertEquals(1, rowCount);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
index 32fa4e4..fff567b 100644
--- a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
+++ b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
@@ -26,6 +26,7 @@ import java.util.*;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.io.Files;
+import org.apache.cassandra.service.reads.NeverSpeculativeRetryPolicy;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -255,6 +256,7 @@ public class TableCQLHelperTest extends CQLTester
                .maxIndexInterval(7)
                .memtableFlushPeriod(8)
                .speculativeRetry(AlwaysSpeculativeRetryPolicy.INSTANCE)
+               .speculativeWriteThreshold(NeverSpeculativeRetryPolicy.INSTANCE)
                .extensions(ImmutableMap.of("ext1", ByteBuffer.wrap("val1".getBytes())))
                .recordColumnDrop(ColumnMetadata.regularColumn(keyspace, table, "reg1", AsciiType.instance),
                                  FBUtilities.timestampMicros());
@@ -272,6 +274,7 @@ public class TableCQLHelperTest extends CQLTester
         "\tAND max_index_interval = 7\n" +
         "\tAND memtable_flush_period_in_ms = 8\n" +
         "\tAND speculative_retry = 'ALWAYS'\n" +
+        "\tAND speculative_write_threshold = 'NEVER'\n" +
         "\tAND comment = 'comment'\n" +
         "\tAND caching = { 'keys': 'ALL', 'rows_per_partition': 'NONE' }\n" +
         "\tAND compaction = { 'max_threshold': '32', 'min_threshold': '4', 'sstable_size_in_mb': '1', 'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' }\n" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/VerifyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
index 0632274..df2acb4 100644
--- a/test/unit/org/apache/cassandra/db/VerifyTest.java
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -421,7 +421,7 @@ public class VerifyTest
 
         // make the sstable repaired:
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
-        sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, System.currentTimeMillis(), sstable.getSSTableMetadata().pendingRepair);
+        sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, System.currentTimeMillis(), sstable.getPendingRepair(), sstable.isTransient());
         sstable.reloadSSTableMetadata();
 
         // break the sstable:
@@ -487,7 +487,7 @@ public class VerifyTest
         fillCF(cfs, 2);
 
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
-        sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, 1, sstable.getSSTableMetadata().pendingRepair);
+        sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, 1, sstable.getPendingRepair(), sstable.isTransient());
         sstable.reloadSSTableMetadata();
         cfs.getTracker().notifySSTableRepairedStatusChanged(Collections.singleton(sstable));
         assertTrue(sstable.isRepaired());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
index a320248..4d62894 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
@@ -109,17 +109,16 @@ public class AbstractPendingRepairTest extends AbstractRepairTest
         SSTableReader sstable = diff.iterator().next();
         if (orphan)
         {
-            Iterables.any(csm.getUnrepaired(), s -> s.getSSTables().contains(sstable));
-            csm.getUnrepaired().forEach(s -> s.removeSSTable(sstable));
+            csm.getUnrepairedUnsafe().allStrategies().forEach(acs -> acs.removeSSTable(sstable));
         }
         return sstable;
     }
 
-    protected static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair)
+    protected static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair, boolean isTransient)
     {
         try
         {
-            sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, repairedAt, pendingRepair);
+            sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
             sstable.reloadSSTableMetadata();
         }
         catch (IOException e)
@@ -130,11 +129,11 @@ public class AbstractPendingRepairTest extends AbstractRepairTest
 
     protected static void mutateRepaired(SSTableReader sstable, long repairedAt)
     {
-        mutateRepaired(sstable, repairedAt, ActiveRepairService.NO_PENDING_REPAIR);
+        mutateRepaired(sstable, repairedAt, ActiveRepairService.NO_PENDING_REPAIR, false);
     }
 
-    protected static void mutateRepaired(SSTableReader sstable, UUID pendingRepair)
+    protected static void mutateRepaired(SSTableReader sstable, UUID pendingRepair, boolean isTransient)
     {
-        mutateRepaired(sstable, ActiveRepairService.UNREPAIRED_SSTABLE, pendingRepair);
+        mutateRepaired(sstable, ActiveRepairService.UNREPAIRED_SSTABLE, pendingRepair, isTransient);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 366c18e..f514ea6 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
+import java.util.function.Predicate;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -39,6 +40,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.RangesAtEndpoint;
+import org.apache.cassandra.locator.Replica;
 import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
@@ -50,7 +53,6 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -58,6 +60,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.Refs;
 import org.apache.cassandra.UpdateBuilder;
@@ -77,16 +80,21 @@ public class AntiCompactionTest
 {
     private static final String KEYSPACE1 = "AntiCompactionTest";
     private static final String CF = "AntiCompactionTest";
+    private static final Collection<Range<Token>> NO_RANGES = Collections.emptyList();
+
     private static TableMetadata metadata;
     private static ColumnFamilyStore cfs;
+    private static InetAddressAndPort local;
+
 
     @BeforeClass
-    public static void defineSchema() throws ConfigurationException
+    public static void defineSchema() throws Throwable
     {
         SchemaLoader.prepareServer();
         metadata = SchemaLoader.standardCFMD(KEYSPACE1, CF).build();
         SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), metadata);
         cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.id);
+        local = InetAddressAndPort.getByName("127.0.0.1");
     }
 
     @After
@@ -97,56 +105,86 @@ public class AntiCompactionTest
         store.truncateBlocking();
     }
 
-    private void registerParentRepairSession(UUID sessionID, Collection<Range<Token>> ranges, long repairedAt, UUID pendingRepair) throws IOException
+    private void registerParentRepairSession(UUID sessionID, Iterable<Range<Token>> ranges, long repairedAt, UUID pendingRepair) throws IOException
     {
         ActiveRepairService.instance.registerParentRepairSession(sessionID,
                                                                  InetAddressAndPort.getByName("10.0.0.1"),
-                                                                 Lists.newArrayList(cfs), ranges,
+                                                                 Lists.newArrayList(cfs), ImmutableSet.copyOf(ranges),
                                                                  pendingRepair != null || repairedAt != UNREPAIRED_SSTABLE,
                                                                  repairedAt, true, PreviewKind.NONE);
     }
 
-    private void antiCompactOne(long repairedAt, UUID pendingRepair) throws Exception
+    private static RangesAtEndpoint atEndpoint(Collection<Range<Token>> full, Collection<Range<Token>> trans)
     {
-        assert repairedAt != UNREPAIRED_SSTABLE || pendingRepair != null;
+        RangesAtEndpoint.Builder builder = RangesAtEndpoint.builder(local);
+        for (Range<Token> range : full)
+            builder.add(new Replica(local, range, true));
 
-        ColumnFamilyStore store = prepareColumnFamilyStore();
-        Collection<SSTableReader> sstables = getUnrepairedSSTables(store);
-        assertEquals(store.getLiveSSTables().size(), sstables.size());
-        Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
-        List<Range<Token>> ranges = Arrays.asList(range);
+        for (Range<Token> range : trans)
+            builder.add(new Replica(local, range, false));
+
+        return builder.build();
+    }
+
+    private static Collection<Range<Token>> range(int l, int r)
+    {
+        return Collections.singleton(new Range<>(new BytesToken(Integer.toString(l).getBytes()), new BytesToken(Integer.toString(r).getBytes())));
+    }
 
-        int repairedKeys = 0;
+    private static class SSTableStats
+    {
+        int numLiveSSTables = 0;
         int pendingKeys = 0;
-        int nonRepairedKeys = 0;
+        int transKeys = 0;
+        int unrepairedKeys = 0;
+    }
+
+    private SSTableStats antiCompactRanges(ColumnFamilyStore store, RangesAtEndpoint ranges) throws IOException
+    {
+        UUID sessionID = UUID.randomUUID();
+        Collection<SSTableReader> sstables = getUnrepairedSSTables(store);
         try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
              Refs<SSTableReader> refs = Refs.ref(sstables))
         {
             if (txn == null)
                 throw new IllegalStateException();
-            UUID parentRepairSession = pendingRepair == null ? UUID.randomUUID() : pendingRepair;
-            registerParentRepairSession(parentRepairSession, ranges, repairedAt, pendingRepair);
-            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt, pendingRepair, parentRepairSession);
+            registerParentRepairSession(sessionID, ranges.ranges(), FBUtilities.nowInSeconds(), sessionID);
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, sessionID);
         }
 
-        assertEquals(2, store.getLiveSSTables().size());
+        SSTableStats stats = new SSTableStats();
+        stats.numLiveSSTables = store.getLiveSSTables().size();
+
+        Predicate<Token> fullContains = t -> Iterables.any(ranges.fullRanges(), r -> r.contains(t));
+        Predicate<Token> transContains = t -> Iterables.any(ranges.transientRanges(), r -> r.contains(t));
         for (SSTableReader sstable : store.getLiveSSTables())
         {
+            assertFalse(sstable.isRepaired());
+            assertEquals(sstable.isPendingRepair() ? sessionID : NO_PENDING_REPAIR, sstable.getPendingRepair());
             try (ISSTableScanner scanner = sstable.getScanner())
             {
                 while (scanner.hasNext())
                 {
                     UnfilteredRowIterator row = scanner.next();
-                    if (sstable.isRepaired() || sstable.isPendingRepair())
+                    Token token = row.partitionKey().getToken();
+                    if (sstable.isPendingRepair() && !sstable.isTransient())
                     {
-                        assertTrue(range.contains(row.partitionKey().getToken()));
-                        repairedKeys += sstable.isRepaired() ? 1 : 0;
-                        pendingKeys += sstable.isPendingRepair() ? 1 : 0;
+                        assertTrue(fullContains.test(token));
+                        assertFalse(transContains.test(token));
+                        stats.pendingKeys++;
+                    }
+                    else if (sstable.isPendingRepair() && sstable.isTransient())
+                    {
+
+                        assertTrue(transContains.test(token));
+                        assertFalse(fullContains.test(token));
+                        stats.transKeys++;
                     }
                     else
                     {
-                        assertFalse(range.contains(row.partitionKey().getToken()));
-                        nonRepairedKeys++;
+                        assertFalse(fullContains.test(token));
+                        assertFalse(transContains.test(token));
+                        stats.unrepairedKeys++;
                     }
                 }
             }
@@ -157,21 +195,40 @@ public class AntiCompactionTest
             assertEquals(1, sstable.selfRef().globalCount());
         }
         assertEquals(0, store.getTracker().getCompacting().size());
-        assertEquals(repairedKeys, repairedAt != UNREPAIRED_SSTABLE ? 4 : 0);
-        assertEquals(pendingKeys, pendingRepair != NO_PENDING_REPAIR ? 4 : 0);
-        assertEquals(nonRepairedKeys, 6);
+        return stats;
     }
 
     @Test
-    public void antiCompactOneRepairedAt() throws Exception
+    public void antiCompactOneFull() throws Exception
     {
-        antiCompactOne(1000, NO_PENDING_REPAIR);
+        ColumnFamilyStore store = prepareColumnFamilyStore();
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(range(0, 4), NO_RANGES));
+        assertEquals(2, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 4);
+        assertEquals(stats.transKeys, 0);
+        assertEquals(stats.unrepairedKeys, 6);
+    }
+
+    @Test
+    public void antiCompactOneMixed() throws Exception
+    {
+        ColumnFamilyStore store = prepareColumnFamilyStore();
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(range(0, 4), range(4, 8)));
+        assertEquals(3, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 4);
+        assertEquals(stats.transKeys, 4);
+        assertEquals(stats.unrepairedKeys, 2);
     }
 
     @Test
-    public void antiCompactOnePendingRepair() throws Exception
+    public void antiCompactOneTransOnly() throws Exception
     {
-        antiCompactOne(UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
+        ColumnFamilyStore store = prepareColumnFamilyStore();
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(NO_RANGES, range(0, 4)));
+        assertEquals(2, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 0);
+        assertEquals(stats.transKeys, 4);
+        assertEquals(stats.unrepairedKeys, 6);
     }
 
     @Test
@@ -190,7 +247,7 @@ public class AntiCompactionTest
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
              Refs<SSTableReader> refs = Refs.ref(sstables))
         {
-            CompactionManager.instance.performAnticompaction(cfs, ranges, refs, txn, 12345, NO_PENDING_REPAIR, parentRepairSession);
+            CompactionManager.instance.performAnticompaction(cfs, atEndpoint(ranges, NO_RANGES), refs, txn, parentRepairSession);
         }
         long sum = 0;
         long rows = 0;
@@ -208,7 +265,7 @@ public class AntiCompactionTest
         File dir = cfs.getDirectories().getDirectoryForNewSSTables();
         Descriptor desc = cfs.newSSTableDescriptor(dir);
 
-        try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, NO_PENDING_REPAIR, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
+        try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, NO_PENDING_REPAIR, false, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
         {
             for (int i = 0; i < count; i++)
             {
@@ -240,7 +297,7 @@ public class AntiCompactionTest
     }
 
     @Test
-    public void antiCompactTen() throws InterruptedException, IOException
+    public void antiCompactTenFull() throws InterruptedException, IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
@@ -250,56 +307,59 @@ public class AntiCompactionTest
         {
             generateSStable(store,Integer.toString(table));
         }
-        Collection<SSTableReader> sstables = getUnrepairedSSTables(store);
-        assertEquals(store.getLiveSSTables().size(), sstables.size());
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(range(0, 4), NO_RANGES));
+        /*
+        Anticompaction will be anti-compacting 10 SSTables but will be doing this two at a time
+        so there will be no net change in the number of sstables
+         */
+        assertEquals(10, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 40);
+        assertEquals(stats.transKeys, 0);
+        assertEquals(stats.unrepairedKeys, 60);
+    }
 
-        Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
-        List<Range<Token>> ranges = Arrays.asList(range);
+    @Test
+    public void antiCompactTenTrans() throws InterruptedException, IOException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.disableAutoCompaction();
 
-        long repairedAt = 1000;
-        UUID parentRepairSession = UUID.randomUUID();
-        registerParentRepairSession(parentRepairSession, ranges, repairedAt, null);
-        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
-             Refs<SSTableReader> refs = Refs.ref(sstables))
+        for (int table = 0; table < 10; table++)
         {
-            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt, NO_PENDING_REPAIR, parentRepairSession);
+            generateSStable(store,Integer.toString(table));
         }
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(NO_RANGES, range(0, 4)));
         /*
         Anticompaction will be anti-compacting 10 SSTables but will be doing this two at a time
         so there will be no net change in the number of sstables
          */
-        assertEquals(10, store.getLiveSSTables().size());
-        int repairedKeys = 0;
-        int nonRepairedKeys = 0;
-        for (SSTableReader sstable : store.getLiveSSTables())
+        assertEquals(10, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 0);
+        assertEquals(stats.transKeys, 40);
+        assertEquals(stats.unrepairedKeys, 60);
+    }
+
+    @Test
+    public void antiCompactTenMixed() throws InterruptedException, IOException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.disableAutoCompaction();
+
+        for (int table = 0; table < 10; table++)
         {
-            try (ISSTableScanner scanner = sstable.getScanner())
-            {
-                while (scanner.hasNext())
-                {
-                    try (UnfilteredRowIterator row = scanner.next())
-                    {
-                        if (sstable.isRepaired())
-                        {
-                            assertTrue(range.contains(row.partitionKey().getToken()));
-                            assertEquals(repairedAt, sstable.getSSTableMetadata().repairedAt);
-                            repairedKeys++;
-                        }
-                        else
-                        {
-                            assertFalse(range.contains(row.partitionKey().getToken()));
-                            assertEquals(ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getSSTableMetadata().repairedAt);
-                            nonRepairedKeys++;
-                        }
-                    }
-                }
-            }
+            generateSStable(store,Integer.toString(table));
         }
-        assertEquals(repairedKeys, 40);
-        assertEquals(nonRepairedKeys, 60);
+        SSTableStats stats = antiCompactRanges(store, atEndpoint(range(0, 4), range(4, 8)));
+        assertEquals(15, stats.numLiveSSTables);
+        assertEquals(stats.pendingKeys, 40);
+        assertEquals(stats.transKeys, 40);
+        assertEquals(stats.unrepairedKeys, 20);
     }
 
-    private void shouldMutate(long repairedAt, UUID pendingRepair) throws InterruptedException, IOException
+    @Test
+    public void shouldMutatePendingRepair() throws InterruptedException, IOException
     {
         ColumnFamilyStore store = prepareColumnFamilyStore();
         Collection<SSTableReader> sstables = getUnrepairedSSTables(store);
@@ -307,35 +367,23 @@ public class AntiCompactionTest
         // the sstables start at "0".getBytes() = 48, we need to include that first token, with "/".getBytes() = 47
         Range<Token> range = new Range<Token>(new BytesToken("/".getBytes()), new BytesToken("9999".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
-        UUID parentRepairSession = pendingRepair == null ? UUID.randomUUID() : pendingRepair;
-        registerParentRepairSession(parentRepairSession, ranges, repairedAt, pendingRepair);
+        UUID pendingRepair = UUID.randomUUID();
+        registerParentRepairSession(pendingRepair, ranges, UNREPAIRED_SSTABLE, pendingRepair);
 
         try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
              Refs<SSTableReader> refs = Refs.ref(sstables))
         {
-            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt, pendingRepair, parentRepairSession);
+            CompactionManager.instance.performAnticompaction(store, atEndpoint(ranges, NO_RANGES), refs, txn, pendingRepair);
         }
 
         assertThat(store.getLiveSSTables().size(), is(1));
-        assertThat(Iterables.get(store.getLiveSSTables(), 0).isRepaired(), is(repairedAt != UNREPAIRED_SSTABLE));
-        assertThat(Iterables.get(store.getLiveSSTables(), 0).isPendingRepair(), is(pendingRepair != NO_PENDING_REPAIR));
+        assertThat(Iterables.get(store.getLiveSSTables(), 0).isRepaired(), is(false));
+        assertThat(Iterables.get(store.getLiveSSTables(), 0).isPendingRepair(), is(true));
         assertThat(Iterables.get(store.getLiveSSTables(), 0).selfRef().globalCount(), is(1));
         assertThat(store.getTracker().getCompacting().size(), is(0));
     }
 
     @Test
-    public void shouldMutateRepairedAt() throws InterruptedException, IOException
-    {
-        shouldMutate(1, NO_PENDING_REPAIR);
-    }
-
-    @Test
-    public void shouldMutatePendingRepair() throws InterruptedException, IOException
-    {
-        shouldMutate(UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
-    }
-
-    @Test
     public void shouldSkipAntiCompactionForNonIntersectingRange() throws InterruptedException, IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
@@ -358,7 +406,7 @@ public class AntiCompactionTest
         try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
              Refs<SSTableReader> refs = Refs.ref(sstables))
         {
-            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1, NO_PENDING_REPAIR, parentRepairSession);
+            CompactionManager.instance.performAnticompaction(store, atEndpoint(ranges, NO_RANGES), refs, txn, parentRepairSession);
         }
         catch (IllegalStateException e)
         {
@@ -428,7 +476,7 @@ public class AntiCompactionTest
             Assert.assertFalse(refs.isEmpty());
             try
             {
-                CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1, missingRepairSession, missingRepairSession);
+                CompactionManager.instance.performAnticompaction(store, atEndpoint(ranges, NO_RANGES), refs, txn, missingRepairSession);
                 Assert.fail("expected RuntimeException");
             }
             catch (RuntimeException e)
@@ -484,8 +532,7 @@ public class AntiCompactionTest
 
         Range<Token> r = new Range<>(t(9), t(100)); // sstable is not intersecting and should not be included
 
-        Iterator<SSTableReader> sstableIterator = sstables.iterator();
-        CompactionManager.findSSTablesToAnticompact(sstableIterator, Collections.singletonList(r), UUID.randomUUID());
+        CompactionManager.validateSSTableBoundsForAnticompaction(UUID.randomUUID(), sstables, atEndpoint(Collections.singletonList(r), NO_RANGES));
     }
 
     @Test(expected = IllegalStateException.class)
@@ -500,8 +547,7 @@ public class AntiCompactionTest
 
         Range<Token> r = new Range<>(t(10), t(11)); // no sstable included, throw
 
-        Iterator<SSTableReader> sstableIterator = sstables.iterator();
-        CompactionManager.findSSTablesToAnticompact(sstableIterator, Collections.singletonList(r), UUID.randomUUID());
+        CompactionManager.validateSSTableBoundsForAnticompaction(UUID.randomUUID(), sstables, atEndpoint(Collections.singletonList(r), NO_RANGES));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java
index c7f1ae8..267c2e4 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.db.compaction;
 
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
@@ -42,29 +41,34 @@ import org.apache.cassandra.utils.FBUtilities;
 public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingRepairTest
 {
 
-    private static boolean strategiesContain(Collection<AbstractCompactionStrategy> strategies, SSTableReader sstable)
+    private boolean transientContains(SSTableReader sstable)
     {
-        return Iterables.any(strategies, strategy -> strategy.getSSTables().contains(sstable));
-    }
-
-    private boolean pendingContains(UUID id, SSTableReader sstable)
-    {
-        return Iterables.any(csm.getPendingRepairManagers(), p -> p.get(id) != null && p.get(id).getSSTables().contains(sstable));
+        return csm.getTransientRepairsUnsafe().containsSSTable(sstable);
     }
 
     private boolean pendingContains(SSTableReader sstable)
     {
-        return Iterables.any(csm.getPendingRepairManagers(), p -> strategiesContain(p.getStrategies(), sstable));
+        return csm.getPendingRepairsUnsafe().containsSSTable(sstable);
     }
 
     private boolean repairedContains(SSTableReader sstable)
     {
-        return strategiesContain(csm.getRepaired(), sstable);
+        return csm.getRepairedUnsafe().containsSSTable(sstable);
     }
 
     private boolean unrepairedContains(SSTableReader sstable)
     {
-        return strategiesContain(csm.getUnrepaired(), sstable);
+        return csm.getUnrepairedUnsafe().containsSSTable(sstable);
+    }
+
+    private boolean hasPendingStrategiesFor(UUID sessionID)
+    {
+        return !Iterables.isEmpty(csm.getPendingRepairsUnsafe().getStrategiesFor(sessionID));
+    }
+
+    private boolean hasTransientStrategiesFor(UUID sessionID)
+    {
+        return !Iterables.isEmpty(csm.getTransientRepairsUnsafe().getStrategiesFor(sessionID));
     }
 
     /**
@@ -75,23 +79,25 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
     {
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
-        Assert.assertTrue(csm.pendingRepairs().isEmpty());
+        Assert.assertTrue(Iterables.isEmpty(csm.getPendingRepairsUnsafe().allStrategies()));
 
         SSTableReader sstable = makeSSTable(true);
         Assert.assertFalse(sstable.isRepaired());
         Assert.assertFalse(sstable.isPendingRepair());
 
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         Assert.assertFalse(sstable.isRepaired());
         Assert.assertTrue(sstable.isPendingRepair());
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNull);
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         // add the sstable
         csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
         Assert.assertFalse(repairedContains(sstable));
         Assert.assertFalse(unrepairedContains(sstable));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNotNull);
-        Assert.assertTrue(pendingContains(repairID, sstable));
+        Assert.assertTrue(pendingContains(sstable));
+        Assert.assertTrue(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
     }
 
     @Test
@@ -101,16 +107,17 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
 
         SSTableReader sstable1 = makeSSTable(true);
-        mutateRepaired(sstable1, repairID);
+        mutateRepaired(sstable1, repairID, false);
 
         SSTableReader sstable2 = makeSSTable(true);
-        mutateRepaired(sstable2, repairID);
+        mutateRepaired(sstable2, repairID, false);
 
         Assert.assertFalse(repairedContains(sstable1));
         Assert.assertFalse(unrepairedContains(sstable1));
         Assert.assertFalse(repairedContains(sstable2));
         Assert.assertFalse(unrepairedContains(sstable2));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNull);
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         // add only
         SSTableListChangedNotification notification;
@@ -119,13 +126,14 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
                                                           OperationType.COMPACTION);
         csm.handleNotification(notification, cfs.getTracker());
 
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNotNull);
         Assert.assertFalse(repairedContains(sstable1));
         Assert.assertFalse(unrepairedContains(sstable1));
-        Assert.assertTrue(pendingContains(repairID, sstable1));
+        Assert.assertTrue(pendingContains(sstable1));
         Assert.assertFalse(repairedContains(sstable2));
         Assert.assertFalse(unrepairedContains(sstable2));
-        Assert.assertFalse(pendingContains(repairID, sstable2));
+        Assert.assertFalse(pendingContains(sstable2));
+        Assert.assertTrue(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         // remove and add
         notification = new SSTableListChangedNotification(Collections.singleton(sstable2),
@@ -135,10 +143,10 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
 
         Assert.assertFalse(repairedContains(sstable1));
         Assert.assertFalse(unrepairedContains(sstable1));
-        Assert.assertFalse(pendingContains(repairID, sstable1));
+        Assert.assertFalse(pendingContains(sstable1));
         Assert.assertFalse(repairedContains(sstable2));
         Assert.assertFalse(unrepairedContains(sstable2));
-        Assert.assertTrue(pendingContains(repairID, sstable2));
+        Assert.assertTrue(pendingContains(sstable2));
     }
 
     @Test
@@ -151,18 +159,20 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         SSTableReader sstable = makeSSTable(false);
         Assert.assertTrue(unrepairedContains(sstable));
         Assert.assertFalse(repairedContains(sstable));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNull);
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         SSTableRepairStatusChanged notification;
 
         // change to pending repaired
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         notification = new SSTableRepairStatusChanged(Collections.singleton(sstable));
         csm.handleNotification(notification, cfs.getTracker());
         Assert.assertFalse(unrepairedContains(sstable));
         Assert.assertFalse(repairedContains(sstable));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNotNull);
-        Assert.assertTrue(pendingContains(repairID, sstable));
+        Assert.assertTrue(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
+        Assert.assertTrue(pendingContains(sstable));
 
         // change to repaired
         mutateRepaired(sstable, System.currentTimeMillis());
@@ -170,7 +180,7 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         csm.handleNotification(notification, cfs.getTracker());
         Assert.assertFalse(unrepairedContains(sstable));
         Assert.assertTrue(repairedContains(sstable));
-        Assert.assertFalse(pendingContains(repairID, sstable));
+        Assert.assertFalse(pendingContains(sstable));
     }
 
     @Test
@@ -180,14 +190,14 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
 
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
-        Assert.assertTrue(pendingContains(repairID, sstable));
+        Assert.assertTrue(pendingContains(sstable));
 
         // delete sstable
         SSTableDeletingNotification notification = new SSTableDeletingNotification(sstable);
         csm.handleNotification(notification, cfs.getTracker());
-        Assert.assertFalse(pendingContains(repairID, sstable));
+        Assert.assertFalse(pendingContains(sstable));
         Assert.assertFalse(unrepairedContains(sstable));
         Assert.assertFalse(repairedContains(sstable));
     }
@@ -209,7 +219,7 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         Assert.assertTrue(strategies.get(2).isEmpty());
 
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
 
         strategies = csm.getStrategies();
@@ -227,11 +237,12 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
         LocalSessionAccessor.finalizeUnsafe(repairID);
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNotNull);
-        Assert.assertNotNull(pendingContains(repairID, sstable));
+        Assert.assertTrue(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
+        Assert.assertTrue(pendingContains(sstable));
         Assert.assertTrue(sstable.isPendingRepair());
         Assert.assertFalse(sstable.isRepaired());
 
@@ -245,7 +256,9 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
 
         Assert.assertTrue(repairedContains(sstable));
         Assert.assertFalse(unrepairedContains(sstable));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNull);
+        Assert.assertFalse(pendingContains(sstable));
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         // sstable should have pendingRepair cleared, and repairedAt set correctly
         long expectedRepairedAt = ActiveRepairService.instance.getParentRepairSession(repairID).repairedAt;
@@ -264,12 +277,13 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
         LocalSessionAccessor.failUnsafe(repairID);
 
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNotNull);
-        Assert.assertNotNull(pendingContains(repairID, sstable));
+        Assert.assertTrue(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
+        Assert.assertTrue(pendingContains(sstable));
         Assert.assertTrue(sstable.isPendingRepair());
         Assert.assertFalse(sstable.isRepaired());
 
@@ -283,11 +297,78 @@ public class CompactionStrategyManagerPendingRepairTest extends AbstractPendingR
 
         Assert.assertFalse(repairedContains(sstable));
         Assert.assertTrue(unrepairedContains(sstable));
-        csm.getForPendingRepair(repairID).forEach(Assert::assertNull);
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
 
         // sstable should have pendingRepair cleared, and repairedAt set correctly
         Assert.assertFalse(sstable.isPendingRepair());
         Assert.assertFalse(sstable.isRepaired());
         Assert.assertEquals(ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getSSTableMetadata().repairedAt);
     }
+
+    @Test
+    public void finalizedSessionTransientCleanup()
+    {
+        Assert.assertTrue(cfs.getLiveSSTables().isEmpty());
+        UUID repairID = registerSession(cfs, true, true);
+        LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
+        SSTableReader sstable = makeSSTable(true);
+        mutateRepaired(sstable, repairID, true);
+        csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
+        LocalSessionAccessor.finalizeUnsafe(repairID);
+
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertTrue(hasTransientStrategiesFor(repairID));
+        Assert.assertTrue(transientContains(sstable));
+        Assert.assertFalse(pendingContains(sstable));
+        Assert.assertFalse(repairedContains(sstable));
+        Assert.assertFalse(unrepairedContains(sstable));
+
+        cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task
+        AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds());
+        Assert.assertNotNull(compactionTask);
+        Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass());
+
+        // run the compaction
+        compactionTask.execute(null);
+
+        Assert.assertTrue(cfs.getLiveSSTables().isEmpty());
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
+    }
+
+    @Test
+    public void failedSessionTransientCleanup()
+    {
+        Assert.assertTrue(cfs.getLiveSSTables().isEmpty());
+        UUID repairID = registerSession(cfs, true, true);
+        LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
+        SSTableReader sstable = makeSSTable(true);
+        mutateRepaired(sstable, repairID, true);
+        csm.handleNotification(new SSTableAddedNotification(Collections.singleton(sstable), null), cfs.getTracker());
+        LocalSessionAccessor.failUnsafe(repairID);
+
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertTrue(hasTransientStrategiesFor(repairID));
+        Assert.assertTrue(transientContains(sstable));
+        Assert.assertFalse(pendingContains(sstable));
+        Assert.assertFalse(repairedContains(sstable));
+        Assert.assertFalse(unrepairedContains(sstable));
+
+        cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task
+        AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds());
+        Assert.assertNotNull(compactionTask);
+        Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass());
+
+        // run the compaction
+        compactionTask.execute(null);
+
+        Assert.assertFalse(cfs.getLiveSSTables().isEmpty());
+        Assert.assertFalse(hasPendingStrategiesFor(repairID));
+        Assert.assertFalse(hasTransientStrategiesFor(repairID));
+        Assert.assertFalse(transientContains(sstable));
+        Assert.assertFalse(pendingContains(sstable));
+        Assert.assertFalse(repairedContains(sstable));
+        Assert.assertTrue(unrepairedContains(sstable));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java
index eeaaf5b..73e6852 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java
@@ -129,12 +129,12 @@ public class CompactionStrategyManagerTest
             if (i % 3 == 0)
             {
                 //make 1 third of sstables repaired
-                cfs.getCompactionStrategyManager().mutateRepaired(newSSTables, System.currentTimeMillis(), null);
+                cfs.getCompactionStrategyManager().mutateRepaired(newSSTables, System.currentTimeMillis(), null, false);
             }
             else if (i % 3 == 1)
             {
                 //make 1 third of sstables pending repair
-                cfs.getCompactionStrategyManager().mutateRepaired(newSSTables, 0, UUIDGen.getTimeUUID());
+                cfs.getCompactionStrategyManager().mutateRepaired(newSSTables, 0, UUIDGen.getTimeUUID(), false);
             }
             previousSSTables = currentSSTables;
         }
@@ -272,19 +272,19 @@ public class CompactionStrategyManagerTest
         DatabaseDescriptor.setAutomaticSSTableUpgradeEnabled(false);
     }
 
-    private static void assertHolderExclusivity(boolean isRepaired, boolean isPendingRepair, Class<? extends AbstractStrategyHolder> expectedType)
+    private static void assertHolderExclusivity(boolean isRepaired, boolean isPendingRepair, boolean isTransient, Class<? extends AbstractStrategyHolder> expectedType)
     {
         ColumnFamilyStore cfs = Keyspace.open(KS_PREFIX).getColumnFamilyStore(TABLE_PREFIX);
         CompactionStrategyManager csm = cfs.getCompactionStrategyManager();
 
-        AbstractStrategyHolder holder = csm.getHolder(isRepaired, isPendingRepair);
+        AbstractStrategyHolder holder = csm.getHolder(isRepaired, isPendingRepair, isTransient);
         assertNotNull(holder);
         assertSame(expectedType, holder.getClass());
 
         int matches = 0;
         for (AbstractStrategyHolder other : csm.getHolders())
         {
-            if (other.managesRepairedGroup(isRepaired, isPendingRepair))
+            if (other.managesRepairedGroup(isRepaired, isPendingRepair, isTransient))
             {
                 assertSame("holder assignment should be mutually exclusive", holder, other);
                 matches++;
@@ -293,13 +293,13 @@ public class CompactionStrategyManagerTest
         assertEquals(1, matches);
     }
 
-    private static void assertInvalieHolderConfig(boolean isRepaired, boolean isPendingRepair)
+    private static void assertInvalieHolderConfig(boolean isRepaired, boolean isPendingRepair, boolean isTransient)
     {
         ColumnFamilyStore cfs = Keyspace.open(KS_PREFIX).getColumnFamilyStore(TABLE_PREFIX);
         CompactionStrategyManager csm = cfs.getCompactionStrategyManager();
         try
         {
-            csm.getHolder(isRepaired, isPendingRepair);
+            csm.getHolder(isRepaired, isPendingRepair, isTransient);
             fail("Expected IllegalArgumentException");
         }
         catch (IllegalArgumentException e)
@@ -315,10 +315,14 @@ public class CompactionStrategyManagerTest
     @Test
     public void testMutualExclusiveHolderClassification() throws Exception
     {
-        assertHolderExclusivity(false, false, CompactionStrategyHolder.class);
-        assertHolderExclusivity(true, false, CompactionStrategyHolder.class);
-        assertHolderExclusivity(false, true, PendingRepairHolder.class);
-        assertInvalieHolderConfig(true, true);
+        assertHolderExclusivity(false, false, false, CompactionStrategyHolder.class);
+        assertHolderExclusivity(true, false, false, CompactionStrategyHolder.class);
+        assertHolderExclusivity(false, true, false, PendingRepairHolder.class);
+        assertHolderExclusivity(false, true, true, PendingRepairHolder.class);
+        assertInvalieHolderConfig(true, true, false);
+        assertInvalieHolderConfig(true, true, true);
+        assertInvalieHolderConfig(false, false, true);
+        assertInvalieHolderConfig(true, false, true);
     }
 
     PartitionPosition forKey(int key)
@@ -337,20 +341,23 @@ public class CompactionStrategyManagerTest
         ColumnFamilyStore cfs = createJBODMockCFS(numDir);
         Keyspace.open(cfs.keyspace.getName()).getColumnFamilyStore(cfs.name).disableAutoCompaction();
         assertTrue(cfs.getLiveSSTables().isEmpty());
-        List<SSTableReader> unrepaired = new ArrayList<>();
+        List<SSTableReader> transientRepairs = new ArrayList<>();
         List<SSTableReader> pendingRepair = new ArrayList<>();
+        List<SSTableReader> unrepaired = new ArrayList<>();
         List<SSTableReader> repaired = new ArrayList<>();
 
         for (int i = 0; i < numDir; i++)
         {
             int key = 100 * i;
-            unrepaired.add(createSSTableWithKey(cfs.keyspace.getName(), cfs.name, key++));
+            transientRepairs.add(createSSTableWithKey(cfs.keyspace.getName(), cfs.name, key++));
             pendingRepair.add(createSSTableWithKey(cfs.keyspace.getName(), cfs.name, key++));
+            unrepaired.add(createSSTableWithKey(cfs.keyspace.getName(), cfs.name, key++));
             repaired.add(createSSTableWithKey(cfs.keyspace.getName(), cfs.name, key++));
         }
 
-        cfs.getCompactionStrategyManager().mutateRepaired(pendingRepair, 0, UUID.randomUUID());
-        cfs.getCompactionStrategyManager().mutateRepaired(repaired, 1000, null);
+        cfs.getCompactionStrategyManager().mutateRepaired(transientRepairs, 0, UUID.randomUUID(), true);
+        cfs.getCompactionStrategyManager().mutateRepaired(pendingRepair, 0, UUID.randomUUID(), false);
+        cfs.getCompactionStrategyManager().mutateRepaired(repaired, 1000, null, false);
 
         DiskBoundaries boundaries = new DiskBoundaries(cfs.getDirectories().getWriteableLocations(),
                                                        Lists.newArrayList(forKey(100), forKey(200), forKey(300)),
@@ -358,7 +365,7 @@ public class CompactionStrategyManagerTest
 
         CompactionStrategyManager csm = new CompactionStrategyManager(cfs, () -> boundaries, true);
 
-        List<GroupedSSTableContainer> grouped = csm.groupSSTables(Iterables.concat(repaired, pendingRepair, unrepaired));
+        List<GroupedSSTableContainer> grouped = csm.groupSSTables(Iterables.concat( transientRepairs, pendingRepair, repaired, unrepaired));
 
         for (int x=0; x<grouped.size(); x++)
         {
@@ -372,7 +379,16 @@ public class CompactionStrategyManagerTest
                 if (sstable.isRepaired())
                     expected = repaired.get(y);
                 else if (sstable.isPendingRepair())
-                    expected = pendingRepair.get(y);
+                {
+                    if (sstable.isTransient())
+                    {
+                        expected = transientRepairs.get(y);
+                    }
+                    else
+                    {
+                        expected = pendingRepair.get(y);
+                    }
+                }
                 else
                     expected = unrepaired.get(y);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
index 599fc74..5370f33 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
@@ -96,9 +96,9 @@ public class CompactionTaskTest
         Assert.assertEquals(Transactional.AbstractTransactional.State.ABORTED, txn.state());
     }
 
-    private static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair) throws IOException
+    private static void mutateRepaired(SSTableReader sstable, long repairedAt, UUID pendingRepair, boolean isTransient) throws IOException
     {
-        sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, repairedAt, pendingRepair);
+        sstable.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable.descriptor, repairedAt, pendingRepair, isTransient);
         sstable.reloadSSTableMetadata();
     }
 
@@ -127,9 +127,9 @@ public class CompactionTaskTest
         SSTableReader pending1 = sstables.get(2);
         SSTableReader pending2 = sstables.get(3);
 
-        mutateRepaired(repaired, FBUtilities.nowInSeconds(), ActiveRepairService.NO_PENDING_REPAIR);
-        mutateRepaired(pending1, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
-        mutateRepaired(pending2, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID());
+        mutateRepaired(repaired, FBUtilities.nowInSeconds(), ActiveRepairService.NO_PENDING_REPAIR, false);
+        mutateRepaired(pending1, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID(), false);
+        mutateRepaired(pending2, ActiveRepairService.UNREPAIRED_SSTABLE, UUIDGen.getTimeUUID(), false);
 
         LifecycleTransaction txn = null;
         List<SSTableReader> toCompact = new ArrayList<>(sstables);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
index c91d2fe..857fa32 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
@@ -411,7 +411,7 @@ public class CompactionsCQLTest extends CQLTester
             cfs.forceBlockingFlush();
         }
         assertEquals(50, cfs.getLiveSSTables().size());
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepaired().get(0);
+        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first();
         AbstractCompactionTask act = lcs.getNextBackgroundTask(0);
         // we should be compacting all 50 sstables:
         assertEquals(50, act.transaction.originals().size());
@@ -445,7 +445,7 @@ public class CompactionsCQLTest extends CQLTester
 
         // mark the L1 sstable as compacting to make sure we trigger STCS in L0:
         LifecycleTransaction txn = cfs.getTracker().tryModify(l1sstable, OperationType.COMPACTION);
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepaired().get(0);
+        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first();
         AbstractCompactionTask act = lcs.getNextBackgroundTask(0);
         // note that max_threshold is 60 (more than the amount of L0 sstables), but MAX_COMPACTING_L0 is 32, which means we will trigger STCS with at most max_threshold sstables
         assertEquals(50, act.transaction.originals().size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 9ebe326..23e88fe 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -364,7 +364,7 @@ public class LeveledCompactionStrategyTest
         SSTableReader sstable1 = unrepaired.manifest.generations[2].get(0);
         SSTableReader sstable2 = unrepaired.manifest.generations[1].get(0);
 
-        sstable1.descriptor.getMetadataSerializer().mutateRepaired(sstable1.descriptor, System.currentTimeMillis(), null);
+        sstable1.descriptor.getMetadataSerializer().mutateRepairMetadata(sstable1.descriptor, System.currentTimeMillis(), null, false);
         sstable1.reloadSSTableMetadata();
         assertTrue(sstable1.isRepaired());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java
index 2b88c9c..d83e063 100644
--- a/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java
@@ -45,7 +45,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
 
@@ -63,7 +63,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
         LocalSessionAccessor.finalizeUnsafe(repairID);
@@ -82,7 +82,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
         LocalSessionAccessor.failUnsafe(repairID);
@@ -94,7 +94,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
     public void needsCleanupNoSession()
     {
         UUID fakeID = UUIDGen.getTimeUUID();
-        PendingRepairManager prm = new PendingRepairManager(cfs, null);
+        PendingRepairManager prm = new PendingRepairManager(cfs, null, false);
         Assert.assertTrue(prm.canCleanup(fakeID));
     }
 
@@ -106,7 +106,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
 
@@ -122,7 +122,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
         Assert.assertNotNull(prm.get(repairID));
@@ -140,13 +140,13 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
 
         repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         LocalSessionAccessor.finalizeUnsafe(repairID);
 
@@ -184,7 +184,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
 
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
         Assert.assertNotNull(prm.get(repairID));
@@ -202,7 +202,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         UUID repairID = registerSession(cfs, true, true);
         LocalSessionAccessor.prepareUnsafe(repairID, COORDINATOR, PARTICIPANTS);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertNotNull(prm.get(repairID));
         Assert.assertNotNull(prm.get(repairID));
@@ -225,7 +225,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         PendingRepairManager prm = csm.getPendingRepairManagers().get(0);
         UUID repairId = registerSession(cfs, true, true);
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairId);
+        mutateRepaired(sstable, repairId, false);
         prm.addSSTable(sstable);
         List<AbstractCompactionTask> tasks = csm.getUserDefinedTasks(Collections.singleton(sstable), 100);
         try
@@ -247,8 +247,8 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
         SSTableReader sstable = makeSSTable(true);
         SSTableReader sstable2 = makeSSTable(true);
 
-        mutateRepaired(sstable, repairId);
-        mutateRepaired(sstable2, repairId2);
+        mutateRepaired(sstable, repairId, false);
+        mutateRepaired(sstable2, repairId2, false);
         prm.addSSTable(sstable);
         prm.addSSTable(sstable2);
         List<AbstractCompactionTask> tasks = csm.getUserDefinedTasks(Lists.newArrayList(sstable, sstable2), 100);
@@ -296,7 +296,7 @@ public class PendingRepairManagerTest extends AbstractPendingRepairTest
 
         Assert.assertFalse(prm.hasDataForSession(repairID));
         SSTableReader sstable = makeSSTable(true);
-        mutateRepaired(sstable, repairID);
+        mutateRepaired(sstable, repairID, false);
         prm.addSSTable(sstable);
         Assert.assertTrue(prm.hasDataForSession(repairID));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
index 6428ab7..1292b7e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java
@@ -56,7 +56,7 @@ public class SingleSSTableLCSTaskTest extends CQLTester
         assertEquals(1, cfs.getLiveSSTables().size());
         cfs.getLiveSSTables().forEach(s -> assertEquals(2, s.getSSTableLevel()));
         // make sure compaction strategy is notified:
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepaired().iterator().next();
+        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first();
         for (int i = 0; i < lcs.manifest.getLevelCount(); i++)
         {
             if (i == 2)
@@ -98,7 +98,7 @@ public class SingleSSTableLCSTaskTest extends CQLTester
                 cfs.forceBlockingFlush();
         }
         // now we have a bunch of data in L0, first compaction will be a normal one, containing all sstables:
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepaired().get(0);
+        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first();
         AbstractCompactionTask act = lcs.getNextBackgroundTask(0);
         act.execute(null);
 
@@ -148,7 +148,7 @@ public class SingleSSTableLCSTaskTest extends CQLTester
         assertEquals(1, cfs.getLiveSSTables().size());
         for (SSTableReader sst : cfs.getLiveSSTables())
             assertEquals(0, sst.getSSTableMetadata().sstableLevel);
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepaired().iterator().next();
+        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first();
         assertEquals(1, lcs.getLevelSize(0));
         assertTrue(cfs.getTracker().getCompacting().isEmpty());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index 5694e86..e5ff138 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -1183,7 +1183,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
 
         SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList());
         StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator)
-                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, null, header)
+                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, null, false, header)
                                                  .get(MetadataType.STATS);
         SSTableReader reader = SSTableReader.internalOpen(descriptor,
                                                           components,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
index 5cda2ad..b7b7d4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
@@ -164,6 +164,7 @@ public class RealTransactionsTest extends SchemaLoader
                                                            0,
                                                            0,
                                                            null,
+                                                           false,
                                                            0,
                                                            SerializationHeader.make(cfs.metadata(), txn.originals()),
                                                            cfs.indexManager.listIndexes(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7431b43/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
index 76ebfd8..3b29cc5 100644
--- a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
@@ -119,11 +119,11 @@ public class CompactionManagerGetSSTablesForValidationTest
         Iterator<SSTableReader> iter = cfs.getLiveSSTables().iterator();
 
         repaired = iter.next();
-        repaired.descriptor.getMetadataSerializer().mutateRepaired(repaired.descriptor, System.currentTimeMillis(), null);
+        repaired.descriptor.getMetadataSerializer().mutateRepairMetadata(repaired.descriptor, System.currentTimeMillis(), null, false);
         repaired.reloadSSTableMetadata();
 
         pendingRepair = iter.next();
-        pendingRepair.descriptor.getMetadataSerializer().mutateRepaired(pendingRepair.descriptor, ActiveRepairService.UNREPAIRED_SSTABLE, sessionID);
+        pendingRepair.descriptor.getMetadataSerializer().mutateRepairMetadata(pendingRepair.descriptor, ActiveRepairService.UNREPAIRED_SSTABLE, sessionID, false);
         pendingRepair.reloadSSTableMetadata();
 
         unrepaired = iter.next();


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