Hi Chandan,

> 1. Why did we took 2 different approaches using different RocksDB apis ?
> We could have used Checkpoint api of RocksDB for fullSnapshot as well .

The reason here is partially historical. Full snapshot in RocksDB backend was 
implemented before incremental and rescaling for incremental snapshot but after 
heap backend. Full snapshot in RocksDB uses approach close to heap backend 
because Flink community plans to support the unified format for savepoints. The 
unified format would make it possible to switch backends and restore from 
savepoint. The formats still differ due to backend specifics to optimise 
snapshotting and restore but it is technically possible to unify them in future.

> 2. Is there any specific reason to use Snapshot API of rocksDB  over 
> Checkpoint api of RocksDB for fullSnapshot?

I think Checkpoint API produces separate SST file list to copy them to HDFS in 
case of incremental snapshot.

Full snapshot does not need the file list, it just needs an iterator over 
snapshotted (frozen) data. Internally RocksDB just hard-links immutable already 
existing SST files and iterates their data for Snapshot API.

Best,
Andrey


> On 24 Oct 2018, at 18:40, chandan prakash <chandanbaran...@gmail.com> wrote:
> 
> Thanks Tzu-Li for redirecting.
> Would also like to be corrected if my any inference from the code is 
> incorrect or incomplete.
> I am sure it will help to clear doubts of more developers like me  :)
> Thanks in advance.
> 
> Regards,
> Chandan
> 
> 
> On Wed, Oct 24, 2018 at 9:19 PM Tzu-Li (Gordon) Tai <tzuli...@apache.org 
> <mailto:tzuli...@apache.org>> wrote:
> Hi,
> 
> I’m forwarding this question to Stefan (cc’ed).
> He would most likely be able to answer your question, as he has done 
> substantial work in the RocksDB state backends.
> 
> Cheers,
> Gordon
> 
> 
> On 24 October 2018 at 8:47:24 PM, chandan prakash (chandanbaran...@gmail.com 
> <mailto:chandanbaran...@gmail.com>) wrote:
> 
>> Hi,
>> I am new to Flink.
>> Was looking into the code to understand how Flink does FullSnapshot and 
>> Incremental Snapshot using RocksDB
>> 
>> What I understood:
>> 1. For full snapshot, we call RocksDb snapshot api which basically an 
>> iterator handle to the entries in RocksDB instance. We iterate over every 
>> entry one by one and serialize that to some distributed file system. 
>> Similarly in restore for fullSnapshot, we read the file to get every entry 
>> and apply that to the rocksDb instance one by one to fully construct the db 
>> instance.
>> 
>> 2. On the other hand in for Incremental Snapshot, we rely on RocksDB 
>> Checkpoint api to copy the sst files to HDFS/S3 incrementally.
>> Similarly on restore, we copy the sst files to local directory and 
>> instantiate rocksDB instance with the path of the directory.
>> 
>> My Question is:
>> 1. Why did we took 2 different approaches using different RocksDB apis ?
>> We could have used Checkpoint api of RocksDB for fullSnapshot as well .
>> 2. Is there any specific reason to use Snapshot API of rocksDB  over 
>> Checkpoint api of RocksDB for fullSnapshot?
>> 
>> I am sure, I am missing some important point, really curious to know that.
>> Any explanation will be really great. Thanks in advance.
>> 
>> 
>> Regards,
>> Chandan
>> 
>> 
>> 
>> 
>> 
>> --
>> Chandan Prakash
>> 
> 
> 
> -- 
> Chandan Prakash
> 

Reply via email to