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/05/09 11:30:51 UTC

[GitHub] [flink] echauchot opened a new pull request, #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

echauchot opened a new pull request, #19680:
URL: https://github.com/apache/flink/pull/19680

   
   
   ## What is the purpose of the change
   
   When migrating to Cassandra 4.x in [this PR](https://github.com/apache/flink/pull/19586) a race condition in the tests between the asynchronous writes and the junit assertions was uncovered. So it was decided to introduce the flush mechanism to asynchronous writes in the Cassandra output formats similarly to what was done in Cassandra sinks.
   
   
   ## Brief change log
   
   The existing class `CassandraOutputFormatBase` that was previously used as a base class only for Tuple and Row outputFormats is now used as a base class for the 3 output formats including Pojo. the base class for column based output formats (tuple and row) is now a new class called CassandraColumnarOutputFormatBase.
   Regarding configuration of the flush I preferred using simple setters to a configuration object as there was no builders for the output formats.
   Regarding other modules: I extracted a utility method for semaphore management (SinkUtils) because it is used by both sinks and output formats now. And I also had to change the exceptions thrown in OutputFormat as some methods can now throw TimeoutException and InterruptedException because of the flush mechanism. I think it is ok as this interface is not user facing.
   
   
   ## Verifying this change
   
   
   
   This change is already covered by existing ITCAse tests 
   This change added UTests for the flush mechanism and can be verified as follows: CassandraOutputFormatBaseTest
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): adding semaphore permit management + flush on close
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes 
     - If yes, how is the feature documented? javadocs
   


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r875588195


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/utils/SinkUtils.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connectors.cassandra.utils;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/** Utility class for sinks. */
+public class SinkUtils implements Serializable {
+
+    /**
+     * Acquire permits on the given semaphore within a given allowed timeout and deal with errors.
+     *
+     * @param permits the mumber of permits to acquire.
+     * @param maxConcurrentRequests the maximum number of permits the semaphore was initialized
+     *     with.
+     * @param maxConcurrentRequestsTimeout the timeout to acquire the permits.
+     * @param semaphore the semaphore to acquire permits to.
+     * @throws InterruptedException if the current thread was interrupted.
+     * @throws TimeoutException if the waiting time elapsed before all permits were acquired.
+     */
+    public static void tryAcquire(

Review Comment:
   and besides `java.util.concurrent.Semaphore#tryAcquire` also throws exceptions. So the naming seems coherent



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876793907


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoOutputFormat.java:
##########
@@ -90,67 +83,20 @@ public void open(int taskNumber, int numTasks) {
                 mapper.setDefaultSaveOptions(optionsArray);
             }
         }
-        this.callback =
-                new FutureCallback<Void>() {
-                    @Override
-                    public void onSuccess(Void ignored) {
-                        onWriteSuccess();
-                    }
-
-                    @Override
-                    public void onFailure(Throwable t) {
-                        onWriteFailure(t);
-                    }
-                };
     }
 
     @Override
-    public void writeRecord(OUT record) throws IOException {
-        if (exception != null) {
-            throw new IOException("write record failed", exception);
-        }
-
-        ListenableFuture<Void> result = mapper.saveAsync(record);
-        Futures.addCallback(result, callback);
-    }
-
-    /**
-     * Callback that is invoked after a record is written to Cassandra successfully.
-     *
-     * <p>Subclass can override to provide its own logic.
-     */
-    protected void onWriteSuccess() {}
-
-    /**
-     * Callback that is invoked when failing to write to Cassandra. Current implementation will
-     * record the exception and fail the job upon next record.
-     *
-     * <p>Subclass can override to provide its own failure handling logic.
-     *
-     * @param t the exception
-     */
-    protected void onWriteFailure(Throwable t) {
-        exception = t;
+    public ListenableFuture<Void> send(OUT record) {

Review Comment:
   ```suggestion
       protected ListenableFuture<Void> send(OUT record) {
   ```



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {
                     @Override
-                    public void onSuccess(ResultSet ignored) {
-                        onWriteSuccess(ignored);
+                    public void onSuccess(V ignored) {
+                        semaphore.release();
                     }
 
                     @Override
                     public void onFailure(Throwable t) {
-                        onWriteFailure(t);
+                        throwable.compareAndSet(null, t);
+                        LOG.error("Error while writing value.", t);
+                        semaphore.release();
                     }
                 };
     }
 
+    private void flush() {
+        tryAcquire(maxConcurrentRequests);
+        semaphore.release(maxConcurrentRequests);
+    }
+
+    private void tryAcquire(int permits) {
+        try {
+            SinkUtils.tryAcquire(
+                    permits, maxConcurrentRequests, maxConcurrentRequestsTimeout, semaphore);
+        } catch (Exception e) {
+            throw new RuntimeException(e);

Review Comment:
   Have you thought about throwing an IOException?



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(

Review Comment:
   avoid raw types



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {

Review Comment:
   Does this actually throw an IOException? Doesn't seem so; if we remove this we could skip the signature change in the pojo output format. (changing it here instead of the pojo format is preferable since the latter is more user-facing than this one)



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(
+            String insertQuery,
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
+        super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /** Opens a Session to Cassandra and initializes the prepared statement. */
+    @Override
+    public void open(int taskNumber, int numTasks) throws IOException {
+        super.open(taskNumber, numTasks);
+        this.prepared = session.prepare(insertQuery);
+    }
+
+    @Override
+    public ListenableFuture<ResultSet> send(OUT record) {

Review Comment:
   ```suggestion
       protected ListenableFuture<ResultSet> send(OUT record) {
   ```



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I'm wondering if it would make sense to move the dependencies on the datastax driver into the sub-classes (columnar/pojo) at the cost of duplicating a few lines.
   This would allow us to test this class in nicer fashion without mocking.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)

Review Comment:
   new tests should be written with JUnit5 and assertj.



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(
+            String insertQuery,
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
+        super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /** Opens a Session to Cassandra and initializes the prepared statement. */

Review Comment:
   ```suggestion
   ```
   The code seems to document this just fine on it's own.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876822053


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");

Review Comment:
   This should fail if no exception is thrown.



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


[GitHub] [flink] zentol merged pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol merged PR #19680:
URL: https://github.com/apache/flink/pull/19680


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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1149818069

   @flinkbot run azure


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879239572


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886663806


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);

Review Comment:
   I agree. Here is [the ticket](https://issues.apache.org/jira/browse/FLINK-27866)
   



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887784305


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I agree it is a pity to use a mock only because of cassandra session/cluster and only for one test. I'll try to refactor to avoid mocking and code duplication. I'll propose you something in a isolated commit for ease of review.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r888836440


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   done.
   I introduced `OutputFormatBase` that has no Cassandra dependencies and that contains only flush and max concurrent requests features. This class is for now in cassandra module waiting for the above promotion decision. The introduction of this class is isolated in commit a3323a1b to ease a cherrypick in case we do a separate PR to address https://issues.apache.org/jira/browse/FLINK-27884



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873543679


##########
pom.xml:
##########
@@ -2134,6 +2134,7 @@ under the License.
 								<exclude>org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object)</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.SinkFunction</exclude>
+								<exclude>org.apache.flink.api.common.io.OutputFormat</exclude>

Review Comment:
   ah, so that's why japicmp complained.



##########
pom.xml:
##########
@@ -2134,6 +2134,7 @@ under the License.
 								<exclude>org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object)</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.SinkFunction</exclude>
+								<exclude>org.apache.flink.api.common.io.OutputFormat</exclude>

Review Comment:
   ah, so that's why japicmp did not complain.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873557179


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
+    }
+
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequests maximum number of concurrent requests allowed
+     */
+    public void setMaxConcurrentRequests(int maxConcurrentRequests) {
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+    }
+
+    /**
+     * Configure the connection to Cassandra.
+     *
+     * @param parameters The configuration with all parameters.
+     */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
     /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
+     * Opens a Session to Cassandra .
      *
      * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
+     * @throws IOException Thrown, if the output format could not be opened due to an I/O problem.
      */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {

Review Comment:
   Is the type V relevant anywhere? Could we just use a wildcard?



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r881588655


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);

Review Comment:
   Right, we would fail again in the flush when trying to acquire the permits.
   
   We may actually want to file a follow-up ticket, because as is we wait (and fail) twice if a permit can't be acquired in writeRecord() and that state persists until close(). Ideally this only happens once.



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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1134505280

   @zentol I'm done addressing your comments, do you see anything else to address?


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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876821169


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }

Review Comment:
   This is too broad. Let's explicitly call close and only guard that call with the try-catch block.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877246715


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);
+            }
+        }
+    }
+
+    private TestCassandraOutputFormat createTestCassandraOutputFormat() throws IOException {

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886525871


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testHostNotFoundErrorHandling() {
+        CassandraOutputFormatBase<Object, Void> cassandraOutputFormatBase =
+                new CassandraOutputFormatBase<Object, Void>(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture<Void> send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        assertThatThrownBy(() -> cassandraOutputFormatBase.open(1, 1))
+                .isInstanceOf(NoHostAvailableException.class);
+    }
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertThat(originalPermits).isGreaterThan(0);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(originalPermits);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+
+        assertThatThrownBy(() -> testCassandraOutputFormat.close())
+                .isInstanceOf(IOException.class)
+                .hasCauseReference(cause);
+    }
+
+    @Test
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            // should fail because the first write failed and the second will check for asynchronous
+            // errors (throwable set by the async callback)
+            assertThatThrownBy(
+                            () -> testCassandraOutputFormat.writeRecord("world"),
+                            "Sending of second value should have failed.")
+                    .isInstanceOf(IOException.class)
+                    .hasCauseReference(cause);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            CheckedThread checkedThread =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                        }
+                    };
+            checkedThread.start();
+            while (checkedThread.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+            // start writing
+            completableFuture.complete(null);
+            checkedThread.sync();
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            // start writing
+            completableFuture.complete(null);
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnFailure() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat();
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+        CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+        testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+        testCassandraOutputFormat.writeRecord("N/A");
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+        completableFuture.completeExceptionally(new RuntimeException());
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
+        } catch (IOException ignored) {
+            // the aim is not to assert on the exception in this test
+        }
+    }
+
+    @Test
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throw new RuntimeException("expected");
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            assertThatThrownBy(() -> testCassandraOutputFormat.writeRecord("none"));

Review Comment:
   added a comment in code and an ignored catch.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873551123


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+public abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(String insertQuery, ClusterBuilder builder) {
+        super(builder);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /**
+     * Opens a Session to Cassandra and initializes the prepared statement.
+     *
+     * @param taskNumber The number of the parallel instance.
+     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.

Review Comment:
   seems strange to document the parameters when this is already done in the interface.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874523367


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+public abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(String insertQuery, ClusterBuilder builder) {
+        super(builder);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /**
+     * Opens a Session to Cassandra and initializes the prepared statement.
+     *
+     * @param taskNumber The number of the parallel instance.
+     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874519948


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+public abstract class CassandraColumnarOutputFormatBase<OUT>

Review Comment:
   yes, thanks for the catch



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876822911


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);
+            }
+        }
+    }
+
+    private TestCassandraOutputFormat createTestCassandraOutputFormat() throws IOException {

Review Comment:
   all these methods can be static



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

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

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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876821775


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;

Review Comment:
   Why not just throw a runtime exception?



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873542903


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/utils/SinkUtils.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connectors.cassandra.utils;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/** Utility class for sinks. */
+public class SinkUtils implements Serializable {
+
+    /**
+     * Acquire permits on the given semaphore within a given allowed timeout and deal with errors.
+     *
+     * @param permits the mumber of permits to acquire.
+     * @param maxConcurrentRequests the maximum number of permits the semaphore was initialized
+     *     with.
+     * @param maxConcurrentRequestsTimeout the timeout to acquire the permits.
+     * @param semaphore the semaphore to acquire permits to.
+     * @throws InterruptedException if the current thread was interrupted.
+     * @throws TimeoutException if the waiting time elapsed before all permits were acquired.
+     */
+    public static void tryAcquire(

Review Comment:
   Naming isn't ideal. "try" implies that not being able to acquire it is an expected/safe state, yet it throws an exception.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879220618


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {

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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877242325


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r888836440


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   done.
   I introduced `OutputFormatBase` that has no Cassandra dependencies and that contains only flush and max concurrent requests features. This class is for now in cassandra module waiting for the above promotion decision. The introduction of this class is isolated in commit a3323a1b to ease a cherrypick in case we do a separate PR to address https://issues.apache.org/jira/browse/FLINK-27884
   
   Also there is no more `CassandraOutputFormatBaseTest` because it contained no cassandra-related tests except a test that NoHostAvailableException is thrown in case of non-existing host. There is now only a `OutputFormatBaseTest` that contains the tests for the flush mechanism. So I could completely remove the mocking as you wished.



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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1129923837

   @zentol thanks for your review. I addressed all your comments PTAL.


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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1149796221

   @zentol it seems that the CI failed. I'm taking a look 


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886675384


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testHostNotFoundErrorHandling() {
+        CassandraOutputFormatBase<Object, Void> cassandraOutputFormatBase =
+                new CassandraOutputFormatBase<Object, Void>(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture<Void> send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        assertThatThrownBy(() -> cassandraOutputFormatBase.open(1, 1))
+                .isInstanceOf(NoHostAvailableException.class);
+    }
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertThat(originalPermits).isGreaterThan(0);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(originalPermits);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+
+        assertThatThrownBy(() -> testCassandraOutputFormat.close())
+                .isInstanceOf(IOException.class)
+                .hasCauseReference(cause);
+    }
+
+    @Test
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            // should fail because the first write failed and the second will check for asynchronous
+            // errors (throwable set by the async callback)
+            assertThatThrownBy(
+                            () -> testCassandraOutputFormat.writeRecord("world"),
+                            "Sending of second value should have failed.")
+                    .isInstanceOf(IOException.class)
+                    .hasCauseReference(cause);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            CheckedThread checkedThread =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                        }
+                    };
+            checkedThread.start();
+            while (checkedThread.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+            // start writing
+            completableFuture.complete(null);
+            checkedThread.sync();
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            // start writing
+            completableFuture.complete(null);
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnFailure() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat();
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+        CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+        testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+        testCassandraOutputFormat.writeRecord("N/A");
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+        completableFuture.completeExceptionally(new RuntimeException());
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
+        } catch (IOException ignored) {
+            // the aim is not to assert on the exception in this test
+        }
+    }
+
+    @Test
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throw new RuntimeException("expected");
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            assertThatThrownBy(() -> testCassandraOutputFormat.writeRecord("none"));
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {

Review Comment:
   agree, 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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886678602


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -202,9 +202,12 @@ public void testReleaseOnFailure() throws Exception {
 
             assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
             assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
         } catch (IOException ignored) {

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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887784305


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I agree it is a pity to use a mock only because of cassandra session/cluster. I'll try to refactor to avoid mocking and code duplication. I'll propose you something in a isolated commit for ease of review.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887784305


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I vastly reused `CassandraSinkBaseTest` code but I agree it is a pity to use a mock only because of Cassandra session/cluster. I'll try to refactor to avoid mocking and code duplication. I'll propose you something in a isolated commit for ease of review.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r891088987


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/OutputFormatBaseTest.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for the {@link OutputFormatBase}. */
+public class OutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {

Review Comment:
   I'll fix this during the merge.



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

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

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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876822534


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);

Review Comment:
   this shouldn't be necessary.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874852268


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   ok



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874497578


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   As said in the other comment, I **proposed** this change as `tryAcquire` already threw exceptions and `OutputFormat` is not user facing so I felt we could change to a more generic exception throwing. But it is true that even if it is not a user braking change, it is a source-compatibility breaking change. So if you prefer I'll make `tryAcquire` throw an unchecked exception in both Sink and OutputFormat so that we don't need to change the throw clause.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874518997


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
+    }
+
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequests maximum number of concurrent requests allowed
+     */
+    public void setMaxConcurrentRequests(int maxConcurrentRequests) {
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+    }
+
+    /**
+     * Configure the connection to Cassandra.
+     *
+     * @param parameters The configuration with all parameters.
+     */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
     /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
+     * Opens a Session to Cassandra .
      *
      * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
+     * @throws IOException Thrown, if the output format could not be opened due to an I/O problem.
      */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {
                     @Override
-                    public void onSuccess(ResultSet ignored) {
-                        onWriteSuccess(ignored);
+                    public void onSuccess(V ignored) {
+                        semaphore.release();
                     }
 
                     @Override
                     public void onFailure(Throwable t) {
-                        onWriteFailure(t);
+                        throwable.compareAndSet(null, t);
+                        LOG.error("Error while writing value.", t);
+                        semaphore.release();
                     }
                 };
     }
 
-    @Override
-    public void writeRecord(OUT record) throws IOException {
-        if (exception != null) {
-            throw new IOException("write record failed", exception);
-        }
-
-        Object[] fields = extractFields(record);
-        ResultSetFuture result = session.executeAsync(prepared.bind(fields));
-        Futures.addCallback(result, callback);
+    private void flush() throws InterruptedException, TimeoutException {
+        tryAcquire(maxConcurrentRequests);
+        semaphore.release(maxConcurrentRequests);
     }
 
-    protected abstract Object[] extractFields(OUT record);
+    private void tryAcquire(int permits) throws InterruptedException, TimeoutException {
+        SinkUtils.tryAcquire(
+                permits, maxConcurrentRequests, maxConcurrentRequestsTimeout, semaphore);
+    }
 
     /**
-     * Callback that is invoked after a record is written to Cassandra successfully.
-     *
-     * <p>Subclass can override to provide its own logic.
+     * Writes a record to Cassandra.
      *
-     * @param ignored the result.
+     * @param record The records to add to the output.
+     * @throws Exception Thrown, if the record could not be written due to an I/O problem or a
+     *     timeout.
      */
-    protected void onWriteSuccess(ResultSet ignored) {}
+    @Override
+    public void writeRecord(OUT record) throws Exception {
+        checkAsyncErrors();
+        tryAcquire(1);
+        final ListenableFuture<V> result;
+        try {
+            result = send(record);
+        } catch (Throwable e) {
+            semaphore.release();
+            throw e;
+        }
+        Futures.addCallback(result, callback);
+    }
+
+    public abstract ListenableFuture<V> send(OUT value);

Review Comment:
   yes, thanks for the catch



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877946074


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879260258


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;

Review Comment:
   I took this code from CassandraSinkBaseTest. I agree, I don't know why it was done in this unnecessary complicated way.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886474742


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -202,9 +202,12 @@ public void testReleaseOnFailure() throws Exception {
 
             assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
             assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
         } catch (IOException ignored) {

Review Comment:
   I did that because the aim of the test is to assert on semaphore count and not on exception as I wrote in the comment in the code. But if you think it is not misleading to assert on the exception, then fine.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887888690


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I'm thinking that if we remove everything Cassandra related from `CassandraOutputFormatBase` then this class can become the base class for all the output formats that want to leverage the flush logic. So we could promote the resulting class to `org.apache.flink.api.common.io` package as `OutputFormatBase`.  `CassandraOutputFormatBase` would extend this class and contain the cassandra deps.
   
   I'll do the above code so that we discuss on the code itself if needed.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873552991


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {

Review Comment:
   these shouldn't be setters but a constructor argument; it clarifies when the format is configured (at construction time).



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r875643870


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   Do you want that I change the exception throwing to unchecked also in Sink for consistency with OutputFormat? If positive, the impact would be a breaking change for the user in exception management: in case of exception they will get RuntimeException(e) in place of e. I could also do the change only on the OutputFormat to avoid this breaking change in the sinks.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r875643870


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   Do you want that I change the exception throwing to unchecked also on Sink for consistency with OutputFormat? If positive, the impact would be a breaking change for the user in exception management: in case of exception they will get RuntimeException(e) in place of e. I could also do the change only on the OutputFormat to avoid this breaking change in the sinks.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874491149


##########
pom.xml:
##########
@@ -2134,6 +2134,7 @@ under the License.
 								<exclude>org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object)</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.SinkFunction</exclude>
+								<exclude>org.apache.flink.api.common.io.OutputFormat</exclude>

Review Comment:
   yes as I said in the commit message as it was not user facing I proposed this exception, but it is just to proposition. If it is not the correct move I can definitely do otherwise. I just did not know what was acceptable: I mean SinkFunctions for example were already in the exceptions.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874497578


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   As said in the other comment, it is just a proposed change. I did this change as `tryAcquire` in `CassandraSink` already threw exceptions and `OutputFormat` is not user facing so I felt we could change to a more generic exception throwing. But it is true that even if it is not a user braking change, it is a source-compatibility breaking change. So if you prefer I'll make `tryAcquire` throw an unchecked exception in both Sink and OutputFormat so that we don't need to change the throw clause.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r875671860


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   I'll do the unchecked exception change only on `CassandraOutputFormat`  and not on `CassandraSink` to avoid a breaking change in exception management for sink users: avoid them getting `RuntimeException(e)` in place of `e` in case of exception.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r886510500


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testHostNotFoundErrorHandling() {
+        CassandraOutputFormatBase<Object, Void> cassandraOutputFormatBase =
+                new CassandraOutputFormatBase<Object, Void>(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture<Void> send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        assertThatThrownBy(() -> cassandraOutputFormatBase.open(1, 1))
+                .isInstanceOf(NoHostAvailableException.class);
+    }
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertThat(originalPermits).isGreaterThan(0);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(originalPermits);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+
+        assertThatThrownBy(() -> testCassandraOutputFormat.close())
+                .isInstanceOf(IOException.class)
+                .hasCauseReference(cause);
+    }
+
+    @Test
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            // should fail because the first write failed and the second will check for asynchronous
+            // errors (throwable set by the async callback)
+            assertThatThrownBy(
+                            () -> testCassandraOutputFormat.writeRecord("world"),
+                            "Sending of second value should have failed.")
+                    .isInstanceOf(IOException.class)
+                    .hasCauseReference(cause);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            CheckedThread checkedThread =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                        }
+                    };
+            checkedThread.start();
+            while (checkedThread.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+            // start writing
+            completableFuture.complete(null);
+            checkedThread.sync();
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            // start writing
+            completableFuture.complete(null);
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnFailure() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat();
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+        CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+        testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+        testCassandraOutputFormat.writeRecord("N/A");
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+        completableFuture.completeExceptionally(new RuntimeException());
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
+        } catch (IOException ignored) {
+            // the aim is not to assert on the exception in this test
+        }
+    }
+
+    @Test
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throw new RuntimeException("expected");
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            assertThatThrownBy(() -> testCassandraOutputFormat.writeRecord("none"));

Review Comment:
   agree, this is not clear enough, there should be no assertion on exception



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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1149976189

   @flinkbot run azure
   


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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r890997078


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/OutputFormatBaseTest.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for the {@link OutputFormatBase}. */
+public class OutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {

Review Comment:
   ```suggestion
   class OutputFormatBaseTest {
   
       private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
               Duration.ofMillis(Long.MAX_VALUE);
   
       @Test
       void testSuccessfulWrite() throws Exception {
   ```
   We can reduce the visibility because of junit5.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877945167


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();

Review Comment:
   ah yeah, did not know this very useful util class, thanks for the pointer.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879264371


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;

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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876816993


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)

Review Comment:
   we _generally_ don't use timeouts in tests anymore.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877945167


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();

Review Comment:
   ah yeah, did not know this util class, thanks for the pointer.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879239572


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }

Review Comment:
   Agree, I was too quick, I catch too much here.



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

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

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877935271


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879271060


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");

Review Comment:
   :+1: 



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);

Review Comment:
   If we don't complete the future, then the callback of the write will not be the one that will throw the timeout exception but it will be thrown later when close() is called (cf checkAsyncErrors). As in this test we check that write throws, I suggest that we leave the test as it is to avoid the ambiguity of asserting that close throws.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877234379


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoOutputFormat.java:
##########
@@ -90,67 +83,20 @@ public void open(int taskNumber, int numTasks) {
                 mapper.setDefaultSaveOptions(optionsArray);
             }
         }
-        this.callback =
-                new FutureCallback<Void>() {
-                    @Override
-                    public void onSuccess(Void ignored) {
-                        onWriteSuccess();
-                    }
-
-                    @Override
-                    public void onFailure(Throwable t) {
-                        onWriteFailure(t);
-                    }
-                };
     }
 
     @Override
-    public void writeRecord(OUT record) throws IOException {
-        if (exception != null) {
-            throw new IOException("write record failed", exception);
-        }
-
-        ListenableFuture<Void> result = mapper.saveAsync(record);
-        Futures.addCallback(result, callback);
-    }
-
-    /**
-     * Callback that is invoked after a record is written to Cassandra successfully.
-     *
-     * <p>Subclass can override to provide its own logic.
-     */
-    protected void onWriteSuccess() {}
-
-    /**
-     * Callback that is invoked when failing to write to Cassandra. Current implementation will
-     * record the exception and fail the job upon next record.
-     *
-     * <p>Subclass can override to provide its own failure handling logic.
-     *
-     * @param t the exception
-     */
-    protected void onWriteFailure(Throwable t) {
-        exception = t;
+    public ListenableFuture<Void> send(OUT record) {

Review Comment:
   :+1: 



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r881572051


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -202,9 +202,12 @@ public void testReleaseOnFailure() throws Exception {
 
             assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
             assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
         } catch (IOException ignored) {

Review Comment:
   use assertThatThrownBy



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testHostNotFoundErrorHandling() {
+        CassandraOutputFormatBase<Object, Void> cassandraOutputFormatBase =
+                new CassandraOutputFormatBase<Object, Void>(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture<Void> send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        assertThatThrownBy(() -> cassandraOutputFormatBase.open(1, 1))
+                .isInstanceOf(NoHostAvailableException.class);
+    }
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertThat(originalPermits).isGreaterThan(0);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(originalPermits);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+
+        assertThatThrownBy(() -> testCassandraOutputFormat.close())
+                .isInstanceOf(IOException.class)
+                .hasCauseReference(cause);
+    }
+
+    @Test
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            // should fail because the first write failed and the second will check for asynchronous
+            // errors (throwable set by the async callback)
+            assertThatThrownBy(
+                            () -> testCassandraOutputFormat.writeRecord("world"),
+                            "Sending of second value should have failed.")
+                    .isInstanceOf(IOException.class)
+                    .hasCauseReference(cause);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            CheckedThread checkedThread =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                        }
+                    };
+            checkedThread.start();
+            while (checkedThread.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+            // start writing
+            completableFuture.complete(null);
+            checkedThread.sync();
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            // start writing
+            completableFuture.complete(null);
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnFailure() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat();
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+        CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+        testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+        testCassandraOutputFormat.writeRecord("N/A");
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+        completableFuture.completeExceptionally(new RuntimeException());
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
+        } catch (IOException ignored) {
+            // the aim is not to assert on the exception in this test
+        }
+    }
+
+    @Test
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throw new RuntimeException("expected");
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            assertThatThrownBy(() -> testCassandraOutputFormat.writeRecord("none"));

Review Comment:
   I think I'd prefer an explicit isNotNull clause or something along those lines; it threw me off that this actually asserts anything (because that wouldn't be the case for assertThat())



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   Alternatively we could move the session stuff into another shared sub-class.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");
+            } catch (Throwable e) {
+                Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat(Duration.ofMillis(1))) {
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("writeRecord #1");
+
+            try {
+                testCassandraOutputFormat.writeRecord("writeRecord #2");
+                Assert.fail("Sending value should have experienced a TimeoutException");
+            } catch (Exception e) {
+                Assert.assertTrue(e.getCause() instanceof TimeoutException);
+            } finally {
+                completableFuture.complete(null);

Review Comment:
   I'm not sure what ambiguity you're referring to.
   
   We do want to/must make sure is that if a timeout occurs and is noticed within `writeRecord()`, that we can close the sink without throwing another exception, independent of whether the future is eventually completed or not.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test
+    public void testHostNotFoundErrorHandling() {
+        CassandraOutputFormatBase<Object, Void> cassandraOutputFormatBase =
+                new CassandraOutputFormatBase<Object, Void>(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture<Void> send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        assertThatThrownBy(() -> cassandraOutputFormatBase.open(1, 1))
+                .isInstanceOf(NoHostAvailableException.class);
+    }
+
+    @Test
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertThat(originalPermits).isGreaterThan(0);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(originalPermits);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+
+        assertThatThrownBy(() -> testCassandraOutputFormat.close())
+                .isInstanceOf(IOException.class)
+                .hasCauseReference(cause);
+    }
+
+    @Test
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            // should fail because the first write failed and the second will check for asynchronous
+            // errors (throwable set by the async callback)
+            assertThatThrownBy(
+                            () -> testCassandraOutputFormat.writeRecord("world"),
+                            "Sending of second value should have failed.")
+                    .isInstanceOf(IOException.class)
+                    .hasCauseReference(cause);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            CheckedThread checkedThread =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                        }
+                    };
+            checkedThread.start();
+            while (checkedThread.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+            // start writing
+            completableFuture.complete(null);
+            checkedThread.sync();
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+            // start writing
+            completableFuture.complete(null);
+
+            assertThat(openedTestCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(openedTestCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testReleaseOnFailure() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat();
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+        CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+        testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+        testCassandraOutputFormat.writeRecord("N/A");
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(0);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(1);
+
+        completableFuture.completeExceptionally(new RuntimeException());
+
+        assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+        assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        try {
+            testCassandraOutputFormat.close();
+        } catch (IOException ignored) {
+            // the aim is not to assert on the exception in this test
+        }
+    }
+
+    @Test
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throw new RuntimeException("expected");
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+
+            assertThatThrownBy(() -> testCassandraOutputFormat.writeRecord("none"));
+            assertThat(testCassandraOutputFormat.getAvailablePermits()).isEqualTo(1);
+            assertThat(testCassandraOutputFormat.getAcquiredPermits()).isEqualTo(0);
+        }
+    }
+
+    @Test
+    public void testTimeoutExceptionOnWriteRecord() throws Exception {

Review Comment:
   We should adjust the name to clarify that this timeout happens due to the maximum number of concurrent requests, and not write timing out.



##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   Alternatively we could move the cluster/session stuff into another shared sub-class.



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

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

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


[GitHub] [flink] flinkbot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1120982216

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "6485a7ae38abe5421eab32e008ad3f7abf971394",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6485a7ae38abe5421eab32e008ad3f7abf971394",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6485a7ae38abe5421eab32e008ad3f7abf971394 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874488570


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/utils/SinkUtils.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connectors.cassandra.utils;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/** Utility class for sinks. */
+public class SinkUtils implements Serializable {
+
+    /**
+     * Acquire permits on the given semaphore within a given allowed timeout and deal with errors.
+     *
+     * @param permits the mumber of permits to acquire.
+     * @param maxConcurrentRequests the maximum number of permits the semaphore was initialized
+     *     with.
+     * @param maxConcurrentRequestsTimeout the timeout to acquire the permits.
+     * @param semaphore the semaphore to acquire permits to.
+     * @throws InterruptedException if the current thread was interrupted.
+     * @throws TimeoutException if the waiting time elapsed before all permits were acquired.
+     */
+    public static void tryAcquire(

Review Comment:
   I just reused what already existed in `CassandraSinkBase` which signature is: `private void tryAcquire(int permits) throws InterruptedException, TimeoutException`



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874497578


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   As said in the other comment, I proposed this change as `tryAcquire` already threw exceptions and `OutputFormat` is not user facing so I felt we could change to a more generic exception throwing. But it is true that even if it is not a user braking change, it is a source-compatibility breaking change. So if you prefer I'll make `tryAcquire` throw an unchecked exception in both Sink and OutputFormat so that we don't need to change the throw clause.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873547606


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
+    }
+
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequests maximum number of concurrent requests allowed
+     */
+    public void setMaxConcurrentRequests(int maxConcurrentRequests) {
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+    }
+
+    /**
+     * Configure the connection to Cassandra.
+     *
+     * @param parameters The configuration with all parameters.
+     */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
     /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
+     * Opens a Session to Cassandra .
      *
      * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
+     * @throws IOException Thrown, if the output format could not be opened due to an I/O problem.
      */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {
                     @Override
-                    public void onSuccess(ResultSet ignored) {
-                        onWriteSuccess(ignored);
+                    public void onSuccess(V ignored) {
+                        semaphore.release();
                     }
 
                     @Override
                     public void onFailure(Throwable t) {
-                        onWriteFailure(t);
+                        throwable.compareAndSet(null, t);
+                        LOG.error("Error while writing value.", t);
+                        semaphore.release();
                     }
                 };
     }
 
-    @Override
-    public void writeRecord(OUT record) throws IOException {
-        if (exception != null) {
-            throw new IOException("write record failed", exception);
-        }
-
-        Object[] fields = extractFields(record);
-        ResultSetFuture result = session.executeAsync(prepared.bind(fields));
-        Futures.addCallback(result, callback);
+    private void flush() throws InterruptedException, TimeoutException {
+        tryAcquire(maxConcurrentRequests);
+        semaphore.release(maxConcurrentRequests);
     }
 
-    protected abstract Object[] extractFields(OUT record);
+    private void tryAcquire(int permits) throws InterruptedException, TimeoutException {
+        SinkUtils.tryAcquire(
+                permits, maxConcurrentRequests, maxConcurrentRequestsTimeout, semaphore);
+    }
 
     /**
-     * Callback that is invoked after a record is written to Cassandra successfully.
-     *
-     * <p>Subclass can override to provide its own logic.
+     * Writes a record to Cassandra.
      *
-     * @param ignored the result.
+     * @param record The records to add to the output.
+     * @throws Exception Thrown, if the record could not be written due to an I/O problem or a
+     *     timeout.
      */
-    protected void onWriteSuccess(ResultSet ignored) {}
+    @Override
+    public void writeRecord(OUT record) throws Exception {
+        checkAsyncErrors();
+        tryAcquire(1);
+        final ListenableFuture<V> result;
+        try {
+            result = send(record);
+        } catch (Throwable e) {
+            semaphore.release();
+            throw e;
+        }
+        Futures.addCallback(result, callback);
+    }
+
+    public abstract ListenableFuture<V> send(OUT value);

Review Comment:
   shouldn't this be protected?



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874815346


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   > OutputFormat is not user facing
   
   It is very much user-facing; they can (and do) implement their own formats and use it with existing sinks. So we must revert to throwing an unchecked exception.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r878323535


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887888690


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I'm thinking that if we remove everything Cassandra related from `CassandraOutputFormatBase` then this class can become the base class for all the output formats that want to leverage the flush logic. So we could promote the resulting class to `org.apache.flink.api.common.io` package as `OutputFormatBase`.  `CassandraOutputFormatBase` would extend this class and contain the cassandra deps.
   
   I think it requires also to create a dedicated ticket: https://issues.apache.org/jira/browse/FLINK-27884
   
   For now I'll just keep the classes in the cassandra connector module waiting for the decision to promote output format flush features to flink common io



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r888836440


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   done.
   I introduced `OutputFormatBase` that has no Cassandra dependencies and that contains only flush and max concurrent requests features. This class is for now in cassandra module waiting for the above promotion decision. The introduction of this class is isolated in commit a3323a1b to ease a cherrypick in case we do a separate PR to address https://issues.apache.org/jira/browse/FLINK-27884
   
   Also there is no more `CassandraOutputFormatBaseTest` because there were it contained no cassandra-related tests exception a test that NoHostAvailableException is thrown in case of non-existing host. There is now only a `OutputFormatBaseTest` that contains the tests for the flush mechanism



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887888690


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I'm thinking that if we remove everything Cassandra related from `CassandraOutputFormatBase` then the resulting class can become the base class for all the output formats that want to leverage the flush logic. So we could promote it to `org.apache.flink.api.common.io` package as `OutputFormatBase`.  `CassandraOutputFormatBase` would extend this class and contain the cassandra deps.
   
   I think it requires also to create a dedicated ticket: https://issues.apache.org/jira/browse/FLINK-27884
   
   For now I'll just keep the classes in the cassandra connector module waiting for the decision to promote output format flush features to flink common io



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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1145913812

   @zentol I'm done addressing your last round of comments, PTAL. I hope now it is fine and we will not need another round of review.


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874541873


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
+    }
+
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequests maximum number of concurrent requests allowed
+     */
+    public void setMaxConcurrentRequests(int maxConcurrentRequests) {
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+    }
+
+    /**
+     * Configure the connection to Cassandra.
+     *
+     * @param parameters The configuration with all parameters.
+     */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
     /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
+     * Opens a Session to Cassandra .
      *
      * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
+     * @throws IOException Thrown, if the output format could not be opened due to an I/O problem.
      */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {

Review Comment:
   Yes, remember (cf PR description) that `CassandraOutputFormat` is no more the base class for row and tuple based output formats, it is now the base class for all the output formats.
   `V` is used as  `Void` in Pojo, and as `ResultSet` for `CassandraColumnarOutputFormatBase` (new base class for tuple and row output formats). 
   Regarding coding style of type variables vs wildcard I used the same coding style as in `CassandraSinkBase` (type variable). Generally speaking I prefer using type variables when possible rather than wildcard to avoid capture.
   
   
   



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874497578


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   As said in the other comment, it is just a proposed change. I did this change as `tryAcquire` already threw exceptions and `OutputFormat` is not user facing so I felt we could change to a more generic exception throwing. But it is true that even if it is not a user braking change, it is a source-compatibility breaking change. So if you prefer I'll make `tryAcquire` throw an unchecked exception in both Sink and OutputFormat so that we don't need to change the throw clause.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873541618


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   -1; this change breaks source-compatibility with all existing implementations.
   
   Need to double-check why japicmp didn't notice 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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873549773


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+public abstract class CassandraColumnarOutputFormatBase<OUT>

Review Comment:
   we could think about making this package-private.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r873552991


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {

Review Comment:
   these shouldn't be setters but a constructor argument.



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876820042


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();

Review Comment:
   You don't need the latch; use `CheckedThread#sync` instead.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877192550


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();
-        this.prepared = session.prepare(insertQuery);
         this.callback =
-                new FutureCallback<ResultSet>() {
+                new FutureCallback<V>() {
                     @Override
-                    public void onSuccess(ResultSet ignored) {
-                        onWriteSuccess(ignored);
+                    public void onSuccess(V ignored) {
+                        semaphore.release();
                     }
 
                     @Override
                     public void onFailure(Throwable t) {
-                        onWriteFailure(t);
+                        throwable.compareAndSet(null, t);
+                        LOG.error("Error while writing value.", t);
+                        semaphore.release();
                     }
                 };
     }
 
+    private void flush() {
+        tryAcquire(maxConcurrentRequests);
+        semaphore.release(maxConcurrentRequests);
+    }
+
+    private void tryAcquire(int permits) {
+        try {
+            SinkUtils.tryAcquire(
+                    permits, maxConcurrentRequests, maxConcurrentRequestsTimeout, semaphore);
+        } catch (Exception e) {
+            throw new RuntimeException(e);

Review Comment:
   yes, it would be clearer thx for the suggestion



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877234797


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(
+            String insertQuery,
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
+        super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /** Opens a Session to Cassandra and initializes the prepared statement. */
+    @Override
+    public void open(int taskNumber, int numTasks) throws IOException {
+        super.open(taskNumber, numTasks);
+        this.prepared = session.prepare(insertQuery);
+    }
+
+    @Override
+    public ListenableFuture<ResultSet> send(OUT record) {

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1121050723

   R @zentol 


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874491149


##########
pom.xml:
##########
@@ -2134,6 +2134,7 @@ under the License.
 								<exclude>org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object)</exclude>
 								<exclude>org.apache.flink.streaming.api.functions.sink.SinkFunction</exclude>
+								<exclude>org.apache.flink.api.common.io.OutputFormat</exclude>

Review Comment:
   yes as I said in the commit message as it was not user facing I proposed this exception, but it is just a proposition. If it is not the correct move I can definitely do otherwise. I just did not know what was acceptable: I mean `SinkFunctions` for example were already in the exceptions.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r874964198


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,191 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
 
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    public CassandraOutputFormatBase(ClusterBuilder builder) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
     }
 
+    /**
+     * Sets the maximum allowed number of concurrent requests for this output format.
+     *
+     * @param maxConcurrentRequestsTimeout timeout duration when acquiring a permit to execute
+     */
+    public void setMaxConcurrentRequestsTimeout(Duration maxConcurrentRequestsTimeout) {

Review Comment:
   :+1: 



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r875643870


##########
flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java:
##########
@@ -71,9 +71,10 @@
      * <p>When this method is called, the output format it guaranteed to be opened.
      *
      * @param record The records to add to the output.
-     * @throws IOException Thrown, if the records could not be added to to an I/O problem.
+     * @throws Exception Thrown, if the records could not be added due to an I/O problem or a
+     *     timeout.
      */
-    void writeRecord(IT record) throws IOException;
+    void writeRecord(IT record) throws Exception;

Review Comment:
   Do you want that I change the exception throwing to unchecked also in Sink for consistency with OutputFormat? If positive, the impact would be a breaking change for the user in exception management: in case of exception they will get `RuntimeException(e)` in place of `e`. I would prefer do the change only on the OutputFormat to avoid this breaking change in the sinks.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r887888690


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   I'm thinking that if we remove everything Cassandra related from `CassandraOutputFormatBase` then this class can become the base class for all the output formats that want to leverage the flush logic. So we could promote the resulting class to `org.apache.flink.api.common.io` package as `OutputFormatBase`.  `CassandraOutputFormatBase` would extend this class and contain the cassandra deps.
   
   I think it requires also to create a dedicated ticket
   
   I'll do the above code so that we discuss on the code itself if needed.



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


[GitHub] [flink] jeromatron commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
jeromatron commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1142382894

   I just wanted to say thank you @echauchot and @zentol for all the work on this and the other patch for https://issues.apache.org/jira/browse/FLINK-26824.  Those using Flink with more recent versions of Cassandra really appreciate this work.


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

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

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


[GitHub] [flink] echauchot commented on pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on PR #19680:
URL: https://github.com/apache/flink/pull/19680#issuecomment-1149603385

   thanks for the review @zentol 


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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877238047


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+        extends CassandraOutputFormatBase<OUT, ResultSet> {
+    private final String insertQuery;
+    private transient PreparedStatement prepared;
+
+    public CassandraColumnarOutputFormatBase(
+            String insertQuery,
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
+        super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+        Preconditions.checkArgument(
+                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+        this.insertQuery = insertQuery;
+    }
+
+    /** Opens a Session to Cassandra and initializes the prepared statement. */

Review Comment:
   :+1: 



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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876816993


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)

Review Comment:
   we _generally_ don't use timeouts.



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)

Review Comment:
   we _generally_ don't use timeouts in tests.



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

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

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


[GitHub] [flink] zentol commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876818398


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued

Review Comment:
   This seems misleading; it's failing because the previous write has failed.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r877207093


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {

Review Comment:
   Indeed. You're totally right ! I missed this pojo signature change. I'll go through all the changes to seak exception signature changes in user-facing classes to ensure I haven't missed another one.



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

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

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r878323055


##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
 
 package org.apache.flink.batch.connectors.cassandra;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.io.RichOutputFormat;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
 import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
 import org.apache.flink.util.Preconditions;
 
 import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
 import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into Apache Cassandra using
+ * output formats.
  *
  * @param <OUT> Type of the elements to write.
  */
-public abstract class CassandraOutputFormatBase<OUT> extends RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends RichOutputFormat<OUT> {
     private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormatBase.class);
 
-    private final String insertQuery;
     private final ClusterBuilder builder;
+    private Semaphore semaphore;
+    private Duration maxConcurrentRequestsTimeout = Duration.ofMillis(Long.MAX_VALUE);
+    private int maxConcurrentRequests = Integer.MAX_VALUE;
 
     private transient Cluster cluster;
-    private transient Session session;
-    private transient PreparedStatement prepared;
-    private transient FutureCallback<ResultSet> callback;
-    private transient Throwable exception = null;
-
-    public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) {
-        Preconditions.checkArgument(
-                !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty");
+    protected transient Session session;
+    private transient FutureCallback<V> callback;
+    private AtomicReference<Throwable> throwable;
+
+    public CassandraOutputFormatBase(
+            ClusterBuilder builder,
+            int maxConcurrentRequests,
+            Duration maxConcurrentRequestsTimeout) {
         Preconditions.checkNotNull(builder, "Builder cannot be null");
-
-        this.insertQuery = insertQuery;
         this.builder = builder;
+        Preconditions.checkArgument(
+                maxConcurrentRequests > 0, "Max concurrent requests is expected to be positive");
+        this.maxConcurrentRequests = maxConcurrentRequests;
+        Preconditions.checkNotNull(
+                maxConcurrentRequestsTimeout, "Max concurrent requests timeout cannot be null");
+        Preconditions.checkArgument(
+                !maxConcurrentRequestsTimeout.isNegative(),
+                "Max concurrent requests timeout is expected to be positive");
+        this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
     }
 
+    /** Configure the connection to Cassandra. */
     @Override
     public void configure(Configuration parameters) {
         this.cluster = builder.getCluster();
     }
 
-    /**
-     * Opens a Session to Cassandra and initializes the prepared statement.
-     *
-     * @param taskNumber The number of the parallel instance.
-     * @throws IOException Thrown, if the output could not be opened due to an I/O problem.
-     */
+    /** Opens a Session to Cassandra . */
     @Override
     public void open(int taskNumber, int numTasks) throws IOException {
+        throwable = new AtomicReference<>();
+        this.semaphore = new Semaphore(maxConcurrentRequests);
         this.session = cluster.connect();

Review Comment:
   Well, I don't like mocks either but remember the pojo and columnar (tuple and row) output formats are properly tested in `CassandraConnectorITCase`; In this ITest they actually write to a real Cassandra testContainer (that I implemented in another PR). So the interactions with the Cassandra driver are already tested there. `CassandraOutputFormatBaseTest` is just for unit tests of the flush mechanism (semaphore management, exceptions etc...) so I think a mock of the Cassandra driver which is not really needed looks fine to me. And I think that I prefer that than duplicating code.



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

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

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879271060


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");

Review Comment:
   If we don't complete the future, then the callback of the write will not be the one that will throw the timeout exception but it will be thrown later when close() is called (cf checkAsyncErrors). As in this test we check that write throws, I suggest that we leave the test as it is to avoid the ambiguity of asserting that close throws.



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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879219892


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)

Review Comment:
   :+1: 



##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)

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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] echauchot commented on a diff in pull request #19680: [FLINK-27457] Implement flush() logic in Cassandra output formats

Posted by GitBox <gi...@apache.org>.
echauchot commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r879271060


##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+    private static final long DEFAULT_TEST_TIMEOUT = 5000;
+    private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+            Duration.ofMillis(Long.MAX_VALUE);
+
+    @Test(expected = NoHostAvailableException.class)
+    public void testHostNotFoundErrorHandling() throws Exception {
+        CassandraOutputFormatBase cassandraOutputFormatBase =
+                new CassandraOutputFormatBase(
+                        new ClusterBuilder() {
+                            @Override
+                            protected Cluster buildCluster(Cluster.Builder builder) {
+                                return builder.addContactPoint("127.0.0.1")
+                                        .withoutJMXReporting()
+                                        .withoutMetrics()
+                                        .build();
+                            }
+                        },
+                        Integer.MAX_VALUE,
+                        DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT) {
+                    @Override
+                    public ListenableFuture send(Object value) {
+                        return null;
+                    }
+                };
+        cassandraOutputFormatBase.configure(new Configuration());
+        cassandraOutputFormatBase.open(1, 1);
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testSuccessfulWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    CompletableFuture.completedFuture(null));
+
+            final int originalPermits = testCassandraOutputFormat.getAvailablePermits();
+            assertTrue(originalPermits > 0);
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            Assert.assertEquals(originalPermits, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnClose() throws Exception {
+        TestCassandraOutputFormat testCassandraOutputFormat = createTestCassandraOutputFormat();
+        testCassandraOutputFormat.open(1, 1);
+
+        Exception cause = new RuntimeException();
+        testCassandraOutputFormat.enqueueCompletableFuture(
+                FutureUtils.completedExceptionally(cause));
+        testCassandraOutputFormat.writeRecord("hello");
+        try {
+            testCassandraOutputFormat.close();
+            Assert.fail("Close should have thrown an exception.");
+        } catch (IOException e) {
+            ExceptionUtils.findThrowable(e, candidate -> candidate == cause).orElseThrow(() -> e);
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testThrowErrorOnWrite() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Exception cause = new RuntimeException();
+            testCassandraOutputFormat.enqueueCompletableFuture(
+                    FutureUtils.completedExceptionally(cause));
+
+            testCassandraOutputFormat.writeRecord("hello");
+
+            try {
+                testCassandraOutputFormat.writeRecord("world");
+                // should fail because only one completableFuture was enqueued
+                Assert.fail("Sending of second value should have failed.");
+            } catch (IOException e) {
+                Assert.assertEquals(cause, e.getCause());
+                Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+            }
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testWaitForPendingUpdatesOnClose() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+
+            testCassandraOutputFormat.writeRecord("hello");
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Thread t =
+                    new CheckedThread("Flink-CassandraOutputFormatBaseTest") {
+                        @Override
+                        public void go() throws Exception {
+                            testCassandraOutputFormat.close();
+                            latch.countDown();
+                        }
+                    };
+            t.start();
+            while (t.getState() != Thread.State.TIMED_WAITING) {
+                Thread.sleep(5);
+            }
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+            // start writing
+            completableFuture.complete(null);
+            latch.await();
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnSuccess() throws Exception {
+        try (TestCassandraOutputFormat openedTestCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            openedTestCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            openedTestCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAcquiredPermits());
+
+            // start writing
+            completableFuture.complete(null);
+
+            Assert.assertEquals(1, openedTestCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, openedTestCassandraOutputFormat.getAcquiredPermits());
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnFailure() throws Exception {
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedTestCassandraOutputFormat()) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            CompletableFuture<ResultSet> completableFuture = new CompletableFuture<>();
+            testCassandraOutputFormat.enqueueCompletableFuture(completableFuture);
+            testCassandraOutputFormat.writeRecord("N/A");
+
+            Assert.assertEquals(0, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(1, testCassandraOutputFormat.getAcquiredPermits());
+
+            completableFuture.completeExceptionally(new RuntimeException());
+
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+        } catch (IOException ignored) {
+            // format.close() throws the exception gathered in format.writeRecord()
+        }
+    }
+
+    @Test(timeout = DEFAULT_TEST_TIMEOUT)
+    public void testReleaseOnThrowingSend() throws Exception {
+        Function<String, ListenableFuture<ResultSet>> failingSendFunction =
+                ignoredMessage -> {
+                    throwCheckedAsUnchecked(new Throwable("expected"));
+                    //noinspection ReturnOfNull
+                    return null;
+                };
+
+        try (TestCassandraOutputFormat testCassandraOutputFormat =
+                createOpenedMockOutputFormat(failingSendFunction)) {
+            Assert.assertEquals(1, testCassandraOutputFormat.getAvailablePermits());
+            Assert.assertEquals(0, testCassandraOutputFormat.getAcquiredPermits());
+
+            //noinspection OverlyBroadCatchBlock,NestedTryStatement
+            try {
+                testCassandraOutputFormat.writeRecord("none");

Review Comment:
   :+1: 



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