flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9083) Add async backpressure support to Cassandra Connector
Date Thu, 11 Oct 2018 15:13:01 GMT

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

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_r224475996
 
 

 ##########
 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;
 
 Review comment:
   maybe minor thing, I would rather say that these default constants belong to the builder
where they are actually used and can be documentation for themselves.

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

Mime
View raw message