[
https://issues.apache.org/jira/browse/FLINK-9083?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646573#comment-16646573
]
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_r224473345
##########
File path: docs/dev/connectors/cassandra.md
##########
@@ -72,10 +72,13 @@ The following configuration methods can be used:
4. _setMapperOptions(MapperOptions options)_
* Sets the mapper options that are used to configure the DataStax
ObjectMapper.
* Only applies when processing __POJO__ data types.
-5. _enableWriteAheadLog([CheckpointCommitter committer])_
+5. _setMaxConcurrentRequests(int maxConcurrentRequests, long timeout, TimeUnit
unit)_
+ * Sets the maximum allowed number of concurrent requests with a timeout
for acquiring permits to execute.
+ * Only applies when __enableWriteAheadLog()__ is not configured.
Review comment:
Potentially `CassandraTupleWriteAheadSink.sendValues` could be also
throttled and send values in batches of concurrent requests instead of trying
to flush at once all values accumulated between checkpoints. It could be useful
the same way. Is it just matter of implementing effort or there are more
concerns about this?
----------------------------------------------------------------
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:
[email protected]
> 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)