flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9325) generate the _meta file for checkpoint only when the writing is truly successful
Date Tue, 15 May 2018 14:38:00 GMT

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

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

Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5982
  
    Good point about the renaming on `close()` in case close is called for cleanup, rather
than success.
    
    We could follow the same semantics as in [CheckpointStateOutputStream](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java#L61)
    
    There the semantics are:
      - `close()` means "close on error / cleanup" and closes the stream and deletes the temp
file.
      - `closeAndPublish()` would mean "close on success" and close the stream and rename
the file.
      - After ``closeAndPublish()` has been called, `close()` becomes a no-op.
    
    The [FsCheckpointMetadataOutputStream](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java)
implements that pattern, I think it worked well and is easy to use.
    



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

Mime
View raw message