You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "yifan-c (via GitHub)" <gi...@apache.org> on 2023/06/12 19:45:35 UTC

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

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