You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/08/19 19:22:00 UTC

[jira] [Commented] (FLINK-9325) generate the _meta file for checkpoint only when the writing is truly successful

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

ASF GitHub Bot commented on FLINK-9325:
---------------------------------------

StephanEwen commented on issue #5982: [FLINK-9325][checkpoint]generate the meta file for checkpoint only when the writing is truly successful
URL: https://github.com/apache/flink/pull/5982#issuecomment-414149487
 
 
   Apologies for the delay.
   
   The interface of the `AtomicCreatingFsDataOutputStream` is good.
   As part of the new `StreamingFileSink` design, we created a [recoverable stream](https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/core/fs/RecoverableFsDataOutputStream.java), which is to some extend an extension of the `AtomicCreatingFsDataOutputStream` in the following way:
   
     - The recoverable stream also does not show the file immediately, but it needs to go through a committer.
     - The committer can be persisted and recovered
     - An intermediate status (some data having been written to the file) can also be persisted and recovered
   
   This is used in the StreamingFileSink to write data to a file, chunk by chunk, and make sure the state of an output file (as of a checkpoint) can be recovered.
   
   Now, can we use the same implementation for both AtomicCreatingFsDataOutputStream and RecoverableFsDataOutputStream? Parts of the logic are very simple (like using a temp file and renaming for HDFS, or using a multipart upload and committing later on S3).
   

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


> generate the _meta file for checkpoint only when the writing is truly successful
> --------------------------------------------------------------------------------
>
>                 Key: FLINK-9325
>                 URL: https://issues.apache.org/jira/browse/FLINK-9325
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.5.0
>            Reporter: Sihua Zhou
>            Assignee: Sihua Zhou
>            Priority: Major
>              Labels: pull-request-available
>
> We should generate the _meta file for checkpoint only when the writing is totally successful. We should write the metadata file first to a temp file and then atomically rename it (with an equivalent workaround for S3). 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)