ratis icon indicating copy to clipboard operation
ratis copied to clipboard

RATIS-1644. Provide a safe async flush.

Open ChenSammi opened this issue 3 years ago • 11 comments

https://issues.apache.org/jira/browse/RATIS-1644

ChenSammi avatar Jul 28 '22 07:07 ChenSammi

  • We should support unsafe flush since the existing unsafe flush still has a better performance than async flush. @SincereXIA , do you need unsafe flush if we have the async flush provided here?

Thanks @szetszwo @ChenSammi. Yes we'd better keep unsafe flush because of the better performance. We can add a separate safe async flush here.

captainzmc avatar Jul 29 '22 03:07 captainzmc

Thanks @szetszwo and @captainzmc for the feedback.

ChenSammi avatar Aug 01 '22 03:08 ChenSammi

@ChenSammi , thanks for working on this. Async flush is a very good idea! Some thoughts

* We should support unsafe flush since the existing unsafe flush still has a better performance than async flush.  @SincereXIA , do you need unsafe flush if we have the async flush provided here?

* Async flush needs to honor stateMachineDataPolicy; otherwise, it is unsafe.

@szetszwo The stateMachineDataPolicy is controlled by raft.server.log.statemachine.data.sync. When it's true, state machine will be flushed before it reaches raft log write and flush. When it's false, the current sync flush is unsafe too.
Since we have defined the raft.server.log.statemachine.data.sync to control the state machine data flush, we can just focus on the write log flush with .async-flush.enabled property. Otherwise, maybe we should merge this two property into one to guarantee a safe flush of both state machine data and log data with sync and async two modes. What do you think?

ChenSammi avatar Aug 01 '22 09:08 ChenSammi

@ChenSammi, yes, when raft.server.log.statemachine.data.sync is false, we don't have to do anything.

When both raft.server.log.statemachine.data.sync and the new Async flush are true, we should wait for both operations to complete before updating the index.

szetszwo avatar Aug 01 '22 16:08 szetszwo

@szetszwo , Right. currently when raft.server.log.statemachine.data.sync is true, state machine data will be waited to be synced before ratis moves on to write the corresponding raft log and flush. So we do not handle any state machine related case in flushIfNecessary(), right? Let me know if I misunderstand something.

ChenSammi avatar Aug 02 '22 01:08 ChenSammi

@szetszwo , the difference is it sticks to call updateIncreasingly on flushIndex in your proposal while updateToMax is called currently. Is there any known side effect of using updateToMax? I'm thinking of using PriorityBlockingList in flushExecutor in another PR to improve the performance by executing the raft flush with high commitIndex in priority, and remove all lower commitIndex flush tasks, so that we will have less actual file flush. In this case, flushIndex will not be update continuously.

ChenSammi avatar Aug 04 '22 04:08 ChenSammi

@ChenSammi , updateIncreasingly is more strict than updateToMax so that it is preferred. For flush index, it should be updated increasingly. Do you agree?

szetszwo avatar Aug 04 '22 06:08 szetszwo

@szetszwo IMHO, the current updateIncreasingly is less strict than updateToMax. updateToMax will atomically check if the new value is greater than the old value before update the index while updateIncreasingly doesn the check after the value change in a way which has no atomic guarantee.

This async flush just put the flush aside, the number of flush is not reduced. In a heavy workload cluster, every flush took up to seconds to finish. My plan is first make it async, then reduce the actual flush count by reverse order executing, flush tasks with high commitIndex got executed first.

ChenSammi avatar Aug 04 '22 07:08 ChenSammi

... the current updateIncreasingly is less strict than updateToMax. updateToMax will atomically check if the new value is greater than the old value before update the index while updateIncreasingly doesn the check after the value change in a way which has no atomic guarantee.

Yes, updateIncreasingly does the check after the value change and it will throw an exception if the check failed. The check is an assertion in order to make sure that the condition always holds. If it fails, it is a bug in the code.

Since updateIncreasingly only allows the new value >= the old value while updateToMax allows any value, we say updateIncreasingly more strict than updateToMax in this sense.

szetszwo avatar Aug 04 '22 16:08 szetszwo

@szetszwo thanks for the explain. It makes sense. Please take a look of the new PR at your convenient time, thanks.

ChenSammi avatar Aug 08 '22 10:08 ChenSammi

@ChenSammi , there is a test failure which can be reproduced. Could you take a look? https://github.com/apache/ratis/runs/7793088310?check_suite_focus=true#step:5:625

szetszwo avatar Aug 11 '22 18:08 szetszwo

TestRaftSnapshotWithNetty failure seems irrelevant. Anyway, it's fixed.

ChenSammi avatar Aug 16 '22 13:08 ChenSammi

Thanks @szetszwo for the code review.

ChenSammi avatar Aug 17 '22 01:08 ChenSammi