[ https://issues.apache.org/jira/browse/FLINK-19016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17181926#comment-17181926 ]
Jiayi Liao edited comment on FLINK-19016 at 8/21/20, 4:23 PM: -------------------------------------------------------------- [~sewen] All of your assumptions are right. (BTW the local recovery is not enabled.) I can share my thoughts here but I'm not sure that I'm completely right. During RocksDB's takeDBNativeCheckpoint, RocksDB will invoke {{create_file_cb}} on current in-progress file (see [here|https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/checkpoint/checkpoint_impl.cc#L292]), which will invoke CreateFile function(see [here|https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/util/file_util.cc#L70]). Since Flink sets sync mode in db's options, the CreateFile and its Append operation may not succeed on disk. (If using fsync, I think it should be guranteed.) I'm not expert on Linux and RocksDB, plz correct me if I'm wrong. was (Author: wind_ljy): [~sewen] All of your assumptions are right. (BTW the local recovery is not enabled.) I can share my thoughts here but I'm not sure that I'm completely right. During RocksDB's takeDBNativeCheckpoint, RocksDB will invoke {{create_file_cb}} on current in-progress file (see [here|https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/checkpoint/checkpoint_impl.cc#L292]), which will invoke CreateFile function(see [here|https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/util/file_util.cc#L70]). Since Flink uses sync operation, the CreateFile and its Append operation may not succeed on disk. (If using fsync, I think it should be guranteed.) I'm not expert on Linux and RocksDB, plz correct me if I'm wrong. > Checksum mismatch when restore from RocksDB > ------------------------------------------- > > Key: FLINK-19016 > URL: https://issues.apache.org/jira/browse/FLINK-19016 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing > Affects Versions: 1.11.1 > Reporter: Jiayi Liao > Priority: Major > > The error stack is shown below: > {code:java} > Caused by: org.apache.flink.util.FlinkException: Could not restore keyed > state backend for > KeyedMapBundleOperator_44cfc1ca74b40bb44eed1f38f72b3ea9_(71/300) from any of > the 1 provided restore options. > at > org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135) > at > org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:307) > at > org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:135) > ... 6 more > Caused by: org.apache.flink.runtime.state.BackendBuildingException: Caught > unexpected exception. > at > org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder.build(RocksDBKeyedStateBackendBuilder.java:333) > at > org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:580) > at > org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:291) > at > org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142) > at > org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121) > ... 8 more > Caused by: java.io.IOException: Error while opening RocksDB instance. > at > org.apache.flink.contrib.streaming.state.RocksDBOperationUtils.openDB(RocksDBOperationUtils.java:74) > at > org.apache.flink.contrib.streaming.state.restore.AbstractRocksDBRestoreOperation.openDB(AbstractRocksDBRestoreOperation.java:131) > at > org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation.restoreFromLocalState(RocksDBIncrementalRestoreOperation.java:214) > at > org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation.restoreFromRemoteState(RocksDBIncrementalRestoreOperation.java:188) > at > org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation.restoreWithoutRescaling(RocksDBIncrementalRestoreOperation.java:162) > at > org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation.restore(RocksDBIncrementalRestoreOperation.java:148) > at > org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder.build(RocksDBKeyedStateBackendBuilder.java:277) > ... 12 more > Caused by: org.rocksdb.RocksDBException: checksum mismatch > at org.rocksdb.RocksDB.open(Native Method) > at org.rocksdb.RocksDB.open(RocksDB.java:286) > at > org.apache.flink.contrib.streaming.state.RocksDBOperationUtils.openDB(RocksDBOperationUtils.java:66) > ... 18 more > {code} > The machine goes down because of hardware problem, then the job cannot > restart successfully anymore. After digging a little bit, I found that > RocksDB in Flink uses sync instead of fsync to synchronized the data with the > disk. With sync operation, the RocksDB cannot guarantee that the current > in-progress file can be persisted on disk in takeDBNativeCheckpoint. -- This message was sent by Atlassian Jira (v8.3.4#803005)