From a765170aa3da287cf8645d266ecb0fa6a47b0f6e Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Fri, 3 Nov 2023 16:20:46 -0400 Subject: [PATCH] metamorphic: add IngestAndExcise, make EFOS determinitic This change updates the metamorphic tests to support a new useExcise testOption and a new IngestAndExcise operation, which calls db.IngestAndExcise() if useExcise is true, or calls it with an sstable containing DeleteRange/RangeKeyDelete for the same span if useExcise is false. Furthermore, a new private option is added, efosAlwaysCreatesIterators. If the test option for useExcise (or the previously added useSharedReplicate) is enabled, one of these two things happen to guarantee EFOS determinism around excises: 1) If efosAlwaysCreatesIterators is true, the behaviour of EFOS is changed to always hold onto a readState and never return errors from NewIter. 2) If efosAlwaysCreatesIterators is false, regular EFOS behaviour is retained, but a flush is completed after every EFOS instantiation to guarantee that it becomes excise-immune right at instantiation. Note that regular snapshots are never created when useExcise or useSharedReplicate is true. Fixes #2885. --- db.go | 21 ++++-- metamorphic/config.go | 2 + metamorphic/generator.go | 54 ++++++++------ metamorphic/ops.go | 152 ++++++++++++++++++++++++++++++++------- metamorphic/options.go | 37 ++++++++++ metamorphic/parser.go | 3 + options.go | 12 ++++ snapshot.go | 52 +++++++++++--- 8 files changed, 271 insertions(+), 62 deletions(-) diff --git a/db.go b/db.go index d9f27bc4b06..85e67dde0dc 100644 --- a/db.go +++ b/db.go @@ -995,9 +995,12 @@ var iterAllocPool = sync.Pool{ // and the specified seqNum will be used as the snapshot seqNum. // - EFOS in file-only state: Only `seqNum` and `vers` are set. All the // relevant SSTs are referenced by the *version. +// - EFOS that has been excised but is in alwaysCreateIters mode (tests only). +// Only `seqNum` and `readState` are set. type snapshotIterOpts struct { - seqNum uint64 - vers *version + seqNum uint64 + vers *version + readState *readState } type batchIterOpts struct { @@ -1051,8 +1054,12 @@ func (d *DB) newIter( // files in the associated version from being deleted if there is a current // compaction. The readState is unref'd by Iterator.Close(). if internalOpts.snapshot.vers == nil { - // NB: loadReadState() calls readState.ref(). - readState = d.loadReadState() + if internalOpts.snapshot.readState != nil { + readState = internalOpts.snapshot.readState + } else { + // NB: loadReadState() calls readState.ref(). + readState = d.loadReadState() + } } else { // vers != nil internalOpts.snapshot.vers.Ref() @@ -1280,7 +1287,11 @@ func (d *DB) newInternalIter( // compaction. The readState is unref'd by Iterator.Close(). var readState *readState if sOpts.vers == nil { - readState = d.loadReadState() + if sOpts.readState != nil { + readState = sOpts.readState + } else { + readState = d.loadReadState() + } } if sOpts.vers != nil { sOpts.vers.Ref() diff --git a/metamorphic/config.go b/metamorphic/config.go index f86c651269f..e37021e5cd4 100644 --- a/metamorphic/config.go +++ b/metamorphic/config.go @@ -44,6 +44,7 @@ const ( writerDelete writerDeleteRange writerIngest + writerIngestAndExcise writerMerge writerRangeKeyDelete writerRangeKeySet @@ -152,6 +153,7 @@ func defaultConfig() config { writerDelete: 100, writerDeleteRange: 50, writerIngest: 100, + writerIngestAndExcise: 50, writerMerge: 100, writerRangeKeySet: 10, writerRangeKeyUnset: 10, diff --git a/metamorphic/generator.go b/metamorphic/generator.go index 1ec5545561c..64fcf1c66b9 100644 --- a/metamorphic/generator.go +++ b/metamorphic/generator.go @@ -170,6 +170,7 @@ func generate(rng *rand.Rand, count uint64, cfg config, km *keyManager) []op { writerDelete: g.writerDelete, writerDeleteRange: g.writerDeleteRange, writerIngest: g.writerIngest, + writerIngestAndExcise: g.writerIngestAndExcise, writerMerge: g.writerMerge, writerRangeKeyDelete: g.writerRangeKeyDelete, writerRangeKeySet: g.writerRangeKeySet, @@ -1220,28 +1221,15 @@ func (g *generator) newSnapshot() { snapID: snapID, } - // With 75% probability, impose bounds on the keys that may be read with the - // snapshot. Setting bounds allows some runs of the metamorphic test to use - // a EventuallyFileOnlySnapshot instead of a Snapshot, testing equivalence - // between the two for reads within those bounds. - // - // If we're in multi-instance mode, we must always create bounds, as we will - // always create EventuallyFileOnlySnapshots to allow commands that use excises - // (eg. replicateOp) to work. - if g.rng.Float64() < 0.75 || g.dbs.Len() > 1 { - s.bounds = g.generateDisjointKeyRanges( - g.rng.Intn(5) + 1, /* between 1-5 */ - ) - g.snapshotBounds[snapID] = s.bounds - } + // Impose bounds on the keys that may be read with the snapshot. Setting bounds + // allows some runs of the metamorphic test to use a EventuallyFileOnlySnapshot + // instead of a Snapshot, testing equivalence between the two for reads within + // those bounds. + s.bounds = g.generateDisjointKeyRanges( + g.rng.Intn(5) + 1, /* between 1-5 */ + ) + g.snapshotBounds[snapID] = s.bounds g.add(s) - if g.dbs.Len() > 1 { - // Do a flush after each EFOS, if we're in multi-instance mode. This limits - // the testing area of EFOS, but allows them to be used alongside operations - // that do an excise (eg. replicateOp). This will be revisited when - // https://github.com/cockroachdb/pebble/issues/2885 is implemented. - g.add(&flushOp{dbID}) - } } func (g *generator) snapshotClose() { @@ -1446,6 +1434,30 @@ func (g *generator) writerIngest() { }) } +func (g *generator) writerIngestAndExcise() { + if len(g.liveBatches) == 0 { + return + } + + dbID := g.dbs.rand(g.rng) + // Ingest between 1 and 3 batches. + batchID := g.liveBatches.rand(g.rng) + g.removeBatchFromGenerator(batchID) + + start := g.randKeyToWrite(0.001) + end := g.randKeyToWrite(0.001) + if g.cmp(start, end) > 0 { + start, end = end, start + } + + g.add(&ingestAndExciseOp{ + dbID: dbID, + batchID: batchID, + exciseStart: start, + exciseEnd: end, + }) +} + func (g *generator) writerMerge() { if len(g.liveWriters) == 0 { return diff --git a/metamorphic/ops.go b/metamorphic/ops.go index bc194e6da85..971be5ba4d9 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -518,13 +518,15 @@ func (o *ingestOp) run(t *test, h historyRecorder) { h.Recordf("%s // %v", o, err) } -func (o *ingestOp) build(t *test, h historyRecorder, b *pebble.Batch, i int) (string, error) { - path := t.opts.FS.PathJoin(t.tmpDir, fmt.Sprintf("ext%d-%d", o.dbID.slot(), i)) +func buildForIngest( + t *test, dbID objID, h historyRecorder, b *pebble.Batch, i int, +) (string, error) { + path := t.opts.FS.PathJoin(t.tmpDir, fmt.Sprintf("ext%d-%d", dbID.slot(), i)) f, err := t.opts.FS.Create(path) if err != nil { return "", err } - db := t.getDB(o.dbID) + db := t.getDB(dbID) iter, rangeDelIter, rangeKeyIter := private.BatchSort(b) defer closeIters(iter, rangeDelIter, rangeKeyIter) @@ -550,7 +552,7 @@ func (o *ingestOp) build(t *test, h historyRecorder, b *pebble.Batch, i int) (st // It's possible that we wrote the key on a batch from a db that supported // DeleteSized, but are now ingesting into a db that does not. Detect // this case and translate the key to an InternalKeyKindDelete. - if key.Kind() == pebble.InternalKeyKindDeleteSized && !t.isFMV(o.dbID, pebble.FormatDeleteSizedAndObsolete) { + if key.Kind() == pebble.InternalKeyKindDeleteSized && !t.isFMV(dbID, pebble.FormatDeleteSizedAndObsolete) { value = pebble.LazyValue{} key.SetKind(pebble.InternalKeyKindDelete) } @@ -621,6 +623,10 @@ func (o *ingestOp) build(t *test, h historyRecorder, b *pebble.Batch, i int) (st return path, nil } +func (o *ingestOp) build(t *test, h historyRecorder, b *pebble.Batch, i int) (string, error) { + return buildForIngest(t, o.dbID, h, b, i) +} + func (o *ingestOp) receiver() objID { return o.dbID } func (o *ingestOp) syncObjs() objIDSlice { // Ingest should not be concurrent with mutating the batches that will be @@ -783,6 +789,69 @@ func (o *ingestOp) String() string { return buf.String() } +type ingestAndExciseOp struct { + dbID objID + batchID objID + exciseStart, exciseEnd []byte +} + +func (o *ingestAndExciseOp) run(t *test, h historyRecorder) { + var err error + b := t.getBatch(o.batchID) + t.clearObj(o.batchID) + if b.Empty() || t.testOpts.Opts.Comparer.Compare(o.exciseEnd, o.exciseStart) <= 0 { + // No-op. + h.Recordf("%s // %v", o, err) + return + } + if !t.testOpts.useExcise { + // Do a rangedel and rangekeydel before the ingestion. This mimics the + // behaviour of an excise. + db := t.getDB(o.dbID) + err = firstError(err, db.DeleteRange(o.exciseStart, o.exciseEnd, t.writeOpts)) + err = firstError(err, db.RangeKeyDelete(o.exciseStart, o.exciseEnd, t.writeOpts)) + } + path, err2 := o.build(t, h, b, 0 /* i */) + if err2 != nil { + h.Recordf("Build(%s) // %v", o.batchID, err2) + } + err = firstError(err, err2) + err = firstError(err, b.Close()) + + if t.testOpts.useExcise { + err = firstError(err, withRetries(func() error { + _, err := t.getDB(o.dbID).IngestAndExcise([]string{path}, nil /* sharedSSTs */, pebble.KeyRange{ + Start: o.exciseStart, + End: o.exciseEnd, + }) + return err + })) + } else { + err = firstError(err, withRetries(func() error { + return t.getDB(o.dbID).Ingest([]string{path}) + })) + } + + h.Recordf("%s // %v", o, err) +} + +func (o *ingestAndExciseOp) build( + t *test, h historyRecorder, b *pebble.Batch, i int, +) (string, error) { + return buildForIngest(t, o.dbID, h, b, i) +} + +func (o *ingestAndExciseOp) receiver() objID { return o.dbID } +func (o *ingestAndExciseOp) syncObjs() objIDSlice { + // Ingest should not be concurrent with mutating the batches that will be + // ingested as sstables. + return []objID{o.batchID} +} + +func (o *ingestAndExciseOp) String() string { + return fmt.Sprintf("%s.IngestAndExcise(%s, %q, %q)", o.dbID, o.batchID, o.exciseStart, o.exciseEnd) +} + // getOp models a Reader.Get operation. type getOp struct { readerID objID @@ -1320,10 +1389,28 @@ type newSnapshotOp struct { } func (o *newSnapshotOp) run(t *test, h historyRecorder) { + bounds := o.bounds + if len(bounds) == 0 { + panic("bounds unexpectedly unset for newSnapshotOp") + } // Fibonacci hash https://probablydance.com/2018/06/16/fibonacci-hashing-the-optimization-that-the-world-forgot-or-a-better-alternative-to-integer-modulo/ - if len(t.dbs) > 1 || (len(o.bounds) > 0 && ((11400714819323198485*uint64(t.idx)*t.testOpts.seedEFOS)>>63) == 1) { - s := t.getDB(o.dbID).NewEventuallyFileOnlySnapshot(o.bounds) + createEfos := ((11400714819323198485 * uint64(t.idx) * t.testOpts.seedEFOS) >> 63) == 1 + // If either of these options is true, an EFOS _must_ be created, regardless + // of what the fibonacci hash returned. + excisePossible := t.testOpts.useSharedReplicate || t.testOpts.useExcise + if createEfos || excisePossible { + s := t.getDB(o.dbID).NewEventuallyFileOnlySnapshot(bounds) t.setSnapshot(o.snapID, s) + // If the EFOS isn't guaranteed to always create iterators, we must force + // a flush on this DB so it transitions this EFOS into a file-only snapshot. + if excisePossible && !t.testOpts.efosAlwaysCreatesIters { + err := t.getDB(o.dbID).Flush() + if err != nil { + h.Recordf("%s // %v", o, err) + h.history.err.Store(errors.Wrap(err, "newSnapshotOp")) + return + } + } } else { s := t.getDB(o.dbID).NewSnapshot() t.setSnapshot(o.snapID, s) @@ -1423,14 +1510,11 @@ func (r *replicateOp) runSharedReplicate( }, func(start, end []byte, keys []keyspan.Key) error { s := keyspan.Span{ - Start: start, - End: end, - Keys: keys, - KeysOrder: 0, + Start: start, + End: end, + Keys: keys, } - return rangekey.Encode(&s, func(k base.InternalKey, v []byte) error { - return w.AddRangeKey(base.MakeInternalKey(k.UserKey, 0, k.Kind()), v) - }) + return rangekey.Encode(&s, w.AddRangeKey) }, func(sst *pebble.SharedSSTMeta) error { sharedSSTs = append(sharedSSTs, *sst) @@ -1442,6 +1526,12 @@ func (r *replicateOp) runSharedReplicate( return } + err = w.Close() + if err != nil { + h.Recordf("%s // %v", r, err) + return + } + _, err = dest.IngestAndExcise([]string{sstPath}, sharedSSTs, pebble.KeyRange{Start: r.start, End: r.end}) h.Recordf("%s // %v", r, err) } @@ -1468,6 +1558,15 @@ func (r *replicateOp) run(t *test, h historyRecorder) { return } + // First, do a RangeKeyDelete and DeleteRange on the whole span. + if err := dest.RangeKeyDelete(r.start, r.end, t.writeOpts); err != nil { + h.Recordf("%s // %v", r, err) + return + } + if err := dest.DeleteRange(r.start, r.end, t.writeOpts); err != nil { + h.Recordf("%s // %v", r, err) + return + } iter, err := source.NewIter(&pebble.IterOptions{ LowerBound: r.start, UpperBound: r.end, @@ -1478,16 +1577,7 @@ func (r *replicateOp) run(t *test, h historyRecorder) { } defer iter.Close() - // Write rangedels and rangekeydels for the range. This mimics the Excise - // that runSharedReplicate would do. - if err := w.DeleteRange(r.start, r.end); err != nil { - panic(err) - } - if err := w.RangeKeyDelete(r.start, r.end); err != nil { - panic(err) - } - - for ok := iter.SeekGE(r.start); ok && iter.Error() != nil; ok = iter.Next() { + for ok := iter.SeekGE(r.start); ok && iter.Error() == nil; ok = iter.Next() { hasPoint, hasRange := iter.HasPointAndRange() if hasPoint { val, err := iter.ValueAndErr() @@ -1501,13 +1591,25 @@ func (r *replicateOp) run(t *test, h historyRecorder) { if hasRange && iter.RangeKeyChanged() { rangeKeys := iter.RangeKeys() rkStart, rkEnd := iter.RangeBounds() + + span := &keyspan.Span{Start: rkStart, End: rkEnd, Keys: make([]keyspan.Key, len(rangeKeys))} for i := range rangeKeys { - if err := w.RangeKeySet(rkStart, rkEnd, rangeKeys[i].Suffix, rangeKeys[i].Value); err != nil { - panic(err) + span.Keys[i] = keyspan.Key{ + Trailer: base.MakeTrailer(0, base.InternalKeyKindRangeKeySet), + Suffix: rangeKeys[i].Suffix, + Value: rangeKeys[i].Value, } } + keyspan.SortKeysByTrailer(&span.Keys) + if err := rangekey.Encode(span, w.AddRangeKey); err != nil { + panic(err) + } } } + if err := iter.Error(); err != nil { + h.Recordf("%s // %v", r, err) + return + } if err := w.Close(); err != nil { panic(err) } diff --git a/metamorphic/options.go b/metamorphic/options.go index b288c440682..7ce03e5bfb9 100644 --- a/metamorphic/options.go +++ b/metamorphic/options.go @@ -124,6 +124,16 @@ func parseOptions( return true } return true + case "TestOptions.use_shared_replicate": + opts.useSharedReplicate = true + return true + case "TestOptions.use_excise": + opts.useExcise = true + return true + case "TestOptions.efos_always_creates_iterators": + opts.efosAlwaysCreatesIters = true + opts.Opts.TestingAlwaysCreateEFOSIterators(true /* value */) + return true default: if customOptionParsers == nil { return false @@ -191,6 +201,15 @@ func optionsToString(opts *TestOptions) string { if opts.ingestSplit { fmt.Fprintf(&buf, " ingest_split=%v\n", opts.ingestSplit) } + if opts.useSharedReplicate { + fmt.Fprintf(&buf, " use_shared_replicate=%v\n", opts.useSharedReplicate) + } + if opts.useExcise { + fmt.Fprintf(&buf, " use_excise=%v\n", opts.useExcise) + } + if opts.efosAlwaysCreatesIters { + fmt.Fprintf(&buf, " efos_always_creates_iterators=%v\n", opts.efosAlwaysCreatesIters) + } for _, customOpt := range opts.CustomOpts { fmt.Fprintf(&buf, " %s=%s\n", customOpt.Name(), customOpt.Value()) } @@ -267,6 +286,16 @@ type TestOptions struct { // Enables ingest splits. Saved here for serialization as Options does not // serialize this. ingestSplit bool + // Enables operations that do excises. Note that a false value for this does + // not guarantee the lack of excises, as useSharedReplicate can also cause + // excises. However !useExcise && !useSharedReplicate can be used to guarantee + // lack of excises. + useExcise bool + // Enables EFOS to always create iterators, even if a conflicting excise + // happens. Used to guarantee EFOS determinism when conflicting excises are + // in play. If false, EFOS determinism is maintained by having the DB do a + // flush after every new EFOS. + efosAlwaysCreatesIters bool } // CustomOption defines a custom option that configures the behavior of an @@ -571,6 +600,14 @@ func randomOptions( testOpts.seedEFOS = rng.Uint64() testOpts.ingestSplit = rng.Intn(2) == 0 opts.Experimental.IngestSplit = func() bool { return testOpts.ingestSplit } + testOpts.useExcise = rng.Intn(2) == 0 + if testOpts.useExcise || testOpts.useSharedReplicate { + //testOpts.efosAlwaysCreatesIters = rng.Intn(2) == 0 + opts.TestingAlwaysCreateEFOSIterators(testOpts.efosAlwaysCreatesIters) + if testOpts.Opts.FormatMajorVersion < pebble.FormatVirtualSSTables { + testOpts.Opts.FormatMajorVersion = pebble.FormatVirtualSSTables + } + } testOpts.Opts.EnsureDefaults() return testOpts } diff --git a/metamorphic/parser.go b/metamorphic/parser.go index 75aa040150d..b4f7f801729 100644 --- a/metamorphic/parser.go +++ b/metamorphic/parser.go @@ -69,6 +69,8 @@ func opArgs(op op) (receiverID *objID, targetID *objID, args []interface{}) { return &t.readerID, nil, []interface{}{&t.key} case *ingestOp: return &t.dbID, nil, []interface{}{&t.batchIDs} + case *ingestAndExciseOp: + return &t.dbID, nil, []interface{}{&t.batchID, &t.exciseStart, &t.exciseEnd} case *initOp: return nil, nil, []interface{}{&t.dbSlots, &t.batchSlots, &t.iterSlots, &t.snapshotSlots} case *iterLastOp: @@ -130,6 +132,7 @@ var methods = map[string]*methodInfo{ "Flush": makeMethod(flushOp{}, dbTag), "Get": makeMethod(getOp{}, dbTag, batchTag, snapTag), "Ingest": makeMethod(ingestOp{}, dbTag), + "IngestAndExcise": makeMethod(ingestAndExciseOp{}, dbTag), "Init": makeMethod(initOp{}, dbTag), "Last": makeMethod(iterLastOp{}, iterTag), "Merge": makeMethod(mergeOp{}, dbTag, batchTag), diff --git a/options.go b/options.go index 8a2c609ef00..244b101249c 100644 --- a/options.go +++ b/options.go @@ -973,6 +973,11 @@ type Options struct { // against the FS are made after the DB is closed, the FS may leak a // goroutine indefinitely. fsCloser io.Closer + + // efosAlwaysCreatesIterators is set by some tests to force + // EventuallyFileOnlySnapshots to always create iterators, even after a + // conflicting excise. + efosAlwaysCreatesIterators bool } } @@ -1146,6 +1151,13 @@ func (o *Options) AddEventListener(l EventListener) { o.EventListener = &l } +// TestingAlwaysCreateEFOSIterators is used to toggle a private option for +// having EventuallyFileOnlySnapshots always create iterators. Meant to only +// be used in tests. +func (o *Options) TestingAlwaysCreateEFOSIterators(value bool) { + o.private.efosAlwaysCreatesIterators = value +} + func (o *Options) equal() Equal { if o.Comparer.Equal == nil { return bytes.Equal diff --git a/snapshot.go b/snapshot.go index 31536b34ab3..0d3ee4f3878 100644 --- a/snapshot.go +++ b/snapshot.go @@ -247,6 +247,10 @@ type EventuallyFileOnlySnapshot struct { snap *Snapshot // The wrapped version reference, if a file-only snapshot. vers *version + + // The readState corresponding to when this EFOS was created. Only set + // if alwaysCreateIters is true. + rs *readState } // Key ranges to watch for an excise on. @@ -259,6 +263,12 @@ type EventuallyFileOnlySnapshot struct { db *DB seqNum uint64 + // If true, this EventuallyFileOnlySnapshot will always generate iterators that + // retain snapshot semantics, by holding onto the readState if a conflicting + // excise were to happen. Only used in some tests to enforce deterministic + // behaviour around excises. + alwaysCreateIters bool + closed chan struct{} } @@ -279,10 +289,14 @@ func (d *DB) makeEventuallyFileOnlySnapshot( } } es := &EventuallyFileOnlySnapshot{ - db: d, - seqNum: seqNum, - protectedRanges: keyRanges, - closed: make(chan struct{}), + db: d, + seqNum: seqNum, + protectedRanges: keyRanges, + closed: make(chan struct{}), + alwaysCreateIters: d.opts.private.efosAlwaysCreatesIterators, + } + if es.alwaysCreateIters { + es.mu.rs = d.loadReadState() } if isFileOnly { es.mu.vers = d.mu.versions.currentVersion() @@ -426,6 +440,9 @@ func (es *EventuallyFileOnlySnapshot) Close() error { if es.mu.vers != nil { es.mu.vers.UnrefLocked() } + if es.mu.rs != nil { + es.mu.rs.unrefLocked() + } return nil } @@ -479,15 +496,21 @@ func (es *EventuallyFileOnlySnapshot) NewIterWithContext( return es.db.newIter(ctx, nil /* batch */, newIterOpts{snapshot: sOpts}, o), nil } + sOpts := snapshotIterOpts{seqNum: es.seqNum} if es.excised.Load() { - return nil, ErrSnapshotExcised + if !es.alwaysCreateIters { + return nil, ErrSnapshotExcised + } + if es.mu.rs == nil { + return nil, errors.AssertionFailedf("unexpected nil readState in EFOS' alwaysCreateIters mode") + } + sOpts.readState = es.mu.rs } - sOpts := snapshotIterOpts{seqNum: es.seqNum} iter := es.db.newIter(ctx, nil /* batch */, newIterOpts{snapshot: sOpts}, o) // If excised is true, then keys relevant to the snapshot might not be // present in the readState being used by the iterator. Error out. - if es.excised.Load() { + if es.excised.Load() && !es.alwaysCreateIters { iter.Close() return nil, ErrSnapshotExcised } @@ -512,7 +535,7 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( if es.db == nil { panic(ErrClosed) } - if es.excised.Load() { + if es.excised.Load() && !es.alwaysCreateIters { return ErrSnapshotExcised } var sOpts snapshotIterOpts @@ -523,8 +546,15 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( vers: es.mu.vers, } } else { - sOpts = snapshotIterOpts{ - seqNum: es.seqNum, + if es.excised.Load() && es.alwaysCreateIters { + sOpts = snapshotIterOpts{ + readState: es.mu.rs, + seqNum: es.seqNum, + } + } else { + sOpts = snapshotIterOpts{ + seqNum: es.seqNum, + } } } es.mu.Unlock() @@ -546,7 +576,7 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( // If excised is true, then keys relevant to the snapshot might not be // present in the readState being used by the iterator. Error out. - if es.excised.Load() { + if es.excised.Load() && !es.alwaysCreateIters { return ErrSnapshotExcised }