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<Token>, 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