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_r258386352
 
 

 ##########
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
 ##########
 @@ -0,0 +1,507 @@
+/*
+ * 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.configuration.ConfigConstants;
+import 
org.apache.flink.contrib.streaming.state.RocksDBIncrementalCheckpointUtils;
+import org.apache.flink.contrib.streaming.state.RocksDBKeySerializationUtils;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOperationUtils;
+import org.apache.flink.contrib.streaming.state.RocksDBStateDownloader;
+import org.apache.flink.contrib.streaming.state.RocksDBWriteBatchWrapper;
+import org.apache.flink.contrib.streaming.state.RocksIteratorWrapper;
+import org.apache.flink.contrib.streaming.state.StateColumnFamilyHandle;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.BackendBuildingException;
+import org.apache.flink.runtime.state.DirectoryStateHandle;
+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.KeyedBackendSerializationProxy;
+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.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
+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 org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+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.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX;
+
+/**
+ * Encapsulates the process of restoring a RocksDB instance from an 
incremental snapshot.
+ */
+public class RocksDBIncrementalRestoreOperation<K> extends 
AbstractRocksDBRestoreOperation<K> {
+       private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBIncrementalRestoreOperation.class);
+
+       private final String operatorIdentifier;
+       protected final SortedMap<Long, Set<StateHandleID>> restoredSstFiles;
+       protected long lastCompletedCheckpointId;
+       protected UUID backendUID;
+       private boolean isKeySerializerCompatibilityChecked;
+
+       public RocksDBIncrementalRestoreOperation(
+               String operatorIdentifier,
+               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,
+               RocksDBNativeMetricOptions nativeMetricOptions,
+               MetricGroup metricGroup,
+               @Nonnull Collection<KeyedStateHandle> restoreStateHandles) {
+               super(keyGroupRange,
+                       keyGroupPrefixBytes,
+                       numberOfTransferringThreads,
+                       cancelStreamRegistry,
+                       userCodeClassLoader,
+                       kvStateInformation,
+                       keySerializerProvider,
+                       instanceBasePath,
+                       instanceRocksDBPath,
+                       dbOptions,
+                       columnOptions,
+                       nativeMetricOptions,
+                       metricGroup,
+                       restoreStateHandles);
+               this.operatorIdentifier = operatorIdentifier;
+               this.restoredSstFiles = new TreeMap<>();
+               this.lastCompletedCheckpointId = -1L;
+               this.backendUID = UUID.randomUUID();
+       }
+
+       /**
+        * Root method that branches for different implementations of {@link 
KeyedStateHandle}.
+        */
+       @Override
+       public RocksDBRestoreResult restore() throws Exception {
+
+               if (restoreStateHandles == null || 
restoreStateHandles.isEmpty()) {
+                       return null;
+               }
+
+               final KeyedStateHandle theFirstStateHandle = 
restoreStateHandles.iterator().next();
+
+               boolean isRescaling = (restoreStateHandles.size() > 1 ||
+                       !Objects.equals(theFirstStateHandle.getKeyGroupRange(), 
keyGroupRange));
+
+               if (isRescaling) {
+                       restoreWithRescaling(restoreStateHandles);
+               } else {
+                       restoreWithoutRescaling(theFirstStateHandle);
+               }
+               return new RocksDBRestoreResult(this.db, 
defaultColumnFamilyHandle,
+                       nativeMetricMonitor, lastCompletedCheckpointId, 
backendUID, restoredSstFiles);
+       }
+
+       /**
+        * Recovery from a single remote incremental state without rescaling.
+        */
+       void restoreWithoutRescaling(KeyedStateHandle rawStateHandle) throws 
Exception {
+               // 1. Prepare for restore without rescaling
+               Path temporaryRestoreInstancePath = new Path(
+                       instanceBasePath.getAbsolutePath(),
+                       UUID.randomUUID().toString()); // used as restore 
source for IncrementalKeyedStateHandle
+               RocksDBIncrementalRestorePrepareResult prepareResult = 
prepareFiles(rawStateHandle, temporaryRestoreInstancePath);
+               Path restoreSourcePath = 
prepareResult.getLocalKeyedStateHandle().getDirectoryStateHandle().getDirectory();
+               if (rawStateHandle instanceof IncrementalKeyedStateHandle) {
+                       backendUID = ((IncrementalKeyedStateHandle) 
rawStateHandle).getBackendIdentifier();
+               } else {
+                       backendUID = ((IncrementalLocalKeyedStateHandle) 
rawStateHandle).getBackendIdentifier();
+               }
+               LOG.debug("Restoring keyed backend uid in operator {} from 
incremental snapshot to {}.",
+                       this.operatorIdentifier, this.backendUID);
+               if (!instanceRocksDBPath.mkdirs()) {
+                       String errMsg = "Could not create RocksDB data 
directory: " + instanceBasePath.getAbsolutePath();
+                       LOG.error(errMsg);
+                       throw new IOException(errMsg);
+               }
+               try {
+                       restoreInstanceDirectoryFromPath(restoreSourcePath, 
dbPath);
+               } finally {
+                       FileSystem restoreFileSystem = 
restoreSourcePath.getFileSystem();
+                       if (restoreFileSystem.exists(restoreSourcePath)) {
+                               restoreFileSystem.delete(restoreSourcePath, 
true);
 
 Review comment:
   If we just to `try-finally`, an exception that happens during IO in the 
`finally` block can cause the loss of the original exception. There are more 
cases of this problem in the class. One way is that you could use 
`ExceptionUtils.firstOrSuppressed` to preserve original exceptions.

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