-
Notifications
You must be signed in to change notification settings - Fork 408
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
RATIS-1656. Leftover usage of ForkJoinPool.commonPool() in RaftServerImpl #702
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@adoroszlai , thanks a lot for catching this! Please see the comment inlined.
@@ -1413,7 +1413,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), NOT_ | |||
getRaftServer().getPeer()); | |||
} | |||
} | |||
return JavaUtils.allOf(futures).whenCompleteAsync( | |||
return JavaUtils.allOf(futures).whenComplete( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actions supplied for dependent completions of non-async methods may be performed by the thread that completes the current CompletableFuture, or by any other caller of a completion method.
According to the above javadoc https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html , it uses the thread, which is SegmentedRaftLogWorker in this case, completing futures
passed to allOf(..)
but not the thread running appendEntriesAsync
. Therefore, we should pass serverExecutor as below
return JavaUtils.allOf(futures).whenCompleteAsync(
(r, t) -> followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)),
serverExecutor
).thenApply(v -> {
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @szetszwo for the review. Passing serverExecutor
was my first attempt, but it caused timeout in TestInstallSnapshotNotificationWithGrpc
.
It may indicate that the test or some other part of Ratis needs further tweak. I'll take another look.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TestInstallSnapshotNotificationWithGrpc
may be flaky after all, though I haven't found recent commits with the same failure.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 the change looks good.
Tested
Timeout happens while cluster is shutting down:
Looking into that I've found With that additional change
|
|
@adoroszlai , thanks a lot for working hard on this! How about we merge the current change? |
Thanks @szetszwo for the review. |
What changes were proposed in this pull request?
RaftServerImpl#appendEntriesAsync
is still using the common pool here:ratis/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
Lines 1416 to 1417 in 3db8f9e
Preceding steps in
appendEntriesAsync
are already running onserverExecutor
, so I think there is no need to further defer this step.https://issues.apache.org/jira/browse/RATIS-1656
How was this patch tested?
Regular CI:
https://github.com/adoroszlai/incubator-ratis/actions/runs/2789205160