[
https://issues.apache.org/jira/browse/RATIS-2208?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17907924#comment-17907924
]
Tsz-wo Sze commented on RATIS-2208:
-----------------------------------
[~William Song], thanks a lot for the update!
bq. Heartbeat will require the RaftLog write ...
updateCommitIndex actually is optional. We may add a timeout.
{code}
diff --git
a/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
index 8a5409bafe..9581e925a5 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
@@ -17,6 +17,7 @@
*/
package org.apache.ratis.util;
+import java.util.Objects;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
@@ -45,6 +46,13 @@ public final class AutoCloseableLock implements
AutoCloseable {
return new AutoCloseableLock(lock, preUnlock);
}
+ public static AutoCloseableLock tryAcquire(final Lock lock, Runnable
preUnlock, TimeDuration timeout)
+ throws InterruptedException {
+ Objects.requireNonNull(timeout, "timeout == null");
+ final boolean locked = lock.tryLock(timeout.getDuration(),
timeout.getUnit());
+ return locked? new AutoCloseableLock(lock, preUnlock): null;
+ }
+
private final Lock underlying;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Runnable preUnlock;
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java
b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java
index 9353612c8c..d6314bc13f 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java
@@ -122,7 +122,7 @@ public abstract class RaftLogBase implements RaftLog {
@Override
public boolean updateCommitIndex(long majorityIndex, long currentTerm,
boolean isLeader) {
- try(AutoCloseableLock writeLock = writeLock()) {
+ try(AutoCloseableLock writeLock = tryWriteLock(TimeDuration.ONE_SECOND)) {
final long oldCommittedIndex = getLastCommittedIndex();
final long newCommitIndex = Math.min(majorityIndex, getFlushIndex());
if (oldCommittedIndex < newCommitIndex) {
@@ -136,6 +136,9 @@ public abstract class RaftLogBase implements RaftLog {
return commitIndex.updateIncreasingly(newCommitIndex,
traceIndexChange);
}
}
+ } catch (InterruptedException e) {
+ LOG.warn("{}: Interrupted to updateCommitIndex: majorityIndex={},
currentTerm={}, isLeader={}",
+ getName(), majorityIndex, currentTerm, isLeader, e);
}
return false;
}
@@ -389,6 +392,10 @@ public abstract class RaftLogBase implements RaftLog {
return AutoCloseableLock.acquire(lock.writeLock());
}
+ public AutoCloseableLock tryWriteLock(TimeDuration timeout) throws
InterruptedException {
+ return AutoCloseableLock.tryAcquire(lock.writeLock(), null, timeout);
+ }
+
public boolean hasWriteLock() {
return this.lock.isWriteLockedByCurrentThread();
}
{code}
bq. Happy an early Christmas!
Merry Christmas!
> 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)