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

 ##########
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
 ##########
 @@ -0,0 +1,447 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import 
org.apache.flink.contrib.streaming.state.restore.AbstractRocksDBRestoreOperation;
+import 
org.apache.flink.contrib.streaming.state.restore.RocksDBFullRestoreOperation;
+import 
org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation;
+import 
org.apache.flink.contrib.streaming.state.restore.RocksDBNoneRestoreOperation;
+import org.apache.flink.contrib.streaming.state.restore.RocksDBRestoreResult;
+import 
org.apache.flink.contrib.streaming.state.snapshot.RocksDBSnapshotStrategyBase;
+import 
org.apache.flink.contrib.streaming.state.snapshot.RocksFullSnapshotStrategy;
+import 
org.apache.flink.contrib.streaming.state.snapshot.RocksIncrementalSnapshotStrategy;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackendBuilder;
+import org.apache.flink.runtime.state.BackendBuildingException;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
+import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.LocalRecoveryConfig;
+import org.apache.flink.runtime.state.PriorityQueueSetFactory;
+import org.apache.flink.runtime.state.StateHandleID;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME;
+
+/**
+ * Builder class for {@link RocksDBKeyedStateBackend} which handles all 
necessary initializations and clean ups.
+ *
+ * @param <K> The data type that the key serializer serializes.
+ */
+public class RocksDBKeyedStateBackendBuilder<K> extends 
AbstractKeyedStateBackendBuilder<K> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBKeyedStateBackendBuilder.class);
+       public static final String DB_INSTANCE_DIR_STRING = "db";
+
+       /** String that identifies the operator that owns this backend. */
+       private final String operatorIdentifier;
+       private final RocksDBStateBackend.PriorityQueueStateType 
priorityQueueStateType;
+       /** The configuration of local recovery. */
+       private final LocalRecoveryConfig localRecoveryConfig;
+
+       /** The column family options from the options factory. */
+       private final ColumnFamilyOptions columnFamilyOptions;
+
+       /** The DB options from the options factory. */
+       private final DBOptions dbOptions;
+
+       /** Path where this configured instance stores its data directory. */
+       private final File instanceBasePath;
+
+       /** Path where this configured instance stores its RocksDB database. */
+       private final File instanceRocksDBPath;
+
+       private final MetricGroup metricGroup;
+
+       /** True if incremental checkpointing is enabled. */
+       private boolean enableIncrementalCheckpointing = false;
+       private RocksDBNativeMetricOptions nativeMetricOptions = new 
RocksDBNativeMetricOptions();
+       private int numberOfTransferingThreads = 
RocksDBOptions.CHECKPOINT_TRANSFER_THREAD_NUM.defaultValue();
+
+       private RocksDB injectedTestDB; // for testing
+
+       public RocksDBKeyedStateBackendBuilder(
+               String operatorIdentifier,
+               ClassLoader userCodeClassLoader,
+               File instanceBasePath,
+               DBOptions dbOptions,
+               ColumnFamilyOptions columnFamilyOptions,
+               TaskKvStateRegistry kvStateRegistry,
+               TypeSerializer<K> keySerializer,
+               int numberOfKeyGroups,
+               KeyGroupRange keyGroupRange,
+               ExecutionConfig executionConfig,
+               LocalRecoveryConfig localRecoveryConfig,
+               RocksDBStateBackend.PriorityQueueStateType 
priorityQueueStateType,
+               TtlTimeProvider ttlTimeProvider,
+               MetricGroup metricGroup,
+               Collection<KeyedStateHandle> stateHandles) {
+
+               super(kvStateRegistry, keySerializer, userCodeClassLoader,
+                       numberOfKeyGroups, keyGroupRange, executionConfig, 
ttlTimeProvider, stateHandles);
+
+               this.operatorIdentifier = operatorIdentifier;
+               this.priorityQueueStateType = priorityQueueStateType;
+               this.localRecoveryConfig = localRecoveryConfig;
+               // ensure that we use the right merge operator, because other 
code relies on this
+               this.columnFamilyOptions = 
Preconditions.checkNotNull(columnFamilyOptions)
+                       .setMergeOperatorName(MERGE_OPERATOR_NAME);
+               this.dbOptions = dbOptions;
+               this.instanceBasePath = instanceBasePath;
+               this.instanceRocksDBPath = new File(instanceBasePath, 
DB_INSTANCE_DIR_STRING);
+               this.metricGroup = metricGroup;
+       }
+
+       @VisibleForTesting
+       RocksDBKeyedStateBackendBuilder(
+               String operatorIdentifier,
+               ClassLoader userCodeClassLoader,
+               File instanceBasePath,
+               DBOptions dbOptions,
+               ColumnFamilyOptions columnFamilyOptions,
+               TaskKvStateRegistry kvStateRegistry,
+               TypeSerializer<K> keySerializer,
+               int numberOfKeyGroups,
+               KeyGroupRange keyGroupRange,
+               ExecutionConfig executionConfig,
+               LocalRecoveryConfig localRecoveryConfig,
+               RocksDBStateBackend.PriorityQueueStateType 
priorityQueueStateType,
+               TtlTimeProvider ttlTimeProvider,
+               MetricGroup metricGroup,
+               Collection<KeyedStateHandle> stateHandles,
+               RocksDB injectedTestDB) {
+               this(operatorIdentifier, userCodeClassLoader, instanceBasePath, 
dbOptions,
+                       columnFamilyOptions, kvStateRegistry, keySerializer, 
numberOfKeyGroups, keyGroupRange,
+                       executionConfig, localRecoveryConfig, 
priorityQueueStateType, ttlTimeProvider, metricGroup, stateHandles);
+               this.injectedTestDB = injectedTestDB;
+       }
+
+       public RocksDBKeyedStateBackendBuilder<K> 
setEnableIncrementalCheckpointing(boolean enableIncrementalCheckpointing) {
+               this.enableIncrementalCheckpointing = 
enableIncrementalCheckpointing;
+               return this;
+       }
+
+       public RocksDBKeyedStateBackendBuilder<K> 
setNativeMetricOptions(RocksDBNativeMetricOptions nativeMetricOptions) {
+               this.nativeMetricOptions = nativeMetricOptions;
+               return this;
+       }
+
+       public RocksDBKeyedStateBackendBuilder<K> 
setNumberOfTransferingThreads(int numberOfTransferingThreads) {
+               this.numberOfTransferingThreads = numberOfTransferingThreads;
+               return this;
+       }
+
+       private static void checkAndCreateDirectory(File directory) throws 
IOException {
+               if (directory.exists()) {
+                       if (!directory.isDirectory()) {
+                               throw new IOException("Not a directory: " + 
directory);
+                       }
+               } else if (!directory.mkdirs()) {
+                       throw new IOException(String.format("Could not create 
RocksDB data directory at %s.", directory));
+               }
+       }
+
+       public RocksDBKeyedStateBackend<K> build() throws 
BackendBuildingException {
+               RocksDBKeyedStateBackend<K> backend = null;
+               RocksDBWriteBatchWrapper writeBatchWrapper = null;
+               ColumnFamilyHandle defaultColumnFamilyHandle = null;
+               RocksDBNativeMetricMonitor nativeMetricMonitor = 
nativeMetricOptions.isEnabled() ?
+                       new RocksDBNativeMetricMonitor(nativeMetricOptions, 
metricGroup) : null;
+               CloseableRegistry cancelStreamRegistry = new 
CloseableRegistry();
+               //The write options to use in the states. We disable write 
ahead logging.
+               WriteOptions writeOptions = new 
WriteOptions().setDisableWAL(true);
+               LinkedHashMap<String, StateColumnFamilyHandle> 
kvStateInformation = new LinkedHashMap<>();
 
 Review comment:
   Will move the backend construction and `return` statement out of the `try` 
clause, so some of the variables will stay out of `try`, and others moved 
closer to their actual usage.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to