You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/02 07:16:00 UTC

[jira] [Commented] (FLINK-9083) Add async backpressure support to Cassandra Connector

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

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

jparkie opened a new pull request #6782: [FLINK-9083][Cassandra Connector] Add async backpressure support to Cassandra Connector
URL: https://github.com/apache/flink/pull/6782
 
 
   ## What is the purpose of the change
   
   This pull requests rewrites `CassandraSinkBase` to use a `Phaser` and `Semaphore` to provide proper synchronization to support `maxConcurrentRequests` as a new configuration. This improves the reliability of the Cassandra Connector as it can currently overwhelm a weak Cassandra cluster if the upstream source has very high throughput.
   
   ## Brief change log
   
     - Rewrote `CassandraSinkBase` to use a `Phaser` and `Semaphore`.
     - Expose `public void setMaxConcurrentRequests(int maxConcurrentRequests, long timeout, TimeUnit unit)` on `CassandraSinkBase`.
     - Modify `CassandraSink` with the new configuration. It currently does not support the WAL.
     - Updated the documentation about the new configuration.
   
   ## Verifying this change
   
   This change is already covered by existing tests, such as `CassandraSinkBaseTest` and `CassandraConnectorITCase`.
   
   This change added tests and can be verified as follows:
   
     - Added tests for acquiring permits from the `Semaphore` and releasing permits from the `Semaphore` when a write succeeds or fails.
   
   ## 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): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, 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? docs / JavaDocs
   

----------------------------------------------------------------
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:
users@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)