Skip to content
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: add Time Bound Iteration to DeleteRange #83676

Merged
merged 1 commit into from
Jul 26, 2022

Conversation

msbutler
Copy link
Collaborator

@msbutler msbutler commented Jun 30, 2022

Previously, a kv client could only pass an AOST timestamp to a DelRange
request. Now, the user can pass a lower bound timestamp, causing
the kvserver to leverage time bound iteration while issuing delete requests.

Specifically, the server uses an MVCCIncrementalIterator to iterate over the
target span at the client provided time bounds, track a continuous run of keys
in that time bound, and flush the run via point and MVCC range tombstones
depending on the size of the run.

In a future pr, this operation will replace the use of RevertRange during IMPORT
INTO rollbacks to make them MVCC compatible.

Informs #70428

Release note: none

@msbutler msbutler self-assigned this Jun 30, 2022
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@erikgrinaker
Copy link
Contributor

Btw, since you'll have to respect existing MVCC range tombstones here. We don't have good documentation on them yet, but I'm planning to write a tech note this weekend. In the meanwhile, you can have a look at the MVCC range tombstone info in the SimpleMVCCIterator doc comment:

// The iterator exposes both point keys and range keys. Range keys are only
// emitted when enabled via IterOptions.KeyTypes. Currently, all range keys are
// MVCC range tombstones, and this is enforced during writes.
//
// Range keys and point keys exist separately in Pebble. A specific key position
// can have both a point key and multiple range keys overlapping it. Their
// properties are accessed via:
//
// HasPointAndRange(): Key types present at the current position.
// UnsafeKey(): Current position (point key if any).
// UnsafeValue(): Current point key value (if any).
// RangeBounds(): Start,end bounds of range keys at current position.
// RangeKeys(): All range keys/values overlapping current position.
//
// Consider the following point keys and range keys:
//
// 4: a4 b4
// 3: [-------)
// 2: [-------)
// 1: b1 c1
// a b c
//
// Range keys cover a span between two roachpb.Key bounds (start inclusive, end
// exclusive) and contain timestamp/value pairs. They overlap *all* point key
// versions within their key bounds regardless of timestamp. For example, when
// the iterator is positioned on b@4, it will also expose [a-c)@3 and [a-c)@2.
//
// During iteration with IterKeyTypePointsAndRanges, range keys are emitted at
// their start key and at every overlapping point key. For example, iterating
// across the above span would emit this sequence:
//
// UnsafeKey HasPointAndRange UnsafeValue RangeKeys
// a false,true - [a-c)@3 [a-c)@2
// a@4 true,true a4 [a-c)@3 [a-c)@2
// b@4 true,true b4 [a-c)@3 [a-c)@2
// b@1 true,true b1 [a-c)@3 [a-c)@2
// c@1 true,false c1 -
//
// MVCCIterator reverse iteration yields the above sequence in reverse.
// Notably, bare range keys are still emitted at their start key (not end key),
// so they will be emitted last in this example.
//
// When using SeekGE within range key bounds, the iterator may land on the bare
// range key first, unless seeking exactly to an existing point key. E.g.:
//
// SeekGE UnsafeKey HasPointAndRange UnsafeValue RangeKeys
// b b false,true - [a-c)@3 [a-c)@2
// b@5 b@5 false,true - [a-c)@3 [a-c)@2
// b@4 b@4 true,true b@4 [a-c)@3 [a-c)@2
// b@3 b@3 false,true - [a-c)@3 [a-c)@2
//
// Note that intents (with timestamp 0) encode to a bare roachpb.Key, so they
// will be colocated with a range key start bound. For example, if there was an
// intent on a in the above example, then both SeekGE(a) and forward iteration
// would land on a@0 and [a-c)@3,[a-c)@2 simultaneously, instead of the bare
// range keys first.
//
// Range keys do not have a stable, discrete identity, and should be
// considered a continuum: they may be merged or fragmented by other range key
// writes, split and merged along with CRDB ranges, partially removed by GC,
// and truncated by iterator bounds.
//
// Range keys are fragmented by Pebble such that all overlapping range keys
// between two keys form a stack of range key fragments at different timestamps.
// For example, writing [a-e)@1 and [c-g)@2 will yield this fragment structure:
//
// 2: |---|---|
// 1: |---|---|
// a c e g
//
// Fragmentation makes all range key properties local, which avoids incurring
// unnecessary access costs across SSTs and CRDB ranges. It is deterministic
// on the current range key state, and does not depend on write history.
// Stacking allows easy access to all range keys overlapping a point key.

@msbutler
Copy link
Collaborator Author

msbutler commented Jul 1, 2022

Thanks for the quick response! One more question: if the iterator encounters a point or range key newer than endTime, I know I should return a WriteTooOldError. To implement, my hunch is to do a quick TBI scan (endtime-1,now] before the main TBI scan (startTime,endTime] used to issue delete tombstones. Seem reasonable to you?

@erikgrinaker
Copy link
Contributor

erikgrinaker commented Jul 1, 2022

Thanks for the quick response! One more question: if the iterator encounters a point or range key newer than endTime, I know I should return a WriteTooOldError. To implement, my hunch is to do a quick TBI scan (endtime-1,now] before the main TBI scan (startTime,endTime] used to issue delete tombstones. Seem reasonable to you?

That'll work, but I suspect it might be slow in some cases, depending on how many SSTs the TBIs can skip.

It might be better to just use a single MVCCIncrementalIterator with range (startTime,MaxTimestamp], and check for any future timestamps in the main loop -- if you find one, just bail out with WriteTooOldError.

@msbutler
Copy link
Collaborator Author

msbutler commented Jul 1, 2022

If we wait to check in the main iterator, there's a chance that we will have already issued deletes that we cannot rollback. Is that ok?

Maybe this check isn't even necessary, as the only use case for this function is an Import Rollback on an offline table. If we see newer writes to the table, we're in a real bad place, regardless of how this function behaves.

@erikgrinaker
Copy link
Contributor

If we wait to check in the main iterator, there's a chance that we will have already issued deletes that we cannot rollback. Is that ok?

I don't think that's an issue, because we're evaluating these writes above Raft and staging them in a Pebble batch. The writes won't take effect until the batch is submitted through Raft and committed to Pebble.

Maybe this check isn't even necessary, as the only use case for this function is an Import Rollback on an offline table. If we see newer writes to the table, we're in a real bad place, regardless of how this function behaves.

We should check for newer writes anyway, to comply with MVCC invariants. Other callers may begin using this for other use-cases, and there's also a possibility that KV requests get delayed or reordered.

@erikgrinaker
Copy link
Contributor

If we wait to check in the main iterator, there's a chance that we will have already issued deletes that we cannot rollback. Is that ok?

I don't think that's an issue, because we're evaluating these writes above Raft and staging them in a Pebble batch. The writes won't take effect until the batch is submitted through Raft and committed to Pebble.

Actually, it might be an issue if we return a resume span -- in that case we'll have committed the batch, but then the client will fail when it tries to delete the remainder.

Of course, that's always going to be an issue here, because some other write may come in before the client resumes the delete -- I don't think we can protect against that unless we make these operations transactional.

@msbutler
Copy link
Collaborator Author

msbutler commented Jul 1, 2022

I think it makes sense to never return a resumeSpan if we surface a WriteTooOldError. Like any other error the user encounters in the middle of a big non transactional job, the user will just have to manually resolve afterwards. For this reason, I'm now convinced to go with your original idea:

use a single MVCCIncrementalIterator with range (startTime,MaxTimestamp], and check for any future timestamps in the main loop -- if you find one, just bail out with WriteTooOldError.

@erikgrinaker
Copy link
Contributor

I think it makes sense to never return a resumeSpan if we surface a WriteTooOldError.

Yeah, totally -- what I meant is that even if we scan the entire span and find no future writes, then go ahead and delete but return a resume span because we hit some key limit, when we come back to resume some other client may have written in the meanwhile, so we'll hit a WriteTooOldError with partial writes anyway. As long as we use resume spans at all here, there's no way to guard against this.

Like any other error the user encounters in the middle of a big non transactional job, the user will just have to manually resolve afterwards. For this reason, I'm now convinced to go with your original idea:

Yep, I agree -- that seems like the most reasonable approach here.

msbutler added a commit to msbutler/cockroach that referenced this pull request Jul 6, 2022
The MVCCInvermentalIterator's NextKeyIgnoreTime() function was previously
deleted in cockroachdb#82691, as there wasn't any use for it at the time. Now, the new
Delete Range with predicate filtering will need it (cockroachdb#83676).

This PR also cleans up duplicate code used to test `NextIgnoringTime` and
`NextKeyIgnoringTime`.

Release note: None
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 5 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @msbutler)


pkg/roachpb/api.proto line 364 at r1 (raw file):

  bool use_experimental_range_tombstone = 5;

  DeleteRangePredicates predicates = 6 [(gogoproto.nullable) = true];

Can this option be used with a transaction?


pkg/roachpb/api.proto line 367 at r1 (raw file):

}

message DeleteRangePredicates {

I got really confused by this since one can't do an MVCC compliant delete in general with a lower-bound, since one can't alter history.
I believe there is a promise being made here that any key found in (experimental_predicate_time, delete_range_time] does not have any version with timestamp <= experimental_predicate_time.

I needed to go back and read the following text

"More formally, we have DeleteRange([k1, k2), latest >= t1) as a non-transactional MVCC operation, that when applied at timestamp t2, will affect all reads >= t2, and [k1, k2) are global keys (not range local keys). The first two cases above will use t1=0. The last case will use t1 corresponding to the IMPORT INTO timestamp. Note that IMPORT INTO does not allow masking an existing key, so if it wrote a value k@t (where t >= t1), then there was not an older value for k at timestamp < t1 prior to the IMPORT INTO. This restriction on IMPORT INTO is what allows this MVCC DeleteRange to be used."

from the first section in https://docs.google.com/document/d/1ItxpitNwuaEnwv95RJORLCGuOczuS2y_GoM2ckJCnFs/edit#heading=h.sygakmt3u8hm to remember this.
Some version of that comment would be useful here.


pkg/storage/mvcc.go line 2454 at r1 (raw file):

// span [key, endKey) which have timestamps in the span (startTime, endTime].
// This can have the apparent effect of "reverting" the range to startTime,
// though all older revisions of deleted keys will be GC'ed. Long runs of keys

I didn't understand "though all older revisions of deleted keys will be GC'ed". Older than startTime?


pkg/storage/mvcc.go line 2542 at r1 (raw file):

	// we need a non-time-bound iter e.g. we need to know if there is an older key
	// under the one we are clearing to know if we're changing the number of live
	// keys. The MVCCIncrementalIterator uses a non-time-bound iter as its source

There shouldn't be an older key under the one we are clearing, since that would cleared too, since this is an MVCC compliant delete.
Which would also mean we want to keep using NextIgnoringTime to ensure that there is no older version, since that would be an error.

I suspect I am forgetting/missing something that is obvious to you all.

msbutler added a commit to msbutler/cockroach that referenced this pull request Jul 7, 2022
The MVCCInvermentalIterator's NextKeyIgnoringTime() function was previously
deleted in cockroachdb#82691, as there wasn't any use for it at the time. Now, the new
Delete Range with predicate filtering will need it (cockroachdb#83676).

This PR also cleans up duplicate code used to test NextIgnoringTime and
NextKeyIgnoringTime.

Release note: None
msbutler added a commit to msbutler/cockroach that referenced this pull request Jul 8, 2022
The MVCCInvermentalIterator's NextKeyIgnoringTime() function was previously
deleted in cockroachdb#82691, as there wasn't any use for it at the time. Now, the new
Delete Range with predicate filtering will need it (cockroachdb#83676).

This PR also cleans up duplicate code used to test NextIgnoringTime and
NextKeyIgnoringTime.

Release note: None
craig bot pushed a commit that referenced this pull request Jul 8, 2022
83729: storage: add nextKeyIgnoringTime() to MVCCIncrementalIterator r=erikgrinaker a=msbutler

The MVCCInvermentalIterator's NextKeyIgnoringTime() function was previously
deleted in #82691, as there wasn't any use for it at the time. Now, the new
Delete Range with predicate filtering will need it (#83676).

This PR also cleans up duplicate code used to test NextIgnoringTime and
NextKeyIgnoringTime.

Release note: None

Co-authored-by: Michael Butler <[email protected]>
@msbutler msbutler force-pushed the butler-import-rollbacks branch from f6784b1 to 5806d5f Compare July 11, 2022 15:17
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @msbutler and @sumeerbhola)


pkg/roachpb/api.proto line 364 at r1 (raw file):

Previously, sumeerbhola wrote…

Can this option be used with a transaction?

It cannot. Added a comment to reflect this.


pkg/roachpb/api.proto line 367 at r1 (raw file):

Previously, sumeerbhola wrote…

I got really confused by this since one can't do an MVCC compliant delete in general with a lower-bound, since one can't alter history.
I believe there is a promise being made here that any key found in (experimental_predicate_time, delete_range_time] does not have any version with timestamp <= experimental_predicate_time.

I needed to go back and read the following text

"More formally, we have DeleteRange([k1, k2), latest >= t1) as a non-transactional MVCC operation, that when applied at timestamp t2, will affect all reads >= t2, and [k1, k2) are global keys (not range local keys). The first two cases above will use t1=0. The last case will use t1 corresponding to the IMPORT INTO timestamp. Note that IMPORT INTO does not allow masking an existing key, so if it wrote a value k@t (where t >= t1), then there was not an older value for k at timestamp < t1 prior to the IMPORT INTO. This restriction on IMPORT INTO is what allows this MVCC DeleteRange to be used."

from the first section in https://docs.google.com/document/d/1ItxpitNwuaEnwv95RJORLCGuOczuS2y_GoM2ckJCnFs/edit#heading=h.sygakmt3u8hm to remember this.
Some version of that comment would be useful here.

Oof, my bad, I hadn't updated the docstrings as well as I could have. I just updated them. Please let me know if anything else needs clarification!


pkg/storage/mvcc.go line 2454 at r1 (raw file):

Previously, sumeerbhola wrote…

I didn't understand "though all older revisions of deleted keys will be GC'ed". Older than startTime?

See updated docstring for this func


pkg/storage/mvcc.go line 2542 at r1 (raw file):

Previously, sumeerbhola wrote…

There shouldn't be an older key under the one we are clearing, since that would cleared too, since this is an MVCC compliant delete.
Which would also mean we want to keep using NextIgnoringTime to ensure that there is no older version, since that would be an error.

I suspect I am forgetting/missing something that is obvious to you all.

Hopefully the updated docs makes this clearer. Essentially, if we've surfaced a point key within (startTime,Endtime), we should issue a tombstone on it; so, there's no need to iterate to older versions of the same key. The DelRange calls in flush will handle all the stats.

@msbutler msbutler force-pushed the butler-import-rollbacks branch from 5806d5f to 35c546b Compare July 11, 2022 15:27
@msbutler msbutler force-pushed the butler-import-rollbacks branch from 35c546b to fc82e6a Compare July 11, 2022 21:34
@msbutler msbutler marked this pull request as ready for review July 11, 2022 21:35
@msbutler msbutler requested review from a team as code owners July 11, 2022 21:35
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 5 files at r2, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/roachpb/api.proto line 367 at r1 (raw file):

Previously, msbutler (Michael Butler) wrote…

Oof, my bad, I hadn't updated the docstrings as well as I could have. I just updated them. Please let me know if anything else needs clarification!

Could you enhance the comment to clearly state the invariants assumed, regarding k@t

  • t must be <= endTime (if I understand correctly)
  • if t \in (startTime, endTime], then there is no other k@t' where t' <= startTime.

pkg/storage/mvcc.go line 2486 at r3 (raw file):

// containing up to maxIntents intents.
//
// If an MVCC key surfaced has a timestamp at or above endTime,

The language in api.proto says "If specified, DeleteRange will issue tombstones to keys within the span [startKey, endKey) that also have MVCC versions with timestamps between (startTime, endTime]." Which suggests that an existing version @EndTime is ok. Here it suggests @EndTime is not ok.


pkg/storage/mvcc.go line 2546 at r3 (raw file):

	//  2) The latest key is not a tombstone, matches the predicates,
	//     and has a timestamp below EndTime.
	continueRun := func(k MVCCKey, iter SimpleMVCCIterator) (bool, bool, error) {

can you name these bool return values so it is more obvious what they mean


pkg/storage/mvcc.go line 2657 at r3 (raw file):

		if err != nil {
			return nil, errors.CombineErrors(err, flushDeleteKeys(k))
		}

IIUC we can be here with k@t where t < EndTime, but there is no lower bound on t (this statement would need to be generalized to include range keys). But the invariant is: t <= StartTime implies isTombstone. So the else if toContinue case means t >= StartTime, which explains why it is being explicitly included in the run.
I think stating these invariants in these blocks will make it easier on a reader.


pkg/storage/mvcc.go line 2677 at r3 (raw file):

				iter.NextIgnoringTime()
			} else {
				iter.NextKeyIgnoringTime()

it is worth clarifying in a code comment that this NextKeyIgnoringTime() is optimizing for the case where we were at k@t with t <= StartTime. There could be many older versions of k that we don't care about.


pkg/storage/mvcc.go line 2700 at r3 (raw file):

			// Move the iterator to the next key/value in linear iteration even if it
			// lies outside (startTime, endTime), to see if there's a need to flush.
			iter.NextKeyIgnoringTime()

There must not be an older version of this key. I'd suggest using iter.NextIgnoringTime and confirming that we are not still on the same key (since otherwise we have an error).


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

data: "a"/2.000000000,0 -> /BYTES/a2
data: "b"/3.000000000,0 -> /BYTES/b3
data: "d"/5.000000000,0 -> /<empty>

Isn't this an error case since d@4 is being rolledback but there is a d@1?

@msbutler msbutler force-pushed the butler-import-rollbacks branch from fc82e6a to 5e8a1a9 Compare July 16, 2022 19:19
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/roachpb/api.proto line 367 at r1 (raw file):

Previously, sumeerbhola wrote…

Could you enhance the comment to clearly state the invariants assumed, regarding k@t

  • t must be <= endTime (if I understand correctly)
  • if t \in (startTime, endTime], then there is no other k@t' where t' <= startTime.

Done


pkg/storage/mvcc.go line 2486 at r3 (raw file):

Previously, sumeerbhola wrote…

The language in api.proto says "If specified, DeleteRange will issue tombstones to keys within the span [startKey, endKey) that also have MVCC versions with timestamps between (startTime, endTime]." Which suggests that an existing version @EndTime is ok. Here it suggests @EndTime is not ok.

ack. We cannot have keys with timestamps >= endtime. Updated docs strings to reflect this.


pkg/storage/mvcc.go line 2546 at r3 (raw file):

Previously, sumeerbhola wrote…

can you name these bool return values so it is more obvious what they mean

Done


pkg/storage/mvcc.go line 2657 at r3 (raw file):

Previously, sumeerbhola wrote…

IIUC we can be here with k@t where t < EndTime, but there is no lower bound on t (this statement would need to be generalized to include range keys). But the invariant is: t <= StartTime implies isTombstone. So the else if toContinue case means t >= StartTime, which explains why it is being explicitly included in the run.
I think stating these invariants in these blocks will make it easier on a reader.

Done


pkg/storage/mvcc.go line 2677 at r3 (raw file):

Previously, sumeerbhola wrote…

it is worth clarifying in a code comment that this NextKeyIgnoringTime() is optimizing for the case where we were at k@t with t <= StartTime. There could be many older versions of k that we don't care about.

I think we're optimizing for a larger set of cases: we can call NextKeyIgnoringTime() on any key whose latest version is point tombstone or that we're about to delete. I've updated the comments to reflect this.


pkg/storage/mvcc.go line 2700 at r3 (raw file):

Previously, sumeerbhola wrote…

There must not be an older version of this key. I'd suggest using iter.NextIgnoringTime and confirming that we are not still on the same key (since otherwise we have an error).

I don't think we should do this, as IMPORT is allowed to write the same key twice (specifically, if a key was not checkpointed near a job pause/retry, Import will write it again). Further, during Import's second attempt at writing the key, I think the new MVCC AddSSTable would write the key at a higher timestamp.


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, sumeerbhola wrote…

Isn't this an error case since d@4 is being rolledback but there is a d@1?

I think we should allow this. See comment above about IMPORT's ability to write more than one version of a given key

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 6 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2700 at r3 (raw file):

Previously, msbutler (Michael Butler) wrote…

I don't think we should do this, as IMPORT is allowed to write the same key twice (specifically, if a key was not checkpointed near a job pause/retry, Import will write it again). Further, during Import's second attempt at writing the key, I think the new MVCC AddSSTable would write the key at a higher timestamp.

Interesting -- I didn't think of this case. Can you add this example here.
Or preferably add an example at the top of MVCCPredicateDeleteRange. Something like @(10,20] with
a@5 and no more recent versions
b@12 and b@14
tombstone at c@8 with older versions at c@7, c@6, ...

I'm being nitpicky because I am having a hard time mapping the many (nicely worded) comments in the code to the various interesting cases it is handling.


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, msbutler (Michael Butler) wrote…

I think we should allow this. See comment above about IMPORT's ability to write more than one version of a given key

Isn't this rolling back @(3,5) because of startTime=3 and ts=5? If yes, d@1 does not represent IMPORT writing multiple versions since they would still need to be in (3,5)?
I still think it is better to use NextIgnoringTime and spend the rare iteration cost of seeing multiple versions (in case IMPORT wrote multiple versions), and find the error case in the code. Typically the cost for this error checking will be zero, but it can save us from a correctness bug.

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, sumeerbhola wrote…

Isn't this rolling back @(3,5) because of startTime=3 and ts=5? If yes, d@1 does not represent IMPORT writing multiple versions since they would still need to be in (3,5)?
I still think it is better to use NextIgnoringTime and spend the rare iteration cost of seeing multiple versions (in case IMPORT wrote multiple versions), and find the error case in the code. Typically the cost for this error checking will be zero, but it can save us from a correctness bug.

I agree with Michael here. Even though in the IMPORT case, we don't expect to shadow existing keys, I interpret the mechanics of this parameter as "delete any point keys that satisfy the predicate". We don't care about keys/versions that don't satisfy the predicate, and should ignore them for our purposes.

This will later be extended with an import ID predicate and corresponding block property, in which case we couldn't detect older versions and error on them anyway (the blocks wouldn't match), which would make the behavior of those two predicates differ in their handling of older versions -- that seems unfortunate and unnecessary.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

I agree with Michael here. Even though in the IMPORT case, we don't expect to shadow existing keys, I interpret the mechanics of this parameter as "delete any point keys that satisfy the predicate". We don't care about keys/versions that don't satisfy the predicate, and should ignore them for our purposes.

This will later be extended with an import ID predicate and corresponding block property, in which case we couldn't detect older versions and error on them anyway (the blocks wouldn't match), which would make the behavior of those two predicates differ in their handling of older versions -- that seems unfortunate and unnecessary.

ack.
is there any description of the import ID block property somewhere (discussion thread/doc/code)?

@msbutler my comments here are drive-by, so don't block on me for merging.

Copy link
Contributor

@erikgrinaker erikgrinaker left a 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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, sumeerbhola wrote…

ack.
is there any description of the import ID block property somewhere (discussion thread/doc/code)?

@msbutler my comments here are drive-by, so don't block on me for merging.

Yep: https://docs.google.com/document/d/16TbkFznqbsu3mialSw6o1sxOEn1pKhhJ9HTxNdw0-WE/edit#heading=h.10hso9s01asu

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Flushing some comments as I head into a four-hour meeting block. I'm going to have a closer look at the MVCC bits again, but we're getting there!

Reviewed 6 of 6 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @msbutler, and @sumeerbhola)


pkg/roachpb/api.proto line 360 at r5 (raw file):

  bool use_range_tombstone = 5;

  DeleteRangePredicates predicates = 6 [(gogoproto.nullable) = false];

Nice documentation.

This should specify that it requires UseRangeTombstone.


pkg/roachpb/api.proto line 372 at r5 (raw file):

  // for deletion. If specified, DeleteRange will only issue tombstones to keys
  // within the span [startKey, endKey) that also have MVCC versions with
  // timestamps between (startTime, endTime).

We don't specify endTime anywhere, we should mention that it's the request timestamp.


pkg/storage/mvcc.go line 2608 at r3 (raw file):

Previously, msbutler (Michael Butler) wrote…

Done.

Thanks! But MVCCDelete will likely be measurably slower than calling through to mvccPutInternal, because it has to construct a new iterator and grab a new buffer for every point key. Fine with adding a TODO to optimize.


pkg/storage/mvcc.go line 2494 at r5 (raw file):

// MVCCPredicateDeleteRange returns a WriteTooOldError without a resumeSpan,
// even if tombstones were already written to disk. To resolve, manual
// intervention is necessary.

I think the preferred way to handle this would be for the caller to retry at a higher timestamp, assuming it has appropriate isolation (e.g. via an offline schema descriptor)? Manual cleanup seems bad.


pkg/storage/mvcc.go line 2556 at r5 (raw file):

		hasPointKey, hasRangeKey := iter.HasPointAndRange()
		if hasRangeKey {
			rangeKeys := iter.RangeKeys()

This will likely be very slow, since we're recreating the range key stack for each key it covers. We typically cache these until the range key bounds change. I'm fine with adding a TODO for now and optimize later. This will also get better with #84379.


pkg/storage/mvcc.go line 2708 at r5 (raw file):

			if runSize == 0 {
				buf = buf[:0]
				buf = buf[:cap(buf)]

Hm, why do we do the :0 then :cap dance?


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 9 at r5 (raw file):

# 6
# 5
# 4  x           d4      f4  x   h4          o-------------------o

Might be worth putting a range tombstone in between the point keys here, to make sure it's handled properly and won't e.g. break up key runs.


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 161 at r5 (raw file):

# its latest value is a point tombstone, and is therefore not counted
# in runByteSize.
run ok

Let's add on stats for all of these, to make sure MVCC stats are computed correctly as well (it will automatically verify that the claimed stats are correct).


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 174 at r5 (raw file):

				continue
			}
			t.Run(name+fmt.Sprintf("_Predicates=%v", runWithPredicates), func(t *testing.T) {

nit: the usual way to do this is with subtests, i.e. nested t.Run for each variable/loop.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 209 at r5 (raw file):

						StartTime: hlc.Timestamp{WallTime: 1},
					}
					h.MaxSpanRequestKeys = maxDeleteRangeBatchBytes

nit: seems a bit odd to set this to a byte value. Maybe math.MaxInt64 or something.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 286 at r5 (raw file):

) {
	iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
		KeyTypes:   storage.IterKeyTypePointsOnly,

Let's use PointAndRanges, and assert that we don't see any range keys at the write timestamp.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 301 at r5 (raw file):

		require.NoError(t, err)
		if !value.IsTombstone() {
			fmt.Println("help")

Stray debug output?

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Read over the MVCC bits again. A few minor considerations, but nothing major, so I think we're essentially there.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2610 at r5 (raw file):

				return err
			}
			batchByteSize += int64(len(runStart) + len(nonMatch.Key))

This underestimates the size of the range key, since it does not take into account the timestamp nor the sentinel bytes, like we do for point keys. Not likely to be a huge problem, but let's use MVCCRangeKey.EncodedSize().


pkg/storage/mvcc.go line 2670 at r5 (raw file):

			// buf. Further, the run can continue as its impossible to delete a live
			// key that does not match the predicates.
			if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint {

Since we've already checked this in continueRun(), I suppose we could return another bool in order to save the cost of another call -- it's currently non-negligible, but we're going to try to optimize it down to be ~free. We could also return an enum instead of a bunch of bools. But I'm also fine with deferring this until we've optimized and benchmarked it.


pkg/storage/mvcc.go line 2701 at r5 (raw file):

			if batchSize+runSize >= maxBatchSize || batchByteSize+runByteSize >= maxBatchByteSize {
				// The matched key will be the start the resume span.
				if err := flushDeleteKeys(MVCCKey{Key: k.Key}); err != nil {

This will greedily extend the MVCC range tombstone all the way up to the non-matching key, rather than ending it above the last matching key. I don't think that's a problem. There are essentially two scenarios:

  1. The key span from the last live matching key to this one is empty. In this case, I don't think the width matters at all, except if we end up writing a lot of data on top of it. But imports are usually ordered, right, so a retry would just write the same keys in sequence and then past the non-matching key?

  2. The key span from the last live matching key is full of point/range tombstones. In that case, it might actually be beneficial to cover them with a range tombstone, since it might make GC more efficient.

So I think this is ok, but worth keeping in mind, especially when benchmarking.

Wdyt @jbowens, anything else to consider?

Copy link
Collaborator

@jbowens jbowens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2688 at r5 (raw file):

				iter.NextIgnoringTime()
			} else {
				// Since the latest version of this key is a point tombstone, skip over

this iterator position isn't necessarily at a point tombstone, it could be a point key that's deleted by a range tombstone, right? If so, I think this comment should change to avoid specifically calling it a point tombstone.


pkg/storage/mvcc.go line 2701 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

This will greedily extend the MVCC range tombstone all the way up to the non-matching key, rather than ending it above the last matching key. I don't think that's a problem. There are essentially two scenarios:

  1. The key span from the last live matching key to this one is empty. In this case, I don't think the width matters at all, except if we end up writing a lot of data on top of it. But imports are usually ordered, right, so a retry would just write the same keys in sequence and then past the non-matching key?

  2. The key span from the last live matching key is full of point/range tombstones. In that case, it might actually be beneficial to cover them with a range tombstone, since it might make GC more efficient.

So I think this is ok, but worth keeping in mind, especially when benchmarking.

Wdyt @jbowens, anything else to consider?

I think the biggest pebble-level concern is the impact on ingestion. A wide range tombstone can have a large impact on the performance of ingests into the deleted keyspace. Any ingested sstable that overlaps the range tombstone's key span will be forced into a level of the LSM higher than the range tombstone.

Maybe that's not such a big deal though, because like you said, a retry of an ordered import will overlap with the previous import until it passes this end key, and then ingest strictly beyond it. An unordered import's ingests are almost always forced into higher levels of the LSM anyways, because the ingested sstables tend to overlap with keys written earlier in the same import.


pkg/storage/mvcc.go line 2713 at r5 (raw file):

			if runSize < rangeTombstoneThreshold {
				// Only buffer keys if there's a possibility of issuing point tombstones.
				buf[runSize] = k.Key.Clone()

when we get to optimizing, maybe it would be worthwhile to amortize these allocations since the lifetime of the cloned key is restricted to this function.

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @jbowens, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2701 at r5 (raw file):

Previously, jbowens (Jackson Owens) wrote…

I think the biggest pebble-level concern is the impact on ingestion. A wide range tombstone can have a large impact on the performance of ingests into the deleted keyspace. Any ingested sstable that overlaps the range tombstone's key span will be forced into a level of the LSM higher than the range tombstone.

Maybe that's not such a big deal though, because like you said, a retry of an ordered import will overlap with the previous import until it passes this end key, and then ingest strictly beyond it. An unordered import's ingests are almost always forced into higher levels of the LSM anyways, because the ingested sstables tend to overlap with keys written earlier in the same import.

True. Also, for unordered imports, the same argument could be made between any of the other points we're dropping the MVCC range tombstone above. I.e. if we're covering keys a and c, and then hit a non-matching key z, then even if we end the range key at c.Next() then we could still ingest a ton of data between a and c, and very little between c and z -- we don't really know.

However, the thing that really gives me pause is that we'll always extend the range key to the end of the key span if we don't hit a non-matching key. I don't know what we're plumbing down from bulk, but if this is the entire table span then we're going to end up dropping the range key across the entire table, and then everything we import later will land on top of it. I think for that reason alone, we should keep track of the last matching key and end the range tombstone after it, i.e. lastMatchingKey.Next(). Also, we should only start it at the first live key, if we don't already.

Copy link
Collaborator

@jbowens jbowens left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2701 at r5 (raw file):

I don't know what we're plumbing down from bulk, but if this is the entire table span then we're going to end up dropping the range key across the entire table, and then everything we import later will land on top of it.

That's a really good point. +1

Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @jbowens, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2688 at r5 (raw file):

Previously, jbowens (Jackson Owens) wrote…

this iterator position isn't necessarily at a point tombstone, it could be a point key that's deleted by a range tombstone, right? If so, I think this comment should change to avoid specifically calling it a point tombstone.

In this codepath, the iterator should be at a pointTombstone. Further above, if continueRun returns isTombstone == true, then the iterator is either at a point or range tombstone. So, within the isTombstone branch, if we're not at range tombstone, then we must be at a point tombstone.


pkg/storage/mvcc.go line 2701 at r5 (raw file):

Previously, jbowens (Jackson Owens) wrote…

I don't know what we're plumbing down from bulk, but if this is the entire table span then we're going to end up dropping the range key across the entire table, and then everything we import later will land on top of it.

That's a really good point. +1

Bulk is going to send the entire table span.

Just to summarize the discussion so far: when we issue a range tombstone, the startKey should be firstLiveMatchedKey in the run, and the endKey should be lastLiveMatchKey.Next().


pkg/storage/mvcc.go line 2708 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Hm, why do we do the :0 then :cap dance?

er, maybe I don't know what the go convention is for clearing the elements of a buffer, but here's my thinking:

buf = buf[:0]: clears all the elements by setting len(buf) to 0

now, to repopulate the buffer, I could call buf = append(buf,nextKey), but I think that b[runSize] = nextKey is more performant (see https://stackoverflow.com/questions/38654729/golang-slice-append-vs-assign-performance)

But, in order to assign elements to a buff, I need to reset the length of the buffer, hence buf = buf[:cap(buf)]

wdyt?

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @jbowens, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2701 at r5 (raw file):

Previously, msbutler (Michael Butler) wrote…

Bulk is going to send the entire table span.

Just to summarize the discussion so far: when we issue a range tombstone, the startKey should be firstLiveMatchedKey in the run, and the endKey should be lastLiveMatchKey.Next().

Yep.


pkg/storage/mvcc.go line 2708 at r5 (raw file):

Previously, msbutler (Michael Butler) wrote…

er, maybe I don't know what the go convention is for clearing the elements of a buffer, but here's my thinking:

buf = buf[:0]: clears all the elements by setting len(buf) to 0

now, to repopulate the buffer, I could call buf = append(buf,nextKey), but I think that b[runSize] = nextKey is more performant (see https://stackoverflow.com/questions/38654729/golang-slice-append-vs-assign-performance)

But, in order to assign elements to a buff, I need to reset the length of the buffer, hence buf = buf[:cap(buf)]

wdyt?

Nah, it doesn't touch the elements, it only changes the slice length: https://go.dev/play/p/Q39FIbtx_qq

As long as we don't resize the backing array (which we don't do here), then I don't think there's a significant difference between append and assign. The second answer on that Stack Overflow thread seems to confirm that as well. I have a slight personal preference for len(buf) rather than runSize, but it doesn't really matter -- whatever you prefer is fine.

@msbutler msbutler force-pushed the butler-import-rollbacks branch from f74ae2d to 577c2d0 Compare July 21, 2022 16:21
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @erikgrinaker, @jbowens, and @sumeerbhola)


pkg/roachpb/api.proto line 360 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Nice documentation.

This should specify that it requires UseRangeTombstone.

Done


pkg/roachpb/api.proto line 372 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

We don't specify endTime anywhere, we should mention that it's the request timestamp.

Good catch. Done.


pkg/storage/mvcc.go line 2608 at r3 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Thanks! But MVCCDelete will likely be measurably slower than calling through to mvccPutInternal, because it has to construct a new iterator and grab a new buffer for every point key. Fine with adding a TODO to optimize.

Done. Seemed like a quick win.


pkg/storage/mvcc.go line 2700 at r3 (raw file):

Previously, sumeerbhola wrote…

Interesting -- I didn't think of this case. Can you add this example here.
Or preferably add an example at the top of MVCCPredicateDeleteRange. Something like @(10,20] with
a@5 and no more recent versions
b@12 and b@14
tombstone at c@8 with older versions at c@7, c@6, ...

I'm being nitpicky because I am having a hard time mapping the many (nicely worded) comments in the code to the various interesting cases it is handling.

Done!


pkg/storage/mvcc.go line 2494 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

I think the preferred way to handle this would be for the caller to retry at a higher timestamp, assuming it has appropriate isolation (e.g. via an offline schema descriptor)? Manual cleanup seems bad.

That makes sense!


pkg/storage/mvcc.go line 2556 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

This will likely be very slow, since we're recreating the range key stack for each key it covers. We typically cache these until the range key bounds change. I'm fine with adding a TODO for now and optimize later. This will also get better with #84379.

Done.


pkg/storage/mvcc.go line 2610 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

This underestimates the size of the range key, since it does not take into account the timestamp nor the sentinel bytes, like we do for point keys. Not likely to be a huge problem, but let's use MVCCRangeKey.EncodedSize().

Interesting, check out what I've done.


pkg/storage/mvcc.go line 2670 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Since we've already checked this in continueRun(), I suppose we could return another bool in order to save the cost of another call -- it's currently non-negligible, but we're going to try to optimize it down to be ~free. We could also return an enum instead of a bunch of bools. But I'm also fine with deferring this until we've optimized and benchmarked it.

Revised it to return three bools! Seems more in line with iter.HasPointAndRange() semantics. It also simplifies the if statement below.


pkg/storage/mvcc.go line 2708 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Nah, it doesn't touch the elements, it only changes the slice length: https://go.dev/play/p/Q39FIbtx_qq

As long as we don't resize the backing array (which we don't do here), then I don't think there's a significant difference between append and assign. The second answer on that Stack Overflow thread seems to confirm that as well. I have a slight personal preference for len(buf) rather than runSize, but it doesn't really matter -- whatever you prefer is fine.

Sweet, using append now since it's cleaner


pkg/storage/mvcc.go line 2713 at r5 (raw file):

Previously, jbowens (Jackson Owens) wrote…

when we get to optimizing, maybe it would be worthwhile to amortize these allocations since the lifetime of the cloned key is restricted to this function.

interesting! Just to clarify, are you saying, instead of calling k.Key.Clone() 3 times in the given iteration, just call it once? Just fixed that.


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 57 at r3 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Yep: https://docs.google.com/document/d/16TbkFznqbsu3mialSw6o1sxOEn1pKhhJ9HTxNdw0-WE/edit#heading=h.10hso9s01asu

Done.


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 9 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Might be worth putting a range tombstone in between the point keys here, to make sure it's handled properly and won't e.g. break up key runs.

done. grep for del_range_pred k=j end=r ts=5 startTime=1 rangeThreshold=2


pkg/storage/testdata/mvcc_histories/delete_range_predicate line 161 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Let's add on stats for all of these, to make sure MVCC stats are computed correctly as well (it will automatically verify that the claimed stats are correct).

Done


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 174 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

nit: the usual way to do this is with subtests, i.e. nested t.Run for each variable/loop.

Done.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 209 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

nit: seems a bit odd to set this to a byte value. Maybe math.MaxInt64 or something.

whoops, good catch! Done.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 286 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Let's use PointAndRanges, and assert that we don't see any range keys at the write timestamp.

Done.


pkg/kv/kvserver/batcheval/cmd_delete_range_test.go line 301 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Stray debug output?

cheers!

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this should be good to go now, but I'm going to do a last pass over this to see if we missed anything.

Reviewed 6 of 6 files at r6, 6 of 6 files at r7, 4 of 4 files at r8, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @EndTime, @jbowens, @msbutler, and @sumeerbhola)


pkg/storage/mvcc.go line 2608 at r3 (raw file):

Previously, msbutler (Michael Butler) wrote…

Done. Seemed like a quick win.

Thanks! I think we can reuse the iterator and buffer between all flushes, which should net us some more performance gains.


pkg/storage/mvcc.go line 2610 at r5 (raw file):

Previously, msbutler (Michael Butler) wrote…

Interesting, check out what I've done.

👍


pkg/storage/mvcc.go line 2713 at r5 (raw file):

Previously, msbutler (Michael Butler) wrote…

interesting! Just to clarify, are you saying, instead of calling k.Key.Clone() 3 times in the given iteration, just call it once? Just fixed that.

I think he means reusing the byte slice, e.g.:

runStart = append(runStart[:0], k.Key...)

This only needs O(1) memory allocations for all key clones, instead of O(n) memory allocations.

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: once comments are addressed.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @EndTime, @jbowens, @msbutler, and @sumeerbhola)

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, thank you for taking this on, much appreciated!

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @EndTime, @jbowens, @msbutler, and @sumeerbhola)

@msbutler msbutler force-pushed the butler-import-rollbacks branch from 577c2d0 to af2eac7 Compare July 25, 2022 12:56
Copy link
Collaborator Author

@msbutler msbutler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for all the reviews! Learned a ton!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @EndTime, @erikgrinaker, @jbowens, and @sumeerbhola)


pkg/storage/mvcc.go line 2608 at r3 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Thanks! I think we can reuse the iterator and buffer between all flushes, which should net us some more performance gains.

Done


pkg/storage/mvcc.go line 2713 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

I think he means reusing the byte slice, e.g.:

runStart = append(runStart[:0], k.Key...)

This only needs O(1) memory allocations for all key clones, instead of O(n) memory allocations.

Done

@msbutler msbutler force-pushed the butler-import-rollbacks branch from af2eac7 to 0e7ca0e Compare July 25, 2022 19:07
Previously, a kv client could only pass an AOST timestamp to a DeleteRange
request. Now, the user can pass a lower bound timestamp, causing
the kvserver to leverage time bound iteration while issuing delete requests.

Specifically, the server uses an MVCCIncrementalIterator to iterate over the
target span at the client provided time bounds, track a continuous run of keys
in that time bound, and flush the run via point and MVCC range tombstones
depending on the size of the run.

In a future pr, this operation will replace the use of RevertRange during IMPORT
INTO rollbacks to make them MVCC compatible.

Informs cockroachdb#70428

Release note: none
@msbutler msbutler force-pushed the butler-import-rollbacks branch from 0e7ca0e to aead1ca Compare July 25, 2022 19:42
@msbutler
Copy link
Collaborator Author

bors r=erikgrinaker

@craig
Copy link
Contributor

craig bot commented Jul 25, 2022

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Jul 26, 2022

Build succeeded:

@craig craig bot merged commit 4ab8292 into cockroachdb:master Jul 26, 2022
@msbutler msbutler deleted the butler-import-rollbacks branch July 26, 2022 10:24
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants