[
https://issues.apache.org/jira/browse/FLINK-3718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15239045#comment-15239045
]
ASF GitHub Bot commented on FLINK-3718:
---------------------------------------
GitHub user aljoscha opened a pull request:
https://github.com/apache/flink/pull/1879
[FLINK-3718] Add Option For Completely Async Backup in RocksDB State Backend
This also refactors the RocksDB backend to keep one RocksDB data base in
the backend where all key/value state is stored. Individual named
key/value states get a reference to the db and store their state in a
column family. This way, we only have to backup one RocksDB data base
and can centrally decide how to do backups.
@tillrohrmann I don't how much these changes clash with your work on
repartitionable state. Would it make sense to wait with this PR or can it go
ahead before your changes go in?
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/aljoscha/flink rocks-refactor
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/1879.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 #1879
----
commit 6b7123a2ff5db0a49896d58d09b0ba541fbb8135
Author: Aljoscha Krettek <[email protected]>
Date: 2016-04-08T12:58:50Z
[FLINK-3718] Add Option For Completely Async Backup in RocksDB State Backend
This also refactors the RocksDB backend to keep one RocksDB data base in
the backend where all key/value state is stored. Individual named
key/value states get a reference to the db and store their state in a
column family. This way, we only have to backup one RocksDB data base
and can centrally decide how to do backups.
----
> Add Option For Completely Async Backup in RocksDB State Backend
> ---------------------------------------------------------------
>
> Key: FLINK-3718
> URL: https://issues.apache.org/jira/browse/FLINK-3718
> Project: Flink
> Issue Type: Bug
> Components: Streaming
> Reporter: Aljoscha Krettek
> Assignee: Aljoscha Krettek
>
> Right now, the snapshotting for RocksDB has a synchronous part where a backup
> of the RocksDB database is drawn and an asynchronous part where this backup
> is written to HDFS.
> We should add an option that uses the snapshot feature of RocksDB to get an
> iterator over all keys at a set point in time. The iterator can be used to
> store everything to HDFS. Normal operation can continue while we store the
> keys. This makes the snapshot completely asynchronous.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)