szetszwo commented on code in PR #1168:
URL: https://github.com/apache/ratis/pull/1168#discussion_r1805770617
##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -613,24 +606,33 @@ private synchronized CompletableFuture<Void>
changeToFollowerAsync(
return future;
}
- synchronized void changeToFollowerAndPersistMetadata(
+ synchronized CompletableFuture<Void> changeToFollowerAndPersistMetadata(
long newTerm,
boolean allowListener,
Object reason) throws IOException {
- if (changeToFollower(newTerm, false, allowListener, reason)) {
- state.persistMetadata();
+ final AtomicBoolean metadataUpdated = new AtomicBoolean();
+ final CompletableFuture<Void> future = changeToFollower(newTerm, false,
allowListener, reason, metadataUpdated);
+ try {
+ if (metadataUpdated.get()) {
+ state.persistMetadata();
+ }
+ } catch (IOException e) {
+ CompletableFuture.runAsync(future::join);
Review Comment:
`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?
##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java:
##########
@@ -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;
Review Comment:
It is better to use `final CompletableFuture<Void> future;`. Then
`future.join()` will never have NPE.
--
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]