[ 
https://issues.apache.org/jira/browse/RATIS-677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16984349#comment-16984349
 ] 

Istvan Fajth commented on RATIS-677:
------------------------------------

Yes default is EXCEPTION, and I have ran into this as well on a cluster where I 
just restarted the DNs without changing any binary. My cluster is running an 
internal Cloudera build, which is pretty close to master and has this change 
in, it seems though if you set the raft.server.log.corruption.policy from 
Cloudera Manager into the Ozone DataNode Safety Valve for ozone-site.xml, the 
value is set in the xml, but the DN does not read that property. This one seems 
to be a Cloudera Manager internal problem, I just wanted to note I have ran 
into the problem.

What I did before the issue came up, is I ran some data generation onto the DNs 
raft was suffering from heavy GC and got into a couple of leader election, so 
that I increased the DN heap, and did a restart, after the restart I lost 25% 
of my DNs (3 out of 12) due to this problem, and SCM did not came out from safe 
mode.

Based on this I would as whether changing the default to WARN_AND_RETURN would 
be feasible, as in almost all of these cases if this option is present and the 
issue comes up, then that would be the way to go anyways, what is the reason 
behind keeping it as EXCEPTION by default?

> Log entry marked corrupt due to ChecksumException
> -------------------------------------------------
>
>                 Key: RATIS-677
>                 URL: https://issues.apache.org/jira/browse/RATIS-677
>             Project: Ratis
>          Issue Type: Bug
>          Components: server
>            Reporter: Sammi Chen
>            Assignee: Tsz-wo Sze
>            Priority: Blocker
>             Fix For: 0.5.0
>
>         Attachments: r677_20190913.patch, r677_20190919.patch, 
> r677_20190919b.patch, r677_20190920.patch
>
>
> Steps:
> 1.  Run Teragen and generated a few GB data in a 4 datanodes cluster.  
> 2.  Stoped the datanodes through ./stop-ozone.sh.
> 3.  Changed the ozone binaries
> 4.  Start the cluster through ./start-ozone.sh.
> 5.  Two datanode regisisterd to SCM. Two datanode fail to appear at SCM side. 
>  
> Checked these two failed node, datanode process is still running. In the 
> logfile, I found a lot of following errors. 
> 2019-09-12 21:06:45,255 [Datanode State Machine Thread - 0] INFO       - 
> Starting XceiverServerRatis ba17ad5e-714e-4d82-85d8-ff2e0737fcf9 at port 9858
> 2019-09-12 21:06:47,255 [Datanode State Machine Thread - 0] INFO       - 
> Attempting to start container services.
> 2019-09-12 21:06:47,255 [Datanode State Machine Thread - 0] INFO       - 
> Background container scanner has been disabled.
> 2019-09-12 21:06:47,255 [Datanode State Machine Thread - 0] INFO       - 
> Starting XceiverServerRatis ba17ad5e-714e-4d82-85d8-ff2e0737fcf9 at port 9858
> 2019-09-12 21:06:47,255 [Datanode State Machine Thread - 0] ERROR      - 
> Unable to communicate to SCM server at 10.120.110.183:9861 for past 2100 
> seconds.
> org.apache.ratis.protocol.ChecksumException: LogEntry is corrupt. Calculated 
> checksum is -134141393 but read checksum 0
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogReader.decodeEntry(SegmentedRaftLogReader.java:299)
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogReader.readEntry(SegmentedRaftLogReader.java:185)
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogInputStream.nextEntry(SegmentedRaftLogInputStream.java:121)
>         at 
> org.apache.ratis.server.raftlog.segmented.LogSegment.readSegmentFile(LogSegment.java:94)
>         at 
> org.apache.ratis.server.raftlog.segmented.LogSegment.loadSegment(LogSegment.java:117)
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache.loadSegment(SegmentedRaftLogCache.java:310)
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.loadLogSegments(SegmentedRaftLog.java:234)
>         at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.openImpl(SegmentedRaftLog.java:204)
>         at org.apache.ratis.server.raftlog.RaftLog.open(RaftLog.java:247)
>         at 
> org.apache.ratis.server.impl.ServerState.initRaftLog(ServerState.java:190)
>         at 
> org.apache.ratis.server.impl.ServerState.<init>(ServerState.java:120)
>         at 
> org.apache.ratis.server.impl.RaftServerImpl.<init>(RaftServerImpl.java:110)
>         at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:208)
>         at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to