[ https://issues.apache.org/jira/browse/FLINK-4218?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15516443#comment-15516443 ]
ASF GitHub Bot commented on FLINK-4218: --------------------------------------- GitHub user StephanEwen opened a pull request: https://github.com/apache/flink/pull/2544 [FLINK-4218] [checkpoints] Do not rely on FileSystem to determine state sizes This prevents failures on eventually consistent S3, where the operations for keys (=entries in the parent directory/bucket) are not guaranteed to be immediately consistent (visible) after a blob was written. Not relying on any operation on keys (= requesting `FileStatus`) should mitigate the problem. This also changes the exception signature from `getStateSize()` from `Exception` to `IOException`, which fits more natural with the exception signatures of some other I/O methods. Related issue: We may still want to have retries on `FileStatus` operations on S3, for other parts of the system (like FileOutputFormats) You can merge this pull request into a Git repository by running: $ git pull https://github.com/StephanEwen/incubator-flink state_size_fix Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2544.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2544 ---- commit 7ce2de7e14b9a1fb24c27dc674f689f08abdf7cc Author: Stephan Ewen <se...@apache.org> Date: 2016-09-23T13:16:27Z [FLINK-4218] [checkpoints] Do not rely on FileSystem to determing state sizes This prevents failures on eventually consistent S3, where the operations for keys (=entries in the parent directory/bucket) are not guaranteed to be immediately consistent (visible) after a blob was written. ---- > Sporadic "java.lang.RuntimeException: Error triggering a checkpoint..." > causes task restarting > ---------------------------------------------------------------------------------------------- > > Key: FLINK-4218 > URL: https://issues.apache.org/jira/browse/FLINK-4218 > Project: Flink > Issue Type: Improvement > Affects Versions: 1.1.0 > Reporter: Sergii Koshel > > Sporadically see exception as below. And restart of task because of it. > {code:title=Exception|borderStyle=solid} > java.lang.RuntimeException: Error triggering a checkpoint as the result of > receiving checkpoint barrier > at > org.apache.flink.streaming.runtime.tasks.StreamTask$3.onEvent(StreamTask.java:785) > at > org.apache.flink.streaming.runtime.tasks.StreamTask$3.onEvent(StreamTask.java:775) > at > org.apache.flink.streaming.runtime.io.BarrierBuffer.processBarrier(BarrierBuffer.java:203) > at > org.apache.flink.streaming.runtime.io.BarrierBuffer.getNextNonBlocked(BarrierBuffer.java:129) > at > org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:183) > at > org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:66) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:265) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:588) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.io.FileNotFoundException: No such file or directory: > s3://<bucket_name_here>/flink/checkpoints/ece317c26960464ba5de75f3bbc38cb2/chk-8810/96eebbeb-de14-45c7-8ebb-e7cde978d6d3 > at > org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:996) > at > org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:77) > at > org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.getFileStatus(HadoopFileSystem.java:351) > at > org.apache.flink.runtime.state.filesystem.AbstractFileStateHandle.getFileSize(AbstractFileStateHandle.java:93) > at > org.apache.flink.runtime.state.filesystem.FileStreamStateHandle.getStateSize(FileStreamStateHandle.java:58) > at > org.apache.flink.runtime.state.AbstractStateBackend$DataInputViewHandle.getStateSize(AbstractStateBackend.java:482) > at > org.apache.flink.streaming.runtime.tasks.StreamTaskStateList.getStateSize(StreamTaskStateList.java:77) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:604) > at > org.apache.flink.streaming.runtime.tasks.StreamTask$3.onEvent(StreamTask.java:779) > ... 8 more > {code} > File actually exists on S3. > I suppose it is related to some race conditions with S3 but would be good to > retry a few times before stop task execution. -- This message was sent by Atlassian JIRA (v6.3.4#6332)