databendlabs / openraft

rust raft with improvements
Apache License 2.0
1.41k stars 158 forks source link

Bug in I/O sequence handling in initialization #1252

Closed schreter closed 1 month ago

schreter commented 1 month ago

We observe the following situation in our project:

It seems like the I/O sequence handling is buggy in presence of vote storing, which is executed inline during AppendEntries is still running. Probably the same will happen also later in similar situation upon I/O hiccup (this was at startup).

I attached a minimized trace from openraft showing the issue: log.txt.

The question is, do we do something wrong, or is the I/O sequence handling in openraft broken for asynchronously-completing calls? If the latter, how to fix it?

It is openraft at master~1.

Thanks.

github-actions[bot] commented 1 month ago

đź‘‹ Thanks for opening this issue!

Get help or engage by:

drmingdrmer commented 1 month ago

It appears that the AppendEntries operation is submitted to the RaftLogStorage before the SaveVote operation, yet SaveVote completes first. This behavior suggests a potential issue with operation ordering.

The Notification channel in RaftCore uses non-async methods for communication. This implies that in your RaftLogStorage implementation, save_vote() likely returns before the preceding append() operation has a chance to send its callback (note that append() uses a callback mechanism to notify Openraft).

IO order is crucial:

It's crucial to maintain the correct order of write operations to ensure system correctness. As stated in the Openraft documentation:

https://github.com/databendlabs/openraft/blob/e54a1fd4b981eec389af2a0dfb4541206d6190cc/openraft/src/storage/v2/raft_log_storage.rs#L22-L24

Furthermore, the completion of write methods (such as save_vote() returning or the callback for append() being invoked) should also occur in the same order as the operations were initiated. While not strictly required—since IO-ids are monotonically increasing and ignoring a smaller IO-id is technically possible—enforcing this ordering helps identify potential IO reordering issues.

IO events in the log:

The RAFT_event log entries mark significant events in Openraft. The IOFlushed events indicate the completion of these two commands. Here's the relevant log output:

cat log.txt | grep RAFT_event.* -o | cat:

RAFT_event id=NID[3]  input: Initialize: {NodeIndex(1): NodeId[127.0.0.1:9990], NodeIndex(2): NodeId[127.0.0.1:9991], NodeIndex(3): NodeId[127.0.0.1:9992]}
RAFT_event id=NID[3]    cmd: AppendInputEntries: vote: <T0-NNone:Q>, entries: [0.0: {voters:[{NID[1]:NodeId[127.0.0.1:9990],NID[2]:NodeId[127.0.0.1:9991],NID[3]:NodeId[127.0.0.1:9992]}], learners:[]}]
RAFT_event id=NID[3]    cmd: SaveVote: <T1-NNID[3]:->
RAFT_event id=NID[3]    cmd: SendVote: {vote:<T1-NNID[3]:->, last_log:0.0}
RAFT_event id=NID[3] notify: IOFlushed: (<T1-NNID[3]:->)
RAFT_event id=NID[3] notify: VoteResponse: from target=NID[3], to sender_vote: <T1-NNID[3]:->, {<T1-NNID[3]:->, last_log:None}
RAFT_event id=NID[3] notify: IOFlushed: (by:<T0-NNone:Q>, 0.0)

This log sequence suggests that the SaveVote operation completed (as indicated by its IOFlushed event) before the AppendEntries operation, despite being initiated later.

schreter commented 1 month ago

The problem is that during initialization, the log append is initiated and completes asynchronously and vote is awaited inline. While awaiting the vote, the callback completes, but it seems to be too late - it wakes up the channel, which ultimately receives the completion, but you can't count on it when the task will continue to run.

FYI: Flushing the log (and thus sending the I/O completion event) before writing vote didn't help. Finally, we added an explicit await for I/O of log entry 0 into log append to work around the problem.

drmingdrmer commented 1 month ago

While awaiting the vote, the callback completes, but it seems to be too late

The callback for append() should be invoked before save_vote() completes its awaiting process. This callback immediately sends a message to the Notification channel. Subsequently, when save_vote() returns (after the callback has been called), the RaftCore puts another message into the Notification channel(notification for save-vote). Consequently, these two notifications should be received in the order they were submitted.

FYI: Flushing the log (and thus sending the I/O completion event) before writing vote didn't help.

Not only flushing the log first, but the callback for append() must be called first, before save_vote() returns.

schreter commented 1 month ago

In our project, log I/O and vote I/O (and some others) are decoupled. I was under the impression that vote I/O is independent, so it was my mistake.

Not only flushing the log first, but the callback for append() must be called first, before save_vote() returns.

Our save_vote() now hard-awaits the actual log I/O before starting writing the vote to the persistent storage. The callback is called from the I/O completion handler in unspecified order compared to the await of I/O completing in the save_vote() task. Should the vote I/O complete immediately (which is impossible, since we use asynchronous direct I/O), then there is no specific ordering of I/O completion messages.

Therefore, I think that our workaround of awaiting log I/O before writing the vote is actually the solution (and I added yield_now() before starting the vote I/O in order to ensure that the I/O starts only after the callback was definitely called).

I need to test whether it's sufficient also for log entry 0.0, which is scheduled somewhat differently in the openraft initialization code. For now, we have a hard I/O await and immediate direct callback in side of append() to ensure that the I/O order is guaranteed. But, in fact, the save_vote() awaiting any previous log I/O must be sufficient.

drmingdrmer commented 1 month ago

If all I/O operations submitted to RaftLogStorage are executed and responded to (via callback or return) in the same order as they are submitted, then no issues will arise.

I need to test whether it's sufficient also for log entry 0.0, which is scheduled somewhat differently in the openraft initialization code.

The IO operations during initialize() have nothing different from other IO, IMHO. What exactly do you mean about the differences?

schreter commented 1 month ago

The IO operations during initialize() have nothing different from other IO, IMHO. What exactly do you mean about the differences?

You are right. I misread the code.

What I meant is that Engine::initialize() calls FollowingHandler::do_append_entries() and then elect() directly, but these both only push commands to the queue (and queue processor will process them).

So the bug is then indeed only on our side. I'm closing this issue and will document it on our side so the next colleague won't run into the problem again.

Thanks for the help and sorry for the misunderstanding.