1996fanrui commented on code in PR #23765: URL: https://github.com/apache/flink/pull/23765#discussion_r1404119092
########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { + + @TempDir java.nio.file.Path folder; + + @Test + public void rocksdbStateFileVerifierTest() throws Exception { + List columnFamilyHandles = new ArrayList<>(1); Review Comment: ```suggestion List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>(1); ``` ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { + + @TempDir java.nio.file.Path folder; + + @Test + public void rocksdbStateFileVerifierTest() throws Exception { Review Comment: ```suggestion void rocksdbStateFileVerifierTest() throws Exception { ``` public isn't needed for junit5. ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java: ########## @@ -96,6 +105,46 @@ void testCheckpointIsIncremental() throws Exception { } } + @Test + void testCheckpointFailIfSstFileCorrupted() throws Exception { + + try (CloseableRegistry closeableRegistry = new CloseableRegistry(); + RocksIncrementalSnapshotStrategy checkpointSnapshotStrategy = + createSnapshotStrategy(closeableRegistry)) { + FsCheckpointStreamFactory checkpointStreamFactory = createFsCheckpointStreamFactory(); + + RocksIncrementalSnapshotStrategy.NativeRocksDBSnapshotResources snapshotResources = + checkpointSnapshotStrategy.syncPrepareResources(1L); + + // Corrupt Sst Files + List<Path> sstFiles = + Arrays.stream( + FileUtils.listDirectory( + snapshotResources.snapshotDirectory.getDirectory())) + .filter(file -> file.getFileName().toString().endsWith(SST_FILE_SUFFIX)) + .collect(Collectors.toList()); + + RocksdbStateFileVerifierTest.corruptSstFile(sstFiles.get(0), sstFiles.get(0)); + + try { + checkpointSnapshotStrategy + .asyncSnapshot( + snapshotResources, + 1L, + 1L, + checkpointStreamFactory, + CheckpointOptions.forCheckpointWithDefaultLocation()) + .get(closeableRegistry) + .getJobManagerOwnedSnapshot(); + + fail("verifySstFilesChecksum should failed"); + } catch (Exception e) { + Assertions.assertTrue( + e.getMessage().contains("Error while verifying Checksum of Sst File")); + } Review Comment: ```suggestion assertThatThrownBy(() -> checkpointSnapshotStrategy .asyncSnapshot( snapshotResources, 1L, 1L, checkpointStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()) .get(closeableRegistry) .getJobManagerOwnedSnapshot()) .hasMessageContaining("Error while verifying Checksum of Sst File"); ``` ########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java: ########## @@ -350,6 +359,10 @@ private long uploadSnapshotFiles( ? CheckpointedStateScope.EXCLUSIVE : CheckpointedStateScope.SHARED; + if (stateFileVerifier != null) { + stateFileVerifier.verifySstFilesChecksum(sstFilePaths); + } + List<HandleAndLocalPath> sstFilesUploadResult = stateUploader.uploadFilesToCheckpointFs( Review Comment: From the code order, we verify the file first, and then `uploadFilesToCheckpointFs`. If the file is corrupted before calling `hdfs.write()`, this check still cannot cover, right? ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { + + @TempDir java.nio.file.Path folder; + + @Test + public void rocksdbStateFileVerifierTest() throws Exception { + List columnFamilyHandles = new ArrayList<>(1); + String rootPath = folder.toAbsolutePath().toString(); + File dbPath = new File(rootPath, "db"); + File cpPath = new File(rootPath, "cp"); + + try (DBOptions dbOptions = new DBOptions().setCreateIfMissing(true); + ColumnFamilyOptions colOptions = new ColumnFamilyOptions(); + Options sstFileReaderOptions = new Options(dbOptions, colOptions); + WriteOptions writeOptions = new WriteOptions().setDisableWAL(true); + RocksDB db = + RocksDB.open( + dbOptions, + dbPath.toString(), + Collections.singletonList( + new ColumnFamilyDescriptor( + "default".getBytes(), colOptions)), + columnFamilyHandles); + RocksDBStateFileVerifier rocksDBStateFileVerifier = + new RocksDBStateFileVerifier(sstFileReaderOptions)) { + + byte[] key = "checkpoint".getBytes(); + byte[] val = "incrementalTest".getBytes(); + db.put(writeOptions, key, val); + + try (Checkpoint checkpoint = Checkpoint.create(db)) { + checkpoint.createCheckpoint(cpPath.toString()); + } + + List<Path> sstFiles = + Arrays.stream(FileUtils.listDirectory(cpPath.toPath())) + .filter(file -> file.getFileName().toString().endsWith(SST_FILE_SUFFIX)) + .collect(Collectors.toList()); + + Assert.assertFalse(sstFiles.isEmpty()); Review Comment: ```suggestion assertThat(sstFiles).isNotEmpty(); ``` It's from `org.assertj.core.api.Assertions#assertThat` ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { Review Comment: ```suggestion class RocksDBStateFileVerifierTest { ``` The public isn't needed, and please use the `RocksDB` instead of `Rocksdb`. I see the prefix of other classes are `RocksDB` ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { + + @TempDir java.nio.file.Path folder; + + @Test + public void rocksdbStateFileVerifierTest() throws Exception { + List columnFamilyHandles = new ArrayList<>(1); + String rootPath = folder.toAbsolutePath().toString(); + File dbPath = new File(rootPath, "db"); + File cpPath = new File(rootPath, "cp"); + + try (DBOptions dbOptions = new DBOptions().setCreateIfMissing(true); + ColumnFamilyOptions colOptions = new ColumnFamilyOptions(); + Options sstFileReaderOptions = new Options(dbOptions, colOptions); + WriteOptions writeOptions = new WriteOptions().setDisableWAL(true); + RocksDB db = + RocksDB.open( + dbOptions, + dbPath.toString(), + Collections.singletonList( + new ColumnFamilyDescriptor( + "default".getBytes(), colOptions)), + columnFamilyHandles); + RocksDBStateFileVerifier rocksDBStateFileVerifier = + new RocksDBStateFileVerifier(sstFileReaderOptions)) { + + byte[] key = "checkpoint".getBytes(); + byte[] val = "incrementalTest".getBytes(); + db.put(writeOptions, key, val); + + try (Checkpoint checkpoint = Checkpoint.create(db)) { + checkpoint.createCheckpoint(cpPath.toString()); + } + + List<Path> sstFiles = + Arrays.stream(FileUtils.listDirectory(cpPath.toPath())) + .filter(file -> file.getFileName().toString().endsWith(SST_FILE_SUFFIX)) + .collect(Collectors.toList()); + + Assert.assertFalse(sstFiles.isEmpty()); + + try { + rocksDBStateFileVerifier.verifySstFilesChecksum(sstFiles); + } catch (IOException e) { + fail(e.getMessage()); + } Review Comment: ```suggestion assertDoesNotThrow(() -> rocksDBStateFileVerifier.verifySstFilesChecksum(sstFiles)); ``` ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; Review Comment: Junit5 should use `org.junit.jupiter.api.Assertions` ########## flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.util.FileUtils; + +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.rocksdb.Checkpoint; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX; +import static org.junit.Assert.fail; + +/** {@link RocksDBStateFileVerifier} test. */ +public class RocksdbStateFileVerifierTest { + + @TempDir java.nio.file.Path folder; + + @Test + public void rocksdbStateFileVerifierTest() throws Exception { + List columnFamilyHandles = new ArrayList<>(1); + String rootPath = folder.toAbsolutePath().toString(); + File dbPath = new File(rootPath, "db"); + File cpPath = new File(rootPath, "cp"); + + try (DBOptions dbOptions = new DBOptions().setCreateIfMissing(true); + ColumnFamilyOptions colOptions = new ColumnFamilyOptions(); + Options sstFileReaderOptions = new Options(dbOptions, colOptions); + WriteOptions writeOptions = new WriteOptions().setDisableWAL(true); + RocksDB db = + RocksDB.open( + dbOptions, + dbPath.toString(), + Collections.singletonList( + new ColumnFamilyDescriptor( + "default".getBytes(), colOptions)), + columnFamilyHandles); + RocksDBStateFileVerifier rocksDBStateFileVerifier = + new RocksDBStateFileVerifier(sstFileReaderOptions)) { + + byte[] key = "checkpoint".getBytes(); + byte[] val = "incrementalTest".getBytes(); + db.put(writeOptions, key, val); + + try (Checkpoint checkpoint = Checkpoint.create(db)) { + checkpoint.createCheckpoint(cpPath.toString()); + } + + List<Path> sstFiles = + Arrays.stream(FileUtils.listDirectory(cpPath.toPath())) + .filter(file -> file.getFileName().toString().endsWith(SST_FILE_SUFFIX)) + .collect(Collectors.toList()); + + Assert.assertFalse(sstFiles.isEmpty()); + + try { + rocksDBStateFileVerifier.verifySstFilesChecksum(sstFiles); + } catch (IOException e) { + fail(e.getMessage()); + } + // corrupt sst file. + Path chosenSstFile = sstFiles.get(0); + File corruptedSstFile = + new File( + chosenSstFile.getParent().toString(), + "corrupted_" + chosenSstFile.getFileName().toString()); + corruptSstFile(chosenSstFile, corruptedSstFile.toPath()); + sstFiles.add(corruptedSstFile.toPath()); + try { + rocksDBStateFileVerifier.verifySstFilesChecksum(sstFiles); + // corrupted sst file would verify failed + fail("verifySstFilesChecksum should failed"); + } catch (IOException e) { + Assertions.assertTrue( + e.getMessage().contains("Error while verifying Checksum of Sst File")); + } Review Comment: ```suggestion assertThatThrownBy(() -> rocksDBStateFileVerifier.verifySstFilesChecksum(sstFiles)) .hasMessageContaining("Error while verifying Checksum of Sst File"); ``` ########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java: ########## @@ -142,6 +143,12 @@ public class EmbeddedRocksDBStateBackend extends AbstractManagedMemoryStateBacke /** Thread number used to transfer (download and upload) state, default value: 1. */ private int numberOfTransferThreads; + /** + * Whether to verify the Checksum of the incremental sst file during Checkpoint in + * RocksDBStateBackend. + */ + private TernaryBoolean enableVerifySstFileChecksum; Review Comment: ```suggestion private final TernaryBoolean enableVerifySstFileChecksum; ``` -- 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