You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/11/03 13:29:11 UTC

[GitHub] [cassandra] adelapena opened a new pull request #801: CASSANDRA-16180 Coordination tests

adelapena opened a new pull request #801:
URL: https://github.com/apache/cassandra/pull/801


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540373687



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.exceptions.ReadFailureException;
+import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.metrics.ClientRequestMetrics;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.reads.DataResolver;
+import org.apache.cassandra.service.reads.ReadCallback;
+import org.apache.cassandra.service.reads.repair.ReadRepair;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
+{
+    private static final Logger logger = LoggerFactory.getLogger(RangeCommandIterator.class);
+
+    private static final ClientRequestMetrics rangeMetrics = new ClientRequestMetrics("RangeSlice");
+
+    private final CloseableIterator<ReplicaPlan.ForRangeRead> replicaPlans;
+    private final int totalRangeCount;
+    private final PartitionRangeReadCommand command;
+    private final boolean enforceStrictLiveness;
+
+    private final long startTime;
+    private final long queryStartNanoTime;
+    private DataLimits.Counter counter;
+    private PartitionIterator sentQueryIterator;
+
+    private final int maxConcurrencyFactor;
+    private int concurrencyFactor;
+    // The two following "metric" are maintained to improve the concurrencyFactor
+    // when it was not good enough initially.
+    private int liveReturned;
+    private int rangesQueried;
+    private int batchesRequested = 0;
+
+    RangeCommandIterator(CloseableIterator<ReplicaPlan.ForRangeRead> replicaPlans,
+                         PartitionRangeReadCommand command,
+                         int concurrencyFactor,
+                         int maxConcurrencyFactor,
+                         int totalRangeCount,
+                         long queryStartNanoTime)
+    {
+        this.replicaPlans = replicaPlans;
+        this.command = command;
+        this.concurrencyFactor = concurrencyFactor;
+        this.maxConcurrencyFactor = maxConcurrencyFactor;
+        this.totalRangeCount = totalRangeCount;
+        this.queryStartNanoTime = queryStartNanoTime;
+
+        startTime = System.nanoTime();
+        enforceStrictLiveness = command.metadata().enforceStrictLiveness();
+    }
+
+    @Override
+    protected RowIterator computeNext()
+    {
+        try
+        {
+            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
+            {
+                // If we don't have more range to handle, we're done
+                if (!replicaPlans.hasNext())
+                    return endOfData();
+
+                // else, sends the next batch of concurrent queries (after having close the previous iterator)
+                if (sentQueryIterator != null)
+                {
+                    liveReturned += counter.counted();
+                    sentQueryIterator.close();
+
+                    // It's not the first batch of queries and we're not done, so we we can use what has been
+                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
+                    updateConcurrencyFactor();
+                }
+                sentQueryIterator = sendNextRequests();
+            }
+
+            return sentQueryIterator.next();
+        }
+        catch (UnavailableException e)
+        {
+            rangeMetrics.unavailables.mark();
+            throw e;
+        }
+        catch (ReadTimeoutException e)
+        {
+            rangeMetrics.timeouts.mark();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            rangeMetrics.failures.mark();
+            throw e;
+        }
+    }
+
+    private void updateConcurrencyFactor()
+    {
+        liveReturned += counter.counted();
+
+        concurrencyFactor = computeConcurrencyFactor(totalRangeCount, rangesQueried, maxConcurrencyFactor, command.limits().count(), liveReturned);
+    }
+
+    @VisibleForTesting
+    static int computeConcurrencyFactor(int totalRangeCount, int rangesQueried, int maxConcurrencyFactor, int limit, int liveReturned)
+    {
+        maxConcurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, totalRangeCount - rangesQueried));
+        if (liveReturned == 0)
+        {
+            // we haven't actually gotten any results, so query up to the limit if not results so far
+            Tracing.trace("Didn't get any response rows; new concurrent requests: {}", maxConcurrencyFactor);
+            return maxConcurrencyFactor;
+        }
+
+        // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
+        // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
+        int remainingRows = limit - liveReturned;
+        float rowsPerRange = (float) liveReturned / (float) rangesQueried;
+        int concurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, Math.round(remainingRows / rowsPerRange)));
+        logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                     rowsPerRange, remainingRows, concurrencyFactor);
+        return concurrencyFactor;
+    }
+
+    /**
+     * Queries the provided sub-range.
+     *
+     * @param replicaPlan 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(ReplicaPlan.ForRangeRead replicaPlan, boolean isFirst)
+    {
+        PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaPlan.range(), isFirst);
+        // If enabled, request repaired data tracking info from full replicas but
+        // only if there are multiple full replicas to compare results from
+        if (DatabaseDescriptor.getRepairedDataTrackingForRangeReadsEnabled()
+            && replicaPlan.contacts().filter(Replica::isFull).size() > 1)
+        {
+            command.trackRepairedStatus();
+            rangeCommand.trackRepairedStatus();
+        }
+
+        ReplicaPlan.SharedForRangeRead sharedReplicaPlan = ReplicaPlan.shared(replicaPlan);
+        ReadRepair<EndpointsForRange, ReplicaPlan.ForRangeRead> readRepair
+                = ReadRepair.create(command, sharedReplicaPlan, queryStartNanoTime);

Review comment:
       nit/formatting: We'd usually have `=` on the line above?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r574665806



##########
File path: test/distributed/org/apache/cassandra/distributed/test/MetricsCountQueriesTest.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+
+public class MetricsCountQueriesTest extends TestBaseImpl
+{
+    @Test
+    public void testMetricsCountQueries() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.create(2)))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+            for (int i = 0; i < 100; i++)
+                cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (?,?,?)"), ALL, i, i, i);
+
+            long readCount1 = readCount(cluster.get(1));
+            long readCount2 = readCount(cluster.get(2));
+            for (int i = 0; i < 100; i++)
+                cluster.coordinator(1).execute(withKeyspace("SELECT * FROM %s.tbl WHERE pk = ? and ck = ?"), ALL, i, i);
+
+            readCount1 = readCount(cluster.get(1)) - readCount1;
+            readCount2 = readCount(cluster.get(2)) - readCount2;
+            Assert.assertEquals(readCount1, readCount2);
+            Assert.assertEquals(100, readCount1);
+        }
+    }
+
+    private static long readCount(IInvokableInstance instance)
+    {
+        return instance.callOnInstance(() -> Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").metric.readLatency.latency.getCount());

Review comment:
       This is another test just extracted from `SimpleReadWriteTest` without changes. I'm afraid we can't get that metric with the current API. The read latency is a `LatencyMetricsTimer` that extends `Timer`. However, the required count can't be accessed with [Metrics#getTimer](https://github.com/apache/cassandra-in-jvm-dtest-api/blob/trunk/src/main/java/org/apache/cassandra/distributed/shared/Metrics.java#L41-L42), which returns a double and fails if we pass it a `MetricValue.COUNT` value.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536337733



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()

Review comment:
       nit: Maybe we can give these better names now, especially since the Jira is already in the JavaDoc above.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r541617179



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;

Review comment:
       Shutting the nodes down might solve the problem of waiting for the timeouts. However I'm finding problems restarting multiple nodes, for example:
   ```java
   new TestCase()
   .nodes(3)
   .nodesToUpgrade(1)
   .upgrade(Versions.Major.v3X, Versions.Major.v4)
   .setup(c -> {})
   .runAfterNodeUpgrade((cluster, node) -> {
       cluster.get(1).shutdown().get();
       cluster.get(2).shutdown().get();
       cluster.get(1).startup();
       cluster.get(2).startup();
   }).run();
   ```
   That produces an `IllegalStateException: Can't use shut down instances, delegate is null` error, when I understand that it should work, unless I'm missing something.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537654306



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;

Review comment:
       I think yes, at least all the tests returning more than one row should require more that one page:
   * testSkinnyTableWithLiveRows
   * testMultipleMissedRows
   * testAscendingOrder
   * testDescendingOrder
   * testSkipEarlyTerminationRows
   * testSkipEarlyTerminationPartitions
   
   Also, even if there is only zero or one results to return and a page size of one, paging is still interesting with a limit greater than one because it can expose failures like CASSANDRA-16307. 
   
   I have added [here](https://github.com/apache/cassandra/pull/801/commits/c77a8870e8865a322706a8b49d69f264f5ed2dcd) some additional page sizes, just in case. These page sizes are not added as a JUnit parameter to not make the tests much slower.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r535839945



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,461 @@
+/*
+ * 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.range;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+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.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.Util.testPartitioner;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNode()
+    {
+        new TokenUpdater().withTokens(10, 20, 30, 40).update();
+
+        // with CLs requiring a single node all ranges are merged (unless they are wrapping)
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(min(), min()));
+            testRanges(cl, range(min(), max(25)));
+            testRanges(cl, range(min(), max(40)));
+            testRanges(cl, range(min(), max(50)));
+            testRanges(cl, range(max(20), max(30)));
+            testRanges(cl, range(max(25), min()));
+            testRanges(cl, range(max(25), max(35)));
+            testRanges(cl, range(max(50), min()));
+            testRanges(cl, range(max(40), max(10)), range(max(40), min()), range(min(), max(10))); // wrapped is split
+            testRanges(cl, range(max(25), max(15)), range(max(25), min()), range(min(), max(15))); // wrapped is split
+        }
+
+        // with CLs requiring more than a single node ranges are not merged due to the RF=2
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(min(), max(25)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(25)));
+            testRanges(cl,
+                       range(min(), max(40)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)));
+            testRanges(cl,
+                       range(min(), max(50)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), max(50)));
+            testRanges(cl,
+                       range(max(20), max(30)));
+            testRanges(cl,
+                       range(max(25), min()),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(max(50), min()));
+            testRanges(cl,
+                       range(max(30), max(10)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10))); // wrapped
+            testRanges(cl,
+                       range(max(25), max(15)),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(15))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithContinuousRanges()
+    {
+        new TokenUpdater().withTokens("127.0.0.1", 10, 20, 30)
+                          .withTokens("127.0.0.2", 40, 50, 60)
+                          .withTokens("127.0.0.3", 70, 80, 90)
+                          .update();
+
+        // with CL=ANY the ranges are fully merged (unless they are wrapping)
+        testMultiNodeFullMerge(ANY);
+
+        // with CL=THREE the ranges are not merged at all
+        testMultiNodeNoMerge(THREE);
+
+        // with CLs requiring a single node the ranges are merged in a per-node basis
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(60)),
+                       range(max(60), min()));
+            testRanges(cl,
+                       range(min(), max(25)));
+            testRanges(cl,
+                       range(min(), max(40)));
+            testRanges(cl,
+                       range(min(), max(50)));
+            testRanges(cl,
+                       range(max(20), max(30)));
+            testRanges(cl,
+                       range(max(25), min()),
+                       range(max(25), max(60)),
+                       range(max(60), min()));
+            testRanges(cl,
+                       range(max(25), max(35)),
+                       range(max(25), max(35)));
+            testRanges(cl,
+                       range(max(50), min()),
+                       range(max(50), max(90)),
+                       range(max(90), min()));
+            testRanges(cl,
+                       range(max(50), max(10)),
+                       range(max(50), max(90)),
+                       range(max(90), min()),
+                       range(min(), max(10))); // wrapping range
+            testRanges(cl,
+                       range(max(25), max(15)),
+                       range(max(25), max(60)),
+                       range(max(60), min()),
+                       range(min(), max(15))); // wrapping range
+        }
+
+        // with other CLs the ranges are merged in a similar per-node basis
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(30)),
+                       range(max(30), max(60)),
+                       range(max(60), max(90)),
+                       range(max(90), min()));
+            testRanges(cl,
+                       range(min(), max(25)));
+            testRanges(cl,
+                       range(min(), max(40)),
+                       range(min(), max(30)),
+                       range(max(30), max(40)));
+            testRanges(cl,
+                       range(min(), max(50)),
+                       range(min(), max(30)),
+                       range(max(30), max(50)));
+            testRanges(cl,
+                       range(max(20), max(30)));
+            testRanges(cl,
+                       range(max(25), min()),
+                       range(max(25), max(30)),
+                       range(max(30), max(60)),
+                       range(max(60), max(90)),
+                       range(max(90), min()));
+            testRanges(cl,
+                       range(max(25), max(35)),
+                       range(max(25), max(30)),
+                       range(max(30), max(35)));
+            testRanges(cl,
+                       range(max(50), min()),
+                       range(max(50), max(60)),
+                       range(max(60), max(90)),
+                       range(max(90), min()));
+            testRanges(cl,
+                       range(max(50), max(10)),
+                       range(max(50), max(60)),
+                       range(max(60), max(90)),
+                       range(max(90), min()),
+                       range(min(), max(10))); // wrapping range
+            testRanges(cl,
+                       range(max(25), max(15)),
+                       range(max(25), max(30)),
+                       range(max(30), max(60)),
+                       range(max(60), max(90)),
+                       range(max(90), min()),
+                       range(min(), max(15))); // wrapping range
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster when the token ranges overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithDiscontinuousRanges() throws UnknownHostException

Review comment:
       nit: Probably don't need to throw `UnknownHostException` (remember to remove its import)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537636539



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,460 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+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.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.Util.testPartitioner;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNode()
+    {
+        new TokenUpdater().withTokens(10, 20, 30, 40).update();
+
+        // with CLs requiring a single node all ranges are merged (unless they are wrapping)
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(min(), min()));
+            testRanges(cl, range(min(), max(25)));
+            testRanges(cl, range(min(), max(40)));
+            testRanges(cl, range(min(), max(50)));
+            testRanges(cl, range(max(20), max(30)));
+            testRanges(cl, range(max(25), min()));
+            testRanges(cl, range(max(25), max(35)));
+            testRanges(cl, range(max(50), min()));
+            testRanges(cl, range(max(40), max(10)), range(max(40), min()), range(min(), max(10))); // wrapped is split
+            testRanges(cl, range(max(25), max(15)), range(max(25), min()), range(min(), max(15))); // wrapped is split
+        }
+
+        // with CLs requiring more than a single node ranges are not merged due to the RF=2
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(min(), max(25)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(25)));
+            testRanges(cl,
+                       range(min(), max(40)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)));
+            testRanges(cl,
+                       range(min(), max(50)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), max(50)));
+            testRanges(cl,
+                       range(max(20), max(30)));
+            testRanges(cl,
+                       range(max(25), min()),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(max(50), min()));
+            testRanges(cl,
+                       range(max(30), max(10)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10))); // wrapped
+            testRanges(cl,
+                       range(max(25), max(15)),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(15))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithContinuousRanges()
+    {
+        new TokenUpdater().withTokens("127.0.0.1", 10, 20, 30)
+                          .withTokens("127.0.0.2", 40, 50, 60)
+                          .withTokens("127.0.0.3", 70, 80, 90)
+                          .update();
+
+        // with CL=ANY the ranges are fully merged (unless they are wrapping)
+        testMultiNodeFullMerge(ANY);
+
+        // with CL=THREE the ranges are not merged at all
+        testMultiNodeNoMerge(THREE);
+
+        // with CLs requiring a single node the ranges are merged in a per-node basis
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(60)),
+                       range(max(60), min()));

Review comment:
       Done [here](https://github.com/apache/cassandra/pull/801/commits/3fd6f859b42a8934b1ee8dd46342f9b3637fc164).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538892515



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));

Review comment:
       @adelapena We might just have to separate the tests by starting version, etc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537714743



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;

Review comment:
       Right, I have reduced the number of fetch sizes. Also, the query without paging was needlessly done inside the  `PAGE_SIZES` loop; I have moved it out. 
   Also, I have moved the selection of coordinator from the JUnit parameters to the inside of the `assertRows` method. This should be valid since read repair strategy is disabled, and it makes the tests significantly faster (from 1m42s to 52s in my laptop).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r572935237



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PagingTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+
+public class PagingTest extends TestBaseImpl
+{
+    @Test
+    public void testPaging() throws Throwable
+    {
+        try (Cluster cluster = init(builder().withNodes(3).start());
+             Cluster singleNode = init(builder().withNodes(1).withSubnet(1).start()))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+            singleNode.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+
+            for (int i = 0; i < 10; i++)
+            {
+                for (int j = 0; j < 10; j++)
+                {
+                    cluster.coordinator(1)
+                           .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                    singleNode.coordinator(1)
+                              .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                }
+            }
+
+            int[] pageSizes = new int[]{ 1, 2, 3, 5, 10, 20, 50 };
+            String[] statements = new String[]{ withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl LIMIT 3"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10)"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10) LIMIT 2")
+            };
+            for (String statement : statements)
+            {
+                for (int pageSize : pageSizes)
+                {
+                    assertRows(cluster.coordinator(1).executeWithPaging(statement, QUORUM, pageSize),
+                               singleNode.coordinator(1).executeWithPaging(statement, QUORUM, Integer.MAX_VALUE));

Review comment:
       non-paged response will be same accross all page sizes.

##########
File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
##########
@@ -477,6 +477,11 @@ public ICoordinator coordinator(int node)
         return instances.get(node - 1).coordinator();
     }
 
+    public List<ICoordinator> coordinators()

Review comment:
       Should we return `stream` as we do with, say, instances here below?

##########
File path: test/distributed/org/apache/cassandra/distributed/test/SSTableSkippingReadTest.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.distributed.test;
+
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertEquals;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+
+public class SSTableSkippingReadTest extends TestBaseImpl
+{
+    @Test
+    public void skippedSSTableWithPartitionDeletionTest() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.create(2)))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY(pk, ck))"));
+            // insert a partition tombstone on node 1, the deletion timestamp should end up being the sstable's minTimestamp
+            cluster.get(1).executeInternal(withKeyspace("DELETE FROM %s.tbl USING TIMESTAMP 1 WHERE pk = 0"));
+            // and a row from a different partition, to provide the sstable's min/max clustering
+            cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, 1, 1) USING TIMESTAMP 2"));
+            cluster.get(1).flush(KEYSPACE);
+            // expect a single sstable, where minTimestamp equals the timestamp of the partition delete
+            cluster.get(1).runOnInstance(() -> {
+                Set<SSTableReader> sstables = Keyspace.open(KEYSPACE)
+                                                      .getColumnFamilyStore("tbl")
+                                                      .getLiveSSTables();

Review comment:
       It might be possible to get this through `LiveSSTableCount` metric.

##########
File path: test/distributed/org/apache/cassandra/distributed/test/PagingTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+
+public class PagingTest extends TestBaseImpl
+{
+    @Test
+    public void testPaging() throws Throwable
+    {
+        try (Cluster cluster = init(builder().withNodes(3).start());
+             Cluster singleNode = init(builder().withNodes(1).withSubnet(1).start()))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+            singleNode.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+
+            for (int i = 0; i < 10; i++)
+            {
+                for (int j = 0; j < 10; j++)
+                {
+                    cluster.coordinator(1)
+                           .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                    singleNode.coordinator(1)
+                              .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                }
+            }
+
+            int[] pageSizes = new int[]{ 1, 2, 3, 5, 10, 20, 50 };
+            String[] statements = new String[]{ withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl LIMIT 3"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10)"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10) LIMIT 2")
+            };
+            for (String statement : statements)
+            {
+                for (int pageSize : pageSizes)
+                {
+                    assertRows(cluster.coordinator(1).executeWithPaging(statement, QUORUM, pageSize),
+                               singleNode.coordinator(1).executeWithPaging(statement, QUORUM, Integer.MAX_VALUE));

Review comment:
       There's no interplay between SELECT, paging, and range tombstones (plus reversed data types), but I think it's best we leave it to automated/fuzz tests.

##########
File path: test/distributed/org/apache/cassandra/distributed/test/MetricsCountQueriesTest.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+
+public class MetricsCountQueriesTest extends TestBaseImpl
+{
+    @Test
+    public void testMetricsCountQueries() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.create(2)))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+            for (int i = 0; i < 100; i++)
+                cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (?,?,?)"), ALL, i, i, i);
+
+            long readCount1 = readCount(cluster.get(1));
+            long readCount2 = readCount(cluster.get(2));
+            for (int i = 0; i < 100; i++)
+                cluster.coordinator(1).execute(withKeyspace("SELECT * FROM %s.tbl WHERE pk = ? and ck = ?"), ALL, i, i);
+
+            readCount1 = readCount(cluster.get(1)) - readCount1;
+            readCount2 = readCount(cluster.get(2)) - readCount2;
+            Assert.assertEquals(readCount1, readCount2);
+            Assert.assertEquals(100, readCount1);
+        }
+    }
+
+    private static long readCount(IInvokableInstance instance)
+    {
+        return instance.callOnInstance(() -> Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").metric.readLatency.latency.getCount());

Review comment:
       We're exposing metrics through `Instance#metrics` now btw.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540185407



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r526195979



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.range;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.rp;
+import static org.apache.cassandra.Util.token;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(rp(""), rp("")));
+            testRanges(cl, range(rp(""), rp("25")));
+            testRanges(cl, range(rp(""), rp("40")));
+            testRanges(cl, range(rp(""), rp("50")));
+            testRanges(cl, range(rp("20"), rp("30")));
+            testRanges(cl, range(rp("25"), rp("")));
+            testRanges(cl, range(rp("25"), rp("35")));
+            testRanges(cl, range(rp("50"), rp("")));
+            testRanges(cl, range(rp("40"), rp("10")), range(rp("40"), rp("")), range(rp(""), rp("10"))); // wrapped is split
+            testRanges(cl, range(rp("25"), rp("15")), range(rp("25"), rp("")), range(rp(""), rp("15"))); // wrapped is split
+        }
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that doesn't allow to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithNotMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(rp(""), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp(""), rp("25")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), rp("25")));
+            testRanges(cl,
+                       range(rp(""), rp("40")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), rp("40")));
+            testRanges(cl,
+                       range(rp(""), rp("50")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("50")));
+            testRanges(cl,
+                       range(rp("20"), rp("30")),
+                       range(rp("20"), endOf("20")),
+                       range(endOf("20"), rp("30")));
+            testRanges(cl,
+                       range(rp("25"), rp("")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp("50"), rp("")));
+            testRanges(cl,
+                       range(rp("30"), rp("10")),
+                       range(rp("30"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), rp("10"))); // wrapped
+            testRanges(cl,
+                       range(rp("25"), rp("15")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), rp("15"))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster and a read consistency level that allows to merge ranges,
+     * particularly when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithMergingConsistencyLevelContinuous() throws UnknownHostException
+    {
+        new TokenUpdater().withTokens(InetAddressAndPort.getByName("127.0.0.1"), "10", "20", "30")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.2"), "40", "50", "60")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.3"), "70", "80", "90")
+                          .update();

Review comment:
       You are totally right, I have added those calls to `Gossiper#initializeNodeUnsafe` and changed the range tests, that weren't calculating the tokens properly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538812923



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)
+                                      .upgrade(Versions.Major.v30, Versions.Major.v4)
+                                      .upgrade(Versions.Major.v3X, Versions.Major.v4)
+                                      .withConfig(cfg -> cfg.with(Feature.NETWORK, Feature.GOSSIP)));
+    }
+
+    private void testConsistency(TestCase testCase) throws Throwable
+    {
+        List<ConsistencyTester> testers = new ArrayList<>();
+        testers.addAll(ConsistencyTester.create(1, ALL, THREE));
+        testers.addAll(ConsistencyTester.create(2, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO));
+        testers.addAll(ConsistencyTester.create(3, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO, ONE, LOCAL_ONE));
+
+        testCase.setup(cluster -> {
+            ConsistencyTester.createTable(cluster);
+            for (ConsistencyTester tester : testers)
+                tester.writeRows(cluster);
+        }).runAfterNodeAndClusterUpgrade(cluster -> {

Review comment:
       One thing we could do in the post-cluster-upgrade section is progressively stop nodes depending on the read CL like `TestAvailability` does. The only problem with that seems to be the way the CLs are grouped into a `ConsistencyTester` right now. I suppose those could be sliced the other way though (the ALL/THREE tests, then the QUORUM tests, etc.) and node stops performed between them...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r557729264



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PutGetTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRow;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Simple put/get tests using different types of query, paticularly when the data is spread across memory and multiple
+ * sstables and using different compressors. All available compressors are tested. Both ascending and descending
+ * clustering orders are tested. The read queries are run using every node as a coordinator, with and without paging.
+ */
+@RunWith(Parameterized.class)
+public class PutGetTest extends TestBaseImpl
+{
+    private static final int NUM_NODES = 4;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final String CREATE_TABLE = "CREATE TABLE %s(k int, c int, v int, PRIMARY KEY (k, c)) " +
+                                               "WITH CLUSTERING ORDER BY (c %s) " +
+                                               "AND COMPRESSION = { 'class': '%s' } " +
+                                               "AND READ_REPAIR = 'none'";
+    private static final String[] COMPRESSORS = new String[]{ NoopCompressor.class.getSimpleName(),
+                                                              LZ4Compressor.class.getSimpleName(),
+                                                              DeflateCompressor.class.getSimpleName(),
+                                                              SnappyCompressor.class.getSimpleName(),
+                                                              ZstdCompressor.class.getSimpleName() };
+
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    /**
+     * The sstable compressor to be used.
+     */
+    @Parameterized.Parameter
+    public String compressor;
+
+    /**
+     * Whether the clustering order is reverse.
+     */
+    @Parameterized.Parameter(1)
+    public boolean reverse;
+
+    private String tableName;
+
+    @Parameterized.Parameters(name = "{index}: compressor={0} reverse={1}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (String compressor : COMPRESSORS)
+            for (boolean reverse : BOOLEANS)
+                result.add(new Object[]{ compressor, reverse });
+        return result;
+    }
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build(NUM_NODES).start(), REPLICATION_FACTOR);
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void before()
+    {
+        // create the table
+        tableName = String.format("%s.t_%d", KEYSPACE, seq.getAndIncrement());
+        cluster.schemaChange(String.format(CREATE_TABLE, tableName, reverse ? "DESC" : "ASC", compressor));
+    }
+
+    @After
+    public void after()
+    {
+        cluster.schemaChange(format("DROP TABLE %s"));
+    }
+
+    /**
+     * Simple put/get on a single partition with a few rows, reading with a single partition query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_putget[_snappy|_deflate]().
+     */
+    @Test
+    public void testPartitionQuery()
+    {
+        int numRows = 10;
+
+        writeRows(1, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s WHERE k=?", 0);
+        Assert.assertEquals(numRows, rows.length);
+        for (int c = 0; c < numRows; c++)
+        {
+            validateRow(rows[c], numRows, 0, c);
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with a range query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_rangeputget().
+     */
+    @Test
+    public void testRangeQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s");
+        Assert.assertEquals(numPartitions * numRows, rows.length);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            for (int c = 0; c < numRows; c++)
+            {
+                Object[] row = rows[k * numRows + c];
+                validateRow(row, numRows, k, c);
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on a single partition with multiple rows, reading with slice queries.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_wide_row().
+     */
+    @Test
+    public void testSliceQuery()
+    {
+        int numRows = 100;
+
+        writeRows(1, numRows);
+
+        String query = "SELECT * FROM %s WHERE k=? AND c>=? AND c<?";
+        for (int sliceSize : Arrays.asList(10, 20, 100))
+        {
+            for (int c = 0; c < numRows; c = c + sliceSize)
+            {
+                Object[][] rows = readRows(query, 0, c, c + sliceSize);
+                Assert.assertEquals(sliceSize, rows.length);
+
+                for (int i = 0; i < sliceSize; i++)
+                {
+                    Object[] row = rows[i];
+                    validateRow(row, numRows, 0, c + i);
+                }
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with IN queries.
+     */
+    @Test
+    public void testInQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        String query = "SELECT * FROM %s WHERE k IN (?, ?)";
+        for (int k = 0; k < numPartitions; k += 2)
+        {
+            Object[][] rows = readRows(query, k, k + 1);
+            Assert.assertEquals(numRows * 2, rows.length);
+
+            for (int i = 0; i < 2; i++)
+            {
+                for (int c = 0; c < numRows; c++)
+                {
+                    Object[] row = rows[i * numRows + c];
+                    validateRow(row, numRows, k + i, c);
+                }
+            }
+        }
+    }
+
+    /**
+     * Writes {@code numPartitions} with {@code numRows} each, with overrides in different sstables and memtables.
+     */
+    private void writeRows(int numPartitions, int numRows)

Review comment:
       nit: `numRows` -> `rowsPerPartition`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522111527



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeIterator.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.RingPosition;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.Pair;
+
+class RangeIterator extends AbstractIterator<ReplicaPlan.ForRangeRead>

Review comment:
       I have changed it to `ReplicaPlanIterator` [here](https://github.com/apache/cassandra/pull/801/commits/f10e0407eb71b9e6ef0ac221513f1899e2f25e84). It's a bit confusing that since the introduction of `ReplicaPlan` we are indistinctly calling `range` to both `ReplicaPlan.ForRangeRead` and actual `Range` instances. I have tried to rename `ReplicaPlan.ForRangeRead` instances too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538808249



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase

Review comment:
       nit: Perhaps `ReadConsistencyTest` or `MixedModeConsistencyTest`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r557732470



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PutGetTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRow;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Simple put/get tests using different types of query, paticularly when the data is spread across memory and multiple
+ * sstables and using different compressors. All available compressors are tested. Both ascending and descending
+ * clustering orders are tested. The read queries are run using every node as a coordinator, with and without paging.
+ */
+@RunWith(Parameterized.class)
+public class PutGetTest extends TestBaseImpl
+{
+    private static final int NUM_NODES = 4;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final String CREATE_TABLE = "CREATE TABLE %s(k int, c int, v int, PRIMARY KEY (k, c)) " +
+                                               "WITH CLUSTERING ORDER BY (c %s) " +
+                                               "AND COMPRESSION = { 'class': '%s' } " +
+                                               "AND READ_REPAIR = 'none'";
+    private static final String[] COMPRESSORS = new String[]{ NoopCompressor.class.getSimpleName(),
+                                                              LZ4Compressor.class.getSimpleName(),
+                                                              DeflateCompressor.class.getSimpleName(),
+                                                              SnappyCompressor.class.getSimpleName(),
+                                                              ZstdCompressor.class.getSimpleName() };
+
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    /**
+     * The sstable compressor to be used.
+     */
+    @Parameterized.Parameter
+    public String compressor;
+
+    /**
+     * Whether the clustering order is reverse.
+     */
+    @Parameterized.Parameter(1)
+    public boolean reverse;
+
+    private String tableName;
+
+    @Parameterized.Parameters(name = "{index}: compressor={0} reverse={1}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (String compressor : COMPRESSORS)
+            for (boolean reverse : BOOLEANS)
+                result.add(new Object[]{ compressor, reverse });
+        return result;
+    }
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build(NUM_NODES).start(), REPLICATION_FACTOR);
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void before()
+    {
+        // create the table
+        tableName = String.format("%s.t_%d", KEYSPACE, seq.getAndIncrement());
+        cluster.schemaChange(String.format(CREATE_TABLE, tableName, reverse ? "DESC" : "ASC", compressor));
+    }
+
+    @After
+    public void after()
+    {
+        cluster.schemaChange(format("DROP TABLE %s"));
+    }
+
+    /**
+     * Simple put/get on a single partition with a few rows, reading with a single partition query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_putget[_snappy|_deflate]().
+     */
+    @Test
+    public void testPartitionQuery()
+    {
+        int numRows = 10;
+
+        writeRows(1, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s WHERE k=?", 0);
+        Assert.assertEquals(numRows, rows.length);
+        for (int c = 0; c < numRows; c++)
+        {
+            validateRow(rows[c], numRows, 0, c);
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with a range query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_rangeputget().
+     */
+    @Test
+    public void testRangeQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s");
+        Assert.assertEquals(numPartitions * numRows, rows.length);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            for (int c = 0; c < numRows; c++)
+            {
+                Object[] row = rows[k * numRows + c];
+                validateRow(row, numRows, k, c);
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on a single partition with multiple rows, reading with slice queries.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_wide_row().
+     */
+    @Test
+    public void testSliceQuery()
+    {
+        int numRows = 100;
+
+        writeRows(1, numRows);
+
+        String query = "SELECT * FROM %s WHERE k=? AND c>=? AND c<?";
+        for (int sliceSize : Arrays.asList(10, 20, 100))
+        {
+            for (int c = 0; c < numRows; c = c + sliceSize)
+            {
+                Object[][] rows = readRows(query, 0, c, c + sliceSize);
+                Assert.assertEquals(sliceSize, rows.length);
+
+                for (int i = 0; i < sliceSize; i++)
+                {
+                    Object[] row = rows[i];
+                    validateRow(row, numRows, 0, c + i);
+                }
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with IN queries.
+     */
+    @Test
+    public void testInQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        String query = "SELECT * FROM %s WHERE k IN (?, ?)";
+        for (int k = 0; k < numPartitions; k += 2)
+        {
+            Object[][] rows = readRows(query, k, k + 1);
+            Assert.assertEquals(numRows * 2, rows.length);
+
+            for (int i = 0; i < 2; i++)
+            {
+                for (int c = 0; c < numRows; c++)
+                {
+                    Object[] row = rows[i * numRows + c];
+                    validateRow(row, numRows, k + i, c);
+                }
+            }
+        }
+    }
+
+    /**
+     * Writes {@code numPartitions} with {@code numRows} each, with overrides in different sstables and memtables.
+     */
+    private void writeRows(int numPartitions, int numRows)
+    {
+        String update = format("UPDATE %s SET v=? WHERE k=? AND c=?");
+        ICoordinator coordinator = cluster.coordinator(1);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            // insert all the partition rows in a single sstable
+            for (int c = 0; c < numRows; c++)
+                coordinator.execute(update, QUORUM, c, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows in a second sstable
+            for (int c = 0; c < numRows; c += 2)
+                coordinator.execute(update, QUORUM, c + numRows, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows only in memtable
+            for (int c = 0; c < numRows; c += 5)
+                coordinator.execute(update, QUORUM, c + numRows * 2, k, c);
+        }
+    }
+
+    /**
+     * Runs the specified query in all coordinators, with and without paging.
+     */
+    private Object[][] readRows(String query, Object... boundValues)
+    {
+        query = format(query);
+        Object[][] rows = null;
+
+        // verify that all coordinators return the same results for the query, regardless of paging
+        for (ICoordinator coordinator : cluster.coordinators())
+        {
+            for (boolean paging : BOOLEANS)
+            {
+                Object[][] readRows = paging
+                                      ? Iterators.toArray(coordinator.executeWithPaging(query, QUORUM, 1, boundValues),
+                                                          Object[].class)
+                                      : coordinator.execute(query, QUORUM, boundValues);
+                if (rows == null)
+                    rows = readRows;
+                else
+                    assertRows(rows, readRows);

Review comment:
       nit: Perhaps we should write an explicit message here indicating the assertion has failed because paged and non-paged versions of the result are different?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537642033



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))

Review comment:
       Same as before, that's probably due to writes having the same timestamp.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536423073



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)
+                    cluster.get(node == NUM_NODES ? 1 : node + 1).executeInternal(formattedQuery);

Review comment:
       This brings me to another point. If we're really focused on coordination here, perhaps we could just use `executeInternal()` for all writes, make things uniform. It isolates us from bugs in replication I, but perhaps I'm being too picky...ignore if you don't think it matters.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522290988



##########
File path: test/unit/org/apache/cassandra/service/reads/range/RangeIteratorTest.java
##########
@@ -236,4 +181,71 @@ public void testGRRExactBounds() throws Throwable
         testGRRKeys(exBounds(rp(""), rp("")), range(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
         testGRRKeys(incExBounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
     }
+
+    /**
+     * Test {@link RangeIterator#getRestrictedRanges(AbstractBounds)} for tokens.
+     */
+    @SafeVarargs
+    private final void testGRR(AbstractBounds<Token> queryRange, AbstractBounds<Token>... expected)
+    {
+        // Testing for tokens
+        List<AbstractBounds<Token>> restricted = RangeIterator.getRestrictedRanges(queryRange);

Review comment:
       What I don't understand is why `getRestrictedRanges` accepts any kind of `RingPosition` bounds when it's only invoked with bounds of `PartitionPosition`s obtained exclusively from `DataRange#keyRange`.
   
   The test takes care of testing bounds of `Token`s but we never use those, so I wonder whether we could just get rid of the generics in `getRestrictedRanges` and the associated tests for tokens. WDYT?
   
   Also, we could add a getter for the ranges and make `getRestrictedRanges` private, so we tests that attribute instead of the static method. Unfortunately testing `computeNext` seems more difficult since it tries to contact nodes that aren't really there.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r542653378



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;
+
+            UUID partitionKey = UUID.randomUUID();
+            coordinator.execute(INSERT, writeConsistencyLevel, partitionKey, 1, 10);
+            coordinator.execute(INSERT, writeConsistencyLevel, partitionKey, 2, 20);
+            assertRows(coordinator.execute(SELECT, readConsistencyLevel, partitionKey),

Review comment:
       Good idea, done [here](https://github.com/apache/cassandra/pull/801/commits/2e180a81f3b922a5a47dc66232a7f7cc258b06b8).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540355622



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));

Review comment:
       Unless there's a way we can clear metaspace between methods, it might have to be separate classes?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #801:
URL: https://github.com/apache/cassandra/pull/801#issuecomment-725602263


   @adelapena This looks like it's going in a good direction. If we want to expand coverage, some basic unit-level tests for `SingleRangeResponse` and `RangeMerger` might make sense, as well as a quick audit of the other now-freestanding classes for corner cases.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540371017



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;
+
+            UUID partitionKey = UUID.randomUUID();
+            coordinator.execute(INSERT, writeConsistencyLevel, partitionKey, 1, 10);
+            coordinator.execute(INSERT, writeConsistencyLevel, partitionKey, 2, 20);
+            assertRows(coordinator.execute(SELECT, readConsistencyLevel, partitionKey),

Review comment:
       @adelapena If this fails, will it be obvious which tester and which R/W CL pair we're dealing with? Perhaps we could  catch any exception here, wrap it in an `AssertionError` with an appropriate description of the current `AvailabilityTester`, and re-throw?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r525565841



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.range;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.rp;
+import static org.apache.cassandra.Util.token;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(rp(""), rp("")));
+            testRanges(cl, range(rp(""), rp("25")));
+            testRanges(cl, range(rp(""), rp("40")));
+            testRanges(cl, range(rp(""), rp("50")));
+            testRanges(cl, range(rp("20"), rp("30")));
+            testRanges(cl, range(rp("25"), rp("")));
+            testRanges(cl, range(rp("25"), rp("35")));
+            testRanges(cl, range(rp("50"), rp("")));
+            testRanges(cl, range(rp("40"), rp("10")), range(rp("40"), rp("")), range(rp(""), rp("10"))); // wrapped is split
+            testRanges(cl, range(rp("25"), rp("15")), range(rp("25"), rp("")), range(rp(""), rp("15"))); // wrapped is split
+        }
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that doesn't allow to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithNotMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(rp(""), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp(""), rp("25")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), rp("25")));
+            testRanges(cl,
+                       range(rp(""), rp("40")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), rp("40")));
+            testRanges(cl,
+                       range(rp(""), rp("50")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("50")));
+            testRanges(cl,
+                       range(rp("20"), rp("30")),
+                       range(rp("20"), endOf("20")),
+                       range(endOf("20"), rp("30")));
+            testRanges(cl,
+                       range(rp("25"), rp("")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp("50"), rp("")));
+            testRanges(cl,
+                       range(rp("30"), rp("10")),
+                       range(rp("30"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), rp("10"))); // wrapped
+            testRanges(cl,
+                       range(rp("25"), rp("15")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), rp("15"))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster and a read consistency level that allows to merge ranges,
+     * particularly when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithMergingConsistencyLevelContinuous() throws UnknownHostException
+    {
+        new TokenUpdater().withTokens(InetAddressAndPort.getByName("127.0.0.1"), "10", "20", "30")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.2"), "40", "50", "60")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.3"), "70", "80", "90")
+                          .update();

Review comment:
       I think this is actually a bug in a tests as well. Without `.2` and `.3` visible to the `Gossiper`, the `ReplicaPlanIterator` produces replica plans that have no candidate endpoints, and that means we merge much less than we should. One other way we might be able to see this is by having the test extend `CQLTester` and calling `requireNetwork()` after `prepareServer()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537675532



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))
+              .assertRows("SELECT DISTINCT id FROM %s LIMIT 1", row(3));
+    }
+
+    /**
+     * Tests SRP when more than one row is missing.
+     * <p>
+     * See CASSANDRA-12872.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_12872()}.
+     */
+    @Test
+    public void testMultipleMissedRows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .allNodes(0, 4, i -> String.format("INSERT INTO %%s (pk, ck) VALUES (0, %d)", i))
+              .onlyNode1("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2, 3)",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 5)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 4)")
+              .assertRows("SELECT ck FROM %s WHERE pk = 0 LIMIT 2", row(0), row(4));

Review comment:
       Interesting. Well, in any case, I can't get it to fail now :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r574629290



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PagingTest.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+
+public class PagingTest extends TestBaseImpl
+{
+    @Test
+    public void testPaging() throws Throwable
+    {
+        try (Cluster cluster = init(builder().withNodes(3).start());
+             Cluster singleNode = init(builder().withNodes(1).withSubnet(1).start()))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+            singleNode.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"));
+
+            for (int i = 0; i < 10; i++)
+            {
+                for (int j = 0; j < 10; j++)
+                {
+                    cluster.coordinator(1)
+                           .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                    singleNode.coordinator(1)
+                              .execute(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, ?, ?)"), QUORUM, i, j, i + i);
+                }
+            }
+
+            int[] pageSizes = new int[]{ 1, 2, 3, 5, 10, 20, 50 };
+            String[] statements = new String[]{ withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 ORDER BY ck DESC LIMIT 3"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck >= 5 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT * FROM %s.tbl WHERE pk = 1 AND ck > 5 AND ck <= 10 ORDER BY ck DESC LIMIT 2"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl LIMIT 3"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10)"),
+                                                withKeyspace("SELECT DISTINCT pk FROM %s.tbl WHERE pk IN (3,5,8,10) LIMIT 2")
+            };
+            for (String statement : statements)
+            {
+                for (int pageSize : pageSizes)
+                {
+                    assertRows(cluster.coordinator(1).executeWithPaging(statement, QUORUM, pageSize),
+                               singleNode.coordinator(1).executeWithPaging(statement, QUORUM, Integer.MAX_VALUE));

Review comment:
       > non-paged response will be same across all page sizes.
   
   Good catch. This test has been moved with minimal changes from [`SimpleReadWriteTest#pagingTests`](https://github.com/apache/cassandra/blob/trunk/test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java#L276-L329) during a final refactor to give `SimpleReadWriteTest` the responsibilities of the Python dtest `TestPutGet`. I guess that it was done this way to use the version of `assertRows` that accepts row iterators, that would exhaust the iterator for the non-paged query if we reuse it. I have changed it to use the version of `assertRows` that accepts arrays so we can reuse the non-paged results.
   
   > There's no interplay between SELECT, paging, and range tombstones (plus reversed data types), but I think it's best we leave it to automated/fuzz tests.
   
   `ShortReadProtectionTest`, `ReadRepairQueryTypesTest` and `ReadRepairEmptyRangeTombstonesTest` include some interaction with paging and reversed types (that's how we found CASSANDRA-15459), but it's true that it would be great to have that more systematically covered with fuzz tests.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538803277



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)
+                                      .upgrade(Versions.Major.v30, Versions.Major.v4)
+                                      .upgrade(Versions.Major.v3X, Versions.Major.v4)
+                                      .withConfig(cfg -> cfg.with(Feature.NETWORK, Feature.GOSSIP)));
+    }
+
+    private void testConsistency(TestCase testCase) throws Throwable
+    {
+        List<ConsistencyTester> testers = new ArrayList<>();
+        testers.addAll(ConsistencyTester.create(1, ALL, THREE));
+        testers.addAll(ConsistencyTester.create(2, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO));
+        testers.addAll(ConsistencyTester.create(3, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO, ONE, LOCAL_ONE));
+
+        testCase.setup(cluster -> {
+            ConsistencyTester.createTable(cluster);
+            for (ConsistencyTester tester : testers)
+                tester.writeRows(cluster);
+        }).runAfterNodeAndClusterUpgrade(cluster -> {

Review comment:
       @adelapena If we're only upgrading one node, wouldn't using `runAfterNodeAndClusterUpgrade()` just mean we run the supplied lambda twice with the cluster in the same exact state? I'm wondering if we actually need `runAfterNodeAndClusterUpgrade()`. If we upgraded all nodes, `runAfterNodeUpgrade()` would already ensure we run when the cluster is fully upgraded, but if we don't update all the nodes, `runAfterClusterUpgrade()` still runs in a mixed cluster.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537671849



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;

Review comment:
       I initially missed that the page size was hard-coded to 1 in `assertRows()`. In any case, this looks good. I'd honestly be fine w/ fetch sizes of just 1 and 10, given the kinds of results we get here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522471632



##########
File path: src/java/org/apache/cassandra/service/reads/range/ReplicaPlanIterator.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.Pair;
+
+class ReplicaPlanIterator extends AbstractIterator<ReplicaPlan.ForRangeRead>
+{
+    private final Keyspace keyspace;
+    private final ConsistencyLevel consistency;
+    @VisibleForTesting
+    final Iterator<? extends AbstractBounds<PartitionPosition>> ranges;
+    private final int rangeCount;
+
+    ReplicaPlanIterator(AbstractBounds<PartitionPosition> keyRange, Keyspace keyspace, ConsistencyLevel consistency)
+    {
+        this.keyspace = keyspace;
+        this.consistency = consistency;
+
+        List<? extends AbstractBounds<PartitionPosition>> l = keyspace.getReplicationStrategy() instanceof LocalStrategy
+                                                              ? keyRange.unwrap()
+                                                              : getRestrictedRanges(keyRange);
+        this.ranges = l.iterator();
+        this.rangeCount = l.size();
+    }
+
+    int rangeCount()

Review comment:
       nit: Maybe we should also make this `size()` or `count()` `elements()` now that clients are using the "plan" terminology.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r539436846



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       There may also be a way to work around it here: https://github.com/apache/cassandra/blob/cassandra-3.11/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReadTest.java#L71-L78
   
   Thanks to @jrwest for the link.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540426178



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;

Review comment:
       @maedhroz I initially I added negative cases, but waiting for the read/write timeouts seemed to make the tests a bit unstable, at least locally. I think that for upgrade tests I'd prefer to live without negative tests and with a limited number of consistency levels, but I'm open to give it another go.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540442338



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;
+
+            UUID partitionKey = UUID.randomUUID();

Review comment:
       One single partition key per test case run, and each `AvailabiltyTester` can be run multiple times. `ONE-ALL` and `ALL-ONE` are run twice, and `QUORUM-QUORUM` is run four times.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536268284



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,460 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+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.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.Util.testPartitioner;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNode()
+    {
+        new TokenUpdater().withTokens(10, 20, 30, 40).update();
+
+        // with CLs requiring a single node all ranges are merged (unless they are wrapping)
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(min(), min()));
+            testRanges(cl, range(min(), max(25)));
+            testRanges(cl, range(min(), max(40)));
+            testRanges(cl, range(min(), max(50)));
+            testRanges(cl, range(max(20), max(30)));
+            testRanges(cl, range(max(25), min()));
+            testRanges(cl, range(max(25), max(35)));
+            testRanges(cl, range(max(50), min()));
+            testRanges(cl, range(max(40), max(10)), range(max(40), min()), range(min(), max(10))); // wrapped is split
+            testRanges(cl, range(max(25), max(15)), range(max(25), min()), range(min(), max(15))); // wrapped is split
+        }
+
+        // with CLs requiring more than a single node ranges are not merged due to the RF=2
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(min(), max(25)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(25)));
+            testRanges(cl,
+                       range(min(), max(40)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)));
+            testRanges(cl,
+                       range(min(), max(50)),
+                       range(min(), max(10)),
+                       range(max(10), max(20)),
+                       range(max(20), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), max(50)));
+            testRanges(cl,
+                       range(max(20), max(30)));
+            testRanges(cl,
+                       range(max(25), min()),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()));
+            testRanges(cl,
+                       range(max(50), min()));
+            testRanges(cl,
+                       range(max(30), max(10)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10))); // wrapped
+            testRanges(cl,
+                       range(max(25), max(15)),
+                       range(max(25), max(30)),
+                       range(max(30), max(40)),
+                       range(max(40), min()),
+                       range(min(), max(10)),
+                       range(max(10), max(15))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithContinuousRanges()
+    {
+        new TokenUpdater().withTokens("127.0.0.1", 10, 20, 30)
+                          .withTokens("127.0.0.2", 40, 50, 60)
+                          .withTokens("127.0.0.3", 70, 80, 90)
+                          .update();
+
+        // with CL=ANY the ranges are fully merged (unless they are wrapping)
+        testMultiNodeFullMerge(ANY);
+
+        // with CL=THREE the ranges are not merged at all
+        testMultiNodeNoMerge(THREE);
+
+        // with CLs requiring a single node the ranges are merged in a per-node basis
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE))
+        {
+            testRanges(cl,
+                       range(min(), min()),
+                       range(min(), max(60)),
+                       range(max(60), min()));

Review comment:
       nit: I know it's not always possible, but if the `expected` ranges fit on one line, that helps visually distinguish them from the query range (rather than having to remember that the first one is the query range).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522107443



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.exceptions.ReadFailureException;
+import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.metrics.ClientRequestMetrics;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.reads.DataResolver;
+import org.apache.cassandra.service.reads.ReadCallback;
+import org.apache.cassandra.service.reads.repair.ReadRepair;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.AbstractIterator;
+
+class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
+{
+    private static final Logger logger = LoggerFactory.getLogger(RangeCommandIterator.class);
+
+    private static final ClientRequestMetrics rangeMetrics = new ClientRequestMetrics("RangeSlice");
+
+    private final Iterator<ReplicaPlan.ForRangeRead> ranges;
+    private final int totalRangeCount;
+    private final PartitionRangeReadCommand command;
+    private final boolean enforceStrictLiveness;
+
+    private final long startTime;
+    private final long queryStartNanoTime;
+    private DataLimits.Counter counter;
+    private PartitionIterator sentQueryIterator;
+
+    private final int maxConcurrencyFactor;
+    private int concurrencyFactor;
+    // The two following "metric" are maintained to improve the concurrencyFactor
+    // when it was not good enough initially.
+    private int liveReturned;
+    private int rangesQueried;
+    private int batchesRequested = 0;
+
+    RangeCommandIterator(Iterator<ReplicaPlan.ForRangeRead> ranges,
+                         PartitionRangeReadCommand command,
+                         int concurrencyFactor,
+                         int maxConcurrencyFactor,
+                         int totalRangeCount,
+                         long queryStartNanoTime)
+    {
+        this.ranges = ranges;
+        this.command = command;
+        this.concurrencyFactor = concurrencyFactor;
+        this.maxConcurrencyFactor = maxConcurrencyFactor;
+        this.totalRangeCount = totalRangeCount;
+        this.queryStartNanoTime = queryStartNanoTime;
+
+        startTime = System.nanoTime();
+        enforceStrictLiveness = command.metadata().enforceStrictLiveness();
+    }
+
+    @Override
+    protected RowIterator computeNext()
+    {
+        try
+        {
+            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
+            {
+                // If we don't have more range to handle, we're done
+                if (!ranges.hasNext())
+                    return endOfData();
+
+                // else, sends the next batch of concurrent queries (after having close the previous iterator)
+                if (sentQueryIterator != null)
+                {
+                    liveReturned += counter.counted();
+                    sentQueryIterator.close();
+
+                    // It's not the first batch of queries and we're not done, so we we can use what has been
+                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
+                    updateConcurrencyFactor();
+                }
+                sentQueryIterator = sendNextRequests();
+            }
+
+            return sentQueryIterator.next();
+        }
+        catch (UnavailableException e)
+        {
+            rangeMetrics.unavailables.mark();
+            throw e;
+        }
+        catch (ReadTimeoutException e)
+        {
+            rangeMetrics.timeouts.mark();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            rangeMetrics.failures.mark();
+            throw e;
+        }
+    }
+
+    private void updateConcurrencyFactor()
+    {
+        liveReturned += counter.counted();
+
+        concurrencyFactor = computeConcurrencyFactor(totalRangeCount, rangesQueried, maxConcurrencyFactor, command.limits().count(), liveReturned);
+    }
+
+    @VisibleForTesting
+    static int computeConcurrencyFactor(int totalRangeCount, int rangesQueried, int maxConcurrencyFactor, int limit, int liveReturned)
+    {
+        maxConcurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, totalRangeCount - rangesQueried));
+        if (liveReturned == 0)
+        {
+            // we haven't actually gotten any results, so query up to the limit if not results so far
+            Tracing.trace("Didn't get any response rows; new concurrent requests: {}", maxConcurrencyFactor);
+            return maxConcurrencyFactor;
+        }
+
+        // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
+        // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
+        int remainingRows = limit - liveReturned;
+        float rowsPerRange = (float) liveReturned / (float) rangesQueried;
+        int concurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, Math.round(remainingRows / rowsPerRange)));
+        logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                     rowsPerRange, remainingRows, concurrencyFactor);
+        return concurrencyFactor;
+    }
+
+    /**
+     * Queries the provided sub-range.
+     *
+     * @param replicaPlan 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(ReplicaPlan.ForRangeRead replicaPlan, boolean isFirst)
+    {
+        PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaPlan.range(), isFirst);
+        // If enabled, request repaired data tracking info from full replicas but
+        // only if there are multiple full replicas to compare results from
+        if (DatabaseDescriptor.getRepairedDataTrackingForRangeReadsEnabled()
+            && replicaPlan.contacts().filter(Replica::isFull).size() > 1)
+        {
+            command.trackRepairedStatus();
+            rangeCommand.trackRepairedStatus();
+        }
+
+        ReplicaPlan.SharedForRangeRead sharedReplicaPlan = ReplicaPlan.shared(replicaPlan);
+        ReadRepair<EndpointsForRange, ReplicaPlan.ForRangeRead> readRepair
+        = ReadRepair.create(command, sharedReplicaPlan, queryStartNanoTime);
+        DataResolver<EndpointsForRange, ReplicaPlan.ForRangeRead> resolver
+        = new DataResolver<>(rangeCommand, sharedReplicaPlan, readRepair, queryStartNanoTime);
+        ReadCallback<EndpointsForRange, ReplicaPlan.ForRangeRead> handler
+        = new ReadCallback<>(resolver, rangeCommand, sharedReplicaPlan, queryStartNanoTime);
+
+
+        if (replicaPlan.contacts().size() == 1 && replicaPlan.contacts().get(0).isSelf())
+        {
+            Stage.READ.execute(new StorageProxy.LocalReadRunnable(rangeCommand, handler));
+        }
+        else
+        {
+            for (Replica replica : replicaPlan.contacts())
+            {
+                Tracing.trace("Enqueuing request to {}", replica);
+                ReadCommand command = replica.isFull() ? rangeCommand : rangeCommand.copyAsTransientQuery(replica);
+                Message<ReadCommand> message = command.createMessage(command.isTrackingRepairedStatus() && replica.isFull());
+                MessagingService.instance().sendWithCallback(message, replica.endpoint(), handler);
+            }
+        }
+
+        return new SingleRangeResponse(resolver, handler, readRepair);
+    }
+
+    private PartitionIterator sendNextRequests()
+    {
+        List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
+        List<ReadRepair<?, ?>> readRepairs = new ArrayList<>(concurrencyFactor);
+
+        try
+        {
+            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); )
+            {
+                ReplicaPlan.ForRangeRead range = ranges.next();
+
+                @SuppressWarnings("resource") // response will be closed by concatAndBlockOnRepair, or in the catch block below

Review comment:
       I'm afraid this is needed to avoid a failure in `ant eclipse-warnings`, which is run by CI. Indeed, I was missing another similar `@SuppressWarnings` annotation [here](https://github.com/apache/cassandra/pull/801/commits/f0562873ee676dabca467f2a2651a3a74b7d725d) that was producing a CI failure due to `ant eclipse-warnings`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r521562839



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeIterator.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.RingPosition;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.Pair;
+
+class RangeIterator extends AbstractIterator<ReplicaPlan.ForRangeRead>

Review comment:
       nit: I'd probably name this `RangePlanIterator` or `ReplicaPlanIterator` to avoid the conflict with the SASI/SAI `RangeIterator`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540186009



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));

Review comment:
       Do you mean a method or a class per starting version?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522111779



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeMerger.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.range;
+
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.utils.AbstractIterator;
+
+class RangeMerger extends AbstractIterator<ReplicaPlan.ForRangeRead>

Review comment:
       Done as mentioned above.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537644188



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()

Review comment:
       Yes, I used more descriptive name for some other tests but not for this one due to a lack of imagination. [Here](https://github.com/apache/cassandra/pull/801/commits/057756b50ed630bf6ce9ddd0f51e8ade58b172f8) I have used `testSkipEarlyTermination*` instead of `test13911*`, wdyt?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538824890



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));

Review comment:
       Trying to run this is generating some noise in the logs and OOMs for me...
   
   ```
   ERROR 21:27:38 LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@182e3d57) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@384421922:Memory@[7f9064b42190..7f9064b424b0) was not released before the reference was garbage collected
   ERROR [Reference-Reaper] node1 2020-12-08 15:27:38,081 Ref.java:228 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@182e3d57) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@384421922:Memory@[7f9064b42190..7f9064b424b0) was not released before the reference was garbage collected
   ERROR 21:27:45 LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@12d65627) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@1473567257:Memory@[7f9057a46640..7f9057a46644) was not released before the reference was garbage collected
   ERROR [Reference-Reaper] node1 2020-12-08 15:27:45,696 Ref.java:228 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@12d65627) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@1473567257:Memory@[7f9057a46640..7f9057a46644) was not released before the reference was garbage collected
   ERROR 21:27:47 LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@3ef99131) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@1724460556:Memory@[7f9057abde30..7f9057abde78) was not released before the reference was garbage collected
   ERROR [Reference-Reaper] node1 2020-12-08 15:27:47,828 Ref.java:228 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@3ef99131) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@1724460556:Memory@[7f9057abde30..7f9057abde78) was not released before the reference was garbage collected
   INFO  [node1_MemtableFlushWriter:1] node1 2020-12-08 15:27:53,538 LogTransaction.java:240 - Unfinished transaction log, deleting /private/var/folders/4d/zfjs7m7s6x5_l93k33r5k6680000gn/T/dtests8093906131697840956/node1/data2/system/compaction_history-b4dbb7b4dc493fb5b3bfce6e434832ca/na-1-big-Data.db 
   INFO  [node1_MemtableFlushWriter:1] node1 2020-12-08 15:27:54,042 LogTransaction.java:240 - Unfinished transaction log, deleting /private/var/folders/4d/zfjs7m7s6x5_l93k33r5k6680000gn/T/dtests8093906131697840956/node1/data2/system/compaction_history-b4dbb7b4dc493fb5b3bfce6e434832ca/na-1-big-Index.db 
   INFO  [node1_MemtableFlushWriter:1] node1 2020-12-08 15:27:56,281 LogTransaction.java:240 - Unfinished transaction log, deleting /private/var/folders/4d/zfjs7m7s6x5_l93k33r5k6680000gn/T/dtests8093906131697840956/node1/data2/system/compaction_history-b4dbb7b4dc493fb5b3bfce6e434832ca/na_txn_flush_2f723f70-399c-11eb-a318-6dfe4db8d7d4.log 
   ERROR 21:28:09 LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@460e3b9c) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@939205974:Memory@[7f91114aab70..7f91114aab74) was not released before the reference was garbage collected
   ERROR 21:28:03 Exception in thread Thread[node1_MemtableFlushWriter:1,5,node1]
   java.lang.OutOfMemoryError: Metaspace
   ERROR [Reference-Reaper] node1 2020-12-08 15:28:09,243 Ref.java:228 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@460e3b9c) to class org.apache.cassandra.io.util.SafeMemory$MemoryTidy@939205974:Memory@[7f91114aab70..7f91114aab74) was not released before the reference was garbage collected
   ERROR [node1_isolatedExecutor:2] node1 2020-12-08 15:28:03,445 CassandraDaemon.java:492 - Exception in thread Thread[node1_MemtableFlushWriter:1,5,node1]
   java.lang.OutOfMemoryError: Metaspace
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r574672374



##########
File path: test/distributed/org/apache/cassandra/distributed/test/SSTableSkippingReadTest.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.distributed.test;
+
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertEquals;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+
+public class SSTableSkippingReadTest extends TestBaseImpl
+{
+    @Test
+    public void skippedSSTableWithPartitionDeletionTest() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.create(2)))
+        {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck int, v int, PRIMARY KEY(pk, ck))"));
+            // insert a partition tombstone on node 1, the deletion timestamp should end up being the sstable's minTimestamp
+            cluster.get(1).executeInternal(withKeyspace("DELETE FROM %s.tbl USING TIMESTAMP 1 WHERE pk = 0"));
+            // and a row from a different partition, to provide the sstable's min/max clustering
+            cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl (pk, ck, v) VALUES (1, 1, 1) USING TIMESTAMP 2"));
+            cluster.get(1).flush(KEYSPACE);
+            // expect a single sstable, where minTimestamp equals the timestamp of the partition delete
+            cluster.get(1).runOnInstance(() -> {
+                Set<SSTableReader> sstables = Keyspace.open(KEYSPACE)
+                                                      .getColumnFamilyStore("tbl")
+                                                      .getLiveSSTables();

Review comment:
       Right, but since we still need the sstables to get the first (and unique) one to get its min timestamp right below, perhaps we can leave it as it is, unless there's a metric for the min timestamp too. wdyt?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522416589



##########
File path: test/unit/org/apache/cassandra/service/reads/range/RangeIteratorTest.java
##########
@@ -236,4 +181,71 @@ public void testGRRExactBounds() throws Throwable
         testGRRKeys(exBounds(rp(""), rp("")), range(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
         testGRRKeys(incExBounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
     }
+
+    /**
+     * Test {@link RangeIterator#getRestrictedRanges(AbstractBounds)} for tokens.
+     */
+    @SafeVarargs
+    private final void testGRR(AbstractBounds<Token> queryRange, AbstractBounds<Token>... expected)
+    {
+        // Testing for tokens
+        List<AbstractBounds<Token>> restricted = RangeIterator.getRestrictedRanges(queryRange);

Review comment:
       [Here](https://github.com/apache/cassandra/pull/801/commits/4ca3e0321a99af6d7457134fe5832bcb2c0621da) I have tried to remove the support for `AbstractBounds<Token>` in `getRestrictedRanges`, that simplifies both that function and the tests. I think that was a remainder from quite older versions.
   
   [Here](https://github.com/apache/cassandra/pull/801/commits/237df2321f26220dddb15a52d3a88cc6b5917b8e) I have modified the test to actually instantiate `ReplicaPlanIterator` and verify the created ranges, including the special case of local replication strategy. The only part that is not covered is the actual building of the `ReplicaPlan` for each range ([this line](https://github.com/apache/cassandra/blob/237df2321f26220dddb15a52d3a88cc6b5917b8e/src/java/org/apache/cassandra/service/reads/range/ReplicaPlanIterator.java#L73)), since I think that it would require to have the replica nodes running.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538865371



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       @adelapena I ran through the 3.0 -> 3.11 upgrade example. Once node 1 is upgraded, attempting a read from node 2 as a coordinator at ALL fails thusly: `Live nodes [/127.0.0.2, /127.0.0.3] do not satisfy ConsistencyLevel (3 required)`. Why doesn't node 2 think node 1 has rejoined the cluster?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536339441



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;

Review comment:
       @adelapena Do any of the tests actually use a fetch size that would cause more than a single page to be processed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r557730667



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PutGetTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRow;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Simple put/get tests using different types of query, paticularly when the data is spread across memory and multiple
+ * sstables and using different compressors. All available compressors are tested. Both ascending and descending
+ * clustering orders are tested. The read queries are run using every node as a coordinator, with and without paging.
+ */
+@RunWith(Parameterized.class)
+public class PutGetTest extends TestBaseImpl
+{
+    private static final int NUM_NODES = 4;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final String CREATE_TABLE = "CREATE TABLE %s(k int, c int, v int, PRIMARY KEY (k, c)) " +
+                                               "WITH CLUSTERING ORDER BY (c %s) " +
+                                               "AND COMPRESSION = { 'class': '%s' } " +
+                                               "AND READ_REPAIR = 'none'";
+    private static final String[] COMPRESSORS = new String[]{ NoopCompressor.class.getSimpleName(),
+                                                              LZ4Compressor.class.getSimpleName(),
+                                                              DeflateCompressor.class.getSimpleName(),
+                                                              SnappyCompressor.class.getSimpleName(),
+                                                              ZstdCompressor.class.getSimpleName() };
+
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    /**
+     * The sstable compressor to be used.
+     */
+    @Parameterized.Parameter
+    public String compressor;
+
+    /**
+     * Whether the clustering order is reverse.
+     */
+    @Parameterized.Parameter(1)
+    public boolean reverse;
+
+    private String tableName;
+
+    @Parameterized.Parameters(name = "{index}: compressor={0} reverse={1}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (String compressor : COMPRESSORS)
+            for (boolean reverse : BOOLEANS)
+                result.add(new Object[]{ compressor, reverse });
+        return result;
+    }
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build(NUM_NODES).start(), REPLICATION_FACTOR);
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void before()
+    {
+        // create the table
+        tableName = String.format("%s.t_%d", KEYSPACE, seq.getAndIncrement());
+        cluster.schemaChange(String.format(CREATE_TABLE, tableName, reverse ? "DESC" : "ASC", compressor));
+    }
+
+    @After
+    public void after()
+    {
+        cluster.schemaChange(format("DROP TABLE %s"));
+    }
+
+    /**
+     * Simple put/get on a single partition with a few rows, reading with a single partition query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_putget[_snappy|_deflate]().
+     */
+    @Test
+    public void testPartitionQuery()
+    {
+        int numRows = 10;
+
+        writeRows(1, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s WHERE k=?", 0);
+        Assert.assertEquals(numRows, rows.length);
+        for (int c = 0; c < numRows; c++)
+        {
+            validateRow(rows[c], numRows, 0, c);
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with a range query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_rangeputget().
+     */
+    @Test
+    public void testRangeQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s");
+        Assert.assertEquals(numPartitions * numRows, rows.length);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            for (int c = 0; c < numRows; c++)
+            {
+                Object[] row = rows[k * numRows + c];
+                validateRow(row, numRows, k, c);
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on a single partition with multiple rows, reading with slice queries.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_wide_row().
+     */
+    @Test
+    public void testSliceQuery()
+    {
+        int numRows = 100;
+
+        writeRows(1, numRows);
+
+        String query = "SELECT * FROM %s WHERE k=? AND c>=? AND c<?";
+        for (int sliceSize : Arrays.asList(10, 20, 100))
+        {
+            for (int c = 0; c < numRows; c = c + sliceSize)
+            {
+                Object[][] rows = readRows(query, 0, c, c + sliceSize);
+                Assert.assertEquals(sliceSize, rows.length);
+
+                for (int i = 0; i < sliceSize; i++)
+                {
+                    Object[] row = rows[i];
+                    validateRow(row, numRows, 0, c + i);
+                }
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with IN queries.
+     */
+    @Test
+    public void testInQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        String query = "SELECT * FROM %s WHERE k IN (?, ?)";
+        for (int k = 0; k < numPartitions; k += 2)
+        {
+            Object[][] rows = readRows(query, k, k + 1);
+            Assert.assertEquals(numRows * 2, rows.length);
+
+            for (int i = 0; i < 2; i++)
+            {
+                for (int c = 0; c < numRows; c++)
+                {
+                    Object[] row = rows[i * numRows + c];
+                    validateRow(row, numRows, k + i, c);
+                }
+            }
+        }
+    }
+
+    /**
+     * Writes {@code numPartitions} with {@code numRows} each, with overrides in different sstables and memtables.
+     */
+    private void writeRows(int numPartitions, int numRows)
+    {
+        String update = format("UPDATE %s SET v=? WHERE k=? AND c=?");
+        ICoordinator coordinator = cluster.coordinator(1);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            // insert all the partition rows in a single sstable
+            for (int c = 0; c < numRows; c++)
+                coordinator.execute(update, QUORUM, c, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows in a second sstable
+            for (int c = 0; c < numRows; c += 2)
+                coordinator.execute(update, QUORUM, c + numRows, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows only in memtable
+            for (int c = 0; c < numRows; c += 5)

Review comment:
       nit: It might be nice to make constants out of the skip intervals 2 and 5 (so at least we can reuse it below during validation).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536419927



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)
+                    cluster.get(node == NUM_NODES ? 1 : node + 1).executeInternal(formattedQuery);

Review comment:
       @adelapena This confused me a bit when I first saw it. (i.e. `onlyNode1()` sometimes actually doesn't write just to node 1.) The reason we have these helpers is to make the higher level tests more intelligible, but I'm afraid this is a bit misleading. Perhaps some simple renaming would be fine, something like `toNodeN()` and a bit of documentation? I'll think it over and see if I can come up with any other ideas...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522471632



##########
File path: src/java/org/apache/cassandra/service/reads/range/ReplicaPlanIterator.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.Pair;
+
+class ReplicaPlanIterator extends AbstractIterator<ReplicaPlan.ForRangeRead>
+{
+    private final Keyspace keyspace;
+    private final ConsistencyLevel consistency;
+    @VisibleForTesting
+    final Iterator<? extends AbstractBounds<PartitionPosition>> ranges;
+    private final int rangeCount;
+
+    ReplicaPlanIterator(AbstractBounds<PartitionPosition> keyRange, Keyspace keyspace, ConsistencyLevel consistency)
+    {
+        this.keyspace = keyspace;
+        this.consistency = consistency;
+
+        List<? extends AbstractBounds<PartitionPosition>> l = keyspace.getReplicationStrategy() instanceof LocalStrategy
+                                                              ? keyRange.unwrap()
+                                                              : getRestrictedRanges(keyRange);
+        this.ranges = l.iterator();
+        this.rangeCount = l.size();
+    }
+
+    int rangeCount()

Review comment:
       nit: Maybe we should also make this `size()` or `count()` or something now that clients are using the "plan" terminology.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536300403



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.range;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.rp;
+import static org.apache.cassandra.Util.token;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(rp(""), rp("")));
+            testRanges(cl, range(rp(""), rp("25")));
+            testRanges(cl, range(rp(""), rp("40")));
+            testRanges(cl, range(rp(""), rp("50")));
+            testRanges(cl, range(rp("20"), rp("30")));
+            testRanges(cl, range(rp("25"), rp("")));
+            testRanges(cl, range(rp("25"), rp("35")));
+            testRanges(cl, range(rp("50"), rp("")));
+            testRanges(cl, range(rp("40"), rp("10")), range(rp("40"), rp("")), range(rp(""), rp("10"))); // wrapped is split
+            testRanges(cl, range(rp("25"), rp("15")), range(rp("25"), rp("")), range(rp(""), rp("15"))); // wrapped is split
+        }
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that doesn't allow to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithNotMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(rp(""), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp(""), rp("25")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), rp("25")));
+            testRanges(cl,
+                       range(rp(""), rp("40")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), rp("40")));
+            testRanges(cl,
+                       range(rp(""), rp("50")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("50")));
+            testRanges(cl,
+                       range(rp("20"), rp("30")),
+                       range(rp("20"), endOf("20")),
+                       range(endOf("20"), rp("30")));
+            testRanges(cl,
+                       range(rp("25"), rp("")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp("50"), rp("")));
+            testRanges(cl,
+                       range(rp("30"), rp("10")),
+                       range(rp("30"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), rp("10"))); // wrapped
+            testRanges(cl,
+                       range(rp("25"), rp("15")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), rp("15"))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster and a read consistency level that allows to merge ranges,
+     * particularly when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithMergingConsistencyLevelContinuous() throws UnknownHostException
+    {
+        new TokenUpdater().withTokens(InetAddressAndPort.getByName("127.0.0.1"), "10", "20", "30")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.2"), "40", "50", "60")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.3"), "70", "80", "90")
+                          .update();

Review comment:
       @adelapena These changes look great. Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538865371



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       @adelapena I ran through the 3.0 -> 3.11 upgrade example. Once node 1 is upgraded, attempting a read from node 2 as a coordinator at ALL fails thusly: `Live nodes [/127.0.0.2, /127.0.0.3] do not satisfy ConsistencyLevel (3 required)`. Why doesn't node 2 think node 1 has rejoined the cluster?
   
   UPDATE: Sleeping for 5 seconds before the read seems to give node 2 enough time to see node 1 as part of the ring. There be a race somwhar...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r538887919



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       Yeah, the reason this works without `NETWORK/GOSSIP` is that the endpoint data is manually maintained there. I'm not 100% sure, but I don't know if we really need to test this with GOSSIP enabled at all. We're testing coordination behavior through upgrades, assuming gossip and other networking systems are functioning normally.
   
   @adelapena WDYT about just avoiding NETWORK and GOSSIP in this patch?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540414682



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;

Review comment:
       @adelapena What do you think about testing the negative cases here as well (i.e. just catching the errors and verifying them)? I don't imagine the verb filtering will suddenly break, but you know...paranoia...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536334421



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))
+              .assertRows("SELECT DISTINCT id FROM %s LIMIT 1", row(3));
+    }
+
+    /**
+     * Tests SRP when more than one row is missing.
+     * <p>
+     * See CASSANDRA-12872.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_12872()}.
+     */
+    @Test
+    public void testMultipleMissedRows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .allNodes(0, 4, i -> String.format("INSERT INTO %%s (pk, ck) VALUES (0, %d)", i))
+              .onlyNode1("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2, 3)",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 5)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 4)")
+              .assertRows("SELECT ck FROM %s WHERE pk = 0 LIMIT 2", row(0), row(4));

Review comment:
       It isn't consistent, but I've observed this failing on the parameterization `[quorum=false flush=true paging=true coordinator=2]`:
   
   ```
   java.lang.AssertionError: Expected: [[0],[4]]
   Actual: [[0],[3]]
   	at org.apache.cassandra.distributed.shared.AssertUtils.fail(AssertUtils.java:193)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertTrue(AssertUtils.java:181)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertRows(AssertUtils.java:70)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.assertRows(ShortReadProtectionTest.java:505)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.access$500(ShortReadProtectionTest.java:418)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest.testMultipleMissedRows(ShortReadProtectionTest.java:193)
   ```
   
   It's always when we coordinate from node 2, which didn't itself delete the row w/ `ck=3`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537668454



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()

Review comment:
       Those names work for me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536422019



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)

Review comment:
       What if we just kept the `consistencyLevel`, called it `readConsistency` (or similar), and got rid of the `quorum` field? (We could just check read CL here instead of needing `quorum`.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537812108



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       @maedhroz I don't know why using these versions make the test fail with an `UnavailableException` on reads.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540433194



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;

Review comment:
       That would be a long wait, especially given we've increased the timeouts above. Would simply shutting down the affected nodes result in quicker failures?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540435048



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.exceptions.ReadFailureException;
+import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.metrics.ClientRequestMetrics;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.reads.DataResolver;
+import org.apache.cassandra.service.reads.ReadCallback;
+import org.apache.cassandra.service.reads.repair.ReadRepair;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
+{
+    private static final Logger logger = LoggerFactory.getLogger(RangeCommandIterator.class);
+
+    private static final ClientRequestMetrics rangeMetrics = new ClientRequestMetrics("RangeSlice");
+
+    private final CloseableIterator<ReplicaPlan.ForRangeRead> replicaPlans;
+    private final int totalRangeCount;
+    private final PartitionRangeReadCommand command;
+    private final boolean enforceStrictLiveness;
+
+    private final long startTime;
+    private final long queryStartNanoTime;
+    private DataLimits.Counter counter;
+    private PartitionIterator sentQueryIterator;
+
+    private final int maxConcurrencyFactor;
+    private int concurrencyFactor;
+    // The two following "metric" are maintained to improve the concurrencyFactor
+    // when it was not good enough initially.
+    private int liveReturned;
+    private int rangesQueried;
+    private int batchesRequested = 0;
+
+    RangeCommandIterator(CloseableIterator<ReplicaPlan.ForRangeRead> replicaPlans,
+                         PartitionRangeReadCommand command,
+                         int concurrencyFactor,
+                         int maxConcurrencyFactor,
+                         int totalRangeCount,
+                         long queryStartNanoTime)
+    {
+        this.replicaPlans = replicaPlans;
+        this.command = command;
+        this.concurrencyFactor = concurrencyFactor;
+        this.maxConcurrencyFactor = maxConcurrencyFactor;
+        this.totalRangeCount = totalRangeCount;
+        this.queryStartNanoTime = queryStartNanoTime;
+
+        startTime = System.nanoTime();
+        enforceStrictLiveness = command.metadata().enforceStrictLiveness();
+    }
+
+    @Override
+    protected RowIterator computeNext()
+    {
+        try
+        {
+            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
+            {
+                // If we don't have more range to handle, we're done
+                if (!replicaPlans.hasNext())
+                    return endOfData();
+
+                // else, sends the next batch of concurrent queries (after having close the previous iterator)
+                if (sentQueryIterator != null)
+                {
+                    liveReturned += counter.counted();
+                    sentQueryIterator.close();
+
+                    // It's not the first batch of queries and we're not done, so we we can use what has been
+                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
+                    updateConcurrencyFactor();
+                }
+                sentQueryIterator = sendNextRequests();
+            }
+
+            return sentQueryIterator.next();
+        }
+        catch (UnavailableException e)
+        {
+            rangeMetrics.unavailables.mark();
+            throw e;
+        }
+        catch (ReadTimeoutException e)
+        {
+            rangeMetrics.timeouts.mark();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            rangeMetrics.failures.mark();
+            throw e;
+        }
+    }
+
+    private void updateConcurrencyFactor()
+    {
+        liveReturned += counter.counted();
+
+        concurrencyFactor = computeConcurrencyFactor(totalRangeCount, rangesQueried, maxConcurrencyFactor, command.limits().count(), liveReturned);
+    }
+
+    @VisibleForTesting
+    static int computeConcurrencyFactor(int totalRangeCount, int rangesQueried, int maxConcurrencyFactor, int limit, int liveReturned)
+    {
+        maxConcurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, totalRangeCount - rangesQueried));
+        if (liveReturned == 0)
+        {
+            // we haven't actually gotten any results, so query up to the limit if not results so far
+            Tracing.trace("Didn't get any response rows; new concurrent requests: {}", maxConcurrencyFactor);
+            return maxConcurrencyFactor;
+        }
+
+        // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
+        // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
+        int remainingRows = limit - liveReturned;
+        float rowsPerRange = (float) liveReturned / (float) rangesQueried;
+        int concurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, Math.round(remainingRows / rowsPerRange)));
+        logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                     rowsPerRange, remainingRows, concurrencyFactor);
+        return concurrencyFactor;
+    }
+
+    /**
+     * Queries the provided sub-range.
+     *
+     * @param replicaPlan 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(ReplicaPlan.ForRangeRead replicaPlan, boolean isFirst)
+    {
+        PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaPlan.range(), isFirst);
+        // If enabled, request repaired data tracking info from full replicas but
+        // only if there are multiple full replicas to compare results from
+        if (DatabaseDescriptor.getRepairedDataTrackingForRangeReadsEnabled()
+            && replicaPlan.contacts().filter(Replica::isFull).size() > 1)
+        {
+            command.trackRepairedStatus();
+            rangeCommand.trackRepairedStatus();
+        }
+
+        ReplicaPlan.SharedForRangeRead sharedReplicaPlan = ReplicaPlan.shared(replicaPlan);
+        ReadRepair<EndpointsForRange, ReplicaPlan.ForRangeRead> readRepair
+                = ReadRepair.create(command, sharedReplicaPlan, queryStartNanoTime);

Review comment:
       Changed [here](https://github.com/apache/cassandra/pull/801/commits/1fa21b90296318c283d6edb7288e7523f7809f93), I was using [the original](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageProxy.java#L2306-L2311) but less common style.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537658125



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)
+                    cluster.get(node == NUM_NODES ? 1 : node + 1).executeInternal(formattedQuery);

Review comment:
       I think all writes are already internal. [Here](https://github.com/apache/cassandra/pull/801/commits/5b74a26df13515b9d1f6e6cb6ade05b7f8533255) I have renamed the methods and added some documentation in an attempt to make the relationship between writes and reads clearer, please let me know what do you think.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r522483578



##########
File path: test/unit/org/apache/cassandra/service/reads/range/RangeIteratorTest.java
##########
@@ -236,4 +181,71 @@ public void testGRRExactBounds() throws Throwable
         testGRRKeys(exBounds(rp(""), rp("")), range(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
         testGRRKeys(incExBounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
     }
+
+    /**
+     * Test {@link RangeIterator#getRestrictedRanges(AbstractBounds)} for tokens.
+     */
+    @SafeVarargs
+    private final void testGRR(AbstractBounds<Token> queryRange, AbstractBounds<Token>... expected)
+    {
+        // Testing for tokens
+        List<AbstractBounds<Token>> restricted = RangeIterator.getRestrictedRanges(queryRange);

Review comment:
       @adelapena I've read through the changes, and they all look good so far.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r542465769



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;

Review comment:
       A workaround for shutting down the nodes is, given that it seems we can't bring them up again, is to run two separate upgrade tests. One of them uses the upgraded node as coordinator and progressively shuts down the other two nodes, and the other uses one of the not upgraded nodes as coordinator and progressively shuts down the other two nodes. Done [here](https://github.com/apache/cassandra/pull/801/commits/e9d9fb5c91abe8a49ac10dba8b196d889dff3db6).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena closed pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena closed pull request #801:
URL: https://github.com/apache/cassandra/pull/801


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r521568209



##########
File path: test/unit/org/apache/cassandra/service/reads/range/RangeIteratorTest.java
##########
@@ -236,4 +181,71 @@ public void testGRRExactBounds() throws Throwable
         testGRRKeys(exBounds(rp(""), rp("")), range(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
         testGRRKeys(incExBounds(rp(""), rp("")), bounds(rp(""), endOf("1")), range(endOf("1"), endOf("6")), exBounds(endOf("6"), rp("")));
     }
+
+    /**
+     * Test {@link RangeIterator#getRestrictedRanges(AbstractBounds)} for tokens.
+     */
+    @SafeVarargs
+    private final void testGRR(AbstractBounds<Token> queryRange, AbstractBounds<Token>... expected)
+    {
+        // Testing for tokens
+        List<AbstractBounds<Token>> restricted = RangeIterator.getRestrictedRanges(queryRange);

Review comment:
       It's kind of funny that `RangeIteratorTest` never actually creates a `RangeIterator` instance. It might make sense to change that and test the public contract of the class rather than reach into `getRestrictedRanges()`. (There is a little bit of conditional logic and iterator logic we miss too, of course.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540179546



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)

Review comment:
       Unfortunately those methods are gone in trunk. Probably we don't really need gossip on these tests, so I have left the original `UpgradeTest#simpleUpgradeWithNetworkAndGossipTest` test as it was.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r542461364



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));

Review comment:
       Done [here](https://github.com/apache/cassandra/pull/801/commits/4e414b982534e31e549fba80d86fbb290b644bbe).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540425867



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase
+{
+    @Test
+    public void testAvailability() throws Throwable
+    {
+        testAvailability(new AvailabiltyTester(ONE, ALL),
+                         new AvailabiltyTester(QUORUM, QUORUM),
+                         new AvailabiltyTester(ALL, ONE));
+    }
+
+    public void testAvailability(AvailabiltyTester... testers) throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30))
+                                    .set("write_request_timeout_in_ms", SECONDS.toMillis(30)))
+        .setup(c -> c.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k uuid, c int, v int, PRIMARY KEY (k, c))")))
+        .runAfterNodeUpgrade((cluster, n) -> {
+
+            // using an upgraded and a not upgraded coordinator...
+            for (int node = 1; node < cluster.size(); node++)
+            {
+                ICoordinator coordinator = cluster.coordinator(node);
+                int nodeDown = node;
+
+                // using 0 to 2 down nodes...
+                for (int numNodesDown = 0; numNodesDown < cluster.size(); numNodesDown++)
+                {
+                    // disable communications to the down nodes
+                    if (numNodesDown > 0)
+                    {
+                        nodeDown = nextNode(nodeDown, cluster.size());
+                        cluster.filters().verbs(READ_REQ.id, MUTATION_REQ.id).to(nodeDown).drop();
+                    }
+
+                    // run the test cases that are compatible with the number of down nodes
+                    for (AvailabiltyTester tester : testers)
+                    {
+                        tester.test(coordinator, numNodesDown);
+                    }
+                }
+
+                // restore communication to all nodes
+                cluster.filters().reset();
+            }
+        }).run();
+    }
+
+    private static int nextNode(int node, int numNodes)
+    {
+        return node == numNodes ? 1 : node + 1;
+    }
+
+    private static class AvailabiltyTester
+    {
+        private static final String INSERT = withKeyspace("INSERT INTO %s.tbl (k, c, v) VALUES (?, ?, ?)");
+        private static final String SELECT = withKeyspace("SELECT * FROM %s.tbl WHERE k = ?");
+
+        private final ConsistencyLevel writeConsistencyLevel;
+        private final ConsistencyLevel readConsistencyLevel;
+
+        private AvailabiltyTester(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel)
+        {
+            this.writeConsistencyLevel = writeConsistencyLevel;
+            this.readConsistencyLevel = readConsistencyLevel;
+        }
+
+        public void test(ICoordinator coordinator, int numNodesDown)
+        {
+            if (numNodesDown > maxNodesDown(writeConsistencyLevel) ||
+                numNodesDown > maxNodesDown(readConsistencyLevel))
+                return;
+
+            UUID partitionKey = UUID.randomUUID();

Review comment:
       @adelapena We're testing with a single key here, given RF=3 anyway?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540363643



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTest.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.distributed.upgrade;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.apache.cassandra.net.Verb.MUTATION_REQ;
+import static org.apache.cassandra.net.Verb.READ_REQ;
+
+public class MixedModeAvailabilityTest extends UpgradeTestBase

Review comment:
       @dcapwell Another new upgrade test here, FYI




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537641173



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))
+              .assertRows("SELECT DISTINCT id FROM %s LIMIT 1", row(3));
+    }
+
+    /**
+     * Tests SRP when more than one row is missing.
+     * <p>
+     * See CASSANDRA-12872.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_12872()}.
+     */
+    @Test
+    public void testMultipleMissedRows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .allNodes(0, 4, i -> String.format("INSERT INTO %%s (pk, ck) VALUES (0, %d)", i))
+              .onlyNode1("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2, 3)",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 5)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 4)")
+              .assertRows("SELECT ck FROM %s WHERE pk = 0 LIMIT 2", row(0), row(4));

Review comment:
       I think this is due to the lack of specific timestamps on the writes done by the test. If we use the automatic ones there is a chance they end up with the same timestamp messing up with reconciliation. Some of the original dtests were specifying that timestamp, while others were missing it. This was probably not a problem is Python but can be problematic with their faster Java version. Fixed [here](https://github.com/apache/cassandra/pull/801/commits/38d41c3e4e543731762ab121aa4bdb4fd4d428f5).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r521561723



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeCommandIterator.java
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.range;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.exceptions.ReadFailureException;
+import org.apache.cassandra.exceptions.ReadTimeoutException;
+import org.apache.cassandra.exceptions.UnavailableException;
+import org.apache.cassandra.locator.EndpointsForRange;
+import org.apache.cassandra.locator.Replica;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.metrics.ClientRequestMetrics;
+import org.apache.cassandra.net.Message;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.reads.DataResolver;
+import org.apache.cassandra.service.reads.ReadCallback;
+import org.apache.cassandra.service.reads.repair.ReadRepair;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.AbstractIterator;
+
+class RangeCommandIterator extends AbstractIterator<RowIterator> implements PartitionIterator
+{
+    private static final Logger logger = LoggerFactory.getLogger(RangeCommandIterator.class);
+
+    private static final ClientRequestMetrics rangeMetrics = new ClientRequestMetrics("RangeSlice");
+
+    private final Iterator<ReplicaPlan.ForRangeRead> ranges;
+    private final int totalRangeCount;
+    private final PartitionRangeReadCommand command;
+    private final boolean enforceStrictLiveness;
+
+    private final long startTime;
+    private final long queryStartNanoTime;
+    private DataLimits.Counter counter;
+    private PartitionIterator sentQueryIterator;
+
+    private final int maxConcurrencyFactor;
+    private int concurrencyFactor;
+    // The two following "metric" are maintained to improve the concurrencyFactor
+    // when it was not good enough initially.
+    private int liveReturned;
+    private int rangesQueried;
+    private int batchesRequested = 0;
+
+    RangeCommandIterator(Iterator<ReplicaPlan.ForRangeRead> ranges,
+                         PartitionRangeReadCommand command,
+                         int concurrencyFactor,
+                         int maxConcurrencyFactor,
+                         int totalRangeCount,
+                         long queryStartNanoTime)
+    {
+        this.ranges = ranges;
+        this.command = command;
+        this.concurrencyFactor = concurrencyFactor;
+        this.maxConcurrencyFactor = maxConcurrencyFactor;
+        this.totalRangeCount = totalRangeCount;
+        this.queryStartNanoTime = queryStartNanoTime;
+
+        startTime = System.nanoTime();
+        enforceStrictLiveness = command.metadata().enforceStrictLiveness();
+    }
+
+    @Override
+    protected RowIterator computeNext()
+    {
+        try
+        {
+            while (sentQueryIterator == null || !sentQueryIterator.hasNext())
+            {
+                // If we don't have more range to handle, we're done
+                if (!ranges.hasNext())
+                    return endOfData();
+
+                // else, sends the next batch of concurrent queries (after having close the previous iterator)
+                if (sentQueryIterator != null)
+                {
+                    liveReturned += counter.counted();
+                    sentQueryIterator.close();
+
+                    // It's not the first batch of queries and we're not done, so we we can use what has been
+                    // returned so far to improve our rows-per-range estimate and update the concurrency accordingly
+                    updateConcurrencyFactor();
+                }
+                sentQueryIterator = sendNextRequests();
+            }
+
+            return sentQueryIterator.next();
+        }
+        catch (UnavailableException e)
+        {
+            rangeMetrics.unavailables.mark();
+            throw e;
+        }
+        catch (ReadTimeoutException e)
+        {
+            rangeMetrics.timeouts.mark();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            rangeMetrics.failures.mark();
+            throw e;
+        }
+    }
+
+    private void updateConcurrencyFactor()
+    {
+        liveReturned += counter.counted();
+
+        concurrencyFactor = computeConcurrencyFactor(totalRangeCount, rangesQueried, maxConcurrencyFactor, command.limits().count(), liveReturned);
+    }
+
+    @VisibleForTesting
+    static int computeConcurrencyFactor(int totalRangeCount, int rangesQueried, int maxConcurrencyFactor, int limit, int liveReturned)
+    {
+        maxConcurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, totalRangeCount - rangesQueried));
+        if (liveReturned == 0)
+        {
+            // we haven't actually gotten any results, so query up to the limit if not results so far
+            Tracing.trace("Didn't get any response rows; new concurrent requests: {}", maxConcurrencyFactor);
+            return maxConcurrencyFactor;
+        }
+
+        // Otherwise, compute how many rows per range we got on average and pick a concurrency factor
+        // that should allow us to fetch all remaining rows with the next batch of (concurrent) queries.
+        int remainingRows = limit - liveReturned;
+        float rowsPerRange = (float) liveReturned / (float) rangesQueried;
+        int concurrencyFactor = Math.max(1, Math.min(maxConcurrencyFactor, Math.round(remainingRows / rowsPerRange)));
+        logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                     rowsPerRange, remainingRows, concurrencyFactor);
+        return concurrencyFactor;
+    }
+
+    /**
+     * Queries the provided sub-range.
+     *
+     * @param replicaPlan 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(ReplicaPlan.ForRangeRead replicaPlan, boolean isFirst)
+    {
+        PartitionRangeReadCommand rangeCommand = command.forSubRange(replicaPlan.range(), isFirst);
+        // If enabled, request repaired data tracking info from full replicas but
+        // only if there are multiple full replicas to compare results from
+        if (DatabaseDescriptor.getRepairedDataTrackingForRangeReadsEnabled()
+            && replicaPlan.contacts().filter(Replica::isFull).size() > 1)
+        {
+            command.trackRepairedStatus();
+            rangeCommand.trackRepairedStatus();
+        }
+
+        ReplicaPlan.SharedForRangeRead sharedReplicaPlan = ReplicaPlan.shared(replicaPlan);
+        ReadRepair<EndpointsForRange, ReplicaPlan.ForRangeRead> readRepair
+        = ReadRepair.create(command, sharedReplicaPlan, queryStartNanoTime);
+        DataResolver<EndpointsForRange, ReplicaPlan.ForRangeRead> resolver
+        = new DataResolver<>(rangeCommand, sharedReplicaPlan, readRepair, queryStartNanoTime);
+        ReadCallback<EndpointsForRange, ReplicaPlan.ForRangeRead> handler
+        = new ReadCallback<>(resolver, rangeCommand, sharedReplicaPlan, queryStartNanoTime);
+
+
+        if (replicaPlan.contacts().size() == 1 && replicaPlan.contacts().get(0).isSelf())
+        {
+            Stage.READ.execute(new StorageProxy.LocalReadRunnable(rangeCommand, handler));
+        }
+        else
+        {
+            for (Replica replica : replicaPlan.contacts())
+            {
+                Tracing.trace("Enqueuing request to {}", replica);
+                ReadCommand command = replica.isFull() ? rangeCommand : rangeCommand.copyAsTransientQuery(replica);
+                Message<ReadCommand> message = command.createMessage(command.isTrackingRepairedStatus() && replica.isFull());
+                MessagingService.instance().sendWithCallback(message, replica.endpoint(), handler);
+            }
+        }
+
+        return new SingleRangeResponse(resolver, handler, readRepair);
+    }
+
+    private PartitionIterator sendNextRequests()
+    {
+        List<PartitionIterator> concurrentQueries = new ArrayList<>(concurrencyFactor);
+        List<ReadRepair<?, ?>> readRepairs = new ArrayList<>(concurrencyFactor);
+
+        try
+        {
+            for (int i = 0; i < concurrencyFactor && ranges.hasNext(); )
+            {
+                ReplicaPlan.ForRangeRead range = ranges.next();
+
+                @SuppressWarnings("resource") // response will be closed by concatAndBlockOnRepair, or in the catch block below

Review comment:
       I think this suppression is unnecessary now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r557725616



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PutGetTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRow;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Simple put/get tests using different types of query, paticularly when the data is spread across memory and multiple
+ * sstables and using different compressors. All available compressors are tested. Both ascending and descending
+ * clustering orders are tested. The read queries are run using every node as a coordinator, with and without paging.
+ */
+@RunWith(Parameterized.class)
+public class PutGetTest extends TestBaseImpl
+{
+    private static final int NUM_NODES = 4;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final String CREATE_TABLE = "CREATE TABLE %s(k int, c int, v int, PRIMARY KEY (k, c)) " +
+                                               "WITH CLUSTERING ORDER BY (c %s) " +
+                                               "AND COMPRESSION = { 'class': '%s' } " +
+                                               "AND READ_REPAIR = 'none'";
+    private static final String[] COMPRESSORS = new String[]{ NoopCompressor.class.getSimpleName(),
+                                                              LZ4Compressor.class.getSimpleName(),
+                                                              DeflateCompressor.class.getSimpleName(),
+                                                              SnappyCompressor.class.getSimpleName(),
+                                                              ZstdCompressor.class.getSimpleName() };
+
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    /**
+     * The sstable compressor to be used.
+     */
+    @Parameterized.Parameter
+    public String compressor;
+
+    /**
+     * Whether the clustering order is reverse.
+     */
+    @Parameterized.Parameter(1)
+    public boolean reverse;
+
+    private String tableName;
+
+    @Parameterized.Parameters(name = "{index}: compressor={0} reverse={1}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (String compressor : COMPRESSORS)
+            for (boolean reverse : BOOLEANS)
+                result.add(new Object[]{ compressor, reverse });
+        return result;
+    }
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build(NUM_NODES).start(), REPLICATION_FACTOR);
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void before()
+    {
+        // create the table
+        tableName = String.format("%s.t_%d", KEYSPACE, seq.getAndIncrement());
+        cluster.schemaChange(String.format(CREATE_TABLE, tableName, reverse ? "DESC" : "ASC", compressor));
+    }
+
+    @After
+    public void after()
+    {
+        cluster.schemaChange(format("DROP TABLE %s"));
+    }
+
+    /**
+     * Simple put/get on a single partition with a few rows, reading with a single partition query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_putget[_snappy|_deflate]().
+     */
+    @Test
+    public void testPartitionQuery()
+    {
+        int numRows = 10;
+
+        writeRows(1, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s WHERE k=?", 0);
+        Assert.assertEquals(numRows, rows.length);
+        for (int c = 0; c < numRows; c++)
+        {
+            validateRow(rows[c], numRows, 0, c);
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with a range query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_rangeputget().
+     */
+    @Test
+    public void testRangeQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s");
+        Assert.assertEquals(numPartitions * numRows, rows.length);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            for (int c = 0; c < numRows; c++)
+            {
+                Object[] row = rows[k * numRows + c];
+                validateRow(row, numRows, k, c);
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on a single partition with multiple rows, reading with slice queries.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_wide_row().
+     */
+    @Test
+    public void testSliceQuery()
+    {
+        int numRows = 100;
+
+        writeRows(1, numRows);
+
+        String query = "SELECT * FROM %s WHERE k=? AND c>=? AND c<?";
+        for (int sliceSize : Arrays.asList(10, 20, 100))
+        {
+            for (int c = 0; c < numRows; c = c + sliceSize)
+            {
+                Object[][] rows = readRows(query, 0, c, c + sliceSize);
+                Assert.assertEquals(sliceSize, rows.length);
+
+                for (int i = 0; i < sliceSize; i++)
+                {
+                    Object[] row = rows[i];
+                    validateRow(row, numRows, 0, c + i);
+                }
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with IN queries.
+     */
+    @Test
+    public void testInQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        String query = "SELECT * FROM %s WHERE k IN (?, ?)";
+        for (int k = 0; k < numPartitions; k += 2)
+        {
+            Object[][] rows = readRows(query, k, k + 1);
+            Assert.assertEquals(numRows * 2, rows.length);
+
+            for (int i = 0; i < 2; i++)
+            {
+                for (int c = 0; c < numRows; c++)
+                {
+                    Object[] row = rows[i * numRows + c];
+                    validateRow(row, numRows, k + i, c);
+                }
+            }
+        }
+    }
+
+    /**
+     * Writes {@code numPartitions} with {@code numRows} each, with overrides in different sstables and memtables.
+     */
+    private void writeRows(int numPartitions, int numRows)
+    {
+        String update = format("UPDATE %s SET v=? WHERE k=? AND c=?");
+        ICoordinator coordinator = cluster.coordinator(1);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            // insert all the partition rows in a single sstable
+            for (int c = 0; c < numRows; c++)
+                coordinator.execute(update, QUORUM, c, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows in a second sstable
+            for (int c = 0; c < numRows; c += 2)
+                coordinator.execute(update, QUORUM, c + numRows, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows only in memtable
+            for (int c = 0; c < numRows; c += 5)
+                coordinator.execute(update, QUORUM, c + numRows * 2, k, c);
+        }
+    }
+
+    /**
+     * Runs the specified query in all coordinators, with and without paging.
+     */
+    private Object[][] readRows(String query, Object... boundValues)
+    {
+        query = format(query);
+        Object[][] rows = null;
+
+        // verify that all coordinators return the same results for the query, regardless of paging
+        for (ICoordinator coordinator : cluster.coordinators())
+        {
+            for (boolean paging : BOOLEANS)
+            {
+                Object[][] readRows = paging
+                                      ? Iterators.toArray(coordinator.executeWithPaging(query, QUORUM, 1, boundValues),
+                                                          Object[].class)
+                                      : coordinator.execute(query, QUORUM, boundValues);
+                if (rows == null)
+                    rows = readRows;
+                else
+                    assertRows(rows, readRows);
+            }
+        }
+        Assert.assertNotNull(rows);
+
+        // undo the clustering reverse sorting to ease validation
+        if (reverse)
+            ArrayUtils.reverse(rows);
+
+        // sort by partition key to ease validation
+        Arrays.sort(rows, Comparator.comparing(row -> (int) row[0]));
+
+        return rows;
+    }
+
+    private void validateRow(Object[] row, int numRows, int k, int c)
+    {
+        Assert.assertNotNull(row);
+
+        if (c % 5 == 0)
+            assertRow(row, row(k, c, c + numRows * 2));
+        else if (c % 2 == 0)
+            assertRow(row, row(k, c, c + numRows));
+        else
+            assertRow(row, row(k, c, c));
+    }
+
+    private String format(String query)

Review comment:
       nit: Perhaps we could call this `withTable()` or something similar?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537660979



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)

Review comment:
       Done. I was using a boolean trying to express that we only use between two read-write consistency models (ONE-ALL and QUORUM-QUORUM), but that way is probably clearer.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536335965



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))

Review comment:
       It isn't consistent, but I've observed this failing on the parameterization `[quorum=false flush=false paging=true coordinator=2]`:
   
   ```
   java.lang.AssertionError: Expected: [[3]]
   Actual: [[9]]
   	at org.apache.cassandra.distributed.shared.AssertUtils.fail(AssertUtils.java:193)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertTrue(AssertUtils.java:181)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertRows(AssertUtils.java:70)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.assertRows(ShortReadProtectionTest.java:505)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.access$500(ShortReadProtectionTest.java:418)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest.testSkinnyTableWithComplementaryDeletions(ShortReadProtectionTest.java:174)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   ```
   
   I'm guessing it's because partition 9 isn't deleted from node 2, and node 2 as a coordinator somehow doesn't filter out partition 9 from node 1's deletion...interesting.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r537658125



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -66,51 +329,192 @@ public void testGroupBySRPRegularRow() throws Throwable
      * See CASSANDRA-15459
      */
     @Test
-    public void testGroupBySRPStaticRow() throws Throwable
-    {
-        testGroupBySRP("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))",
-                       asList("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (0, null)",
-                              "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0"),
-                       asList("INSERT INTO %s (pk, s) VALUES (1, null)",
-                              "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
-                              "INSERT INTO %s (pk, s) VALUES (2, null)"),
-                       asList("SELECT * FROM %s LIMIT 1",
-                              "SELECT * FROM %s LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk LIMIT 10",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 1",
-                              "SELECT * FROM %s GROUP BY pk, ck LIMIT 10"));
-    }
-
-    private void testGroupBySRP(String createTable,
-                                List<String> node1Queries,
-                                List<String> node2Queries,
-                                List<String> coordinatorQueries) throws Throwable
-    {
-        try (Cluster cluster = init(Cluster.build()
-                                           .withNodes(2)
-                                           .withConfig(config -> config.set("hinted_handoff_enabled", false))
-                                           .start()))
+    public void testGroupByStaticRow()
+    {
+        Assume.assumeFalse(paging); // paging fails due to CASSANDRA-16307
+        tester.createTable("CREATE TABLE %s (pk int, ck int, s int static, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, s) VALUES (1, 1) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (0, null)",
+                         "INSERT INTO %s (pk, s) VALUES (2, 2) USING TIMESTAMP 0")
+              .onlyNode2("INSERT INTO %s (pk, s) VALUES (1, null)",
+                         "INSERT INTO %s (pk, s) VALUES (0, 0) USING TIMESTAMP 0",
+                         "INSERT INTO %s (pk, s) VALUES (2, null)")
+              .assertRows("SELECT * FROM %s LIMIT 1")
+              .assertRows("SELECT * FROM %s LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk LIMIT 10")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 1")
+              .assertRows("SELECT * FROM %s GROUP BY pk, ck LIMIT 10");
+    }
+
+    /**
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911()}.
+     */
+    @Test
+    public void test13911()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0)")
+              .onlyNode2("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2)")
+              .assertRows("SELECT DISTINCT pk FROM %s", row(0));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDoneForPartition()} to
+     * abort single partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_rows_srp()}.
+     */
+    @Test
+    public void test13911rows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 1")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck = 0",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 2) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 3", row(0, 0), row(0, 1), row(2, 2));
+    }
+
+    /**
+     * A regression test to prove that we can no longer rely on {@code !singleResultCounter.isDone()} to abort ranged
+     * partition SRP early if a per partition limit is set.
+     * <p>
+     * See CASSANDRA-13911.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13911_partitions_srp()}.
+     */
+    @Test
+    public void test13911partitions()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .onlyNode1("INSERT INTO %s (pk, ck) VALUES (0, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 1) USING TIMESTAMP 42",
+                         "DELETE FROM %s USING TIMESTAMP 42 WHERE pk = 2 AND ck IN  (0, 1)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 2) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 3) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (2, 1) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 0) USING TIMESTAMP 42",
+                         "INSERT INTO %s (pk, ck) VALUES (4, 1) USING TIMESTAMP 42")
+              .assertRows("SELECT pk, ck FROM %s PER PARTITION LIMIT 2 LIMIT 4",
+                          row(0, 0), row(0, 1), row(4, 0), row(4, 1));
+    }
+
+    private static long token(int key)
+    {
+        return (long) Murmur3Partitioner.instance.getToken(ByteBufferUtil.bytes(key)).getTokenValue();
+    }
+
+    private static class Tester
+    {
+        private static final AtomicInteger seqNumber = new AtomicInteger();
+
+        private final ICoordinator coordinator;
+        private final boolean quorum, flush, paging;
+        private final String qualifiedTableName;
+        private final ConsistencyLevel consistencyLevel;
+
+        private boolean flushed = false;
+
+        private Tester(boolean quorum, boolean flush, boolean paging, int coordinator)
+        {
+            this.coordinator = cluster.coordinator(coordinator);
+            this.quorum = quorum;
+            this.flush = flush;
+            this.paging = paging;
+            qualifiedTableName = KEYSPACE + ".t_" + seqNumber.getAndIncrement();
+            consistencyLevel = quorum ? QUORUM : ALL;
+        }
+
+        private Tester createTable(String query)
+        {
+            cluster.schemaChange(format(query) + " WITH read_repair='NONE'");
+            return this;
+        }
+
+        private Tester allNodes(int startInclusive, int endExclusive, Function<Integer, String> querySupplier)
+        {
+            IntStream.range(startInclusive, endExclusive).mapToObj(querySupplier::apply).forEach(this::allNodes);
+            return this;
+        }
+
+        private Tester allNodes(String... queries)
+        {
+            for (String query : queries)
+                allNodes(query);
+            return this;
+        }
+
+        private Tester allNodes(String query)
+        {
+            coordinator.execute(format(query), ALL);
+            return this;
+        }
+
+        private Tester onlyNode1(String... queries)
+        {
+            onlyNodeN(1, queries);
+            return this;
+        }
+
+        private Tester onlyNode2(String... queries)
+        {
+            onlyNodeN(2, queries);
+            return this;
+        }
+
+        private Tester onlyNode3(String... queries)
+        {
+            onlyNodeN(3, queries);
+            return this;
+        }
+
+        private void onlyNodeN(int node, String... queries)
+        {
+            for (String query : queries)
+            {
+                String formattedQuery = format(query);
+                cluster.get(node).executeInternal(formattedQuery);
+                if (quorum)
+                    cluster.get(node == NUM_NODES ? 1 : node + 1).executeInternal(formattedQuery);

Review comment:
       I think all writes are already internal. I have renamed the methods and added some documentation in an attempt to make the relationship between writes and reads clearer, please let me know what do you think.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r521564190



##########
File path: src/java/org/apache/cassandra/service/reads/range/RangeMerger.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.range;
+
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.locator.ReplicaPlan;
+import org.apache.cassandra.locator.ReplicaPlans;
+import org.apache.cassandra.utils.AbstractIterator;
+
+class RangeMerger extends AbstractIterator<ReplicaPlan.ForRangeRead>

Review comment:
       nit: We could also rename this one `ReplicaPlanMerger` or something similar if we rename `RangeIterator`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r540185280



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ConsistencyTest.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.distributed.upgrade;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.Versions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.*;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ConsistencyTest extends UpgradeTestBase
+{
+    @Test
+    public void testConsistency() throws Throwable
+    {
+        testConsistency(allUpgrades(3, 1));
+    }
+
+    @Test
+    public void testConsistencyWithNetworkAndGossip() throws Throwable
+    {
+        testConsistency(new TestCase().nodes(3)
+                                      .nodesToUpgrade(1)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v30)
+                                      // .upgrade(Versions.Major.v22, Versions.Major.v3X)
+                                      // .upgrade(Versions.Major.v30, Versions.Major.v3X)
+                                      .upgrade(Versions.Major.v30, Versions.Major.v4)
+                                      .upgrade(Versions.Major.v3X, Versions.Major.v4)
+                                      .withConfig(cfg -> cfg.with(Feature.NETWORK, Feature.GOSSIP)));
+    }
+
+    private void testConsistency(TestCase testCase) throws Throwable
+    {
+        List<ConsistencyTester> testers = new ArrayList<>();
+        testers.addAll(ConsistencyTester.create(1, ALL, THREE));
+        testers.addAll(ConsistencyTester.create(2, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO));
+        testers.addAll(ConsistencyTester.create(3, ALL, THREE, QUORUM, LOCAL_QUORUM, TWO, ONE, LOCAL_ONE));
+
+        testCase.setup(cluster -> {
+            ConsistencyTester.createTable(cluster);
+            for (ConsistencyTester tester : testers)
+                tester.writeRows(cluster);
+        }).runAfterNodeAndClusterUpgrade(cluster -> {

Review comment:
       I have added a new `MixedModeAvailabilityTest` to verify the behaviour with down nodes, messing with internode messaging. I'm limiting the number of tested consistency levels because, as we said in CASSANDRA-16181, the details of each CL are not the main purpose of the test.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r574617028



##########
File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
##########
@@ -477,6 +477,11 @@ public ICoordinator coordinator(int node)
         return instances.get(node - 1).coordinator();
     }
 
+    public List<ICoordinator> coordinators()

Review comment:
       We can return a `Stream`, yes. Not sure which one is going to be more useful, though.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r536334421



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ShortReadProtectionTest.java
##########
@@ -18,46 +18,309 @@
 
 package org.apache.cassandra.distributed.test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.IntStream;
 
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.api.ICoordinator;
-import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static java.lang.String.format;
-import static java.util.Arrays.asList;
+import static com.google.common.collect.Iterators.toArray;
 import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
-import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
 
 /**
  * Tests short read protection, the mechanism that ensures distributed queries at read consistency levels > ONE/LOCAL_ONE
  * avoid short reads that might happen when a limit is used and reconciliation accepts less rows than such limit.
  */
+@RunWith(Parameterized.class)
 public class ShortReadProtectionTest extends TestBaseImpl
 {
+    private static final int NUM_NODES = 3;
+
+    private static Cluster cluster;
+    private Tester tester;
+
+    /**
+     * {@code true} for CL=QUORUM writes and CL=QUORUM reads, {@code false} for CL=ONE writes and CL=ALL reads.
+     */
+    @Parameterized.Parameter
+    public boolean quorum;
+
+    /**
+     * Whether to flush data after mutations.
+     */
+    @Parameterized.Parameter(1)
+    public boolean flush;
+
+    /**
+     * Whether paging is used for the distributed queries.
+     */
+    @Parameterized.Parameter(2)
+    public boolean paging;
+
+    /**
+     * The node to be used as coordinator.
+     */
+    @Parameterized.Parameter(3)
+    public int coordinator;
+
+    @Parameterized.Parameters(name = "{index}: quorum={0} flush={1} paging={2} coordinator={3}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (boolean quorum : BOOLEANS)
+            for (boolean flush : BOOLEANS)
+                for (boolean paging : BOOLEANS)
+                    for (int coordinator = 1; coordinator < NUM_NODES; coordinator++)
+                        result.add(new Object[]{ quorum, flush, paging, coordinator });
+        return result;
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build()
+                              .withNodes(NUM_NODES)
+                              .withConfig(config -> config.set("hinted_handoff_enabled", false))
+                              .start());
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void setupTester()
+    {
+        tester = new Tester(quorum, flush, paging, coordinator);
+    }
+
+    @After
+    public void teardownTester()
+    {
+        tester.dropTable();
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with a deleted row.
+     * <p>
+     * See CASSANDRA-13880.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13880()}.
+     */
+    @Test
+    public void testSkinnyTableWithoutLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes("INSERT INTO %s (id) VALUES (0)")
+              .onlyNode1("DELETE FROM %s WHERE id = 0")
+              .assertRows("SELECT DISTINCT id FROM %s WHERE id = 0")
+              .assertRows("SELECT id FROM %s WHERE id = 0 LIMIT 1");
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with alternated live and deleted rows.
+     * <p>
+     * See CASSANDRA-13747.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13747()}.
+     */
+    @Test
+    public void testSkinnyTableWithLiveRows()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (1, 0, 4, 6, 3)") // delete every other row
+              .assertRows("SELECT DISTINCT token(id), id FROM %s",
+                          row(token(5), 5), row(token(8), 8), row(token(2), 2), row(token(7), 7), row(token(9), 9));
+    }
+
+    /**
+     * Tests SRP for tables with no clustering columns and with complementary deleted rows.
+     * <p>
+     * See CASSANDRA-13595.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_13595()}.
+     */
+    @Test
+    public void testSkinnyTableWithComplementaryDeletions()
+    {
+        tester.createTable("CREATE TABLE %s (id int PRIMARY KEY)")
+              .allNodes(0, 10, i -> String.format("INSERT INTO %%s (id) VALUES (%d)", i)) // order is 5,1,8,0,2,4,7,6,9,3
+              .onlyNode1("DELETE FROM %s WHERE id IN (5, 8, 2, 7, 9)") // delete every other row
+              .onlyNode2("DELETE FROM %s WHERE id IN (1, 0, 4, 6)") // delete every other row but the last one
+              .assertRows("SELECT id FROM %s LIMIT 1", row(3))
+              .assertRows("SELECT DISTINCT id FROM %s LIMIT 1", row(3));
+    }
+
+    /**
+     * Tests SRP when more than one row is missing.
+     * <p>
+     * See CASSANDRA-12872.
+     * <p>
+     * Replaces Python dtest {@code consistency_test.py:TestConsistency.test_12872()}.
+     */
+    @Test
+    public void testMultipleMissedRows()
+    {
+        tester.createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))")
+              .allNodes(0, 4, i -> String.format("INSERT INTO %%s (pk, ck) VALUES (0, %d)", i))
+              .onlyNode1("DELETE FROM %s WHERE pk = 0 AND ck IN (1, 2, 3)",
+                         "INSERT INTO %s (pk, ck) VALUES (0, 5)")
+              .onlyNode2("INSERT INTO %s (pk, ck) VALUES (0, 4)")
+              .assertRows("SELECT ck FROM %s WHERE pk = 0 LIMIT 2", row(0), row(4));

Review comment:
       It isn't consistent, but I've observed this failing on the parameterization `[quorum=false flush=true paging=true coordinator=2]`:
   
   ```
   java.lang.AssertionError: Expected: [[0],[4]]
   Actual: [[0],[3]]
   	at org.apache.cassandra.distributed.shared.AssertUtils.fail(AssertUtils.java:193)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertTrue(AssertUtils.java:181)
   	at org.apache.cassandra.distributed.shared.AssertUtils.assertRows(AssertUtils.java:70)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.assertRows(ShortReadProtectionTest.java:505)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest$Tester.access$500(ShortReadProtectionTest.java:418)
   	at org.apache.cassandra.distributed.test.ShortReadProtectionTest.testMultipleMissedRows(ShortReadProtectionTest.java:193)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r525556179



##########
File path: test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.range;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.rp;
+import static org.apache.cassandra.Util.token;
+import static org.apache.cassandra.db.ConsistencyLevel.ALL;
+import static org.apache.cassandra.db.ConsistencyLevel.ANY;
+import static org.apache.cassandra.db.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.LOCAL_SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.ONE;
+import static org.apache.cassandra.db.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
+import static org.apache.cassandra.db.ConsistencyLevel.THREE;
+import static org.apache.cassandra.db.ConsistencyLevel.TWO;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Tests for {@link ReplicaPlanMerger}.
+ */
+public class ReplicaPlanMergerTest
+{
+    private static final String KEYSPACE = "ReplicaPlanMergerTest";
+    private static Keyspace keyspace;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2));
+        keyspace = Keyspace.open(KEYSPACE);
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that allows to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ONE, LOCAL_ONE, ANY))
+        {
+            testRanges(cl, range(rp(""), rp("")));
+            testRanges(cl, range(rp(""), rp("25")));
+            testRanges(cl, range(rp(""), rp("40")));
+            testRanges(cl, range(rp(""), rp("50")));
+            testRanges(cl, range(rp("20"), rp("30")));
+            testRanges(cl, range(rp("25"), rp("")));
+            testRanges(cl, range(rp("25"), rp("35")));
+            testRanges(cl, range(rp("50"), rp("")));
+            testRanges(cl, range(rp("40"), rp("10")), range(rp("40"), rp("")), range(rp(""), rp("10"))); // wrapped is split
+            testRanges(cl, range(rp("25"), rp("15")), range(rp("25"), rp("")), range(rp(""), rp("15"))); // wrapped is split
+        }
+    }
+
+    /**
+     * Tests range merging with a single node cluster and a read consistency level that doesn't allow to merge ranges.
+     */
+    @Test
+    public void testSingleNodeWithNotMergingConsistencyLevel()
+    {
+        new TokenUpdater().withTokens("10", "20", "30", "40").update();
+
+        for (ConsistencyLevel cl : Arrays.asList(ALL, QUORUM, LOCAL_QUORUM, EACH_QUORUM, TWO, THREE, SERIAL, LOCAL_SERIAL))
+        {
+            testRanges(cl,
+                       range(rp(""), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp(""), rp("25")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), rp("25")));
+            testRanges(cl,
+                       range(rp(""), rp("40")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), rp("40")));
+            testRanges(cl,
+                       range(rp(""), rp("50")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), endOf("20")),
+                       range(endOf("20"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("50")));
+            testRanges(cl,
+                       range(rp("20"), rp("30")),
+                       range(rp("20"), endOf("20")),
+                       range(endOf("20"), rp("30")));
+            testRanges(cl,
+                       range(rp("25"), rp("")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")));
+            testRanges(cl,
+                       range(rp("50"), rp("")));
+            testRanges(cl,
+                       range(rp("30"), rp("10")),
+                       range(rp("30"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), rp("10"))); // wrapped
+            testRanges(cl,
+                       range(rp("25"), rp("15")),
+                       range(rp("25"), endOf("30")),
+                       range(endOf("30"), endOf("40")),
+                       range(endOf("40"), rp("")),
+                       range(rp(""), endOf("10")),
+                       range(endOf("10"), rp("15"))); // wrapped
+        }
+    }
+
+    /**
+     * Tests range merging with a multinode cluster and a read consistency level that allows to merge ranges,
+     * particularly when the token ranges don't overlap between replicas.
+     */
+    @Test
+    public void testMultiNodeWithMergingConsistencyLevelContinuous() throws UnknownHostException
+    {
+        new TokenUpdater().withTokens(InetAddressAndPort.getByName("127.0.0.1"), "10", "20", "30")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.2"), "40", "50", "60")
+                          .withTokens(InetAddressAndPort.getByName("127.0.0.3"), "70", "80", "90")
+                          .update();

Review comment:
       @adelapena These tests throw a bunch of errors at the logs for the missing endpoints in `Gossiper`. We might have to either initialize them (via `Gossiper#initializeNodeUnsafe()` a la `LeaveAndBootstrapTest`) or find a way to mock out the `ReplicaLayout` liveness check.
   
   ```
   ERROR [main] 2020-11-17 16:00:31,012 FailureDetector.java:281 - Unknown endpoint: /127.0.0.3:7012
   java.lang.IllegalArgumentException: 
   	at org.apache.cassandra.gms.FailureDetector.isAlive(FailureDetector.java:281)
   	at org.apache.cassandra.gms.FailureDetector.lambda$static$0(FailureDetector.java:76)
   	at org.apache.cassandra.locator.AbstractReplicaCollection.filter(AbstractReplicaCollection.java:427)
   	at org.apache.cassandra.locator.AbstractReplicaCollection.filter(AbstractReplicaCollection.java:412)
   	at org.apache.cassandra.locator.ReplicaLayout.forRangeReadLiveSorted(ReplicaLayout.java:335)
   	at org.apache.cassandra.locator.ReplicaPlans.forRangeRead(ReplicaPlans.java:606)
   	at org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:76)
   	at org.apache.cassandra.service.reads.range.ReplicaPlanIterator.computeNext(ReplicaPlanIterator.java:42)
   	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
   	at com.google.common.collect.Iterators$PeekingImpl.hasNext(Iterators.java:1126)
   	at org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:56)
   	at org.apache.cassandra.service.reads.range.ReplicaPlanMerger.computeNext(ReplicaPlanMerger.java:32)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] adelapena commented on a change in pull request #801: CASSANDRA-16180 Coordination tests

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #801:
URL: https://github.com/apache/cassandra/pull/801#discussion_r558328464



##########
File path: test/distributed/org/apache/cassandra/distributed/test/PutGetTest.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRow;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Simple put/get tests using different types of query, paticularly when the data is spread across memory and multiple
+ * sstables and using different compressors. All available compressors are tested. Both ascending and descending
+ * clustering orders are tested. The read queries are run using every node as a coordinator, with and without paging.
+ */
+@RunWith(Parameterized.class)
+public class PutGetTest extends TestBaseImpl
+{
+    private static final int NUM_NODES = 4;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final String CREATE_TABLE = "CREATE TABLE %s(k int, c int, v int, PRIMARY KEY (k, c)) " +
+                                               "WITH CLUSTERING ORDER BY (c %s) " +
+                                               "AND COMPRESSION = { 'class': '%s' } " +
+                                               "AND READ_REPAIR = 'none'";
+    private static final String[] COMPRESSORS = new String[]{ NoopCompressor.class.getSimpleName(),
+                                                              LZ4Compressor.class.getSimpleName(),
+                                                              DeflateCompressor.class.getSimpleName(),
+                                                              SnappyCompressor.class.getSimpleName(),
+                                                              ZstdCompressor.class.getSimpleName() };
+
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    /**
+     * The sstable compressor to be used.
+     */
+    @Parameterized.Parameter
+    public String compressor;
+
+    /**
+     * Whether the clustering order is reverse.
+     */
+    @Parameterized.Parameter(1)
+    public boolean reverse;
+
+    private String tableName;
+
+    @Parameterized.Parameters(name = "{index}: compressor={0} reverse={1}")
+    public static Collection<Object[]> data()
+    {
+        List<Object[]> result = new ArrayList<>();
+        for (String compressor : COMPRESSORS)
+            for (boolean reverse : BOOLEANS)
+                result.add(new Object[]{ compressor, reverse });
+        return result;
+    }
+
+    private static Cluster cluster;
+
+    @BeforeClass
+    public static void setupCluster() throws IOException
+    {
+        cluster = init(Cluster.build(NUM_NODES).start(), REPLICATION_FACTOR);
+    }
+
+    @AfterClass
+    public static void teardownCluster()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+
+    @Before
+    public void before()
+    {
+        // create the table
+        tableName = String.format("%s.t_%d", KEYSPACE, seq.getAndIncrement());
+        cluster.schemaChange(String.format(CREATE_TABLE, tableName, reverse ? "DESC" : "ASC", compressor));
+    }
+
+    @After
+    public void after()
+    {
+        cluster.schemaChange(format("DROP TABLE %s"));
+    }
+
+    /**
+     * Simple put/get on a single partition with a few rows, reading with a single partition query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_putget[_snappy|_deflate]().
+     */
+    @Test
+    public void testPartitionQuery()
+    {
+        int numRows = 10;
+
+        writeRows(1, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s WHERE k=?", 0);
+        Assert.assertEquals(numRows, rows.length);
+        for (int c = 0; c < numRows; c++)
+        {
+            validateRow(rows[c], numRows, 0, c);
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with a range query.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_rangeputget().
+     */
+    @Test
+    public void testRangeQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        Object[][] rows = readRows("SELECT * FROM %s");
+        Assert.assertEquals(numPartitions * numRows, rows.length);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            for (int c = 0; c < numRows; c++)
+            {
+                Object[] row = rows[k * numRows + c];
+                validateRow(row, numRows, k, c);
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on a single partition with multiple rows, reading with slice queries.
+     * <p>
+     * Migrated from Python dtests putget_test.py:TestPutGet.test_wide_row().
+     */
+    @Test
+    public void testSliceQuery()
+    {
+        int numRows = 100;
+
+        writeRows(1, numRows);
+
+        String query = "SELECT * FROM %s WHERE k=? AND c>=? AND c<?";
+        for (int sliceSize : Arrays.asList(10, 20, 100))
+        {
+            for (int c = 0; c < numRows; c = c + sliceSize)
+            {
+                Object[][] rows = readRows(query, 0, c, c + sliceSize);
+                Assert.assertEquals(sliceSize, rows.length);
+
+                for (int i = 0; i < sliceSize; i++)
+                {
+                    Object[] row = rows[i];
+                    validateRow(row, numRows, 0, c + i);
+                }
+            }
+        }
+    }
+
+    /**
+     * Simple put/get on multiple partitions with multiple rows, reading with IN queries.
+     */
+    @Test
+    public void testInQuery()
+    {
+        int numPartitions = 10;
+        int numRows = 10;
+
+        writeRows(numPartitions, numRows);
+
+        String query = "SELECT * FROM %s WHERE k IN (?, ?)";
+        for (int k = 0; k < numPartitions; k += 2)
+        {
+            Object[][] rows = readRows(query, k, k + 1);
+            Assert.assertEquals(numRows * 2, rows.length);
+
+            for (int i = 0; i < 2; i++)
+            {
+                for (int c = 0; c < numRows; c++)
+                {
+                    Object[] row = rows[i * numRows + c];
+                    validateRow(row, numRows, k + i, c);
+                }
+            }
+        }
+    }
+
+    /**
+     * Writes {@code numPartitions} with {@code numRows} each, with overrides in different sstables and memtables.
+     */
+    private void writeRows(int numPartitions, int numRows)
+    {
+        String update = format("UPDATE %s SET v=? WHERE k=? AND c=?");
+        ICoordinator coordinator = cluster.coordinator(1);
+        for (int k = 0; k < numPartitions; k++)
+        {
+            // insert all the partition rows in a single sstable
+            for (int c = 0; c < numRows; c++)
+                coordinator.execute(update, QUORUM, c, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows in a second sstable
+            for (int c = 0; c < numRows; c += 2)
+                coordinator.execute(update, QUORUM, c + numRows, k, c);
+            cluster.forEach(i -> i.flush(KEYSPACE));
+
+            // override some rows only in memtable
+            for (int c = 0; c < numRows; c += 5)
+                coordinator.execute(update, QUORUM, c + numRows * 2, k, c);
+        }
+    }
+
+    /**
+     * Runs the specified query in all coordinators, with and without paging.
+     */
+    private Object[][] readRows(String query, Object... boundValues)
+    {
+        query = format(query);
+        Object[][] rows = null;
+
+        // verify that all coordinators return the same results for the query, regardless of paging
+        for (ICoordinator coordinator : cluster.coordinators())
+        {
+            for (boolean paging : BOOLEANS)
+            {
+                Object[][] readRows = paging
+                                      ? Iterators.toArray(coordinator.executeWithPaging(query, QUORUM, 1, boundValues),
+                                                          Object[].class)
+                                      : coordinator.execute(query, QUORUM, boundValues);
+                if (rows == null)
+                    rows = readRows;
+                else
+                    assertRows(rows, readRows);

Review comment:
       Done. Indeed testing the different coordinators and paging here instead of just using Junit params makes things a bit more complex, but there is a significant performance gain over Junit parameterization.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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