StefanRRichter commented on a change in pull request #7674: [FLINK-10043] [State Backends] Refactor RocksDBKeyedStateBackend object construction/initialization/restore code URL: https://github.com/apache/flink/pull/7674#discussion_r258060842
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/AbstractRocksDBRestoreOperation.java ########## @@ -0,0 +1,135 @@ +/* + * 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.restore; + +import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricMonitor; +import org.apache.flink.contrib.streaming.state.RocksDBOperationUtils; +import org.apache.flink.contrib.streaming.state.StateColumnFamilyHandle; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateSerializerProvider; +import org.apache.flink.util.IOUtils; + +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.RocksDB; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.UUID; + +/** + * Base implementation of RocksDB restore operation. + * + * @param <K> The data type that the serializer serializes. + */ +public abstract class AbstractRocksDBRestoreOperation<K> implements RocksDBRestoreOperation, AutoCloseable { + protected final KeyGroupRange keyGroupRange; + protected final int keyGroupPrefixBytes; + protected final int numberOfTransferringThreads; + protected final CloseableRegistry cancelStreamRegistry; + protected final ClassLoader userCodeClassLoader; + protected final ColumnFamilyOptions columnOptions; + protected final DBOptions dbOptions; + protected final Map<String, StateColumnFamilyHandle> kvStateInformation; + protected final RocksDBNativeMetricMonitor nativeMetricMonitor; + protected final File instanceBasePath; + protected final File instanceRocksDBPath; + protected final String dbPath; + protected List<ColumnFamilyHandle> columnFamilyHandles; + protected List<ColumnFamilyDescriptor> columnFamilyDescriptors; + protected final StateSerializerProvider<K> keySerializerProvider; + protected final SortedMap<Long, Set<StateHandleID>> restoredSstFiles; Review comment: Just wondering why sst-files belong to the abstract class, even if they are actually only relevant for the incremental snapshot case? Is this just picking unused variables over more differences between the classes and results from incremental and full cases? If you think that this is intended and want to keep it like this, you could actually also make the call to `new RocksDBRestoreResult(...)` part of the abstract class, because it is duplicated in all subclasses, passing in the same fields. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services