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

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

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

    https://github.com/apache/flink/pull/5582#discussion_r192065096
  
    --- Diff: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeySerializationUtils.java
 ---
    @@ -138,4 +138,12 @@ private static void writeVariableIntBytes(
                        value >>>= 8;
                } while (value != 0);
        }
    +
    +   public static byte[] serializeKeyGroup(int keyGroup, int 
keyGroupPrefixBytes) {
    +           byte[] startKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes];
    --- End diff --
    
    Yes, I did notice the "sstable ingestion feature", and also did some 
experiment on it. You are right that currently the ingestion feature only works 
for the sstables written by the sstable writer. I tried to use the sstable 
writer to generate external sstables in parallel and ingest the sstables into 
the target db, but unfortunately the performance of the sstable writer is quite 
poor in RocksJava...I left the experiment conclusion in 
[FLINK-8845](https://issues.apache.org/jira/browse/FLINK-8845)(that is the 
reason why I took a step back to use the `WriteBatch` to speed up the recovery 
for full checkpoint), I pasted the comments below:
    
    **Unfortunately, even though according to RocksDB wiki, the best way to 
load data into RocksDB is "Generate SST files (using SstFileWriter) with 
non-overlapping ranges in parallel and bulk load the SST files.". But after 
implementing this and test with a simple bench mark, I found that the 
performance is not that good as expected, it's almost the same or worst that as 
using Rocks.put(). After a bit analysis I found that when building SST it 
consumed a lot of time to create DirectSlice and currently we can't reuse the 
DirectSlice in java api. Even though in C++ this could help to get a 
outperformance result, but in java I think we can't use this to improve the 
performance currently (maybe somedays RocksDB might improve this to enable us 
get a approximate performance in java as using C++) ...**
    
    And regarding to https://github.com/facebook/rocksdb/issues/499, If I'm not 
misunderstand, I think we might also can't use the `repairDB()` because we have 
many column families, and the other opinions in that thread is quite similar 
with the approach that I've tried to build the sstables in parallel and it 
turned out that it didn't work properly with Java API.


> Improve performance for recovery from incremental checkpoint
> ------------------------------------------------------------
>
>                 Key: FLINK-8790
>                 URL: https://issues.apache.org/jira/browse/FLINK-8790
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.5.0
>            Reporter: Sihua Zhou
>            Assignee: Sihua Zhou
>            Priority: Major
>             Fix For: 1.6.0
>
>
> When there are multi state handle to be restored, we can improve the 
> performance as follow:
> 1. Choose the best state handle to init the target db
> 2. Use the other state handles to create temp db, and clip the db according 
> to the target key group range (via rocksdb.deleteRange()), this can help use 
> get rid of the `key group check` in 
>  `data insertion loop` and also help us get rid of traversing the useless 
> record.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to