[
https://issues.apache.org/jira/browse/FLINK-17820?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17112643#comment-17112643
]
Stephan Ewen commented on FLINK-17820:
--------------------------------------
I would simply avoid flushing here. It makes no difference, persistence wise.
Persistence is only guaranteed once you call {{closeAndGetHandle()}}.
There are few reasons ever to call flush on a stream (usually only when it is
pipelined to a receiver and you want to make sure that the receiver receives
buffered data, which I assume is not the case here, as this is no network
stream).
(The reference to {{sync}} is not quite right in the JavaDocs. For some
FileSystems, like S3, {{sync()}} does not help at all).
I would not change {{FsCheckpointStateOutputStream}}, this is a class that does
exactly what it should do at the moment.
Wrapping also seems like unnecessary complexity, which we should avoid whenever
possible.
> Memory threshold is ignored for channel state
> ---------------------------------------------
>
> Key: FLINK-17820
> URL: https://issues.apache.org/jira/browse/FLINK-17820
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing, Runtime / Task
> Affects Versions: 1.11.0
> Reporter: Roman Khachatryan
> Assignee: Roman Khachatryan
> Priority: Critical
> Labels: pull-request-available
> Fix For: 1.11.0
>
>
> Config parameter state.backend.fs.memory-threshold is ignored for channel
> state. Causing each subtask to have a file per checkpoint. Regardless of the
> size of channel state (of this subtask).
> This also causes slow cleanup and delays the next checkpoint.
>
> The problem is that {{ChannelStateCheckpointWriter.finishWriteAndResult}}
> calls flush(); which actually flushes the data on disk.
>
> From FSDataOutputStream.flush Javadoc:
> A completed flush does not mean that the data is necessarily persistent. Data
> persistence can is only assumed after calls to close() or sync().
>
> Possible solutions:
> 1. not to flush in {{ChannelStateCheckpointWriter.finishWriteAndResult (which
> can lead to data loss in a wrapping stream).}}
> {{2. change }}{{FsCheckpointStateOutputStream.flush behavior}}
> {{3. wrap }}{{FsCheckpointStateOutputStream to prevent flush}}{{}}{{}}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)