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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to