[ https://issues.apache.org/jira/browse/FLINK-8790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16496411#comment-16496411 ]
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_r192072170 --- Diff: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java --- @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.contrib.streaming.state; + +import org.apache.flink.runtime.state.KeyGroupRange; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; + +import java.util.List; + +/** + * Utils for RocksDB Incremental Checkpoint. + */ +public class RocksDBIncrementalCheckpointUtils { + + public static void clipDBWithKeyGroupRange( + RocksDB db, + List<ColumnFamilyHandle> columnFamilyHandles, + KeyGroupRange targetGroupRange, + KeyGroupRange currentGroupRange, + int keyGroupPrefixBytes) throws RocksDBException { + + for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) { + if (currentGroupRange.getStartKeyGroup() < targetGroupRange.getStartKeyGroup()) { + byte[] beginKey = RocksDBKeySerializationUtils.serializeKeyGroup( + currentGroupRange.getStartKeyGroup(), keyGroupPrefixBytes); + byte[] endKye = RocksDBKeySerializationUtils.serializeKeyGroup( + targetGroupRange.getStartKeyGroup(), keyGroupPrefixBytes); + db.deleteRange(columnFamilyHandle, beginKey, endKye); + } + + if (currentGroupRange.getEndKeyGroup() > targetGroupRange.getEndKeyGroup()) { + byte[] beginKey = RocksDBKeySerializationUtils.serializeKeyGroup( + targetGroupRange.getEndKeyGroup() + 1, keyGroupPrefixBytes); + + byte[] endKey = new byte[keyGroupPrefixBytes]; + for (int i = 0; i < keyGroupPrefixBytes; ++i) { + endKey[i] = (byte) (0xFF); + } + db.deleteRange(columnFamilyHandle, beginKey, endKey); + } + } + } + + public static int evaluateGroupRange(KeyGroupRange range1, KeyGroupRange range2) { --- End diff -- I think this code could also be based on something like `KeyGroupRange.getIntersection(KeyGroupRange).getNumberOfKeyGroups()`. > 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)