Description
This problem was seen with Ozone in datanodes state machine.
Before restart of the datanode, a snapshot was taken at log index 6. Please note that the commit entry for this will be after this log index in the Raft Log.
2019-03-19 08:56:30,225 INFO impl.StateMachineUpdater (StateMachineUpdater.java:stopAndJoin(109)) - StateMachineUpdater-4c165953-147b-48fb-89e1-951579e828eb: set stopIndex = 6 2019-03-19 08:56:30,225 INFO ratis.ContainerStateMachine (ContainerStateMachine.java:takeSnapshot(245)) - Taking snapshot at termIndex:(t:1, i:6) 2019-03-19 08:56:30,226 INFO ratis.ContainerStateMachine (ContainerStateMachine.java:takeSnapshot(249)) - Taking a snapshot to file /Users/msingh/code/apache/ozone/oz_new1/hadoop-ozone/integration-test/t arget/test-dir/MiniOzoneClusterImpl-0fa66624-f533-44bc-8f6f-99cf251fe4c3/datanode-0/data/ratis/34393916-e10d-4b3c-b212-5c910eea4935/sm/snapshot.1_6 2019-03-19 08:56:30,231 INFO impl.RaftServerImpl (ServerState.java:close(386)) - 4c165953-147b-48fb-89e1-951579e828eb closes. The last applied log index is 6
After restart, the state machine register 6 as the log index in the snapshot.
2019-03-19 08:56:33,351 INFO ratis.ContainerStateMachine (ContainerStateMachine.java:loadSnapshot(209)) - Setting the last applied index to (t:1, i:6)
Now, after applying all the transactions after the snapshot, the state machine will encounter a commit entry for this log index (5). this hits the assert in the state machine.
java.io.IOException: java.lang.IllegalStateException: Failed to updateIncreasingly for commitIndex: 6 -> 5 at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:54) at org.apache.ratis.util.IOUtils.toIOException(IOUtils.java:61) at org.apache.ratis.util.IOUtils.getFromFuture(IOUtils.java:70) at org.apache.ratis.server.impl.RaftServerProxy.getImpls(RaftServerProxy.java:283) at org.apache.ratis.server.impl.RaftServerProxy.start(RaftServerProxy.java:295) at org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.start(XceiverServerRatis.java:417) at org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:182) at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.start(DatanodeStateMachine.java:165) at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$0(DatanodeStateMachine.java:334) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.IllegalStateException: Failed to updateIncreasingly for commitIndex: 6 -> 5 at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:72) at org.apache.ratis.server.storage.RaftLogIndex.updateIncreasingly(RaftLogIndex.java:60) at org.apache.ratis.server.storage.RaftLog.lambda$open$7(RaftLog.java:245) at java.util.Optional.ifPresent(Optional.java:159) at org.apache.ratis.server.storage.RaftLog.open(RaftLog.java:244) at org.apache.ratis.server.impl.ServerState.initLog(ServerState.java:191) at org.apache.ratis.server.impl.ServerState.<init>(ServerState.java:114) at org.apache.ratis.server.impl.RaftServerImpl.<init>(RaftServerImpl.java:103) at org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:207) at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590) at java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1582) at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
Attachments
Attachments
Issue Links
- relates to
-
RATIS-506 ServerRestartTests.testRestartCommitIndex may fail intermittently
- Resolved