Thanks Aljoscha. Yes - that is exactly what I am looking for.

On Thu, Jun 30, 2016 at 5:07 AM, Aljoscha Krettek <aljos...@apache.org>
wrote:

> Hi,
> are you taking about *enableFullyAsyncSnapshots()* in the RocksDB
> backend. If not, there is this switch that is described in the JavaDoc:
>
> /**
> * Enables fully asynchronous snapshotting of the partitioned state held in
> RocksDB.
> *
> * <p>By default, this is disabled. This means that RocksDB state is copied
> in a synchronous
> * step, during which normal processing of elements pauses, followed by an
> asynchronous step
> * of copying the RocksDB backup to the final checkpoint location. Fully
> asynchronous
> * snapshots take longer (linear time requirement with respect to number of
> unique keys)
> * but normal processing of elements is not paused.
> */
> public void enableFullyAsyncSnapshots()
>
> This also describes the implications on checkpointing time but please let
> me know if I should provide more details. We should probably also add more
> description to the documentation for this.
>
> Cheers,
> Aljoscha
>
> On Wed, 29 Jun 2016 at 23:04 Daniel Li <daniell...@gmail.com> wrote:
>
>> When RocksDB holds a very large state, is there a concern over the time
>> takes in checkpointing the RocksDB data to HDFS? Is asynchronous
>> checkpointing a recommended practice here?
>>
>>
>>
>> https://ci.apache.org/projects/flink/flink-docs-master/apis/streaming/state_backends.html
>>
>> "The RocksDBStateBackend holds in-flight data in a RocksDB
>> <http://rocksdb.org/> data base that is (per default) stored in the
>> TaskManager data directories. Upon checkpointing, the whole RocksDB data
>> base will be checkpointed into the configured file system and directory.
>> Minimal metadata is stored in the JobManager’s memory (or, in
>> high-availability mode, in the metadata checkpoint).
>>
>> The RocksDBStateBackend is encouraged for:
>>
>>    - Jobs with very large state, long windows, large key/value states.
>>    - All high-availability setups."
>>
>>
>> thx
>> Daniel
>>
>

Reply via email to