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

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

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

    https://github.com/apache/flink/pull/5582#discussion_r192052899
  
    --- 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 --
    
    Cool, no problem, I am also completing this review soon. One general thing 
that I was wondering about: did you ever see the sstable ingestion feature? It 
would be super nice for the rescaling of incremental checkpoints if we could 
simply ingest the sstables from multiple checkpoints into one database and the 
just clip the range boundaries. Unfortunately, from what I have seen this only 
works for external sstables written bei the sstable writer (see here: 
https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files). I 
wonder if there is any way to modify the sstables if incremental checkpoints to 
make them usable for ingestion, but maybe it is just completely impossible. I 
also found this interesting discussion that outlines another potential 
approach: https://github.com/facebook/rocksdb/issues/499. Any thoughts?


> 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