-
Notifications
You must be signed in to change notification settings - Fork 422
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-2162. When closing leaderState, if the logAppender thread sends a snapshot, a deadlock may occur #1154
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.
@133tosakarin , thanks a lot for working on this! It is great that you have idenified the problem that it join()
the leader state with the RaftServerImpl
lock. Let's change the code to join()
without the lock. See https://issues.apache.org/jira/secure/attachment/13071769/1154_review.patch and also the comments inlined.
ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
Show resolved
Hide resolved
return senders.removeAll(c); | ||
} | ||
|
||
CompletableFuture<Void> stopAll() { | ||
synchronized CompletableFuture<Void> stopAll() { |
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.
Since it is a CopyOnWriteArrayList
, we should use iterator()
or stream()
instead of get(i)
, i.e.
CompletableFuture<Void> stopAll() {
return CompletableFuture.allOf(senders.stream().map(LogAppender::stopAsync).toArray(CompletableFuture[]::new));
}
Then, we don't need synchronized
for all the methods in SenderList
.
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 for the review, I will make changes
private final RaftGroupMemberId memberId; | ||
private final AtomicReference<RaftGroupMemberId> memberId = new AtomicReference<>(); |
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.
Let's don't add AtomicReference
since memberId
is never changed.
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 for the review, I will make changes
synchronized (server) { | ||
final SnapshotChunkProto.Builder b = LeaderProtoUtils.toSnapshotChunkProtoBuilder( | ||
requestId, requestIndex++, snapshot.getTermIndex(), chunk, totalSize, done); | ||
return LeaderProtoUtils.toInstallSnapshotRequestProto(server, followerId, b); | ||
} | ||
final SnapshotChunkProto.Builder b = LeaderProtoUtils.toSnapshotChunkProtoBuilder( | ||
requestId, requestIndex++, snapshot.getTermIndex(), chunk, totalSize, done); | ||
return LeaderProtoUtils.toInstallSnapshotRequestProto(server, followerId, b); |
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.
Let's keep synchronized (server)
. Otherwise, it could create an inconsistent request.
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 for the review, I will make changes
@@ -128,7 +128,7 @@ public void start() { | |||
|
|||
@Override | |||
public boolean isRunning() { | |||
return daemon.isWorking(); | |||
return daemon.isWorking() && server.getInfo().isLeader(); |
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.
This is a good 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.
@133tosakarin , thanks for the update! Please see the comments inlined.
private RaftConfigurationImpl currentConf; | ||
/** Cache the peer corresponding to {@link #id}. */ | ||
@SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type | ||
private volatile RaftPeer currentPeer; | ||
private RaftPeer currentPeer; |
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 remove also the @SuppressWarnings({"squid:S3077"})
for both fields. It was for the volatile
false positive.
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!
@@ -219,16 +219,13 @@ void addAll(Collection<LogAppender> newSenders) { | |||
Preconditions.assertTrue(changed); | |||
} | |||
|
|||
boolean removeAll(Collection<LogAppender> c) { | |||
synchronized boolean removeAll(Collection<LogAppender> c) { |
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.
We don't need synchronized
for both addAll
and removeAll
since it uses a CopyOnWriteArrayList
.
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.
There are some other deadlock problem, I need to check |
@szetszwo It seems that there are also cases where the unit (misc) test fails in the master branch |
@133tosakarin , the test failures in misc may be related to this change. In #1152 , the misc tests did not fail. |
I see. I will further check |
… a snapshot, a deadlock may occur. (apache#1154)
Let's try testing it over release-3.1.1; see #1157 |
#1157 has passed over 3.1.1. |
… a snapshot, a deadlock may occur. (#1154)
Hi @133tosakarin Here is the full jstack: https://issues.apache.org/jira/secure/attachment/13073619/13073619_celeborn_master_jstack.txt And here is the ticket in celeborn community: https://issues.apache.org/jira/browse/CELEBORN-1795?focusedCommentId=17907470&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17907470 BTW: The ratis rpc type is |
Thank you for providing the question. I hope you can post the specific information to Jira, and we will keep following up. |
Thanks for the reminder, I have update the info in RATIS-2162. I can create a new RATIS ticket if necessary. Thank you. |
What changes were proposed in this pull request?
I'm thinking of replacing the sychronized (server) block in InstallSnapshotRequests with some atomic variables newInstallSnapshotRequest
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2162