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/02/05 16:42:48 UTC

[GitHub] [cassandra] ifesdjeen opened a new pull request #437: Extract in-jvm-dtest API

ifesdjeen opened a new pull request #437: Extract in-jvm-dtest API
URL: https://github.com/apache/cassandra/pull/437
 
 
   Patch by Alex Petrov for CASSANDRA-15539; reviewed by TBD.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395483853
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
+{
+    @Override
+    public <I extends IInstance, C extends ICluster> Builder<I, C> builder() {
+        // This is definitely not the smartest solution, but given the complexity of the alternatives and low risk, we can just rely on the
+        // fact that this code is going to work accross _all_ versions.
+        return (Builder<I, C>) Cluster.build();
+    }
+
+    public static void assertRows(ResultSet actual, Object[]... expected)
 
 Review comment:
   This was moved to `AssertBase`

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396604600
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
 ##########
 @@ -39,6 +39,7 @@
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 
 Review comment:
   dead code?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396696786
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java
 ##########
 @@ -107,7 +107,7 @@ private NodeToolResult repair(int node, String... args) {
         return DistributedRepairUtils.repair(CLUSTER, node, repairType, withNotifications, args);
     }
 
-    @Test(timeout = 1 * 60 * 1000)
 
 Review comment:
   why remove timeout?  this is going to conflict with CASSANDRA-15650

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398724588
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -231,13 +231,14 @@ private static void insert(ICoordinator coordinator, int start, int count)
     /**
      * returns a pair with [repair success, was inconsistent]
      */
-    private static IIsolatedExecutor.SerializableCallable<Pair<Boolean, Boolean>> repair(Map<String, String> options)
+    private static IIsolatedExecutor.SerializableCallable<RepairResult> repair(Map<String, String> options)
     {
         return () -> {
             SimpleCondition await = new SimpleCondition();
             AtomicBoolean success = new AtomicBoolean(true);
             AtomicBoolean wasInconsistent = new AtomicBoolean(false);
             StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
+                System.out.println(String.format("EVENT %s %s", event.getMessage(), event.getType()));
 
 Review comment:
   drop!

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395509699
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -197,11 +206,14 @@ public void validate()
             throw new IllegalArgumentException("In-JVM dtests do not support vnodes as of now.");
     }
 
-    private void propagate(Object writeToConfig, String fieldName, Object value, boolean ignoreMissing)
 
 Review comment:
   implemented the same behaviour via mapping..

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396698518
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -248,7 +249,15 @@ else if (event.getType() == ProgressEventType.NOTIFICATION && event.getMessage()
                     wasInconsistent.set(true);
                 }
                 else if (event.getType() == ProgressEventType.COMPLETE)
+                {
+                    await.signalAll();
+                }
+                else if (event.getType() == ProgressEventType.SUCCESS)
 
 Review comment:
   why add a success when this is followed by complete?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398721524
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
 ##########
 @@ -29,31 +29,36 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.InstanceConfig;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 import org.apache.cassandra.utils.progress.ProgressEventType;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.cassandra.distributed.impl.ExecUtil.rethrow;
+import static org.apache.cassandra.distributed.test.ExecUtil.rethrow;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 
-public class RepairTest extends DistributedTestBase
+public class RepairTest extends TestBaseImpl
 {
     private static final String insert = withKeyspace("INSERT INTO %s.test (k, c1, c2) VALUES (?, 'value1', 'value2');");
     private static final String query = withKeyspace("SELECT k, c1, c2 FROM %s.test WHERE k = ?;");
-    private static Cluster cluster;
 
-    private static void insert(Cluster cluster, int start, int end, int ... nodes)
+    public static TestBaseImpl inst = new RepairTest();
 
 Review comment:
   Yes, sorry; I've copied these tests from the ported repository. Here it's not necesasry. 

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398738636
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -188,71 +188,29 @@ public void testMessageMatching() throws Throwable
                                                                Verb.MUTATION_REQ.id,
                                                                Verb.MUTATION_RSP.id));
 
-            for (boolean inbound : Arrays.asList(true, false))
-            {
-                counter.set(0);
-                // Reads and writes are going to time out in both directions
-                IMessageFilters.Filter filter = cluster.filters()
-                                                       .allVerbs()
-                                                       .inbound(inbound)
-                                                       .from(1)
-                                                       .to(2)
-                                                       .messagesMatching((from, to, msg) -> {
-                                                           // Decode and verify message on instance; return the result back here
-                                                           Integer id = cluster.get(1).callsOnInstance((IIsolatedExecutor.SerializableCallable<Integer>) () -> {
-                                                               Message decoded = Instance.deserializeMessage(msg);
-                                                               return (Integer) decoded.verb().id;
-                                                           }).call();
-                                                           Assert.assertTrue(verbs.contains(id));
-                                                           counter.incrementAndGet();
-                                                           return false;
-                                                       }).drop();
-
-                for (int i : new int[]{ 1, 2 })
-                    cluster.coordinator(i).execute(read, ConsistencyLevel.ALL);
-                for (int i : new int[]{ 1, 2 })
-                    cluster.coordinator(i).execute(write, ConsistencyLevel.ALL);
-
-                filter.off();
-                Assert.assertEquals(4, counter.get());
-            }
-        }
-    }
-
-    @Test
-    public void outboundBeforeInbound() throws Throwable
 
 Review comment:
   Hrm i thought i did it; must've missed this one

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399371095
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -63,17 +66,17 @@ public void simpleOutboundFiltersTest()
 
     private static void simpleFiltersTest(boolean inbound)
     {
-        int VERB1 = Verb.READ_REQ.id;
 
 Review comment:
   reverted to prev version

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395485027
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
 
 Review comment:
   In other repo you've asked to remove dependency of `DistributedTestBase` on junit, which I did. Now, however, we need `TestBaseImpl` for before/after hooks. Also, if you take a look at the test runner repo, you'll see that we do fake dependency injection using this class, relying on its stable API, to be able to run tests for all other versions.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396574061
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
 ##########
 @@ -34,6 +39,29 @@
 public class DistributedTestSnitch extends AbstractNetworkTopologySnitch
 {
     private static NetworkTopology mapping = null;
+    private static final BiMap<InetAddressAndPort, InetSocketAddress> cache = HashBiMap.create();
+
+    static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort)
 
 Review comment:
   yeah, looking at `com.google.common.collect.HashBiMap#seekByKey` you need to walk the linked list, so this could be problematic

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396717742
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -188,71 +188,29 @@ public void testMessageMatching() throws Throwable
                                                                Verb.MUTATION_REQ.id,
                                                                Verb.MUTATION_RSP.id));
 
-            for (boolean inbound : Arrays.asList(true, false))
-            {
-                counter.set(0);
-                // Reads and writes are going to time out in both directions
-                IMessageFilters.Filter filter = cluster.filters()
-                                                       .allVerbs()
-                                                       .inbound(inbound)
-                                                       .from(1)
-                                                       .to(2)
-                                                       .messagesMatching((from, to, msg) -> {
-                                                           // Decode and verify message on instance; return the result back here
-                                                           Integer id = cluster.get(1).callsOnInstance((IIsolatedExecutor.SerializableCallable<Integer>) () -> {
-                                                               Message decoded = Instance.deserializeMessage(msg);
-                                                               return (Integer) decoded.verb().id;
-                                                           }).call();
-                                                           Assert.assertTrue(verbs.contains(id));
-                                                           counter.incrementAndGet();
-                                                           return false;
-                                                       }).drop();
-
-                for (int i : new int[]{ 1, 2 })
-                    cluster.coordinator(i).execute(read, ConsistencyLevel.ALL);
-                for (int i : new int[]{ 1, 2 })
-                    cluster.coordinator(i).execute(write, ConsistencyLevel.ALL);
-
-                filter.off();
-                Assert.assertEquals(4, counter.get());
-            }
-        }
-    }
-
-    @Test
-    public void outboundBeforeInbound() throws Throwable
 
 Review comment:
   we shouldn't delete tests in this PR, can you add back?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384700000
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -128,96 +93,10 @@ public void readRepairTest() throws Throwable
         }
     }
 
-    @Test
-    public void readRepairTimeoutTest() throws Throwable
 
 Review comment:
   same, please don't delete tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399371270
 
 

 ##########
 File path: build.properties.default
 ##########
 @@ -1,4 +1,4 @@
 # Maven2 Repository Locations (you can override these in "build.properties" to point to a local proxy, e.g. Nexus)
 artifact.remoteRepository.central:     https://repo1.maven.org/maven2
 artifact.remoteRepository.apache:      https://repo.maven.apache.org/maven2
-
+artifact.remoteRepository.apacheSnapshots: https://repository.apache.org/content/groups/snapshots/
 
 Review comment:
   removed

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398725389
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -248,7 +249,15 @@ else if (event.getType() == ProgressEventType.NOTIFICATION && event.getMessage()
                     wasInconsistent.set(true);
                 }
                 else if (event.getType() == ProgressEventType.COMPLETE)
+                {
+                    await.signalAll();
+                }
+                else if (event.getType() == ProgressEventType.SUCCESS)
+                {
+                    success.set(true);
 
 Review comment:
   Removing my changes to this file

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396721726
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -73,9 +73,11 @@
 import org.apache.cassandra.service.StorageService;
 
 @RunWith(Parameterized.class)
-public class FailingRepairTest extends DistributedTestBase implements Serializable
+public class FailingRepairTest extends TestBaseImpl implements Serializable
 {
-    private static Cluster CLUSTER;
+    public static TestBaseImpl inst = new RepairTest();
 
 Review comment:
   please remove

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398738370
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
 ##########
 @@ -32,44 +32,44 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
 import org.apache.cassandra.distributed.impl.TracingUtil;
 import org.apache.cassandra.utils.UUIDGen;
 
-public class MessageForwardingTest extends DistributedTestBase
+public class MessageForwardingTest extends TestBaseImpl
 {
     @Test
     public void mutationsForwardedToAllReplicasTest()
     {
         String originalTraceTimeout = TracingUtil.setWaitForTracingEventTimeoutSecs("1");
         final int numInserts = 100;
-        Map<InetAddress,Integer> forwardFromCounts = new HashMap<>();
-        Map<InetAddress,Integer> commitCounts = new HashMap<>();
+        Map<InetAddress, Integer> forwardFromCounts = new HashMap<>();
+        Map<InetAddress, Integer> commitCounts = new HashMap<>();
 
-        try (Cluster cluster = init(Cluster.build()
-                                           .withDC("dc0", 1)
-                                           .withDC("dc1", 3)
-                                           .start()))
+        try (Cluster cluster = (Cluster) init(builder()
+                                              .withDC("dc0", 1)
+                                              .withDC("dc1", 3)
+                                              .start()))
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v text, PRIMARY KEY (pk, ck))");
 
-            cluster.forEach(instance -> commitCounts.put(instance.broadcastAddressAndPort().address, 0));
+            cluster.forEach(instance -> commitCounts.put(instance.broadcastAddress().getAddress(), 0));
             final UUID sessionId = UUIDGen.getTimeUUID();
             Stream<Future<Object[][]>> inserts = IntStream.range(0, numInserts).mapToObj((idx) ->
-                cluster.coordinator(1).asyncExecuteWithTracing(sessionId,
-                                                         "INSERT INTO " + KEYSPACE + ".tbl(pk,ck,v) VALUES (1, 1, 'x')",
-                                                         ConsistencyLevel.ALL)
+                                                                                         cluster.coordinator(1).asyncExecuteWithTracing(sessionId,
 
 Review comment:
   done 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396635025
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
 
 Review comment:
   > however, we need TestBaseImpl for before/after hooks
   
   Ok.
   
   > if you take a look at the test runner repo, you'll see that we do fake dependency injection using this class, relying on its stable API, to be able to run tests for all other versions.
   
   I'll need to look at that,  I don't see why you would need this "to be able to run tests for all other versions".
   
   > relying on its stable API,
   
   This isn't documented and maintaining that in all the branches can be the same issues we keep seeing with dtest compatibility.  Ill need to look at the other repo to see what's going on; won't block this patch on this.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398644256
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -286,9 +299,8 @@ public void receiveMessage(IMessage message)
             Message.Header header = messageIn.header;
             TraceState state = Tracing.instance.initializeFromMessage(header);
             if (state != null) state.trace("{} message received from {}", header.verb, header.from);
-            header.verb.stage.execute(() -> {
-                MessagingService.instance().inboundSink.accept(messageIn);
-            }, ExecutorLocals.create(state));
+            header.verb.stage.execute(ThrowingRunnable.toRunnable(() -> MessagingService.instance().inboundSink.accept(messageIn)),
 
 Review comment:
   Removed 

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398660739
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -179,16 +194,10 @@ private InstanceConfig forceSet(String fieldName, Object value)
         return this;
     }
 
-    public void propagateIfSet(Object writeToConfig, String fieldName)
-    {
-        if (params.containsKey(fieldName))
-            propagate(writeToConfig, fieldName, params.get(fieldName), true);
-    }
-
-    public void propagate(Object writeToConfig)
+    public void propagate(Object writeToConfig, Map<Class<?>, Function<Object, Object>> mapping)
 
 Review comment:
   I would leave it as-is for now, since this would require API change, and API library is already voted in :/

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396570973
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
 ##########
 @@ -34,6 +39,29 @@
 public class DistributedTestSnitch extends AbstractNetworkTopologySnitch
 {
     private static NetworkTopology mapping = null;
+    private static final BiMap<InetAddressAndPort, InetSocketAddress> cache = HashBiMap.create();
+
+    static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort)
 
 Review comment:
   shouldn't this need a lock?  `cache` isn't thread safe and is going to be mutated by many threads.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396604100
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -197,11 +206,14 @@ public void validate()
             throw new IllegalArgumentException("In-JVM dtests do not support vnodes as of now.");
     }
 
-    private void propagate(Object writeToConfig, String fieldName, Object value, boolean ignoreMissing)
 
 Review comment:
   what do you mean?  before we skipped now we fail?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395300279
 
 

 ##########
 File path: build.xml
 ##########
 @@ -541,8 +543,10 @@
 
           <dependency groupId="org.yaml" artifactId="snakeyaml" version="1.11"/>
           <dependency groupId="junit" artifactId="junit" version="4.12" />
+          <dependency groupId="org.mockito" artifactId="mockito-core" version="3.2.4" />
 
 Review comment:
   this is already on trunk, is something up with the rebase?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384694774
 
 

 ##########
 File path: src/java/org/apache/cassandra/gms/Gossiper.java
 ##########
 @@ -1578,7 +1578,7 @@ public void start(int generationNbr, Map<ApplicationState, VersionedValue> prelo
         boolean isSeed = DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort());
         // We double RING_DELAY if we're not a seed to increase chance of successful startup during a full cluster bounce,
         // giving the seeds a chance to startup before we fail the shadow round
-        int shadowRoundDelay =  isSeed ? StorageService.RING_DELAY : StorageService.RING_DELAY * 2;
+        int shadowRoundDelay =  Integer.MAX_VALUE;
 
 Review comment:
   why?  this doesn't look like we should do that in this JIRA

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384697689
 
 

 ##########
 File path: test/unit/org/apache/cassandra/CassandraIsolatedJunit4ClassRunner.java
 ##########
 @@ -66,7 +67,7 @@ public CassandraIsolatedJunit4ClassRunner(Class<?> clazz) throws InitializationE
     {
         public CassandraIsolatedClassLoader()
         {
-            super(Versions.CURRENT.classpath);
+            super( new Versions.Version(FBUtilities.getReleaseVersionString(), Versions.getClassPath()).classpath);
 
 Review comment:
   we really should remove this dependency...  I won't force in this JIRA though, if you want to tackle outside can you file another JIRA?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395309422
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/shared/RepairResult.java
 ##########
 @@ -16,13 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.distributed.api;
+package org.apache.cassandra.distributed.shared;
 
-public interface IListen
+public class RepairResult
 
 Review comment:
   this diff is weird... you added this to replace the usage of `Pair` yet git thinks you replace IListen o_O

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398630166
 
 

 ##########
 File path: test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
 ##########
 @@ -316,6 +316,7 @@ public void testCreateAlterNetworkTopologyWithDefaults() throws Throwable
                    row("tbl1", map("class", "org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy",
                                   "min_threshold", "7",
                                   "max_threshold", "32")));
+        metadata.clearUnsafe();
 
 Review comment:
   It's not a weird rebase; I haven't rebased on top of my own commit.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398744306
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/CasWriteTest.java
 ##########
 @@ -54,12 +54,16 @@
 import org.hamcrest.Description;
 
 import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.fail;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 
-public class CasWriteTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class CasWriteTest extends TestBaseImpl
 {
+    // for static initializers
+    private static TestBaseImpl inst = new CasWriteTest();
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399350125
 
 

 ##########
 File path: build.properties.default
 ##########
 @@ -1,4 +1,4 @@
 # Maven2 Repository Locations (you can override these in "build.properties" to point to a local proxy, e.g. Nexus)
 artifact.remoteRepository.central:     https://repo1.maven.org/maven2
 artifact.remoteRepository.apache:      https://repo.maven.apache.org/maven2
-
+artifact.remoteRepository.apacheSnapshots: https://repository.apache.org/content/groups/snapshots/
 
 Review comment:
   spoke in slack, please remove; I am fine if on-commit you drop but we should avoid snapshots

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395485027
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
 
 Review comment:
   In other repo you've asked to remove dependency of `DistributedTestBase` on junit, which I did. Now, however, we need `TestBaseImpl` for before/after hooks. Also, if you take a look at the test runner repo, you'll see that we do fake dependency injection using this class, relying on its stable API, to be able to run tests for all other repos.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384694132
 
 

 ##########
 File path: in-jvm-dtest-api/src/main/java/org/apache/cassandra/distributed/shared/Versions.java
 ##########
 @@ -182,7 +156,7 @@ public static Versions find()
             Matcher m = pattern.matcher(file.getName());
             if (!m.matches())
                 continue;
-            String version = m.group("fullversion");
 
 Review comment:
   this looks like you are regressing?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398741682
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -128,15 +124,19 @@ private static void simpleFiltersTest(boolean inbound)
         Assert.assertEquals(2, counter.get());
     }
 
-    private static IMessage msg(int verb, String msg)
 
 Review comment:
   sorry; merge issues

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384693147
 
 

 ##########
 File path: build.properties.default
 ##########
 @@ -1,4 +1,4 @@
 # Maven2 Repository Locations (you can override these in "build.properties" to point to a local proxy, e.g. Nexus)
 artifact.remoteRepository.central:     https://repo1.maven.org/maven2
 artifact.remoteRepository.apache:      https://repo.maven.apache.org/maven2
-
+artifact.remoteRepository.apache2:     https://repository.apache.org/content/groups/snapshots/
 
 Review comment:
   This looks like the snapshots version and not the release version; maybe rename apache2 to apacheSnapshot?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396696468
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorSlow.java
 ##########
 @@ -52,7 +51,7 @@ public RepairCoordinatorSlow(RepairType repairType, RepairParallelism parallelis
         super(repairType, parallelism, withNotifications);
     }
 
-    @Test(timeout = 1 * 60 * 1000)
 
 Review comment:
   why did you remove the timeout?  this is also going to conflict with CASSANDRA-15650

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396699532
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/NativeProtocolTest.java
 ##########
 @@ -18,58 +18,61 @@
 
 package org.apache.cassandra.distributed.test;
 
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
 import com.datastax.driver.core.ConsistencyLevel;
 import com.datastax.driver.core.ResultSet;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.SimpleStatement;
 import com.datastax.driver.core.Statement;
-import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.RowUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Iterator;
+import org.apache.cassandra.distributed.api.ICluster;
 
 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.*;
 
-public class NativeProtocolTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class NativeProtocolTest extends TestBaseImpl
 {
 
     @Test
     public void withClientRequests() throws Throwable
     {
-        try (Cluster ignored = init(Cluster.create(3,
-                                                   config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))))
+        try (ICluster ignored = init(builder().withNodes(3)
+                                              .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))
+                                              .start()))
         {
-            final com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
-            Session session = cluster.connect();
-            session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck));");
-            session.execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) values (1,1,1);");
-            Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
-            final ResultSet resultSet = session.execute(select);
-            assertRows(resultSet, row(1, 1, 1));
-            Assert.assertEquals(3, cluster.getMetadata().getAllHosts().size());
-            session.close();
-            cluster.close();
+
+            try (com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
+                 Session session = cluster.connect())
+            {
+                session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck));");
+                session.execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) values (1,1,1);");
+                Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
+                final ResultSet resultSet = session.execute(select);
+                assertRows(RowUtil.toObjects(resultSet), row(1, 1, 1));
+                Assert.assertEquals(3, cluster.getMetadata().getAllHosts().size());
+            }
         }
     }
 
     @Test
     public void withCounters() throws Throwable
     {
-        try (Cluster dtCluster = init(Cluster.create(3,
-                config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))))
+        try (ICluster ignored = init(builder().withNodes(3)
+                                              .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))
+                                              .start()))
         {
             final com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
             Session session = cluster.connect();
             session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck counter, PRIMARY KEY (pk));");
             session.execute("UPDATE " + KEYSPACE + ".tbl set ck = ck + 10 where pk = 1;");
             Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
             final ResultSet resultSet = session.execute(select);
-            assertRows(resultSet, row(1, 10L));
+            assertRows(RowUtil.toObjects(resultSet), row(1, 10L));
 
 Review comment:
   why?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384700335
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java
 ##########
 @@ -0,0 +1,272 @@
+package org.apache.cassandra.distributed.test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class DistributedReadWritePathTest extends TestBaseImpl
 
 Review comment:
   this looks like you are based off old trunk, you are adding back something Benedict renamed...

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398728645
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/NativeProtocolTest.java
 ##########
 @@ -18,58 +18,61 @@
 
 package org.apache.cassandra.distributed.test;
 
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
 import com.datastax.driver.core.ConsistencyLevel;
 import com.datastax.driver.core.ResultSet;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.SimpleStatement;
 import com.datastax.driver.core.Statement;
-import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.RowUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Iterator;
+import org.apache.cassandra.distributed.api.ICluster;
 
 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.*;
 
-public class NativeProtocolTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class NativeProtocolTest extends TestBaseImpl
 {
 
     @Test
     public void withClientRequests() throws Throwable
     {
-        try (Cluster ignored = init(Cluster.create(3,
-                                                   config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))))
+        try (ICluster ignored = init(builder().withNodes(3)
+                                              .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))
+                                              .start()))
         {
-            final com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
-            Session session = cluster.connect();
-            session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck));");
-            session.execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) values (1,1,1);");
-            Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
-            final ResultSet resultSet = session.execute(select);
-            assertRows(resultSet, row(1, 1, 1));
-            Assert.assertEquals(3, cluster.getMetadata().getAllHosts().size());
-            session.close();
-            cluster.close();
+
+            try (com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
+                 Session session = cluster.connect())
+            {
+                session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck));");
+                session.execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) values (1,1,1);");
+                Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
+                final ResultSet resultSet = session.execute(select);
+                assertRows(RowUtil.toObjects(resultSet), row(1, 1, 1));
+                Assert.assertEquals(3, cluster.getMetadata().getAllHosts().size());
+            }
         }
     }
 
     @Test
     public void withCounters() throws Throwable
     {
-        try (Cluster dtCluster = init(Cluster.create(3,
-                config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))))
+        try (ICluster ignored = init(builder().withNodes(3)
+                                              .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL))
+                                              .start()))
         {
             final com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1").build();
             Session session = cluster.connect();
             session.execute("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck counter, PRIMARY KEY (pk));");
             session.execute("UPDATE " + KEYSPACE + ".tbl set ck = ck + 10 where pk = 1;");
             Statement select = new SimpleStatement("select * from " + KEYSPACE + ".tbl;").setConsistencyLevel(ConsistencyLevel.ALL);
             final ResultSet resultSet = session.execute(select);
-            assertRows(resultSet, row(1, 10L));
+            assertRows(RowUtil.toObjects(resultSet), row(1, 10L));
 
 Review comment:
   In short, for some reason static imports didn't work well with a static method in the parents class and compiler couldn't figure out which `assertRows` I was referring to. And I couldn't add `assertRows(ResultSet)` to the parent class because it has a driver dependency. 

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398744160
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -73,9 +73,11 @@
 import org.apache.cassandra.service.StorageService;
 
 @RunWith(Parameterized.class)
-public class FailingRepairTest extends DistributedTestBase implements Serializable
+public class FailingRepairTest extends TestBaseImpl implements Serializable
 {
-    private static Cluster CLUSTER;
+    public static TestBaseImpl inst = new RepairTest();
 
 Review comment:
   Done 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396603356
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -179,16 +194,10 @@ private InstanceConfig forceSet(String fieldName, Object value)
         return this;
     }
 
-    public void propagateIfSet(Object writeToConfig, String fieldName)
-    {
-        if (params.containsKey(fieldName))
-            propagate(writeToConfig, fieldName, params.get(fieldName), true);
-    }
-
-    public void propagate(Object writeToConfig)
+    public void propagate(Object writeToConfig, Map<Class<?>, Function<Object, Object>> mapping)
 
 Review comment:
   also because of class loaders `Class` equality can be iffy, so the mapping thing only makes sense IN class loader, so unsafe to expose to users.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396699181
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/NetworkTopologyTest.java
 ##########
 @@ -61,9 +62,9 @@ public void namedDcTest() throws Throwable
     public void automaticNamedDcTest() throws Throwable
 
     {
-        try (Cluster cluster = Cluster.build()
-                                      .withRacks(2, 1, 3)
-                                      .start())
+        try (ICluster cluster = builder()
+                                .withRacks(2, 1, 3)
 
 Review comment:
   why remove the space?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398726697
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/NetworkTopologyTest.java
 ##########
 @@ -61,9 +62,9 @@ public void namedDcTest() throws Throwable
     public void automaticNamedDcTest() throws Throwable
 
     {
-        try (Cluster cluster = Cluster.build()
-                                      .withRacks(2, 1, 3)
-                                      .start())
+        try (ICluster cluster = builder()
+                                .withRacks(2, 1, 3)
 
 Review comment:
   Formatted the way it was before

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396696641
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFast.java
 ##########
 @@ -51,7 +51,7 @@ public RepairCoordinatorFast(RepairType repairType, RepairParallelism parallelis
         super(repairType, parallelism, withNotifications);
     }
 
-    @Test(timeout = 1 * 60 * 1000)
+    @Test
 
 Review comment:
   why remove the timeout?  this is going to conflict with CASSANDRA-15650

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395300000
 
 

 ##########
 File path: build.xml
 ##########
 @@ -541,8 +543,10 @@
 
           <dependency groupId="org.yaml" artifactId="snakeyaml" version="1.11"/>
           <dependency groupId="junit" artifactId="junit" version="4.12" />
+          <dependency groupId="org.mockito" artifactId="mockito-core" version="3.2.4" />
           <dependency groupId="org.quicktheories" artifactId="quicktheories" version="0.25" />
           <dependency groupId="com.google.code.java-allocation-instrumenter" artifactId="java-allocation-instrumenter" version="${allocation-instrumenter.version}" />
+          <dependency groupId="org.apache.cassandra" artifactId="dtest-api" version="0.0.1-SNAPSHOT" />
 
 Review comment:
   snapshots are unsafe since a change to dtest now breaks cassandra...  should only rely on release.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395490700
 
 

 ##########
 File path: build.xml
 ##########
 @@ -541,8 +543,10 @@
 
           <dependency groupId="org.yaml" artifactId="snakeyaml" version="1.11"/>
           <dependency groupId="junit" artifactId="junit" version="4.12" />
+          <dependency groupId="org.mockito" artifactId="mockito-core" version="3.2.4" />
           <dependency groupId="org.quicktheories" artifactId="quicktheories" version="0.25" />
           <dependency groupId="com.google.code.java-allocation-instrumenter" artifactId="java-allocation-instrumenter" version="${allocation-instrumenter.version}" />
+          <dependency groupId="org.apache.cassandra" artifactId="dtest-api" version="0.0.1-SNAPSHOT" />
 
 Review comment:
   I will release a 0.0.1 before commit 

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396718797
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -128,15 +124,19 @@ private static void simpleFiltersTest(boolean inbound)
         Assert.assertEquals(2, counter.get());
     }
 
-    private static IMessage msg(int verb, String msg)
 
 Review comment:
   why remove the `private static`?  its now visible and slightly slower

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396719905
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -188,71 +188,29 @@ public void testMessageMatching() throws Throwable
                                                                Verb.MUTATION_REQ.id,
                                                                Verb.MUTATION_RSP.id));
 
-            for (boolean inbound : Arrays.asList(true, false))
 
 Review comment:
   why are you dropping the loop?  The test was testing both filters and now it only tests one?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398725255
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -248,7 +249,15 @@ else if (event.getType() == ProgressEventType.NOTIFICATION && event.getMessage()
                     wasInconsistent.set(true);
                 }
                 else if (event.getType() == ProgressEventType.COMPLETE)
+                {
+                    await.signalAll();
+                }
+                else if (event.getType() == ProgressEventType.SUCCESS)
+                {
+                    success.set(true);
 
 Review comment:
   I know; I can drop it if you insist, but I thouoght it was more intuitive.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398743457
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -136,36 +138,37 @@ public static void setupCluster() throws IOException
     {
         // streaming requires networking ATM
         // streaming also requires gossip or isn't setup properly
-        CLUSTER = init(Cluster.build(2)
-                    .withConfig(c -> c.with(Feature.NETWORK)
-                                      .with(Feature.GOSSIP)
-                                      .set("disk_failure_policy", "die"))
-                    .start());
+        cluster = init(inst.builder()
 
 Review comment:
   I've just tried to use same tests in both codebases; you're right it's not unreasonable. Fixed 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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384700000
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -128,96 +93,10 @@ public void readRepairTest() throws Throwable
         }
     }
 
-    @Test
-    public void readRepairTimeoutTest() throws Throwable
 
 Review comment:
   same, please don't delete tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399358835
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -63,17 +66,17 @@ public void simpleOutboundFiltersTest()
 
     private static void simpleFiltersTest(boolean inbound)
     {
-        int VERB1 = Verb.READ_REQ.id;
 
 Review comment:
   if trying to remove the import of Verb, can you document these numbers?  1, 2, and 3 are less readable than before

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398664854
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java
 ##########
 @@ -39,6 +39,7 @@
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 
 Review comment:
   It is dead, but I've used it a few times while developing this patch, and had to work around and use `Function` to emulate the same. I'd leave it since it's quite useful.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396561021
 
 

 ##########
 File path: build.xml
 ##########
 @@ -96,11 +96,11 @@
     <property name="maven-repository-url" value="https://repository.apache.org/content/repositories/snapshots"/>
     <property name="maven-repository-id" value="apache.snapshots.https"/>
 
-    <property name="test.timeout" value="240000" />
+    <property name="test.timeout" value="600000" />
     <property name="test.memory.timeout" value="480000" />
     <property name="test.long.timeout" value="600000" />
     <property name="test.burn.timeout" value="60000000" />
-    <property name="test.distributed.timeout" value="360000" />
+    <property name="test.distributed.timeout" value="600000" />
 
 Review comment:
   same, please revert.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396587171
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -179,16 +194,10 @@ private InstanceConfig forceSet(String fieldName, Object value)
         return this;
     }
 
-    public void propagateIfSet(Object writeToConfig, String fieldName)
-    {
-        if (params.containsKey(fieldName))
-            propagate(writeToConfig, fieldName, params.get(fieldName), true);
-    }
-
-    public void propagate(Object writeToConfig)
+    public void propagate(Object writeToConfig, Map<Class<?>, Function<Object, Object>> mapping)
 
 Review comment:
   should this be public?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396694374
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -19,35 +19,29 @@
 package org.apache.cassandra.distributed.test;
 
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
-import static org.apache.cassandra.net.Verb.READ_REPAIR_RSP;
-import static org.junit.Assert.assertEquals;
-
-import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD;
+import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ;
+import static org.apache.cassandra.net.Verb.READ_REPAIR_RSP;
 
-public class SimpleReadWriteTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class SimpleReadWriteTest extends TestBaseImpl
 {
-    @BeforeClass
 
 Review comment:
   why remove?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399350477
 
 

 ##########
 File path: build.xml
 ##########
 @@ -96,11 +96,11 @@
     <property name="maven-repository-url" value="https://repository.apache.org/content/repositories/snapshots"/>
     <property name="maven-repository-id" value="apache.snapshots.https"/>
 
-    <property name="test.timeout" value="240000" />
+    <property name="test.timeout" value="600000" />
 
 Review comment:
   spoke in slack, Alex will remove on-commit.  There are two JIRAs fixing so this is just to make things stable 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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398743563
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -136,36 +138,37 @@ public static void setupCluster() throws IOException
     {
         // streaming requires networking ATM
         // streaming also requires gossip or isn't setup properly
-        CLUSTER = init(Cluster.build(2)
-                    .withConfig(c -> c.with(Feature.NETWORK)
-                                      .with(Feature.GOSSIP)
-                                      .set("disk_failure_policy", "die"))
-                    .start());
+        cluster = init(inst.builder()
+                           .withNodes(2)
+                           .withConfig(c -> c.with(Feature.NETWORK)
+                                             .with(Feature.GOSSIP)
+                                             .set("disk_failure_policy", "die"))
+                           .start());
     }
 
     @AfterClass
-    public static void teardownCluster()
+    public static void teardownCluster() throws Exception
     {
-        if (CLUSTER != null)
-            CLUSTER.close();
+        if (cluster != null)
+            cluster.close();
     }
 
     @Before
     public void cleanupState()
     {
-        for (int i = 1; i <= CLUSTER.size(); i++)
-            CLUSTER.get(i).runOnInstance(() -> InstanceKiller.clear());
+        for (int i = 1; i <= cluster.size(); i++)
+            cluster.get(i).runOnInstance(InstanceKiller::clear);
     }
 
-    @Test(timeout = 10 * 60 * 1000)
 
 Review comment:
   Reverted.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395483754
 
 

 ##########
 File path: test/unit/org/apache/cassandra/CassandraIsolatedJunit4ClassRunner.java
 ##########
 @@ -66,7 +67,7 @@ public CassandraIsolatedJunit4ClassRunner(Class<?> clazz) throws InitializationE
     {
         public CassandraIsolatedClassLoader()
         {
-            super(Versions.CURRENT.classpath);
+            super( new Versions.Version(FBUtilities.getReleaseVersionString(), Versions.getClassPath()).classpath);
 
 Review comment:
   Remove the runner? Or remove these calls?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396635342
 
 

 ##########
 File path: test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
 ##########
 @@ -316,6 +316,7 @@ public void testCreateAlterNetworkTopologyWithDefaults() throws Throwable
                    row("tbl1", map("class", "org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy",
                                   "min_threshold", "7",
                                   "max_threshold", "32")));
+        metadata.clearUnsafe();
 
 Review comment:
   is this a ninja or something off with the rebase?  I see it in trunk so looks like a weird rebase

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398643561
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -105,9 +113,18 @@
 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.impl.DistributedTestSnitch.fromCassandraInetAddressAndPort;
+import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort;
 
 public class Instance extends IsolatedExecutor implements IInvokableInstance
 {
+    private static final Map<Class<?>, Function<Object, Object>> mapper = new HashMap<Class<?>, Function<Object, Object>>() {{
 
 Review 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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396725673
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/CasWriteTest.java
 ##########
 @@ -54,12 +54,16 @@
 import org.hamcrest.Description;
 
 import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.fail;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 
-public class CasWriteTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class CasWriteTest extends TestBaseImpl
 {
+    // for static initializers
+    private static TestBaseImpl inst = new CasWriteTest();
 
 Review comment:
   please remove

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384699569
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -389,30 +268,4 @@ public void pagingTests() throws Throwable
 
         }
     }
-
-    @Test
-    public void metricsCountQueriesTest() throws Throwable
 
 Review comment:
   please don't delete any tests.  if you want to move them into different files (some code can be extracted some can't) im ok with this, but gripping for `metricsCountQueriesTest` shows no matches other than this removal.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384699569
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -389,30 +268,4 @@ public void pagingTests() throws Throwable
 
         }
     }
-
-    @Test
-    public void metricsCountQueriesTest() throws Throwable
 
 Review comment:
   please don't delete any tests.  if you want to move them into different files (some code can be extracted some can't) im ok with this, but gripping for `metricsCountQueriesTest` shows no matches other than this removal.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395496376
 
 

 ##########
 File path: build.xml
 ##########
 @@ -541,8 +543,10 @@
 
           <dependency groupId="org.yaml" artifactId="snakeyaml" version="1.11"/>
           <dependency groupId="junit" artifactId="junit" version="4.12" />
+          <dependency groupId="org.mockito" artifactId="mockito-core" version="3.2.4" />
 
 Review comment:
   doesn't show up in the latest rebase

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398742341
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageFiltersTest.java
 ##########
 @@ -188,71 +188,29 @@ public void testMessageMatching() throws Throwable
                                                                Verb.MUTATION_REQ.id,
                                                                Verb.MUTATION_RSP.id));
 
-            for (boolean inbound : Arrays.asList(true, false))
 
 Review comment:
   reverted: bad merge

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396721515
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -136,36 +138,37 @@ public static void setupCluster() throws IOException
     {
         // streaming requires networking ATM
         // streaming also requires gossip or isn't setup properly
-        CLUSTER = init(Cluster.build(2)
-                    .withConfig(c -> c.with(Feature.NETWORK)
-                                      .with(Feature.GOSSIP)
-                                      .set("disk_failure_policy", "die"))
-                    .start());
+        cluster = init(inst.builder()
+                           .withNodes(2)
+                           .withConfig(c -> c.with(Feature.NETWORK)
+                                             .with(Feature.GOSSIP)
+                                             .set("disk_failure_policy", "die"))
+                           .start());
     }
 
     @AfterClass
-    public static void teardownCluster()
+    public static void teardownCluster() throws Exception
     {
-        if (CLUSTER != null)
-            CLUSTER.close();
+        if (cluster != null)
+            cluster.close();
     }
 
     @Before
     public void cleanupState()
     {
-        for (int i = 1; i <= CLUSTER.size(); i++)
-            CLUSTER.get(i).runOnInstance(() -> InstanceKiller.clear());
+        for (int i = 1; i <= cluster.size(); i++)
+            cluster.get(i).runOnInstance(InstanceKiller::clear);
     }
 
-    @Test(timeout = 10 * 60 * 1000)
 
 Review comment:
   Why remove the timeouts?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395485341
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java
 ##########
 @@ -0,0 +1,272 @@
+package org.apache.cassandra.distributed.test;
+
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class DistributedReadWritePathTest extends TestBaseImpl
 
 Review comment:
   Right; that got preserved the right one.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398724335
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java
 ##########
 @@ -107,7 +107,7 @@ private NodeToolResult repair(int node, String... args) {
         return DistributedRepairUtils.repair(CLUSTER, node, repairType, withNotifications, args);
     }
 
-    @Test(timeout = 1 * 60 * 1000)
 
 Review comment:
   Dropped this commit

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395491477
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/shared/RepairResult.java
 ##########
 @@ -16,13 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.distributed.api;
+package org.apache.cassandra.distributed.shared;
 
-public interface IListen
+public class RepairResult
 
 Review comment:
   Hm yes; not sure why / how this happened. 

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r399362928
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorSlow.java
 ##########
 @@ -52,7 +51,7 @@ public RepairCoordinatorSlow(RepairType repairType, RepairParallelism parallelis
         super(repairType, parallelism, withNotifications);
     }
 
-    @Test(timeout = 1 * 60 * 1000)
 
 Review comment:
   spoke in slack, please revert.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396560905
 
 

 ##########
 File path: build.xml
 ##########
 @@ -96,11 +96,11 @@
     <property name="maven-repository-url" value="https://repository.apache.org/content/repositories/snapshots"/>
     <property name="maven-repository-id" value="apache.snapshots.https"/>
 
-    <property name="test.timeout" value="240000" />
+    <property name="test.timeout" value="600000" />
 
 Review comment:
   please revert back, unrelated.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384698969
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
 
 Review comment:
   I don't see why we need this class.  I assume its because of `Cluster.build()` but even then thats a api thing and should fix the API rather than try to hide this.
   
   Can we remove this 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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398744289
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java
 ##########
 @@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.distributed.test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-import com.google.common.collect.Iterators;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-import com.datastax.driver.core.ResultSet;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.distributed.impl.AbstractCluster;
-import org.apache.cassandra.distributed.impl.IsolatedExecutor;
-import org.apache.cassandra.distributed.impl.RowUtil;
-
-public class DistributedTestBase
 
 Review comment:
   Let's postpone it to the next release

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398645063
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -386,13 +400,15 @@ public void startup(ICluster cluster)
                 }
                 registerInboundFilter(cluster);
                 registerOutboundFilter(cluster);
+
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
                 // TODO: this is more than just gossip
                 if (config.has(GOSSIP))
                 {
                     StorageService.instance.initServer();
                     StorageService.instance.removeShutdownHook();
+                    Gossiper.waitToSettle();
 
 Review comment:
   I don't remember the details anymore, but it did seem to help with one of the flakiness of one of the tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396580156
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -286,9 +299,8 @@ public void receiveMessage(IMessage message)
             Message.Header header = messageIn.header;
             TraceState state = Tracing.instance.initializeFromMessage(header);
             if (state != null) state.trace("{} message received from {}", header.verb, header.from);
-            header.verb.stage.execute(() -> {
-                MessagingService.instance().inboundSink.accept(messageIn);
-            }, ExecutorLocals.create(state));
+            header.verb.stage.execute(ThrowingRunnable.toRunnable(() -> MessagingService.instance().inboundSink.accept(messageIn)),
 
 Review comment:
   why did you wrap in a `ThrowingRunnable`?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396697812
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -231,13 +231,14 @@ private static void insert(ICoordinator coordinator, int start, int count)
     /**
      * returns a pair with [repair success, was inconsistent]
      */
-    private static IIsolatedExecutor.SerializableCallable<Pair<Boolean, Boolean>> repair(Map<String, String> options)
+    private static IIsolatedExecutor.SerializableCallable<RepairResult> repair(Map<String, String> options)
     {
         return () -> {
             SimpleCondition await = new SimpleCondition();
             AtomicBoolean success = new AtomicBoolean(true);
             AtomicBoolean wasInconsistent = new AtomicBoolean(false);
             StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> {
+                System.out.println(String.format("EVENT %s %s", event.getMessage(), event.getType()));
 
 Review comment:
   logger or drop?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384694855
 
 

 ##########
 File path: src/java/org/apache/cassandra/gms/Gossiper.java
 ##########
 @@ -1597,7 +1597,7 @@ public void start(int generationNbr, Map<ApplicationState, VersionedValue> prelo
             {
                 if (slept % 5000 == 0)
                 { // CASSANDRA-8072, retry at the beginning and every 5 seconds
-                    logger.trace("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
+                    logger.info("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
 
 Review comment:
   same, why is this changed in this JIRA?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384694230
 
 

 ##########
 File path: in-jvm-dtest-api/src/main/java/org/apache/cassandra/distributed/shared/Versions.java
 ##########
 @@ -192,7 +166,7 @@ public static Versions find()
             if (e.getValue().isEmpty())
                 continue;
             Collections.sort(e.getValue(), Comparator.comparing(v -> v.version, e.getKey()::compare));
-            logger.info("Found " + e.getValue().stream().map(v -> v.version).collect(Collectors.joining(", ")));
+            System.out.println("Found " + e.getValue().stream().map(v -> v.version).collect(Collectors.joining(", ")));
 
 Review comment:
   should use logger

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396724928
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java
 ##########
 @@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.distributed.test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-import com.google.common.collect.Iterators;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-import com.datastax.driver.core.ResultSet;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.distributed.impl.AbstractCluster;
-import org.apache.cassandra.distributed.impl.IsolatedExecutor;
-import org.apache.cassandra.distributed.impl.RowUtil;
-
-public class DistributedTestBase
 
 Review comment:
   honestly speaking, I feel that this class should stay with the tests.  since https://github.com/apache/cassandra-in-jvm-dtest-api is only the API at the moment it shouldn't have this 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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398666806
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/SimpleReadWriteTest.java
 ##########
 @@ -19,35 +19,29 @@
 package org.apache.cassandra.distributed.test;
 
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
 
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
-import static org.apache.cassandra.net.Verb.READ_REPAIR_RSP;
-import static org.junit.Assert.assertEquals;
-
-import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD;
+import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ;
+import static org.apache.cassandra.net.Verb.READ_REPAIR_RSP;
 
-public class SimpleReadWriteTest extends DistributedTestBase
+// TODO: this test should be removed after running in-jvm dtests is set up via the shared API repository
+public class SimpleReadWriteTest extends TestBaseImpl
 {
-    @BeforeClass
 
 Review comment:
   It was never needed there. Moreover, double-initialization might lead to some problems (seen someone mentioning it somewhere)
   

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384698090
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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 com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+import org.apache.cassandra.distributed.shared.Builder;
+import org.apache.cassandra.distributed.shared.DistributedTestBase;
+
+public class TestBaseImpl extends DistributedTestBase
+{
+    @Override
+    public <I extends IInstance, C extends ICluster> Builder<I, C> builder() {
+        // This is definitely not the smartest solution, but given the complexity of the alternatives and low risk, we can just rely on the
+        // fact that this code is going to work accross _all_ versions.
+        return (Builder<I, C>) Cluster.build();
+    }
+
+    public static void assertRows(ResultSet actual, Object[]... expected)
 
 Review comment:
   why not put in DistributedTestBase?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396700380
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/MessageForwardingTest.java
 ##########
 @@ -32,44 +32,44 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
 import org.apache.cassandra.distributed.Cluster;
 import org.apache.cassandra.distributed.impl.IsolatedExecutor;
 import org.apache.cassandra.distributed.impl.TracingUtil;
 import org.apache.cassandra.utils.UUIDGen;
 
-public class MessageForwardingTest extends DistributedTestBase
+public class MessageForwardingTest extends TestBaseImpl
 {
     @Test
     public void mutationsForwardedToAllReplicasTest()
     {
         String originalTraceTimeout = TracingUtil.setWaitForTracingEventTimeoutSecs("1");
         final int numInserts = 100;
-        Map<InetAddress,Integer> forwardFromCounts = new HashMap<>();
-        Map<InetAddress,Integer> commitCounts = new HashMap<>();
+        Map<InetAddress, Integer> forwardFromCounts = new HashMap<>();
+        Map<InetAddress, Integer> commitCounts = new HashMap<>();
 
-        try (Cluster cluster = init(Cluster.build()
-                                           .withDC("dc0", 1)
-                                           .withDC("dc1", 3)
-                                           .start()))
+        try (Cluster cluster = (Cluster) init(builder()
+                                              .withDC("dc0", 1)
+                                              .withDC("dc1", 3)
+                                              .start()))
         {
             cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v text, PRIMARY KEY (pk, ck))");
 
-            cluster.forEach(instance -> commitCounts.put(instance.broadcastAddressAndPort().address, 0));
+            cluster.forEach(instance -> commitCounts.put(instance.broadcastAddress().getAddress(), 0));
             final UUID sessionId = UUIDGen.getTimeUUID();
             Stream<Future<Object[][]>> inserts = IntStream.range(0, numInserts).mapToObj((idx) ->
-                cluster.coordinator(1).asyncExecuteWithTracing(sessionId,
-                                                         "INSERT INTO " + KEYSPACE + ".tbl(pk,ck,v) VALUES (1, 1, 'x')",
-                                                         ConsistencyLevel.ALL)
+                                                                                         cluster.coordinator(1).asyncExecuteWithTracing(sessionId,
 
 Review comment:
   can you fix the spacing?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396695310
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairTest.java
 ##########
 @@ -29,31 +29,36 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.distributed.Cluster;
-import org.apache.cassandra.distributed.impl.InstanceConfig;
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 import org.apache.cassandra.utils.progress.ProgressEventType;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.cassandra.distributed.impl.ExecUtil.rethrow;
+import static org.apache.cassandra.distributed.test.ExecUtil.rethrow;
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.apache.cassandra.distributed.shared.AssertUtils.*;
 
-public class RepairTest extends DistributedTestBase
+public class RepairTest extends TestBaseImpl
 {
     private static final String insert = withKeyspace("INSERT INTO %s.test (k, c1, c2) VALUES (?, 'value1', 'value2');");
     private static final String query = withKeyspace("SELECT k, c1, c2 FROM %s.test WHERE k = ?;");
-    private static Cluster cluster;
 
-    private static void insert(Cluster cluster, int start, int end, int ... nodes)
+    public static TestBaseImpl inst = new RepairTest();
 
 Review comment:
   why?  this looks like dead code to me.  Only reference looks to be https://github.com/apache/cassandra/blob/692138bbe6546a43d0e9960ee5cae45ad78a04ba/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java#L91 but since this class extends you could call builder directly.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395307623
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java
 ##########
 @@ -197,11 +206,14 @@ public void validate()
             throw new IllegalArgumentException("In-JVM dtests do not support vnodes as of now.");
     }
 
-    private void propagate(Object writeToConfig, String fieldName, Object value, boolean ignoreMissing)
 
 Review comment:
   why drop `ignoreMissing`?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r384695059
 
 

 ##########
 File path: src/java/org/apache/cassandra/gms/Gossiper.java
 ##########
 @@ -1607,7 +1607,10 @@ public void start(int generationNbr, Map<ApplicationState, VersionedValue> prelo
                     {
                         logger.trace("Sending shadow round GOSSIP DIGEST SYN to known peers {}", peers);
                         for (InetAddressAndPort peer : peers)
+                        {
+                            System.out.println("peer = " + peer);
 
 Review comment:
   avoid println in real code, this is also traced above

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398643000
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
 ##########
 @@ -34,6 +39,29 @@
 public class DistributedTestSnitch extends AbstractNetworkTopologySnitch
 {
     private static NetworkTopology mapping = null;
+    private static final BiMap<InetAddressAndPort, InetSocketAddress> cache = HashBiMap.create();
+
+    static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort)
 
 Review comment:
   I've replaced this with two concurrent hashmaps. Even though access to each map is not syncornised with its inverse, I think it's fine, since the worse case scenario we'll just create some extra objects.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395483032
 
 

 ##########
 File path: src/java/org/apache/cassandra/gms/Gossiper.java
 ##########
 @@ -1597,7 +1597,7 @@ public void start(int generationNbr, Map<ApplicationState, VersionedValue> prelo
             {
                 if (slept % 5000 == 0)
                 { // CASSANDRA-8072, retry at the beginning and every 5 seconds
-                    logger.trace("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
+                    logger.info("Sending shadow round GOSSIP DIGEST SYN to seeds {}", seeds);
 
 Review comment:
   THese are leftovers from messaging service problems debugging; shouldn've been here at all

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396580973
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -386,13 +400,15 @@ public void startup(ICluster cluster)
                 }
                 registerInboundFilter(cluster);
                 registerOutboundFilter(cluster);
+
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
                 // TODO: this is more than just gossip
                 if (config.has(GOSSIP))
                 {
                     StorageService.instance.initServer();
                     StorageService.instance.removeShutdownHook();
+                    Gossiper.waitToSettle();
 
 Review comment:
   was this why some alters looked flaky?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396576066
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -105,9 +113,18 @@
 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.impl.DistributedTestSnitch.fromCassandraInetAddressAndPort;
+import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort;
 
 public class Instance extends IsolatedExecutor implements IInvokableInstance
 {
+    private static final Map<Class<?>, Function<Object, Object>> mapper = new HashMap<Class<?>, Function<Object, Object>>() {{
 
 Review comment:
   partial functions in java... teehee =)

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r395299687
 
 

 ##########
 File path: build.properties.default
 ##########
 @@ -1,4 +1,4 @@
 # Maven2 Repository Locations (you can override these in "build.properties" to point to a local proxy, e.g. Nexus)
 artifact.remoteRepository.central:     https://repo1.maven.org/maven2
 artifact.remoteRepository.apache:      https://repo.maven.apache.org/maven2
-
+artifact.remoteRepository.apache2:     https://repository.apache.org/content/groups/snapshots/
 
 Review comment:
   can we rename this to be clear its for snapshots?

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398643000
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java
 ##########
 @@ -34,6 +39,29 @@
 public class DistributedTestSnitch extends AbstractNetworkTopologySnitch
 {
     private static NetworkTopology mapping = null;
+    private static final BiMap<InetAddressAndPort, InetSocketAddress> cache = HashBiMap.create();
+
+    static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort)
 
 Review comment:
   I've replaced this with two concurrent hashmaps. Even though access to each map is not syncornised with its inverse, I think it's fine, since the worse case scenario we'll just create one extra object.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396721136
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java
 ##########
 @@ -136,36 +138,37 @@ public static void setupCluster() throws IOException
     {
         // streaming requires networking ATM
         // streaming also requires gossip or isn't setup properly
-        CLUSTER = init(Cluster.build(2)
-                    .withConfig(c -> c.with(Feature.NETWORK)
-                                      .with(Feature.GOSSIP)
-                                      .set("disk_failure_policy", "die"))
-                    .start());
+        cluster = init(inst.builder()
 
 Review comment:
   why are you making this lower case?  not seen anything in C* style guide to say to do this and java convention is upper case.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r396698251
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java
 ##########
 @@ -248,7 +249,15 @@ else if (event.getType() == ProgressEventType.NOTIFICATION && event.getMessage()
                     wasInconsistent.set(true);
                 }
                 else if (event.getType() == ProgressEventType.COMPLETE)
+                {
+                    await.signalAll();
+                }
+                else if (event.getType() == ProgressEventType.SUCCESS)
+                {
+                    success.set(true);
 
 Review comment:
   this is the default.

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


With regards,
Apache Git Services

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


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #437: Extract in-jvm-dtest API: https://issues.apache.org/jira/browse/CASSANDRA-15539
URL: https://github.com/apache/cassandra/pull/437#discussion_r398666907
 
 

 ##########
 File path: test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
 ##########
 @@ -316,6 +316,7 @@ public void testCreateAlterNetworkTopologyWithDefaults() throws Throwable
                    row("tbl1", map("class", "org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy",
                                   "min_threshold", "7",
                                   "max_threshold", "32")));
+        metadata.clearUnsafe();
 
 Review comment:
   Rebased now; fixed

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


With regards,
Apache Git Services

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