[
https://issues.apache.org/jira/browse/RATIS-2208?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17907773#comment-17907773
]
Song Ziyang commented on RATIS-2208:
------------------------------------
Hi [~szetszwo], thank you for your suggestion on fixing the issue and for
providing the patch to improve the debugging experience! I spent several days
thoroughly investigating this problem and discovered another tricky issue.
{quote}*How could it cause the problem for RATIS-2203?*
{quote}
RATIS-2203 tracks the issue where membership changes get stuck. The root cause
of this problem lies in the event sequence that leads to a membership change
being stuck, which occurs as follows:
# Leader A receives a SetConfiguration request from the client to add node D
to an existing group (A, B, C).
# Leader A sends an AppendEntries RPC containing over 8000 log entries to D.
# It takes D about 20 seconds to process these 8000+ log entries (with each
entry taking 1-3 ms). D fails to respond to A within the timeout.
# Leader A assumes D’s bootstrapping is in NOPROGRESS state, and the
SetConfiguration operation fails.
# The client retries the operation, but it fails each time, causing the
membership change to be stuck.
{quote}*Should we avoid this exception by forcing AppendEntries to wait for the
previous AppendEntries to complete?*
{quote}
That seems reasonable. The {{IllegalStatementException}} occurs when the client
retries the SetConfiguration operation while a previous AppendEntries is still
in progress.
h3. Are there other ways to solve this problem?
Yes and no.
RATIS provides an optional feature to decouple heartbeat and AppendEntries by
using a separate channel.
[https://github.com/apache/ratis/blob/551b8c94b78b8e8611c6c77794617d12682a9140/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java#L266]
If we enable this feature, the expected behavior is that even if AppendEntries
is delayed (e.g., taking 20 seconds), the leader should continue sending
heartbeat requests, and the follower should keep responding to those heartbeat
requests. In this case, the leader would be able to update the last RPC
response time and would not classify the follower as NOPROGRESS. *This could be
a perfect solution for this issue.*
However, even with this separate channel option enabled, the problem still
persists. I{*}t turns out that heartbeat and AppendEntries still experience a
data race due to RaftLog’s lock. Even with a separate channel, heartbeat
requests can still be blocked by a preceding AppendEntries request.{*}
# AppendEntries will hold the RaftLog write lock in
[https://github.com/apache/ratis/blob/551b8c94b78b8e8611c6c77794617d12682a9140/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java#L499]
# Heartbeat will require the RaftLog write in
[https://github.com/apache/ratis/blob/551b8c94b78b8e8611c6c77794617d12682a9140/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java#L1657]which
boils down to
[https://github.com/apache/ratis/blob/551b8c94b78b8e8611c6c77794617d12682a9140/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java#L125]
Do you have any suggestion on this lock race? Thanks in advance!
Happy an early Christmas! :D
> IllegalStateException: SegmentedRaftLog: Already running a method by
> --------------------------------------------------------------------
>
> Key: RATIS-2208
> URL: https://issues.apache.org/jira/browse/RATIS-2208
> Project: Ratis
> Issue Type: Bug
> Components: gRPC, Leader, server
> Affects Versions: 3.1.2
> Reporter: Song Ziyang
> Assignee: Song Ziyang
> Priority: Major
>
>
> {code:java}
> 2024-12-06 18:19:18,750 [4-server-thread3] ERROR
> o.a.r.s.i.RaftServerImpl:1481 - 4@group-000200000030: Failed appendEntries*
> 9->4#3-t1,previous=(t:0, i:0),leaderCommit=9097,initializing? true,entries:
> size=9098, first=(t:1, i:0),
> CONFIGURATIONENTRY(current:id:"9"address:"172.16.2.9:10750"startupRole:FOLLOWER,
> old:) java.lang.IllegalStateException:
> 4@group-000200000030-SegmentedRaftLog: Already running a method by
> Thread[4-server-thread2,5,main], current=Thread[4-server-thread3,5,main]
> at
> org.apache.ratis.server.raftlog.RaftLogSequentialOps$Runner.runSequentially(RaftLogSequentialOps.java:80)
>
> at org.apache.ratis.server.raftlog.RaftLogBase.append(RaftLogBase.java:359)
>
> at
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1590)
>
> at
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1479)
>
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:645)
>
> at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
> at
> org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:899)
>
> at
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
>
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
>
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>
> at java.base/java.lang.Thread.run(Thread.java:833) {code}
> How this issue was triggered?
>
> # Client C (IoTDB Application) adds a new node A to an existing Raft Group
> via SetConf request.
> # Leader tries to bootstrap A by sending AppendEntries with (9000+ log
> entries)
> # appendEntries operation in new node A +*takes exceptionally long time,*+
> (~1-3 ms each entry, 20+ seconds in total by estimation). Therefore, A fails
> to respond this AppendEntries request within timeout (12s as configured in
> IoTDB).
> # Leader think the bootstrapping process failed and respond to client
> notifying SetConf failure.
> # Client C retries SetConf immediately.
> # Leader tries to bootstrap A by sending AppendEntries, {+}*again*{+}.
> However, at this moment, +*the previous AppendEntries is still ongoing. That
> triggered IllegalStateException.*+
>
> This exception suggests that even one AppendEntries request size is small
> within 4-16MB, the time need to process this AppendEntries request is still
> very long if it is consisted of large amount of tiny chunk of logs. Possible
> solutions:
> # Constraint max number of entries within a AppendEntries.
> # Batch write tasks at follower side.
> # Other solutions.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)