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_r258056445
 
 

 ##########
 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;
+       protected final Collection<KeyedStateHandle> restoreStateHandles;
+
+       protected RocksDB db;
+       protected long lastCompletedCheckpointId = -1L;
+       protected UUID backendUID = UUID.randomUUID();
+       protected ColumnFamilyHandle defaultColumnFamilyHandle;
+
+       protected AbstractRocksDBRestoreOperation(
+               KeyGroupRange keyGroupRange,
+               int keyGroupPrefixBytes,
+               int numberOfTransferringThreads,
+               CloseableRegistry cancelStreamRegistry,
+               ClassLoader userCodeClassLoader,
+               Map<String, StateColumnFamilyHandle> kvStateInformation,
+               StateSerializerProvider<K> keySerializerProvider,
+               File instanceBasePath,
+               File instanceRocksDBPath,
+               DBOptions dbOptions,
+               ColumnFamilyOptions columnOptions,
+               RocksDBNativeMetricMonitor nativeMetricMonitor,
+               @Nonnull Collection<KeyedStateHandle> stateHandles) {
+               this.keyGroupRange = keyGroupRange;
+               this.keyGroupPrefixBytes = keyGroupPrefixBytes;
+               this.numberOfTransferringThreads = numberOfTransferringThreads;
+               this.cancelStreamRegistry = cancelStreamRegistry;
+               this.userCodeClassLoader = userCodeClassLoader;
+               this.kvStateInformation = kvStateInformation;
+               this.keySerializerProvider = keySerializerProvider;
+               this.instanceBasePath = instanceBasePath;
+               this.instanceRocksDBPath = instanceRocksDBPath;
+               this.dbPath = instanceRocksDBPath.getAbsolutePath();
+               this.dbOptions = dbOptions;
+               this.columnOptions = columnOptions;
+               this.nativeMetricMonitor = nativeMetricMonitor;
+               this.restoreStateHandles = stateHandles;
+               this.restoredSstFiles = new TreeMap<>();
+               this.columnFamilyHandles = new ArrayList<>(1);
+               this.columnFamilyDescriptors = Collections.emptyList();
+       }
+
+       public void openDB() throws IOException {
+               db = RocksDBOperationUtils.openDB(dbPath, 
columnFamilyDescriptors,
+                       columnFamilyHandles, columnOptions, dbOptions);
+               // remove the default column family which is located at the 
first index
+               defaultColumnFamilyHandle = columnFamilyHandles.remove(0);
+               if (nativeMetricMonitor != null) {
+                       nativeMetricMonitor.setRocksDB(db);
 
 Review comment:
   I think I see why this was done, because you need the metric when the KV 
states are registered. But can't we just create the metric in this class if 
required, after the DB was opened? then the builder can get the metrics with a 
getter before passing it into the `RocksDBKeyedStateBackend`.

----------------------------------------------------------------
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

Reply via email to