You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Song Ziyang (Jira)" <ji...@apache.org> on 2022/08/16 13:21:00 UTC

[jira] [Created] (RATIS-1674) Frequent inconsistency entries during normal operations

Song Ziyang created RATIS-1674:
----------------------------------

             Summary: Frequent inconsistency entries during normal operations
                 Key: RATIS-1674
                 URL: https://issues.apache.org/jira/browse/RATIS-1674
             Project: Ratis
          Issue Type: Bug
            Reporter: Song Ziyang


*After grpc 1.48.1 release, we packaged ratis with master branch and thirdparty 1.0.2 and test wether out of direct memory issue is fixed. Unfortunately, this error is again reproduced.*

*See the error stack:*

2022-08-16 15:28:45,962 [172.20.70.24_40010@group-000100000001->172.20.70.23_40010-GrpcLogAppender-LogAppenderDaemon] ERROR o.a.r.s.l.LogAppenderDaemon:86 - 172.20.70.24_40010@group-000100000001->172.20.70.23_40010-GrpcLogAppender-LogAppenderDaemon failed
org.apache.ratis.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 2097152 byte(s) of direct memory (used: 17177772311, max: 17179869184)
        at org.apache.ratis.thirdparty.io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:806)
        at org.apache.ratis.thirdparty.io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:735)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:649)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:624)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:203)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena.tcacheAllocateNormal(PoolArena.java:187)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena.allocate(PoolArena.java:136)
        at org.apache.ratis.thirdparty.io.netty.buffer.PoolArena.allocate(PoolArena.java:126)
        at org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:396)
        at org.apache.ratis.thirdparty.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188)
        at org.apache.ratis.thirdparty.io.netty.buffer.AbstractByteBufAllocator.buffer(AbstractByteBufAllocator.java:124)
        at org.apache.ratis.thirdparty.io.grpc.netty.NettyWritableBufferAllocator.allocate(NettyWritableBufferAllocator.java:51)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.writeRaw(MessageFramer.java:285)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.access$400(MessageFramer.java:43)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer$OutputStreamAdapter.write(MessageFramer.java:375)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.write(CodedOutputStream.java:3005)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeLazy(CodedOutputStream.java:3013)
        at org.apache.ratis.thirdparty.com.google.protobuf.ByteString$LiteralByteString.writeTo(ByteString.java:1411)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeBytesNoTag(CodedOutputStream.java:2801)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeBytes(CodedOutputStream.java:2775)
        at org.apache.ratis.proto.RaftProtos$StateMachineLogEntryProto.writeTo(RaftProtos.java:7869)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeMessageNoTag(CodedOutputStream.java:2855)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeMessage(CodedOutputStream.java:2824)
        at org.apache.ratis.proto.RaftProtos$LogEntryProto.writeTo(RaftProtos.java:9503)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeMessageNoTag(CodedOutputStream.java:2855)
        at org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream$OutputStreamEncoder.writeMessage(CodedOutputStream.java:2824)
        at org.apache.ratis.proto.RaftProtos$AppendEntriesRequestProto.writeTo(RaftProtos.java:17799)
        at org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessageLite.writeTo(AbstractMessageLite.java:83)
        at org.apache.ratis.thirdparty.io.grpc.protobuf.lite.ProtoInputStream.drainTo(ProtoInputStream.java:52)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.writeToOutputStream(MessageFramer.java:267)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.writeKnownLengthUncompressed(MessageFramer.java:229)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.writeUncompressed(MessageFramer.java:168)
        at org.apache.ratis.thirdparty.io.grpc.internal.MessageFramer.writePayload(MessageFramer.java:141)
        at org.apache.ratis.thirdparty.io.grpc.internal.AbstractStream.writeMessage(AbstractStream.java:65)
        at org.apache.ratis.thirdparty.io.grpc.internal.ForwardingClientStream.writeMessage(ForwardingClientStream.java:37)
        at org.apache.ratis.thirdparty.io.grpc.internal.RetriableStream.sendMessage(RetriableStream.java:545)
        at org.apache.ratis.thirdparty.io.grpc.internal.ClientCallImpl.sendMessageInternal(ClientCallImpl.java:520)
        at org.apache.ratis.thirdparty.io.grpc.internal.ClientCallImpl.sendMessage(ClientCallImpl.java:506)
        at org.apache.ratis.thirdparty.io.grpc.stub.ClientCalls$CallToStreamObserverAdapter.onNext(ClientCalls.java:374)
        at org.apache.ratis.grpc.server.GrpcLogAppender$StreamObservers.onNext(GrpcLogAppender.java:236)
        at org.apache.ratis.grpc.server.GrpcLogAppender.lambda$sendRequest$1(GrpcLogAppender.java:285)
        at java.util.Optional.map(Optional.java:215)
        at org.apache.ratis.grpc.server.GrpcLogAppender.sendRequest(GrpcLogAppender.java:284)
        at org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:276)
        at org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:160)
        at org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:77)
        at java.lang.Thread.run(Thread.java:748)
2022-08-16 15:28:45,963 [grpc-default-executor-2] WARN  o.a.ratis.util.LogUtils:122 - 172.20.70.24_40010@group-000100000001->172.20.70.23_40010-AppendLogResponseHandler: Failed appendEntries: org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException: CANCELLED: Client sendMessage() failed with Error
2022-08-16 15:28:45,963 [172.20.70.24_40010@group-000100000001->172.20.70.23_40010-GrpcLogAppender-LogAppenderDaemon] INFO  o.a.r.s.i.LeaderStateImpl:531 - 172.20.70.24_40010@group-000100000001-LeaderStateImpl: Restarting GrpcLogAppender for 172.20.70.24_40010@group-000100000001->172.20.70.23_40010

 

{color:#FF0000}*However, we discover some other abnormalities.* {color}

*That is, one follower often report that it receives inconsistent AppendEntries, which keeps writing log entries from time to time. See follower log:*

2022-08-16 15:25:43,253 [172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker] INFO  o.a.r.s.r.s.SegmentedRaftLogWorker$StartLogSegment:629 - 172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker: created new log segment /data/cluster/first-rest-test/DataNode/apache-iotdb/datanode/sbin/../data/consensus/data_region/47474747-4747-4747-4747-000100000001/current/log_inprogress_1376
2022-08-16 15:25:43,345 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1283) already exists (snapshotIndex: -1, commitIndex: 1393)
2022-08-16 15:25:43,345 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2960:FAIL-t1,INCONSISTENCY,nextIndex=1394,followerCommit=1393,matchIndex=-1
2022-08-16 15:25:43,358 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1287) already exists (snapshotIndex: -1, commitIndex: 1393)
2022-08-16 15:25:43,359 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2961:FAIL-t1,INCONSISTENCY,nextIndex=1394,followerCommit=1393,matchIndex=-1
2022-08-16 15:25:43,374 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1290) already exists (snapshotIndex: -1, commitIndex: 1393)
2022-08-16 15:25:43,374 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2962:FAIL-t1,INCONSISTENCY,nextIndex=1394,followerCommit=1393,matchIndex=-1
2022-08-16 15:25:43,387 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1294) already exists (snapshotIndex: -1, commitIndex: 1393)
2022-08-16 15:25:43,387 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2963:FAIL-t1,INCONSISTENCY,nextIndex=1394,followerCommit=1393,matchIndex=-1
2022-08-16 15:25:43,983 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1297) already exists (snapshotIndex: -1, commitIndex: 1393)
2022-08-16 15:25:43,984 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2964:FAIL-t1,INCONSISTENCY,nextIndex=1394,followerCommit=1393,matchIndex=-1
2022-08-16 15:25:44,011 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.r.s.SegmentedRaftLogWorker:430 - 172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker: Rolling segment log-1376_1394 to index:1394
2022-08-16 15:25:44,017 [172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker] INFO  o.a.r.s.r.s.SegmentedRaftLogWorker$FinalizeLogSegment:586 - 172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker: Rolled log segment from /data/cluster/first-rest-test/DataNode/apache-iotdb/datanode/sbin/../data/consensus/data_region/47474747-4747-4747-4747-000100000001/current/log_inprogress_1376 to /data/cluster/first-rest-test/DataNode/apache-iotdb/datanode/sbin/../data/consensus/data_region/47474747-4747-4747-4747-000100000001/current/log_1376-1394
2022-08-16 15:25:44,028 [172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker] INFO  o.a.r.s.r.s.SegmentedRaftLogWorker$StartLogSegment:629 - 172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker: created new log segment /data/cluster/first-rest-test/DataNode/apache-iotdb/datanode/sbin/../data/consensus/data_region/47474747-4747-4747-4747-000100000001/current/log_inprogress_1395
2022-08-16 15:25:44,137 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1394) already exists (snapshotIndex: -1, commitIndex: 1395)
2022-08-16 15:25:44,137 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2966:FAIL-t1,INCONSISTENCY,nextIndex=1396,followerCommit=1395,matchIndex=-1
2022-08-16 15:25:44,158 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1394) already exists (snapshotIndex: -1, commitIndex: 1395)
2022-08-16 15:25:44,158 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2967:FAIL-t1,INCONSISTENCY,nextIndex=1396,followerCommit=1395,matchIndex=-1
2022-08-16 15:25:44,229 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.r.s.SegmentedRaftLogWorker:430 - 172.20.70.23_40010@group-000100000001-SegmentedRaftLogWorker: Rolling segment log-1395_1411 to index:1411
2022-08-16 15:25:44,247 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1475 - 172.20.70.23_40010@group-000100000001: Failed appendEntries as the first entry (index 1394) already exists (snapshotIndex: -1, commitIndex: 1403)
2022-08-16 15:25:44,248 [172.20.70.23_40010-server-thread2] INFO  o.a.r.s.i.RaftServerImpl:1455 - 172.20.70.23_40010@group-000100000001: inconsistency entries. Reply:172.20.70.24_40010<-172.20.70.23_40010#2980:FAIL-t1,INCONSISTENCY,nextIndex=1404,followerCommit=1403,matchIndex=-1

 

*Meanwhile, the leader keeps reporting changing this follower's nextIndex*

2022-08-16 15:25:44,612 [grpc-default-executor-2] INFO  o.a.r.s.i.FollowerInfoImpl:48 - 172.20.70.24_40010@group-000100000001->172.20.70.23_40010: nextIndex: updateUnconditionally 1403 -> 1403
2022-08-16 15:25:44,649 [grpc-default-executor-2] INFO  o.a.r.s.i.FollowerInfoImpl:48 - 172.20.70.24_40010@group-000100000001->172.20.70.23_40010: nextIndex: updateUnconditionally 1405 -> 1403

 

{color:#FF0000}*I looked back at previous directOOM logs and discovered this similar log pattern. {color:#172b4d}Since these "inconsistent AppendEntries" are sent from leader almost with 0 waiting time (according to follower reply timestamp), I guess directOOM may related to continuous Leader "inconsistent" AppendEntries, which consumes lots of direct memory for network transimission.{color}*{color}

{color:#de350b}*Through, I'm confused why a follower will report inconsistent from time to time while still can successfully append logs between these "inconsistent AppendEntries".*{color}

{color:#172b4d}I've set forceFlush to false, FYI. Which means appendLog won't wait for disk operation to complete before reply successfully. I'm not sure wether this async disk flush strategy is the cause for above abnormalities. {color}

{color:#172b4d}I'll be grateful if anyone can help me with this issue. Thanks in advance!{color}

 

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)