kvserver: support withDiff with TBI in rangefeed catch up scan #69191
kvserver: support withDiff with TBI in rangefeed catch up scan #69191stevendanna wants to merge 1 commit intocockroachdb:masterfrom
Conversation
95f99a4 to
0a7621d
Compare
|
|
||
| var valueBuf []byte | ||
|
|
||
| reorderBuf := make([]roachpb.RangeFeedEvent, 0, 5) |
There was a problem hiding this comment.
that's some magic constant action.
0a7621d to
48da0c1
Compare
48da0c1 to
ab635a6
Compare
|
cc @sumeerbhola You mentioned being willing to take a look at a hacky implementation of the withDiff support. I've put a release justification on here to appease the linter, don't feel pressured to take a look at this before the branch cut if you don't have time. |
The MVCCIncrementalIterator previously could not be used in catch up scan when the withDiff option was provided because the time-based filtering resulted in some previous values of a given key being skipped. To account for this, we use the non-timebound iterator inside the MVCCIncrementalIterator to return the previous value of a given key (previous from the perspective of the timestamp, not the iterator) any time the time optimization would have skipped it by moving to a new key. I could not think of a good name for this rather odd function; suggestions are welcome. Release note: None Release justification: Expands scope of catch up scan performance improvements to address recent high priority escalations.
ab635a6 to
c23b038
Compare
Last week was affected by release blockers and that will continue this week (+ breather week). So I am unlikely to get to it this week. |
sumeerbhola
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @stevendanna, and @sumeerbhola)
pkg/storage/mvcc_incremental_iterator.go, line 269 at r4 (raw file):
// // This is used by rangefeed catchup scans. func (i *MVCCIncrementalIterator) NextSavingSkipped(key roachpb.Key) bool {
This looks like a clean enough change and avoids an additional iterator, which is good.
However, given that this is for 22.1, I'd prefer to wait until we're rewritten MVCCIncrementalIterator to no longer contain iter. The rewritten one would only contain the timeBoundIter and a lock table iter, since we no longer have interleaved intents. I think we'd need to wrap that MVCCIncrementalIterator and a regular MVCCIterator (with MVCCKeyIterKind) to produce this diff behavior, so it would be different from what is done in this PR.
Actually if you want to write this wrapping-with-diff iterator now, it is probably ok to proceed with it. Hmm, I think you will run into trouble due to the iterator caching behavior in pebbleBatch and pebbleReadOnly. The MVCCIncrementalIterator.iter uses up both the cache "slots" so the code will panic when constructing the regular iterator for the wrapper, since it needs another pebbleReadOnly.normalIter. Once the rewrite is done, this will not be a problem since the MVCCIncrementalIterator will no longer use the pebbleReadOnly.normalIter "slot".
Thanks for taking a look. This sounds like a good plan to me. Also, the Changefeed Nemeses tests revealed a problem with this PR that I haven't been able to look into yet, so this isn't merge-able as is. |
Build on #66312
The MVCCIncrementalIterator previously could not be used in catchup
scan when the withDiff option was provided because the time-based
filtering resulted in some previous values of a given key being
skipped.
To account for this, we use the non-timebound iterator inside the
MVCCIncrementalIterator to return the previous value of a given
key (previous from the perspective of the timestamp, not the iterator)
any time the time optimization would have skipped it by moving to a
new key.
I could not think of a good name for this rather odd function;
suggestions are welcome.
Release Notes: None