From 1d7852b392560ae6dffa520a9af1e9ec487dcf8c Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Mon, 22 Jan 2024 15:18:11 -0500 Subject: [PATCH] db: have excises wait for EFOS below them, reducing EFOS waits Previously, we had to make EventuallyFileOnlySnapshots (EFOS) wait for all relevant keys in their key bounds to be flushed, just in case an excise came in before that happened. This change avoids the need to do that by assigning a seqnum to every excise (which will be useful when we do #2676), and by flushing any memtables that overlap with the protected ranges of any EFOS that has a sequence number below the excise. The reduction in waiting is desirable as actual conflicts between excises and EventuallyFileOnlySnapshots are rare, while users of EventuallyFileOnlySnapshots were previously stuck waiting or stuck doing excessive flushes at times when no such excises were happening. Fixes #3210. --- compaction.go | 8 --- ingest.go | 70 ++++++++++++++++++++-- metamorphic/ops.go | 8 --- metamorphic/options.go | 16 ----- options.go | 12 ---- snapshot.go | 113 +++-------------------------------- testdata/checkpoint | 20 +++---- testdata/concurrent_excise | 9 ++- testdata/excise | 29 +++++---- testdata/ingest_shared | 47 +++++++++------ testdata/ingest_shared_lower | 38 +++++++----- testdata/metrics | 10 ++-- testdata/table_stats | 40 ++++++------- 13 files changed, 183 insertions(+), 237 deletions(-) diff --git a/compaction.go b/compaction.go index 09482da4b2..f08d7dc151 100644 --- a/compaction.go +++ b/compaction.go @@ -2067,14 +2067,6 @@ func (d *DB) flush1() (bytesFlushed uint64, err error) { s = s.next continue } - if s.efos.excised.Load() { - // If a concurrent excise has happened that overlaps with one of the key - // ranges this snapshot is interested in, this EFOS cannot transition to - // a file-only snapshot as keys in that range could now be deleted. Move - // onto the next snapshot. - s = s.next - continue - } currentVersion.Ref() // NB: s.efos.transitionToFileOnlySnapshot could close s, in which diff --git a/ingest.go b/ingest.go index 03e1c9c733..cd7849800d 100644 --- a/ingest.go +++ b/ingest.go @@ -1392,6 +1392,52 @@ func (d *DB) ingest( d.mu.Lock() defer d.mu.Unlock() + // Check if any of the currently-open EventuallyFileOnlySnapshots overlap + // in key ranges with the excise span. If so, we need to check for memtable + // overlaps with all bounds of that EventuallyFileOnlySnapshot in addition + // to the ingestion's own bounds too. + + if exciseSpan.Valid() { + for s := d.mu.snapshots.root.next; s != &d.mu.snapshots.root; s = s.next { + if s.efos == nil { + continue + } + if base.Visible(seqNum, s.efos.seqNum, base.InternalKeySeqNumMax) { + // We only worry about snapshots older than the excise. Any snapshots + // created after the excise should see the excised view of the LSM + // anyway. + // + // Since we delay publishing the excise seqnum as visible until after + // the apply step, this case will never be hit in practice until we + // make excises flushable ingests. + continue + } + if invariants.Enabled { + if s.efos.hasTransitioned() { + panic("unexpected transitioned EFOS in snapshots list") + } + } + for i := range s.efos.protectedRanges { + if !s.efos.protectedRanges[i].OverlapsKeyRange(d.cmp, exciseSpan) { + continue + } + // Our excise conflicts with this EFOS. We need to add its protected + // ranges to our overlapBounds. Grow overlapBounds in one allocation + // if necesary. + prs := s.efos.protectedRanges + if cap(overlapBounds) < len(overlapBounds)+len(prs) { + oldOverlapBounds := overlapBounds + overlapBounds = make([]bounded, len(oldOverlapBounds), len(oldOverlapBounds)+len(prs)) + copy(overlapBounds, oldOverlapBounds) + } + for i := range prs { + overlapBounds = append(overlapBounds, &prs[i]) + } + break + } + } + } + // Check to see if any files overlap with any of the memtables. The queue // is ordered from oldest to newest with the mutable memtable being the // last element in the slice. We want to wait for the newest table that @@ -1419,7 +1465,8 @@ func (d *DB) ingest( // flushable queue and switching to a new memtable. metaFlushableOverlaps[v.FileNum] = true case *KeyRange: - // An excise span; not a file. + // An excise span or an EventuallyFileOnlySnapshot protected range; + // not a file. default: panic("unreachable") } @@ -1491,6 +1538,13 @@ func (d *DB) ingest( return } + // If there's an excise being done atomically with the same ingest, we + // assign the lowest sequence number in the set of sequence numbers for this + // ingestion to the excise. Note that we've already allocated fileCount+1 + // sequence numbers in this case. + if exciseSpan.Valid() { + seqNum++ // the first seqNum is reserved for the excise. + } // Update the sequence numbers for all ingested sstables' // metadata. When the version edit is applied, the metadata is // written to the manifest, persisting the sequence number. @@ -1524,8 +1578,12 @@ func (d *DB) ingest( // the commit mutex which would prevent unrelated batches from writing their // changes to the WAL and memtable. This will cause a bigger commit hiccup // during ingestion. + seqNumCount := loadResult.fileCount + if exciseSpan.Valid() { + seqNumCount++ + } d.commit.ingestSem <- struct{}{} - d.commit.AllocateSeqNum(loadResult.fileCount, prepare, apply) + d.commit.AllocateSeqNum(seqNumCount, prepare, apply) <-d.commit.ingestSem if err != nil { @@ -2253,7 +2311,10 @@ func (d *DB) ingestApply( } } // Check for any EventuallyFileOnlySnapshots that could be watching for - // an excise on this span. + // an excise on this span. There should be none as the + // computePossibleOverlaps steps should have forced these EFOS to transition + // to file-only snapshots by now. If we see any that conflict with this + // excise, panic. if exciseSpan.Valid() { for s := d.mu.snapshots.root.next; s != &d.mu.snapshots.root; s = s.next { if s.efos == nil { @@ -2266,8 +2327,7 @@ func (d *DB) ingestApply( // snapshot. for i := range efos.protectedRanges { if efos.protectedRanges[i].OverlapsKeyRange(d.cmp, exciseSpan) { - efos.excised.Store(true) - break + panic("unexpected excise of an EventuallyFileOnlySnapshot's bounds") } } } diff --git a/metamorphic/ops.go b/metamorphic/ops.go index ed4c8c64a6..5bb57e8852 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -1664,14 +1664,6 @@ func (o *newSnapshotOp) run(t *Test, h historyRecorder) { 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) - } - } } else { s := t.getDB(o.dbID).NewSnapshot() t.setSnapshot(o.snapID, s) diff --git a/metamorphic/options.go b/metamorphic/options.go index 9f2fa46673..c99f1a6577 100644 --- a/metamorphic/options.go +++ b/metamorphic/options.go @@ -135,10 +135,6 @@ func parseOptions( 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 @@ -215,9 +211,6 @@ func optionsToString(opts *TestOptions) string { 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()) } @@ -320,11 +313,6 @@ type TestOptions struct { // 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 @@ -666,10 +654,6 @@ func RandomOptions( testOpts.Opts.FormatMajorVersion = pebble.FormatVirtualSSTables } } - if testOpts.useExcise || testOpts.useSharedReplicate { - testOpts.efosAlwaysCreatesIters = rng.Intn(2) == 0 - opts.TestingAlwaysCreateEFOSIterators(testOpts.efosAlwaysCreatesIters) - } testOpts.Opts.EnsureDefaults() return testOpts } diff --git a/options.go b/options.go index c51746d448..799ace9528 100644 --- a/options.go +++ b/options.go @@ -1056,11 +1056,6 @@ 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 } } @@ -1237,13 +1232,6 @@ 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 1714765a32..192b5ae757 100644 --- a/snapshot.go +++ b/snapshot.go @@ -9,20 +9,13 @@ import ( "io" "math" "sync" - "sync/atomic" "time" - "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/invariants" "github.com/cockroachdb/pebble/rangekey" "github.com/cockroachdb/pebble/sstable" ) -// ErrSnapshotExcised is returned from WaitForFileOnlySnapshot if an excise -// overlapping with one of the EventuallyFileOnlySnapshot's KeyRanges gets -// applied before the transition of that EFOS to a file-only snapshot. -var ErrSnapshotExcised = errors.New("pebble: snapshot excised before conversion to file-only snapshot") - // Snapshot provides a read-only point-in-time view of the DB state. type Snapshot struct { // The db the snapshot was created from. @@ -228,13 +221,12 @@ func (l *snapshotList) remove(s *Snapshot) { // the snapshot is closed may prefer EventuallyFileOnlySnapshots for their // reduced write amplification. Callers that desire the benefits of the file-only // state that requires no pinning of memtables should call -// `WaitForFileOnlySnapshot()` (and possibly re-mint an EFOS if it returns -// ErrSnapshotExcised) before relying on the EFOS to keep producing iterators +// `WaitForFileOnlySnapshot()` before relying on the EFOS to keep producing iterators // with zero write-amp and zero pinning of memtables in memory. // // EventuallyFileOnlySnapshots interact with the IngestAndExcise operation in -// subtle ways. No new iterators can be created once -// EventuallyFileOnlySnapshot.excised is set to true. +// subtle ways. The IngestAndExcise can force the transition of an EFOS to a +// file-only snapshot if an excise overlaps with the EFOS bounds. type EventuallyFileOnlySnapshot struct { mu struct { // NB: If both this mutex and db.mu are being grabbed, db.mu should be @@ -250,28 +242,14 @@ 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. protectedRanges []KeyRange - // excised, if true, signals that the above ranges were excised during the - // lifetime of this snapshot. - excised atomic.Bool // The db the snapshot was created from. 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{} } @@ -289,14 +267,10 @@ func (d *DB) makeEventuallyFileOnlySnapshot(keyRanges []KeyRange) *EventuallyFil }, sliceAsBounded(keyRanges)...) } es := &EventuallyFileOnlySnapshot{ - db: d, - seqNum: seqNum, - protectedRanges: keyRanges, - closed: make(chan struct{}), - alwaysCreateIters: d.opts.private.efosAlwaysCreatesIterators, - } - if es.alwaysCreateIters { - es.mu.rs = d.loadReadState() + db: d, + seqNum: seqNum, + protectedRanges: keyRanges, + closed: make(chan struct{}), } if isFileOnly { es.mu.vers = d.mu.versions.currentVersion() @@ -390,9 +364,6 @@ func (es *EventuallyFileOnlySnapshot) waitForFlush(ctx context.Context, dur time earliestUnflushedSeqNum = es.db.getEarliestUnflushedSeqNumLocked() } - if es.excised.Load() { - return ErrSnapshotExcised - } return nil } @@ -440,9 +411,6 @@ 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 } @@ -478,31 +446,6 @@ func (es *EventuallyFileOnlySnapshot) NewIter(o *IterOptions) (*Iterator, error) return es.NewIterWithContext(context.Background(), o) } -func (es *EventuallyFileOnlySnapshot) newAlwaysCreateIterWithContext( - ctx context.Context, o *IterOptions, -) (*Iterator, error) { - // Grab the db mutex. This avoids races down below, where we could get - // excised between the es.excised.Load() call, and the newIter call. - es.db.mu.Lock() - defer es.db.mu.Unlock() - es.mu.Lock() - defer es.mu.Unlock() - if es.mu.vers != nil { - sOpts := snapshotIterOpts{seqNum: es.seqNum, vers: es.mu.vers} - return es.db.newIter(ctx, nil /* batch */, newIterOpts{snapshot: sOpts}, o), nil - } - - sOpts := snapshotIterOpts{seqNum: es.seqNum} - if es.excised.Load() { - if es.mu.rs == nil { - return nil, errors.AssertionFailedf("unexpected nil readState in EFOS' alwaysCreateIters mode") - } - sOpts.readState = es.mu.rs - } - iter := es.db.newIter(ctx, nil /* batch */, newIterOpts{snapshot: sOpts}, o) - return iter, nil -} - // NewIterWithContext is like NewIter, and additionally accepts a context for // tracing. func (es *EventuallyFileOnlySnapshot) NewIterWithContext( @@ -514,9 +457,6 @@ func (es *EventuallyFileOnlySnapshot) NewIterWithContext( default: } - if es.alwaysCreateIters { - return es.newAlwaysCreateIterWithContext(ctx, o) - } es.mu.Lock() defer es.mu.Unlock() if es.mu.vers != nil { @@ -525,17 +465,7 @@ func (es *EventuallyFileOnlySnapshot) NewIterWithContext( } sOpts := snapshotIterOpts{seqNum: es.seqNum} - if es.excised.Load() { - return nil, ErrSnapshotExcised - } 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. - if es.excised.Load() { - iter.Close() - return nil, ErrSnapshotExcised - } return iter, nil } @@ -557,9 +487,6 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( if es.db == nil { panic(ErrClosed) } - if es.excised.Load() && !es.alwaysCreateIters { - return ErrSnapshotExcised - } var sOpts snapshotIterOpts opts := &scanInternalOptions{ CategoryAndQoS: categoryAndQoS, @@ -574,11 +501,6 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( visitSharedFile: visitSharedFile, skipSharedLevels: visitSharedFile != nil, } - if es.alwaysCreateIters { - // Grab the db mutex. This avoids races down below as it prevents excises - // from taking effect until the iterator is instantiated. - es.db.mu.Lock() - } es.mu.Lock() if es.mu.vers != nil { sOpts = snapshotIterOpts{ @@ -586,15 +508,8 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( vers: es.mu.vers, } } else { - if es.excised.Load() && es.alwaysCreateIters { - sOpts = snapshotIterOpts{ - readState: es.mu.rs, - seqNum: es.seqNum, - } - } else { - sOpts = snapshotIterOpts{ - seqNum: es.seqNum, - } + sOpts = snapshotIterOpts{ + seqNum: es.seqNum, } } es.mu.Unlock() @@ -603,16 +518,6 @@ func (es *EventuallyFileOnlySnapshot) ScanInternal( return err } defer iter.close() - if es.alwaysCreateIters { - // See the similar conditional above where we grab this mutex. - es.db.mu.Unlock() - } - - // 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() && !es.alwaysCreateIters { - return ErrSnapshotExcised - } return scanInternalImpl(ctx, lower, upper, iter, opts) } diff --git a/testdata/checkpoint b/testdata/checkpoint index edf0f63841..2e001d92f1 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -432,7 +432,7 @@ lsm db 6: 000012:[a#0,SET-b#0,SET] 000013:[d#0,SET-g#0,SET] - 000011:[i#19,SET-k#19,SET] + 000011:[i#20,SET-k#20,SET] build db ext2 format=pebblev2 set z z @@ -447,9 +447,9 @@ lsm db 6: 000012:[a#0,SET-b#0,SET] 000013:[d#0,SET-g#0,SET] - 000015:[i#19,SET-i#19,SET] - 000016:[k#19,SET-k#19,SET] - 000014:[z#20,SET-z#20,SET] + 000015:[i#20,SET-i#20,SET] + 000016:[k#20,SET-k#20,SET] + 000014:[z#22,SET-z#22,SET] # scan db so that it is known what to expect from the checkpoints. scan db @@ -655,9 +655,9 @@ lsm checkpoints/checkpoint5 000017:[h#18,SET-h#18,SET] 6: 000013:[d#0,SET-g#0,SET] - 000015:[i#19,SET-i#19,SET] - 000016:[k#19,SET-k#19,SET] - 000014:[z#20,SET-z#20,SET] + 000015:[i#20,SET-i#20,SET] + 000016:[k#20,SET-k#20,SET] + 000014:[z#22,SET-z#22,SET] close checkpoints/checkpoint5 ---- @@ -742,6 +742,6 @@ lsm checkpoints/checkpoint6 0.0: 000017:[h#18,SET-h#18,SET] 6: - 000015:[i#19,SET-i#19,SET] - 000016:[k#19,SET-k#19,SET] - 000014:[z#20,SET-z#20,SET] + 000015:[i#20,SET-i#20,SET] + 000016:[k#20,SET-k#20,SET] + 000014:[z#22,SET-z#22,SET] diff --git a/testdata/concurrent_excise b/testdata/concurrent_excise index 135566f22d..6ee8e71db3 100644 --- a/testdata/concurrent_excise +++ b/testdata/concurrent_excise @@ -89,7 +89,7 @@ ok lsm ---- 6: - 000010:[d#13,SET-d#13,SET] + 000010:[d#14,SET-d#14,SET] 000011:[f#11,SET-f#11,SET] compact a-z @@ -165,7 +165,7 @@ replicated 1 shared SSTs wait-for-file-only-snapshot s2 ---- -pebble: snapshot excised before conversion to file-only snapshot +ok iter snapshot=s2 first @@ -173,4 +173,7 @@ next next next ---- -pebble: snapshot excised before conversion to file-only snapshot +d: (something, .) +dd: (memory, .) +e: (bar, .) +. diff --git a/testdata/excise b/testdata/excise index 116581b26d..0de6c40117 100644 --- a/testdata/excise +++ b/testdata/excise @@ -106,7 +106,7 @@ lsm ---- 6: 000019:[a#10,SET-a#10,SET] - 000018:[d#13,SET-e#13,SET] + 000018:[d#14,SET-e#14,SET] 000020:[l#10,SET-l#10,SET] iter @@ -211,7 +211,7 @@ lsm 000007:[a#10,SET-a#10,SET] 000008:[d#12,SET-ee#inf,RANGEKEYSET] 6: - 000006:[z#14,SET-z#14,SET] + 000006:[z#15,SET-z#15,SET] # Regression test for https://github.com/cockroachdb/pebble/issues/2947. reset @@ -251,7 +251,7 @@ lsm 000007:[a#10,SET-c#12,SET] 000008:[e#14,SET-j#19,SET] 6: - 000006:[z#20,SET-z#20,SET] + 000006:[z#21,SET-z#21,SET] build ext3 set zz zz @@ -268,8 +268,8 @@ lsm 000010:[e#14,SET-f#15,SET] 000011:[h#17,SET-j#19,SET] 6: - 000006:[z#20,SET-z#20,SET] - 000009:[zz#21,SET-zz#21,SET] + 000006:[z#21,SET-z#21,SET] + 000009:[zz#23,SET-zz#23,SET] confirm-backing 7 10 11 ---- @@ -287,8 +287,8 @@ lsm 000010:[e#14,SET-f#15,SET] 000011:[h#17,SET-j#19,SET] 6: - 000006:[z#20,SET-z#20,SET] - 000009:[zz#21,SET-zz#21,SET] + 000006:[z#21,SET-z#21,SET] + 000009:[zz#23,SET-zz#23,SET] confirm-backing 7 10 11 ---- @@ -326,7 +326,7 @@ lsm 0.0: 000007:[a#10,SET-c#12,SET] 6: - 000006:[z#15,SET-z#15,SET] + 000006:[z#16,SET-z#16,SET] reopen ---- @@ -336,7 +336,7 @@ lsm 0.0: 000007:[a#10,SET-c#12,SET] 6: - 000006:[z#15,SET-z#15,SET] + 000006:[z#16,SET-z#16,SET] # Regression test for #3128. A key at the upper bound of a virtual sstable # should not be skipped in reverse iteration with range key masking enabled. @@ -402,7 +402,7 @@ lsm ---- 0.0: 000011:[cc#17,RANGEKEYSET-f#17,MERGE] - 000010:[z#18,SET-z#18,SET] + 000010:[z#19,SET-z#19,SET] 6: 000012:[a@3#0,SET-bbsomethinglong@4#0,MERGE] 000013:[d@6#0,MERGE-z#0,MERGE] @@ -502,13 +502,16 @@ set c something ingest-and-excise ext5 excise=c-cc ---- +memtable flushed lsm ---- +0.0: + 000012:[x#14,SET-x#14,SET] 6: - 000011:[a#0,SET-b#0,SET] - 000010:[c#16,SET-c#16,SET] - 000012:[d@6#15,DEL-d@6#0,SET] + 000013:[a#0,SET-b#0,SET] + 000010:[c#17,SET-c#17,SET] + 000014:[d@6#15,DEL-d@6#0,SET] iter lower=c upper=e last diff --git a/testdata/ingest_shared b/testdata/ingest_shared index b7f659eef0..f7938d2a40 100644 --- a/testdata/ingest_shared +++ b/testdata/ingest_shared @@ -50,7 +50,7 @@ replicated 1 shared SSTs lsm ---- 6: - 000005:[d#10,SET-f#10,SET] + 000005:[d#11,SET-f#11,SET] iter first @@ -366,9 +366,9 @@ ok lsm ---- 0.0: - 000004:[a@3#11,SET-d#inf,RANGEDEL] + 000004:[a@3#12,SET-d#inf,RANGEDEL] 6: - 000005:[a@3#10,SET-e#10,SET] + 000005:[a@3#11,SET-e#11,SET] iter first @@ -465,11 +465,11 @@ ok lsm ---- 0.0: - 000004:[a@3#12,SET-c@9#12,SET] + 000004:[a@3#13,SET-c@9#13,SET] 5: - 000005:[b#11,RANGEDEL-d#inf,RANGEDEL] + 000005:[b#12,RANGEDEL-d#inf,RANGEDEL] 6: - 000006:[a@3#10,SET-e#10,SET] + 000006:[a@3#11,SET-e#11,SET] iter first @@ -540,7 +540,7 @@ lsm ---- 6: 000008:[a#10,RANGEKEYSET-aaa#inf,RANGEKEYSET] - 000007:[b#13,SET-c#13,SET] + 000007:[b#14,SET-c#14,SET] 000009:[d#11,SET-e#12,SET] iter @@ -621,9 +621,9 @@ replicated 2 shared SSTs lsm ---- 5: - 000007:[bb#12,RANGEDEL-f#inf,RANGEDEL] + 000007:[bb#13,RANGEDEL-f#inf,RANGEDEL] 6: - 000008:[b@5#11,SET-e#11,SET] + 000008:[b@5#12,SET-e#12,SET] 000005:[ff#10,SET-ff#10,SET] iter @@ -719,9 +719,9 @@ replicated 2 shared SSTs lsm ---- 5: - 000007:[bb#12,RANGEKEYSET-f#inf,RANGEKEYSET] + 000007:[bb#13,RANGEKEYSET-f#inf,RANGEKEYSET] 6: - 000008:[b@5#11,SET-e#11,SET] + 000008:[b@5#12,SET-e#12,SET] 000005:[ff#10,SET-ff#10,SET] iter @@ -958,9 +958,8 @@ set f foobar # The below file-only snapshot is the more challenging case of a partial overlap # between an excise and a file-only snapshot. In this case the EFOS transition -# blocks on the memtable but the excise proceeds through, causing the EFOS' -# WaitForFileOnlySnapshot() call to error out. Opening iterators also returns -# the same errors. +# blocks on the memtable. The excise should detect that and force the flush +# before it happens, and snapshot semantics should be maintained. file-only-snapshot s3 c g @@ -1004,7 +1003,7 @@ replicated 2 shared SSTs wait-for-file-only-snapshot s3 ---- -pebble: snapshot excised before conversion to file-only snapshot +ok iter snapshot=s3 first @@ -1013,7 +1012,11 @@ next next next ---- -pebble: snapshot excised before conversion to file-only snapshot +a: (foo, .) +b: (bar, .) +c: (baz, .) +f: (foobar, .) +. iter snapshot=s3 first @@ -1024,7 +1027,13 @@ next next next ---- -pebble: snapshot excised before conversion to file-only snapshot +a: (foo, .) +b: (bar, .) +. +a: (foo, .) +b: (bar, .) +c: (baz, .) +f: (foobar, .) iter first @@ -1220,7 +1229,7 @@ ok lsm ---- 6: - 000008:[c#11,RANGEKEYSET-e#inf,RANGEKEYSET] + 000008:[c#12,RANGEKEYSET-e#inf,RANGEKEYSET] switch 1 ---- @@ -1289,7 +1298,7 @@ ok lsm ---- 6: - 000005:[a#10,SET-a#10,DEL] + 000005:[a#11,SET-a#11,DEL] iter first diff --git a/testdata/ingest_shared_lower b/testdata/ingest_shared_lower index 98b7ba8715..400d1f4171 100644 --- a/testdata/ingest_shared_lower +++ b/testdata/ingest_shared_lower @@ -50,7 +50,7 @@ replicated 1 shared SSTs lsm ---- 6: - 000005:[d#10,SET-f#10,SET] + 000005:[d#11,SET-f#11,SET] iter first @@ -366,9 +366,9 @@ ok lsm ---- 0.0: - 000004:[a@3#11,SET-d#inf,RANGEDEL] + 000004:[a@3#12,SET-d#inf,RANGEDEL] 6: - 000005:[a@3#10,SET-e#10,SET] + 000005:[a@3#11,SET-e#11,SET] iter first @@ -465,11 +465,11 @@ ok lsm ---- 0.0: - 000004:[a@3#12,SET-c@9#12,SET] + 000004:[a@3#13,SET-c@9#13,SET] 5: - 000005:[b#11,RANGEDEL-d#inf,RANGEDEL] + 000005:[b#12,RANGEDEL-d#inf,RANGEDEL] 6: - 000006:[a@3#10,SET-e#10,SET] + 000006:[a@3#11,SET-e#11,SET] iter first @@ -540,7 +540,7 @@ lsm ---- 6: 000009:[a#10,RANGEKEYSET-aaa#inf,RANGEKEYSET] - 000008:[b#13,SET-c#13,SET] + 000008:[b#14,SET-c#14,SET] 000010:[d#11,SET-e#12,SET] iter @@ -621,9 +621,9 @@ replicated 2 shared SSTs lsm ---- 5: - 000008:[bb#12,RANGEDEL-f#inf,RANGEDEL] + 000008:[bb#13,RANGEDEL-f#inf,RANGEDEL] 6: - 000009:[b@5#11,SET-e#11,SET] + 000009:[b@5#12,SET-e#12,SET] 000006:[ff#10,SET-ff#10,SET] iter @@ -702,9 +702,9 @@ replicated 2 shared SSTs lsm ---- 5: - 000008:[bb#12,RANGEKEYSET-f#inf,RANGEKEYSET] + 000008:[bb#13,RANGEKEYSET-f#inf,RANGEKEYSET] 6: - 000009:[b@5#11,SET-e#11,SET] + 000009:[b@5#12,SET-e#12,SET] 000006:[ff#10,SET-ff#10,SET] iter @@ -987,7 +987,7 @@ replicated 2 shared SSTs wait-for-file-only-snapshot s3 ---- -pebble: snapshot excised before conversion to file-only snapshot +ok iter snapshot=s3 first @@ -996,7 +996,11 @@ next next next ---- -pebble: snapshot excised before conversion to file-only snapshot +a: (foo, .) +b: (bar, .) +c: (baz, .) +f: (foobar, .) +. iter snapshot=s3 first @@ -1007,7 +1011,13 @@ next next next ---- -pebble: snapshot excised before conversion to file-only snapshot +a: (foo, .) +b: (bar, .) +. +a: (foo, .) +b: (bar, .) +c: (baz, .) +f: (foobar, .) iter first diff --git a/testdata/metrics b/testdata/metrics index e44288bde1..250744d849 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -563,7 +563,7 @@ lsm 000008:[a#0,SET-b#0,SET] 000013:[c@20#0,SET-c@16#0,SET] 000014:[c@15#0,SET-c@14#0,SET] - 000028:[z#32,SET-z#32,SET] + 000028:[z#33,SET-z#33,SET] # There should be 2 backing tables. Note that tiny sstables have inaccurate # virtual sstable sizes. @@ -640,8 +640,8 @@ lsm 000008:[a#0,SET-b#0,SET] 000013:[c@20#0,SET-c@16#0,SET] 000014:[c@15#0,SET-c@14#0,SET] - 000028:[z#32,SET-z#32,SET] - 000031:[zz#33,SET-zz#33,SET] + 000028:[z#33,SET-z#33,SET] + 000031:[zz#35,SET-zz#35,SET] metrics-value num-backing @@ -663,8 +663,8 @@ compact a-z 000013:[c@20#0,SET-c@16#0,SET] 000014:[c@15#0,SET-c@14#0,SET] 000033:[d#0,SET-m#0,SET] - 000028:[z#32,SET-z#32,SET] - 000031:[zz#33,SET-zz#33,SET] + 000028:[z#33,SET-z#33,SET] + 000031:[zz#35,SET-zz#35,SET] # Virtual sstables metrics should be gone after the compaction. metrics-value diff --git a/testdata/table_stats b/testdata/table_stats index a14ef69812..65d748ebd3 100644 --- a/testdata/table_stats +++ b/testdata/table_stats @@ -634,7 +634,7 @@ lsm 000007:[a#10,SET-a#10,SET] 000008:[d#12,DEL-d#12,DEL] 6: - 000006:[f#13,SET-f#13,SET] + 000006:[f#14,SET-f#14,SET] metadata-stats file=7 ---- @@ -696,12 +696,12 @@ range-key-set e ee @1 foo flush ---- 0.1: - 000010:[a#14,SET-ee#inf,RANGEKEYSET] + 000010:[a#15,SET-ee#inf,RANGEKEYSET] 0.0: 000007:[a#10,SET-a#10,SET] 000008:[d#12,DEL-d#12,DEL] 6: - 000006:[f#13,SET-f#13,SET] + 000006:[f#14,SET-f#14,SET] properties file=10 rocksdb @@ -750,14 +750,14 @@ ingest-and-excise ext2 excise=b-c lsm ---- 0.1: - 000012:[a#14,SET-a#14,SET] - 000013:[d#16,SET-ee#inf,RANGEKEYSET] + 000012:[a#15,SET-a#15,SET] + 000013:[d#17,SET-ee#inf,RANGEKEYSET] 0.0: 000007:[a#10,SET-a#10,SET] 000008:[d#12,DEL-d#12,DEL] 6: - 000006:[f#13,SET-f#13,SET] - 000011:[z#18,SET-z#18,SET] + 000006:[f#14,SET-f#14,SET] + 000011:[z#20,SET-z#20,SET] metadata-stats file=12 ---- @@ -809,9 +809,9 @@ compact a-z 6: 000014:[a#0,SET-a#0,SET] 000015:[d#0,SETWITHDEL-d#0,SETWITHDEL] - 000016:[e#17,RANGEKEYSET-ee#inf,RANGEKEYSET] - 000006:[f#13,SET-f#13,SET] - 000011:[z#18,SET-z#18,SET] + 000016:[e#18,RANGEKEYSET-ee#inf,RANGEKEYSET] + 000006:[f#14,SET-f#14,SET] + 000011:[z#20,SET-z#20,SET] batch del-range a e @@ -820,13 +820,13 @@ del-range a e flush ---- 0.0: - 000018:[a#19,RANGEDEL-e#inf,RANGEDEL] + 000018:[a#21,RANGEDEL-e#inf,RANGEDEL] 6: 000014:[a#0,SET-a#0,SET] 000015:[d#0,SETWITHDEL-d#0,SETWITHDEL] - 000016:[e#17,RANGEKEYSET-ee#inf,RANGEKEYSET] - 000006:[f#13,SET-f#13,SET] - 000011:[z#18,SET-z#18,SET] + 000016:[e#18,RANGEKEYSET-ee#inf,RANGEKEYSET] + 000006:[f#14,SET-f#14,SET] + 000011:[z#20,SET-z#20,SET] properties file=18 rocksdb @@ -866,15 +866,15 @@ ingest-and-excise ext3 excise=b-c lsm ---- 0.0: - 000020:[a#19,RANGEDEL-b#inf,RANGEDEL] - 000021:[c#19,RANGEDEL-e#inf,RANGEDEL] + 000020:[a#21,RANGEDEL-b#inf,RANGEDEL] + 000021:[c#21,RANGEDEL-e#inf,RANGEDEL] 6: 000014:[a#0,SET-a#0,SET] 000015:[d#0,SETWITHDEL-d#0,SETWITHDEL] - 000016:[e#17,RANGEKEYSET-ee#inf,RANGEKEYSET] - 000006:[f#13,SET-f#13,SET] - 000011:[z#18,SET-z#18,SET] - 000019:[zz#20,SET-zz#20,SET] + 000016:[e#18,RANGEKEYSET-ee#inf,RANGEKEYSET] + 000006:[f#14,SET-f#14,SET] + 000011:[z#20,SET-z#20,SET] + 000019:[zz#23,SET-zz#23,SET] properties file=20 ----