[ 
https://issues.apache.org/jira/browse/FLINK-9083?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646575#comment-16646575
 ] 

ASF GitHub Bot commented on FLINK-9083:
---------------------------------------

azagrebin commented on a change in pull request #6782: [FLINK-9083][Cassandra 
Connector] Add async backpressure support to Cassandra Connector
URL: https://github.com/apache/flink/pull/6782#discussion_r224486540
 
 

 ##########
 File path: 
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java
 ##########
 @@ -204,23 +206,113 @@ public void go() throws Exception {
                        Thread.sleep(5);
                }
 
-               Assert.assertEquals(1, casSinkFunc.getNumOfPendingRecords());
+               Assert.assertEquals(1, casSinkFunc.getAcquiredPermits());
                completableFuture.complete(null);
-               Assert.assertEquals(0, casSinkFunc.getNumOfPendingRecords());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
        }
 
-       private static class TestCassandraSink extends 
CassandraSinkBase<String, ResultSet> {
+       @Test(timeout = 5000)
+       public void testAcquireOnInvoke() throws Exception {
+               TestCassandraSink casSinkFunc = new TestCassandraSink();
+               casSinkFunc.setMaxConcurrentRequests(1, 5, 
TimeUnit.MILLISECONDS);
+
+               casSinkFunc.open(new Configuration());
+
+               CompletableFuture<ResultSet> completableFuture = new 
CompletableFuture<>();
+
+               
casSinkFunc.setResultSetFuture(ResultSetFutures.fromCompletableFuture(completableFuture));
+
+               Assert.assertEquals(1, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
+
+               casSinkFunc.invoke("hello");
+
+               Assert.assertEquals(0, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(1, casSinkFunc.getAcquiredPermits());
+
+               completableFuture.complete(null);
+
+               casSinkFunc.close();
+       }
+
+       @Test(timeout = 5000)
+       public void testReleaseOnSuccess() throws Exception {
+               TestCassandraSink casSinkFunc = new TestCassandraSink();
+               casSinkFunc.setMaxConcurrentRequests(1, 5, 
TimeUnit.MILLISECONDS);
+
+               casSinkFunc.open(new Configuration());
+
+               CompletableFuture<ResultSet> completableFuture = new 
CompletableFuture<>();
+
+               
casSinkFunc.setResultSetFuture(ResultSetFutures.fromCompletableFuture(completableFuture));
+
+               Assert.assertEquals(1, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
+
+               casSinkFunc.invoke("hello");
+
+               Assert.assertEquals(0, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(1, casSinkFunc.getAcquiredPermits());
+
+               completableFuture.complete(null);
+
+               Assert.assertEquals(1, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
+
+               casSinkFunc.close();
+       }
+
+       @Test(timeout = 5000)
+       public void testReleaseOnFailure() throws Exception {
+               TestCassandraSink casSinkFunc = new TestCassandraSink();
+               casSinkFunc.setMaxConcurrentRequests(1, 5, 
TimeUnit.MILLISECONDS);
+
+               casSinkFunc.open(new Configuration());
+
+               CompletableFuture<ResultSet> completableFuture = new 
CompletableFuture<>();
+
+               
casSinkFunc.setResultSetFuture(ResultSetFutures.fromCompletableFuture(completableFuture));
+
+               Assert.assertEquals(1, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
+
+               casSinkFunc.invoke("hello");
+
+               Assert.assertEquals(0, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(1, casSinkFunc.getAcquiredPermits());
+
+               completableFuture.completeExceptionally(new RuntimeException());
+
+               Assert.assertEquals(1, casSinkFunc.getAvailablePermits());
+               Assert.assertEquals(0, casSinkFunc.getAcquiredPermits());
+       }
+
+       @Test(timeout = 5000)
+       public void testTimeoutExceptionOnInvoke() throws Exception {
+               TestCassandraSink casSinkFunc = new TestCassandraSink();
+               casSinkFunc.setMaxConcurrentRequests(0, 5, 
TimeUnit.MILLISECONDS);
+
+               casSinkFunc.open(new Configuration());
+
+               
casSinkFunc.setResultSetFuture(ResultSetFutures.fromCompletableFuture(CompletableFuture.completedFuture(null)));
 
+               try {
+                       casSinkFunc.invoke("hello");
+                       Assert.fail("Sending value should have experienced a 
TimeoutException");
+               } catch (Exception e) {
+                       Assert.assertTrue(e instanceof TimeoutException);
+               }
+       }
+
+       private static class TestCassandraSink extends 
CassandraSinkBase<String, ResultSet> {
 
 Review comment:
   can be `AutoClosable` to use with try with resources

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Add async backpressure support to Cassandra Connector
> -----------------------------------------------------
>
>                 Key: FLINK-9083
>                 URL: https://issues.apache.org/jira/browse/FLINK-9083
>             Project: Flink
>          Issue Type: Improvement
>          Components: Cassandra Connector
>            Reporter: Jacob Park
>            Assignee: Jacob Park
>            Priority: Minor
>              Labels: pull-request-available
>
> As the CassandraSinkBase derivatives utilize async writes, they do not block 
> the task to introduce any backpressure.
> I am currently using a semaphore to provide backpressure support by blocking 
> at a maximum concurrent requests limit like how DataStax's Spark Cassandra 
> Connector functions: 
> [https://github.com/datastax/spark-cassandra-connector/blob/v2.0.7/spark-cassandra-connector/src/main/scala/com/datastax/spark/connector/writer/AsyncExecutor.scala#L18]
> This improvement has greatly improved the fault-tolerance of our Cassandra 
> Sink Connector implementation on Apache Flink in production. I would like to 
> contribute this feature back upstream.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to