[ https://issues.apache.org/jira/browse/FLINK-9083?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16693718#comment-16693718 ]
ASF GitHub Bot commented on FLINK-9083: --------------------------------------- zentol 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_r235144945 ########## File path: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java ########## @@ -84,111 +90,108 @@ public void testThrowErrorOnClose() throws Exception { casSinkFunc.open(new Configuration()); Exception cause = new RuntimeException(); - casSinkFunc.setResultFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); + casSinkFunc.enqueueResultSetFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); casSinkFunc.invoke("hello"); try { casSinkFunc.close(); Assert.fail("Close should have thrown an exception."); } catch (IOException e) { Assert.assertEquals(cause, e.getCause()); - Assert.assertEquals(0, casSinkFunc.getNumOfPendingRecords()); + Assert.assertEquals(0, casSinkFunc.getAcquiredPermits()); } } @Test(timeout = 5000) public void testThrowErrorOnInvoke() throws Exception { - TestCassandraSink casSinkFunc = new TestCassandraSink(); + try (TestCassandraSink casSinkFunc = new TestCassandraSink()) { + casSinkFunc.open(new Configuration()); - casSinkFunc.open(new Configuration()); - - Exception cause = new RuntimeException(); - casSinkFunc.setResultFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); + Exception cause = new RuntimeException(); + casSinkFunc.enqueueResultSetFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); - casSinkFunc.invoke("hello"); + casSinkFunc.invoke("hello"); - try { - casSinkFunc.invoke("world"); - Assert.fail("Sending of second value should have failed."); - } catch (IOException e) { - Assert.assertEquals(cause, e.getCause()); - Assert.assertEquals(0, casSinkFunc.getNumOfPendingRecords()); + try { + casSinkFunc.invoke("world"); + Assert.fail("Sending of second value should have failed."); + } catch (IOException e) { + Assert.assertEquals(cause, e.getCause()); + Assert.assertEquals(0, casSinkFunc.getAcquiredPermits()); + } } } @Test(timeout = 5000) public void testIgnoreError() throws Exception { Exception cause = new RuntimeException(); CassandraFailureHandler failureHandler = failure -> Assert.assertEquals(cause, failure); - TestCassandraSink casSinkFunc = new TestCassandraSink(failureHandler); + try (TestCassandraSink casSinkFunc = new TestCassandraSink(failureHandler)) { + casSinkFunc.open(new Configuration()); - casSinkFunc.open(new Configuration()); + casSinkFunc.enqueueResultSetFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); + casSinkFunc.enqueueResultSetFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); - casSinkFunc.setResultFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); - - casSinkFunc.invoke("hello"); - casSinkFunc.invoke("world"); + casSinkFunc.invoke("hello"); + casSinkFunc.invoke("world"); + } } @Test(timeout = 5000) public void testThrowErrorOnSnapshot() throws Exception { TestCassandraSink casSinkFunc = new TestCassandraSink(); - OneInputStreamOperatorTestHarness<String, Object> testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(casSinkFunc)); - - testHarness.open(); + try (OneInputStreamOperatorTestHarness<String, Object> testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(casSinkFunc))) { + testHarness.open(); - Exception cause = new RuntimeException(); - casSinkFunc.setResultFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); + Exception cause = new RuntimeException(); + casSinkFunc.enqueueResultSetFuture(ResultSetFutures.fromCompletableFuture(FutureUtils.getFailedFuture(cause))); - casSinkFunc.invoke("hello"); + casSinkFunc.invoke("hello"); - try { - testHarness.snapshot(123L, 123L); + try { Review comment: I would appreciate if we would move the majority of these changes into a separate commit. (try-catch introduction mostly). ---------------------------------------------------------------- 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)