4646
4747import java .io .IOException ;
4848import java .util .Optional ;
49+ import java .util .concurrent .CompletableFuture ;
4950import java .util .concurrent .atomic .AtomicBoolean ;
5051import java .util .concurrent .atomic .AtomicInteger ;
5152import java .util .concurrent .atomic .AtomicLong ;
@@ -122,12 +123,12 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt
122123 if (installSnapshotEnabled ) {
123124 // Leader has sent InstallSnapshot request with SnapshotInfo. Install the snapshot.
124125 if (request .hasSnapshotChunk ()) {
125- reply = checkAndInstallSnapshot (request , leaderId );
126+ reply = checkAndInstallSnapshot (request , leaderId ). join () ;
126127 }
127128 } else {
128129 // Leader has only sent a notification to install snapshot. Inform State Machine to install snapshot.
129130 if (request .hasNotification ()) {
130- reply = notifyStateMachineToInstallSnapshot (request , leaderId );
131+ reply = notifyStateMachineToInstallSnapshot (request , leaderId ). join () ;
131132 }
132133 }
133134
@@ -156,21 +157,22 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt
156157 return failedReply ;
157158 }
158159
159- private InstallSnapshotReplyProto checkAndInstallSnapshot (InstallSnapshotRequestProto request ,
160+ private CompletableFuture < InstallSnapshotReplyProto > checkAndInstallSnapshot (InstallSnapshotRequestProto request ,
160161 RaftPeerId leaderId ) throws IOException {
161162 final long currentTerm ;
162163 final long leaderTerm = request .getLeaderTerm ();
163164 final InstallSnapshotRequestProto .SnapshotChunkProto snapshotChunkRequest = request .getSnapshotChunk ();
164165 final TermIndex lastIncluded = TermIndex .valueOf (snapshotChunkRequest .getTermIndex ());
165166 final long lastIncludedIndex = lastIncluded .getIndex ();
167+ final CompletableFuture <Void > future ;
166168 synchronized (server ) {
167169 final boolean recognized = state .recognizeLeader (RaftServerProtocol .Op .INSTALL_SNAPSHOT , leaderId , leaderTerm );
168170 currentTerm = state .getCurrentTerm ();
169171 if (!recognized ) {
170- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
171- currentTerm , snapshotChunkRequest .getRequestIndex (), InstallSnapshotResult .NOT_LEADER );
172+ return CompletableFuture . completedFuture ( toInstallSnapshotReplyProto (leaderId , getMemberId (),
173+ currentTerm , snapshotChunkRequest .getRequestIndex (), InstallSnapshotResult .NOT_LEADER )) ;
172174 }
173- server .changeToFollowerAndPersistMetadata (leaderTerm , true , "installSnapshot" );
175+ future = server .changeToFollowerAndPersistMetadata (leaderTerm , true , "installSnapshot" );
174176 state .setLeader (leaderId , "installSnapshot" );
175177
176178 server .updateLastRpcTime (FollowerState .UpdateType .INSTALL_SNAPSHOT_START );
@@ -186,8 +188,9 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest
186188 // have a lot of requests
187189 if (state .getLog ().getLastCommittedIndex () >= lastIncludedIndex ) {
188190 nextChunkIndex .set (snapshotChunkRequest .getRequestIndex () + 1 );
189- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
191+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (),
190192 currentTerm , snapshotChunkRequest .getRequestIndex (), InstallSnapshotResult .ALREADY_INSTALLED );
193+ return future .thenApply (dummy -> reply );
191194 }
192195
193196 //TODO: We should only update State with installed snapshot once the request is done.
@@ -210,25 +213,27 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest
210213 if (snapshotChunkRequest .getDone ()) {
211214 LOG .info ("{}: successfully install the entire snapshot-{}" , getMemberId (), lastIncludedIndex );
212215 }
213- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
216+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (),
214217 currentTerm , snapshotChunkRequest .getRequestIndex (), InstallSnapshotResult .SUCCESS );
218+ return future .thenApply (dummy -> reply );
215219 }
216220
217- private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot (
221+ private CompletableFuture < InstallSnapshotReplyProto > notifyStateMachineToInstallSnapshot (
218222 InstallSnapshotRequestProto request , RaftPeerId leaderId ) throws IOException {
219223 final long currentTerm ;
220224 final long leaderTerm = request .getLeaderTerm ();
221225 final TermIndex firstAvailableLogTermIndex = TermIndex .valueOf (
222226 request .getNotification ().getFirstAvailableTermIndex ());
223227 final long firstAvailableLogIndex = firstAvailableLogTermIndex .getIndex ();
228+ final CompletableFuture <Void > future ;
224229 synchronized (server ) {
225230 final boolean recognized = state .recognizeLeader ("notifyInstallSnapshot" , leaderId , leaderTerm );
226231 currentTerm = state .getCurrentTerm ();
227232 if (!recognized ) {
228- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
229- currentTerm , InstallSnapshotResult .NOT_LEADER );
233+ return CompletableFuture . completedFuture ( toInstallSnapshotReplyProto (leaderId , getMemberId (),
234+ currentTerm , InstallSnapshotResult .NOT_LEADER )) ;
230235 }
231- server .changeToFollowerAndPersistMetadata (leaderTerm , true , "installSnapshot" );
236+ future = server .changeToFollowerAndPersistMetadata (leaderTerm , true , "installSnapshot" );
232237 state .setLeader (leaderId , "installSnapshot" );
233238 server .updateLastRpcTime (FollowerState .UpdateType .INSTALL_SNAPSHOT_NOTIFICATION );
234239
@@ -245,8 +250,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot(
245250 inProgressInstallSnapshotIndex .compareAndSet (firstAvailableLogIndex , INVALID_LOG_INDEX );
246251 LOG .info ("{}: InstallSnapshot notification result: {}, current snapshot index: {}" , getMemberId (),
247252 InstallSnapshotResult .ALREADY_INSTALLED , snapshotIndex );
248- return toInstallSnapshotReplyProto (leaderId , getMemberId (), currentTerm ,
253+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (), currentTerm ,
249254 InstallSnapshotResult .ALREADY_INSTALLED , snapshotIndex );
255+ return future .thenApply (dummy -> reply );
250256 }
251257
252258 final RaftPeerProto leaderProto ;
@@ -323,8 +329,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot(
323329 inProgressInstallSnapshotIndex .set (INVALID_LOG_INDEX );
324330 server .getStateMachine ().event ().notifySnapshotInstalled (
325331 InstallSnapshotResult .SNAPSHOT_UNAVAILABLE , INVALID_LOG_INDEX , server .getPeer ());
326- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
332+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (),
327333 currentTerm , InstallSnapshotResult .SNAPSHOT_UNAVAILABLE );
334+ return future .thenApply (dummy -> reply );
328335 }
329336
330337 // If a snapshot has been installed, return SNAPSHOT_INSTALLED with the installed snapshot index and reset
@@ -341,17 +348,19 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot(
341348 server .getStateMachine ().event ().notifySnapshotInstalled (
342349 InstallSnapshotResult .SNAPSHOT_INSTALLED , latestInstalledIndex , server .getPeer ());
343350 installedIndex .set (latestInstalledIndex );
344- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
351+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (),
345352 currentTerm , InstallSnapshotResult .SNAPSHOT_INSTALLED , latestInstalledSnapshotTermIndex .getIndex ());
353+ return future .thenApply (dummy -> reply );
346354 }
347355
348356 // Otherwise, Snapshot installation is in progress.
349357 if (LOG .isDebugEnabled ()) {
350358 LOG .debug ("{}: InstallSnapshot notification result: {}" , getMemberId (),
351359 InstallSnapshotResult .IN_PROGRESS );
352360 }
353- return toInstallSnapshotReplyProto (leaderId , getMemberId (),
361+ final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto (leaderId , getMemberId (),
354362 currentTerm , InstallSnapshotResult .IN_PROGRESS );
363+ return future .thenApply (dummy -> reply );
355364 }
356365 }
357366
0 commit comments