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/13 22:50:35 UTC

[GitHub] [cassandra] maedhroz opened a new pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

maedhroz opened a new pull request #821:
URL: https://github.com/apache/cassandra/pull/821


   


----------------------------------------------------------------
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 pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   @maedhroz Regarding `HintedHandoffNodetoolTest`, `get/setMaxHintWindow` are already covered by Python dtests, but I guess it wouldn't hurt to also add them in the Java dtest for completeness, so it covers all the six nodetool commands for hh, 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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable

Review comment:
       See https://github.com/apache/cassandra-dtest/pull/112/files




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {

Review comment:
       Yes, you are right, there is no value in using `runAfterClusterUpgrade `.




----------------------------------------------------------------
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] ekaterinadimitrova2 commented on a change in pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable

Review comment:
       Shall we elaborate on what partially means? If I see it for the first time I will be confused probably




----------------------------------------------------------------
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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-764010187


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/204/workflows/1fefb1bb-e9b0-4cd3-8b2a-5883bb0ac3e5


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */

Review comment:
       Done here: https://github.com/apache/cassandra-dtest/pull/112/commits/23ee8360345eb7738b06b323e64cd3d839123321




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable

Review comment:
       This is looking really nice. Are we going to add some additional consistency levels? The problem with lower CLs is that we can't only make assumptions about the minimum number of written replicas.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable

Review comment:
       I think we agreed to use the `@ported_to_in_jvm('4.0')` annotation on these instead of removing for 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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-760524407


   Current CI status: https://app.circleci.com/pipelines/github/maedhroz/cassandra/193/workflows/6cf141e0-ad68-4559-9e61-334660dd61e4
   


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/locator/LocalStrategy.java
##########
@@ -69,6 +69,7 @@ public void validateOptions() throws ConfigurationException
     {
     }
 
+    @Override
     public Collection<String> recognizedOptions()
     {

Review comment:
       I like to leave the type information around sometimes when there isn't an obvious return type or assignment in view, but this one meets that description anyway, so I'll go for it.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-16181


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+            String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";
+
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                // ...and sample enough keys that we cover the ring.
+                for (int j = 0; j < 10; j++)
+                {
+                    int key = j + (i * 10);
+                    Object[] row = row(key, "foo");
+                    Long token = tokenFrom(key);
+
+                    cluster.coordinator(i).execute(INSERT, ConsistencyLevel.ALL, row);
+
+                    int node = calculatePrimary(validTokens, token);
+                    assertRows(cluster.get(node).executeInternal(SELECT, key), row);
+
+                    node = advance(node, cluster.size());
+                    assertRows(cluster.get(node).executeInternal(SELECT, key), row);
+
+                    // At RF=2, this node should not have received the write.
+                    node = advance(node, cluster.size());
+                    assertRows(cluster.get(node).executeInternal(SELECT, key));
+                }
+            }
+        })
+        .run();
+    }
+
+    private int calculatePrimary(List<Long> initialTokens, Long token)
+    {
+        int primary = 1;
+
+        for (Long initialToken : initialTokens)
+        {
+            if (token <= initialToken)
+            {
+                break;
+            }
+            
+            primary++;
+        }
+        
+        return primary;
+    }
+
+    private Long tokenFrom(int key)
+    {
+        DecoratedKey dk = Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.bytes(key));
+        return (Long) dk.getToken().getTokenValue();
+    }
+    
+    private int advance(int current, int nodes)

Review comment:
       Maybe `nextNode(int node, int numNodes)`?




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/177/workflows/086ea849-3973-48ef-a659-5956cd3873a4


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -77,12 +79,17 @@ public int size()
     {
         return decodedMutations.size() + encodedMutations.size();
     }
-
-    static final class Serializer implements IVersionedSerializer<Batch>
+    
+    public boolean isLocal()

Review comment:
       We could add some brief JavaDoc telling what local means, similar to the one for the both `createLocal` and `createRemote` methods right above. By the way, `createRemote` has an unneeded `<Mutation>` that we could remove here.

##########
File path: src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -189,7 +188,7 @@ else if (replicaPlan.consistencyLevel() == ConsistencyLevel.EACH_QUORUM && (this
             else
             {
                 //Construct a delegate response handler to use to track the ideal consistency level
-                AbstractWriteResponseHandler idealHandler = getWriteResponseHandler(replicaPlan.withConsistencyLevel(idealConsistencyLevel),
+                AbstractWriteResponseHandler<T> idealHandler = getWriteResponseHandler(replicaPlan.withConsistencyLevel(idealConsistencyLevel),

Review comment:
       Nit: misaligned parameters

##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       Indeed exposing the encoded mutations sounds less aggressive, although I'm not against the test method. If we preserve the `reserialize` method we should add some JavaDoc indicating why it exists.

##########
File path: src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -318,7 +317,7 @@ private static AbstractReplicationStrategy createInternal(String keyspaceName,
         throws ConfigurationException
     {
         AbstractReplicationStrategy strategy;
-        Class [] parameterTypes = new Class[] {String.class, TokenMetadata.class, IEndpointSnitch.class, Map.class};
+        Class[] parameterTypes = new Class[] {String.class, TokenMetadata.class, IEndpointSnitch.class, Map.class};

Review comment:
       Nit: It could be `Class<?>[] parameterTypes`.

##########
File path: src/java/org/apache/cassandra/batchlog/BatchlogManager.java
##########
@@ -447,8 +447,7 @@ private void writeHintsForUndeliveredEndpoints(int startFrom, Set<InetAddressAnd
             for (Mutation mutation : mutations)
             {
                 ReplayWriteResponseHandler<Mutation> handler = sendSingleReplayMutation(mutation, writtenAt, hintedNodes);
-                if (handler != null)

Review comment:
       Good catch

##########
File path: src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -451,7 +451,7 @@ protected void validateReplicationFactor(String s) throws ConfigurationException
 
     protected void validateExpectedOptions() throws ConfigurationException
     {
-        Collection expectedOptions = recognizedOptions();
+        Collection<String> expectedOptions = recognizedOptions();

Review comment:
       Nit: there is a missed unneeded `<String, String>` in line 69.

##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -318,6 +320,28 @@ private static IMessage serializeMessage(InetAddressAndPort from, InetAddressAnd
 
         try (DataOutputBuffer out = new DataOutputBuffer(1024))
         {
+            // On a 4.0+ node, C* makes a distinction between "local" and "remote" batches, where only the former can 
+            // be serialized and sent to a remote node, where they are deserialized and written to the batch commitlog
+            // without first being converted into mutation objects. Batch serialization is therfore not symmetric, and
+            // we use a special procedure here that "re-serializes" a "remote" batch to build the message.

Review comment:
       Nice comment




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable

Review comment:
       @adelapena Yeah, writing at the lower CLs doesn't seem to be super valuable if the point of all this is to make sure replication works properly. We just use ALL to make the tests deterministic, although I think I'll probably work out some additional tests elsewhere to cover missing hint delivery scenarios, if there are any. (I think we had identified a few in the [audit doc](https://docs.google.com/document/d/1yPbquhAALIkkTRMmyOv5cceD5N5sPFMB1O4iOd3O7FM/edit?disco=AAAAKxI9jj4).)




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.IMessageFilters;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+
+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.BATCH_STORE_REQ;
+import static org.apache.cassandra.net.Verb.REQUEST_RSP;
+
+public class MixedModeBatchTestBase extends UpgradeTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            if (isLogged)
+            {
+                // If we're testing logged batches, exercise the case were batchlog writes fail.
+                IMessageFilters.Filter dropBatchlogWrite = cluster.filters().inbound().verbs(BATCH_STORE_REQ.id, REQUEST_RSP.id).drop();
+                dropBatchlogWrite.on();
+                testBatches(isLogged, true, insert, select, cluster, upgraded);

Review comment:
       ```suggestion
                   testBatches(true, true, insert, select, cluster, upgraded);
   ```




----------------------------------------------------------------
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 pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   The last changes in the tests for logged/unlogged batches look good to 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 pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/204/workflows/1fefb1bb-e9b0-4cd3-8b2a-5883bb0ac3e5


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                StringBuilder batchBuilder = new StringBuilder("BEGIN " + (isLogged ? "" : "UNLOGGED ") + "BATCH\n");

Review comment:
       @adelapena I've been looking for some tests somewhere that actually verify atomicity (without isolation, of course) in the case where a mutation in the batch fails, but I don't see any. I'm trying to think through what kind of failure we would actually want to induce. I'm reading a lot of this code for the first time, but it seems like there isn't actually a rollback mechanism, and it's just the batchlog along with its periodic replay that actually makes the "all or nothing behavior possible". The only way it seems we would actually induce a failure here is to have the coordinator's batchlog write fail. (i.e. We'd get nothing in this case, because if the batchlog writes don't succeed, we don't even start the normal writes?)
   
   It might be possible to parameterize the existing tests to optionally activate a `ByteBuddy` rule or network filter that fails batchlog writes. 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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTestBase.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MixedModeReplicationTestBase extends UpgradeTestBase

Review comment:
       I'll start by at least pushing the up to the `UpgradeTestBase`...




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.List;
+
+import org.junit.AfterClass;
+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.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.hints.HintsService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class HintedHandoffNodetoolTest extends TestBaseImpl
+{
+    private static Cluster cluster;
+
+    @Parameterized.Parameter
+    public int node;
+
+    @Parameterized.Parameters(name = "node={0}")
+    public static List<Object[]> data()
+    {   
+        List<Object[]> result = new ArrayList<>();
+        result.add(new Object[]{ 1 });
+        result.add(new Object[]{ 2 });
+        return result;
+    }
+
+    @BeforeClass
+    public static void before() throws IOException
+    {
+        cluster = init(Cluster.build().withNodes(2).withDCs(2).start());
+    }
+
+    @AfterClass
+    public static void after()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+    
+    @Before
+    public void enableHandoff()
+    {
+        cluster.get(1).nodetoolResult("enablehandoff");
+        cluster.get(2).nodetoolResult("enablehandoff");
+    }
+
+    @Test
+    public void testEnableHandoff()
+    {
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is running");

Review comment:
       We could also use JMX to verify that hh is enabled:
   ```
   assertTrue(cluster.get(node).callOnInstance(() -> StorageProxy.instance.getHintedHandoffEnabled()));
   ```
   Also, we could test `enablehandoff` both when it is initially enabled and when it's initially disabled, wdyt?

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.List;
+
+import org.junit.AfterClass;
+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.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.hints.HintsService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class HintedHandoffNodetoolTest extends TestBaseImpl
+{
+    private static Cluster cluster;
+
+    @Parameterized.Parameter
+    public int node;
+
+    @Parameterized.Parameters(name = "node={0}")
+    public static List<Object[]> data()
+    {   
+        List<Object[]> result = new ArrayList<>();
+        result.add(new Object[]{ 1 });
+        result.add(new Object[]{ 2 });
+        return result;
+    }
+
+    @BeforeClass
+    public static void before() throws IOException
+    {
+        cluster = init(Cluster.build().withNodes(2).withDCs(2).start());
+    }
+
+    @AfterClass
+    public static void after()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+    
+    @Before
+    public void enableHandoff()
+    {
+        cluster.get(1).nodetoolResult("enablehandoff");
+        cluster.get(2).nodetoolResult("enablehandoff");
+    }
+
+    @Test
+    public void testEnableHandoff()
+    {
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is running");
+    }
+
+    @Test
+    public void testDisableHandoff()
+    {
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is running");
+        cluster.get(node).nodetoolResult("disablehandoff");
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is not running");
+    }
+
+    @Test
+    public void testDisableForDC()

Review comment:
       I think we are missing the complementary `enablehintsfordc`




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       This is where I'd be interested to know what our second reviewer thinks ;)




----------------------------------------------------------------
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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-765599330


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/206/workflows/91c0a021-8122-449c-8bbf-ef14533b36dc


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {

Review comment:
       Would it make sense to also run checks after the cluster upgrade?




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?
+    private void verifyReplication(UpgradeableCluster cluster, int nodes)
+    {
+        String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+        String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        List<Long> initialTokens = new ArrayList<>(nodes + 1);
+        initialTokens.add(null);
+
+        for (int i = 1; i <= nodes; i++)
+        {
+            initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+        }
+
+        // TODO: Pick some random keys to cover all three nodes as primaries.
+        int key = 1;
+        Object[] row = row(key, "foo");
+        Long token = tokenFrom(key);
+
+        int primary = calculatePrimary(initialTokens.subList(1, nodes + 1), token);
+
+        cluster.coordinator(1).execute(INSERT, ConsistencyLevel.ALL, row);
+        
+        assertRows(cluster.get(primary).executeInternal(SELECT, key), row);
+        assertRows(cluster.get(next(primary, nodes)).executeInternal(SELECT, key), row);
+        assertRows(cluster.get(next(primary + 1, nodes)).executeInternal(SELECT, key));

Review comment:
       This `primary + 1` can be out of bounds if primary is 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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +127,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException

Review comment:
       Agreed. If we keep this method, rather than exposing the encoded mutations, I'll add it...




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   @adelapena The only thing that's a little odd is the failures here: https://app.circleci.com/pipelines/github/maedhroz/cassandra/178/workflows/b6ae3c51-dfcc-4c1f-93a8-d3fc11bbab40/jobs/980
   
   Re-running here: https://app.circleci.com/pipelines/github/maedhroz/cassandra/178/workflows/239b4b60-14ee-4e75-bdfe-ab7c7ddae071


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+            String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";
+
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                // ...and sample enough keys that we cover the ring.
+                for (int j = 0; j < 10; j++)
+                {
+                    int key = j + (i * 10);
+                    Object[] row = row(key, "foo");
+                    Long token = tokenFrom(key);
+
+                    cluster.coordinator(i).execute(INSERT, ConsistencyLevel.ALL, row);
+
+                    int node = calculatePrimary(validTokens, token);
+                    assertRows(cluster.get(node).executeInternal(SELECT, key), row);
+
+                    node = advance(node, cluster.size());
+                    assertRows(cluster.get(node).executeInternal(SELECT, key), row);
+
+                    // At RF=2, this node should not have received the write.
+                    node = advance(node, cluster.size());
+                    assertRows(cluster.get(node).executeInternal(SELECT, key));
+                }
+            }
+        })
+        .run();
+    }
+
+    private int calculatePrimary(List<Long> initialTokens, Long token)
+    {
+        int primary = 1;
+
+        for (Long initialToken : initialTokens)
+        {
+            if (token <= initialToken)
+            {
+                break;
+            }
+            
+            primary++;
+        }
+        
+        return primary;
+    }
+
+    private Long tokenFrom(int key)

Review comment:
       Nit: Thee three private methods could be `static`.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+            String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";

Review comment:
       Nit: Can we lower case these since they aren't class constants? We can always mark them as `final` to underline that they are constants, or make them class constants.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   Haven't broken anything yet: https://app.circleci.com/pipelines/github/maedhroz/cassandra/162/workflows/3e788562-20b1-4e4e-a03f-f63ec0bc38c0


----------------------------------------------------------------
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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-765599330






----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   Committed in https://github.com/apache/cassandra/commit/953c18df33ab3e009ced15a16785e2753843418a


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeUnloggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeUnloggedBatchTest extends MixedModeBatchTestBase
+{
+    @Test
+    public void testSimpleStrategy22to30() throws Throwable

Review comment:
       It seems that this is missing 3.0 to 3.x, while we test that path for logged batches, is that intentional? Also, the tests for logged batches are split into two separate classes, while the tests for unlogged batches are inside a single class.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/schema/MigrationManager.java
##########
@@ -213,6 +213,11 @@ public static void announce(Collection<Mutation> schema)
         FBUtilities.waitOnFuture(f);
     }
 
+    public static Future<?> announceWithoutPush(Collection<Mutation> schema)
+    {
+        return MIGRATION.submit(() -> Schema.instance.mergeAndAnnounceVersion(schema));
+    }
+

Review comment:
       This commit would be pushed to CASSANDRA-16387 eventually, but it's easier to test here for 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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeLoggedBatchTest extends MixedModeBatchTestBase
+{
+    @Test
+    public void testSimpleStrategy22to30() throws Throwable
+    {
+        testSimpleStrategy(Versions.Major.v22, Versions.Major.v30, true);

Review comment:
       Shouldn't we include `v22` to `v3X` and `v30` to `v3X`?

##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTestBase.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MixedModeReplicationTestBase extends UpgradeTestBase
+{
+    protected static int calculatePrimary(List<Long> initialTokens, Long token)

Review comment:
       We could name this `calculatePrimaryReplica`.

##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                StringBuilder batchBuilder = new StringBuilder("BEGIN " + (isLogged ? "" : "UNLOGGED ") + "BATCH\n");

Review comment:
       We are not differentiating a lot between logged and unlogged batches. Perhaps we could make some batches to fail to see that the atomicity guarantees are kept in mixed mode, wdyt?

##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeLoggedBatchTest extends MixedModeBatchTestBase

Review comment:
       Would it sound very bad using `MixedModeBatchLoggedTest` and `MixedModeBatchUnloggedTest` instead, so all the three tests get together alphabetical order? Feel free to ignore if you don't agree.

##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTestBase.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MixedModeReplicationTestBase extends UpgradeTestBase

Review comment:
       The utility functions here seems useful beyond mixed mode replication tests, we could put them in `UpgradeTestBase`. I think they would be useful even for not-upgrade tests extending `TestBaseImpl`, although we don't have a common accessible superclass for them, nor a utility class for this kind of things. Perhaps we should create it?




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/schema/MigrationManager.java
##########
@@ -213,6 +213,11 @@ public static void announce(Collection<Mutation> schema)
         FBUtilities.waitOnFuture(f);
     }
 
+    public static Future<?> announceWithoutPush(Collection<Mutation> schema)
+    {
+        return MIGRATION.submit(() -> Schema.instance.mergeAndAnnounceVersion(schema));
+    }
+

Review comment:
       This commit would be pushed to CASSANDRA-16387 eventually, but it's easier to test here for 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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-747525879


   @adelapena The only thing that's a little odd is the failures here: https://app.circleci.com/pipelines/github/maedhroz/cassandra/178/workflows/b6ae3c51-dfcc-4c1f-93a8-d3fc11bbab40/jobs/980
   
   Re-running here: https://app.circleci.com/pipelines/github/maedhroz/cassandra/178/workflows/239b4b60-14ee-4e75-bdfe-ab7c7ddae071


----------------------------------------------------------------
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 closed pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: .circleci/config.yml
##########
@@ -2199,11 +2199,11 @@ jobs:
           export PATH=$JAVA_HOME/bin:$PATH
           cd ~/cassandra
           mkdir ~/dtest_jars
-          git remote add apache git://github.com/apache/cassandra.git
-          for branch in cassandra-2.2 cassandra-3.0 cassandra-3.11 trunk; do
+          git remote add maedhroz git@github.com:maedhroz/cassandra.git
+          for branch in cassandra-2.2 CASSANDRA-16387-3.0 CASSANDRA-16387-3.11 trunk; do
             # check out the correct cassandra version:
-            git remote set-branches --add apache '$branch'
-            git fetch --depth 1 apache $branch
+            git remote set-branches --add maedhroz '$branch'
+            git fetch --depth 1 maedhroz $branch

Review comment:
       @adelapena @ekaterinadimitrova2 This commit will just be reverted once CASSANDRA-16387 commits...




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?

Review comment:
       Probably, though that's going to be a bit more tricky because IIR the write consistency level only guarantees a minimum of written replicas, but there could be more. Also, any replica can be written. 




----------------------------------------------------------------
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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-740787368


   Haven't broken anything yet: https://app.circleci.com/pipelines/github/maedhroz/cassandra/162/workflows/3e788562-20b1-4e4e-a03f-f63ec0bc38c0


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeLoggedBatchTest extends MixedModeBatchTestBase

Review comment:
       I see the appeal...although it feels more important to keep the phrases "[logged|unlogged] batch" intact for the reader.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/locator/ReplicationFactor.java
##########
@@ -76,11 +76,11 @@ static void validate(int totalRF, int transientRF)
                                                                     .filter(endpoint -> Gossiper.instance.getReleaseVersion(endpoint) != null && Gossiper.instance.getReleaseVersion(endpoint).major < 4)
                                                                     .collect(Collectors.toList());
             if (!badVersionEndpoints.isEmpty())
-                throw new AssertionError("Transient replication is not supported in mixed version clusters with nodes < 4.0. Bad nodes: " + badVersionEndpoints);
+                throw new IllegalArgumentException("Transient replication is not supported in mixed version clusters with nodes < 4.0. Bad nodes: " + badVersionEndpoints);

Review comment:
       Nice 👍 

##########
File path: src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -451,7 +451,7 @@ protected void validateReplicationFactor(String s) throws ConfigurationException
 
     protected void validateExpectedOptions() throws ConfigurationException
     {
-        Collection expectedOptions = recognizedOptions();
+        Collection<String> expectedOptions = recognizedOptions();

Review comment:
       There are some other generics-related warnings in this file, we could fix them here even if it's not strictly related to the patch.

##########
File path: test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
##########
@@ -57,7 +54,7 @@ public void setup(Class stratClass, Map<String, String> strategyOptions) throws
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddressAndPort.getByName("127.0.0.2"));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(30)), InetAddressAndPort.getByName("127.0.0.3"));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(40)), InetAddressAndPort.getByName("127.0.0.4"));
-        //tmd.updateNormalToken(new BigIntegerToken(String.valueOf(50)), InetAddressAndPort.getByName("127.0.0.5", null, null));
+        tmd.updateNormalToken(new BigIntegerToken(String.valueOf(50)), InetAddressAndPort.getByName("127.0.0.5"));

Review comment:
       Do you know why was this commented? By the way, this block could be (perhaps) simplified to:
   ```java
   for (int i = 1; i <= 8; i++)
       tmd.updateNormalToken(new BigIntegerToken(String.valueOf(i * 10)), 
                                                 InetAddressAndPort.getByName("127.0.0." + i));
   ```

##########
File path: test/unit/org/apache/cassandra/locator/ReplicationFactorTest.java
##########
@@ -52,32 +54,42 @@ private static void assertRfParseFailure(String s)
     private static void assertRfParse(String s, int expectedReplicas, int expectedTrans)
     {
         ReplicationFactor rf = ReplicationFactor.fromString(s);
-        Assert.assertEquals(expectedReplicas, rf.allReplicas);
-        Assert.assertEquals(expectedTrans, rf.transientReplicas());
-        Assert.assertEquals(expectedReplicas - expectedTrans, rf.fullReplicas);
+        assertEquals(expectedReplicas, rf.allReplicas);
+        assertEquals(expectedTrans, rf.transientReplicas());
+        assertEquals(expectedReplicas - expectedTrans, rf.fullReplicas);
     }
 
     @Test
-    public void parseTest()
+    public void shouldParseValidRF()
     {
+        assertRfParse("0", 0, 0);
         assertRfParse("3", 3, 0);
         assertRfParse("3/1", 3, 1);
-
         assertRfParse("5", 5, 0);
         assertRfParse("5/2", 5, 2);
+    }
 
+    @Test
+    public void shouldFailOnInvalidRF()
+    {
         assertRfParseFailure("-1");
         assertRfParseFailure("3/3");
+        assertRfParseFailure("3/-1");
         assertRfParseFailure("3/4");
+        assertRfParseFailure("3/");

Review comment:
       We could check the message of the exception caught in `assertRfParseFailure`, passing the expected message as an argument. Additionally, we could try strings producing `NumberFormatException`, like `""`, `"a/1"` and `"1/a"`:
   ```java
   assertRfParseFailure("-1", "Replication factor must be non-negative, found -1");
   assertRfParseFailure("3/3", "Transient replicas must be zero, or less than total replication factor. For 3/3");
   assertRfParseFailure("3/-1", "Amount of transient nodes should be strictly positive, but was: '-1'");
   assertRfParseFailure("3/4", "Transient replicas must be zero, or less than total replication factor. For 3/4");
   assertRfParseFailure("3/", "Replication factor format is <replicas> or <replicas>/<transient>");
   assertRfParseFailure("1/a", "For input string: \"a\"");
   assertRfParseFailure("a/1", "For input string: \"a\"");
   assertRfParseFailure("", "For input string: \"\"");
   ```
   We could also consider throwing a `NumberFormatException` with a more RF-specific message, although that doesn't seem especially useful.

##########
File path: src/java/org/apache/cassandra/locator/SimpleStrategy.java
##########
@@ -73,17 +72,19 @@ public EndpointsForRange calculateNaturalReplicas(Token token, TokenMetadata met
         return replicas.build();
     }
 
+    @Override
     public ReplicationFactor getReplicationFactor()
     {
         return rf;
     }
 
-    private final static void validateOptionsInternal(Map<String, String> configOptions) throws ConfigurationException
+    private static void validateOptionsInternal(Map<String, String> configOptions) throws ConfigurationException
     {
         if (configOptions.get(REPLICATION_FACTOR) == null)
             throw new ConfigurationException("SimpleStrategy requires a replication_factor strategy option.");
     }
 
+    @Override

Review comment:
       The method `recognizedOptions` right below could also benefit from an `@Override` annotation.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeUnloggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeUnloggedBatchTest extends MixedModeBatchTestBase
+{
+    @Test
+    public void testSimpleStrategy22to30() throws Throwable

Review comment:
       Just an oversight. Will fix...




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {

Review comment:
       The "cluster upgrade" being finished really just means the last individual node upgrade is complete, whether or not every node has been upgraded, correct? I could change this so that all three nodes are upgraded, but I think it would still involve just using `runAfterNodeUpgrade()` (which would subsume `runAfterClusterUpgrade()`).




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?
+    private void verifyReplication(UpgradeableCluster cluster, int nodes)
+    {
+        String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+        String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        List<Long> initialTokens = new ArrayList<>(nodes + 1);
+        initialTokens.add(null);
+
+        for (int i = 1; i <= nodes; i++)
+        {
+            initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+        }
+
+        // TODO: Pick some random keys to cover all three nodes as primaries.
+        int key = 1;
+        Object[] row = row(key, "foo");
+        Long token = tokenFrom(key);
+
+        int primary = calculatePrimary(initialTokens.subList(1, nodes + 1), token);
+
+        cluster.coordinator(1).execute(INSERT, ConsistencyLevel.ALL, row);
+        
+        assertRows(cluster.get(primary).executeInternal(SELECT, key), row);
+        assertRows(cluster.get(next(primary, nodes)).executeInternal(SELECT, key), row);
+        assertRows(cluster.get(next(primary + 1, nodes)).executeInternal(SELECT, key));

Review comment:
       Ouch :(




----------------------------------------------------------------
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] ekaterinadimitrova2 commented on a change in pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/locator/LocalStrategy.java
##########
@@ -69,6 +69,7 @@ public void validateOptions() throws ConfigurationException
     {
     }
 
+    @Override
     public Collection<String> recognizedOptions()
     {

Review comment:
       Same as before, there is one last warning on line 76 but I do not insist on dealing with this here, up to you :-) 

##########
File path: src/java/org/apache/cassandra/locator/LocalStrategy.java
##########
@@ -69,6 +69,7 @@ public void validateOptions() throws ConfigurationException
     {
     }
 
+    @Override

Review comment:
       Do we need to add @Override also to other methods, like for example `getReplicationFactor()`?

##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable

Review comment:
       This all looks great, can we add the jira ticket from the dtests for reference? CASSANDRA-9673

##########
File path: src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -451,7 +451,7 @@ protected void validateReplicationFactor(String s) throws ConfigurationException
 
     protected void validateExpectedOptions() throws ConfigurationException
     {
-        Collection expectedOptions = recognizedOptions();
+        Collection<String> expectedOptions = recognizedOptions();

Review comment:
       Also a few others, but I'll leave this up to Caleb if he wants to take care of them as they are unrelated to this work

##########
File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
##########
@@ -90,6 +94,16 @@ public static ByteBuffer tuple(Object... values)
         return TupleType.buildValue(bbs);
     }
 
+    protected void bootstrapAndJoinNode(Cluster cluster)
+    {
+        IInstanceConfig config = cluster.newInstanceConfig();
+        config.set("auto_bootstrap", true);
+        IInvokableInstance newInstance = cluster.bootstrap(config);
+        withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                     () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+        newInstance.nodetoolResult("join").asserts().success();
+    }
+

Review comment:
       Nice one :-) 

##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       Do we need to say more about the test scenarios  where it is acceptable to use it if we keep it here?
   I kind of feel it is better to be in Instance

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */

Review comment:
       Do we need also to say that we still keep the original ones and why?

##########
File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
##########
@@ -112,7 +112,7 @@
  */
 public abstract class AbstractCluster<I extends IInstance> implements ICluster<I>, AutoCloseable
 {
-    public static Versions.Version CURRENT_VERSION = new Versions.Version(FBUtilities.getReleaseVersionString(), Versions.getClassPath());;
+    public static Versions.Version CURRENT_VERSION = new Versions.Version(FBUtilities.getReleaseVersionString(), Versions.getClassPath());
 

Review comment:
       nit: can we also remove at least the unused imports?

##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -77,12 +79,17 @@ public int size()
     {
         return decodedMutations.size() + encodedMutations.size();
     }
-
-    static final class Serializer implements IVersionedSerializer<Batch>
+    
+    public boolean isLocal()

Review comment:
       I think the unneeded `<Mutation>` was not removed

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.List;
+
+import org.junit.AfterClass;
+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.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.hints.HintsService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class HintedHandoffNodetoolTest extends TestBaseImpl
+{
+    private static Cluster cluster;
+
+    @Parameterized.Parameter
+    public int node;
+
+    @Parameterized.Parameters(name = "node={0}")
+    public static List<Object[]> data()
+    {   
+        List<Object[]> result = new ArrayList<>();
+        result.add(new Object[]{ 1 });
+        result.add(new Object[]{ 2 });
+        return result;
+    }
+
+    @BeforeClass
+    public static void before() throws IOException
+    {
+        cluster = init(Cluster.build().withNodes(2).withDCs(2).start());
+    }
+
+    @AfterClass
+    public static void after()
+    {
+        if (cluster != null)
+            cluster.close();
+    }
+    
+    @Before
+    public void enableHandoff()
+    {
+        cluster.get(1).nodetoolResult("enablehandoff");
+        cluster.get(2).nodetoolResult("enablehandoff");
+    }
+
+    @Test
+    public void testEnableHandoff()
+    {
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is running");
+    }
+
+    @Test
+    public void testDisableHandoff()
+    {
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is running");
+        cluster.get(node).nodetoolResult("disablehandoff");
+        cluster.get(node).nodetoolResult("statushandoff").asserts().success().stdoutContains("Hinted handoff is not running");
+    }
+
+    @Test
+    public void testDisableForDC()

Review comment:
       I think we still miss it? 

##########
File path: test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java
##########
@@ -276,9 +276,14 @@ public void accept(IInvokableInstance instance)
 
     public static InstanceAction decomission()
     {
-        return (target) -> target.nodetoolResult("decommission").asserts().success();
+        return decomission(false);
     }
 
+    public static InstanceAction decomission(boolean force)
+    {
+        return force ? (target) -> target.nodetoolResult("decommission",  "--force").asserts().success()
+                     : (target) -> target.nodetoolResult("decommission").asserts().success();
+    }

Review comment:
       nit: Can we remove at least the unused imports to reduce a bit the noice of warnings?

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffNodetoolTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.List;
+
+import org.junit.AfterClass;
+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.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.hints.HintsService;
+import org.apache.cassandra.service.StorageProxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class HintedHandoffNodetoolTest extends TestBaseImpl

Review comment:
       Shall we add again java doc, similar to the previous test class, explanation that this was pytest?

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl

Review comment:
       nit: I would probably call it `HintedHandoffAddRemoveNodesTest`

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */

Review comment:
       Or maybe actually such comment to be added probably to the dtests because they are marked as ported so good to know probably why and until when the original pytests will be kept

##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       The "test framework" link you point to does not work, probably squashed commits?




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?
+    private void verifyReplication(UpgradeableCluster cluster, int nodes)
+    {
+        String INSERT = "INSERT INTO test_simple.names (key, name) VALUES (?, ?)";
+        String SELECT = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        List<Long> initialTokens = new ArrayList<>(nodes + 1);
+        initialTokens.add(null);
+
+        for (int i = 1; i <= nodes; i++)
+        {
+            initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+        }
+
+        // TODO: Pick some random keys to cover all three nodes as primaries.

Review comment:
       Instead of random we can use a bunch of increasing int keys, say one to ten (or more). The murmur hash makes the node selection virtually random, so with a few keys is almost sure that we are going to cover all the nodes.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                StringBuilder batchBuilder = new StringBuilder("BEGIN " + (isLogged ? "" : "UNLOGGED ") + "BATCH\n");

Review comment:
       > I've been looking for some tests somewhere that actually verify atomicity (without isolation, of course) in the case where a mutation in the batch fails, but I don't see any.
   
   That seems an important omission in dtests, being the kind of thing we want to detect and fix in this ticket. We might want to test batch atomicity in a regular not-upgrade dtest too.
   > It might be possible to parameterize the existing tests to optionally activate a ByteBuddy rule or network filter that fails batchlog writes. WDYT?
   
   A parameter to activate/deactivate a ByteBuddy rule seems to make sense, although I'm not sure whether the same rule will be compatible with all the previous versions, so the network filter might be more flexible. 




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.IMessageFilters;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+
+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.BATCH_STORE_REQ;
+import static org.apache.cassandra.net.Verb.REQUEST_RSP;
+
+public class MixedModeBatchTestBase extends UpgradeTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            if (isLogged)
+            {
+                // If we're testing logged batches, exercise the case were batchlog writes fail.
+                IMessageFilters.Filter dropBatchlogWrite = cluster.filters().inbound().verbs(BATCH_STORE_REQ.id, REQUEST_RSP.id).drop();
+                dropBatchlogWrite.on();
+                testBatches(isLogged, true, insert, select, cluster, upgraded);
+                cluster.filters().reset();
+            }
+
+            cluster.coordinator(1).execute("TRUNCATE test_simple.names", ConsistencyLevel.ALL);
+            testBatches(isLogged, false, insert, select, cluster, upgraded);
+            cluster.coordinator(1).execute("TRUNCATE test_simple.names", ConsistencyLevel.ALL);
+        })
+        .run();
+    }
+
+    private void testBatches(boolean isLogged, boolean failBatchlog, String insert, String select, UpgradeableCluster cluster, int upgraded)
+    {
+        List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+        initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+        for (int i = 1; i <= cluster.size(); i++)
+            initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+        List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);

Review comment:
       I think we don't need to put the `null` there, especially when we are removing it in the later call to `subList` to prepare a list valid for `primaryReplica`. We could just add them directly to the same list that we are going to pass to `primaryReplica`:
   ```suggestion
           List<Long> initialTokens = new ArrayList<>(cluster.size());
           for (int i = 1; i <= cluster.size(); i++)
               initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
   ```
   If anything, we could specify in the JavaDoc for `primaryReplica` that the parameter uses zero-based numbering but the result is one-based.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                StringBuilder batchBuilder = new StringBuilder("BEGIN " + (isLogged ? "" : "UNLOGGED ") + "BATCH\n");
+                String name = "Julia";
+                Runnable[] tests = new Runnable[KEYS_PER_BATCH];
+
+                // ...and sample enough keys that we cover the ring.
+                for (int j = 0; j < KEYS_PER_BATCH; j++)
+                {
+                    int key = j + (i * KEYS_PER_BATCH);
+                    batchBuilder.append(String.format(insert, key, name)).append('\n');
+
+                    // Track the test that will later verify that this mutation was replicated properly.
+                    tests[j] = () -> {
+                        Object[] row = row(key, name);
+                        Long token = tokenFrom(key);
+                        int node = calculatePrimary(validTokens, token);
+                        assertRows(cluster.get(node).executeInternal(select, key), row);
+
+                        node = nextNode(node, cluster.size());
+                        assertRows(cluster.get(node).executeInternal(select, key), row);
+
+                        // At RF=2, this node should not have received the write.
+                        node = nextNode(node, cluster.size());
+                        assertRows(cluster.get(node).executeInternal(select, key));

Review comment:
       TODO: These assertions probably need better supporting information on failure, similar to what happens below for writes.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       This block now exists only because of the in-JVM [test framework](https://github.com/apache/cassandra/pull/821/commits/a715388a1c7029f5a76e0061715856fe1c3bb391#diff-321b52220c5bd0aaadf275a845143eb208c889c2696ba0d48a5fc880551131d8R313). I could be persuaded to do something less invasive, like perhaps just exposing the encoded mutations, given the other elements are already accessible. (i.e. The `reserialize()` method would move to `Instance`.)
   
   CC @ifesdjeen @adelapena 




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable

Review comment:
       @adelapena I think this subsumes the mixed cluster cases in the Python dtests, so I'm planning on wholesale removing those eventually (i.e. the ones that don't replay the log). 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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.shared.Versions;
+
+public class MixedModeLoggedBatchTest extends MixedModeBatchTestBase
+{
+    @Test
+    public void testSimpleStrategy22to30() throws Throwable
+    {
+        testSimpleStrategy(Versions.Major.v22, Versions.Major.v30, true);

Review comment:
       Probably. I'll split things out a bit more though to avoid those pesky metaspace problems...




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */
+    @SuppressWarnings("Convert2MethodRef")
+    @Test
+    public void shouldStreamHintsDuringDecomission() throws Exception
+    {
+        try (Cluster cluster = builder().withNodes(4)
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP, NATIVE_PROTOCOL))
+                                        .start())
+        {
+            cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");

Review comment:
       Nit: Do we prefer the `withKeyspace` method? Not really a big difference, but feels a bit more flexible.

##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */
+    @SuppressWarnings("Convert2MethodRef")
+    @Test
+    public void shouldStreamHintsDuringDecomission() throws Exception
+    {
+        try (Cluster cluster = builder().withNodes(4)
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP, NATIVE_PROTOCOL))
+                                        .start())
+        {
+            cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".decom_hint_test (key int PRIMARY KEY, value int)");
+            
+            cluster.get(4).shutdown().get();
+            
+            // Write data using the second node as the coordinator...
+            populate(cluster, "decom_hint_test", 2, 0, 128, ConsistencyLevel.ONE);
+            Long totalHints = cluster.get(2).callOnInstance(() -> StorageMetrics.totalHints.getCount());

Review comment:
       Nit: we can put this is a separate method, so the `SuppressWarnings` is only needed on it:
   ```java
   @SuppressWarnings("Convert2MethodRef")
   private static long hintsCount(Cluster cluster, int node)
   {
       return cluster.get(node).callOnInstance(() -> StorageMetrics.totalHints.getCount());
   }
   ```




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable

Review comment:
       Agree, we don't really need to test a lot of consistency levels 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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-760524407


   Current CI status: https://app.circleci.com/pipelines/github/maedhroz/cassandra/193/workflows/6cf141e0-ad68-4559-9e61-334660dd61e4
   


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
##########
@@ -166,6 +166,9 @@ public void run() throws Throwable
             if (nodesToUpgrade.isEmpty())
                 for (int n = 1; n <= nodeCount; n++)
                     nodesToUpgrade.add(n);
+                
+            if (configConsumer == null)
+                configConsumer = config -> config.set("auto_bootstrap", true);

Review comment:
       CC @ifesdjeen




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   Current CI status: https://app.circleci.com/pipelines/github/maedhroz/cassandra/193/workflows/6cf141e0-ad68-4559-9e61-334660dd61e4
   


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?
+    private void verifyReplication(UpgradeableCluster cluster, int nodes)

Review comment:
       Maybe we can get rid of the `nodes` argument and make a `NUM_NODES` constant for the class. Otherwise, the assertions below should probably be based on the `nodes`/`numNodes` argument. Also, the number of nodes can be get from `cluster.size()`.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/206/workflows/91c0a021-8122-449c-8bbf-ef14533b36dc


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java
##########
@@ -166,6 +166,9 @@ public void run() throws Throwable
             if (nodesToUpgrade.isEmpty())
                 for (int n = 1; n <= nodeCount; n++)
                     nodesToUpgrade.add(n);
+                
+            if (configConsumer == null)
+                configConsumer = config -> config.set("auto_bootstrap", true);

Review comment:
       @ekaterinadimitrova2 @adelapena This is wild. I'll have more commentary in CASSANDRA-16387 soon, but I don't know how it's ever been safe for `AbstractCluster#startup()` to start multiple nodes concurrently. 3.0 nodes trying to create the traces keyspace concurrently seem to be able message each other with schema changes before the other creates the local entry in the `Gossiper` endpoint state map, and you get ugly assertions.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +127,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException

Review comment:
       I still think it would we useful to have a comment about the purpose of the method, probably very similar to the comment in the `Instance#serializeMessage` method where `reserialize` is called.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
##########
@@ -316,8 +320,37 @@ public void testSimpleStrategyThrowsConfigurationException() throws Configuratio
 
         Map<String, String> configOptions = new HashMap<>();
 
+        @SuppressWarnings("unused")
         SimpleStrategy strategy = new SimpleStrategy("ks", metadata, snitch, configOptions);
     }
+    
+    @Test
+    public void shouldReturnNoEndpointsForEmptyRing()
+    {
+        TokenMetadata metadata = new TokenMetadata();
+        
+        HashMap<String, String> configOptions = new HashMap<>();
+        configOptions.put("replication_factor", "1");
+        
+        SimpleStrategy strategy = new SimpleStrategy("ks", metadata, new SimpleSnitch(), configOptions);
+
+        EndpointsForRange replicas = strategy.calculateNaturalReplicas(null, metadata);
+        assertTrue(replicas.endpoints().isEmpty());
+    }
+
+    @Test
+    public void shouldWarnOnHigherReplicationFactorThanNodes()

Review comment:
       There is already coverage for this in [`AlterTest#testCreateAlterKeyspacesRFWarnings`](https://github.com/apache/cassandra/blob/trunk/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java#L279-L329), but it's nice to have it too. We could add an analogous test for `NetworkTopologyStrategyTest`:
   ```java
   @Test
   public void shouldWarnOnHigherReplicationFactorThanNodes()
   {
       IEndpointSnitch snitch = new SimpleSnitch();
   
       Map<String, String> configOptions = new HashMap<>();
       configOptions.put(snitch.getDatacenter((InetAddressAndPort) null), "2");
   
       NetworkTopologyStrategy strategy = new NetworkTopologyStrategy("ks", new TokenMetadata(), snitch, configOptions);
   
       StorageService.instance.getTokenMetadata().updateHostId(UUID.randomUUID(), FBUtilities.getBroadcastAddressAndPort());
       ClientWarn.instance.captureWarnings();
       strategy.maybeWarnOnOptions();
       assertTrue(ClientWarn.instance.getWarnings().stream().anyMatch(s -> s.contains("Your replication factor")));
   }
   ```




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
##########
@@ -55,24 +55,53 @@ public void testBatchAndList() throws Throwable
         createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>)");
 
         execute("BEGIN BATCH " +
-                "UPDATE %1$s SET l = l +[ 1 ] WHERE k = 0; " +
-                "UPDATE %1$s SET l = l + [ 2 ] WHERE k = 0; " +
-                "UPDATE %1$s SET l = l + [ 3 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [1] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [2] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [3] WHERE k = 0; " +
                 "APPLY BATCH");
 
         assertRows(execute("SELECT l FROM %s WHERE k = 0"),
                    row(list(1, 2, 3)));
 
         execute("BEGIN BATCH " +
-                "UPDATE %1$s SET l =[ 1 ] + l WHERE k = 1; " +
-                "UPDATE %1$s SET l = [ 2 ] + l WHERE k = 1; " +
-                "UPDATE %1$s SET l = [ 3 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [1] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [2] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [3] + l WHERE k = 1; " +
                 "APPLY BATCH ");
 
         assertRows(execute("SELECT l FROM %s WHERE k = 1"),
                    row(list(3, 2, 1)));
     }
 
+    @Test
+    public void testBatchAndMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<int, int>)");
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET m[1] = 1 WHERE k = 0; " +
+                "UPDATE %1$s SET m[2] = 2 WHERE k = 0; " +

Review comment:
       Nit: we could use different values for keys and values, to detect if they are somehow confused with each other, for example:
   ```java
   execute("BEGIN BATCH " +
           "UPDATE %1$s SET m[1] = 10 WHERE k = 0; " +
           "UPDATE %1$s SET m[2] = 20 WHERE k = 0; " +
           "APPLY BATCH");
   
   assertRows(execute("SELECT m FROM %s WHERE k = 0"),
              row(map(1, 10, 2, 20)));
   ```




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   Current CI status: https://app.circleci.com/pipelines/github/maedhroz/cassandra/192/workflows/ed5305e6-e4f9-420e-9f0a-6153333746dc


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?

Review comment:
       I'm on the fence w/ this one. I might leave this as it is for now and return to it once some of the other major areas for possible coverage expansion have been dealt with.




----------------------------------------------------------------
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 removed a comment on pull request #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

Posted by GitBox <gi...@apache.org>.
maedhroz removed a comment on pull request #821:
URL: https://github.com/apache/cassandra/pull/821#issuecomment-759839061


   Current CI status: https://app.circleci.com/pipelines/github/maedhroz/cassandra/192/workflows/ed5305e6-e4f9-420e-9f0a-6153333746dc


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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


   https://app.circleci.com/pipelines/github/maedhroz/cassandra/204/workflows/1fefb1bb-e9b0-4cd3-8b2a-5883bb0ac3e5


----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.StorageService;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.cassandra.distributed.action.GossipHelper.decomission;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests around removing and adding nodes from and to a cluster while hints are still outstanding.
+ */
+public class HintedHandoffAddRemoveTest extends TestBaseImpl
+{
+    /**
+     * Replaces Python dtest {@code hintedhandoff_test.py:TestHintedHandoff.test_hintedhandoff_decom()}.
+     */

Review comment:
       JVM dtests can't be run with vnodes nor compression, so I understand that in general the migrated Python dtests are going to be kept until we have support for these configs in JVM dtests. Also, Python dtests should be kept for branches where we don't have JVM versions of them, which in the case of this ticket are those lower than 4.0.
   
   I don't think we need to comment this porting/migration policy in every dtest, if anything maybe we could add a general comment about this in `fixture_ported_to_in_jvm`?




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: src/java/org/apache/cassandra/batchlog/Batch.java
##########
@@ -113,6 +120,22 @@ public void serialize(Batch batch, DataOutputPlus out, int version) throws IOExc
             }
         }
 
+        @VisibleForTesting
+        public void reserialize(Batch batch, DataOutputPlus out, int version) throws IOException
+        {
+            assert !batch.isLocal() : "attempted to reserialize a 'local' batch";
+
+            UUIDSerializer.serializer.serialize(batch.id, out, version);
+            out.writeLong(batch.creationTime);
+
+            out.writeUnsignedVInt(batch.encodedMutations.size());
+            
+            for (ByteBuffer mutation : batch.encodedMutations)
+            {
+                out.write(mutation);
+            }
+        }

Review comment:
       See https://github.com/apache/cassandra/pull/821/files?file-filters%5B%5D=.java#diff-321b52220c5bd0aaadf275a845143eb208c889c2696ba0d48a5fc880551131d8R339
   
   @adelapena @ekaterinadimitrova2 Given the only user is `Instance` at this point, I'm fine with just exposing the encoded mutations, if I can get that to work.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeBatchTestBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.exceptions.WriteFailureException;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class MixedModeBatchTestBase extends MixedModeReplicationTestBase
+{
+    private static final int KEYS_PER_BATCH = 10;
+
+    protected void testSimpleStrategy(Versions.Major from, Versions.Major to, boolean isLogged) throws Throwable
+    {
+        String insert = "INSERT INTO test_simple.names (key, name) VALUES (%d, '%s')";
+        String select = "SELECT * FROM test_simple.names WHERE key = ?";
+
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(from, to)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, upgraded) -> {
+            List<Long> initialTokens = new ArrayList<>(cluster.size() + 1);
+            initialTokens.add(null); // The first valid token is at 1 to avoid offset math below.
+
+            for (int i = 1; i <= cluster.size(); i++)
+                initialTokens.add(Long.valueOf(cluster.get(i).config().get("initial_token").toString()));
+
+            List<Long> validTokens = initialTokens.subList(1, cluster.size() + 1);
+
+            // Exercise all the coordinators...
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                StringBuilder batchBuilder = new StringBuilder("BEGIN " + (isLogged ? "" : "UNLOGGED ") + "BATCH\n");
+                String name = "Julia";
+                Runnable[] tests = new Runnable[KEYS_PER_BATCH];
+
+                // ...and sample enough keys that we cover the ring.
+                for (int j = 0; j < KEYS_PER_BATCH; j++)
+                {
+                    int key = j + (i * KEYS_PER_BATCH);
+                    batchBuilder.append(String.format(insert, key, name)).append('\n');
+
+                    // Track the test that will later verify that this mutation was replicated properly.
+                    tests[j] = () -> {
+                        Object[] row = row(key, name);
+                        Long token = tokenFrom(key);
+                        int node = calculatePrimary(validTokens, token);
+                        assertRows(cluster.get(node).executeInternal(select, key), row);
+
+                        node = nextNode(node, cluster.size());
+                        assertRows(cluster.get(node).executeInternal(select, key), row);
+
+                        // At RF=2, this node should not have received the write.
+                        node = nextNode(node, cluster.size());
+                        assertRows(cluster.get(node).executeInternal(select, key));

Review comment:
       Going to resolve this. The trace is actually pretty good if one of these assertions fails.




----------------------------------------------------------------
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 #821: CASSANDRA-16181 - 4.0 Quality: Replication Test Audit

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



##########
File path: test/distributed/org/apache/cassandra/distributed/upgrade/ReplicationUpgradeTest.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class ReplicationUpgradeTest extends UpgradeTestBase
+{
+    @Test
+    public void testSimpleStrategy() throws Throwable
+    {
+        new TestCase()
+        .nodes(3)
+        .nodesToUpgrade(1, 2)
+        .upgrade(Versions.Major.v22, Versions.Major.v30)
+        .upgrade(Versions.Major.v3X, Versions.Major.v4)
+        .upgrade(Versions.Major.v30, Versions.Major.v4)
+        .setup(cluster -> {
+            cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+            cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)");
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            verifyReplication(cluster, 3);
+        })
+        .run();
+    }
+
+    // TODO: Generalize this for other replication factors?
+    private void verifyReplication(UpgradeableCluster cluster, int nodes)

Review comment:
       > the number of nodes can be get from cluster.size()
   
   good point




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