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_r224481690
 
 

 ##########
 File path: 
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
 ##########
 @@ -43,70 +48,82 @@
  */
 public abstract class CassandraSinkBase<IN, V> extends RichSinkFunction<IN> 
implements CheckpointedFunction {
        protected final Logger log = LoggerFactory.getLogger(getClass());
-       protected transient Cluster cluster;
-       protected transient Session session;
 
-       protected transient volatile Throwable exception;
-       protected transient FutureCallback<V> callback;
+       // ------------------------ Default Configurations 
------------------------
+
+       /**
+        * The default maximum number of concurrent requests. By default, 
{@code Integer.MAX_VALUE}.
+        */
+       public static final int DEFAULT_MAX_CONCURRENT_REQUESTS = 
Integer.MAX_VALUE;
+
+       /**
+        * The default timeout duration when acquiring a permit to execute. By 
default, {@code Long.MAX_VALUE}.
+        */
+       public static final long DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT = 
Long.MAX_VALUE;
+
+       /**
+        * The default timeout unit when acquiring a permit to execute. By 
default, milliseconds.
+        */
+       public static final TimeUnit 
DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT_UNIT = TimeUnit.MILLISECONDS;
+
+       // ------------------------- Configuration Fields 
-------------------------
+
+       private int maxConcurrentRequests = DEFAULT_MAX_CONCURRENT_REQUESTS;
+       private long maxConcurrentRequestsTimeout = 
DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT;
+       private TimeUnit maxConcurrentRequestsTimeoutUnit = 
DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT_UNIT;
+
+       // --------------------------- Cassandra Fields 
---------------------------
 
        private final ClusterBuilder builder;
 
-       private final AtomicInteger updatesPending = new AtomicInteger();
+       protected transient Cluster cluster;
+       protected transient Session session;
+
+       // ------------------------ Synchronization Fields 
------------------------
+
+       private AtomicReference<Throwable> throwable;
+       private Semaphore semaphore;
+       private Phaser phaser;
 
        CassandraSinkBase(ClusterBuilder builder) {
                this.builder = builder;
                ClosureCleaner.clean(builder, true);
        }
 
-       @Override
-       public void open(Configuration configuration) {
-               this.callback = new FutureCallback<V>() {
-                       @Override
-                       public void onSuccess(V ignored) {
-                               int pending = updatesPending.decrementAndGet();
-                               if (pending == 0) {
-                                       synchronized (updatesPending) {
-                                               updatesPending.notifyAll();
-                                       }
-                               }
-                       }
+       // ----------------------------- Sink Methods 
-----------------------------
 
+       @Override
+       public void open(Configuration parameters) {
+               cluster = createCluster();
+               session = createSession();
+
+               throwable = new AtomicReference<>();
+               semaphore = new Semaphore(maxConcurrentRequests);
+               /*
+                * A Phaser is a flexible and reusable synchronization barrier 
similar to CyclicBarrier and CountDownLatch.
+                *
+                * This Phaser is configured to support "1 + N" parties.
+                *   - "1" for the CassandraSinkBase to arrive and to await at 
the Phaser during a flush() call.
+                *   - "N" for the varying number of invoke() calls that 
register and de-register with the Phaser.
+                *
+                * The Phaser awaits the completion of the advancement of a 
phase prior to returning from a register() call.
+                * This behavior ensures that no backlogged invoke() calls 
register to execute while the Semaphore's permits
+                * are being released during a flush() call.
+                */
+               phaser = new Phaser(1) {
 
 Review comment:
   Phaser solution looks good. I am wondering whether we need such low level 
approach. `send` and `snapshotState` are called synchronously. We could have a 
concurrent set of futures and use `FutureUtil.waitForAll` to sync in flush. 
This looks simpler to me. Is there a particular performance reason to use the 
`Phaser`?
   
   I would also abstract semaphore/Phaser/futures/addFuture/waitForAll/error 
away into another reusable and pluggable component. It can be also tested 
separately.

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


With regards,
Apache Git Services

Reply via email to