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: split snapshot SSTables for mvcc keys into multiple SSTs #127997

Merged
merged 1 commit into from
Aug 14, 2024

Conversation

itsbilal
Copy link
Member

Previously, we'd only create one sstable for all mvcc keys in a range when ingesting a rebalance/recovery snapshot into Pebble. This increased write-amp in Pebble as more sstables would have to be compacted into it (or the sstable then split into smaller ones in Pebble), and had other consequences such as massive filter blocks in the large singular sstable.

This change adds a new cluster setting,
kv.snapshot_rebalance.max_sst_size, that sets the max size of the sstables containing user/mvcc keys in a range. If an sstable exceeds this size in multiSSTWriter, we roll over that sstable and create a new one.

Epic: CRDB-8471
Fixes: #67284

Release note (performance improvement): Reduce the write-amplification impact of rebalances by splitting snapshot sstable files into smaller ones before ingesting them into Pebble.

@itsbilal itsbilal requested review from a team and jbowens July 31, 2024 18:25
@itsbilal itsbilal requested a review from a team as a code owner July 31, 2024 18:25
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@itsbilal
Copy link
Member Author

This PR is actually missing the range key fragmenting piece. I'll change it to a draft PR as we'll need that.

@itsbilal itsbilal marked this pull request as draft July 31, 2024 18:37
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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @jbowens)


pkg/kv/kvserver/store_snapshot.go line 270 at r1 (raw file):

		// Finish the current SST, write to the file, and move to the next key
		// range.
		if err := msstw.finalizeSST(ctx); err != nil {

[nit] It would be nice if we would pass key to finalizeSST so it can assert that the sst's largest key is strictly smaller than key


pkg/kv/kvserver/store_snapshot.go line 285 at r1 (raw file):

		return errors.AssertionFailedf("client error: expected %s to fall in one of %s or %s", key, msstw.localKeySpans, msstw.mvccKeySpans)
	}
	if msstw.currSpanIsMVCCSpan() && msstw.maxSSTSize > 0 && msstw.currSST.DataSize > msstw.maxSSTSize {

How does this work if we already did a PutInternalRangeDelete or PutInternalRangeKey that ends after the current key? Feels like we need to keep track of that watermark and never split before it.


pkg/kv/kvserver/store_snapshot.go line 288 at r1 (raw file):

		// We're in an MVCC / user keys span, and the current sstable has exceeded
		// the max size for MVCC sstables that we should be creating. Split this
		// sstable into smaller ones. We do this by splitting the mvccKeySpan into

[nit] remove "into"


pkg/kv/kvserver/store_snapshot.go line 299 at r1 (raw file):

		newSpan := roachpb.Span{Key: currSpan.EndKey, EndKey: oldEndKey}
		msstw.mvccKeySpans = append(msstw.mvccKeySpans, newSpan)
		if msstw.currSpan < len(msstw.localKeySpans)+len(msstw.mvccKeySpans)-1 {

[nit] you can use slices.Insert(mvccKeySpans, curSpan, newSpan)

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 all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @RaduBerinde)


pkg/kv/kvserver/store_snapshot.go line 71 at r3 (raw file):

	"kv.snapshot_rebalance.max_sst_size",
	"maximum size of a rebalance or recovery SST size",
	32<<20, // 32 MB

nit: should we make this 128 MB to match our expectation that these tables will be ingested into L6?


pkg/kv/kvserver/store_snapshot.go line 150 at r3 (raw file):

	// having this bool to true also disables all range key fragmentation
	// and splitting of sstables in the mvcc span.
	skipRangeDelForMVCCSpan bool

nit: call it skipClearForMVCCSpanor the like so that it's not suggesting that we're writing a range deletion but not a range key deletion


pkg/kv/kvserver/store_snapshot.go line 176 at r3 (raw file):

		skipRangeDelForMVCCSpan: skipRangeDelForMVCCSpan,
	}
	if st != nil && !skipRangeDelForMVCCSpan {

do any callers actually pass in a nil st and is it possible to update the caller instead?


pkg/kv/kvserver/store_snapshot.go line 179 at r3 (raw file):

		// If skipRangeDelForMVCCSpan is true, we don't split the MVCC span across
		// multiple sstables, as addRangeDelForMVCCSpan could be called by the caller
		// at any time.

I didn't understand the meaning of this "addRangeDelForMVCCSpan could be called by the caller at any time."

Since we write range deletions even when performing an excise now, when wouldn't we add a range deletion?


pkg/kv/kvserver/store_snapshot.go line 231 at r3 (raw file):

		// we're writing.
		if !msstw.skipRangeDelForMVCCSpan {
			if err := msstw.currSST.ClearRawRange(

how does this interact with range key fragmentation since ClearRawRange also writes a RangeKeyDelete

@itsbilal
Copy link
Member Author

itsbilal commented Aug 2, 2024

TFTRs! I'll be incorporating the feedback soon. Might need to rework a decent chunk of my approach here as we can't fragment at any EngineKeys due to expectations from Pebble around range keys having suffixless keys at the external (ingestion) point, even though fragmentation within Pebble of range keys at suffixed keys is allowed.

That plus other instances of nuances means a lot of this PR is going to remain in flux for the next couple days, so I'd encourage holding off on more reviews until I've marked it as ready-for-review. Thanks!

itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 6, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 6, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to cockroachdb/pebble that referenced this pull request Aug 6, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
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! This is ready for a look now. I've filed cockroachdb/pebble#3829 based on a conversation with Jackson about how this change veers very close to some missing assertions around range key invariants in Pebble on the ingestion path.

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


pkg/kv/kvserver/store_snapshot.go line 270 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] It would be nice if we would pass key to finalizeSST so it can assert that the sst's largest key is strictly smaller than key

We get this sort of checking for free in the ingestSortAndVerify check on the ingestion path, but also I'm not aware if the sst writer keeps track of the largest key added to it. Should we add it still?


pkg/kv/kvserver/store_snapshot.go line 285 at r1 (raw file):

Previously, RaduBerinde wrote…

How does this work if we already did a PutInternalRangeDelete or PutInternalRangeKey that ends after the current key? Feels like we need to keep track of that watermark and never split before it.

Both of those methods have a panic if we call them with !skipClearForMVCCSpan. And in the skipClearForMVCCSpan case, we don't do sstable splits, so we're covered.


pkg/kv/kvserver/store_snapshot.go line 288 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] remove "into"

Done.


pkg/kv/kvserver/store_snapshot.go line 299 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] you can use slices.Insert(mvccKeySpans, curSpan, newSpan)

Done (unnecessary as this code got reworked)


pkg/kv/kvserver/store_snapshot.go line 71 at r3 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: should we make this 128 MB to match our expectation that these tables will be ingested into L6?

Done.


pkg/kv/kvserver/store_snapshot.go line 150 at r3 (raw file):

Previously, jbowens (Jackson Owens) wrote…

nit: call it skipClearForMVCCSpanor the like so that it's not suggesting that we're writing a range deletion but not a range key deletion

Done.


pkg/kv/kvserver/store_snapshot.go line 176 at r3 (raw file):

Previously, jbowens (Jackson Owens) wrote…

do any callers actually pass in a nil st and is it possible to update the caller instead?

Done. I don't think any callers were passing it in.


pkg/kv/kvserver/store_snapshot.go line 179 at r3 (raw file):

Previously, jbowens (Jackson Owens) wrote…

I didn't understand the meaning of this "addRangeDelForMVCCSpan could be called by the caller at any time."

Since we write range deletions even when performing an excise now, when wouldn't we add a range deletion?

Disaggregated storage is the only case where we wouldn't add range deletions.


pkg/kv/kvserver/store_snapshot.go line 231 at r3 (raw file):

Previously, jbowens (Jackson Owens) wrote…

how does this interact with range key fragmentation since ClearRawRange also writes a RangeKeyDelete

Done. Reworked this code so it only adds a point key rangedel, and we explicitly fragment the rangekeydel in the same fragmenter as all other range keys.

@itsbilal itsbilal marked this pull request as ready for review August 6, 2024 20:42
@itsbilal itsbilal requested a review from jbowens August 6, 2024 21:19
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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @jbowens)


pkg/kv/kvserver/store_snapshot.go line 270 at r1 (raw file):

Previously, itsbilal (Bilal Akhtar) wrote…

We get this sort of checking for free in the ingestSortAndVerify check on the ingestion path, but also I'm not aware if the sst writer keeps track of the largest key added to it. Should we add it still?

The largest key is readily available in the sst writer Metadata() once we finish 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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @jbowens and @RaduBerinde)


pkg/kv/kvserver/store_snapshot.go line 270 at r1 (raw file):

Previously, RaduBerinde wrote…

The largest key is readily available in the sst writer Metadata() once we finish it.

Ah right. Fixed!

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.

:lgtm: but I'd wait for an OK from Jackson too as I'm not familiar with much of this code.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @itsbilal and @jbowens)


pkg/kv/kvserver/store_snapshot.go line 209 at r6 (raw file):

	for i := range key.Keys {
		if err := msstw.currSST.PutInternalRangeKey(key.Start, key.End, key.Keys[i]); err != nil {
			panic("failed to put range key in sst")

[nit] print the error too


pkg/kv/kvserver/store_snapshot.go line 219 at r6 (raw file):

func (msstw *multiSSTWriter) currentSpan() roachpb.Span {
	if msstw.currSpan >= len(msstw.localKeySpans) {
		if msstw.currSpan >= len(msstw.localKeySpans)+len(msstw.mvccSSTSpans) {

[nit] maybe move this check inside currSpanIsMVCCSpan and use that method here


pkg/kv/kvserver/store_snapshot.go line 238 at r6 (raw file):

	newSST := storage.MakeIngestionSSTWriter(ctx, msstw.st, newSSTFile)
	msstw.currSST = newSST
	if msstw.currSpan < len(msstw.localKeySpans) || (!msstw.skipClearForMVCCSpan && msstw.currSpan <= len(msstw.localKeySpans)) {

if !curSpanIsMVCCSpan() ||


pkg/kv/kvserver/store_snapshot.go line 240 at r6 (raw file):

	if msstw.currSpan < len(msstw.localKeySpans) || (!msstw.skipClearForMVCCSpan && msstw.currSpan <= len(msstw.localKeySpans)) {
		// We're either in a local key span, or we're in the first MVCC sstable
		// span (before any splits). Add a RangeKeyDel for the whole MVCC span

"for the whole span. If this is the first MVCC span, we don't need to keep re-adding it.."


pkg/kv/kvserver/replica_sst_snapshot_storage_test.go line 362 at r6 (raw file):

	_, err = multiSSTWriter.Finish(ctx)
	require.NoError(t, err)
	require.Greater(t, len(scratch.SSTs()), len(keySpans))

[nit] Ideally we would check the contents of the ssts too - it could be a datadriven test where we input the keys written and then we output the contents of the resulting SSTs.

We should also test the use of ClearEngineRange and skipRangeDelForMVCCSpan=true

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: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @jbowens and @RaduBerinde)


pkg/kv/kvserver/store_snapshot.go line 209 at r6 (raw file):

Previously, RaduBerinde wrote…

[nit] print the error too

Done.


pkg/kv/kvserver/store_snapshot.go line 238 at r6 (raw file):

Previously, RaduBerinde wrote…

if !curSpanIsMVCCSpan() ||

Done.


pkg/kv/kvserver/store_snapshot.go line 240 at r6 (raw file):

Previously, RaduBerinde wrote…

"for the whole span. If this is the first MVCC span, we don't need to keep re-adding it.."

Done.


pkg/kv/kvserver/replica_sst_snapshot_storage_test.go line 362 at r6 (raw file):

Previously, RaduBerinde wrote…

[nit] Ideally we would check the contents of the ssts too - it could be a datadriven test where we input the keys written and then we output the contents of the resulting SSTs.

We should also test the use of ClearEngineRange and skipRangeDelForMVCCSpan=true

The skipRangeDelForMVCCSpan case disables splitting and is already tested in TestMultiSSTWriterAddLastSpan, at least. I updated this test to also verify the contents of the SSTs though!

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.

:lgtm:

Reviewed 1 of 2 files at r7, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @itsbilal and @RaduBerinde)


pkg/kv/kvserver/store_snapshot.go line 134 at r7 (raw file):

	currSST storage.SSTWriter
	// localKeySpans are key spans that are considered unsplittable across sstables.
	// mvccKeySpan can be split across multiple sstables if one of them exceeds

super nit: maybe precede this with explaining that localKeySpans define the range's range local key spans. this comment makes it seem like the defining characteristic is that we won't split them.

also, may precede the second sentence with "In contrast mvccKeySpan can..."

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: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @jbowens and @RaduBerinde)


pkg/kv/kvserver/store_snapshot.go line 134 at r7 (raw file):

Previously, jbowens (Jackson Owens) wrote…

super nit: maybe precede this with explaining that localKeySpans define the range's range local key spans. this comment makes it seem like the defining characteristic is that we won't split them.

also, may precede the second sentence with "In contrast mvccKeySpan can..."

Done.

@itsbilal itsbilal added backport-24.1.x Flags PRs that need to be backported to 24.1. backport-24.2.x Flags PRs that need to be backported to 24.2 labels Aug 14, 2024
Previously, we'd only create one sstable for all mvcc keys
in a range when ingesting a rebalance/recovery snapshot into
Pebble. This increased write-amp in Pebble as more sstables
would have to be compacted into it (or the sstable then split
into smaller ones in Pebble), and had other consequences
such as massive filter blocks in the large singular sstable.

This change adds a new cluster setting,
kv.snapshot_rebalance.max_sst_size, that sets the max size of the
sstables containing user/mvcc keys in a range. If an sstable exceeds
this size in multiSSTWriter, we roll over that sstable and create a
new one.

Epic: CRDB-8471
Fixes: cockroachdb#67284

Release note (performance improvement): Reduce the write-amplification
impact of rebalances by splitting snapshot sstable files into smaller ones
before ingesting them into Pebble.
@itsbilal
Copy link
Member Author

bors r=jbowens

@craig craig bot merged commit e5eb944 into cockroachdb:master Aug 14, 2024
22 of 23 checks passed
Copy link

blathers-crl bot commented Aug 14, 2024

Based on the specified backports for this PR, I applied new labels to the following linked issue(s). Please adjust the labels as needed to match the branches actually affected by the issue(s), including adding any known older branches.


Issue #67284: branch-release-24.1, branch-release-24.2.


🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

Copy link

blathers-crl bot commented Aug 14, 2024

Encountered an error creating backports. Some common things that can go wrong:

  1. The backport branch might have already existed.
  2. There was a merge conflict.
  3. The backport branch contained merge commits.

You might need to create your backport manually using the backport tool.


error creating merge commit from 93daeb4 to blathers/backport-release-24.1-127997: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict []

you may need to manually resolve merge conflicts with the backport tool.

Backport to branch 24.1.x failed. See errors above.


🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 14, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to itsbilal/pebble that referenced this pull request Aug 14, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to cockroachdb/pebble that referenced this pull request Aug 15, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to cockroachdb/pebble that referenced this pull request Aug 15, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to itsbilal/pebble that referenced this pull request Nov 6, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
itsbilal added a commit to cockroachdb/pebble that referenced this pull request Nov 7, 2024
Currently, we export MakeInternalKey but not MakeTrailer. This
change exports MakeTrailer so we can use it directly in
cockroachdb/cockroach#127997 .
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
backport-24.1.x Flags PRs that need to be backported to 24.1. backport-24.2.x Flags PRs that need to be backported to 24.2
Projects
None yet
Development

Successfully merging this pull request may close these issues.

kv/kvserver: split incoming snapshot user keys into multiple sstables
4 participants