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

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_r191444464
  
    --- 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 --
    
    Maybe we can rather pass the `startKeyGroupPrefixBytes` array directly 
instead of creating it in every invocation from `keyGroupPrefixBytes`. Like 
that, the caller can reuse the same array.


> 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