-
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
kvserver: update TruncatedState before writing #131063
base: master
Are you sure you want to change the base?
kvserver: update TruncatedState before writing #131063
Conversation
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!
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.
4fe206e
to
2561983
Compare
b8e0954
to
9a3d8f0
Compare
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 @sumeerbhola and @tbg)
pkg/kv/kvserver/raft_log_truncator.go
line 593 at r1 (raw file):
Previously, sumeerbhola wrote…
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.
Done.
pkg/kv/kvserver/replica_application_result.go
line 506 at r1 (raw file):
Previously, pav-kv (Pavel Kalinnikov) wrote…
Yeah, seems movable a bit up the stack, I'll consider. Though not critical.
This is now in the right place.
@tbg @sumeerbhola All the pieces are now in the right places. There is some renaming / commenting to be done, but this PR is reviewable now. I also updated the PR description. |
bff7f34
to
407e68a
Compare
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
Epic: none Release note: none
Epic: none Release note: none
Epic: none Release note: none
4139769
to
90dd7ee
Compare
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.
⚡⚡⚡
|
||
// finalizeTruncationRaftMuLocked is a post-apply handler for the raft log |
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.
stage/apply sounds reasonable for naming at first glance, but then there are questions:
stage
doesn't actually "stage" anything that really deletes the log entries: you'd expect it to populate a batch, or something like that, to be committed infinalize
. The real staging either happens in the app batch (tight) or in the truncator (loose).finalize
similarly only deals with sideloaded entries
I can see how this all fits into how things currently work (the truncator needs a slim handle to a Replica to update the cache etc) but it would be useful to leave a blurb on these methods that hint at the bigger picture.
It might also be helpful to untether these methods from *Replica
. Instead, raftTruncatorReplica
could inline these methods directly, and in replicaAppBatch
we could cast the *Replica
as a raftTruncatorReplica
for access to these methods. That way, we don't end up with methods that are confusing when considered as methods on *Replica
in isolation.
Just suggestions - take any or leave it. Once we go back to only having "one kind of truncation", it will be easier to streamline this.
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.
Naming is hard :) So we have 2.5 steps, essentially:
- Logical truncation (called
stagePendingTruncationRaftMuLocked
here). Updates the in-memoryRaftTruncatedState
, size, etc. It begins looking like the truncation has been applied. - Physical truncation (could be fully asynchronous).
a. Writes a batch that carries out the same truncation in log storage.
b. More physical truncation (calledfinalize
here). Removes sideloaded files after (a) is synced.
On where to put the methods: I'd like them to be part of replicaLogStorage / logstore
or something like that. Need to think more about how to consolidate things: #136109.
r.handleRaftLogDeltaResultRaftMuLockedReplicaMuLocked(pt.logDeltaBytes, isDeltaTrusted) | ||
// Ensure the raft log size is not negative since it isn't persisted between | ||
// server restarts. | ||
// TODO(pav-kv): should we distrust the log size if it goes negative? |
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.
That makes sense to me at least. But (outside of the expectedFirstIndex==0 case where all bets are off) it should not be possible to arrive at a negative number through trusted updates, correct?
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.
Yes, we shouldn't see a negative here, post all the cleanups. There are still a couple of possibilities:
- The sideloaded storage size tracking is faulty and can silently skew the log size: logstore: sideloaded storage is not atomic #136416.
- I don't have full confidence that the leader/leaseholder-evaluated size delta exactly matches what the local replica would have computed. Maybe not today, but I could imagine mixed-version scenarios in which raft log encodings could differ.
In some future, I would have liked if the local replica keeps track of its own size precisely and in a self-contained way. One way to achieve this is #136358 and a more general approach is 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.
Reviewed 4 of 14 files at r20, 5 of 7 files at r21, 1 of 1 files at r23, 2 of 2 files at r24, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @tbg)
This PR fixes a truncation "race" that causes raft log entries being not found with
ErrCompacted
during normal operation.Previously, truncations were carried out as follows:
Replica.raftMu
, write a batch to Pebble that removes a prefix of the log and updates theRaftTruncatedState
in the log storage.Replica.{raftMu,mu}
, move the in-memoryRaftTruncatedState
forward.Between steps 1 and 2, there can be a goroutine holding
Replica.mu
that reads the log based on the previousRaftTruncatedState
. It can unwillingly observe that the entries in(previous.Index, next.Index]
interval are missing (the raft log returnsErrCompacted
for them).There are at least 2
Replica.mu
-onlyRawNode.Step
paths affected by this.This PR swaps the order of the updates: the truncated state is moved forward first (along with updating the log size stats), signifying a logical deletion; and only then the entries are deleted from storage, physically. This removes the possibility of the race, and eliminates the need to handle
ErrCompacted
, as long as the reader respects theRaftTruncatedState
when accessing the log. It makes the raft log always consistent underReplica.mu
and/orReplica.raftMu
.Part of #132114, #143355
Related to #130955, #131041