[
https://issues.apache.org/jira/browse/FLINK-5355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15823234#comment-15823234
]
ASF GitHub Bot commented on FLINK-5355:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/3078#discussion_r96145754
--- Diff:
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
---
@@ -253,6 +262,35 @@ public String getShardIterator(KinesisStreamShard
shard, String shardIteratorTyp
return getShardIteratorResult.getShardIterator();
}
+ /**
+ * Determines whether the exception can be recovered from using
+ * exponential-backoff
+ *
+ * @param ex
+ * Exception to inspect
+ * @return <code>true</code> if the exception can be recovered from,
else
+ * <code>false</code>
+ */
+ protected static boolean isRecoverableException(AmazonServiceException
ex) {
+ if (ex.getErrorType() == null) {
+ return false;
+ }
+
+ switch (ex.getErrorType()) {
--- End diff --
The indentation for the cases here seem to be missing.
> Handle AmazonKinesisException gracefully in Kinesis Streaming Connector
> -----------------------------------------------------------------------
>
> Key: FLINK-5355
> URL: https://issues.apache.org/jira/browse/FLINK-5355
> Project: Flink
> Issue Type: Improvement
> Components: Kinesis Connector
> Affects Versions: 1.2.0, 1.1.3
> Reporter: Scott Kidder
> Assignee: Scott Kidder
>
> My Flink job that consumes from a Kinesis stream must be restarted at least
> once daily due to an uncaught AmazonKinesisException when reading from
> Kinesis. The complete stacktrace looks like:
> {noformat}
> com.amazonaws.services.kinesis.model.AmazonKinesisException: null (Service:
> AmazonKinesis; Status Code: 500; Error Code: InternalFailure; Request ID:
> dc1b7a1a-1b97-1a32-8cd5-79a896a55223)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1545)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1183)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:964)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:676)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:650)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:633)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$300(AmazonHttpClient.java:601)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:583)
> at
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:447)
> at
> com.amazonaws.services.kinesis.AmazonKinesisClient.doInvoke(AmazonKinesisClient.java:1747)
> at
> com.amazonaws.services.kinesis.AmazonKinesisClient.invoke(AmazonKinesisClient.java:1723)
> at
> com.amazonaws.services.kinesis.AmazonKinesisClient.getRecords(AmazonKinesisClient.java:858)
> at
> org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy.getRecords(KinesisProxy.java:193)
> at
> org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer.getRecords(ShardConsumer.java:268)
> at
> org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer.run(ShardConsumer.java:176)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> It's interesting that the Kinesis endpoint returned a 500 status code, but
> that's outside the scope of this issue.
> I think we can handle this exception in the same manner as a
> ProvisionedThroughputException: performing an exponential backoff and
> retrying a finite number of times before throwing an exception.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)