Skip to content

Commit

Permalink
kvserver: split snapshot SSTables for mvcc keys into multiple SSTs
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
itsbilal committed Aug 14, 2024
1 parent a905886 commit 1c4ce42
Show file tree
Hide file tree
Showing 5 changed files with 391 additions and 67 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -522,6 +522,7 @@ go_test(
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_pebble//sstable",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_cockroachdb_redact//:redact",
"@com_github_dustin_go_humanize//:go-humanize",
Expand Down
127 changes: 122 additions & 5 deletions pkg/kv/kvserver/replica_sst_snapshot_storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
"golang.org/x/time/rate"
)
Expand Down Expand Up @@ -276,10 +278,12 @@ func TestMultiSSTWriterInitSST(t *testing.T) {
EndKey: roachpb.RKeyMax,
}
keySpans := rditer.MakeReplicatedKeySpans(&desc)
localSpans := keySpans[:len(keySpans)-1]
mvccSpan := keySpans[len(keySpans)-1]

msstw, err := newMultiSSTWriter(
ctx, cluster.MakeTestingClusterSettings(), scratch, keySpans, 0,
false, /* skipRangeDelForLastSpan */
ctx, cluster.MakeTestingClusterSettings(), scratch, localSpans, mvccSpan, 0,
false, /* skipRangeDelForMVCCSpan */
)
require.NoError(t, err)
_, err = msstw.Finish(ctx)
Expand Down Expand Up @@ -315,6 +319,117 @@ func TestMultiSSTWriterInitSST(t *testing.T) {
}
}

func buildIterForScratch(
t *testing.T, keySpans []roachpb.Span, scratch *SSTSnapshotStorageScratch,
) (storage.MVCCIterator, error) {
var openFiles []sstable.ReadableFile
for _, sstPath := range scratch.SSTs()[len(keySpans)-1:] {
f, err := vfs.Default.Open(sstPath)
require.NoError(t, err)
openFiles = append(openFiles, f)
}
mvccSpan := keySpans[len(keySpans)-1]

return storage.NewSSTIterator([][]sstable.ReadableFile{openFiles}, storage.IterOptions{
LowerBound: mvccSpan.Key,
UpperBound: mvccSpan.EndKey,
})
}

// TestMultiSSTWriterSize tests the effect of lowering the max size
// of sstables in a multiSSTWriter, and ensuring that the produced sstables
// are still correct.
func TestMultiSSTWriterSize(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
testRangeID := roachpb.RangeID(1)
testSnapUUID := uuid.Must(uuid.FromBytes([]byte("foobar1234567890")))
testLimiter := rate.NewLimiter(rate.Inf, 0)

cleanup, eng := newOnDiskEngine(ctx, t)
defer cleanup()
defer eng.Close()

sstSnapshotStorage := NewSSTSnapshotStorage(eng, testLimiter)
ref := sstSnapshotStorage.NewScratchSpace(testRangeID, testSnapUUID)
scratch := sstSnapshotStorage.NewScratchSpace(testRangeID, testSnapUUID)
settings := cluster.MakeTestingClusterSettings()

desc := roachpb.RangeDescriptor{
StartKey: roachpb.RKey("d"),
EndKey: roachpb.RKeyMax,
}
keySpans := rditer.MakeReplicatedKeySpans(&desc)
localSpans := keySpans[:len(keySpans)-1]
mvccSpan := keySpans[len(keySpans)-1]

// Make a reference msstw with the default size.
referenceMsstw, err := newMultiSSTWriter(ctx, settings, ref, localSpans, mvccSpan, 0, false)
require.NoError(t, err)
require.Equal(t, int64(0), referenceMsstw.dataSize)

for i := range localSpans {
require.NoError(t, referenceMsstw.Put(ctx, storage.EngineKey{Key: localSpans[i].Key}, []byte("foo")))
}

for i := 0; i < 100; i++ {
require.NoError(t, referenceMsstw.Put(ctx, storage.EngineKey{Key: roachpb.Key(append(desc.StartKey, byte(i)))}, []byte("foobarbaz")))
}
_, err = referenceMsstw.Finish(ctx)
require.NoError(t, err)

refIter, err := buildIterForScratch(t, keySpans, ref)
require.NoError(t, err)
defer refIter.Close()

MaxSnapshotSSTableSize.Override(ctx, &settings.SV, 100)

multiSSTWriter, err := newMultiSSTWriter(ctx, settings, scratch, localSpans, mvccSpan, 0, false)
require.NoError(t, err)
require.Equal(t, int64(0), multiSSTWriter.dataSize)

for i := range localSpans {
require.NoError(t, multiSSTWriter.Put(ctx, storage.EngineKey{Key: localSpans[i].Key}, []byte("foo")))
}

for i := 0; i < 100; i++ {
require.NoError(t, multiSSTWriter.Put(ctx, storage.EngineKey{Key: roachpb.Key(append(desc.StartKey, byte(i)))}, []byte("foobarbaz")))
}

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

iter, err := buildIterForScratch(t, keySpans, scratch)
require.NoError(t, err)
defer iter.Close()

iter.SeekGE(storage.MVCCKey{Key: mvccSpan.Key})
refIter.SeekGE(storage.MVCCKey{Key: mvccSpan.Key})
valid, err := iter.Valid()
valid2, err2 := refIter.Valid()
require.NoError(t, err)
require.NoError(t, err2)

for valid && valid2 {

require.Equal(t, iter.UnsafeKey(), refIter.UnsafeKey())
val, err := iter.UnsafeValue()
require.NoError(t, err)
val2, err2 := refIter.UnsafeValue()
require.NoError(t, err2)
require.Equal(t, val, val2)
iter.Next()
refIter.Next()
valid, err = iter.Valid()
valid2, err2 = refIter.Valid()
require.NoError(t, err)
require.NoError(t, err2)
}
require.Equal(t, valid, valid2)
}

// TestMultiSSTWriterAddLastSpan tests that multiSSTWriter initializes each of
// the SST files associated with the replicated key ranges by writing a range
// deletion tombstone that spans the entire range of each respectively, except
Expand Down Expand Up @@ -342,14 +457,16 @@ func TestMultiSSTWriterAddLastSpan(t *testing.T) {
EndKey: roachpb.RKeyMax,
}
keySpans := rditer.MakeReplicatedKeySpans(&desc)
localSpans := keySpans[:len(keySpans)-1]
mvccSpan := keySpans[len(keySpans)-1]

msstw, err := newMultiSSTWriter(
ctx, cluster.MakeTestingClusterSettings(), scratch, keySpans, 0,
true, /* skipRangeDelForLastSpan */
ctx, cluster.MakeTestingClusterSettings(), scratch, localSpans, mvccSpan, 0,
true, /* skipRangeDelForMVCCSpan */
)
require.NoError(t, err)
if addRangeDel {
require.NoError(t, msstw.addRangeDelForLastSpan())
require.NoError(t, msstw.addClearForMVCCSpan())
}
testKey := storage.MVCCKey{Key: roachpb.RKey("d1").AsRawKey(), Timestamp: hlc.Timestamp{WallTime: 1}}
testEngineKey, _ := storage.DecodeEngineKey(storage.EncodeMVCCKey(testKey))
Expand Down
Loading

0 comments on commit 1c4ce42

Please sign in to comment.