[ https://issues.apache.org/jira/browse/FLINK-10310?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16614371#comment-16614371 ]
Jayant Ameta commented on FLINK-10310: -------------------------------------- Hi [~till.rohrmann] {{CassandraSinkBase}} would have a field similar to {{ActionRequestFailureHandler}}. In the {{checkAsyncErrors}} method, the failureHandler would be called instead of throwing the {{IOException}} Current code snippet {code:java} private void checkAsyncErrors() throws Exception { Throwable error = exception; if (error != null) { // prevent throwing duplicated error exception = null; throw new IOException("Error while sending value.", error); } } {code} would change to: {code:java} private void checkAsyncErrors() throws Exception { Throwable error = exception; if (error != null) { failureHandler.onFailure(error); } } {code} Here the {{failureHandler}} can decide what steps to take based on the {{Throwable}}. > Cassandra Sink - Handling failing requests > ------------------------------------------ > > Key: FLINK-10310 > URL: https://issues.apache.org/jira/browse/FLINK-10310 > Project: Flink > Issue Type: Improvement > Components: Cassandra Connector > Reporter: Jayant Ameta > Priority: Major > > The cassandra sink fails for any kind of error. For some exceptions (e.g > WriteTimeoutException), ignoring the exception may be acceptable as well. > Can we discuss having a FailureHandler on the lines of > ActionRequestFailureHandler? > http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/WriteTimeoutException-in-Cassandra-sink-kill-the-job-tp22706p22707.html -- This message was sent by Atlassian JIRA (v7.6.3#76005)