logstore: funnel all raft log deletions through logstore package#169640
logstore: funnel all raft log deletions through logstore package#169640iskettaneh wants to merge 5 commits into
Conversation
|
Merging to
After your PR is submitted to the merge queue, this comment will be automatically updated with its status. If the PR fails, failure details will also be posted here |
|
Detected infrastructure failure (matched: self-hosted runner lost communication with the server). Automatically rerunning failed jobs. (run link) |
2074ef3 to
e126c8c
Compare
b805fc5 to
3d58222
Compare
| prefixBuf := keys.MakeRangeIDPrefixBuf(rangeID) | ||
| // We want to delete all raft log entries < hi. Since Raft log doesn't have | ||
| // holes, we can get the first index, calculate the log size, and call | ||
| // ClearRangeSizeKnown(). If no entries exist in [RaftLogPrefix, hi) (e.g., |
There was a problem hiding this comment.
If no entries exist in [RaftLogPrefix, hi)
@pav-kv Is this possible? could we destroy an uninitialized replica and it have the Addr.index = 0?
|
Detected infrastructure failure (matched: self-hosted runner lost communication with the server). Automatically rerunning failed jobs. (run link) |
1 similar comment
|
Detected infrastructure failure (matched: self-hosted runner lost communication with the server). Automatically rerunning failed jobs. (run link) |
3d58222 to
8d70456
Compare
| return start, end | ||
| } | ||
|
|
||
| // ClearRange clears raft log entries in the range [lo, hi). It calls |
There was a problem hiding this comment.
This is marketed as "log size not known" (as opposed to the SizeKnown below), but isn't it true that we know the size when lo is not zero?
There was a problem hiding this comment.
and if the hi is not MaxUint64. I will see if I can just use the knownsize function
| if hi == math.MaxUint64 { | ||
| end = prefixBuf.RaftLogPrefix().PrefixEnd() |
There was a problem hiding this comment.
Is this ever the case that we don't know the upper bound? All deletion paths I can think of do know: (a) Replica always knows its log bounds, (b) replay of replica destructions always knows the last index of the removed log.
Is there a (c) exception?
There was a problem hiding this comment.
This is just written this way to match the current pattern in the places where we clear the whole raft log. but I agree that if we move away from this pattern (and only clear the raft log part that is populated), we will always know the size.
| ctx context.Context, | ||
| r storage.Reader, | ||
| w storage.Writer, | ||
| prefixBuf keys.RangeIDPrefixBuf, |
There was a problem hiding this comment.
Since we are cloning both keys in raftLogBounds, do we need the buf? Or there is some case in which we save an allocation?
There was a problem hiding this comment.
I changed the functions to take raftLogPrefix, since they don't need the other functionalities in RangeIDPrefixBuf,
| // force a single range tombstone over the whole log span, without scanning. | ||
| if err := logstore.ClearRangeSizeKnown( | ||
| raftWO, sl.RangeIDPrefixBuf, | ||
| 0 /* lo */, math.MaxUint64 /* hi */, 0, /* pointKeyThreshold */ |
There was a problem hiding this comment.
We can probably make it a "size known" case. If we are applying a snapshot to an existing replica, it knows its log. Otherwise, the log is empty.
Btw, I just realized we need more code here for separated engines case. We shouldn't be deleting the entire log - the applied part is deferred to the WAG truncator.
There was a problem hiding this comment.
I want this PR to be as much of a mechanical refactor as possible. I left a TODO to address this in another PR
| } | ||
| if err := logstore.ClearRange( | ||
| ctx, rw.Raft.RO, rw.Raft.WO, buf, | ||
| info.RaftAppliedIndex+1 /* lo */, math.MaxUint64 /* hi */, ClearRangeThresholdPointKeys(), |
There was a problem hiding this comment.
We could make hi known here if the DestroyReplicaInfo passes down the log's last index. Dunno if we want to do it (it's "safer" to clear the whole suffix pretending we don't know where it ends), but it's a possibility.
There was a problem hiding this comment.
True. I want this PR to be as much of a mechanical refactor as possible. I left a TODO to address this in another PR
|
There is also the bit in |
8d70456 to
b0caad3
Compare
Introduce three helpers that centralize how raft log entries are deleted from the engine: - ClearRange: clears entries in [lo, hi) when the entry count is unknown. Defers to storage.ClearRangeWithHeuristic, which scans up to a threshold to choose between point deletes and a single Pebble range tombstone. - ClearRangeSizeKnown: clears entries in [lo, hi) when the caller already knows how many entries the range contains. The choice between point deletes and a range tombstone is made arithmetically (hi - lo vs threshold), without scanning. Optionally uses SingleDeletes if the caller indicates it and if the # of entries <= the point deletes threshold. These helpers don't yet have any callers; subsequent commits funnel existing raft log deletion sites through them. Both helpers (and the underlying raftLogBounds) take a raftLogPrefix roachpb.Key directly rather than a keys.RangeIDPrefixBuf, since the prefix is all they need; this also keeps the call sites narrowed to a single unambiguous key. Release note: None Co-Authored-By: roachdev-claude <roachdev-claude-bot@cockroachlabs.com>
Replace the inline raft log clear in Compact with a call to ClearRangeSizeKnown, which uses the size-based branch (range tombstone vs. per-entry point or single deletes) added in the previous commit. Release note: None Co-Authored-By: roachdev-claude <roachdev-claude-bot@cockroachlabs.com>
Replace the inline ClearRawRange in RewriteRaftState with a call to logstore.ClearRangeSizeKnown. Release note: None Co-Authored-By: roachdev-claude <roachdev-claude-bot@cockroachlabs.com>
Update TestStoreRangeMergeRaftSnapshot's expected SST construction to mirror the new three-clear shape (HardState..RaftLogPrefix, RaftLogPrefix..PrefixEnd via the heuristic, then the post-truncated-state tail) so the snapshot SSTs continue to compare equal to the receiving store's SST output. Release note: None Co-Authored-By: roachdev-claude <roachdev-claude-bot@cockroachlabs.com>
…Known Release note: None Co-Authored-By: roachdev-claude <roachdev-claude-bot@cockroachlabs.com>
b0caad3 to
f680d3e
Compare
iskettaneh
left a comment
There was a problem hiding this comment.
@iskettaneh made 5 comments.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on pav-kv).
| // force a single range tombstone over the whole log span, without scanning. | ||
| if err := logstore.ClearRangeSizeKnown( | ||
| raftWO, sl.RangeIDPrefixBuf, | ||
| 0 /* lo */, math.MaxUint64 /* hi */, 0, /* pointKeyThreshold */ |
There was a problem hiding this comment.
I want this PR to be as much of a mechanical refactor as possible. I left a TODO to address this in another PR
| } | ||
| if err := logstore.ClearRange( | ||
| ctx, rw.Raft.RO, rw.Raft.WO, buf, | ||
| info.RaftAppliedIndex+1 /* lo */, math.MaxUint64 /* hi */, ClearRangeThresholdPointKeys(), |
There was a problem hiding this comment.
True. I want this PR to be as much of a mechanical refactor as possible. I left a TODO to address this in another PR
| if hi == math.MaxUint64 { | ||
| end = prefixBuf.RaftLogPrefix().PrefixEnd() |
There was a problem hiding this comment.
This is just written this way to match the current pattern in the places where we clear the whole raft log. but I agree that if we move away from this pattern (and only clear the raft log part that is populated), we will always know the size.
| return start, end | ||
| } | ||
|
|
||
| // ClearRange clears raft log entries in the range [lo, hi). It calls |
There was a problem hiding this comment.
and if the hi is not MaxUint64. I will see if I can just use the knownsize function
| ctx context.Context, | ||
| r storage.Reader, | ||
| w storage.Writer, | ||
| prefixBuf keys.RangeIDPrefixBuf, |
There was a problem hiding this comment.
I changed the functions to take raftLogPrefix, since they don't need the other functionalities in RangeIDPrefixBuf,
There are three main ways to clear a raft log range, all now expressed via ClearRange:
Delete a range using ClearRangeWithHeuristic when the entry count is not known.
When the entry count is known (during log truncations) we do an arithmatic calculation to determine whether we issue point/range deletions.
Blindly delete the whole raft log span using ClearRawRange. This is done when applying a raft snapshot.
This PR introduces three helpers ClearRange(), ClearRangeSizeKnown() to funnel all of these use cases through logstore.
References: #8979
Release note: None
Co-Authored-By: Claude Opus 4.6 noreply@anthropic.com