[ https://issues.apache.org/jira/browse/RATIS-377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16667827#comment-16667827 ]
Ted Yu commented on RATIS-377: ------------------------------ In SegmentedRaftLogFormat : {code} private static void assertHeader() { Preconditions.assertTrue(Arrays.equals(HEADER_BYTES, HEADER_BYTES_CLONE)); {code} It seems if {{HEADER_BYTES_CLONE}} is modified, the assertion becomes less protective. Can we consider using something such as https://static.javadoc.io/org.apache.fluo/fluo-api/1.2.0/org/apache/fluo/api/data/Bytes.html ? > Tolerate partially written log header > ------------------------------------- > > Key: RATIS-377 > URL: https://issues.apache.org/jira/browse/RATIS-377 > Project: Ratis > Issue Type: Bug > Affects Versions: 0.3.0 > Reporter: Nilotpal Nandi > Assignee: Tsz Wo Nicholas Sze > Priority: Blocker > Fix For: 0.3.0 > > Attachments: r377_20181028c.patch > > > steps taken : > ------------------ > # wrote 5GB files through ozonefs > # stopped datanodes, scm , om. > # started all services. > # Tried to read the file. > One of the datanodes failed to start. Throwing > "java.lang.IllegalStateException: Corrupted log header" > > {noformat} > 2018-10-26 10:26:01,317 ERROR org.apache.ratis.server.storage.LogInputStream: > caught exception initializing log_inprogress_293 > java.lang.IllegalStateException: Corrupted log header: ^@^@^@^@^@^@^@^@ > at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60) > at > org.apache.ratis.server.storage.LogInputStream.init(LogInputStream.java:93) > at > org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:120) > at > org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111) > at > org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133) > at > org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110) > at > org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151) > at > org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120) > 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:106) > at > org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196) > 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) > 2018-10-26 10:26:03,671 INFO > org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer: Listening HDDS > REST traffic on /0.0.0.0:9880 > 2018-10-26 10:26:03,672 INFO org.apache.hadoop.ozone.HddsDatanodeService: > Started plug-in org.apache.hadoop.ozone.web.OzoneHddsDatanodeService@1e411d81 > 2018-10-26 10:26:03,676 INFO > org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer: Attempting to > start container services. > 2018-10-26 10:26:03,676 INFO > org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis: > Starting XceiverServerRatis 0d7f5327-df16-40fe-ac88-7ed06e76a20f at port 9858 > 2018-10-26 10:26:03,702 ERROR > org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine: > Unable to start the DatanodeState Machine > java.io.IOException: java.lang.IllegalStateException: Corrupted log header: > ^@^@^@^@^@^@^@^@ > at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:51) > at > org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:123) > at > org.apache.ratis.server.storage.LogSegment.readSegmentFile(LogSegment.java:111) > at > org.apache.ratis.server.storage.LogSegment.loadSegment(LogSegment.java:133) > at > org.apache.ratis.server.storage.RaftLogCache.loadSegment(RaftLogCache.java:110) > at > org.apache.ratis.server.storage.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:151) > at > org.apache.ratis.server.storage.SegmentedRaftLog.open(SegmentedRaftLog.java:120) > 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:106) > at > org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:196) > 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) > Caused by: java.lang.IllegalStateException: Corrupted log header: > ^@^@^@^@^@^@^@^@ > at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60) > at > org.apache.ratis.server.storage.LogInputStream.init(LogInputStream.java:93) > at > org.apache.ratis.server.storage.LogInputStream.nextEntry(LogInputStream.java:120) > ... 15 more > 2018-10-26 10:26:03,703 ERROR org.apache.hadoop.ozone.HddsDatanodeService: > Exception in HddsDatanodeService. > java.lang.NullPointerException > at > org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.join(DatanodeStateMachine.java:331) > at > org.apache.hadoop.ozone.HddsDatanodeService.join(HddsDatanodeService.java:191) > at > org.apache.hadoop.ozone.HddsDatanodeService.main(HddsDatanodeService.java:250) > 2018-10-26 10:26:03,706 INFO org.apache.hadoop.util.ExitUtil: Exiting with > status 1: java.lang.NullPointerException > 2018-10-26 10:26:03,714 WARN org.apache.hadoop.fs.CachingGetSpaceUsed: Thread > Interrupted waiting to refresh disk information: sleep interrupted > 2018-10-26 10:26:03,715 INFO org.apache.hadoop.ozone.HddsDatanodeService: > SHUTDOWN_MSG: > /************************************************************ > SHUTDOWN_MSG: Shutting down HddsDatanodeService at > ctr-e138-1518143905142-541661-01-000005.hwx.site/172.27.68. > {noformat} > -- This message was sent by Atlassian JIRA (v7.6.3#76005)