-
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
Changes from 18 commits
2b0ce8f
d4e072c
0b5df03
4c6719f
60a0c3d
139be1e
a7d8a9c
5a228b4
bb4520d
9123b1e
309dd5c
f28665a
62e74b6
ea7ff59
3f65d6f
091bce9
3363969
8eb3dba
756a305
d0f00ae
421583d
f8a72fc
9a50ce8
3d051e0
a445d49
d0cd3de
da1cc16
f9b74f6
a7264ef
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -46,6 +46,7 @@ | |
|
||
import java.io.IOException; | ||
import java.util.Optional; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
@@ -122,12 +123,12 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt | |
if (installSnapshotEnabled) { | ||
// Leader has sent InstallSnapshot request with SnapshotInfo. Install the snapshot. | ||
if (request.hasSnapshotChunk()) { | ||
reply = checkAndInstallSnapshot(request, leaderId); | ||
reply = checkAndInstallSnapshot(request, leaderId).join(); | ||
} | ||
} else { | ||
// Leader has only sent a notification to install snapshot. Inform State Machine to install snapshot. | ||
if (request.hasNotification()) { | ||
reply = notifyStateMachineToInstallSnapshot(request, leaderId); | ||
reply = notifyStateMachineToInstallSnapshot(request, leaderId).join(); | ||
} | ||
} | ||
|
||
|
@@ -156,21 +157,23 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt | |
return failedReply; | ||
} | ||
|
||
private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequestProto request, | ||
private CompletableFuture<InstallSnapshotReplyProto> checkAndInstallSnapshot(InstallSnapshotRequestProto request, | ||
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; | ||
synchronized (server) { | ||
final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); | ||
currentTerm = state.getCurrentTerm(); | ||
if (!recognized) { | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); | ||
return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER)); | ||
} | ||
server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); | ||
future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); | ||
state.setLeader(leaderId, "installSnapshot"); | ||
|
||
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); | ||
|
@@ -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 commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. Ok |
||
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.ALREADY_INSTALLED); | ||
return future.thenApply(dummy -> reply); | ||
} | ||
|
||
//TODO: We should only update State with installed snapshot once the request is done. | ||
|
@@ -210,25 +214,27 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest | |
if (snapshotChunkRequest.getDone()) { | ||
LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); | ||
} | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); | ||
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 commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. Ok |
||
} | ||
|
||
private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( | ||
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; | ||
final InstallSnapshotReplyProto replyProto; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. Ok |
||
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 +251,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( | |
inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); | ||
LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), | ||
InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, | ||
replyProto = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, | ||
InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); | ||
return future.thenApply(dummy -> replyProto); | ||
} | ||
|
||
final RaftPeerProto leaderProto; | ||
|
@@ -323,8 +330,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( | |
inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); | ||
server.getStateMachine().event().notifySnapshotInstalled( | ||
InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
replyProto = toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); | ||
return future.thenApply(dummy -> replyProto); | ||
} | ||
|
||
// If a snapshot has been installed, return SNAPSHOT_INSTALLED with the installed snapshot index and reset | ||
|
@@ -341,21 +349,23 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( | |
server.getStateMachine().event().notifySnapshotInstalled( | ||
InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); | ||
installedIndex.set(latestInstalledIndex); | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
replyProto = toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); | ||
return future.thenApply(dummy -> replyProto); | ||
} | ||
|
||
// Otherwise, Snapshot installation is in progress. | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), | ||
InstallSnapshotResult.IN_PROGRESS); | ||
} | ||
return toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
replyProto = toInstallSnapshotReplyProto(leaderId, getMemberId(), | ||
currentTerm, InstallSnapshotResult.IN_PROGRESS); | ||
return future.thenApply(dummy -> replyProto); | ||
} | ||
} | ||
|
||
private RoleInfoProto getRoleInfoProto(RaftPeer leader) { | ||
private RoleInfoProto getRoleInfoProto (RaftPeer leader){ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. Ok |
||
final RoleInfo role = server.getRole(); | ||
final Optional<FollowerState> fs = role.getFollowerState(); | ||
final ServerRpcProto leaderInfo = toServerRpcProto(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.
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