1996fanrui commented on code in PR #27035: URL: https://github.com/apache/flink/pull/27035#discussion_r2382282336
########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/state/rocksdb/restore/RestoredDBInstance.java: ########## @@ -0,0 +1,171 @@ +/* + * 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.state.rocksdb.restore; + +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.state.rocksdb.RocksDBOperationUtils; +import org.apache.flink.state.rocksdb.ttl.RocksDbTtlCompactFiltersManager; +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.ReadOptions; +import org.rocksdb.RocksDB; + +import javax.annotation.Nonnull; + +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +/** Utility classes for RocksDB restore operations. */ +public class RocksDBRestoreOperationUtils { + + /** + * Restores a RocksDB instance from local state for the given state handle. + * + * @param stateHandle the state handle to restore from + * @param columnFamilyOptionsFactory factory for creating column family options + * @param dbOptions database options + * @param ttlCompactFiltersManager TTL compact filters manager (can be null) + * @param writeBufferManagerCapacity write buffer manager capacity (can be null) + * @return restored DB instance with all necessary handles and metadata + * @throws Exception on any restore error + */ + public static RestoredDBInstance restoreTempDBInstanceFromLocalState( + IncrementalLocalKeyedStateHandle stateHandle, + List<StateMetaInfoSnapshot> stateMetaInfoSnapshots, + Function<String, ColumnFamilyOptions> columnFamilyOptionsFactory, + DBOptions dbOptions, + RocksDbTtlCompactFiltersManager ttlCompactFiltersManager, + Long writeBufferManagerCapacity) + throws Exception { + + List<ColumnFamilyDescriptor> columnFamilyDescriptors = + createColumnFamilyDescriptors( + stateMetaInfoSnapshots, + columnFamilyOptionsFactory, + ttlCompactFiltersManager, + writeBufferManagerCapacity, + true); + + Path restoreSourcePath = stateHandle.getDirectoryStateHandle().getDirectory(); + + List<ColumnFamilyHandle> columnFamilyHandles = + new ArrayList<>(stateMetaInfoSnapshots.size() + 1); + + for (ColumnFamilyDescriptor columnFamilyDescriptor : columnFamilyDescriptors) { + columnFamilyDescriptor.getOptions().setDisableAutoCompactions(true); Review Comment: I prefer to keep it as is: - `setDisableAutoCompactions` return this, so it is same. - If using returned ColumnFamilyOptions, we have to change several methods for some reasons: - `RestoredDBInstance#restoreTempDBInstanceFromLocalStateonly` only accesses `ColumnFamilyDescriptor`, and cannot set a new `ColumnFamilyOptions` for ColumnFamilyDescriptor. - We have to disable auto compaction in the code where ColumnFamilyOptions is created - It is needed to pass the `boolean disableAutoCompactions` to several methods WDYT? ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/state/rocksdb/restore/DistributeStateHandlerHelperTest.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.state.rocksdb.restore; + +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.DoubleSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.runtime.state.CompositeKeySerializationUtils; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.ExportImportFilesMetaData; +import org.rocksdb.FlushOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test class for {@link DistributeStateHandlerHelper}. */ +public class DistributeStateHandlerHelperTest extends TestLogger { + + private static final int NUM_KEY_GROUPS = 128; + private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, NUM_KEY_GROUPS - 1); + private static final int KEY_GROUP_PREFIX_BYTES = + CompositeKeySerializationUtils.computeRequiredBytesInKeyGroupPrefix(NUM_KEY_GROUPS); + private static final String CF_NAME = "test-column-family"; + + @TempDir private Path tempDir; + + /** Test whether sst files are exported when the key group all in range. */ + @Test + public void testAutoCompactionIsDisabled() throws Exception { + Path rocksDir = tempDir.resolve("rocksdb_dir"); + Path dbPath = rocksDir.resolve("db"); + Path chkDir = rocksDir.resolve("chk"); + Path exportDir = rocksDir.resolve("export"); + + Files.createDirectories(dbPath); + Files.createDirectories(exportDir); + + ArrayList<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>(2); + + try (RocksDB db = openDB(dbPath.toString(), columnFamilyHandles)) { + ColumnFamilyHandle testCfHandler = columnFamilyHandles.get(1); + + // Create SST files and verify their creation + for (int i = 0; i < 4; i++) { + db.flush(new FlushOptions().setWaitForFlush(true), testCfHandler); + for (int j = 10; j < NUM_KEY_GROUPS / 2; j++) { + byte[] bytes = new byte[KEY_GROUP_PREFIX_BYTES]; + CompositeKeySerializationUtils.serializeKeyGroup(j, bytes); + db.delete(testCfHandler, bytes); + } + assertThat( + dbPath.toFile() + .listFiles( + (file, name) -> + name.toLowerCase().endsWith(".sst"))) + .hasSize(i); + } + + // Create checkpoint + try (Checkpoint checkpoint = Checkpoint.create(db)) { + checkpoint.createCheckpoint(chkDir.toString()); + } + } + + // Verify there are 4 sst files in level 0, compaction will be triggered once the DB is + // opened. + assertThat(chkDir.toFile().listFiles((file, name) -> name.toLowerCase().endsWith(".sst"))) + .hasSize(4); + + // Create IncrementalLocalKeyedStateHandle for testing + IncrementalLocalKeyedStateHandle stateHandle = createTestStateHandle(chkDir.toString()); + + try (DistributeStateHandlerHelper helper = + createDistributeStateHandlerHelper( + stateHandle, (name) -> new ColumnFamilyOptions())) { + + // This simulates the delay that allows background compaction to clean up SST files if + // auto compaction is enabled. + Thread.sleep(500); + Map<RegisteredStateMetaInfoBase.Key, List<ExportImportFilesMetaData>> + exportedColumnFamiliesOut = new HashMap<>(); + List<IncrementalLocalKeyedStateHandle> skipped = new ArrayList<>(); + + KeyGroupRange result = helper.distribute(exportDir, exportedColumnFamiliesOut, skipped); + assertThat(result).isNotNull(); + assertThat(exportedColumnFamiliesOut).isNotEmpty(); + assertThat(skipped).isEmpty(); + } + } + + /** + * Creates a RocksDB instance with debug logging enabled. The instance is automatically tracked + * for cleanup in tearDown(). + */ + private RocksDB openDB(String path, ArrayList<ColumnFamilyHandle> columnFamilyHandles) + throws RocksDBException { + + List<ColumnFamilyDescriptor> columnFamilyDescriptors = new ArrayList<>(2); + columnFamilyDescriptors.add( + new ColumnFamilyDescriptor( + RocksDB.DEFAULT_COLUMN_FAMILY, new ColumnFamilyOptions())); + columnFamilyDescriptors.add( + new ColumnFamilyDescriptor( + CF_NAME.getBytes(ConfigConstants.DEFAULT_CHARSET), + new ColumnFamilyOptions())); + + return RocksDB.open( + new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true), + path, + columnFamilyDescriptors, + columnFamilyHandles); Review Comment: I expected it not to compact since the test only created 3 sst files and the 4th sst file was created by the checkpoint. But I agree with you, it is better to disable auto compaction explicitly, updated. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org