-
Notifications
You must be signed in to change notification settings - Fork 426
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-2245. Ratis should wait for all apply transaction futures before taking snapshot and group remove #1218
base: master
Are you sure you want to change the base?
Conversation
…e taking snapshot and group remove
ed7c066
to
858aa1c
Compare
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.
@swamirishi , thanks a lot for working on this!
Please see the comments inlined.
@@ -263,7 +261,7 @@ private MemoizedSupplier<List<CompletableFuture<Message>>> applyLog() throws Raf | |||
final long incremented = appliedIndex.incrementAndGet(debugIndexChange); | |||
Preconditions.assertTrue(incremented == nextIndex); | |||
if (f != null) { | |||
futures.get().add(f); | |||
applyLogFutures = applyLogFutures.thenCombine(f, (previous, current) -> previous); |
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.
It should return current
, i.e. (previous, current) -> current
.
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.
current is a Message class returned and previous is null value Void
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 the Message returned after apply transactionLog
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.
You are right. Then, let's return null, i.e. (v, message) -> null
.
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.
@swamirishi , any update for this?
ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
Outdated
Show resolved
Hide resolved
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.
@swamirishi , thanks for the update! Please see the comments inlined.
@@ -263,7 +261,7 @@ private MemoizedSupplier<List<CompletableFuture<Message>>> applyLog() throws Raf | |||
final long incremented = appliedIndex.incrementAndGet(debugIndexChange); | |||
Preconditions.assertTrue(incremented == nextIndex); | |||
if (f != null) { | |||
futures.get().add(f); | |||
applyLogFutures = applyLogFutures.thenCombine(f, (previous, current) -> previous); |
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.
@swamirishi , any update for this?
ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
Show resolved
Hide resolved
@@ -190,11 +189,10 @@ public void run() { | |||
reload(); | |||
} | |||
|
|||
final MemoizedSupplier<List<CompletableFuture<Message>>> futures = applyLog(); | |||
checkAndTakeSnapshot(futures); | |||
applyLogFutures = applyLog(applyLogFutures); |
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 check if applyLogFutures.isCompletedExceptionally()
before applying logs.
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.
But in this case we have already submitted next set of tasks right? What would be the point of checking the exception?
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.
The point is to fail fast. Otherwise, it will keep applying log even if the state machine has failed.
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.
The statemachine can take a call right if it has failed. From what I understand statemachine should receive all the transactions, and it is on the statemachine to have the guardrail if it should apply a transaction or not.
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.
If we want to fail fast we should fail the group remove as well.
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.
@swamirishi I have few concern about implementation, please refer comment. Do we need this impact only in stop() case or all case? ....
for(; state != State.STOP; ) { | ||
try { | ||
waitForCommit(); | ||
waitForCommit(applyLogFutures); |
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.
Trying to understand changes and impact,
- waitForCoimmit(applyLogFutures) --> call takeSnapshot(applyLogFutures) --> future.get() ==> No Ops
- applyLog(applyLogFutures): can add some future object to be wait
- checkAndTakeSnapshot(applyLogFutures) : This is moved before STOP(), and does future.get() by internal takeSnapshot()
- In Stop() check, just do future.get() for waiting task completion which was not there earlier
Q1: Step 3 changes checkAndTakeSnapshot() moving out of stop() and force checking to take snapshot do have performance impact?
Q2: Let No snapshot to be taken, in this case, future.get() is never called, do this is intended ?
- wait on future is called if need take snapshot in checkAndTakeSnapshot()
- wait on future is called if stop()
- else its not called
Q3: We can refactor code as above behavior, that,
- applyLog() can return existing future set
- check for future.get() before stop()
- move checkAndTakeSnapshot() out before stop()
This have similar impact instead of passing future deep inside takeSnapshot().
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.
ratis/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
Line 196 in e25ba08
applyLogFutures.get(); |
Future.get() will be called inconsequential of whether the snapshot is taken or not.
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 need to do a future.get() just before taking a snapshot. We cannot be having transactions still being applied in the background which could cause some inconsistency. It makes more sense to wait just before taking a snapshot that is why I moved the future.get() inside takeSnapshot method
ratis/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
Line 286 in e25ba08
applyLogFutures.get(); |
final MemoizedSupplier<List<CompletableFuture<Message>>> futures = applyLog(); | ||
checkAndTakeSnapshot(futures); | ||
applyLogFutures = applyLog(applyLogFutures); | ||
checkAndTakeSnapshot(applyLogFutures); |
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.
In one of case where no snpashot to be taken and no stop(), future.get() will not be called. Do this will fix the issue as expected for applyTransaction to finish ? I think no, it should wait for all case.
And related to performance impact if always need wait, do this call is always in stop() flow or all flow ? this part is not clear.
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.
as discussed, this is not a problem.
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.
LGTM
@szetszwo there are some flaky tests failure here, can you please retrigger the CI here |
try { | ||
applyLogFutures.get(); | ||
} catch (InterruptedException | ExecutionException e) { | ||
LOG.info("{}: interrupted while waiting for apply transactions", this, t); | ||
} |
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.
@swamirishi , Why calling applyLogFutures.get()
when there is already an exception/error? It could cause problems such as:
- The future may never complete.
- The thread may already be interrupted.
- In may throw another runtime exception/error (such as OOM)
The test failures could be caused by this 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.
I have updated the catch block to not wait when the exception is ExecutionException & InterruptedException. We should wait for apply transactions in all other cases
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.
@swamirishi , Why calling
applyLogFutures.get()
when there is already an exception/error? It could cause problems such as:
- The future may never complete.
It is guaranteed it will always complete right?
- The thread may already be interrupted.
- In may throw another runtime exception/error (such as OOM)
It won't be thrown by the future right? If the future throws exception we anyways handle it and mark the future as completed.
https://github.com/apache/ratis/blob/b635b137329f011e715c9083e93c7bb7f22b221f/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java#L271C11-L274C25
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.
It is guaranteed it will always complete right?
@swamirishi , Who will guarantee that? Ozone? Any doc to support this statement? It seems impossible in an erroneous condition.
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.
It won't be thrown by the future right? ...
It definitely could throw OOM.
The question is why?
How about (1) and (3) above? |
What changes were proposed in this pull request?
On Ratis Snapshot and group removal the statemachine just waits for apply transactions that have been applied on a single iteration. If there are no more transactions added onto the state machine and all of the apply transaction future are still in progress. The state machine ends up not waiting for the updater thread and ends up calling the notifyGroupRemove function and deletes the raft group directory. So this could lead to some node not being able to apply some of the transactions still in flight in case of a restart.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2245
How was this patch tested?
Adding unit tests.