-
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
kvserver: update TruncatedState before writing #131063
base: master
Are you sure you want to change the base?
kvserver: update TruncatedState before writing #131063
Conversation
Update the Replica's in-memory TruncatedState before applying the write batch to storage. Readers of the raft log storage who synchronize with it via Replica.mu, and read TruncatedState, will then expect to find entries at indices > TruncatedState.Index in the log. If we write the batch first, and only then update TruncatedState, there is a time window during which the log storage appears to have a gap. Epic: none Release note: none
I feel much more enlightened by our conversation than am by the commit message here. Below is my attempt at capturing the motivation and impact of this PR more fully, feel free to use pieces of it for an updated commit message as you deem appropriate. Log truncations delete a prefix of the raft log. Currently, we commit the write batch that contains the truncation to pebble (under raftMu) prior to updating the in-memory metadata about the "beginning" of the log, the |
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.
Question about locking, but generally looks good!
// first, and only then update TruncatedState, there is a time window | ||
// during which the log storage appears to have a gap. | ||
isDeltaTrusted := res.RaftExpectedFirstIndex != 0 | ||
b.r.setTruncatedStateMuLocked(res.State.TruncatedState, |
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 I don't think we hold replicaMu here?
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.
Yeah, we don't. It's locked inside the method though, just need to remove MuLocked
from the name.
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 are going to grab LogSnapshot
while holding raftMu
(and holding Replica.mu
) and will continue holding raftMu
until we use the LogSnapshot
. So it's unclear to me why this change is necessary -- if this change were straightforward, this would definitely make sense, but given errors in committing the truncation batch, it doesn't seem so.
Reviewed 5 of 7 files at r1, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @pav-kv, and @tbg)
pkg/kv/kvserver/raft_log_truncator.go
line 593 at r1 (raw file):
r.setTruncatedState(trunc.RaftTruncatedState, trunc.expectedFirstIndex, trunc.isDeltaTrusted) r.setTruncationDelta(trunc.logDeltaBytes) })
What if the batch.Commit
fails? Now we think the log is truncated when it is not.
pkg/kv/kvserver/raft_log_truncator.go
line 608 at r1 (raw file):
return } r.applySideEffects(ctx, &trunc.RaftTruncatedState)
Here we are passing a *kvserverpb.RaftTruncatedState
and in setTruncatedState
we pass a kvserverpb.RaftTruncatedState
. Why this inconsistency?
We used to always a pointer before.
pkg/kv/kvserver/replica_application_result.go
line 506 at r1 (raw file):
func (r *Replica) handleTruncatedStateResult( ctx context.Context, t *kvserverpb.RaftTruncatedState, ) (raftLogDelta int64) {
why do we delay clearing the cache entries if we've already updated Replica.mu.state.TruncatedState
?
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 are going to grab
LogSnapshot
while holdingraftMu
(and holdingReplica.mu
) and will continue holdingraftMu
until we use theLogSnapshot
. So it's unclear to me why this change is necessary
"We" being RACv2 - yes. There is still a class of log storage reads done while only holding Replica.mu
, 2 linked from the PR description. Since they don't lock raftMu
before mu
, they can load an outdated TruncatedState
and observe a gap in the log.
This race is the only case that can cause ErrCompacted errors in raft codebase which is also exposed via the API etc/etc. I don't think it's worth keeping this for the sake of one race condition / quirk.
The way truncations are done is also inconsistent with how snapshots are handled (which are special kinds of truncations writing both to "log storage" and "state machine storage"). With snapshots, raft knows first, and registers an "intent" in memory not to try reading below this index (see unstable.snapshot
field and unstable.maybeFirstIndex
method, correspondingly). Only then the snapshot and log wipe are written/synced, and acked back to raft.
Log truncations invert this order: first the truncation is enacted, and only then TruncatedState
is updated (which is an equivalent of notifying raft with an intent). When we get closer to separate-raft-log project again, this discrepancy will become more pressing, so I thought it's not worth waiting and fixed it.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @tbg)
pkg/kv/kvserver/raft_log_truncator.go
line 593 at r1 (raw file):
Previously, sumeerbhola wrote…
What if the
batch.Commit
fails? Now we think the log is truncated when it is not.
In the synchronous truncations flow applies can't fail. If they do, we would panic. In the decoupled truncations flow this seems more liberal, so a valid question.
What can be a reason of a fail here? Any legitimate ones?
I think: if a log index is already planned to be truncated at, it doesn't matter if the storage write fails. Logically, the prefix is already unused. If updating TruncatedState
seems risky, we should make raft aware of truncations (it's a matter of adding one int to the unstable
struct), and notify raft about this intent first thing.
pkg/kv/kvserver/raft_log_truncator.go
line 608 at r1 (raw file):
Previously, sumeerbhola wrote…
Here we are passing a
*kvserverpb.RaftTruncatedState
and insetTruncatedState
we pass akvserverpb.RaftTruncatedState
. Why this inconsistency?
We used to always a pointer before.
No reason, will fix.
pkg/kv/kvserver/replica_application_result.go
line 506 at r1 (raw file):
Previously, sumeerbhola wrote…
why do we delay clearing the cache entries if we've already updated
Replica.mu.state.TruncatedState
?
Yeah, seems movable a bit up the stack, I'll consider. Though not critical.
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.
There is still a class of log storage reads done while only holding
Replica.mu
, 2 linked from the PR description.
thanks for the pointer.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @pav-kv, and @tbg)
pkg/kv/kvserver/raft_log_truncator.go
line 593 at r1 (raw file):
What can be a reason of a fail here? Any legitimate ones?
I couldn't think of one, and neither could Jackson. So adding a panic here is fine, which should unblock this PR. We just don't want a situation where we keep running without the log truncated and think it is truncated.
Update the
Replica
's in-memoryTruncatedState
before applying the write batch to storage.Readers of the raft log storage who synchronize with it via
Replica.mu
, and readTruncatedState
, will then expect to find entries at indices >TruncatedState.Index
in the log, if read in the sameReplica.mu
critical section. If we write the batch first, and only then updateTruncatedState
, there is a time window during whichReplica.mu
can be acquired, under which the log storage will appear to have a gap.There are at least 2
Replica.mu
-onlyRawNode.Step
paths affected by this.Part of #132114
Related to #130955, #131041