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

*: add IngestAndExcise operation #2538

Merged
merged 1 commit into from
Jun 12, 2023

Conversation

itsbilal
Copy link
Member

This change adds an IngestAndExcise operation that does the below additional things alongside a regular ingestion:

  1. It ingests some SharedSSTMeta files, which are provider-backed
    sstables that could be owned by other nodes.
  2. It excises existing sstables within the provided excise span (within
    which all sstables from 1 must fit) by creating new virtual sstables
    that exclude keys from the excise span.

While this change can be implemented independently of #2455, some of the end-to-end tests in future changes will rely on both that and this.

Fixes #2520.

@itsbilal itsbilal requested a review from a team May 17, 2023 22:49
@itsbilal itsbilal self-assigned this May 17, 2023
@itsbilal itsbilal requested a review from bananabrick May 17, 2023 22:49
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@itsbilal itsbilal force-pushed the excise-operation branch 3 times, most recently from b94f4be to 16dedfc Compare May 18, 2023 21:20
Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

Nice work!

Reviewable status: 0 of 12 files reviewed, 8 unresolved discussions (waiting on @bananabrick and @itsbilal)


ingest.go line 76 at r2 (raw file):

			meta.LargestRangeKey.SetSeqNum(base.SeqNumL6RangeKey)
		default:
			panic(fmt.Sprintf("unexpected level in shared meta: %d", sm.Level))

Even though we currently store all levels on blob storage, we will only share L5+ when rebalancing?


ingest.go line 327 at r2 (raw file):

}

func ingestSortAndVerify(

[nit] I'd consolidate some of the comments below in a function comment explaining the slices. Maybe it would be more clear to have localMeta, localPaths, sharedMeta


ingest.go line 1215 at r2 (raw file):

func (d *DB) excise(
	exciseSpan KeyRange, m *fileMetadata, ve *versionEdit, level int,
) (*versionEdit, error) {

we're modifying ve in place, why are we returning it also?


ingest.go line 1234 at r2 (raw file):

	var rangeKeyIter keyspan.FragmentIterator
	backingTableCreated := false
	// Create a file to the left of the excise span, if one exists.

[nit] "if necessary"


ingest.go line 1539 at r2 (raw file):

					FileNum: m.FileNum,
				}]; ok {
					levelMetrics.NumFiles--

[nit] This block is the same path with the block below.. Maybe move everything here or invert it to if !deleted { continue }


ingest.go line 1544 at r2 (raw file):

					continue
				}
				// Take advantage of the fact that excise() must have added files at

Maybe it would be cleaner if excise would return the subslice of files it added?


ingest.go line 1546 at r2 (raw file):

				// Take advantage of the fact that excise() must have added files at
				// the end of ve.NewFiles to update levelMetrics accordingly.
				for i := len(ve.NewFiles) - 1; i >= 0; i-- {

[nit] i >= 0 && ve.NewFiles[i].Meta.FileBacking == m.FileBacking would be more clear


ingest.go line 1427 at r4 (raw file):

		}
	}

Do we have a facility for checking that the Smallest/Largest keys are correct in the metadata? I assume they can be cross-checked against an interator. We could (under invariants) randomly run that check, or we could trigger it from tests.

Copy link
Member Author

@itsbilal itsbilal left a 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 12 files reviewed, 1 unresolved discussion (waiting on @bananabrick and @RaduBerinde)


ingest.go line 76 at r2 (raw file):

Previously, RaduBerinde wrote…

Even though we currently store all levels on blob storage, we will only share L5+ when rebalancing?

That's right. The stuff in ScanInternal already filters for just sharing levels above sharedLevelStart.


ingest.go line 327 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] I'd consolidate some of the comments below in a function comment explaining the slices. Maybe it would be more clear to have localMeta, localPaths, sharedMeta

Done.


ingest.go line 1215 at r2 (raw file):

Previously, RaduBerinde wrote…

we're modifying ve in place, why are we returning it also?

Done.


ingest.go line 1234 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] "if necessary"

Done.


ingest.go line 1539 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] This block is the same path with the block below.. Maybe move everything here or invert it to if !deleted { continue }

Done.


ingest.go line 1544 at r2 (raw file):

Previously, RaduBerinde wrote…

Maybe it would be cleaner if excise would return the subslice of files it added?

Done.


ingest.go line 1546 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] i >= 0 && ve.NewFiles[i].Meta.FileBacking == m.FileBacking would be more clear

Done (simplified due to the new return value).


ingest.go line 1427 at r4 (raw file):

Previously, RaduBerinde wrote…

Do we have a facility for checking that the Smallest/Largest keys are correct in the metadata? I assume they can be cross-checked against an interator. We could (under invariants) randomly run that check, or we could trigger it from tests.

By correct you mean that they're calculated accurately, or that they just obey some common invariants like Largest > Smallest ? I added meta.Validate calls for the latter, but the former would sort of require re-running this logic or something like it to verify. I'm gonna give this more thought though.

@itsbilal itsbilal force-pushed the excise-operation branch 2 times, most recently from b2daacc to cc5e9a8 Compare May 24, 2023 18:54
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 3 of 11 files at r1, 6 of 9 files at r5.
Reviewable status: 3 of 13 files reviewed, 6 unresolved discussions (waiting on @bananabrick, @itsbilal, and @RaduBerinde)


compaction.go line 2566 at r5 (raw file):

			c.inputs[i].files.Each(func(m *manifest.FileMetadata) {
				if m.Deleted {
					err = firstError(err, errors.New("pebble: file deleted by a concurrent operation, will retry compaction"))

Can we ensure/check that this code path gets exercised a lot, both in a regular unit test and the metamorphic test?


ingest.go line 87 at r5 (raw file):

		case 5:
			meta.SmallestPointKey.SetSeqNum(base.SeqNumL5Point)
			meta.LargestPointKey.SetSeqNum(base.SeqNumL5RangeDel)

why is SeqNumL5RangeDel not equal to SeqNumL5Point? With the same seqnum, the rangedel will not delete the point, as desired.


ingest.go line 946 at r5 (raw file):

}

func (d *DB) ingest(

nit: even though this is not public, this code is getting complicated enough that it is worth have a code comment explaining its behavior and the parameters.


ingest.go line 963 at r5 (raw file):

	}
	for i := range shared {
		pendingOutputs[len(paths)+i] = d.mu.versions.getNextFileNum().DiskFileNum()

it's not quite fair to use DiskFileNum for things that are not on disk. We have FileNum for that. We even say this:

// A DiskFileNum is just a FileNum belonging to a file which exists on disk.
// Note that a FileNum is an internal DB identifier and it could belong to files
// which don't exist on disk. An example would be virtual sstable FileNums.
// Converting a DiskFileNum to a FileNum is always valid, whereas converting a
// FileNum to DiskFileNum may not be valid and care should be taken to prove
// that the FileNum actually exists on disk.

internal/manifest/l0_sublevels.go line 904 at r5 (raw file):

// Contains returns whether the specified key exists in the UserKeyRange.
func (k *UserKeyRange) Contains(cmp base.Compare, key []byte) bool {
	return cmp(k.Start, key) <= 0 && cmp(key, k.End) < 0

The comment for UserKeyRange says that End is inclusive.

Also compaction.elideTombstone could use this I think.

@itsbilal itsbilal force-pushed the excise-operation branch 2 times, most recently from a72c4e8 to 8182b6a Compare May 24, 2023 21:53
Copy link
Member Author

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

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

TFTR!

Reviewable status: 1 of 13 files reviewed, 6 unresolved discussions (waiting on @bananabrick, @RaduBerinde, and @sumeerbhola)


compaction.go line 2566 at r5 (raw file):

Previously, sumeerbhola wrote…

Can we ensure/check that this code path gets exercised a lot, both in a regular unit test and the metamorphic test?

Good point. In my testing it fires pretty regularly, and once we wire up the metamorphic test we should see many background errors with this case. I'm writing an end-to-end test in a different branch, and I'll try to add an assertion there that we encountered at least one call to EventListener.BackgroundError.


ingest.go line 87 at r5 (raw file):

Previously, sumeerbhola wrote…

why is SeqNumL5RangeDel not equal to SeqNumL5Point? With the same seqnum, the rangedel will not delete the point, as desired.

It's just clearer this way, but also I forgot that we don't delete points at the same seqnums as a rangedel. Happy to collapse the two seqnums if you think it's better that way.


ingest.go line 946 at r5 (raw file):

Previously, sumeerbhola wrote…

nit: even though this is not public, this code is getting complicated enough that it is worth have a code comment explaining its behavior and the parameters.

There's already the giant comment at Ingest which talks about the process. I'll add another at IngestAndExcise and one here that references to it.


ingest.go line 963 at r5 (raw file):

Previously, sumeerbhola wrote…

it's not quite fair to use DiskFileNum for things that are not on disk. We have FileNum for that. We even say this:

// A DiskFileNum is just a FileNum belonging to a file which exists on disk.
// Note that a FileNum is an internal DB identifier and it could belong to files
// which don't exist on disk. An example would be virtual sstable FileNums.
// Converting a DiskFileNum to a FileNum is always valid, whereas converting a
// FileNum to DiskFileNum may not be valid and care should be taken to prove
// that the FileNum actually exists on disk.

I made this comment elsewhere including in the DiskFIleNum PR, that the correct name for it should be BackingFileNum because we use it to reference FileBackings and we already use it in FileBacking even if an sstable is in shared.Storage and not in the provider. Which is to say I fully agree with you and that this type is wrongly tagged. I just think we should do it in a separate PR because it'll be a wholesale rename of the type and a lot of comment edits.

And since we're creating filenums in the provider that the metadatas will now reference, we'll need to do the conversion to DiskFileNum (which will soon be BackingFileNum, and mentally we should treat it as such) somewhere anyway between here and InitProviderBacking (and also provider.AttachSharedObjects which specifically asks for DiskFileNums to attach to).


internal/manifest/l0_sublevels.go line 904 at r5 (raw file):

Previously, sumeerbhola wrote…

The comment for UserKeyRange says that End is inclusive.

Also compaction.elideTombstone could use this I think.

Ah, oops. Good catch. I'll split the types and create a new KeyRange in the toplevel package that is inclusive,exclusive.

I'll keep compaction.elideTombstone as-is as it's the inclusive, inclusive case and is more directly dealing with key ranges coming from L0 Sublevels (and has an optimization to break based on the sort order of key ranges).

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.

flushing some comments

Reviewed 1 of 7 files at r6, all commit messages.
Reviewable status: 2 of 13 files reviewed, 15 unresolved discussions (waiting on @bananabrick, @itsbilal, @RaduBerinde, and @sumeerbhola)


ingest.go line 43 at r7 (raw file):

}

// Valid returns true if the UserKeyRange is defined.

nit: s/UserKeyRange/KeyRange/


ingest.go line 48 at r7 (raw file):

}

// Contains returns whether the specified key exists in the UserKeyRange.

nit: s/UserKeyRange/KeyRange/


ingest.go line 74 at r7 (raw file):

	meta.FileNum = fileNum.FileNum()
	meta.CreationTime = time.Now().Unix()
	meta.Virtual = true

I didn't realize that all shared sstables are virtual—if the underlying sst's bounds are narrower than the KV range, could we use the shared sstable directly without marking it as "virtual," but continuing to retrieve the object from the object provider.


ingest.go line 76 at r7 (raw file):

	meta.Virtual = true
	meta.Size = sm.Size
	meta.InitProviderBacking(fileNum, uint64(size))

how does this fit into the virtual sstable compaction heuristics? eg, we inflate virtual sstable's compaction priority by the non-live data of the backing sstable under the assumption that if we compact the virtual sstable(s), we can remove the backing sstable. The introduction of shared sstables complicates this, because removing this store's reference to the shared object won't necessarily allow us to delete the sstable from external storage.

Typically, we expect incoming shared sstables to already be a subset of the data of the underlying cloud sstable. I'm mildly concerned that we'll be over-prioritizing compacting shared sstables that we just ingested.


ingest.go line 84 at r7 (raw file):

		case 5:
			meta.SmallestRangeKey.SetSeqNum(base.SeqNumL5RangeKeySet)
			meta.LargestRangeKey.SetSeqNum(base.SeqNumL5RangeKeyUnsetDel)

similarly to @sumeerbhola's comment about rangedels, i don't think there's any requirement for separate sequence numbers here:

// Additionally, internal range keys at the same sequence number have subtle
// mechanics:
//   - RANGEKEYSETs shadow RANGEKEYUNSETs of the same suffix.
//   - RANGEKEYDELs only apply to keys at lower sequence numbers.

Once coalesced, for each unique fragment bounds, each suffix only appears once within RANGEKYSET or RANGEKEYUNSET. RANGEKEYDELs can safely share the same sequence number and not affect anything other than L6 range keys.

Also, since point keys and range keys have parallel existences, the same sequence number can be used for point keys and range keys.


ingest.go line 120 at r7 (raw file):

	opts *Options, fmv FormatMajorVersion, path string, cacheID uint64, fileNum base.DiskFileNum,
) (*fileMetadata, error) {
	var readable objstorage.Readable

nit: are the changes to this func intentional?


ingest.go line 299 at r7 (raw file):

	sharedMeta := make([]*fileMetadata, 0, len(shared))
	levels := make([]uint8, 0, len(shared))
	var sharedObjs []objstorage.SharedObjectToAttach

nit: should this also be pre-allocated to len(shared) capacity?


ingest.go line 365 at r7 (raw file):

	for i := range lr.sharedMeta {
		f := lr.sharedMeta[i]
		if !exciseSpan.Contains(cmp, f.Smallest.UserKey) || !exciseSpan.Contains(cmp, f.Largest.UserKey) {

what if f is exclusive in the end boundary? that seems like it must be possible if not probable, in which case the user key of the exclusive bound does not fall within exciseSpan.


ingest.go line 854 at r7 (raw file):

// ingestion process.
//
// Panics if this DB instance was not instantiated with a shared.Storage.

is this true? Even without shared storage, we want to use IngestAndExcise for snapshot reception, no?

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: 2 of 13 files reviewed, 15 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @RaduBerinde)


ingest.go line 87 at r5 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

It's just clearer this way, but also I forgot that we don't delete points at the same seqnums as a rangedel. Happy to collapse the two seqnums if you think it's better that way.

Up to you. We will use one seqnum with #2559 and the existing globalSeqNum logic will suffice.


ingest.go line 963 at r5 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

I made this comment elsewhere including in the DiskFIleNum PR, that the correct name for it should be BackingFileNum because we use it to reference FileBackings and we already use it in FileBacking even if an sstable is in shared.Storage and not in the provider. Which is to say I fully agree with you and that this type is wrongly tagged. I just think we should do it in a separate PR because it'll be a wholesale rename of the type and a lot of comment edits.

And since we're creating filenums in the provider that the metadatas will now reference, we'll need to do the conversion to DiskFileNum (which will soon be BackingFileNum, and mentally we should treat it as such) somewhere anyway between here and InitProviderBacking (and also provider.AttachSharedObjects which specifically asks for DiskFileNums to attach to).

You are right, and I was not thinking clearly. DiskFileNum is the filenum that we use with the Provider, and whether it maps to local fs, or something else is the provider's responsibility. And yes, it could use a better name.


ingest.go line 74 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

I didn't realize that all shared sstables are virtual—if the underlying sst's bounds are narrower than the KV range, could we use the shared sstable directly without marking it as "virtual," but continuing to retrieve the object from the object provider.

Hmm, given that the objstorage.Provider will map this to fileNum, not virtualizing should be easy to do in this case.


ingest.go line 76 at r7 (raw file):
Glad you raised this. I remember a past discussion in which I had glibly claimed that doing such inflation was fine, since most sstables would fall into a single CRDB range, and we can tolerate some amount of rewriting. Maybe that's ok as a starting point, but we will need to fix this before serious usage: we could have workloads where all ranges in the cluster are say 64MB, because of load-based splits, and LSMs with all levels populated (so L6 with 128MB files), and we don't want rewrites in that case.

The RFC says

Shared sstables do not span range boundaries. Note that this is
best-effort behavior and does not eliminate the need to apply a key
span constraint at iteration time (due to range splits).

But I don't know if this is worth doing -- it introduces more metadata coordination between CRDB and Pebble (to virtualize at split time) and potentially unnecessary code complexity. Also, it does not easily address the problem for workloads with load-based splits where there could be lots of splits and merges (say for various parts of the keyspace the workload has 1h of high load and 1h of no load and this pattern repeats). Ideally we don't want such splits and merges to result in excessive compaction activity.

I made a simple example with a file of size 100 written by N1. N1 shares it with N2 and retains an interest in only 20 out of 100, while N2 is interested in 80. Eventually N1's interest declines to 10, and N2's to 20 and no one else is sharing (say there were further splits and the ranges got transferred to a different region/AZ).
The intent here was to see if we can encode this information in the references so that every node with a reference can periodically scan all the references (say every few hours), sum the useful bytes in the file and use that for compaction prioritization. This would avoid the need for explicit communication.
If references are not updated, then they encode what the node was interested in when it created the ref. So, ref(N1).bytes=100, and ref(N2).bytes=80. If N1 sums its current interest (when it drops to 10) with ref(N2).bytes, it gets 90. Similarly, N2 sums 20 + 100 = 120. Since the file size is 100, these sums are clearly not representing the actual waste. We could fix this by updating the reference whenever a node reduces FileBacking.VirtualizedSize.

@RaduBerinde

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: 2 of 13 files reviewed, 20 unresolved discussions (waiting on @bananabrick, @itsbilal, @RaduBerinde, and @sumeerbhola)


ingest.go line 311 at r7 (raw file):

		})
	}
	sharedObjMetas, err := provider.AttachSharedObjects(sharedObjs)

this is where the references will be durably persisted, right? in that case for shared files this is behaving a little more like ingestLoad and ingestLink. should the attachment be deferred until ingestLink?


ingest.go line 1131 at r7 (raw file):

			// flushable ingests and need to wait synchronously. Either remove this
			// caveat by fleshing out flushable ingest logic to also account for these
			// cases, or remove this TODO.

unfortunately at the moment it appears just about all snapshot ingests overlap the memtable (cockroachdb/cockroach#99273) which will mean we'll need to wait for the flush during all snapshot ingests. hopefully KV will be able to remove that overlap in this release, otherwise we might want to look for creative options to avoid holding up the commit pipeline. @sumeerbhola suggested waiting for the flush before entering the commit pipeline. in cockroach the fact that the Replica performing the ingest is blocked waiting on the ingest should ensure we won't overlap the memtable once we do enter the pipeline.


ingest.go line 1274 at r7 (raw file):

func (d *DB) excise(
	exciseSpan KeyRange, m *fileMetadata, ve *versionEdit, level int,
) ([]manifest.NewFileEntry, error) {

this function both mutates ve and returns the slice of new file entries? do we use the returned new file entries somewhere?


ingest.go line 1277 at r7 (raw file):

d.cmp(exciseSpan.End, m.Smallest.UserKey) < 0

should this be <=? I think exciseSpan.End is exclusive, but m.Smallest.UserKey is inclusive

nit: could use

if v := d.cmp(exciseSpan.Start, m.Largest.UserKey); v > 0 || (m.Largest.IsExclusiveSentinel() && v == 0) || ...

to perform the exciseSpan.Start, m.Largest.UserKey comparison once


ingest.go line 1295 at r7 (raw file):

	backingTableCreated := false
	// Create a file to the left of the excise span, if necessary.
	// The bounds of this file will be [m.Smallest, exciseSpan.Start].

this code appears to find a tighter end boundary than exciseSpan.Start.

can you remind me why you decided to read the table to look up tight bounds instead of using the laxer exciseSpan.Start bound (or a key computed with Comparer.Separator?

Copy link
Member Author

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

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

TFTRs!

Reviewable status: 1 of 16 files reviewed, 16 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


ingest.go line 87 at r5 (raw file):

Previously, sumeerbhola wrote…

Up to you. We will use one seqnum with #2559 and the existing globalSeqNum logic will suffice.

Done.


ingest.go line 43 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: s/UserKeyRange/KeyRange/

Done.


ingest.go line 48 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: s/UserKeyRange/KeyRange/

Done.


ingest.go line 74 at r7 (raw file):

Previously, sumeerbhola wrote…

Hmm, given that the objstorage.Provider will map this to fileNum, not virtualizing should be easy to do in this case.

Yeah, we could get away with that. We'd just need to pass this fact over in SharedSSTMeta because if virtual is false, we won't do the read-time bounds-enforcement and end up exposing keys that are outside of bounds.


ingest.go line 76 at r7 (raw file):

Previously, sumeerbhola wrote…

Glad you raised this. I remember a past discussion in which I had glibly claimed that doing such inflation was fine, since most sstables would fall into a single CRDB range, and we can tolerate some amount of rewriting. Maybe that's ok as a starting point, but we will need to fix this before serious usage: we could have workloads where all ranges in the cluster are say 64MB, because of load-based splits, and LSMs with all levels populated (so L6 with 128MB files), and we don't want rewrites in that case.

The RFC says

Shared sstables do not span range boundaries. Note that this is
best-effort behavior and does not eliminate the need to apply a key
span constraint at iteration time (due to range splits).

But I don't know if this is worth doing -- it introduces more metadata coordination between CRDB and Pebble (to virtualize at split time) and potentially unnecessary code complexity. Also, it does not easily address the problem for workloads with load-based splits where there could be lots of splits and merges (say for various parts of the keyspace the workload has 1h of high load and 1h of no load and this pattern repeats). Ideally we don't want such splits and merges to result in excessive compaction activity.

I made a simple example with a file of size 100 written by N1. N1 shares it with N2 and retains an interest in only 20 out of 100, while N2 is interested in 80. Eventually N1's interest declines to 10, and N2's to 20 and no one else is sharing (say there were further splits and the ranges got transferred to a different region/AZ).
The intent here was to see if we can encode this information in the references so that every node with a reference can periodically scan all the references (say every few hours), sum the useful bytes in the file and use that for compaction prioritization. This would avoid the need for explicit communication.
If references are not updated, then they encode what the node was interested in when it created the ref. So, ref(N1).bytes=100, and ref(N2).bytes=80. If N1 sums its current interest (when it drops to 10) with ref(N2).bytes, it gets 90. Similarly, N2 sums 20 + 100 = 120. Since the file size is 100, these sums are clearly not representing the actual waste. We could fix this by updating the reference whenever a node reduces FileBacking.VirtualizedSize.

@RaduBerinde

Good point. Maybe a good solution here is to use the virtual sstable size as the underlying backing sstable size as well. That way our math for the number of non-live bytes starts off at 0 and we don't give special priority right off the bat to cleaning away references to foreign sstables (which might be living in a cheaper tier of storage anyway).

Important to note that for progressive restore we would need to prioritize these sstables for compaction anyway. But we can probably do that with a new compaction type (a "rewrite" compaction or so) that kicks in when we need to move things from SharedStorage to vfs.FS, instead of messing with existing compaction scoring which should be agnostic to that preference (because not every deployment will need to prioritize moving things from SharedStorage to a closer SharedStorage or to a vfs.FS).


ingest.go line 84 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

similarly to @sumeerbhola's comment about rangedels, i don't think there's any requirement for separate sequence numbers here:

// Additionally, internal range keys at the same sequence number have subtle
// mechanics:
//   - RANGEKEYSETs shadow RANGEKEYUNSETs of the same suffix.
//   - RANGEKEYDELs only apply to keys at lower sequence numbers.

Once coalesced, for each unique fragment bounds, each suffix only appears once within RANGEKYSET or RANGEKEYUNSET. RANGEKEYDELs can safely share the same sequence number and not affect anything other than L6 range keys.

Also, since point keys and range keys have parallel existences, the same sequence number can be used for point keys and range keys.

Done.


ingest.go line 120 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: are the changes to this func intentional?

Oops. Done.


ingest.go line 299 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: should this also be pre-allocated to len(shared) capacity?

Done.


ingest.go line 311 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

this is where the references will be durably persisted, right? in that case for shared files this is behaving a little more like ingestLoad and ingestLink. should the attachment be deferred until ingestLink?

Good point. Done.


ingest.go line 365 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

what if f is exclusive in the end boundary? that seems like it must be possible if not probable, in which case the user key of the exclusive bound does not fall within exciseSpan.

Done. Good catch!


ingest.go line 854 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

is this true? Even without shared storage, we want to use IngestAndExcise for snapshot reception, no?

It's not. Removed. Though the question of when we decide to call this method in Cockroach is still a bit of an open question (do we want to "pilot" it with disagg storage or do we want to roll it out more widely rightaway?)


ingest.go line 1131 at r7 (raw file):

@sumeerbhola suggested waiting for the flush before entering the commit pipeline. in cockroach the fact that the Replica performing the ingest is blocked waiting on the ingest should ensure we won't overlap the memtable once we do enter the pipeline.

This sounds like a good idea, but we'd still need a second check within the pipeline for concurrency reasons right? Like you said we won't apply any operations on the Replica until the ingestion is unblocked, but Pebble should retain a second check within the pipeline in case this is violated.


ingest.go line 1274 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

this function both mutates ve and returns the slice of new file entries? do we use the returned new file entries somewhere?

Yeah, we use the returned slice to adjust level metrics in the caller. This was a suggestion from Radu higher up in this PR as it cleaned up some code in the caller (ingestApply). Though we could also refactor ingestApply to do all metric calculations once we have a complete ve, which would remove the need to return this.


ingest.go line 1277 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

d.cmp(exciseSpan.End, m.Smallest.UserKey) < 0

should this be <=? I think exciseSpan.End is exclusive, but m.Smallest.UserKey is inclusive

nit: could use

if v := d.cmp(exciseSpan.Start, m.Largest.UserKey); v > 0 || (m.Largest.IsExclusiveSentinel() && v == 0) || ...

to perform the exciseSpan.Start, m.Largest.UserKey comparison once

Done.


ingest.go line 1295 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

this code appears to find a tighter end boundary than exciseSpan.Start.

can you remind me why you decided to read the table to look up tight bounds instead of using the laxer exciseSpan.Start bound (or a key computed with Comparer.Separator?

Comment updated. And yeah I found a tight user-key bound because in my testing with the prototype I saw a lot of cases like these:

  1. Wide memtable gets flushed, with some local and some table keys
  2. Another wide memtable gets flushed, with the local keys sitting in between the local and table keys from 1)
  3. A bunch more of the above ingestions, with table keys occasionally slotting in between other ranges' table keys.

If we just created vsstables with loose bounds without confirming if there are any keys within those bounds, we'll likely end up with a lot of ghost vsstables that don't contain any keys - forcing us to needlessly open vsstables that turn out to be empty. It can also have an effect on compaction picking / read amp in higher levels because we will think that that level has an overlapping file even though it doesn't.

Maybe that's not too much of a problem if we are unlikely to iterate from one range to another anyway (which would be the only case where we'd try reading empty vsstables), and if our vsst compaction picking heuristics are good enough at compacting tiny vsstables.

@itsbilal itsbilal force-pushed the excise-operation branch 4 times, most recently from 0500df4 to 5057eb4 Compare May 29, 2023 23:48
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 6 of 11 files at r8, 4 of 6 files at r9, all commit messages.
Reviewable status: 11 of 16 files reviewed, 18 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @RaduBerinde)


compaction.go line 2566 at r5 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Good point. In my testing it fires pretty regularly, and once we wire up the metamorphic test we should see many background errors with this case. I'm writing an end-to-end test in a different branch, and I'll try to add an assertion there that we encountered at least one call to EventListener.BackgroundError.

sounds good


compaction.go line 1339 at r9 (raw file):

	) (keyspan.FragmentIterator, error) {
		iter, rangeDelIter, err := newIters(context.Background(), f.FileMetadata,
			&IterOptions{level: l}, internalIterOpts{bytesIterated: &c.bytesIterated})

was not passing the level an oversight? Is this needed for NewFixedSeqnumRangeDelIter?


ingest.go line 74 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Yeah, we could get away with that. We'd just need to pass this fact over in SharedSSTMeta because if virtual is false, we won't do the read-time bounds-enforcement and end up exposing keys that are outside of bounds.

nit: if this is a TODO for a later PR can we track it somewhere?


ingest.go line 76 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Good point. Maybe a good solution here is to use the virtual sstable size as the underlying backing sstable size as well. That way our math for the number of non-live bytes starts off at 0 and we don't give special priority right off the bat to cleaning away references to foreign sstables (which might be living in a cheaper tier of storage anyway).

Important to note that for progressive restore we would need to prioritize these sstables for compaction anyway. But we can probably do that with a new compaction type (a "rewrite" compaction or so) that kicks in when we need to move things from SharedStorage to vfs.FS, instead of messing with existing compaction scoring which should be agnostic to that preference (because not every deployment will need to prioritize moving things from SharedStorage to a closer SharedStorage or to a vfs.FS).

+1 to not messing with compaction scoring for progressive restore.


ingest.go line 365 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Done. Good catch!

It seems we are always doing this containment check when we have InternalKeys, in which case we should just define Contains as taking an InternalKey param and move all this logic in there. Not that it matters, but it will also save an unnecessary key comparison since we don't need to compare End and f.Largest.UserKey twice.


ingest.go line 854 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

It's not. Removed. Though the question of when we decide to call this method in Cockroach is still a bit of an open question (do we want to "pilot" it with disagg storage or do we want to roll it out more widely rightaway?)

IMO, its easier to "pilot" it with disagg storage, since it reduce the number of times we need to evaluate the performance implication of this change.


ingest.go line 1131 at r7 (raw file):

This sounds like a good idea, but we'd still need a second check within the pipeline for concurrency reasons right?

Yep. This is just about reducing the probability of waiting for the flush in the commit pipeline. We could enable this behavior with ingest options. There would be some coordination to prevent N ingests all flushing memtables -- they'd all wait in the same place and after duration D, if the memtable was not naturally flushed, a flush would be done, where D is perhaps a DB option (to pace the maximum flush rate).


ingest.go line 1277 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Done.

Can you make this a method on excise span? It's easier to read when its abstracted out, and also more amenable to narrow unit tests. And I think we need some such unit tests, since its easy for all of us to miss/introduce such bugs when reading/writing code.


ingest.go line 1295 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Comment updated. And yeah I found a tight user-key bound because in my testing with the prototype I saw a lot of cases like these:

  1. Wide memtable gets flushed, with some local and some table keys
  2. Another wide memtable gets flushed, with the local keys sitting in between the local and table keys from 1)
  3. A bunch more of the above ingestions, with table keys occasionally slotting in between other ranges' table keys.

If we just created vsstables with loose bounds without confirming if there are any keys within those bounds, we'll likely end up with a lot of ghost vsstables that don't contain any keys - forcing us to needlessly open vsstables that turn out to be empty. It can also have an effect on compaction picking / read amp in higher levels because we will think that that level has an overlapping file even though it doesn't.

Maybe that's not too much of a problem if we are unlikely to iterate from one range to another anyway (which would be the only case where we'd try reading empty vsstables), and if our vsst compaction picking heuristics are good enough at compacting tiny vsstables.

These are compelling reasons. Could you add a code comment to that effect.


ingest.go line 1606 at r9 (raw file):

				// L6, or sharedLevelsStart - 1 if we have shared files.
				if len(lr.sharedMeta) > 0 {
					f.Level = sharedLevelsStart - 1

If the ingest does not have shared files (say due to AddSSTable in CRDB), and we ingest a local file into a shared level won't we hit the following error? Should this decision be a property of the DB and not the individual ingest?

// If visitSharedFile is not nil, ScanInternal iterates in skip-shared iteration
// mode. In this iteration mode, sstables in levels L5 and L6 are skipped, and
// their metadatas truncated to [lower, upper) and passed into visitSharedFile.
// ErrInvalidSkipSharedIteration is returned if visitSharedFile is not nil and an
// sstable in L5 or L6 is found that is not in shared storage according to
// provider.IsShared.

itsbilal added a commit to itsbilal/pebble that referenced this pull request May 31, 2023
Currently, if we identify boundary overlap in a level
during ingest target level calculation, but no data overlap,
we are forced to find a target level above the file we saw
the overlap with (if we can't fall below it, such as if the
existing file is in L6, which happens commonly).

This change takes advantage of virtual sstables to split
existing sstables into two virtual sstables when an ingested
sstable would be able to go into the same level had the sstables
been split that way to begin with. Doing this split reduces a
lot of write-amp as it avoids us from having to compact the
newly-ingested sstable with the sstable it boundary-overlapped with.

Biggest part of cockroachdb#1683. First commit is cockroachdb#2538, which this shares
a lot of logic with (mostly just the excise function).
Copy link
Member

@RaduBerinde RaduBerinde 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: 11 of 16 files reviewed, 22 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @sumeerbhola)


ingest.go line 1277 at r7 (raw file):

Previously, sumeerbhola wrote…

Can you make this a method on excise span? It's easier to read when its abstracted out, and also more amenable to narrow unit tests. And I think we need some such unit tests, since its easy for all of us to miss/introduce such bugs when reading/writing code.

I agree, I saw the same logic reproduced many places.. Could also be an IntersectsSpan method onFileMetadata


ingest.go line 65 at r9 (raw file):

}

// ingestLoad1Shared loads the fileMetadata for one shared sstable.

[nit] maybe it would help to mention the sequence number rewrite here


ingest.go line 82 at r9 (raw file):

		meta.LargestRangeKey = sm.LargestRangeKey
		switch sm.Level {
		case 5:

[nit, maybe suggestion for future clean-up] we have logic around these L5/L6 sequence numbers in a few places, I'm wondering if it would help to factor it out into some helpers that are all in the same place.

For example, this switch can be moved to a function that is given a level and returns a sequence number (and panics internally if the level is invalid), and this code would become

seq := sharedMetaSeqNum(sm.Level)
meta.SmallestRangeKey.SetSeqNum(seq)
...

ingest.go line 353 at r9 (raw file):

		if !exciseSpan.Contains(cmp, f.Smallest.UserKey) || !(exciseSpan.Contains(cmp, f.Largest.UserKey) ||
			(cmp(exciseSpan.End, f.Largest.UserKey) == 0 && f.Largest.IsExclusiveSentinel())) {
			return errors.New("pebble: shared file outside of excise span")

Should this be an AssertionFailed?


ingest.go line 1640 at r9 (raw file):

		//
		// Note that since there's a `defer d.mu.Unlock()` above, we need to relock
		// the db mutex before we return an error.

This is fragile, consider using defer

err := func() error {
  d.mu.Unlock()
  defer d.mu.Lock()
  ...
}()`

Copy link
Member Author

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

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

TFTRs! Flushing updates so far, only remaining big piece is the flush/memtable overlap wait before we go into the commit pipeline. Will knock that out very soon.

Reviewable status: 8 of 16 files reviewed, 18 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


compaction.go line 1339 at r9 (raw file):

Previously, sumeerbhola wrote…

was not passing the level an oversight? Is this needed for NewFixedSeqnumRangeDelIter?

Yeah, it was an oversight that got detected in the testing PR. Also now that we're only using one seqnum per level/sstable, we need NewFixedSeqnumRangeDelIter less as our existing sequence number fixing logic for globally-seqnum'd sstables works as it always has. But we can't just sub NewFixedSeqnumRangeDelIter with NewRawRangeDelIter yet because the latter won't elide rangedels at the same sequence number.


ingest.go line 74 at r7 (raw file):

Previously, sumeerbhola wrote…

nit: if this is a TODO for a later PR can we track it somewhere?

Filed #2583 to track this.


ingest.go line 76 at r7 (raw file):

Previously, sumeerbhola wrote…

+1 to not messing with compaction scoring for progressive restore.

Added the short-term solution for now. Will write up a follow-up issue for the more general case of compaction heuristics around shared sstables.


ingest.go line 365 at r7 (raw file):

Previously, sumeerbhola wrote…

It seems we are always doing this containment check when we have InternalKeys, in which case we should just define Contains as taking an InternalKey param and move all this logic in there. Not that it matters, but it will also save an unnecessary key comparison since we don't need to compare End and f.Largest.UserKey twice.

Done.


ingest.go line 854 at r7 (raw file):

Previously, sumeerbhola wrote…

IMO, its easier to "pilot" it with disagg storage, since it reduce the number of times we need to evaluate the performance implication of this change.

Agreed. I'll keep that in mind for the KV replication change (cockroachdb/cockroach#103028 ).


ingest.go line 1277 at r7 (raw file):

Previously, RaduBerinde wrote…

I agree, I saw the same logic reproduced many places.. Could also be an IntersectsSpan method onFileMetadata

Done.


ingest.go line 1295 at r7 (raw file):

Previously, sumeerbhola wrote…

These are compelling reasons. Could you add a code comment to that effect.

Done.


ingest.go line 82 at r9 (raw file):

Previously, RaduBerinde wrote…

[nit, maybe suggestion for future clean-up] we have logic around these L5/L6 sequence numbers in a few places, I'm wondering if it would help to factor it out into some helpers that are all in the same place.

For example, this switch can be moved to a function that is given a level and returns a sequence number (and panics internally if the level is invalid), and this code would become

seq := sharedMetaSeqNum(sm.Level)
meta.SmallestRangeKey.SetSeqNum(seq)
...

Done. This makes sense to do now that we have only one seqnum per level. The original design had a different seqnum for different kinds within a level.


ingest.go line 353 at r9 (raw file):

Previously, RaduBerinde wrote…

Should this be an AssertionFailed?

Done.


ingest.go line 1606 at r9 (raw file):

Previously, sumeerbhola wrote…

If the ingest does not have shared files (say due to AddSSTable in CRDB), and we ingest a local file into a shared level won't we hit the following error? Should this decision be a property of the DB and not the individual ingest?

// If visitSharedFile is not nil, ScanInternal iterates in skip-shared iteration
// mode. In this iteration mode, sstables in levels L5 and L6 are skipped, and
// their metadatas truncated to [lower, upper) and passed into visitSharedFile.
// ErrInvalidSkipSharedIteration is returned if visitSharedFile is not nil and an
// sstable in L5 or L6 is found that is not in shared storage according to
// provider.IsShared.

We won't hit this because we already moved the file to shared storage in ingestLink (see the PreferSharedStorage: true there).


ingest.go line 1640 at r9 (raw file):

Previously, RaduBerinde wrote…

This is fragile, consider using defer

err := func() error {
  d.mu.Unlock()
  defer d.mu.Lock()
  ...
}()`

Done (removed entirely due to bug mentioned elsewhere, see comment)

Copy link
Member

@RaduBerinde RaduBerinde 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: 8 of 16 files reviewed, 19 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @sumeerbhola)


ingest.go line 1427 at r4 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

By correct you mean that they're calculated accurately, or that they just obey some common invariants like Largest > Smallest ? I added meta.Validate calls for the latter, but the former would sort of require re-running this logic or something like it to verify. I'm gonna give this more thought though.

That they're accurate. That e.g. SmallestPoint is the first key returned by a point iterator, etc. Feels like it could be useful to check e.g. when opening an SST file (in invariants mode, maybe with a random probability if it's expensive).

The logic would be along the lines of using some iterators to find the first and last point and range keys and then checking all the fields.

Not for this PR of course, maybe we should just file an issue for now?


ingest.go line 111 at r10 (raw file):

		meta.SmallestSeqNum = seqNum
		meta.LargestSeqNum = seqNum
		meta.ExtendPointKeyBounds(opts.Comparer.Compare, meta.SmallestPointKey, meta.LargestPointKey)

[nit] add a comment saying Initialize the bounds (I was initially confused by the "extend" part, but we're really initializing it)

Copy link
Member Author

@itsbilal itsbilal 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: 8 of 16 files reviewed, 18 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


ingest.go line 1427 at r4 (raw file):

Previously, RaduBerinde wrote…

That they're accurate. That e.g. SmallestPoint is the first key returned by a point iterator, etc. Feels like it could be useful to check e.g. when opening an SST file (in invariants mode, maybe with a random probability if it's expensive).

The logic would be along the lines of using some iterators to find the first and last point and range keys and then checking all the fields.

Not for this PR of course, maybe we should just file an issue for now?

Thanks for prompting. Filed #2593 to track this.


ingest.go line 1131 at r7 (raw file):

Previously, sumeerbhola wrote…

This sounds like a good idea, but we'd still need a second check within the pipeline for concurrency reasons right?

Yep. This is just about reducing the probability of waiting for the flush in the commit pipeline. We could enable this behavior with ingest options. There would be some coordination to prevent N ingests all flushing memtables -- they'd all wait in the same place and after duration D, if the memtable was not naturally flushed, a flush would be done, where D is perhaps a DB option (to pace the maximum flush rate).

Trying to implement this change, I realized that we can't really rotate the memtable without entering the commit pipeline temporarily anyway (and grabbing the commit pipeline mutex and the DB mutex). We'd need to do that if we overlap with the mutable memtable, which I imagine will be the common case anyway. And just grabbing the mutex is still easier than adding a batch to the pipeline to track our ingestion (or at least the AllocateSeqNum call), so that would still be an improvement over the status quo.

Is it still worth to do this change in the same PR? I also realized we have #2112 filed to track just this fix. I'm good with completing it through in one PR if we expect the KV issue to be a significant blocker.


ingest.go line 111 at r10 (raw file):

Previously, RaduBerinde wrote…

[nit] add a comment saying Initialize the bounds (I was initially confused by the "extend" part, but we're really initializing it)

Done.

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 3 files at r7, 1 of 6 files at r9, 2 of 4 files at r10, 1 of 1 files at r11, all commit messages.
Reviewable status: 13 of 16 files reviewed, 18 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @RaduBerinde)


compaction.go line 1339 at r9 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Yeah, it was an oversight that got detected in the testing PR. Also now that we're only using one seqnum per level/sstable, we need NewFixedSeqnumRangeDelIter less as our existing sequence number fixing logic for globally-seqnum'd sstables works as it always has. But we can't just sub NewFixedSeqnumRangeDelIter with NewRawRangeDelIter yet because the latter won't elide rangedels at the same sequence number.

I didn't understand the "we can't just sub ..." part given we do

func (r *Reader) NewRawRangeDelIter() (keyspan.FragmentIterator, error) {
	return r.NewFixedSeqnumRangeDelIter(r.Properties.GlobalSeqNum)
}

ingest.go line 854 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Agreed. I'll keep that in mind for the KV replication change (cockroachdb/cockroach#103028 ).

For the metamorphic test will we make this operation be metamorphic with a batch consisting of a RANGEDEL for the excise span followed by the behavior akin to ingestUsingApply?


ingest.go line 1131 at r7 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Trying to implement this change, I realized that we can't really rotate the memtable without entering the commit pipeline temporarily anyway (and grabbing the commit pipeline mutex and the DB mutex). We'd need to do that if we overlap with the mutable memtable, which I imagine will be the common case anyway. And just grabbing the mutex is still easier than adding a batch to the pipeline to track our ingestion (or at least the AllocateSeqNum call), so that would still be an improvement over the status quo.

Is it still worth to do this change in the same PR? I also realized we have #2112 filed to track just this fix. I'm good with completing it through in one PR if we expect the KV issue to be a significant blocker.

As discussed in the sync, fine to do it later.


ingest.go line 1606 at r9 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

We won't hit this because we already moved the file to shared storage in ingestLink (see the PreferSharedStorage: true there).

So all the local paths are being moved to shared storage via PreferSharedStorage? In the CRDB range snapshot case they will typically not get ingested into L5 or L6 since there will be foreign ssts being ingested there, so have we unnecessarily placed them in shared storage?
We seem to have an interface mismatch here for ingests since we don't know what level these will get ingested in at the time of linking them.

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: 13 of 16 files reviewed, 20 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @RaduBerinde)


ingest.go line 1606 at r9 (raw file):

Previously, sumeerbhola wrote…

So all the local paths are being moved to shared storage via PreferSharedStorage? In the CRDB range snapshot case they will typically not get ingested into L5 or L6 since there will be foreign ssts being ingested there, so have we unnecessarily placed them in shared storage?
We seem to have an interface mismatch here for ingests since we don't know what level these will get ingested in at the time of linking them.

Looking again at this, given that we are determining the level simply based on whether there are shared files or not, and not based on whether there is a shared file below that overlaps, we could use this same decision in setting the value of PreferSharedStorage.


ingest.go line 352 at r11 (raw file):

			return errors.AssertionFailedf("pebble: shared file outside of excise span")
		}
	}

where do we verify that the shared metas for the same level do not overlap?


ingest.go line 1602 at r11 (raw file):

		var err error
		if sharedIdx >= 0 {
			f.Level = int(lr.sharedLevels[sharedIdx])

is this Level from SharedSSTMeta.Level? If yes, we should verify this before getting to this stage.

Copy link
Member Author

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

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

TFTRs!

Dismissed @jbowens and @sumeerbhola from a discussion.
Reviewable status: 13 of 16 files reviewed, 19 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


compaction.go line 1339 at r9 (raw file):

Previously, sumeerbhola wrote…

I didn't understand the "we can't just sub ..." part given we do

func (r *Reader) NewRawRangeDelIter() (keyspan.FragmentIterator, error) {
	return r.NewFixedSeqnumRangeDelIter(r.Properties.GlobalSeqNum)
}

Ah right, I had forgotten that we were doing the collapse in both cases. Cleaned up the iterator stack to not rely on FixedSeqnumRangeDelIter (and made the same change for range keys/points).


ingest.go line 854 at r7 (raw file):

Previously, sumeerbhola wrote…

For the metamorphic test will we make this operation be metamorphic with a batch consisting of a RANGEDEL for the excise span followed by the behavior akin to ingestUsingApply?

That might not be sufficient for metamorphic test equivalence/determinism as we might have snapshots open that could read the excised keys and see different results between the excise and rangedel cases. But yes if we fallback to doing a RANGEDEL+RANGEKEYDEL if there's a snapshot open, then this would work well.


ingest.go line 1606 at r9 (raw file):

Previously, sumeerbhola wrote…

Looking again at this, given that we are determining the level simply based on whether there are shared files or not, and not based on whether there is a shared file below that overlaps, we could use this same decision in setting the value of PreferSharedStorage.

That's right. I am still a little wary of passing false in any case to PreferSharedStorage or moving the level-determination logic here, as we haven't worked on move compactions yet, or at least come up with a good way to do move compactions to L5/L6 more efficiently than doing regular compactions. It'll just result in more instances of us not being able to fast-rebalance because there was one local file that got move-compacted into L5 and is now forcing regular replication when we call ScanInternal.

I will revisit all these PreferSharedStorage values when we have a move compaction implemented for local -> shared. But for now I think an unconditional true makes sense. I'll file an issue for implementing said move compaction and revisiting these PreferSharedStorage values.


ingest.go line 352 at r11 (raw file):

Previously, sumeerbhola wrote…

where do we verify that the shared metas for the same level do not overlap?

It would panic in logAndApply, but I just added a check in ingestSortAndVerify.


ingest.go line 1602 at r11 (raw file):

Previously, sumeerbhola wrote…

is this Level from SharedSSTMeta.Level? If yes, we should verify this before getting to this stage.

Yes, we set it in ingestLoad ourselves, it's why it's in the ingestLoadResult. Is there another check we should have? I moved the lookup in sharedLevels around to make it clearer.

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.

Reviewed 2 of 11 files at r8, 1 of 9 files at r12, all commit messages.
Reviewable status: 8 of 17 files reviewed, 20 unresolved discussions (waiting on @bananabrick, @itsbilal, @RaduBerinde, and @sumeerbhola)


ingest.go line 854 at r7 (raw file):

But yes if we fallback to doing a RANGEDEL+RANGEKEYDEL if there's a snapshot open, then this would work well.

We should be careful to make sure that we actually exercise IngestAndExcise if we do this. I suspect almost all the time some snapshot is open.


ingest.go line 50 at r12 (raw file):

func (k *KeyRange) Contains(cmp base.Compare, key InternalKey) bool {
	v := cmp(key.UserKey, k.End)
	return cmp(k.Start, key.UserKey) <= 0 && (v < 0 || (v == 0 && key.IsExclusiveSentinel()))

nit: because of the ordering of these conditions, i think we'll always perform both comparisons. if we flip the order, we can avoid the start key comparison if the end key comparison already precludes the possibility of overlap


ingest.go line 1212 at r12 (raw file):

		//
		// For shared sstables, we do not need to update sequence numbers. These
		// sequence numbers are already set in ingestLoad.

i've completely forgotten why we need these unique shared-sstable sequence numbers. when we're excising, there are local internal keys above the ingested shared sstable; we've excised them. we allocate n sequence numbers for n sstables. why couldn't we just use the local engine's sequence numbers for setting the global seq num for each sstable, ensuring that overlapping sstables are appropriately sequenced according to the level they're slotted into?


ingest.go line 1349 at r12 (raw file):

	// directly into the space occupied by them. We choose to incur the cost of
	// calculating tight bounds at this time instead of creating more work in the
	// future.

This I/O can also opportunistically be performed before acquiring the manifest lock, like the target-file size I/O can be as described in #2112. I suspect that could help p99 commit latency significantly in the presence of ingests. Maybe leave a TODO referencing #2112?

Copy link
Member Author

@itsbilal itsbilal 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: 8 of 17 files reviewed, 19 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


ingest.go line 854 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

But yes if we fallback to doing a RANGEDEL+RANGEKEYDEL if there's a snapshot open, then this would work well.

We should be careful to make sure that we actually exercise IngestAndExcise if we do this. I suspect almost all the time some snapshot is open.

Agreed


ingest.go line 50 at r12 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: because of the ordering of these conditions, i think we'll always perform both comparisons. if we flip the order, we can avoid the start key comparison if the end key comparison already precludes the possibility of overlap

Done.


ingest.go line 1212 at r12 (raw file):

Previously, jbowens (Jackson Owens) wrote…

i've completely forgotten why we need these unique shared-sstable sequence numbers. when we're excising, there are local internal keys above the ingested shared sstable; we've excised them. we allocate n sequence numbers for n sstables. why couldn't we just use the local engine's sequence numbers for setting the global seq num for each sstable, ensuring that overlapping sstables are appropriately sequenced according to the level they're slotted into?

The RFC called for two reserved seqnums in L5 (+ one for L6 for three in total), and based on our discussion in NYC we bumped that count up to 5 reserved seqnums. To make seqnum assignment easier at ingest time we just went with the reserved seqnum approach, but you're right that given the seqnum changes in this PR, we can just do seqnum assignment the old-fashioned way for shared sstables too (as long as they get lower seqnums than local sstables in the same ingestion) and we'll be okay. It's just that the seqnum changes are recent enough that the rest of the design (or even past PRs, like #2455, which really should have had the seqnum simplifiication and not this PR) don't account for it all.


ingest.go line 1349 at r12 (raw file):

Previously, jbowens (Jackson Owens) wrote…

This I/O can also opportunistically be performed before acquiring the manifest lock, like the target-file size I/O can be as described in #2112. I suspect that could help p99 commit latency significantly in the presence of ingests. Maybe leave a TODO referencing #2112?

Yeah that would make sense, there's an optimization to do as much of this work as possible before grabbing the manifest lock. Added the TODO.

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.

:lgtm: (I didn't look carefully at the tests)

Reviewed 5 of 9 files at r12, 1 of 1 files at r13, all commit messages.
Reviewable status: 13 of 17 files reviewed, 20 unresolved discussions (waiting on @bananabrick, @itsbilal, @jbowens, and @RaduBerinde)


ingest.go line 1606 at r9 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

That's right. I am still a little wary of passing false in any case to PreferSharedStorage or moving the level-determination logic here, as we haven't worked on move compactions yet, or at least come up with a good way to do move compactions to L5/L6 more efficiently than doing regular compactions. It'll just result in more instances of us not being able to fast-rebalance because there was one local file that got move-compacted into L5 and is now forcing regular replication when we call ScanInternal.

I will revisit all these PreferSharedStorage values when we have a move compaction implemented for local -> shared. But for now I think an unconditional true makes sense. I'll file an issue for implementing said move compaction and revisiting these PreferSharedStorage values.

Ack


ingest.go line 1602 at r11 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

Yes, we set it in ingestLoad ourselves, it's why it's in the ingestLoadResult. Is there another check we should have? I moved the lookup in sharedLevels around to make it clearer.

I was not clear in my comment. I didn't mean to move the sharedLevel logic around. I was saying that this if f.Level < sharedLevelsStart should be validated earlier. I added a comment for that.


ingest.go line 1212 at r12 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

The RFC called for two reserved seqnums in L5 (+ one for L6 for three in total), and based on our discussion in NYC we bumped that count up to 5 reserved seqnums. To make seqnum assignment easier at ingest time we just went with the reserved seqnum approach, but you're right that given the seqnum changes in this PR, we can just do seqnum assignment the old-fashioned way for shared sstables too (as long as they get lower seqnums than local sstables in the same ingestion) and we'll be okay. It's just that the seqnum changes are recent enough that the rest of the design (or even past PRs, like #2455, which really should have had the seqnum simplifiication and not this PR) don't account for it all.

Yes, it doesn't seem we need these reserved seqnums. That thinking predates the use of excising across all the levels.
Fine to do it later.


ingest.go line 308 at r13 (raw file):

			return ingestLoadResult{}, err
		}
		if m != nil {

why would m be nil when err is nil?

I was suggesting validating that shared[i].Level < sharedLevelsStart here. That is, the argument error checking should happen as early as possible.

This change adds an IngestAndExcise operation that does the below
additional things alongside a regular ingestion:

1) It ingests some SharedSSTMeta files, which are provider-backed
   sstables that could be owned by other nodes.
2) It excises existing sstables within the provided excise span (within
   which all sstables from 1 must fit) by creating new virtual sstables
   that exclude keys from the excise span.

While this change can be implemented independently of cockroachdb#2455, some of the
end-to-end tests in future changes will rely on both that and this.

Fixes cockroachdb#2520.
Copy link
Member Author

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

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

TFTR! The more interesting tests are in #2574 anyway.

Dismissed @sumeerbhola from a discussion.
Reviewable status: 10 of 17 files reviewed, 19 unresolved discussions (waiting on @bananabrick, @jbowens, @RaduBerinde, and @sumeerbhola)


ingest.go line 1606 at r9 (raw file):

Previously, sumeerbhola wrote…

Ack

Filed #2626.


ingest.go line 1602 at r11 (raw file):

Previously, sumeerbhola wrote…

I was not clear in my comment. I didn't mean to move the sharedLevel logic around. I was saying that this if f.Level < sharedLevelsStart should be validated earlier. I added a comment for that.

Done.


ingest.go line 308 at r13 (raw file):

Previously, sumeerbhola wrote…

why would m be nil when err is nil?

I was suggesting validating that shared[i].Level < sharedLevelsStart here. That is, the argument error checking should happen as early as possible.

Ah right, this is a leftover from the past where ingestLoad1Shared could return nil. Updated.

@itsbilal
Copy link
Member Author

Race test failure is #2613.

@itsbilal itsbilal merged commit a6e9021 into cockroachdb:master Jun 12, 2023
itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 10, 2023
Currently, if we identify boundary overlap in a level
during ingest target level calculation, but no data overlap,
we are forced to find a target level above the file we saw
the overlap with (if we can't fall below it, such as if the
existing file is in L6, which happens commonly).

This change takes advantage of virtual sstables to split
existing sstables into two virtual sstables when an ingested
sstable would be able to go into the same level had the sstables
been split that way to begin with. Doing this split reduces a
lot of write-amp as it avoids us from having to compact the
newly-ingested sstable with the sstable it boundary-overlapped with.

Biggest part of cockroachdb#1683. First commit is cockroachdb#2538, which this shares
a lot of logic with (mostly just the excise function).
itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 10, 2023
Currently, if we identify boundary overlap in a level
during ingest target level calculation, but no data overlap,
we are forced to find a target level above the file we saw
the overlap with (if we can't fall below it, such as if the
existing file is in L6, which happens commonly).

This change takes advantage of virtual sstables to split
existing sstables into two virtual sstables when an ingested
sstable would be able to go into the same level had the sstables
been split that way to begin with. Doing this split reduces a
lot of write-amp as it avoids us from having to compact the
newly-ingested sstable with the sstable it boundary-overlapped with.

Biggest part of cockroachdb#1683. First commit is cockroachdb#2538, which this shares
a lot of logic with (mostly just the excise function).
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.

db: implement IngestAndExcise operation for shared sstables
5 participants