-
Notifications
You must be signed in to change notification settings - Fork 477
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
*: add simpleLevelIterator, reduce merging levels in external iter #1905
Conversation
8ac76b9
to
a370e71
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.
Reviewed all commit messages.
Reviewable status: 0 of 5 files reviewed, 3 unresolved discussions (waiting on @itsbilal)
external_iterator.go
line 25 at r1 (raw file):
// a key with an identical user key contained within a file at a higher index // subarray. Each subarray must be sorted in internal key order, where lower // index files contain keys that sort left of files with higer indexes.
Unfortunately, I think we need to dictate that range keys may overlap arbitrarily, and tie ourselves to that functionality because I believe backups allow for it.
@msbutler Is that right? I think earlier on we talked about range keys in adjacent files within the same backup overlapping if the backup sstables were split between versions of the same user key.
external_iterator.go
line 36 at r1 (raw file):
iterOpts *IterOptions, files [][]sstable.ReadableFile, extraReaderOpts ...sstable.ReaderOption,
Can we add an explicit option to "opt in" to allowing use of the simpleLevelIter? Eg, something like "forward only". Then we can preserve the ability to use the full breadth of Iterator
functionality if the option is omitted.
Maybe we should have a variadic ExternalIterOption
and allow the caller to provide sstable.ReaderOption
s by wrapping them, eg, ExternalIterReaderOptions(...stable.ReaderOption)
?
external_iterator.go
line 295 at r1 (raw file):
s.filtered = append(s.filtered, s.iters[i]) } }
I'm wondering if we can keep this even simpler and only allow positioning methods to move the iterator forward, including SeekGE
. Then SeekGE
can just next until it finds a key >= the seek key.
@msbutler do we ever seek on the top-level Iterator
? If we only scan the iterator, I think we should keep it as simple as possible for now given stability.
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: 0 of 5 files reviewed, 3 unresolved discussions (waiting on @erikgrinaker, @itsbilal, and @jbowens)
external_iterator.go
line 25 at r1 (raw file):
Previously, jbowens (Jackson Owens) wrote…
Unfortunately, I think we need to dictate that range keys may overlap arbitrarily, and tie ourselves to that functionality because I believe backups allow for it.
@msbutler Is that right? I think earlier on we talked about range keys in adjacent files within the same backup overlapping if the backup sstables were split between versions of the same user key.
Yeah, a key's versions can get split across different backup SSTs. @erikgrinaker told me that RangeKeys can overlap across the SSTs returned by the ExportRequest, but I had not realized this occurred because of "between-version-splitting"
external_iterator.go
line 295 at r1 (raw file):
Previously, jbowens (Jackson Owens) wrote…
I'm wondering if we can keep this even simpler and only allow positioning methods to move the iterator forward, including
SeekGE
. ThenSeekGE
can just next until it finds a key >= the seek key.@msbutler do we ever seek on the top-level
Iterator
? If we only scan the iterator, I think we should keep it as simple as possible for now given stability.
hm. I don't quite understand the question. If the question is: does bulk ever call SeekGE
on an underlying storage.PebbleIterator
, then the answer is yes-- this occurs in restore_data_processor.go.
a370e71
to
62e1fc6
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: 0 of 5 files reviewed, 3 unresolved discussions (waiting on @erikgrinaker, @itsbilal, and @jbowens)
external_iterator.go
line 25 at r1 (raw file):
Previously, msbutler (Michael Butler) wrote…
Yeah, a key's versions can get split across different backup SSTs. @erikgrinaker told me that RangeKeys can overlap across the SSTs returned by the ExportRequest, but I had not realized this occurred because of "between-version-splitting"
Ah I see - but I don't see how this affects this method? Unless @jbowens you're saying we should just call out the arbitrary overlap-ness of range keys in this comment, which I have now done. The keyspan merging iter already takes that into account.
external_iterator.go
line 36 at r1 (raw file):
Previously, jbowens (Jackson Owens) wrote…
Can we add an explicit option to "opt in" to allowing use of the simpleLevelIter? Eg, something like "forward only". Then we can preserve the ability to use the full breadth of
Iterator
functionality if the option is omitted.Maybe we should have a variadic
ExternalIterOption
and allow the caller to providesstable.ReaderOption
s by wrapping them, eg,ExternalIterReaderOptions(...stable.ReaderOption)
?
Done.
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: 0 of 5 files reviewed, 3 unresolved discussions (waiting on @erikgrinaker, @itsbilal, and @msbutler)
external_iterator.go
line 25 at r1 (raw file):
Previously, itsbilal (Bilal Akhtar) wrote…
Ah I see - but I don't see how this affects this method? Unless @jbowens you're saying we should just call out the arbitrary overlap-ness of range keys in this comment, which I have now done. The keyspan merging iter already takes that into account.
Yeah, just wanted to call it out in this documentation.
external_iterator.go
line 295 at r1 (raw file):
Previously, msbutler (Michael Butler) wrote…
hm. I don't quite understand the question. If the question is: does bulk ever call
SeekGE
on an underlyingstorage.PebbleIterator
, then the answer is yes-- this occurs in restore_data_processor.go.
I'm wondering if our usage of SeekGE
is only to
(a) initially position the iterator
and
(b) seek the iterator forward to a key greater than > the current Iterator.Key()
It looks like the usage in restore_data_processor.go is only (a)?
testdata/external_iterator
line 81 at r2 (raw file):
b: (b, .) bb: (ac, .) d: (., [d-e) @3=bar UPDATED)
now that we have the option, can we bring these test cases back and add some that pass in the new forward-only option?
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: 0 of 5 files reviewed, 3 unresolved discussions (waiting on @erikgrinaker and @itsbilal)
external_iterator.go
line 295 at r1 (raw file):
Previously, jbowens (Jackson Owens) wrote…
I'm wondering if our usage of
SeekGE
is only to
(a) initially position the iterator
and
(b) seek the iterator forward to a key greater than > the currentIterator.Key()
It looks like the usage in restore_data_processor.go is only (a)?
Yeah, all iterators used in the bulk codebase, at least on the client side, only seek forward, as they wrap a SimpleMVCCIterator
.
I'm not as familiar with the iterators used on the kvserver side to handle AddSSTable and ExportToMVCC requests. (i.e. do we ever seek backwards during CheckSSTConflicts? Probably not, but perhaps @itsbilal @erikgrinaker could confirm?
3cc5593
to
0984569
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.
Dismissed @jbowens from a discussion.
Reviewable status: 0 of 5 files reviewed, 2 unresolved discussions (waiting on @erikgrinaker, @itsbilal, and @jbowens)
external_iterator.go
line 25 at r1 (raw file):
Previously, jbowens (Jackson Owens) wrote…
Yeah, just wanted to call it out in this documentation.
Done.
external_iterator.go
line 295 at r1 (raw file):
Previously, msbutler (Michael Butler) wrote…
Yeah, all iterators used in the bulk codebase, at least on the client side, only seek forward, as they wrap a
SimpleMVCCIterator
.I'm not as familiar with the iterators used on the kvserver side to handle AddSSTable and ExportToMVCC requests. (i.e. do we ever seek backwards during CheckSSTConflicts? Probably not, but perhaps @itsbilal @erikgrinaker could confirm?
CheckSSTCollisions
can SeekGE backwards but only on rare cases. This was introduced recently and only happens in the case of certain range key overlaps, but it is possible. Maybe that justifies keeping the current logic?
I also made the logic a little more performant by just saving the first keys in each file when we set s.filtered
- really speeds up the SeekGE as we can binary search on the saved first keys instead of binary searching on SeekGE-values.
testdata/external_iterator
line 81 at r2 (raw file):
Previously, jbowens (Jackson Owens) wrote…
now that we have the option, can we bring these test cases back and add some that pass in the new forward-only option?
Done.
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: 0 of 5 files reviewed, 2 unresolved discussions (waiting on @erikgrinaker and @itsbilal)
external_iterator.go
line 333 at r3 (raw file):
} else { s.opts.UpperBound = nil }
what necessitates the bounds copying? the Pebble Iterator
stores its own copies of the bounds and propagates those Pebble-owned bounds down the iterator stack. can we avoid these bound copies?
Currently, we create a new merging iterator level for each file passed into NewExternalIter. This is unnecessary for most use-cases of creating ExternalIters around lots of sstables, as we can externally guarantee that many of those sstables won't have overlapping points with each other. We can have the caller pass this knowledge by specifying a [][]sstable.ReadableFile where each sub-slice obeys level invariants for files within it, and is also already sorted by user keys. This change makes the interface change to allow for the above optimization, and also adds a `simpleLevelIter` that implements forward iteration within a single "level". For files that don't contain range deletes, we shove all the point iters into one `simpleLevelIter`, greatly reducing merging iterator levels and speeding up its operations by a lot. Fixes cockroachdb/cockroach#83051.
0984569
to
6c8ced4
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.
TFTR!
Reviewable status: 0 of 5 files reviewed, 2 unresolved discussions (waiting on @erikgrinaker, @itsbilal, and @jbowens)
external_iterator.go
line 333 at r3 (raw file):
Previously, jbowens (Jackson Owens) wrote…
what necessitates the bounds copying? the Pebble
Iterator
stores its own copies of the bounds and propagates those Pebble-owned bounds down the iterator stack. can we avoid these bound copies?
Ah yes, I was getting the contracts of internalIterator vs Iterator confused when it comes to bounds-setting. Changed.
I'm very eager to see benchmark numbers here. Have any lying around, or should I wait for this to land in CRDB? |
@erikgrinaker Don't have it lying around, but going to try it with a vendor bump now! |
I think besides the vendor bump, we'll also need to change some interfaces in bulk to pass in the backup SSTs in a structured manner. |
In cockroachdb#1905 we added a simpleLevelIterator to allow for reductions in mergingIter levels in the External iterator if the caller can guarantee that point keys won't overlap between some external SSTs. However, that iterator did not propagate errors. This change fixes that.
In cockroachdb#1905 we added a simpleLevelIterator to allow for reductions in mergingIter levels in the External iterator if the caller can guarantee that point keys won't overlap between some external SSTs. However, that iterator did not propagate errors. This change fixes that.
In #1905 we added a simpleLevelIterator to allow for reductions in mergingIter levels in the External iterator if the caller can guarantee that point keys won't overlap between some external SSTs. However, that iterator did not propagate errors. This change fixes that.
Currently, we create a new merging iterator level for each file
passed into NewExternalIter. This is unnecessary for most use-cases
of creating ExternalIters around lots of sstables, as we can externally
guarantee that many of those sstables won't have overlapping points
with each other. We can have the caller pass this knowledge
by specifying a [][]sstable.ReadableFile where each sub-slice
obeys level invariants for files within it, and is also already
sorted by user keys.
This change makes the interface change to allow for the above
optimization, and also adds a
simpleLevelIter
that implementsforward iteration within a single "level". For files that don't
contain range deletes, we shove all the point iters into one
simpleLevelIter
, greatly reducing merging iterator levelsand speeding up its operations by a lot.
Fixes cockroachdb/cockroach#83051.