diff --git a/ingest.go b/ingest.go index 2b8cd30f3d5..69a9097f167 100644 --- a/ingest.go +++ b/ingest.go @@ -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 } diff --git a/internal/testkeys/testkeys.go b/internal/testkeys/testkeys.go index ab2f5d1dee7..afaf6b2f1f2 100644 --- a/internal/testkeys/testkeys.go +++ b/internal/testkeys/testkeys.go @@ -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 { diff --git a/metamorphic/build.go b/metamorphic/build.go index a33c02e63fa..dae743bf119 100644 --- a/metamorphic/build.go +++ b/metamorphic/build.go @@ -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 @@ -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...) @@ -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 } } @@ -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( @@ -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 @@ -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(), ) @@ -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) @@ -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(), ) diff --git a/metamorphic/generator.go b/metamorphic/generator.go index b9f00aecd38..abceafa44dd 100644 --- a/metamorphic/generator.go +++ b/metamorphic/generator.go @@ -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. diff --git a/metamorphic/key_generator.go b/metamorphic/key_generator.go index 615c80e7657..de96c418104 100644 --- a/metamorphic/key_generator.go +++ b/metamorphic/key_generator.go @@ -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) @@ -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) } } } diff --git a/metamorphic/ops.go b/metamorphic/ops.go index 00d25851617..98ea5fe6ae6 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -74,7 +74,7 @@ func (o *initOp) run(t *Test, h historyRecorder) { t.batches = make([]*pebble.Batch, o.batchSlots) t.iters = make([]*retryableIter, o.iterSlots) t.snapshots = make([]readerCloser, o.snapshotSlots) - t.externalObjs = make([]*sstable.WriterMetadata, o.externalObjSlots) + t.externalObjs = make([]externalObjMeta, o.externalObjSlots) h.Recordf("%s", o) } @@ -926,22 +926,56 @@ type ingestExternalFilesOp struct { type externalObjWithBounds struct { externalObjID objID bounds pebble.KeyRange + // We will only apply the synthetic suffix if it compares before all the + // suffixes inside the sst. + syntheticSuffix sstable.SyntheticSuffix } func (o *ingestExternalFilesOp) run(t *Test, h historyRecorder) { db := t.getDB(o.dbID) + + // We modify objs to eliminate empty objects and clear invalid synthetic suffixes. + var objs []externalObjWithBounds + for _, obj := range o.objs { + // Make sure the object exists and is not empty. + objMeta := t.getExternalObj(obj.externalObjID) + if m := objMeta.sstMeta; !m.HasPointKeys && !m.HasRangeKeys && !m.HasRangeDelKeys { + continue + } + if externalObjIsEmpty(t, obj.externalObjID, obj.bounds) { + // Currently we don't support ingesting external objects that have no point + // or range keys in the given range. Filter out any such objects. + // TODO(radu): even though we don't expect this case in practice, eventually + // we want to make sure that it doesn't cause failures. + continue + } + if obj.syntheticSuffix != nil { + // Verify that the version supports suffix replacement, and verify that + // the suffix comes before any suffix present in the object. + if t.opts.Comparer.Compare(obj.syntheticSuffix, objMeta.minSuffix) >= 0 { + obj.syntheticSuffix = nil + } + if objMeta.sstMeta.HasRangeDelKeys { + // Disable synthetic suffix if we have range dels. + // TODO(radu): we will want to support this at some point. + obj.syntheticSuffix = nil + } + } + objs = append(objs, obj) + } + if len(objs) == 0 { + h.Recordf("%s // no-op", o) + return + } var err error if !t.testOpts.externalStorageEnabled { // Emulate the operation by crating local, truncated SST files and ingesting // them. var paths []string - for i, obj := range o.objs { + for i, obj := range objs { // Make sure the object exists and is not empty. - if objMeta := t.getExternalObj(obj.externalObjID); !objMeta.HasPointKeys && !objMeta.HasRangeKeys && !objMeta.HasRangeDelKeys { - continue - } - path, meta := buildForIngestExternalEmulation(t, o.dbID, obj.externalObjID, obj.bounds, i) - if meta.HasPointKeys || meta.HasRangeKeys || meta.HasRangeDelKeys { + path, sstMeta := buildForIngestExternalEmulation(t, o.dbID, obj.externalObjID, obj.bounds, obj.syntheticSuffix, i) + if sstMeta.HasPointKeys || sstMeta.HasRangeKeys || sstMeta.HasRangeDelKeys { paths = append(paths, path) } } @@ -949,39 +983,24 @@ func (o *ingestExternalFilesOp) run(t *Test, h historyRecorder) { err = db.Ingest(paths) } } else { - // Currently we don't support ingesting external objects that have no point - // or range keys in the given range. Filter out any such objects. - // TODO(radu): even though we don't expect this case in practice, eventually - // we want to make sure that it doesn't cause failures. - var objs []externalObjWithBounds - for _, obj := range o.objs { - objMeta := t.getExternalObj(obj.externalObjID) - if objMeta.HasPointKeys || objMeta.HasRangeKeys || objMeta.HasRangeDelKeys { - if !externalObjIsEmpty(t, obj.externalObjID, obj.bounds) { - objs = append(objs, obj) - } - } - } - external := make([]pebble.ExternalFile, len(objs)) for i, obj := range objs { meta := t.getExternalObj(obj.externalObjID) external[i] = pebble.ExternalFile{ Locator: "external", ObjName: externalObjName(obj.externalObjID), - Size: meta.Size, + Size: meta.sstMeta.Size, SmallestUserKey: obj.bounds.Start, LargestUserKey: obj.bounds.End, // Note: if the table has point/range keys, we don't know for sure whether // this particular range has any, but that's acceptable. - HasPointKey: meta.HasPointKeys || meta.HasRangeDelKeys, - HasRangeKey: meta.HasRangeKeys, + HasPointKey: meta.sstMeta.HasPointKeys || meta.sstMeta.HasRangeDelKeys, + HasRangeKey: meta.sstMeta.HasRangeKeys, + SyntheticSuffix: obj.syntheticSuffix, // TODO(radu): test prefix replacement. } } - if len(external) > 0 { - _, err = db.IngestExternalFiles(external) - } + _, err = db.IngestExternalFiles(external) } h.Recordf("%s // %v", o, err) @@ -1001,7 +1020,7 @@ func (o *ingestExternalFilesOp) syncObjs() objIDSlice { func (o *ingestExternalFilesOp) String() string { strs := make([]string, len(o.objs)) for i, obj := range o.objs { - strs[i] = fmt.Sprintf("%s, %q, %q", obj.externalObjID, obj.bounds.Start, obj.bounds.End) + strs[i] = fmt.Sprintf("%s, %q, %q, %q", obj.externalObjID, obj.bounds.Start, obj.bounds.End, obj.syntheticSuffix) } return fmt.Sprintf("%s.IngestExternalFiles(%s)", o.dbID, strings.Join(strs, ", ")) } @@ -1758,16 +1777,21 @@ func (o *newExternalObjOp) run(t *Test, h historyRecorder) { rangeKeyIter = nil } - meta, err := writeSSTForIngestion( + sstMeta, minSuffix, err := writeSSTForIngestion( t, iter, rangeDelIter, rangeKeyIter, + true, /* uniquePrefixes */ + nil, /* syntheticSuffix */ writable, t.minFMV(), ) if err != nil { panic(err) } - t.setExternalObj(o.externalObjID, meta) + t.setExternalObj(o.externalObjID, externalObjMeta{ + sstMeta: sstMeta, + minSuffix: minSuffix, + }) h.Recordf("%s", o) } diff --git a/metamorphic/options.go b/metamorphic/options.go index 2e5b2a8f6f0..0a439f6af93 100644 --- a/metamorphic/options.go +++ b/metamorphic/options.go @@ -478,9 +478,22 @@ func standardOptions() []*TestOptions { format_major_version=%s [TestOptions] shared_storage_enabled=true - external_storage_enabled=true secondary_cache_enabled=true `, pebble.FormatMinForSharedObjects), + 28: fmt.Sprintf(` +[Options] + format_major_version=%s +[TestOptions] + external_storage_enabled=true +`, pebble.FormatSyntheticPrefixSuffix), + 29: fmt.Sprintf(` +[Options] + format_major_version=%s +[TestOptions] + shared_storage_enabled=true + external_storage_enabled=true + secondary_cache_enabled=false +`, pebble.FormatSyntheticPrefixSuffix), } opts := make([]*TestOptions, len(stdOpts)) @@ -662,8 +675,8 @@ func RandomOptions( // 50% of time, enable external storage. if rng.Intn(2) == 0 { testOpts.externalStorageEnabled = true - if testOpts.Opts.FormatMajorVersion < pebble.FormatMinForSharedObjects { - testOpts.Opts.FormatMajorVersion = pebble.FormatMinForSharedObjects + if testOpts.Opts.FormatMajorVersion < pebble.FormatSyntheticPrefixSuffix { + testOpts.Opts.FormatMajorVersion = pebble.FormatSyntheticPrefixSuffix } testOpts.externalStorageFS = remote.NewInMem() } diff --git a/metamorphic/parser.go b/metamorphic/parser.go index 4dd04c28f94..431bd1ccdfd 100644 --- a/metamorphic/parser.go +++ b/metamorphic/parser.go @@ -466,22 +466,25 @@ func (p *parser) parseCheckpointSpans(list []listElem) []pebble.CheckpointSpan { } func (p *parser) parseExternalObjsWithBounds(list []listElem) []externalObjWithBounds { - if len(list)%3 != 0 { - panic(p.errorf(list[0].pos, "expected number of arguments to be multiple of 3")) + const numArgs = 4 + if len(list)%numArgs != 0 { + panic(p.errorf(list[0].pos, "expected number of arguments to be multiple of %d", numArgs)) } - objs := make([]externalObjWithBounds, len(list)/3) + objs := make([]externalObjWithBounds, len(list)/numArgs) for i := range objs { list[0].expectToken(p, token.IDENT) list[1].expectToken(p, token.STRING) list[2].expectToken(p, token.STRING) + list[3].expectToken(p, token.STRING) objs[i] = externalObjWithBounds{ externalObjID: p.parseObjID(list[0].pos, list[0].lit), bounds: pebble.KeyRange{ Start: unquoteBytes(list[1].lit), End: unquoteBytes(list[2].lit), }, + syntheticSuffix: unquoteBytes(list[3].lit), } - list = list[3:] + list = list[numArgs:] } return objs } diff --git a/metamorphic/test.go b/metamorphic/test.go index fca5276a9b3..a3262f44aa1 100644 --- a/metamorphic/test.go +++ b/metamorphic/test.go @@ -59,7 +59,7 @@ type Test struct { batches []*pebble.Batch iters []*retryableIter snapshots []readerCloser - externalObjs []*sstable.WriterMetadata + externalObjs []externalObjMeta // externalStorage is used to write external objects. If external storage is // enabled, this is the same with testOpts.externalStorageFS; otherwise, this @@ -67,6 +67,13 @@ type Test struct { externalStorage remote.Storage } +type externalObjMeta struct { + sstMeta *sstable.WriterMetadata + // minSuffix is the minimum (according to the comparator) non-empty suffix in + // the object. + minSuffix []byte +} + func newTest(ops []op) *Test { return &Test{ ops: ops, @@ -420,15 +427,15 @@ func (t *Test) setSnapshot(id objID, s readerCloser) { t.snapshots[id.slot()] = s } -func (t *Test) setExternalObj(id objID, meta *sstable.WriterMetadata) { +func (t *Test) setExternalObj(id objID, meta externalObjMeta) { if id.tag() != externalObjTag { panic(fmt.Sprintf("invalid external object ID: %s", id)) } t.externalObjs[id.slot()] = meta } -func (t *Test) getExternalObj(id objID) *sstable.WriterMetadata { - if id.tag() != externalObjTag || t.externalObjs[id.slot()] == nil { +func (t *Test) getExternalObj(id objID) externalObjMeta { + if id.tag() != externalObjTag || t.externalObjs[id.slot()].sstMeta == nil { panic(fmt.Sprintf("metamorphic test internal error: invalid external object ID: %s", id)) } return t.externalObjs[id.slot()] diff --git a/sstable/block_property_test_utils.go b/sstable/block_property_test_utils.go index 38b32528e7e..4016b9c4880 100644 --- a/sstable/block_property_test_utils.go +++ b/sstable/block_property_test_utils.go @@ -5,6 +5,7 @@ package sstable import ( + "fmt" "math" "github.com/cockroachdb/pebble/internal/base" @@ -35,7 +36,26 @@ func NewTestKeysBlockPropertyCollector() BlockPropertyCollector { // and keys with suffixes within the range [filterMin, filterMax). For keys with // suffixes outside the range, iteration is nondeterministic. func NewTestKeysBlockPropertyFilter(filterMin, filterMax uint64) *BlockIntervalFilter { - return NewBlockIntervalFilter(testKeysBlockPropertyName, filterMin, filterMax, nil) + return NewBlockIntervalFilter(testKeysBlockPropertyName, filterMin, filterMax, testKeysBlockIntervalSyntheticReplacer{}) +} + +var _ BlockIntervalSyntheticReplacer = testKeysBlockIntervalSyntheticReplacer{} + +type testKeysBlockIntervalSyntheticReplacer struct{} + +// AdjustIntervalWithSyntheticSuffix implements BlockIntervalSyntheticReplacer. +func (sr testKeysBlockIntervalSyntheticReplacer) AdjustIntervalWithSyntheticSuffix( + lower uint64, upper uint64, suffix []byte, +) (adjustedLower uint64, adjustedUpper uint64, err error) { + decoded, err := testkeys.ParseSuffix(suffix) + if err != nil { + return 0, 0, err + } + // The testKeysSuffixIntervalCollector below maps keys with no suffix to MaxUint64; ignore it. + if upper != math.MaxUint64 && uint64(decoded) < upper { + panic(fmt.Sprintf("the synthetic suffix %d is less than the property upper bound %d", decoded, upper)) + } + return uint64(decoded), uint64(decoded) + 1, nil } // NewTestKeysMaskingFilter constructs a TestKeysMaskingFilter that implements @@ -70,7 +90,7 @@ func (f TestKeysMaskingFilter) Intersects(prop []byte) (bool, error) { // SyntheticSuffixIntersects implements the BlockPropertyFilter interface. func (f TestKeysMaskingFilter) SyntheticSuffixIntersects(prop []byte, suffix []byte) (bool, error) { - panic("unimplemented") + return f.BlockIntervalFilter.SyntheticSuffixIntersects(prop, suffix) } var _ DataBlockIntervalCollector = (*testKeysSuffixIntervalCollector)(nil) diff --git a/testdata/ingest_external b/testdata/ingest_external index 776b8c742db..901fcdeecea 100644 --- a/testdata/ingest_external +++ b/testdata/ingest_external @@ -237,6 +237,17 @@ a@5: (foo, .) b@5: (foo, .) c@5: (foo, .) +# Verify that we require bounds without suffix if we use suffix replacement. +ingest-external suffix-replace=@5 +f6,10,a@1,z@10 +---- +pebble: synthetic suffix is set but smallest key has suffix + +ingest-external suffix-replace=@5 +f6,10,a,z@10 +---- +pebble: synthetic suffix is set but largest key has suffix + # Test the case when we are ingesting part of a RANGEDEL. reset