You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "JeetKunDoug (via GitHub)" <gi...@apache.org> on 2023/06/05 13:14:48 UTC

[GitHub] [cassandra-sidecar] JeetKunDoug opened a new pull request, #48: Use in-jvm dtest framework for integration tests.

JeetKunDoug opened a new pull request, #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48

   This commit replaces the existing TestContainers-based test template with one that leverages Cassandra's in-jvm dtest framework now that it has JMX support. This will allow for more complete testing of endpoints, especially when those endpoints need to react to cluster changes like moves, joins, and leaves.


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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c closed pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c closed pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.
URL: https://github.com/apache/cassandra-sidecar/pull/48


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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1230216178


##########
client/build.gradle:
##########
@@ -40,6 +42,13 @@ test {
     testLogging {
         events "passed", "skipped", "failed"
     }
+    reports {

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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227353445


##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   Is it possible to drop those entirely? What is the scenario that we want to developers to run with other build tools? 



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227352125


##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.sidecar.common.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.ClusterMembershipOperations;
+import org.apache.cassandra.sidecar.common.ICassandraAdapter;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.TableOperations;
+import org.apache.cassandra.sidecar.common.dns.DnsResolver;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
+ */
+public class CassandraAdapter implements ICassandraAdapter

Review Comment:
   There will be more version supported. The adapters should have the version in the name at the beginning, instead of a refactoring (guaranteed) in the future. The 3.11 example in fact supports that we should have version in the class name in the case of adapting to multiple versions, except Sidecar does not support 3.11 so we drop it. 
   
   The concern in my original comment is not about conflicts. It is about clarity of the code, e.g. full qualified class name vs having version string in the simple class name, `org.apache.cassandra.sidecar.adapters.trunk.CassandraAdapter` vs. `Cassasndra50Adapter`.
   
   Btw, I think it is not a good idea to use `trunk`. It is the "pointer" that can change to other versions. Sidecar should use 50 for clarity. 



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228849085


##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.sidecar.common.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.ClusterMembershipOperations;
+import org.apache.cassandra.sidecar.common.ICassandraAdapter;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.TableOperations;
+import org.apache.cassandra.sidecar.common.dns.DnsResolver;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
+ */
+public class CassandraAdapter implements ICassandraAdapter

Review Comment:
   After much discussion, we decided to leave this as-is (although renaming `latest` to `base` in the package name)



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228850649


##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   Dropped



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227348549


##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   The reason is sidecar supports 4.0 and above. The default jar should be 4.0, so that compiler enforces it by building with it. 
   In order to track changes in 4.1 and trunk, there are integration test that runs with both configurations. 
   There are 2 goals of those integration tests.
   1. to catch any ABI-incompatible changes in Cassandra 4.1.
   2. to discover any 5.0 specific changes required and add it to the corresponding sidecar "adapter". 



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227331855


##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   I'll also update the README to note it's not needed if you run via Gradle.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227329563


##########
src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java:
##########
@@ -216,7 +220,7 @@ private String createTestTableAndPopulate(CassandraTestContext cassandraTestCont
     private void assertNotFoundOnDeleteSnapshot(VertxTestContext context, String testRoute) throws InterruptedException
     {
         WebClient client = WebClient.create(vertx);
-        client.delete(config.getPort(), "localhost", testRoute)
+        client.delete(config.getPort(), "127.0.0.1", testRoute)

Review Comment:
   Depends on the context - in a multi-node in-jvm test module you need IP addresses... in unit tests where we're mocking things it was set to `localhost` and changing it broke stuff, so I reverted some of them back. The two sets could likely be consolidated at some point but right now that's the split required to get tests to pass in all cases.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1224552509


##########
src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java:
##########
@@ -187,9 +187,9 @@ static class RingHandlerTestModule extends AbstractModule
         @Singleton
         public InstancesConfig instanceConfig() throws IOException
         {
-            final int instanceId = 100;
-            final String host = "127.0.0.1";
-            final InstanceMetadata instanceMetadata = mock(InstanceMetadata.class);
+            int instanceId = 100;
+            String host = "localhost";

Review Comment:
   why did this change? I'm wondering why unit tests needed to change. I would expect only integration tests to fail.



##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraStorageOperations.java:
##########
@@ -64,8 +73,28 @@ public void takeSnapshot(@NotNull String tag, @NotNull String keyspace, @NotNull
         requireNonNull(tag, "snapshot tag must be non-null");
         requireNonNull(keyspace, "keyspace for the  must be non-null");
         requireNonNull(table, "table must be non-null");
-        jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME)
-                 .takeSnapshot(tag, options, keyspace + "." + table);
+        try
+        {
+            jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME)
+                     .takeSnapshot(tag, options, keyspace + "." + table);
+        }
+        catch (IOException e)
+        {
+            if (StringUtils.contains(e.getMessage(),
+                                     "Snapshot " + tag + " already exists"))
+            {
+                throw new SnapshotAlreadyExistsException(e.getMessage());
+            }
+            else if (StringUtils.contains(e.getMessage(), "Keyspace " + keyspace + " does not exist"))
+            {
+                throw new IllegalArgumentException(e.getMessage());
+            }
+            else if (StringUtils.contains(e.getMessage(),
+                                          "Cannot snapshot until bootstrap completes"))
+            {
+                throw new NodeBootstrappingException(e.getMessage());
+            }

Review Comment:
   you are swallowing any other exception error message. We should propagate it anyway
   ```suggestion
               }
               throw new Runtime(e);
   ```



##########
src/test/java/org/apache/cassandra/sidecar/routes/SchemaHandlerTest.java:
##########
@@ -165,7 +165,7 @@ public class SchemaHandlerTestModule extends AbstractModule
         public InstancesConfig instanceConfig() throws IOException
         {
             final int instanceId = 100;
-            final String host = "127.0.0.1";
+            final String host = "localhost";

Review Comment:
   revert this change?



##########
src/test/java/org/apache/cassandra/sidecar/routes/ExtractHostAddressWithoutPortTest.java:
##########
@@ -30,15 +30,15 @@
     @Test
     void testAddressWithIPv4Host()
     {
-        final String host = AbstractHandler.extractHostAddressWithoutPort("127.0.0.1");
-        assertEquals("127.0.0.1", host);
+        final String host = AbstractHandler.extractHostAddressWithoutPort("localhost");

Review Comment:
   revert changes in this file? this is actually changing the test as described by the name of the test



##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   yeah, I think this is unnecessary. Don't we already have these in build.gradle? Someone interested in building it manually without using build tools can look inside build.gradle?



##########
adapters/latest/build.gradle:
##########
@@ -18,10 +19,11 @@ test {
 }
 
 dependencies {
-    implementation(project(":common"))
+    api(project(":common"))
+    api("com.google.guava:guava:${project.rootProject.guavaVersion}")
     compileOnly('org.jetbrains:annotations:23.0.0')
     compileOnly('com.datastax.cassandra:cassandra-driver-core:3.11.3')
-    implementation("com.google.guava:guava:${project.rootProject.guavaVersion}")
+    implementation("org.apache.commons:commons-lang3:3.9")

Review Comment:
   can we avoid adding this dependency? I generally get very nervous when we add new dependencies. So I would only add it if it's absolutely necessary.



##########
common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java:
##########
@@ -49,4 +49,5 @@
      * @return the {@link TableOperations} implementation for the Cassandra cluster
      */
     TableOperations tableOperations();
+

Review Comment:
   remove this line maybe?



##########
src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/BaseUploadsHandlerTest.java:
##########
@@ -181,7 +181,7 @@ public CassandraAdapterDelegate delegate()
         public Configuration abstractConfig(InstancesConfig instancesConfig)
         {
             when(mockConfiguration.getInstancesConfig()).thenReturn(instancesConfig);
-            when(mockConfiguration.getHost()).thenReturn("127.0.0.1");
+            when(mockConfiguration.getHost()).thenReturn("localhost");

Review Comment:
   can we revert the change in this file?



##########
src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java:
##########
@@ -129,7 +129,7 @@ static class GossipInfoHandlerTestModule extends AbstractModule
         public InstancesConfig instanceConfig() throws IOException
         {
             final int instanceId = 100;
-            final String host = "127.0.0.1";
+            final String host = "localhost";

Review Comment:
   revert this change?



##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.sidecar.common.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.ClusterMembershipOperations;
+import org.apache.cassandra.sidecar.common.ICassandraAdapter;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.TableOperations;
+import org.apache.cassandra.sidecar.common.dns.DnsResolver;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
+ */
+public class CassandraAdapter implements ICassandraAdapter

Review Comment:
   recollecting how the versions work, I think we annotate a factory with `@MinimumVersion("....")`. So if the minimum version is `4.0.0` that means the adapter is good for any version 4.0.0+, although there might be breaking changes in new versions in which case a new `ICassandraFactory` must be provided.
   
   The provided  factory in the code base  is good for `4.0.0`, `4.1.0`, and `5.0.0` looking at the test results. But the implementation we have was based on the minimum version of 4.0.0, so I think it would make more sense to have it be the `Cassandra40Factory`, which gives you a `Cassandra40Adapter`.
   
   I think we should test against trunk for sure, this will allow us to detect any breaking changes and would let us determine whether Sidecar is _forward-compatible_ with the tip of Cassandra's trunk.
   
   The question is what would be the expectation when latest is no longer compatible and we need to break it.
   
   - latest remains compatible with the tip of Cassandra's trunk
   - We introduce a Cassandra40Adapter then, that will work correctly with versions 4.0.0 through tip of trunk (not inclusive)



##########
src/test/java/org/apache/cassandra/sidecar/utils/SSTableImporterTest.java:
##########
@@ -67,7 +67,7 @@ public void setup() throws InterruptedException
         mockTableOperations1 = mock(TableOperations.class);
         TableOperations mockTableOperations2 = mock(TableOperations.class);
         when(mockConfiguration.getSSTableImportPollIntervalMillis()).thenReturn(10);
-        when(mockMetadataFetcher.delegate("127.0.0.1")).thenReturn(mockCassandraAdapterDelegate1);

Review Comment:
   can we undo the changes to this file? They seem unnecessary and unrelated to the integration tests. Any reason for these changes?



##########
src/test/java/org/apache/cassandra/sidecar/TestSslModule.java:
##########
@@ -58,7 +58,7 @@ public Configuration abstractConfig(InstancesConfig instancesConfig)
 
         return new Configuration.Builder()
                            .setInstancesConfig(instancesConfig)
-                           .setHost("127.0.0.1")
+                           .setHost("localhost")

Review Comment:
   revert this change?



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228863784


##########
src/test/java/org/apache/cassandra/sidecar/TestSslModule.java:
##########
@@ -58,7 +58,7 @@ public Configuration abstractConfig(InstancesConfig instancesConfig)
 
         return new Configuration.Builder()
                            .setInstancesConfig(instancesConfig)
-                           .setHost("127.0.0.1")
+                           .setHost("localhost")

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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228858330


##########
adapters/latest/build.gradle:
##########
@@ -18,10 +19,11 @@ test {
 }
 
 dependencies {
-    implementation(project(":common"))
+    api(project(":common"))
+    api("com.google.guava:guava:${project.rootProject.guavaVersion}")
     compileOnly('org.jetbrains:annotations:23.0.0')
     compileOnly('com.datastax.cassandra:cassandra-driver-core:3.11.3')
-    implementation("com.google.guava:guava:${project.rootProject.guavaVersion}")
+    implementation("org.apache.commons:commons-lang3:3.9")

Review Comment:
   There were already a ton of usages of commons-lang3 in other subprojects so I don't think it's a big deal - the other ones are actually depending on some transitive dependency to pull it in it seems.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227326791


##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   Why would the default be 4.0? We (@frankgh and I) had talked about this quite a bit and landed on using 4.1 and 5.0 for the default branches so we'd continue to track `trunk` and catch breakage faster.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1222176443


##########
README.md:
##########
@@ -27,9 +27,88 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+We rely on the in-jvm dtest framework for testing. You must manually build the dtest jars before you start integration tests.
+At the moment, the JMX feature is unreleased in Cassandra, so you can use the following to build from the PR branches:
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+```shell
+./scripts/build-dtest-jars.sh
+```
+
+The build script supports two parameters:
+- `REPO` - the Cassandra git repository to use for the source files. This is helpful if you need to test with a fork of the Cassandra codebase.
+  - default: `git@github.com:apache/cassandra.git`
+- `BRANCHES` - a space-delimited list of branches to build.
+  -default: `"cassandra-3.11 cassandra-4.1"`

Review Comment:
   Good point - we were originally going to default to 3.11 and 4.1 for testing, but ended up changing our minds, so I'll fix 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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227359469


##########
docs/src/development.adoc:
##########


Review Comment:
   At line#5, the "Building" section, we need to add a step `./scripts/build-dtest-jars.sh` before `./gradlew build`, in order for it to build successfully. 
   
   
   ```
   ## Building
       ./gradlew build
   ```



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228863253


##########
src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/BaseUploadsHandlerTest.java:
##########
@@ -181,7 +181,7 @@ public CassandraAdapterDelegate delegate()
         public Configuration abstractConfig(InstancesConfig instancesConfig)
         {
             when(mockConfiguration.getInstancesConfig()).thenReturn(instancesConfig);
-            when(mockConfiguration.getHost()).thenReturn("127.0.0.1");
+            when(mockConfiguration.getHost()).thenReturn("localhost");

Review Comment:
   Done



##########
src/test/java/org/apache/cassandra/sidecar/routes/SchemaHandlerTest.java:
##########
@@ -165,7 +165,7 @@ public class SchemaHandlerTestModule extends AbstractModule
         public InstancesConfig instanceConfig() throws IOException
         {
             final int instanceId = 100;
-            final String host = "127.0.0.1";
+            final String host = "localhost";

Review Comment:
   Done



##########
src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java:
##########
@@ -129,7 +129,7 @@ static class GossipInfoHandlerTestModule extends AbstractModule
         public InstancesConfig instanceConfig() throws IOException
         {
             final int instanceId = 100;
-            final String host = "127.0.0.1";
+            final String host = "localhost";

Review Comment:
   Done



##########
src/test/java/org/apache/cassandra/sidecar/routes/ExtractHostAddressWithoutPortTest.java:
##########
@@ -30,15 +30,15 @@
     @Test
     void testAddressWithIPv4Host()
     {
-        final String host = AbstractHandler.extractHostAddressWithoutPort("127.0.0.1");
-        assertEquals("127.0.0.1", host);
+        final String host = AbstractHandler.extractHostAddressWithoutPort("localhost");

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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227328663


##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraClusterMembershipOperations.java:
##########
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.sidecar.cassandra40;
+package org.apache.cassandra.sidecar.adapters.trunk;

Review Comment:
   See my previous comment about the "default" or "latest" implementation vs. forking a new implementation for some older version that doesn't support something. Calling this 40 doesn't make sense because it is, essentially, the _default_ implementation. Again, with 3.11 also in the codebase this would have been clearer.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227133813


##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.sidecar.common.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.ClusterMembershipOperations;
+import org.apache.cassandra.sidecar.common.ICassandraAdapter;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.TableOperations;
+import org.apache.cassandra.sidecar.common.dns.DnsResolver;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
+ */
+public class CassandraAdapter implements ICassandraAdapter

Review Comment:
   The class names of the version specific implementations should contain the version string. Although the package could contains the version string already, it is forced to have the full qualified names if the consumer happen to have classes of different versions. 
   
   So, `CassandraAdapter` should be `Cassandra40Adapter`. Similar goes for the other classes in the package. 



##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;

Review Comment:
   package name does not match with the file system structure.



##########
common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java:
##########
@@ -49,4 +49,5 @@
      * @return the {@link TableOperations} implementation for the Cassandra cluster
      */
     TableOperations tableOperations();
+

Review Comment:
   nit: unnecessary change



##########
common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java:
##########
@@ -206,4 +208,14 @@ private static JMXServiceURL buildJmxServiceURL(String host, int port)
             throw new RuntimeException(errorMessage, e);
         }
     }
+
+    public synchronized void close() throws IOException

Review Comment:
   add `@Override`



##########
src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java:
##########
@@ -216,7 +220,7 @@ private String createTestTableAndPopulate(CassandraTestContext cassandraTestCont
     private void assertNotFoundOnDeleteSnapshot(VertxTestContext context, String testRoute) throws InterruptedException
     {
         WebClient client = WebClient.create(vertx);
-        client.delete(config.getPort(), "localhost", testRoute)
+        client.delete(config.getPort(), "127.0.0.1", testRoute)

Review Comment:
   This changes `localhost` to `127.0.0.1`. The other change is the opposite direction. Why are they made so? 



##########
src/test/java/org/apache/cassandra/sidecar/TestModule.java:
##########
@@ -67,7 +67,7 @@ protected Configuration abstractConfig(InstancesConfig instancesConfig)
                                                                            30000);
         return new Configuration.Builder()
                .setInstancesConfig(instancesConfig)
-               .setHost("127.0.0.1")
+               .setHost("localhost")

Review Comment:
   This is the other change ("127.0.0.1" -> "localhost") I am talking about.



##########
src/main/java/org/apache/cassandra/sidecar/routes/SnapshotsHandler.java:
##########
@@ -256,38 +256,29 @@ private void processCreateSnapshotFailure(Throwable cause, RoutingContext contex
         logger.error("SnapshotsHandler failed for request={}, remoteAddress={}, instance={}, method={}",
                      requestParams, remoteAddress, host, context.request().method(), cause);
 
-        Throwable rootCause = cause instanceof UndeclaredThrowableException
-                              ? ((UndeclaredThrowableException) cause).getUndeclaredThrowable()
-                              : cause;
-
-        if (rootCause instanceof IOException)
+        if (cause instanceof SnapshotAlreadyExistsException)
         {
-            if (StringUtils.contains(rootCause.getMessage(),
-                                     "Snapshot " + requestParams.snapshotName() + " already exists"))
-            {
-                context.fail(wrapHttpException(HttpResponseStatus.CONFLICT, rootCause.getMessage()));
-                return;
-            }
-            else if (StringUtils.contains(rootCause.getMessage(),
-                                          "Cannot snapshot until bootstrap completes"))
-            {
-                // Cassandra does not allow taking snapshots while the node is JOINING the ring
-                context.fail(wrapHttpException(HttpResponseStatus.SERVICE_UNAVAILABLE,
-                                               "The Cassandra instance " + host + " is not available"));
-            }
+            context.fail(wrapHttpException(HttpResponseStatus.CONFLICT, cause.getMessage()));
+            return;
+        }
+        else if (cause instanceof NodeBootstrappingException)

Review Comment:
   👍



##########
src/test/integration/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java:
##########
@@ -62,7 +58,7 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
 {

Review Comment:
   There are a few "docker" in the file.



##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   It does not seem that I need to add those options individually to run integrationTest. They are configured as part of the `integrationTest` task in the build file already. 



##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraClusterMembershipOperations.java:
##########
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.sidecar.cassandra40;
+package org.apache.cassandra.sidecar.adapters.trunk;

Review Comment:
   Trunk is 5.0. It is concerning that there is only adapter for trunk, given the project support Cassandra 4.0+. 
   
   Effectively, there is not difference in supporting 4.0, 4.1 and trunk. It is probably the reason that the package name is changed to "latest". 
   I think it makes better sense to me that we keep the adapter name as "cassandra40" (the original name) for now, since it should works on the releases after 4.0.
   
   In the future, we could take Scala projects' layout as example, where the "scala" package contains the common implementation, and "scala-VERSION" packages contain the version specific overrides/implementation. But it is not an action item as of now.  



##########
common/src/main/java/org/apache/cassandra/sidecar/common/exceptions/NodeBootstrappingException.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.sidecar.common.exceptions;
+
+/**
+ * Exception thrown when a node is bootstrapping while an operation cannot be performed when the node is bootstrapping
+ */
+public class NodeBootstrappingException extends RuntimeException

Review Comment:
   How about `IllegalCassandraStateException`? The exception is thrown when then state of Cassandra is not expected. Regarding node vs cassandra, I'd prefer using "Cassandra", although at many places both are used interchangeably. 
   My understanding of `NodeBootstrappingException` is that there is some exception during node bootstrapping. It confused me at the beginning. 



##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   The default should be 4.0 jar



##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC
+-XX:+CMSParallelRemarkEnabled
+-XX:SurvivorRatio=8
+-XX:MaxTenuringThreshold=1
+-XX:CMSInitiatingOccupancyFraction=75
+-XX:+UseCMSInitiatingOccupancyOnly
+-XX:CMSWaitDuration=10000
+-XX:+CMSParallelInitialMarkEnabled
+-XX:+CMSEdenChunksRecordAlways
+--add-exports
+java.base/jdk.internal.misc=ALL-UNNAMED
+--add-exports
+java.base/jdk.internal.ref=ALL-UNNAMED
+--add-exports
+java.base/sun.nio.ch=ALL-UNNAMED
+--add-exports
+java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.registry=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.server=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.transport.tcp=ALL-UNNAMED
+--add-exports
+java.sql/java.sql=ALL-UNNAMED
+--add-opens
+java.base/java.lang.module=ALL-UNNAMED
+--add-opens
+java.base/java.net=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.loader=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.ref=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.reflect=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.math=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.module=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.util.jar=ALL-UNNAMED
+--add-opens
+jdk.management/com.sun.management.internal=ALL-UNNAMED
+-Dcassandra-foreground=yes
+-Dcassandra.config=file:///Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/cassandra.yaml
+-Dcassandra.jmx.local.port=7199
+-Dcassandra.logdir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/data/logs
+-Dcassandra.reads.thresholds.coordinator.defensive_checks_enabled=true
+-Dcassandra.storagedir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/data
+-Dcassandra.triggers_dir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/triggers
+-Djava.library.path=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/lib/sigar-bin
+-Dlogback.configurationFile=file:///Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/logback.xml

Review Comment:
   Normalize the paths?



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1222179958


##########
src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java:
##########
@@ -32,31 +36,59 @@
  */
 class DelegateTest
 {
+    private static CassandraAdapterDelegate getCassandraAdapterDelegate(CassandraTestContext context)
+    {
+        CassandraVersionProvider versionProvider = new CassandraVersionProvider.Builder()
+                                                       .add(new Cassandra311Factory(DnsResolver.DEFAULT))
+                                                       .add(new CassandraFactory(DnsResolver.DEFAULT))
+                                                       .build();
+        InstanceMetadata instanceMetadata = context.instancesConfig.instances().get(0);
+        CQLSessionProvider sessionProvider = new CQLSessionProvider(instanceMetadata.host(), instanceMetadata.port(), 1000);
+        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(versionProvider, sessionProvider, context.jmxClient());
+        return delegate;
+    }
+
     @CassandraIntegrationTest
     void testCorrectVersionIsEnabled(CassandraTestContext context)
     {
-        CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
+        CassandraAdapterDelegate delegate = getCassandraAdapterDelegate(context);
         SimpleCassandraVersion version = delegate.version();
         assertThat(version).isNotNull();
+        assertThat(version.major).isEqualTo(context.version.major);
+        assertThat(version.minor).isEqualTo(context.version.minor);
+        assertThat(version).isGreaterThanOrEqualTo(context.version);
     }
 
     @CassandraIntegrationTest
-    void testHealthCheck(CassandraTestContext context) throws IOException, InterruptedException
+    void testHealthCheck(CassandraTestContext context) throws InterruptedException
     {
-        CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
+        CassandraAdapterDelegate delegate = getCassandraAdapterDelegate(context);
 
         delegate.healthCheck();
 
         assertThat(delegate.isUp()).as("health check succeeds").isTrue();
 
-        context.container.execInContainer("nodetool", "disablebinary");
+        NodeToolResult nodetoolResult = context.cluster.get(1).nodetoolResult("disablebinary");
+        assertThat(nodetoolResult.getRc())
+        .withFailMessage("Failed to disable binary:\nstdout:" + nodetoolResult.getStdout()
+                         + "\nstderr: " + nodetoolResult.getStderr())
+        .isEqualTo(0);
 
-        delegate.healthCheck();
+        for (int i = 0; i < 10; i++)

Review Comment:
   During testing I think it always settled by 1 or 2, but I wasn't sure about how it would work in CI so I wanted to leave some extra time just in 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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] bbotella commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "bbotella (via GitHub)" <gi...@apache.org>.
bbotella commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1222138911


##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   Should we move this default to gradle.properties?



##########
README.md:
##########
@@ -27,9 +27,88 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+We rely on the in-jvm dtest framework for testing. You must manually build the dtest jars before you start integration tests.
+At the moment, the JMX feature is unreleased in Cassandra, so you can use the following to build from the PR branches:
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+```shell
+./scripts/build-dtest-jars.sh
+```
+
+The build script supports two parameters:
+- `REPO` - the Cassandra git repository to use for the source files. This is helpful if you need to test with a fork of the Cassandra codebase.
+  - default: `git@github.com:apache/cassandra.git`
+- `BRANCHES` - a space-delimited list of branches to build.
+  -default: `"cassandra-3.11 cassandra-4.1"`

Review Comment:
   Just a nit: I think the actual defaults on the script are different?



##########
src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java:
##########
@@ -32,31 +36,59 @@
  */
 class DelegateTest
 {
+    private static CassandraAdapterDelegate getCassandraAdapterDelegate(CassandraTestContext context)
+    {
+        CassandraVersionProvider versionProvider = new CassandraVersionProvider.Builder()
+                                                       .add(new Cassandra311Factory(DnsResolver.DEFAULT))
+                                                       .add(new CassandraFactory(DnsResolver.DEFAULT))
+                                                       .build();
+        InstanceMetadata instanceMetadata = context.instancesConfig.instances().get(0);
+        CQLSessionProvider sessionProvider = new CQLSessionProvider(instanceMetadata.host(), instanceMetadata.port(), 1000);
+        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(versionProvider, sessionProvider, context.jmxClient());
+        return delegate;
+    }
+
     @CassandraIntegrationTest
     void testCorrectVersionIsEnabled(CassandraTestContext context)
     {
-        CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
+        CassandraAdapterDelegate delegate = getCassandraAdapterDelegate(context);
         SimpleCassandraVersion version = delegate.version();
         assertThat(version).isNotNull();
+        assertThat(version.major).isEqualTo(context.version.major);
+        assertThat(version.minor).isEqualTo(context.version.minor);
+        assertThat(version).isGreaterThanOrEqualTo(context.version);
     }
 
     @CassandraIntegrationTest
-    void testHealthCheck(CassandraTestContext context) throws IOException, InterruptedException
+    void testHealthCheck(CassandraTestContext context) throws InterruptedException
     {
-        CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
+        CassandraAdapterDelegate delegate = getCassandraAdapterDelegate(context);
 
         delegate.healthCheck();
 
         assertThat(delegate.isUp()).as("health check succeeds").isTrue();
 
-        context.container.execInContainer("nodetool", "disablebinary");
+        NodeToolResult nodetoolResult = context.cluster.get(1).nodetoolResult("disablebinary");
+        assertThat(nodetoolResult.getRc())
+        .withFailMessage("Failed to disable binary:\nstdout:" + nodetoolResult.getStdout()
+                         + "\nstderr: " + nodetoolResult.getStderr())
+        .isEqualTo(0);
 
-        delegate.healthCheck();
+        for (int i = 0; i < 10; i++)

Review Comment:
   Just curious. Why 10?



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1222174438


##########
.circleci/config.yml:
##########
@@ -56,56 +57,14 @@ jobs:
      - store_test_results:
          path: ~/repo/build/test-results/
 
-  # Runs java 8 tests on a VM to be able to run Docker for integration tests
-  java8:
-    <<: *base_job
-
-    steps:
-      - checkout
-      - install_common
-
-      - install_java:
-          version: adoptopenjdk-8-hotspot
-      - run: sudo update-java-alternatives -s adoptopenjdk-8-hotspot-amd64 && java -version
-      - run: ./gradlew -i clean test integrationTest --stacktrace
-
-      - store_artifacts:
-          path: build/reports
-          destination: test-reports
-
-      - store_test_results:
-          path: ~/repo/build/test-results/
-
-      - store_test_results:
-          path: ~/repo/cassandra-integration-tests/build/test-results/
-
   # Runs java 11 tests on a docker image
-  java11_docker:
+  java11:
     docker:
       - image: circleci/openjdk:11-jdk-stretch
     steps:
       - checkout
 
-      - run: ./gradlew -i clean test -x integrationTest --stacktrace
-
-      - store_artifacts:
-          path: build/reports
-          destination: test-reports
-
-      - store_test_results:
-          path: ~/repo/build/test-results/
-
-  # Runs java 11 tests on a VM to be able to run Docker for integration tests
-  java11:
-    <<: *base_job
-
-    steps:
-      - checkout
-      - install_common
-
-      - install_java:
-          version: adoptopenjdk-11-hotspot
-      - run: sudo update-java-alternatives -s adoptopenjdk-11-hotspot-amd64 && java -version
+      - run: scripts/build-dtest-jars.sh

Review Comment:
   Interestingly, the `environment` setting just wouldn't work for some odd reason, so I ended up just putting it in the command-line.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1218719797


##########
.circleci/config.yml:
##########
@@ -56,56 +57,14 @@ jobs:
      - store_test_results:
          path: ~/repo/build/test-results/
 
-  # Runs java 8 tests on a VM to be able to run Docker for integration tests
-  java8:
-    <<: *base_job
-
-    steps:
-      - checkout
-      - install_common
-
-      - install_java:
-          version: adoptopenjdk-8-hotspot
-      - run: sudo update-java-alternatives -s adoptopenjdk-8-hotspot-amd64 && java -version
-      - run: ./gradlew -i clean test integrationTest --stacktrace
-
-      - store_artifacts:
-          path: build/reports
-          destination: test-reports
-
-      - store_test_results:
-          path: ~/repo/build/test-results/
-
-      - store_test_results:
-          path: ~/repo/cassandra-integration-tests/build/test-results/
-
   # Runs java 11 tests on a docker image
-  java11_docker:
+  java11:
     docker:
       - image: circleci/openjdk:11-jdk-stretch
     steps:
       - checkout
 
-      - run: ./gradlew -i clean test -x integrationTest --stacktrace
-
-      - store_artifacts:
-          path: build/reports
-          destination: test-reports
-
-      - store_test_results:
-          path: ~/repo/build/test-results/
-
-  # Runs java 11 tests on a VM to be able to run Docker for integration tests
-  java11:
-    <<: *base_job
-
-    steps:
-      - checkout
-      - install_common
-
-      - install_java:
-          version: adoptopenjdk-11-hotspot
-      - run: sudo update-java-alternatives -s adoptopenjdk-11-hotspot-amd64 && java -version
+      - run: scripts/build-dtest-jars.sh

Review Comment:
   ```suggestion
         - run:
                command: scripts/build-dtest-jars.sh
                environment:
                  CASSANDRA_USE_JDK11: true
   ```



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227357128


##########
src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java:
##########
@@ -216,7 +220,7 @@ private String createTestTableAndPopulate(CassandraTestContext cassandraTestCont
     private void assertNotFoundOnDeleteSnapshot(VertxTestContext context, String testRoute) throws InterruptedException
     {
         WebClient client = WebClient.create(vertx);
-        client.delete(config.getPort(), "localhost", testRoute)
+        client.delete(config.getPort(), "127.0.0.1", testRoute)

Review Comment:
   I see. `CassandraTestContext` builds the `InstancesConfig` from in-jvm instances that use IP addresses. 



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227331421


##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC
+-XX:+CMSParallelRemarkEnabled
+-XX:SurvivorRatio=8
+-XX:MaxTenuringThreshold=1
+-XX:CMSInitiatingOccupancyFraction=75
+-XX:+UseCMSInitiatingOccupancyOnly
+-XX:CMSWaitDuration=10000
+-XX:+CMSParallelInitialMarkEnabled
+-XX:+CMSEdenChunksRecordAlways
+--add-exports
+java.base/jdk.internal.misc=ALL-UNNAMED
+--add-exports
+java.base/jdk.internal.ref=ALL-UNNAMED
+--add-exports
+java.base/sun.nio.ch=ALL-UNNAMED
+--add-exports
+java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.registry=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.server=ALL-UNNAMED
+--add-exports
+java.rmi/sun.rmi.transport.tcp=ALL-UNNAMED
+--add-exports
+java.sql/java.sql=ALL-UNNAMED
+--add-opens
+java.base/java.lang.module=ALL-UNNAMED
+--add-opens
+java.base/java.net=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.loader=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.ref=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.reflect=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.math=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.module=ALL-UNNAMED
+--add-opens
+java.base/jdk.internal.util.jar=ALL-UNNAMED
+--add-opens
+jdk.management/com.sun.management.internal=ALL-UNNAMED
+-Dcassandra-foreground=yes
+-Dcassandra.config=file:///Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/cassandra.yaml
+-Dcassandra.jmx.local.port=7199
+-Dcassandra.logdir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/data/logs
+-Dcassandra.reads.thresholds.coordinator.defensive_checks_enabled=true
+-Dcassandra.storagedir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/data
+-Dcassandra.triggers_dir=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/triggers
+-Djava.library.path=/Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/lib/sigar-bin
+-Dlogback.configurationFile=file:///Users/drohrer/p/apache-cassandra-bare/cassandra-4.1/conf/logback.xml

Review Comment:
   They actually aren't necessary (the paths - see below for why I include this in the readme). I'll remove the extra parameters.



##########
README.md:
##########
@@ -27,9 +45,71 @@ While setting up cassandra instance, make sure the data directories of cassandra
 Testing
 -------
 
-We rely on docker containers for integration tests.
+The test framework is set up to run 4.1 and 5.0 (Trunk) tests (see `TestVersionSupplier.java`) by default.  
+You can change this via the Java property `cassandra.sidecar.versions_to_test` by supplying a comma-delimited string.
+For example, `-Dcassandra.sidecar.versions_to_test=4.0,4.1,5.0`.
 
-The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
+In order for tests to run successfully under JDK11, you'll need to add the following JVM arguments to your test runner of choice.
+You should also set your test framework to fork a new process at least every class, if not every method, as there are still
+a few unresolved memory-related issues in the in-jvm dtest framework.
+```
+-Djdk.attach.allowAttachSelf=true
+-XX:+UseConcMarkSweepGC

Review Comment:
   Yeah - I originally wrote it before I fixed the integrationTest task - but if someone wants to run them outside of `Gradle` they may need them, so I wanted to have them documented.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227338221


##########
common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java:
##########
@@ -49,4 +49,5 @@
      * @return the {@link TableOperations} implementation for the Cassandra cluster
      */
     TableOperations tableOperations();
+

Review Comment:
   Will fix.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228859565


##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   Also just note that the only reason this is included is to use for the dtest classes - we're not otherwise using the jar, so in theory we could just pick one, but I thought it would be good to use trunk so we would catch ABI changes in trunk more easily (and at one point 4.0 didn't even _work_ because it was missing some packages, but I've fixed that in the dtest jar build process 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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228860028


##########
src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java:
##########
@@ -187,9 +187,9 @@ static class RingHandlerTestModule extends AbstractModule
         @Singleton
         public InstancesConfig instanceConfig() throws IOException
         {
-            final int instanceId = 100;
-            final String host = "127.0.0.1";
-            final InstanceMetadata instanceMetadata = mock(InstanceMetadata.class);
+            int instanceId = 100;
+            String host = "localhost";

Review Comment:
   There was a reason that some of them changed to "localhost" but I have to go check - it's possible I over-applied a fix and changed too many of them.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1228859741


##########
src/test/integration/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java:
##########
@@ -62,7 +58,7 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
 {

Review Comment:
   Will fix.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1222175997


##########
build.gradle:
##########
@@ -24,6 +24,8 @@ plugins {
     id 'com.google.cloud.tools.jib' version '2.2.0'
 }
 
+ext.dtestJar = System.getenv("DTEST_JAR") ?: "dtest-5.0.jar" // trunk is currently 5.0.jar - update when trunk moves

Review Comment:
   I thought about that but it felt like, given this is the only place it's used, adding a level of indirection to change it seemed unnecessary.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #48: CASSANDRASC-51: Use in-jvm dtest framework for integration tests.

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #48:
URL: https://github.com/apache/cassandra-sidecar/pull/48#discussion_r1227328161


##########
adapters/latest/src/main/java/org/apache/cassandra/sidecar/adapters/latest/CassandraAdapter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sidecar.adapters.trunk;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.sidecar.common.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.ClusterMembershipOperations;
+import org.apache.cassandra.sidecar.common.ICassandraAdapter;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.TableOperations;
+import org.apache.cassandra.sidecar.common.dns.DnsResolver;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
+ */
+public class CassandraAdapter implements ICassandraAdapter

Review Comment:
   So this would make way more sense if we had left the 3.11 adapter in here - the idea is to track "anything from trunk backwards to 4.0" in the "trunk" package, and then if you have to split it some day (like the 3.11 stuff did) you'd create a new package and all of _those_ classes would have version-specific class names, so you'd never actually have a conflict. This way the default (and now only) classes don't need version-specific names because they handle significantly more than one version (4.0+ in this case), so it didn't make a lot of sense for it to be called `40`.



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


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