-
Notifications
You must be signed in to change notification settings - Fork 421
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-2174. Move future.join outside the lock #1168
RATIS-2174. Move future.join outside the lock #1168
Conversation
2943809
to
2ec2d11
Compare
@szetszwo |
bcc128e
to
2b0ce8f
Compare
ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
Outdated
Show resolved
Hide resolved
Please check the comments below. could you help me refer to which one to use as the final solution?
|
@133tosakarin , sorry for the delay. Will review this soon. |
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.
@133tosakarin , thanks a lot for working on this! The idea is great.
- We should avoid using
Optional
. Usingfinal
is better since the compiler will enforce that the variable must be initialized before use. - Let's pass an
AtomicBoolean
instead of adding the newPair
class. It will have much less code change.
See https://issues.apache.org/jira/secure/attachment/13072253/1168_review.patch
Sze, thank you for your advice! Our current commit is to move future.join out of the lock, but OneSizeFitsQuorum thinks there may be concurrency issues. What do you think? |
Suppose we have the following code: synchronized(server) {
...
final CompletableFuture<Void> future = state.shudown()
f.join()
}
|
Thank you for your valuable advice! We have reached a consensus Regarding the question raised by OneSizeFitsQuorum, the concern might be that when thread A waits for the state to shut down outside the OneSizeFitsQuorum recommends writing it as follows: For Thread A:
For Thread State:
In our code, the State thread may be LogAppender, FollowerState, LeaderElection |
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.
@133tosakarin , thanks for the update!
Please don't use CompletableFuture.runAsync(future::join);
. We either want to
- sync: call
future.join()
, or - async: do nothing (the code is already async).
See the details inlined.
@@ -333,7 +333,7 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, | |||
LOG.info("{} {} round {}: submit vote requests at term {} for {}", this, phase, round, electionTerm, conf); | |||
final ResultAndTerm r = submitRequestAndWaitResult(phase, conf, electionTerm); | |||
LOG.info("{} {} round {}: result {}", this, phase, round, r); | |||
|
|||
CompletableFuture<Void> future = null; |
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.
It is better to use final CompletableFuture<Void> future;
. Then future.join()
will never have NPE.
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.
Thank you for your valuable advice
state.persistMetadata(); | ||
} | ||
} catch (IOException e) { | ||
CompletableFuture.runAsync(future::join); |
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.
CompletableFuture.runAsync
means use another thread to call join()
. It is not useful. More details:
- Without this line, the shutdown thread will run and complete the future.
- With this line, the shutdown thread will run and complete the future. In addition, we use another thread to wait for the shutdown thread. Why use one more thread?
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.
Assuming we encounter an exception here, we either do not handle the future or need another thread to handle it. If we do not use another thread to handle the future, there is still a deadlock problem of holding the lock and waiting for the future.
The same consideration applies to using runAsync elsewhere
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.
If we remove line 620, how could we have a deadlock? Could you give more details?
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.
Replacing CompleteFuture.runAsync(future::join) with future.join still presents the deadlock issue where thread A holds the lock while waiting for the future, and thread B needs to acquire the lock.
If we simply ignore the future and throw an exception without handling it, then no deadlock will occur.
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.
Replacing CompleteFuture.runAsync(future::join) with future.join ...
I mean "remove the line", not "replace it with future.join", i.e. case 3 below.
Three cases:
- With
future.join()
, the current thread will wait for the shutdown thread to complete - With
CompleteFuture.runAsync(future::join)
, the current thread and the shutdown thread will run in parallel. In addition, it uses one more thread to wait for the shutdown thread. - With an empty line, the current thread and the shutdown thread will run in parallel.
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.
I see.
case 1. We should avoid this situation, which seems to contradict our issue
case 2. This writing style does not seem to be recommended
case 3. It seems that we can only use this
Then we can use the third case
synchronized (server) {
future = state.shutdown();
while (!future.isDone()) {
server.wait();
}
} try {
run();
} finally {
stopped.complete();
synchronized (server) {
server.notify();
}
}
synchronized (server) {
future = state.shutdown();
}
future.join() try {
run();
} finally {
stopped.complete();
} Version 1 and Version 2 seem to be the same. Could you point out when they will be different? |
For version 1, we need to wait for another thread to wake up before continuing execution. For version 2, the current thread and the thread that needs to be shut down may execute in parallel. |
No. |
Thank you very much for your explanation. |
state.setLeader(leaderId, "installSnapshot"); | ||
|
||
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); | ||
if (snapshotChunkRequest.getRequestIndex() == 0) { | ||
nextChunkIndex.set(0); | ||
} else if (nextChunkIndex.get() != snapshotChunkRequest.getRequestIndex()) { | ||
// exception new throw, we need join future in another thread | ||
CompletableFuture.runAsync(future::join); |
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.
@133tosakarin , we need to remove all CompletableFuture.runAsync(future::join)
. They are not useful. See #1168 (comment)
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.
Ok, I will modify this part of the code.
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.
@133tosakarin , thanks for the update! Please see the comment below and the comments inlined.
- In
LeaderElection
, it should NOT completestopped
inshutdown()
since the thread is still running at that time.
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
@@ -227,7 +227,6 @@ class LeaderElection implements Runnable {
CompletableFuture<Void> shutdown() {
lifeCycle.checkStateAndClose();
- stopped.complete(null);
return stopped;
}
RaftPeerId leaderId) throws IOException { | ||
final long currentTerm; | ||
final long leaderTerm = request.getLeaderTerm(); | ||
final InstallSnapshotRequestProto.SnapshotChunkProto snapshotChunkRequest = request.getSnapshotChunk(); | ||
final TermIndex lastIncluded = TermIndex.valueOf(snapshotChunkRequest.getTermIndex()); | ||
final long lastIncludedIndex = lastIncluded.getIndex(); | ||
final CompletableFuture<Void> future; | ||
final InstallSnapshotReplyProto reply; |
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.
Remove reply
and declare it in the code below.
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.
ok, thanks for your comment
@@ -186,8 +189,9 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest | |||
// have a lot of requests | |||
if (state.getLog().getLastCommittedIndex() >= lastIncludedIndex) { | |||
nextChunkIndex.set(snapshotChunkRequest.getRequestIndex() + 1); | |||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | |||
reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), |
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.
final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(),
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.
Ok
return future.thenApply(dummy -> toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS)); |
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.
Create the reply first since it does not have to wait for the shutdown.
final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(),
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS);
return future.thenApply(dummy -> reply);
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.
Ok
InstallSnapshotRequestProto request, RaftPeerId leaderId) throws IOException { | ||
final long currentTerm; | ||
final long leaderTerm = request.getLeaderTerm(); | ||
final TermIndex firstAvailableLogTermIndex = TermIndex.valueOf( | ||
request.getNotification().getFirstAvailableTermIndex()); | ||
final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); | ||
final CompletableFuture<Void> future; | ||
final InstallSnapshotReplyProto replyProto; |
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.
Similar to before, move down the declaration, create the reply and return future.thanApply(..).
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.
Ok
} | ||
} | ||
|
||
private RoleInfoProto getRoleInfoProto(RaftPeer leader) { | ||
private RoleInfoProto getRoleInfoProto (RaftPeer leader){ |
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.
Please revert this whitespace change.
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.
Ok
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
(Used the web editor to revert a whitespace change but it did not go well. Then, edited it locally and pushed another whitespace reverts.)
Remove join in FollowerState and LeaderElection. For FollowerState, changing to Candidate will close FollowerState and then return to FollowerState.stopped, which is equivalent to waiting for oneself to close. LeaderElection is similar to FollowerState. When changed to Leader, LeaderElection will be closed and then returned to LeaderElection.stop. |
There seems to be a strange error: the address has already been used |
It happens occasionally. It may be a test problem (or library, OS bugs.) Not sure if there is a way to prevent it. Let's simply rerun the tests for now. |
#[d0cd3de] |
In this commit [d0cd3de ] , the following modifications are mainly made
|
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.
@133tosakarin , good job! The latest change can pass everything at the first run.
What changes were proposed in this pull request?
In recent observations, I found that some functions wait for a future while holding a lock, which is a very dangerous thing.
To avoid the deadlock problem caused by holding the lock and waiting for the future, I am considering moving the future waiting outside the lock.
Currently, I found that calling changeToFollower in the following functions causes the above situation:
see https://issues.apache.org/jira/browse/RATIS-2174