Skip to content

Commit

Permalink
metamorphic: add testing for synthetic suffix
Browse files Browse the repository at this point in the history
Add a random synthetic suffix for external ingestion.

One difficulty is that the synthetic suffix has to be larger (as an
integer value) than all suffixes in the object. We can't easily
determine this at generation time, so we generate a random suffix and
at runtime determine if it's ok to use or not.
  • Loading branch information
RaduBerinde committed Feb 16, 2024
1 parent 7127d66 commit c516ca4
Show file tree
Hide file tree
Showing 10 changed files with 224 additions and 76 deletions.
17 changes: 12 additions & 5 deletions ingest.go
Original file line number Diff line number Diff line change
Expand Up @@ -1107,35 +1107,42 @@ type ExternalFile struct {
// Locator is the shared.Locator that can be used with objProvider to
// resolve a reference to this external sstable.
Locator remote.Locator

// ObjName is the unique name of this sstable on Locator.
ObjName string

// Size of the referenced proportion of the virtualized sstable. An estimate
// is acceptable in lieu of the backing file size.
Size uint64

// SmallestUserKey and LargestUserKey are the [smallest,largest) user key
// bounds of the sstable. Both these bounds are loose i.e. it's possible for
// the sstable to not span the entirety of this range. However, multiple
// ExternalFiles in one ingestion must all have non-overlapping
// [smallest, largest) spans. Note that this Largest bound is exclusive.
SmallestUserKey, LargestUserKey []byte

// HasPointKey and HasRangeKey denote whether this file contains point keys
// or range keys. If both structs are false, an error is returned during
// ingestion.
HasPointKey, HasRangeKey bool

// ContentPrefix and SyntheticPrefix denote a prefix replacement rule causing
// a file, in which all keys have prefix ContentPrefix, to appear whenever it
// is accessed as if those keys all instead have prefix SyntheticPrefix.
//
// SyntheticPrefix must be a prefix of both SmallestUserKey and LargestUserKey.
ContentPrefix, SyntheticPrefix []byte

// SyntheticSuffix will replace the suffix of every key in the file during
// iteration. Note that the file itself is not modified, rather, every key
// returned by an iterator will have the synthetic suffix.
//
// The SyntheticSuffix must sort before any non-empty suffixes in the backing
// sstable.
//
// If SyntheticSuffix is set, then SmallestUserKey and LargestUserKey must not
// have suffixes.
// SyntheticSuffix can only be used under the following conditions:
// - the synthetic suffix must sort before any non-empty suffixes in the
// backing sst;
// - SmallestUserKey and LargestUserKey must not have suffixes;
// - the backing sst must not contain multiple keys with the same prefix.
SyntheticSuffix []byte
}

Expand Down
5 changes: 5 additions & 0 deletions internal/testkeys/testkeys.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,11 @@ var Comparer = &base.Comparer{
Name: "pebble.internal.testkeys",
}

// The comparator is similar to the one in Cockroach; when the prefixes are
// equal:
// - a key without a suffix is smaller than one with a suffix;
// - when both keys have a suffix, the key with the larger (decoded) suffix
// value is smaller.
func compare(a, b []byte) int {
ai, bi := split(a), split(b)
if v := bytes.Compare(a[:ai], b[:bi]); v != 0 {
Expand Down
94 changes: 69 additions & 25 deletions metamorphic/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,19 +22,18 @@ import (
// writeSSTForIngestion writes an SST that is to be ingested, either directly or
// as an external file.
//
// If convertDelSizedToDel is set, then any DeleteSized keys are converted to
// Delete keys; this is useful when the database that will ingest the file is at
// a format that doesn't support DeleteSized.
//
// Returns the sstable metadata and the minimum non-empty suffix.
// Closes the iterators in all cases.
func writeSSTForIngestion(
t *Test,
pointIter base.InternalIterator,
rangeDelIter keyspan.FragmentIterator,
rangeKeyIter keyspan.FragmentIterator,
uniquePrefixes bool,
syntheticSuffix sstable.SyntheticSuffix,
writable objstorage.Writable,
targetFMV pebble.FormatMajorVersion,
) (*sstable.WriterMetadata, error) {
) (_ *sstable.WriterMetadata, minSuffix []byte, _ error) {
writerOpts := t.opts.MakeWriterOptions(0, targetFMV.MaxTableFormat())
if t.testOpts.disableValueBlocksForIngestSSTables {
writerOpts.DisableValueBlocks = true
Expand All @@ -47,11 +46,37 @@ func writeSSTForIngestion(
rangeKeyIterCloser := base.CloseHelper(rangeKeyIter)
defer rangeKeyIterCloser.Close()

outputKey := func(key []byte) []byte {
n := t.opts.Comparer.Split(key)
if suffix := key[n:]; len(suffix) > 0 {
if minSuffix == nil || t.opts.Comparer.Compare(suffix, minSuffix) < 0 {
minSuffix = slices.Clone(suffix)
}
}
return slices.Clone(key)
}

if len(syntheticSuffix) > 0 {
minSuffix = slices.Clone(syntheticSuffix)
outputKey = func(key []byte) []byte {
n := t.opts.Comparer.Split(key)
return append(key[:n:n], syntheticSuffix...)
}
}

var lastUserKey []byte
for key, value := pointIter.First(); key != nil; key, value = pointIter.Next() {
// Ignore duplicate keys.
if t.opts.Comparer.Equal(lastUserKey, key.UserKey) {
continue
if lastUserKey != nil {
last := lastUserKey
this := key.UserKey
if uniquePrefixes {
last = last[:t.opts.Comparer.Split(last)]
this = this[:t.opts.Comparer.Split(this)]
}
if t.opts.Comparer.Equal(last, this) {
continue
}
}
lastUserKey = append(lastUserKey[:0], key.UserKey...)

Expand All @@ -65,28 +90,30 @@ func writeSSTForIngestion(
}
valBytes, _, err := value.Value(nil)
if err != nil {
return nil, err
return nil, nil, err
}
if err := w.Add(*key, valBytes); err != nil {
return nil, err
k := *key
k.UserKey = outputKey(k.UserKey)
if err := w.Add(k, valBytes); err != nil {
return nil, nil, err
}
}
if err := pointIterCloser.Close(); err != nil {
return nil, err
return nil, nil, err
}

if rangeDelIter != nil {
span, err := rangeDelIter.First()
for ; span != nil; span, err = rangeDelIter.Next() {
if err := w.DeleteRange(slices.Clone(span.Start), slices.Clone(span.End)); err != nil {
return nil, err
if err := w.DeleteRange(outputKey(span.Start), outputKey(span.End)); err != nil {
return nil, nil, err
}
}
if err != nil {
return nil, err
return nil, nil, err
}
if err := rangeDelIterCloser.Close(); err != nil {
return nil, err
return nil, nil, err
}
}

Expand All @@ -103,8 +130,8 @@ func writeSSTForIngestion(
// same sequence number is nonsensical, so we "coalesce" or collapse
// the keys.
collapsed := keyspan.Span{
Start: slices.Clone(span.Start),
End: slices.Clone(span.End),
Start: outputKey(span.Start),
End: outputKey(span.End),
Keys: make([]keyspan.Key, 0, len(span.Keys)),
}
rangekey.Coalesce(
Expand All @@ -115,21 +142,25 @@ func writeSSTForIngestion(
}
keyspan.SortKeysByTrailer(&collapsed.Keys)
if err := rangekey.Encode(&collapsed, w.AddRangeKey); err != nil {
return nil, err
return nil, nil, err
}
}
if err != nil {
return nil, err
return nil, nil, err
}
if err := rangeKeyIterCloser.Close(); err != nil {
return nil, err
return nil, nil, err
}
}

if err := w.Close(); err != nil {
return nil, err
return nil, nil, err
}
return w.Metadata()
sstMeta, err := w.Metadata()
if err != nil {
return nil, nil, err
}
return sstMeta, minSuffix, nil
}

// buildForIngest builds a local SST file containing the keys in the given batch
Expand All @@ -147,9 +178,11 @@ func buildForIngest(
iter, rangeDelIter, rangeKeyIter := private.BatchSort(b)

writable := objstorageprovider.NewFileWritable(f)
meta, err := writeSSTForIngestion(
meta, _, err := writeSSTForIngestion(
t,
iter, rangeDelIter, rangeKeyIter,
false, /* uniquePrefixes */
nil, /* syntheticSuffix */
writable,
db.FormatMajorVersion(),
)
Expand All @@ -160,7 +193,12 @@ func buildForIngest(
// external object (truncated to the given bounds) and returns its path and
// metadata.
func buildForIngestExternalEmulation(
t *Test, dbID objID, externalObjID objID, bounds pebble.KeyRange, i int,
t *Test,
dbID objID,
externalObjID objID,
bounds pebble.KeyRange,
syntheticSuffix sstable.SyntheticSuffix,
i int,
) (path string, _ *sstable.WriterMetadata) {
path = t.opts.FS.PathJoin(t.tmpDir, fmt.Sprintf("ext%d-%d", dbID.slot(), i))
f, err := t.opts.FS.Create(path)
Expand All @@ -170,9 +208,15 @@ func buildForIngestExternalEmulation(
defer reader.Close()

writable := objstorageprovider.NewFileWritable(f)
meta, err := writeSSTForIngestion(
// The underlying file should already have unique prefixes. Plus we are
// emulating the external ingestion path which won't remove duplicate prefixes
// if they exist.
const uniquePrefixes = false
meta, _, err := writeSSTForIngestion(
t,
pointIter, rangeDelIter, rangeKeyIter,
uniquePrefixes,
syntheticSuffix,
writable,
t.minFMV(),
)
Expand Down
16 changes: 16 additions & 0 deletions metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -1337,6 +1337,22 @@ func (g *generator) writerIngestExternalFiles() {
o.bounds.Start = keys[2*i]
}
o.bounds.End = keys[2*i+1]

if g.rng.Intn(2) == 0 {
// Try to set a synthetic suffix. We can only do so if the bounds don't
// have suffixes, so try to trim them.
start := g.keyGenerator.prefix(o.bounds.Start)
end := g.keyGenerator.prefix(o.bounds.End)
// If the trimmed bounds overlap with adjacent file bounds, we just don't
// set the suffix.
if g.cmp(start, end) < 0 &&
(i == 0 || g.cmp(sorted[i-1].bounds.End, start) <= 0) &&
(i == len(sorted)-1 || g.cmp(end, sorted[i+1].bounds.Start) <= 0) {
o.bounds.Start = start
o.bounds.End = end
o.syntheticSuffix = g.keyGenerator.SkewedSuffix(0.1)
}
}
}
// The batches we're ingesting may contain single delete tombstones that
// when applied to the writer result in nondeterminism in the deleted key.
Expand Down
15 changes: 12 additions & 3 deletions metamorphic/key_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,12 @@ func (kg *keyGenerator) randKey(newKeyProbability float64, bounds *pebble.KeyRan
panic(fmt.Sprintf("invalid bounds [%q, %q)", bounds.Start, bounds.End))
}
suffix = kg.SkewedSuffixInt(0.01)
for !(cmpSuffix(startSuffix, suffix) <= 0 && cmpSuffix(suffix, endSuffix) < 0) {
for i := 0; !(cmpSuffix(startSuffix, suffix) <= 0 && cmpSuffix(suffix, endSuffix) < 0); i++ {
if i > 10 {
// This value is always >= startSuffix and < endSuffix.
suffix = (startSuffix + endSuffix) / 2
break
}
// The suffix we want must exist in the current suffix range, we don't
// want to keep increasing it here.
suffix = kg.SkewedSuffixInt(0)
Expand All @@ -217,8 +222,12 @@ func (kg *keyGenerator) randKey(newKeyProbability float64, bounds *pebble.KeyRan
suffix = kg.SkewedSuffixInt(0.01)
if kg.equal(prefix, startPrefix) {
// We can't use a suffix which sorts before startSuffix.
for cmpSuffix(suffix, startSuffix) < 0 {
suffix = kg.SkewedSuffixInt(0.01)
for i := 0; cmpSuffix(suffix, startSuffix) < 0; i++ {
if i > 10 {
suffix = startSuffix
break
}
suffix = kg.SkewedSuffixInt(0)
}
}
}
Expand Down
Loading

0 comments on commit c516ca4

Please sign in to comment.