133tosakarin commented on PR #1168: URL: https://github.com/apache/ratis/pull/1168#issuecomment-2421362741
> > ... . Even if we release the lock held at the changeToFollower level, the upper-layer locks are still in place. ... > > This is a good. We need to make sure that it is not holding the server lock at all. Have you found any cases that https://issues.apache.org/jira/secure/attachment/13072253/1168_review.patch is still hold the server lock when calling join()? Except for the last future.thenApply(dummy -> reply), it seems that the other instances are joining within the lock. ``` private CompletableFuture<InstallSnapshotReplyProto> notifyStateMachineToInstallSnapshot( 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; synchronized (server) { final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, InstallSnapshotResult.NOT_LEADER); + return CompletableFuture.completedFuture(toInstallSnapshotReplyProto( + leaderId, getMemberId(), currentTerm, InstallSnapshotResult.NOT_LEADER)); } - server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); @@ -245,8 +250,9 @@ class SnapshotInstallationHandler { inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); + return future.thenApply(dummy -> reply); } final RaftPeerProto leaderProto; @@ -323,8 +329,9 @@ class SnapshotInstallationHandler { inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); + return future.thenApply(dummy -> reply); } // If a snapshot has been installed, return SNAPSHOT_INSTALLED with the installed snapshot index and reset @@ -341,8 +348,9 @@ class SnapshotInstallationHandler { server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); + return future.thenApply(dummy -> reply); } // Otherwise, Snapshot installation is in progress. @@ -350,8 +358,9 @@ class SnapshotInstallationHandler { LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); + return future.thenApply(dummy -> reply); } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
