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-6364) Implement incremental checkpointing in RocksDBStateBackend
Date Tue, 02 May 2017 17:33:05 GMT

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

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

Github user StefanRRichter commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3801#discussion_r114325070
  
    --- Diff: flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
---
    @@ -265,9 +281,64 @@ private boolean hasRegisteredState() {
     			final CheckpointStreamFactory streamFactory,
     			CheckpointOptions checkpointOptions) throws Exception {
     
    +		if (checkpointOptions.getCheckpointType() == CheckpointOptions.CheckpointType.INCREMENTAL_CHECKPOINT)
{
    +			return snapshotIncrementally(checkpointId, timestamp, streamFactory);
    +		} else {
    +			return snapshotFully(checkpointId, timestamp, streamFactory);
    +		}
    +	}
    +
    +	private RunnableFuture<KeyedStateHandle> snapshotIncrementally(
    +			final long checkpointId,
    --- End diff --
    
    This method has some amount of duplicated code with `snapshotFully`. My suggestion would
be target for some common abstract strategy common to all checkpointing, e.g. `takeSnapshot`,
`materialize`, and `releaseResources`. Then we could have a factory that, depending on the
checkpoint type, instantiates the right implementation. This would also help with a second
concern: I can see that the code in this class has grown a lot over time, so this could be
the time to move some aspects like checkpointing strategies into separated classes (they are
already static inner classes anyways). By splitting a bit between processing logic and snapshot/restore
logic, we can keep things more modular and separation of concerns.
    
    We can also do this cleanup in a followup PR.


> Implement incremental checkpointing in RocksDBStateBackend
> ----------------------------------------------------------
>
>                 Key: FLINK-6364
>                 URL: https://issues.apache.org/jira/browse/FLINK-6364
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>            Reporter: Xiaogang Shi
>            Assignee: Xiaogang Shi
>
> {{RocksDBStateBackend}} is well suited for incremental checkpointing because RocksDB
is base on LSM trees,  which record updates in new sst files and all sst files are immutable.
By only materializing those new sst files, we can significantly improve the performance of
checkpointing.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message