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_r255944251
 
 

 ##########
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
 ##########
 @@ -0,0 +1,490 @@
+/*
+ * 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.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import 
org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation;
+import 
org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestorePrepareResult;
+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.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.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.metainfo.StateMetaInfoSnapshot;
+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.apache.flink.util.StateMigrationException;
+
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Objects;
+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;
+
+       /** The write options to use in the states. We disable write ahead 
logging. */
+       private final WriteOptions writeOptions;
+
+       /**
+        * Information about the k/v states, maintained in the order as we 
create them. This is used to retrieve the
+        * column family that is used for a state and also for sanity checks 
when restoring.
+        */
+       private final LinkedHashMap<String, StateColumnFamilyHandle> 
kvStateInformation;
+
+       /**
+        * Number of bytes required to prefix the key groups.
+        */
+       private final int keyGroupPrefixBytes;
+
+       private final MetricGroup metricGroup;
+       private final CloseableRegistry cancelStreamRegistry;
+
+       /** True if incremental checkpointing is enabled. */
+       private boolean enableIncrementalCheckpointing = false;
+       private RocksDBNativeMetricOptions nativeMetricOptions = new 
RocksDBNativeMetricOptions();
+       private int numberOfTransferingThreads = 
RocksDBOptions.CHECKPOINT_TRANSFER_THREAD_NUM.defaultValue();
+
+       // The DB instance to initiate and parameters for initialization
+       private RocksDB db;
+       private List<ColumnFamilyHandle> columnFamilyHandles = new 
ArrayList<>(1);
+       private List<ColumnFamilyDescriptor> columnFamilyDescriptors = 
Collections.emptyList();
+       private final String dbPath;
+       private final ResourceGuard rocksDBResourceGuard;
+       private RocksDBSnapshotStrategyBase<K> checkpointSnapshotStrategy;
+       private RocksDBSnapshotStrategyBase<K> savepointSnapshotStrategy;
+
+       // Variables for snapshot strategy when incremental checkpoint is 
enabled
+       private UUID backendUID = UUID.randomUUID();
+       private SortedMap<Long, Set<StateHandleID>> materializedSstFiles = new 
TreeMap<>();
+       private long lastCompletedCheckpointId = -1L;
+
+       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) {
+
+               super(kvStateRegistry, keySerializer, userCodeClassLoader,
+                       numberOfKeyGroups, keyGroupRange, executionConfig, 
ttlTimeProvider);
+
+               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.kvStateInformation = new LinkedHashMap<>();
+               this.keyGroupPrefixBytes =
+                       
RocksDBKeySerializationUtils.computeRequiredBytesInKeyGroupPrefix(numberOfKeyGroups);
+               this.writeOptions = new WriteOptions().setDisableWAL(true);
 
 Review comment:
   This is a native object that must always be closed after use. I think that 
is not guaranteed here, for example what happens if there is an Exception in 
the beginning of `build()`? In general, I would suggest to avoid fields in this 
class, except for arguments that are configured on the builder. Keep objects 
that are only used for building the backend as local variables inside the 
`build()` method and ensure that either i) backend building is successful or 
ii) all native resources are closed/deleted. Even if native RocksDB objects are 
guarding against double-close, for cleanliness I suggest to still avoid 
double-closing. This means that tracking object ownership is important. For the 
example of `WriteOptions`, first the build method would create and own the 
object, i.e. it is responsible to call `close()` in exceptional cases. This is 
true until you successfully create `new RocksDBKeyedStateBackend` and transfer 
ownership from the local variable in the `build()` method to the field inside 
`RocksDBKeyedStateBackend`. From this point, the `WriteOptions` should no 
longer be closed by `build()` but by `RocksDBKeyedStateBackend::dispose()`. You 
could do something like nulling out the local variable after the ownership was 
transferred. This principle should be applied to everything create in build and 
passed into the backend. I have seen that the method respects the ownership 
transfer for many objects but maybe not all, yet.

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