You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/26 15:50:23 UTC

[GitHub] [flink] zentol commented on a diff in pull request #19586: [FLINK-26824] Upgrade Flink's supported Cassandra versions to match all Apache Cassandra supported versions

zentol commented on code in PR #19586:
URL: https://github.com/apache/flink/pull/19586#discussion_r858837814


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorBaseTest.java:
##########
@@ -220,7 +225,7 @@ private static Class<? extends Pojo> annotatePojoWithTable(String keyspace, Stri
     }
 
     @NotNull
-    private static Table createTableAnnotation(String keyspace, String tableName) {
+    private Table createTableAnnotation(String keyspace, String tableName) {

Review Comment:
   Why is this no longer static?



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorBaseTest.java:
##########
@@ -97,29 +94,33 @@
 import static org.hamcrest.Matchers.samePropertyValuesAs;
 import static org.junit.Assert.assertTrue;
 
-/** IT cases for all cassandra sinks. */
+/**
+ * Base class for IT cases for all Cassandra sinks. This class relies on Cassandra testContainer
+ * that needs to use a ClassRule. Parametrized tests to not work with ClassRules so the actual
+ * testCase classes define the tested version and manage the container.
+ */
 @SuppressWarnings("serial")
 // NoHostAvailableException is raised by Cassandra client under load while connecting to the cluster
 @RetryOnException(times = 3, exception = NoHostAvailableException.class)
-public class CassandraConnectorITCase
+public abstract class CassandraConnectorBaseTest

Review Comment:
   This isn't a test, and hence the name shouldn't end in "Test". switch "Base" and "Test".



##########
flink-connectors/flink-connector-cassandra/pom.xml:
##########
@@ -61,7 +63,8 @@ under the License.
 							<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
 							<artifactSet>
 								<includes>
-									<include>com.datastax.cassandra:cassandra-driver-core</include>
+									<!--driver was relocated by datastax-->

Review Comment:
   ```suggestion
   ```



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java:
##########
@@ -140,7 +152,12 @@ public void invoke(IN value) throws Exception {
             semaphore.release();
             throw e;
         }
-        Futures.addCallback(result, callback);
+        if (synchronousWrites) {
+            result.get(timeout, TimeUnit.SECONDS);
+            semaphore.release();

Review Comment:
   If this times out the semaphore is never released.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Cassandra30ConnectorITCase.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.cassandra;
+
+import org.apache.flink.util.DockerImageVersions;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.testcontainers.containers.CassandraContainer;
+
+/** Class for IT cases for all Cassandra sinks tested on Cassandra 3.0.x . */
+public class Cassandra30ConnectorITCase extends CassandraConnectorBaseTest {
+    private static final String TESTED_VERSION = DockerImageVersions.CASSANDRA_3_0;
+
+    @ClassRule
+    public static final CassandraContainer CASSANDRA_CONTAINER =

Review Comment:
   If we're going with several ITCases I'd be tempted to only have this in the sub-classes:
   
   ```
       @BeforeClass
       public static void setup() {
           startAndInitializeCassandra(<version>);
       }
   ```
   and have startAndInitialize also create the container, and manually manage the lifecycle of the cassandra container instead of relying on `ClassRule`



##########
flink-connectors/flink-connector-cassandra/pom.xml:
##########
@@ -37,8 +37,10 @@ under the License.
 
 	<!-- Allow users to pass custom connector versions -->
 	<properties>
-		<cassandra.version>2.2.5</cassandra.version>
-		<driver.version>3.0.0</driver.version>
+		<!-- Cassandra version 4.x allow to address clusters of version 3.x as well -->
+		<cassandra.version>4.0.3</cassandra.version>
+		<!--driver 3.x works with 3.x and 4.x versions of Cassandra but driver 4.x is a complete refactoring with different API-->
+		<driver.version>3.11.1</driver.version>
 		<guava.version>18.0</guava.version>

Review Comment:
   did you check whether the driver now uses a different guava version?



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java:
##########
@@ -72,6 +74,16 @@
         ClosureCleaner.clean(builder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
     }
 
+    /**
+     * Set writes to be synchronous (block until writes are completed).
+     *
+     * @param timeout Maximum number of seconds to wait for write completion
+     */
+    public void setSynchronousWrites(int timeout) {

Review Comment:
   This seems strange. The CassandraSinkBase explicitly contains logic to make sure all async requests have completed when shutting down or creating a checkpoint (#flush).
   
   If some issue now appears in the test that implies that either
   a) this logic is now broken, which would be problematic in general since it could cause data-loss
   b) despite writes being acknowledged they aren't persisted properly, at which point I wonder what makes the synchronous call that different (maybe it just works because data is sent at a lower rate)
   
   In any case this needs further investigations.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/Cassandra40ConnectorITCase.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.streaming.connectors.cassandra;
+
+import org.apache.flink.util.DockerImageVersions;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.testcontainers.containers.CassandraContainer;
+
+/** Class for IT cases for all Cassandra sinks tested on Cassandra 4.0.x . */
+public class Cassandra40ConnectorITCase extends CassandraConnectorBaseTest {

Review Comment:
   I'm not sure if we really need tests for different cassandra versions.



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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