-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
raft: support lazy appends #125002
raft: support lazy appends #125002
Conversation
Epic: none Release note: none
Epic: none Release note: none
It looks like your PR touches production code but doesn't add or edit any test code. Did you consider adding tests to your PR? 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
This commit introduces the "lazy appends" API in raft RawNode. In this mode, all MsgApp messages with a non-empty Entries slice in StateReplicate are sent using the RawNode.SendAppend method. This gives the caller (typically, the one handling Ready processing) direct control of when the raft node accesses Storage and sends replication messages. The API will be used by Replication Admission Control, which ultimately helps solving follower overload issues. Previously, all MsgApp messages would be sent eagerly as soon as the node sees that a follower is behind. Any message Step-ped into raft could cause an immediate read of entries from Storage and a message construction. Epic: CRDB-37515 Release note: None
TODO: add tests, though the demo in #124948 already provides good coverage. |
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'm a bit confused.
First one is perhaps a nit. The RawNode.SendAppend
method is not defined in this PR. It seems to be in the other PR. Shouldn't it be defined here given the commit comment?
Second, is the following meant to be an intermediate step in the refactor?
// SendAppend sends a log replication (MsgApp) message to a particular node. The
// message will be available via next Ready.
Asking since this does not conform to the RaftInterface
in the RACv2 prototype. In there we get a Ready
with entries, and based on the sizes of the entries and availability of tokens pull within the same handleRaftReadyRaftMuLocked
. Having the entries first is important since we need to figure out the priorities and size. And we also don't want to delay the sending until the next Ready
. I guess we could call Ready
a second time within handleRaftReadyRaftMuLocked
but this again is slightly convoluted (akin to our discussion in https://cockroachlabs.slack.com/archives/C06UFBJ743F/p1717185326307039?thread_ts=1717076603.877549&cid=C06UFBJ743F, though not as convoluted) in that we shouldn't have to tell RawNode what we want, and then call it again with no parameter for it to give us what we want -- the first call should simply return what we want.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @kvoli)
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.
RawNode.SendApped
is in this PR, see the rawnode.go
file. Not sure why you can't see it, maybe Reviewable is confusing things.
This is not the final version of the method, see TODOs in it. The main goal is to add the plumbing first, and then we will add the required parameters as we go, as well as the state tracking outside raft. As of today, there are no extra parameters needed to drive replication using this method (see the other demo PR). Also, this doesn't introduce additional code for handling messages: instead of returning MsgApp
, all the messages are in Ready
- this can be changed in the future but is not needed now for simplicity.
In the future, the approximate flow is:
- The upper layer learns
Next
and entry sizes continuously (not necessarily throughReady
). At any point in time (when we holdraftMu
), we can access theraftLog / unstable
and learn the last index, theNext
index, and we know all the entry sizes inunstable
(we remember them either when we append them, or we can also scanraftLog/unstable
at any time if needed). - So, at any point in time we're able to call
SendAppend
if the tracking state indicates the stream is ready. Typically we will do it inhandleRaftReadyRaftMuLocked
, right before callingRawNode.Ready
.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @arulajmani and @kvoli)
Superseded by #131588. |
This PR introduces the "lazy appends" API in raft
RawNode
. In this mode, allMsgApp
messages with a non-emptyEntries
slice inStateReplicate
are sent using theRawNode.SendAppend
method. This gives the caller (typically, the one handlingReady
processing) direct control of when the raft node accessesStorage
and sends replication messages. The API will be used by Replication Admission Control, which ultimately helps solving follower overload issues.Previously, all
MsgApp
messages would be sent eagerly as soon as the node sees that a follower is behind. Any messageStep
-ped into raft could cause an immediate read of entries fromStorage
and a message construction.The new behaviour is disabled by default, and hidden by
Config.EnableLazyAppends
flag. In the future, it will be the default.PR #124948 demonstrates the usage of this API, and its effect can be seen in the
testdata
traces of the data-driven tests: messages typically contain more entries, since their construction is delayed untilReady
processing.Epic: CRDB-37515