-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
storage: sideloaded Raft entries don't play well with Raft memory limits #31618
Comments
Raft should never see "thin" entries and so should correctly account for the inlined payload. We call On the other hand, the method that loads persisted entries back into memory ( Unless there's a bug that results in a
but that seems unlikely. I just caught up on the discussion on Slack that you had on this in which you posted this message:
from Raft. Can't this happen naturally without any sideloading accounting problems because it does all-or-nothing across multiple entries? Assume you can have 100 bytes of uncommitted log. The replica proposes two proposals, 70 and 40 bytes. But both get dropped because the local Raft instance doesn't know the leader. After a few ticks, the replica proposes both again. Is it possible that both proposals reliably end up in the same Hmm, looks like the answer is "no, at least not in our code". We put each proposal in individually. But this still looks good to fix (is it possible to propose multiple entries at once?) Back to the bug at hand, what are our safeguards against putting in a proposal that's simply getting dropped due to size? Wouldn't that just be reproposed and dropped, over and over again? https://github.com/cockroachdb/cockroach/blob/master/pkg/storage/replica.go#L3805-L3812 |
If this is the case, there are two choices: a) we make sure we detect this condition or prevent it (i.e. make sure these proposals are never that big) or b) we tweak semantics so that the limiting doesn't apply to the first uncommitted entry (after all, we're doing this to make sure there isn't a long log, not to make sure there isn't a single thing in it). But this seems like such an obvious bug that I'm afraid I might be missing something. |
Hmm, yeah, I'm definitely missing something. On a local 3 node cluster, I would otherwise expect to not be able to insert a 12mb blob, but that just works. Ah, I'm seeing that it already does b) from above: Ok, you must be right about there being something about sideloading. I've got to page this back in but this snippet strongly suggests that the committed entries are "thin", which would indeed confuse Raft's internal tracking: cockroach/pkg/storage/replica.go Lines 4322 to 4334 in 38b4827
|
Wait a minute... I don't see why this needs to sideload. The proposals should already have been inlined. Is this just an inefficiency that has gone overlooked? I've commented this out and all the sideloading tests still pass. |
Entries are "thinned" only when passed to `r.append()` (i.e. written to disk) and they are always returned "fat" from `Entries()` (i.e. Raft's way to get entries from disk). Consequently Raft never sees thin entries and won't ask us to commit them. Touches cockroachdb#31618. Release note: None
If this were passed as nil, we'd be returning "thin" (i.e. with sideloading payloads not inlined) Entries. This isn't supposed to happen, but check it. See: cockroachdb#31618 (comment). Release note: None
Hah, yeah, I did that too last night. I can’t see any reason why you’d need to fatten entries anywhere other than |
Finally caught a repro. Here's /debug/requests:
|
Probably not the root cause here, but looks like a buglet: If we hit the config change path then we'll have added the original size but we'll commit a no-op. |
Also another buglet: If you go into this method you see that it updates the term and index. Those are included in |
Going to try with this diff:
and @benesch's one-liner 20% chance repro:
|
Two nodes immediately folded, way before AddSSTable came around. Perhaps due to the buglets above, perhaps because my assertion is somehow flawed:
|
Ok, @benesch points out the obvious: if we leak even a single byte, the AddSSTables are going to bounce off the Raft log forever. So these buglets above are actually the real bugs. |
I'm going to switch this code over to using only |
Is that sufficient? Won't dropped conf changes still cause misaccounting? It seems like you could just move the call to |
Right, I have to do that and count only the data size (to make sure changing term and index is irrelevant). |
Ah, gotcha. SGTM. |
Gotta run this a little longer, but I've imported the first table without hitting the assertion (or getting stuck). Decent chance that this is the fix, I'd say. Gonna give it a couple runs. |
Passed perhaps a dozen times without flinching 🤷♂️ I think I can get started polishing this up for upstream. |
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
Thanks for both of you looking into this. I think you're spot on about both issues, and correct that the first one was the actual cause of the stall while the second one would harmlessly underestimate the uncommitted log size. Looks like etcd-io/etcd#10199 fixes both of the issues. I'm also happy to see #31627. |
It passed many more times. I'm going to test with the newly opened upstream PR now: Hopefully I didn't un-fix anything there. |
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
The previous code was using the proto-generated `Size()` method to track the size of an incoming proposal at the leader. This includes the Index and Term, which were mutated after the call to `Size()` when appending to the log. Additionally, it was not taking into account that an ignored configuration change would ignore the original proposal and append an empty entry instead. As a result, a fully committed Raft group could end up with a non- zero tracked uncommitted Raft log counter that would eventually hit the ceiling and drop all future proposals indiscriminately. It would also immediately imply that proposals exceeding the threshold alone would get refused (as the "first uncommitted proposal" gets special treatment and is always allowed in). Track only the size of the payload actually appended to the Raft log instead. For context, see: cockroachdb/cockroach#31618 (comment)
Entries are "thinned" only when passed to `r.append()` (i.e. written to disk) and they are always returned "fat" from `Entries()` (i.e. Raft's way to get entries from disk). Consequently Raft never sees thin entries and won't ask us to commit them. Touches cockroachdb#31618. Release note: None
If this were passed as nil, we'd be returning "thin" (i.e. with sideloading payloads not inlined) Entries. This isn't supposed to happen, but check it. See: cockroachdb#31618 (comment). Release note: None
31556: importccl: re-enable job control tests r=mjibson a=mjibson I tracked down the problem. It was that after the CANCEL JOB was issued, sometimes the 2nd stage of the IMPORT (the shuffle) would have started, and sometimes it wouldn't have. If it did not start then RunJob would block forever trying to send on the allowResponse chan. Fix this by making a draining go routine after the first block. Closes #24623 Closes #24658 Release note: None 31627: storage: remove spurious call to maybeInlineSideloadedRaftCommand r=nvanbenschoten,benesch a=tschottdorf Entries are "thinned" only when passed to `r.append()` (i.e. written to disk) and they are always returned "fat" from `Entries()` (i.e. Raft's way to get entries from disk). Consequently Raft never sees thin entries and won't ask us to commit them. Touches #31618. Release note: None 31695: bitarray: don't allow FromEncodingParts to return invalid bit array r=knz a=benesch It is invalid for a bit array's lastBitsUsed field to be greater than 64. The FromEncodingParts function, however, would happily construct an invalid bitarray if given a too-large lastBitsUsed value. Teach the FromEncodingParts to return an error instead. This presented as a panic when attempting to pretty-print a key with a bitarray whose lastBitsUsed encoded value was 65. Such a key can be created when calling PrefixEnd on a key with a bitarray whose lastBitsUsed value is 64. By returning an error instead, the pretty-printing code will try again after calling UndoPrefixEnd and be able to print the key. Fix #31115. Release note: None 31697: partitionccl: deflake TestDropIndexWithZoneConfigCCL r=danhhz,eriktrinh a=benesch A particularly adversarial goroutine schedule can cause this test to observe the moment in time where the data is dropped but the zone config is not. Deflake by retrying the check for the dropped zone config until it succeeds (or times out). Fix #31678. Release note: None Co-authored-by: Matt Jibson <[email protected]> Co-authored-by: Tobias Schottdorf <[email protected]> Co-authored-by: Nikhil Benesch <[email protected]>
Fixed in #31689. |
Entries are "thinned" only when passed to `r.append()` (i.e. written to disk) and they are always returned "fat" from `Entries()` (i.e. Raft's way to get entries from disk). Consequently Raft never sees thin entries and won't ask us to commit them. Touches cockroachdb#31618. Release note: None
If this were passed as nil, we'd be returning "thin" (i.e. with sideloading payloads not inlined) Entries. This isn't supposed to happen, but check it. See: cockroachdb#31618 (comment). Release note: None
etcd/raft
has a number of memory limits that it uses to avoid pulling too many entries into memory at once, sending messages that are too large, and applying batches of entries that are too large. These limits work by looking at the size reported byraftpb.Entry.Size
. This doesn't play well with sideloaded entries, which at any given time may or may not be inlined. Entries may flow intoetcd/raft
(proposals, MsgApps, etc.) while inlined and exit it non-inlined (MsgApps, CommittedEntries, etc.). This breaks a few safeguards:I think the fix here is to ensure that
etcd/raft
always sees inlined entries. I think that's already the case in most code-paths, but not in all. I'm sure I'm also missing a lot of context about the history of this and the decisions that have been made.cc. @tschottdorf
The text was updated successfully, but these errors were encountered: