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_r258054045
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java ########## @@ -0,0 +1,580 @@ +/* + * 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.api.common.typeutils.TypeSerializerSchemaCompatibility; +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.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricMonitor; +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.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.RegisteredStateMetaInfoBase; +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.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.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.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; + 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, + RocksDBNativeMetricMonitor nativeMetricMonitor, + @Nonnull Collection<KeyedStateHandle> restoreStateHandles) { + super(keyGroupRange, + keyGroupPrefixBytes, + numberOfTransferringThreads, + cancelStreamRegistry, + userCodeClassLoader, + kvStateInformation, + keySerializerProvider, + instanceBasePath, + instanceRocksDBPath, + dbOptions, + columnOptions, + nativeMetricMonitor, + restoreStateHandles); + this.operatorIdentifier = operatorIdentifier; + } + + /** + * 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, lastCompletedCheckpointId, backendUID, + restoredSstFiles, defaultColumnFamilyHandle); + } + + /** + * 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, cancelStreamRegistry); + 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); + } + } + // 2. Open db instance + openDB(); + // 3. Use the restore sst files as the base for succeeding checkpoints + IncrementalLocalKeyedStateHandle restoreStateHandle = prepareResult.getLocalKeyedStateHandle(); + restoredSstFiles.put( + restoreStateHandle.getCheckpointId(), + restoreStateHandle.getSharedStateHandleIDs()); + lastCompletedCheckpointId = restoreStateHandle.getCheckpointId(); + // 4. Register column family handles + registerCFHandles(prepareResult.getStateMetaInfoSnapshots()); + } + + private void registerCFHandles(List<StateMetaInfoSnapshot> metaInfoSnapshots) + throws BackendBuildingException { + // Register CF handlers + for (int i = 0; i < metaInfoSnapshots.size(); ++i) { + try { + getOrRegisterColumnFamilyHandle( + kvStateInformation, + nativeMetricMonitor, + db, + columnFamilyDescriptors.get(i), + columnFamilyHandles.get(i), + metaInfoSnapshots.get(i)); + } catch (RocksDBException e) { + String errMsg = "Failed to register CF handle."; + LOG.error(errMsg, e); + throw new BackendBuildingException(errMsg, e); + } + } + } + + private RocksDBIncrementalRestorePrepareResult prepareFiles( + KeyedStateHandle stateHandle, + Path restoreInstancePath, + CloseableRegistry cancelStreamRegistry) throws IOException { + RocksDBIncrementalRestorePrepareResult prepareResult; + try { + prepareResult = prepareFiles( + stateHandle, + restoreInstancePath, + numberOfTransferringThreads, + cancelStreamRegistry, + userCodeClassLoader, + columnOptions + ); + } catch (Exception e) { + // clean up + FileSystem restoreFileSystem = restoreInstancePath.getFileSystem(); + if (restoreFileSystem.exists(restoreInstancePath)) { + restoreFileSystem.delete(restoreInstancePath, true); + } + // log and rethrow + String errMsg = "Failed to prepare files for restore from incremental state handle."; + LOG.error(errMsg, e); + throw new BackendBuildingException(errMsg, e); + } + return prepareResult; + } + + public RocksDBIncrementalRestorePrepareResult prepareFiles( + KeyedStateHandle rawStateHandle, + Path temporaryRestoreInstancePath, + int numberOfTransferingThreads, + CloseableRegistry cancelStreamRegistry, + ClassLoader userCodeClassLoader, + ColumnFamilyOptions columnFamilyOptions) throws Exception { + + IncrementalLocalKeyedStateHandle localKeyedStateHandle; + List<StateMetaInfoSnapshot> stateMetaInfoSnapshots; + KeyedBackendSerializationProxy<K> serializationProxy; + if (rawStateHandle instanceof IncrementalKeyedStateHandle) { + IncrementalKeyedStateHandle restoreStateHandle = (IncrementalKeyedStateHandle) rawStateHandle; + + // read state data. + try (RocksDBStateDownloader rocksDBStateDownloader = + new RocksDBStateDownloader(numberOfTransferingThreads)) { + rocksDBStateDownloader.transferAllStateDataToDirectory( + restoreStateHandle, + temporaryRestoreInstancePath, + cancelStreamRegistry); + } + + serializationProxy = readMetaData(restoreStateHandle.getMetaStateHandle(), cancelStreamRegistry, userCodeClassLoader); + stateMetaInfoSnapshots = serializationProxy.getStateMetaInfoSnapshots(); + columnFamilyDescriptors = createAndRegisterColumnFamilyDescriptors(stateMetaInfoSnapshots, columnFamilyOptions); + + // since we transferred all remote state to a local directory, we can use the same code as for + // local recovery. + localKeyedStateHandle = new IncrementalLocalKeyedStateHandle( + restoreStateHandle.getBackendIdentifier(), + restoreStateHandle.getCheckpointId(), + new DirectoryStateHandle(temporaryRestoreInstancePath), + restoreStateHandle.getKeyGroupRange(), + restoreStateHandle.getMetaStateHandle(), + restoreStateHandle.getSharedState().keySet()); + } else if (rawStateHandle instanceof IncrementalLocalKeyedStateHandle) { + // Recovery from local incremental state. + localKeyedStateHandle = (IncrementalLocalKeyedStateHandle) rawStateHandle; + serializationProxy = readMetaData(localKeyedStateHandle.getMetaDataState(), cancelStreamRegistry, userCodeClassLoader); + stateMetaInfoSnapshots = serializationProxy.getStateMetaInfoSnapshots(); + columnFamilyDescriptors = createAndRegisterColumnFamilyDescriptors(stateMetaInfoSnapshots, columnFamilyOptions); + } else { + throw new IllegalStateException("Unexpected state handle type, " + + "expected " + IncrementalKeyedStateHandle.class + " or " + IncrementalLocalKeyedStateHandle.class + + ", but found " + rawStateHandle.getClass()); + } + columnFamilyHandles = new ArrayList<>(columnFamilyDescriptors.size() + 1); + return new RocksDBIncrementalRestorePrepareResult(stateMetaInfoSnapshots, localKeyedStateHandle); + } + + /** + * Recovery from multi incremental states with rescaling. For rescaling, this method creates a temporary + * RocksDB instance for a key-groups shard. All contents from the temporary instance are copied into the + * real restore instance and then the temporary instance is discarded. + */ + void restoreWithRescaling(Collection<KeyedStateHandle> restoreStateHandles) throws Exception { + // Init DB instance + initDBWithRescaling(restoreStateHandles); + // Do restore + byte[] startKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; + RocksDBKeySerializationUtils.serializeKeyGroup(keyGroupRange.getStartKeyGroup(), startKeyGroupPrefixBytes); + + byte[] stopKeyGroupPrefixBytes = new byte[keyGroupPrefixBytes]; + RocksDBKeySerializationUtils.serializeKeyGroup(keyGroupRange.getEndKeyGroup() + 1, stopKeyGroupPrefixBytes); + + for (KeyedStateHandle rawStateHandle : restoreStateHandles) { + + if (!(rawStateHandle instanceof IncrementalKeyedStateHandle)) { + throw new IllegalStateException("Unexpected state handle type, " + + "expected " + IncrementalKeyedStateHandle.class + + ", but found " + rawStateHandle.getClass()); + } + + Path temporaryRestoreInstancePath = new Path(instanceBasePath.getAbsolutePath() + UUID.randomUUID().toString()); + try (RestoredDBInstance tmpRestoreDBInfo = restoreDBInstanceFromStateHandle( + (IncrementalKeyedStateHandle) rawStateHandle, + temporaryRestoreInstancePath); + RocksDBWriteBatchWrapper writeBatchWrapper = new RocksDBWriteBatchWrapper(this.db)) { + + List<ColumnFamilyDescriptor> tmpColumnFamilyDescriptors = tmpRestoreDBInfo.columnFamilyDescriptors; + List<ColumnFamilyHandle> tmpColumnFamilyHandles = tmpRestoreDBInfo.columnFamilyHandles; + + // iterating only the requested descriptors automatically skips the default column family handle + for (int i = 0; i < tmpColumnFamilyDescriptors.size(); ++i) { + ColumnFamilyHandle tmpColumnFamilyHandle = tmpColumnFamilyHandles.get(i); + ColumnFamilyDescriptor tmpColumnFamilyDescriptor = tmpColumnFamilyDescriptors.get(i); + + ColumnFamilyHandle targetColumnFamilyHandle = getOrRegisterColumnFamilyHandle( + kvStateInformation, nativeMetricMonitor, this.db, + tmpColumnFamilyDescriptor, null, tmpRestoreDBInfo.stateMetaInfoSnapshots.get(i)); + + try (RocksIteratorWrapper iterator = RocksDBKeyedStateBackend.getRocksIterator(tmpRestoreDBInfo.db, tmpColumnFamilyHandle)) { + + iterator.seek(startKeyGroupPrefixBytes); + + while (iterator.isValid()) { + + if (RocksDBIncrementalCheckpointUtils.beforeThePrefixBytes(iterator.key(), stopKeyGroupPrefixBytes)) { + writeBatchWrapper.put(targetColumnFamilyHandle, iterator.key(), iterator.value()); + } else { + // Since the iterator will visit the record according to the sorted order, + // we can just break here. + break; + } + + iterator.next(); + } + } // releases native iterator resources + } + } finally { + FileSystem restoreFileSystem = temporaryRestoreInstancePath.getFileSystem(); + if (restoreFileSystem.exists(temporaryRestoreInstancePath)) { + restoreFileSystem.delete(temporaryRestoreInstancePath, true); + } + } + } + } + + private void initDBWithRescaling(Collection<KeyedStateHandle> restoreStateHandles) throws IOException { + // Prepare for restore with rescaling + IncrementalKeyedStateHandle initialHandle = + (IncrementalKeyedStateHandle) RocksDBIncrementalCheckpointUtils.chooseTheBestStateHandleForInitial( + restoreStateHandles, keyGroupRange); + Path instancePath = new Path(dbPath); + if (initialHandle != null) { + restoreStateHandles.remove(initialHandle); + // 1. Download target files + RocksDBIncrementalRestorePrepareResult prepareResult = + prepareFiles(initialHandle, instancePath, cancelStreamRegistry); + // 2. Open db instance + openDB(); + // 3. Clip the DB instance + try { + RocksDBIncrementalCheckpointUtils.clipDBWithKeyGroupRange( + db, + columnFamilyHandles, + keyGroupRange, + initialHandle.getKeyGroupRange(), + keyGroupPrefixBytes); + } catch (RocksDBException e) { + String errMsg = "Failed to clip DB after initialization."; + LOG.error(errMsg, e); + throw new BackendBuildingException(errMsg, e); + } + // 4. Register column family handles + registerCFHandles(prepareResult.getStateMetaInfoSnapshots()); + } else { + openDB(); + } + } + + /** + * Entity to hold the temporary RocksDB instance created for restore. + */ + private static class RestoredDBInstance implements AutoCloseable { + + @Nonnull + private final RocksDB db; + + @Nonnull + private final ColumnFamilyHandle defaultColumnFamilyHandle; + + @Nonnull + private final List<ColumnFamilyHandle> columnFamilyHandles; + + @Nonnull + private final List<ColumnFamilyDescriptor> columnFamilyDescriptors; + + @Nonnull + private final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots; + + private RestoredDBInstance( + @Nonnull RocksDB db, + @Nonnull List<ColumnFamilyHandle> columnFamilyHandles, + @Nonnull List<ColumnFamilyDescriptor> columnFamilyDescriptors, + @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { + this.db = db; + this.defaultColumnFamilyHandle = columnFamilyHandles.remove(0); + this.columnFamilyHandles = columnFamilyHandles; + this.columnFamilyDescriptors = columnFamilyDescriptors; + this.stateMetaInfoSnapshots = stateMetaInfoSnapshots; + } + + @Override + public void close() { + + IOUtils.closeQuietly(defaultColumnFamilyHandle); + + for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) { + IOUtils.closeQuietly(columnFamilyHandle); Review comment: Could use `closeAllQuietly` instead of loop ---------------------------------------------------------------- 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