diff --git a/compaction.go b/compaction.go index d973fe1495..cf95cf7ee8 100644 --- a/compaction.go +++ b/compaction.go @@ -2042,7 +2042,8 @@ func (d *DB) runCompaction( } c.allowedZeroSeqNum = c.allowZeroSeqNum() iter := newCompactionIter(c.cmp, c.formatKey, d.merge, iiter, snapshots, - &c.rangeDelFrag, c.allowedZeroSeqNum, c.elideTombstone, c.elideRangeTombstone) + &c.rangeDelFrag, c.allowedZeroSeqNum, c.elideTombstone, + c.elideRangeTombstone, d.FormatMajorVersion()) var ( filenames []string diff --git a/compaction_iter.go b/compaction_iter.go index 83aa339f40..17d66fcd5b 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -7,6 +7,7 @@ package pebble import ( "io" "sort" + "strconv" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" @@ -161,6 +162,10 @@ type compactionIter struct { // determine when iteration has advanced to a new user key and thus a new // snapshot stripe. keyBuf []byte + // Temporary buffer used for storing the previous value, which may be an + // unsafe, i.iter-owned slice that could be altered when the iterator is + // advanced. + valueBuf []byte // Is the current entry valid? valid bool iterKey *InternalKey @@ -199,6 +204,9 @@ type compactionIter struct { allowZeroSeqNum bool elideTombstone func(key []byte) bool elideRangeTombstone func(start, end []byte) bool + // The on-disk format major version. This informs the types of keys that + // may be written to disk during a compaction. + formatVersion FormatMajorVersion } func newCompactionIter( @@ -211,6 +219,7 @@ func newCompactionIter( allowZeroSeqNum bool, elideTombstone func(key []byte) bool, elideRangeTombstone func(start, end []byte) bool, + formatVersion FormatMajorVersion, ) *compactionIter { i := &compactionIter{ cmp: cmp, @@ -221,6 +230,7 @@ func newCompactionIter( allowZeroSeqNum: allowZeroSeqNum, elideTombstone: elideTombstone, elideRangeTombstone: elideRangeTombstone, + formatVersion: formatVersion, } i.rangeDelFrag.Cmp = cmp i.rangeDelFrag.Format = formatKey @@ -338,12 +348,12 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { continue } - case InternalKeyKindSet: - i.saveKey() - i.value = i.iterValue - i.valid = true - i.skip = true - i.maybeZeroSeqnum(i.curSnapshotIdx) + case InternalKeyKindSet, InternalKeyKindSetWithDelete: + // The key we emit for this entry is a function of the current key + // kind, and whether this entry is followed by a DEL entry. + // setNext() does the work to move the iterator forward, preserving + // the original value, and potentially mutating the key kind. + i.setNext() return &i.key, i.value case InternalKeyKindMerge: @@ -485,6 +495,87 @@ func (i *compactionIter) nextInStripe() stripeChangeType { return newStripe } +func (i *compactionIter) setNext() { + // Save the current key. + i.saveKey() + i.value = i.iterValue + i.valid = true + i.maybeZeroSeqnum(i.curSnapshotIdx) + + // There are two cases where we can early return and skip the remaining + // records in the stripe: + // - If the DB does not SETWITHDEL. + // - If this key is already a SETWITHDEL. + if i.formatVersion < FormatSetWithDelete || + i.iterKey.Kind() == InternalKeyKindSetWithDelete { + i.skip = true + return + } + + // We are iterating forward. Save the current value. + i.valueBuf = append(i.valueBuf[:0], i.iterValue...) + i.value = i.valueBuf + + // Else, we continue to loop through entries in the stripe looking for a + // DEL. Note that we may stop *before* encountering a DEL, if one exists. + for { + switch t := i.nextInStripe(); t { + case newStripe, sameStripeNonSkippable: + i.pos = iterPosNext + if t == sameStripeNonSkippable { + // We iterated onto a key that we cannot skip. We can + // conservatively transform the original SET into a SETWITHDEL + // as an indication that there *may* still be a DEL under this + // SET, even if we did not actually encounter one. + // + // This is safe to do, as: + // + // - in the case that there *is not* actually a DEL under this + // entry, any SINGLEDEL above this now-transformed SETWITHDEL + // will become a DEL when the two encounter in a compaction. The + // DEL will eventually be elided in a subsequent compaction. The + // cost for ensuring correctness is that this entry is kept + // around for an additional compaction cycle(s). + // + // - in the case there *is* indeed a DEL under us (but in a + // different stripe or sstable), then we will have already done + // the work to transform the SET into a SETWITHDEL, and we will + // skip any additional iteration when this entry is encountered + // again in a subsequent compaction. + // + // Ideally, this codepath would be smart enough to handle the + // case of SET <- RANGEDEL <- ... <- DEL <- .... This requires + // preserving any RANGEDEL entries we encounter along the way, + // then emitting the original (possibly transformed) key, + // followed by the RANGEDELs. This requires a sizable + // refactoring of the existing code, as nextInStripe currently + // returns a sameStripeNonSkippable when it encounters a + // RANGEDEL. + // TODO(travers): optimize to handle the RANGEDEL case if it + // turns out to be a performance problem. + i.key.SetKind(InternalKeyKindSetWithDelete) + + // By setting i.skip=true, we are saying that after the + // non-skippable key is emitted (which is likely a RANGEDEL), + // the remaining point keys that share the same user key as this + // saved key should be skipped. + i.skip = true + } + return + case sameStripeSkippable: + // We're still in the same stripe. If this is a DEL, we stop looking + // and emit a SETWITHDEL. Subsequent keys are eligible for skipping. + if i.iterKey.Kind() == InternalKeyKindDelete { + i.key.SetKind(InternalKeyKindSetWithDelete) + i.skip = true + return + } + default: + panic("pebble: unexpected stripeChangeType: " + strconv.Itoa(int(t))) + } + } +} + func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { // Save the current key. i.saveKey() @@ -520,7 +611,7 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { i.skip = true return sameStripeSkippable - case InternalKeyKindSet: + case InternalKeyKindSet, InternalKeyKindSetWithDelete: if i.rangeDelFrag.Deleted(*key, i.curSnapshotSeqNum) { // We change the kind of the result key to a Set so that it shadows // keys in lower levels. That is, MERGE+RANGEDEL -> SET. This isn't @@ -531,9 +622,10 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { return sameStripeSkippable } - // We've hit a Set value. Merge with the existing value and return. We - // change the kind of the resulting key to a Set so that it shadows keys - // in lower levels. That is, MERGE+SET -> SET. + // We've hit a Set or SetWithDel value. Merge with the existing + // value and return. We change the kind of the resulting key to a + // Set so that it shadows keys in lower levels. That is: + // MERGE + (SET*) -> SET. i.err = valueMerger.MergeOlder(i.iterValue) if i.err != nil { i.valid = false @@ -585,8 +677,9 @@ func (i *compactionIter) singleDeleteNext() bool { key := i.iterKey switch key.Kind() { - case InternalKeyKindDelete, InternalKeyKindMerge: - // We've hit a Delete or Merge, transform the SingleDelete into a full Delete. + case InternalKeyKindDelete, InternalKeyKindMerge, InternalKeyKindSetWithDelete: + // We've hit a Delete, Merge or SetWithDelete, transform the + // SingleDelete into a full Delete. i.key.SetKind(InternalKeyKindDelete) i.skip = true return true diff --git a/compaction_iter_test.go b/compaction_iter_test.go index 872e807ff7..dc0c3be0b8 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -79,7 +79,16 @@ func TestCompactionIter(t *testing.T) { var elideTombstones bool var allowZeroSeqnum bool - newIter := func() *compactionIter { + // The input to the data-driven test is dependent on the format major + // version we are testing against. + fileFunc := func(formatVersion FormatMajorVersion) string { + if formatVersion < FormatSetWithDelete { + return "testdata/compaction_iter" + } + return "testdata/compaction_iter_set_with_del" + } + + newIter := func(formatVersion FormatMajorVersion) *compactionIter { // To adhere to the existing assumption that range deletion blocks in // SSTables are not released while iterating, and therefore not // susceptible to use-after-free bugs, we skip the zeroing of @@ -105,96 +114,113 @@ func TestCompactionIter(t *testing.T) { func(_, _ []byte) bool { return elideTombstones }, + formatVersion, ) } - datadriven.RunTest(t, "testdata/compaction_iter", func(d *datadriven.TestData) string { - switch d.Cmd { - case "define": - keys = keys[:0] - vals = vals[:0] - for _, key := range strings.Split(d.Input, "\n") { - j := strings.Index(key, ":") - keys = append(keys, base.ParseInternalKey(key[:j])) - vals = append(vals, []byte(key[j+1:])) - } - return "" - - case "iter": - snapshots = snapshots[:0] - elideTombstones = false - allowZeroSeqnum = false - for _, arg := range d.CmdArgs { - switch arg.Key { - case "snapshots": - for _, val := range arg.Vals { - seqNum, err := strconv.Atoi(val) + runTest := func(t *testing.T, formatVersion FormatMajorVersion) { + datadriven.RunTest(t, fileFunc(formatVersion), func(d *datadriven.TestData) string { + switch d.Cmd { + case "define": + keys = keys[:0] + vals = vals[:0] + for _, key := range strings.Split(d.Input, "\n") { + j := strings.Index(key, ":") + keys = append(keys, base.ParseInternalKey(key[:j])) + vals = append(vals, []byte(key[j+1:])) + } + return "" + + case "iter": + snapshots = snapshots[:0] + elideTombstones = false + allowZeroSeqnum = false + for _, arg := range d.CmdArgs { + switch arg.Key { + case "snapshots": + for _, val := range arg.Vals { + seqNum, err := strconv.Atoi(val) + if err != nil { + return err.Error() + } + snapshots = append(snapshots, uint64(seqNum)) + } + case "elide-tombstones": + var err error + elideTombstones, err = strconv.ParseBool(arg.Vals[0]) if err != nil { return err.Error() } - snapshots = append(snapshots, uint64(seqNum)) - } - case "elide-tombstones": - var err error - elideTombstones, err = strconv.ParseBool(arg.Vals[0]) - if err != nil { - return err.Error() - } - case "allow-zero-seqnum": - var err error - allowZeroSeqnum, err = strconv.ParseBool(arg.Vals[0]) - if err != nil { - return err.Error() + case "allow-zero-seqnum": + var err error + allowZeroSeqnum, err = strconv.ParseBool(arg.Vals[0]) + if err != nil { + return err.Error() + } + default: + return fmt.Sprintf("%s: unknown arg: %s", d.Cmd, arg.Key) } - default: - return fmt.Sprintf("%s: unknown arg: %s", d.Cmd, arg.Key) - } - } - sort.Slice(snapshots, func(i, j int) bool { - return snapshots[i] < snapshots[j] - }) - - iter := newIter() - var b bytes.Buffer - for _, line := range strings.Split(d.Input, "\n") { - parts := strings.Fields(line) - if len(parts) == 0 { - continue } - switch parts[0] { - case "first": - iter.First() - case "next": - iter.Next() - case "tombstones": - var key []byte - if len(parts) == 2 { - key = []byte(parts[1]) + sort.Slice(snapshots, func(i, j int) bool { + return snapshots[i] < snapshots[j] + }) + + iter := newIter(formatVersion) + var b bytes.Buffer + for _, line := range strings.Split(d.Input, "\n") { + parts := strings.Fields(line) + if len(parts) == 0 { + continue } - for _, v := range iter.Tombstones(key, false) { - fmt.Fprintf(&b, "%s-%s#%d\n", - v.Start.UserKey, v.End, v.Start.SeqNum()) + switch parts[0] { + case "first": + iter.First() + case "next": + iter.Next() + case "tombstones": + var key []byte + if len(parts) == 2 { + key = []byte(parts[1]) + } + for _, v := range iter.Tombstones(key, false) { + fmt.Fprintf(&b, "%s-%s#%d\n", + v.Start.UserKey, v.End, v.Start.SeqNum()) + } + fmt.Fprintf(&b, ".\n") + continue + default: + return fmt.Sprintf("unknown op: %s", parts[0]) } - fmt.Fprintf(&b, ".\n") - continue - default: - return fmt.Sprintf("unknown op: %s", parts[0]) - } - if iter.Valid() { - fmt.Fprintf(&b, "%s:%s\n", iter.Key(), iter.Value()) - if iter.Key().Kind() == InternalKeyKindRangeDelete { - iter.rangeDelFrag.Add(iter.cloneKey(iter.Key()), iter.Value()) + if iter.Valid() { + fmt.Fprintf(&b, "%s:%s\n", iter.Key(), iter.Value()) + if iter.Key().Kind() == InternalKeyKindRangeDelete { + iter.rangeDelFrag.Add(iter.cloneKey(iter.Key()), iter.Value()) + } + } else if err := iter.Error(); err != nil { + fmt.Fprintf(&b, "err=%v\n", err) + } else { + fmt.Fprintf(&b, ".\n") } - } else if err := iter.Error(); err != nil { - fmt.Fprintf(&b, "err=%v\n", err) - } else { - fmt.Fprintf(&b, ".\n") } + return b.String() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) } - return b.String() + }) + } - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) + // Rather than testing against all format version, we test against the + // significant boundaries. + formatVersions := []FormatMajorVersion{ + FormatMostCompatible, + FormatSetWithDelete - 1, + FormatSetWithDelete, + FormatNewest, + } + for _, formatVersion := range formatVersions { + t.Run(fmt.Sprintf("version-%s", formatVersion), func(t *testing.T) { + runTest(t, formatVersion) + }) + } } diff --git a/compaction_test.go b/compaction_test.go index f492bf7e1d..23049b20b0 100644 --- a/compaction_test.go +++ b/compaction_test.go @@ -1115,7 +1115,15 @@ func TestManualCompaction(t *testing.T) { } }() - reset := func() { + seed := time.Now().UnixNano() + rng := rand.New(rand.NewSource(seed)) + t.Logf("seed: %d", seed) + + randVersion := func(min, max FormatMajorVersion) FormatMajorVersion { + return FormatMajorVersion(int(min) + rng.Intn(int(max)-int(min)+1)) + } + + reset := func(minVersion, maxVersion FormatMajorVersion) { if d != nil { require.NoError(t, d.Close()) } @@ -1123,26 +1131,24 @@ func TestManualCompaction(t *testing.T) { require.NoError(t, mem.MkdirAll("ext", 0755)) opts := &Options{ - FS: mem, - DebugCheck: DebugCheckLevels, + FS: mem, + DebugCheck: DebugCheckLevels, + FormatMajorVersion: randVersion(minVersion, maxVersion), } opts.private.disableAutomaticCompactions = true - opts.testingRandomized() var err error d, err = Open("", opts) require.NoError(t, err) } - reset() - var ongoingCompaction *compaction - - paths := []string{"testdata/manual_compaction", "testdata/singledel_manual_compaction"} - for _, path := range paths { - datadriven.RunTest(t, path, func(td *datadriven.TestData) string { + runTest := func(t *testing.T, testData string, minVersion, maxVersion FormatMajorVersion) { + reset(minVersion, maxVersion) + var ongoingCompaction *compaction + datadriven.RunTest(t, testData, func(td *datadriven.TestData) string { switch td.Cmd { case "reset": - reset() + reset(minVersion, maxVersion) return "" case "batch": @@ -1174,8 +1180,9 @@ func TestManualCompaction(t *testing.T) { mem = vfs.NewMem() opts := &Options{ - FS: mem, - DebugCheck: DebugCheckLevels, + FS: mem, + DebugCheck: DebugCheckLevels, + FormatMajorVersion: randVersion(minVersion, maxVersion), } opts.private.disableAutomaticCompactions = true @@ -1301,6 +1308,39 @@ func TestManualCompaction(t *testing.T) { } }) } + + testCases := []struct { + testData string + minVersion FormatMajorVersion + maxVersion FormatMajorVersion // inclusive + }{ + { + testData: "testdata/manual_compaction", + minVersion: FormatMostCompatible, + maxVersion: FormatSetWithDelete - 1, + }, + { + testData: "testdata/manual_compaction_set_with_del", + minVersion: FormatSetWithDelete, + maxVersion: FormatNewest, + }, + { + testData: "testdata/singledel_manual_compaction", + minVersion: FormatMostCompatible, + maxVersion: FormatSetWithDelete - 1, + }, + { + testData: "testdata/singledel_manual_compaction_set_with_del", + minVersion: FormatSetWithDelete, + maxVersion: FormatNewest, + }, + } + + for _, tc := range testCases { + t.Run(tc.testData, func(t *testing.T) { + runTest(t, tc.testData, tc.minVersion, tc.maxVersion) + }) + } } func TestCompactionFindGrandparentLimit(t *testing.T) { diff --git a/error_test.go b/error_test.go index 165f5a83ca..0ba36f86c9 100644 --- a/error_test.go +++ b/error_test.go @@ -6,6 +6,7 @@ package pebble import ( "bytes" + "fmt" "math" "strings" "sync/atomic" @@ -164,13 +165,14 @@ func TestErrors(t *testing.T) { // cannot require operations fail since it involves flush/compaction, which retry // internally and succeed following an injected error. func TestRequireReadError(t *testing.T) { - run := func(index int32) (err error) { + run := func(formatVersion FormatMajorVersion, index int32) (err error) { // Perform setup with error injection disabled as it involves writes/background ops. inj := errorfs.OnIndex(-1) fs := errorfs.Wrap(vfs.NewMem(), inj) opts := &Options{ - FS: fs, - Logger: panicLogger{}, + FS: fs, + Logger: panicLogger{}, + FormatMajorVersion: formatVersion, } opts.private.disableTableStats = true d, err := Open("", opts) @@ -192,12 +194,21 @@ func TestRequireReadError(t *testing.T) { require.NoError(t, d.DeleteRange(key1, key2, nil)) require.NoError(t, d.Set(key1, value, nil)) require.NoError(t, d.Flush()) - expectLSM(` + if formatVersion < FormatSetWithDelete { + expectLSM(` 0.0: 000007:[a1#4,SET-a2#72057594037927935,RANGEDEL] 6: 000005:[a1#1,SET-a2#2,SET] `, d, t) + } else { + expectLSM(` +0.0: + 000007:[a1#4,SETWITHDEL-a2#72057594037927935,RANGEDEL] +6: + 000005:[a1#1,SET-a2#2,SET] +`, d, t) + } // Now perform foreground ops with error injection enabled. inj.SetIndex(index) @@ -235,12 +246,17 @@ func TestRequireReadError(t *testing.T) { return nil } - for i := int32(0); ; i++ { - err := run(i) - if err == nil { - t.Logf("no failures reported at index %d", i) - break - } + versions := []FormatMajorVersion{FormatMostCompatible, FormatSetWithDelete} + for _, version := range versions { + t.Run(fmt.Sprintf("version-%s", version), func(t *testing.T) { + for i := int32(0); ; i++ { + err := run(version, i) + if err == nil { + t.Logf("no failures reported at index %d", i) + break + } + } + }) } } @@ -248,15 +264,16 @@ func TestRequireReadError(t *testing.T) { // corruption and return an error. In this case the filesystem reads return // successful status but the data they return is corrupt. func TestCorruptReadError(t *testing.T) { - run := func(index int32) (err error) { + run := func(formatVersion FormatMajorVersion, index int32) (err error) { // Perform setup with corruption injection disabled as it involves writes/background ops. fs := &corruptFS{ FS: vfs.NewMem(), index: -1, } opts := &Options{ - FS: fs, - Logger: panicLogger{}, + FS: fs, + Logger: panicLogger{}, + FormatMajorVersion: formatVersion, } opts.private.disableTableStats = true d, err := Open("", opts) @@ -279,13 +296,23 @@ func TestCorruptReadError(t *testing.T) { require.NoError(t, d.DeleteRange(key1, key2, nil)) require.NoError(t, d.Set(key1, value, nil)) require.NoError(t, d.Flush()) - expectLSM(` + if formatVersion < FormatSetWithDelete { + expectLSM(` 0.0: 000007:[a1#4,SET-a2#72057594037927935,RANGEDEL] 6: 000005:[a1#1,SET-a2#2,SET] `, d, t) + } else { + expectLSM(` +0.0: + 000007:[a1#4,SETWITHDEL-a2#72057594037927935,RANGEDEL] +6: + 000005:[a1#1,SET-a2#2,SET] +`, d, t) + } + // Now perform foreground ops with corruption injection enabled. atomic.StoreInt32(&fs.index, index) iter := d.NewIter(nil) @@ -322,12 +349,17 @@ func TestCorruptReadError(t *testing.T) { } return nil } - for i := int32(0); ; i++ { - err := run(i) - if err == nil { - t.Logf("no failures reported at index %d", i) - break - } + versions := []FormatMajorVersion{FormatMostCompatible, FormatSetWithDelete} + for _, version := range versions { + t.Run(fmt.Sprintf("version-%s", version), func(t *testing.T) { + for i := int32(0); ; i++ { + err := run(version, i) + if err == nil { + t.Logf("no failures reported at index %d", i) + break + } + } + }) } } diff --git a/format_major_version.go b/format_major_version.go index 4eb67528ab..05ec13ca23 100644 --- a/format_major_version.go +++ b/format_major_version.go @@ -62,8 +62,12 @@ const ( // Pebble versions will be unable to open the database unless // they're aware of format versions. FormatVersioned + // FormatSetWithDelete is a format major version that introduces a new key + // kind, base.InternalKeyKindSetWithDelete. Previous Pebble versions will be + // unable to open this database. + FormatSetWithDelete // FormatNewest always contains the most recent format major version. - FormatNewest FormatMajorVersion = FormatVersioned + FormatNewest FormatMajorVersion = FormatSetWithDelete ) // formatMajorVersionMigrations defines the migrations from one format @@ -138,6 +142,11 @@ var formatMajorVersionMigrations = map[FormatMajorVersion]func(*DB) error{ } return d.finalizeFormatVersUpgrade(FormatVersioned) }, + // As SetWithDelete is a new key kind, there is nothing to migrate. We can + // simply finalize the format version and we're done. + FormatSetWithDelete: func(d *DB) error { + return d.finalizeFormatVersUpgrade(FormatSetWithDelete) + }, } const formatVersionMarkerName = `format-version` diff --git a/format_major_version_test.go b/format_major_version_test.go index b6776bd5ce..275bd60047 100644 --- a/format_major_version_test.go +++ b/format_major_version_test.go @@ -31,13 +31,15 @@ func TestRatchetFormat(t *testing.T) { require.Equal(t, FormatVersioned, d.FormatMajorVersion()) require.NoError(t, d.RatchetFormatMajorVersion(FormatVersioned)) require.Equal(t, FormatVersioned, d.FormatMajorVersion()) + require.NoError(t, d.RatchetFormatMajorVersion(FormatSetWithDelete)) + require.Equal(t, FormatSetWithDelete, d.FormatMajorVersion()) require.NoError(t, d.Close()) // If we Open the database again, leaving the default format, the - // database should Open using the persisted FormatVersioned format. + // database should Open using the persisted FormatNewest. d, err = Open("", &Options{FS: fs}) require.NoError(t, err) - require.Equal(t, FormatVersioned, d.FormatMajorVersion()) + require.Equal(t, FormatNewest, d.FormatMajorVersion()) require.NoError(t, d.Close()) // Move the marker to a version that does not exist. diff --git a/internal.go b/internal.go index 00e38b33b8..563785ee67 100644 --- a/internal.go +++ b/internal.go @@ -18,6 +18,7 @@ const ( InternalKeyKindSingleDelete = base.InternalKeyKindSingleDelete InternalKeyKindRangeDelete = base.InternalKeyKindRangeDelete InternalKeyKindMax = base.InternalKeyKindMax + InternalKeyKindSetWithDelete = base.InternalKeyKindSetWithDelete InternalKeyKindInvalid = base.InternalKeyKindInvalid InternalKeySeqNumBatch = base.InternalKeySeqNumBatch InternalKeySeqNumMax = base.InternalKeySeqNumMax diff --git a/internal/base/internal.go b/internal/base/internal.go index 3afd6f38b5..a85274c9f1 100644 --- a/internal/base/internal.go +++ b/internal/base/internal.go @@ -36,6 +36,11 @@ const ( // InternalKeyKindColumnFamilyBlobIndex = 16 // InternalKeyKindBlobIndex = 17 + // InternalKeyKindSetWithDelete keys are SET keys that have met with a + // DELETE key in a prior compaction. This key kind is specific to Pebble. + // See https://github.com/cockroachdb/pebble/issues/1255. + InternalKeyKindSetWithDelete InternalKeyKind = 18 + // This maximum value isn't part of the file format. It's unlikely, // but future extensions may increase this value. // @@ -45,7 +50,7 @@ const ( // which sorts 'less than or equal to' any other valid internalKeyKind, when // searching for any kind of internal key formed by a certain user key and // seqNum. - InternalKeyKindMax InternalKeyKind = 17 + InternalKeyKindMax InternalKeyKind = 18 // A marker for an invalid key. InternalKeyKindInvalid InternalKeyKind = 255 @@ -66,14 +71,14 @@ const ( ) var internalKeyKindNames = []string{ - InternalKeyKindDelete: "DEL", - InternalKeyKindSet: "SET", - InternalKeyKindMerge: "MERGE", - InternalKeyKindLogData: "LOGDATA", - InternalKeyKindSingleDelete: "SINGLEDEL", - InternalKeyKindRangeDelete: "RANGEDEL", - InternalKeyKindMax: "MAX", - InternalKeyKindInvalid: "INVALID", + InternalKeyKindDelete: "DEL", + InternalKeyKindSet: "SET", + InternalKeyKindMerge: "MERGE", + InternalKeyKindLogData: "LOGDATA", + InternalKeyKindSingleDelete: "SINGLEDEL", + InternalKeyKindRangeDelete: "RANGEDEL", + InternalKeyKindSetWithDelete: "SETWITHDEL", + InternalKeyKindInvalid: "INVALID", } func (k InternalKeyKind) String() string { @@ -130,13 +135,14 @@ func MakeRangeDeleteSentinelKey(userKey []byte) InternalKey { } var kindsMap = map[string]InternalKeyKind{ - "DEL": InternalKeyKindDelete, - "SINGLEDEL": InternalKeyKindSingleDelete, - "RANGEDEL": InternalKeyKindRangeDelete, - "SET": InternalKeyKindSet, - "MERGE": InternalKeyKindMerge, - "INVALID": InternalKeyKindInvalid, - "MAX": InternalKeyKindMax, + "DEL": InternalKeyKindDelete, + "SINGLEDEL": InternalKeyKindSingleDelete, + "RANGEDEL": InternalKeyKindRangeDelete, + "SET": InternalKeyKindSet, + "MERGE": InternalKeyKindMerge, + "INVALID": InternalKeyKindInvalid, + "MAX": InternalKeyKindMax, + "SETWITHDEL": InternalKeyKindSetWithDelete, } // ParseInternalKey parses the string representation of an internal key. The diff --git a/internal/base/internal_test.go b/internal/base/internal_test.go index b058d2ac30..0156c151d0 100644 --- a/internal/base/internal_test.go +++ b/internal/base/internal_test.go @@ -37,7 +37,7 @@ func TestInvalidInternalKey(t *testing.T) { "\x01\x02\x03\x04\x05\x06\x07", "foo", "foo\x08\x07\x06\x05\x04\x03\x02", - "foo\x12\x07\x06\x05\x04\x03\x02\x01", + "foo\x14\x07\x06\x05\x04\x03\x02\x01", } for _, tc := range testCases { k := DecodeInternalKey([]byte(tc)) diff --git a/iterator.go b/iterator.go index 6bfa85eb9c..4d7ebfb9b8 100644 --- a/iterator.go +++ b/iterator.go @@ -245,7 +245,7 @@ func (i *Iterator) findNextEntry(limit []byte) { i.nextUserKey() continue - case InternalKeyKindSet: + case InternalKeyKindSet, InternalKeyKindSetWithDelete: i.keyBuf = append(i.keyBuf[:0], key.UserKey...) i.key = i.keyBuf i.value = i.iterValue @@ -459,7 +459,7 @@ func (i *Iterator) findPrevEntry(limit []byte) { } continue - case InternalKeyKindSet: + case InternalKeyKindSet, InternalKeyKindSetWithDelete: i.keyBuf = append(i.keyBuf[:0], key.UserKey...) i.key = i.keyBuf // iterValue is owned by i.iter and could change after the Prev() @@ -570,7 +570,7 @@ func (i *Iterator) mergeNext(key InternalKey, valueMerger ValueMerger) { // point. return - case InternalKeyKindSet: + case InternalKeyKindSet, InternalKeyKindSetWithDelete: // We've hit a Set value. Merge with the existing value and return. i.err = valueMerger.MergeOlder(i.iterValue) return diff --git a/iterator_test.go b/iterator_test.go index fa253c7d60..62e1c00427 100644 --- a/iterator_test.go +++ b/iterator_test.go @@ -218,7 +218,7 @@ type invalidatingIter struct { iter internalIterator lastKey *InternalKey lastValue []byte - ignoreKinds [base.InternalKeyKindMax]bool + ignoreKinds [base.InternalKeyKindMax + 1]bool } func newInvalidatingIter(iter internalIterator) *invalidatingIter { diff --git a/level_checker.go b/level_checker.go index d779546e5e..dcf6010995 100644 --- a/level_checker.go +++ b/level_checker.go @@ -167,7 +167,7 @@ func (m *simpleMergingIter) step() bool { m.err = closer.Close() } m.valueMerger = nil - case InternalKeyKindSet: + case InternalKeyKindSet, InternalKeyKindSetWithDelete: m.err = m.valueMerger.MergeOlder(item.value) if m.err == nil { var closer io.Closer diff --git a/open_test.go b/open_test.go index 58385bae3a..385754c861 100644 --- a/open_test.go +++ b/open_test.go @@ -99,7 +99,7 @@ func TestNewDBFilenames(t *testing.T) { "LOCK", "MANIFEST-000001", "OPTIONS-000003", - "marker.format-version.000002.003", + "marker.format-version.000003.004", "marker.manifest.000001.MANIFEST-000001", }, } diff --git a/range_del_test.go b/range_del_test.go index dd406e9ffa..db1866c3b3 100644 --- a/range_del_test.go +++ b/range_del_test.go @@ -156,73 +156,75 @@ func TestDeleteRangeFlushDelay(t *testing.T) { // problem is that range tombstones are not truncated to sstable boundaries on // disk, only in memory. func TestRangeDelCompactionTruncation(t *testing.T) { - // Use a small target file size so that there is a single key per sstable. - d, err := Open("", &Options{ - FS: vfs.NewMem(), - Levels: []LevelOptions{ - {TargetFileSize: 100}, - {TargetFileSize: 100}, - {TargetFileSize: 1}, - }, - DebugCheck: DebugCheckLevels, - }) - require.NoError(t, err) - defer d.Close() - - d.mu.Lock() - d.mu.versions.dynamicBaseLevel = false - d.mu.Unlock() + runTest := func(formatVersion FormatMajorVersion) { + // Use a small target file size so that there is a single key per sstable. + d, err := Open("", &Options{ + FS: vfs.NewMem(), + Levels: []LevelOptions{ + {TargetFileSize: 100}, + {TargetFileSize: 100}, + {TargetFileSize: 1}, + }, + DebugCheck: DebugCheckLevels, + FormatMajorVersion: formatVersion, + }) + require.NoError(t, err) + defer d.Close() - lsm := func() string { d.mu.Lock() - s := d.mu.versions.currentVersion().DebugString(base.DefaultFormatter) + d.mu.versions.dynamicBaseLevel = false d.mu.Unlock() - return s - } - expectLSM := func(expected string) { - t.Helper() - expected = strings.TrimSpace(expected) - actual := strings.TrimSpace(lsm()) - if expected != actual { - t.Fatalf("expected\n%s\nbut found\n%s", expected, actual) - } - } - - require.NoError(t, d.Set([]byte("a"), bytes.Repeat([]byte("b"), 100), nil)) - snap1 := d.NewSnapshot() - defer snap1.Close() - // Flush so that each version of "a" ends up in its own L0 table. If we - // allowed both versions in the same L0 table, compaction could trivially - // move the single L0 table to L1. - require.NoError(t, d.Flush()) - require.NoError(t, d.Set([]byte("b"), bytes.Repeat([]byte("c"), 100), nil)) - snap2 := d.NewSnapshot() - defer snap2.Close() - require.NoError(t, d.DeleteRange([]byte("a"), []byte("d"), nil)) + lsm := func() string { + d.mu.Lock() + s := d.mu.versions.currentVersion().DebugString(base.DefaultFormatter) + d.mu.Unlock() + return s + } + expectLSM := func(expected string) { + t.Helper() + expected = strings.TrimSpace(expected) + actual := strings.TrimSpace(lsm()) + if expected != actual { + t.Fatalf("expected\n%s\nbut found\n%s", expected, actual) + } + } - // Compact to produce the L1 tables. - require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) - expectLSM(` + require.NoError(t, d.Set([]byte("a"), bytes.Repeat([]byte("b"), 100), nil)) + snap1 := d.NewSnapshot() + defer snap1.Close() + // Flush so that each version of "a" ends up in its own L0 table. If we + // allowed both versions in the same L0 table, compaction could trivially + // move the single L0 table to L1. + require.NoError(t, d.Flush()) + require.NoError(t, d.Set([]byte("b"), bytes.Repeat([]byte("c"), 100), nil)) + + snap2 := d.NewSnapshot() + defer snap2.Close() + require.NoError(t, d.DeleteRange([]byte("a"), []byte("d"), nil)) + + // Compact to produce the L1 tables. + require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) + expectLSM(` 1: 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] `) - // Compact again to move one of the tables to L2. - require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) - expectLSM(` + // Compact again to move one of the tables to L2. + require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) + expectLSM(` 1: 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] 2: 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] `) - // Write "b" and "c" to a new table. - require.NoError(t, d.Set([]byte("b"), []byte("d"), nil)) - require.NoError(t, d.Set([]byte("c"), []byte("e"), nil)) - require.NoError(t, d.Flush()) - expectLSM(` + // Write "b" and "c" to a new table. + require.NoError(t, d.Set([]byte("b"), []byte("d"), nil)) + require.NoError(t, d.Set([]byte("c"), []byte("e"), nil)) + require.NoError(t, d.Flush()) + expectLSM(` 0.0: 000011:[b#4,SET-c#5,SET] 1: @@ -231,36 +233,37 @@ func TestRangeDelCompactionTruncation(t *testing.T) { 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] `) - // "b" is still visible at this point as it should be. - if _, closer, err := d.Get([]byte("b")); err != nil { - t.Fatalf("expected success, but found %v", err) - } else { - closer.Close() - } + // "b" is still visible at this point as it should be. + if _, closer, err := d.Get([]byte("b")); err != nil { + t.Fatalf("expected success, but found %v", err) + } else { + closer.Close() + } - keys := func() string { - iter := d.NewIter(nil) - defer iter.Close() - var buf bytes.Buffer - var sep string - for iter.First(); iter.Valid(); iter.Next() { - fmt.Fprintf(&buf, "%s%s", sep, iter.Key()) - sep = " " + keys := func() string { + iter := d.NewIter(nil) + defer iter.Close() + var buf bytes.Buffer + var sep string + for iter.First(); iter.Valid(); iter.Next() { + fmt.Fprintf(&buf, "%s%s", sep, iter.Key()) + sep = " " + } + return buf.String() } - return buf.String() - } - if expected, actual := `b c`, keys(); expected != actual { - t.Fatalf("expected %q, but found %q", expected, actual) - } + if expected, actual := `b c`, keys(); expected != actual { + t.Fatalf("expected %q, but found %q", expected, actual) + } - // Compact the L0 table. This will compact the L0 table into L1 and do to the - // sstable target size settings will create 2 tables in L1. Then L1 table - // containing "c" will be compacted again with the L2 table creating two - // tables in L2. Lastly, the L2 table containing "c" will be compacted - // creating the L3 table. - require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) - expectLSM(` + // Compact the L0 table. This will compact the L0 table into L1 and do to the + // sstable target size settings will create 2 tables in L1. Then L1 table + // containing "c" will be compacted again with the L2 table creating two + // tables in L2. Lastly, the L2 table containing "c" will be compacted + // creating the L3 table. + require.NoError(t, d.Compact([]byte("c"), []byte("c\x00"))) + if formatVersion < FormatSetWithDelete { + expectLSM(` 1: 000012:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] 3: @@ -268,17 +271,40 @@ func TestRangeDelCompactionTruncation(t *testing.T) { 000018:[b#3,RANGEDEL-c#72057594037927935,RANGEDEL] 000019:[c#5,SET-d#72057594037927935,RANGEDEL] `) + } else { + expectLSM(` +1: + 000012:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] +3: + 000017:[b#4,SETWITHDEL-b#4,SETWITHDEL] + 000018:[b#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000019:[c#5,SETWITHDEL-d#72057594037927935,RANGEDEL] +`) + } - // The L1 table still contains a tombstone from [a,d) which will improperly - // delete the newer version of "b" in L2. - if _, closer, err := d.Get([]byte("b")); err != nil { - t.Errorf("expected success, but found %v", err) - } else { - closer.Close() + // The L1 table still contains a tombstone from [a,d) which will improperly + // delete the newer version of "b" in L2. + if _, closer, err := d.Get([]byte("b")); err != nil { + t.Errorf("expected success, but found %v", err) + } else { + closer.Close() + } + + if expected, actual := `b c`, keys(); expected != actual { + t.Errorf("expected %q, but found %q", expected, actual) + } } - if expected, actual := `b c`, keys(); expected != actual { - t.Errorf("expected %q, but found %q", expected, actual) + versions := []FormatMajorVersion{ + FormatMostCompatible, + FormatSetWithDelete - 1, + FormatSetWithDelete, + FormatNewest, + } + for _, version := range versions { + t.Run(fmt.Sprintf("version-%s", version), func(t *testing.T) { + runTest(version) + }) } } diff --git a/testdata/checkpoint b/testdata/checkpoint index f0bcb14f50..3ae9338270 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -25,6 +25,9 @@ rename: db/CURRENT.000000.dbtmp -> db/CURRENT create: db/marker.format-version.000002.003 close: db/marker.format-version.000002.003 sync: db +create: db/marker.format-version.000003.004 +close: db/marker.format-version.000003.004 +sync: db sync: db/MANIFEST-000001 create: db/000002.log sync: db @@ -89,9 +92,9 @@ close: open-dir: checkpoints/checkpoint1 link: db/OPTIONS-000003 -> checkpoints/checkpoint1/OPTIONS-000003 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.003 -sync: checkpoints/checkpoint1/marker.format-version.000001.003 -close: checkpoints/checkpoint1/marker.format-version.000001.003 +create: checkpoints/checkpoint1/marker.format-version.000001.004 +sync: checkpoints/checkpoint1/marker.format-version.000001.004 +close: checkpoints/checkpoint1/marker.format-version.000001.004 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 create: checkpoints/checkpoint1/MANIFEST-000001 @@ -146,7 +149,7 @@ CURRENT LOCK MANIFEST-000001 OPTIONS-000003 -marker.format-version.000002.003 +marker.format-version.000003.004 marker.manifest.000001.MANIFEST-000001 list checkpoints/checkpoint1 @@ -156,7 +159,7 @@ list checkpoints/checkpoint1 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.003 +marker.format-version.000001.004 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint1 readonly diff --git a/testdata/compaction_iter_set_with_del b/testdata/compaction_iter_set_with_del new file mode 100644 index 0000000000..5235fe308a --- /dev/null +++ b/testdata/compaction_iter_set_with_del @@ -0,0 +1,1419 @@ +define +a.SET.1:b +---- + +iter +first +next +---- +a#1,1:b +. + +define +a.SET.2:c +a.SET.1:b +---- + +iter +first +next +---- +a#2,1:c +. + +iter snapshots=0 +first +next +---- +a#2,1:c +. + +iter snapshots=1 +first +next +---- +a#2,1:c +. + +iter snapshots=2 +first +next +next +---- +a#2,1:c +a#1,1:b +. + +define +a.DEL.2: +a.SET.1:b +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +iter elide-tombstones=true snapshots=2 +first +next +next +---- +a#2,0: +a#1,1:b +. + +iter elide-tombstones=true snapshots=1 +first +next +---- +a#2,0: +. + +define +a.DEL.2: +a.SET.1:b +b.SET.3:c +---- + +iter +first +next +next +---- +a#2,0: +b#3,1:c +. + +iter snapshots=1 +first +next +next +---- +a#2,0: +b#3,1:c +. + +iter snapshots=2 +first +next +next +next +---- +a#2,0: +a#1,1:b +b#3,1:c +. + +define +a.SET.1:a +b.SET.2:b +c.SET.3:c +---- + +iter +first +next +next +next +---- +a#1,1:a +b#2,1:b +c#3,1:c +. + +define +a.MERGE.3:d +a.MERGE.2:c +a.SET.1:b +b.MERGE.2:b +b.MERGE.1:a +---- + +iter +first +next +next +---- +a#3,1:bcd[base] +b#2,2:ab +. + +iter snapshots=3 +first +next +next +next +---- +a#3,2:d +a#2,1:bc[base] +b#2,2:ab +. + +define +a.SET.9:b +a.DEL.8: +a.SET.7:d +a.DEL.6: +a.SET.5:f +---- + +iter +first +next +---- +a#9,18:b +. + +iter snapshots=6 +first +next +next +---- +a#9,18:b +a#5,1:f +. + +iter snapshots=7 +first +next +next +---- +a#9,18:b +a#6,0: +. + +iter snapshots=8 +first +next +next +---- +a#9,18:b +a#7,18:d +. + +iter snapshots=9 +first +next +next +---- +a#9,1:b +a#8,0: +. + +iter snapshots=10 +first +next +---- +a#9,18:b +. + +iter snapshots=(5,6,7,8,9) +first +next +next +next +next +next +---- +a#9,1:b +a#8,0: +a#7,1:d +a#6,0: +a#5,1:f +. + +define +a.INVALID.2:b +a.SET.1:c +---- + +iter +first +---- +err=invalid internal key kind: 255 + +define +a.SET.2:b +a.INVALID.1:c +---- + +iter +first +next +---- +a#2,18:b +err=invalid internal key kind: 255 + +define +a.MERGE.2:b +a.INVALID.1:c +---- + +iter +first +next +---- +a#2,2:b +err=invalid internal key kind: 255 + +define +a.INVALID.2:c +a.RANGEDEL.1:d +---- + +iter +first +tombstones +---- +err=invalid internal key kind: 255 +. + +define +a.MERGE.2:b +a.MERGE.1:c +a.MERGE.0:d +---- + +iter snapshots=(1,2) +first +next +next +next +---- +a#2,2:b +a#1,2:c +a#0,2:d +. + +define +a.SET.2:b +a.RANGEDEL.1:c +b.RANGEDEL.4:d +b.SET.2:e +c.SET.3:f +---- + +iter +first +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +. +a-b#1 +b-c#4 +c-d#4 +. + +iter snapshots=2 +first +next +next +next +tombstones +---- +a#2,1:b +a#1,15:c +b#4,15:d +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +iter snapshots=3 +first +next +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +b#2,1:e +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +iter snapshots=4 +first +next +next +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +b#2,1:e +c#3,1:f +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +define +a.RANGEDEL.3:e +b.SET.4:b +c.SET.3:c +d.SET.2:d +e.SET.1:e +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:e +b#4,1:b +c#3,1:c +e#1,1:e +. +a-e#3 +. + +define +a.RANGEDEL.3:e +b.MERGE.4:b +c.MERGE.3:c +d.MERGE.2:d +e.MERGE.1:e +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:e +b#4,2:b +c#3,2:c +e#1,2:e +. +a-e#3 +. + +define +a.RANGEDEL.3:c +b.MERGE.5:e +b.MERGE.4:d +b.MERGE.2:c +b.MERGE.1:b +d.MERGE.5:c +d.MERGE.4:b +d.RANGEDEL.3:f +d.MERGE.2:e +d.MERGE.1:d +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:c +b#5,1:de[base] +d#5,2:bc +d#3,15:f +. +a-c#3 +d-f#3 +. + +define +a.RANGEDEL.3:d +b.RANGEDEL.2:e +c.RANGEDEL.1:f +---- + +iter +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +c-d#3 +d-e#2 +e-f#1 +. + +iter snapshots=2 +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +c-d#3 +c-d#1 +d-e#2 +d-e#1 +e-f#1 +. + +iter snapshots=3 +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +b-c#2 +c-d#3 +c-d#2 +d-e#2 +e-f#1 +. + +iter snapshots=(2,3) +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +b-c#2 +c-d#3 +c-d#2 +c-d#1 +d-e#2 +d-e#1 +e-f#1 +. + +define +a.RANGEDEL.10:k +f.SET.9:f +f.SET.8:f +---- + +iter snapshots=(9,10) +first +next +tombstones f +next +tombstones +---- +a#10,15:k +f#9,1:f +a-k#10 +. +f#8,1:f +f-k#10 +. + +define +f.RANGEDEL.10:k +f.SET.9:f +f.SET.8:f +---- + +iter snapshots=(9,10) +first +next +tombstones f +next +tombstones +---- +f#10,15:k +f#9,1:f +f-k#10 +. +f#8,1:f +f-k#10 +. + +define +a.SET.1:a +b.RANGEDEL.2:d +c.RANGEDEL.3:e +d.SET.4:d +---- + +iter +first +next +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#3,15:e +d#4,1:d +b-c#2 +c-d#3 +. +c-d#3 +d-e#3 +. + +iter snapshots=3 +first +next +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#3,15:e +d#4,1:d +b-c#2 +c-d#3 +c-d#2 +. +c-d#3 +c-d#2 +d-e#3 +. + +define +a.SET.1:a +b.RANGEDEL.2:d +c.SET.4:d +---- + +iter +first +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#4,1:d +b-d#2 +. +c-d#2 +. + +define +a.RANGEDEL.2:d +a.SET.2:a +b.SET.2:b +c.SET.2:c +---- + +iter +first +next +next +next +next +---- +a#2,15:d +a#2,1:a +b#2,1:b +c#2,1:c +. + +define +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#1,7: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#2,7: +. + +define +a.SINGLEDEL.3: +a.SINGLEDEL.2: +a.SET.1:a +---- + +iter +first +---- +. + +define +a.SET.3:a +b.SINGLEDEL.2: +b.DEL.1: +---- + +iter +first +next +next +---- +a#3,1:a +b#2,0: +. + +define +a.SINGLEDEL.2: +a.DEL.1: +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.MERGE.1: +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.SET.1:b +---- + +iter +first +---- +. + +define +a.SET.2:b +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#2,1:b +. + +define +a.MERGE.6:b +a.SINGLEDEL.5: +a.SET.4:a +---- + +iter +first +next +---- +a#6,1:b[base] +. + +# Non-deterministic use of SINGLEDEL where there are two older SETs that have +# not been deleted or single deleted. It is permitted to shadow both. +define +a.MERGE.6:b +a.SINGLEDEL.5: +a.SET.4:a +a.SET.3:a +---- + +iter +first +next +---- +a#6,1:b[base] +. + +define +a.SINGLEDEL.2: +a.SET.1:b +b.SET.3:c +---- + +iter +first +next +---- +b#3,1:c +. + +define +a.SINGLEDEL.3: +a.SET.2:b +a.SET.1:a +---- + +iter +first +next +---- +a#1,1:a +. + +define +a.SINGLEDEL.3: +a.MERGE.2:b +a.MERGE.1:a +---- + +iter +first +next +---- +a#3,0: +. + +define +a.SINGLEDEL.4: +a.SET.3:val +a.SINGLEDEL.2: +a.SET.1:val +---- + +iter +first +---- +. + +iter snapshots=2 +first +next +next +---- +a#2,7: +a#1,1:val +. + +define +a.SINGLEDEL.4: +a.SET.3:val +a.DEL.2: +a.SET.1:val +---- + +iter +first +next +---- +a#2,0: +. + +iter snapshots=2 +first +next +next +---- +a#2,0: +a#1,1:val +. + +iter snapshots=3 +first +next +---- +a#2,0: +. + +iter snapshots=(2,3) +first +next +next +---- +a#2,0: +a#1,1:val +. + +define +a.SINGLEDEL.4: +a.SET.3:c +a.MERGE.2:b +a.SET.1:a +---- + +iter +first +next +---- +a#2,1:ab[base] +. + +iter snapshots=2 +first +next +next +---- +a#2,2:b +a#1,1:a +. + +iter snapshots=3 +first +next +---- +a#2,1:ab[base] +. + +iter snapshots=(2,3,4) +first +next +next +next +next +---- +a#4,7: +a#3,1:c +a#2,2:b +a#1,1:a +. + +define +a.SINGLEDEL.3: +a.RANGEDEL.2:c +a.SET.1:val +---- + +iter +first +next +next +tombstones +---- +a#3,7: +a#2,15:c +. +a-c#2 +. + +define +a.RANGEDEL.3:d +a.DEL.2: +a.SET.1:a +d.DEL.2: +---- + +iter +first +next +next +tombstones +---- +a#3,15:d +d#2,0: +. +a-d#3 +. + +iter snapshots=3 +first +next +next +next +---- +a#3,15:d +a#2,0: +d#2,0: +. + +iter snapshots=2 +first +next +next +next +---- +a#3,15:d +a#1,1:a +d#2,0: +. + +iter snapshots=1 +first +next +next +---- +a#3,15:d +d#2,0: +. + +define +a.MERGE.2:a +b.RANGEDEL.1:c +---- + +iter +first +tombstones a +next +next +tombstones +---- +a#2,2:a +. +b#1,15:c +. +b-c#1 +. + +define +a.MERGE.2:v2 +a.RANGEDEL.1:b +a.MERGE.1:v1 +---- + +iter allow-zero-seqnum=true +first +next +next +next +tombstones +---- +a#2,2:v2 +a#1,15:b +a#0,2:v1 +. +a-b#1 +. + +# Verify that we transform merge+del -> set. + +define +a.MERGE.5:5 +a.DEL.3: +a.MERGE.1:1 +---- + +iter +first +next +---- +a#5,1:5[base] +. + +iter allow-zero-seqnum=true +first +next +---- +a#0,1:5[base] +. + +iter elide-tombstones=true +first +next +---- +a#5,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#5,1:5[base] +a#1,2:1 +. + +iter snapshots=2 elide-tombstones=true +first +next +next +---- +a#5,1:5[base] +a#1,2:1 +. + +# Verify that we transform merge+rangedel -> set. This isn't strictly +# necessary, but provides consistency with the behavior for merge+del. + +define +a.RANGEDEL.3:c +b.MERGE.5:5 +b.SET.2:2 +b.MERGE.1:1 +---- + +iter +first +next +next +---- +a#3,15:c +b#5,1:5[base] +. + +iter allow-zero-seqnum=true +first +next +next +---- +a#3,15:c +b#0,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#3,15:c +b#5,1:5[base] +b#1,2:1 + +define +a.RANGEDEL.3:c +b.MERGE.5:5 +b.MERGE.2:2 +b.MERGE.1:1 +---- + +iter +first +next +next +---- +a#3,15:c +b#5,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#3,15:c +b#5,1:5[base] +b#1,2:1 + +# SET that meets a DEL is transformed into a SETWITHDEL. + +define +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#2,1:b +a#1,0: +. + +define +a.SET.3:c +a.DEL.2: +a.SET.1:b +---- + +iter +first +next +---- +a#3,18:c +. + +iter snapshots=2 +first +next +next +---- +a#3,18:c +a#1,1:b +. + +define +a.SET.3:c +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#3,18:c +. + +iter snapshots=3 +first +next +next +---- +a#3,1:c +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#3,1:c +a#1,0: +. + +define +a.DEL.3: +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#3,0: +. + +iter snapshots=3 +first +next +next +---- +a#3,0: +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#3,0: +a#1,0: +. + +# SETWITHDEL-eligible entries at or under a RANGEDEL at the same user key should +# be skipped. +define +a.SET.3:c +a.RANGEDEL.2:z +a.SET.2:b +a.DEL.1: +---- + +iter allow-zero-seqnum=true +first +next +next +---- +a#0,18:c +a#2,15:z +. + +iter allow-zero-seqnum=true snapshots=3 +first +next +next +next +---- +a#3,1:c +a#2,15:z +a#0,18:b +. + +iter allow-zero-seqnum=true snapshots=2 +first +next +next +next +---- +a#3,18:c +a#2,15:z +a#1,0: +. + +define +a.SET.4:c +a.RANGEDEL.3:z +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +next +---- +a#4,18:c +a#3,15:z +. + +# Invalid keys are emitted under SETWITHDEL. + +define +a.SET.2:b +a.INVALID.1: +---- + +iter +first +next +---- +a#2,18:b +err=invalid internal key kind: 255 + +define +a.SET.3:c +a.INVALID.2: +a.SET.1:b +---- + +iter +first +next +---- +a#3,18:c +err=invalid internal key kind: 255 + +# SINGLEDEL that meets a SETWITHDEL is transformed into a DEL. + +define +a.SINGLEDEL.3: +a.SETWITHDEL.2:d +b.SET.1:c +---- + +iter +first +next +next +---- +a#3,0: +b#1,1:c +. + +iter snapshots=2 +first +next +next +---- +a#3,0: +b#1,1:c +. + +iter snapshots=3 +first +next +next +next +---- +a#3,7: +a#2,18:d +b#1,1:c +. + +define +a.SETWITHDEL.3:3 +a.SET.2:d +b.SET.1:c +---- + +iter +first +next +next +---- +a#3,18:3 +b#1,1:c +. + +iter snapshots=3 +first +next +next +next +---- +a#3,18:3 +a#2,1:d +b#1,1:c +. diff --git a/testdata/event_listener b/testdata/event_listener index d0fa1da352..4e4b053929 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -30,6 +30,10 @@ create: db/marker.format-version.000002.003 close: db/marker.format-version.000002.003 sync: db upgraded to format version: 003 +create: db/marker.format-version.000003.004 +close: db/marker.format-version.000003.004 +sync: db +upgraded to format version: 004 create: db/MANIFEST-000003 close: db/MANIFEST-000001 sync: db/MANIFEST-000003 @@ -195,9 +199,9 @@ close: open-dir: checkpoint link: db/OPTIONS-000004 -> checkpoint/OPTIONS-000004 open-dir: checkpoint -create: checkpoint/marker.format-version.000001.003 -sync: checkpoint/marker.format-version.000001.003 -close: checkpoint/marker.format-version.000001.003 +create: checkpoint/marker.format-version.000001.004 +sync: checkpoint/marker.format-version.000001.004 +close: checkpoint/marker.format-version.000001.004 sync: checkpoint close: checkpoint create: checkpoint/MANIFEST-000017 diff --git a/testdata/manual_compaction_set_with_del b/testdata/manual_compaction_set_with_del new file mode 100644 index 0000000000..0e75649cc3 --- /dev/null +++ b/testdata/manual_compaction_set_with_del @@ -0,0 +1,867 @@ +batch +set a 1 +set b 2 +---- + +compact a-b +---- +6: + 000005:[a#1,SET-b#2,SET] + +batch +set c 3 +set d 4 +---- + +compact c-d +---- +6: + 000005:[a#1,SET-b#2,SET] + 000007:[c#3,SET-d#4,SET] + +batch +set b 5 +set c 6 +---- + +compact a-d +---- +6: + 000010:[a#0,SET-d#0,SET] + +# This also tests flushing a memtable that only contains range +# deletions. + +batch +del-range a e +---- + +compact a-d +---- + +# Test that a multi-output-file compaction generates non-overlapping files. + +define target-file-sizes=(100, 1) +L0 + b.SET.1:v +L0 + a.SET.2:v +---- +0.0: + 000005:[a-a] + 000004:[b-b] + +compact a-b +---- +1: + 000006:[a#0,SET-a#0,SET] + 000007:[b#0,SET-b#0,SET] + +# A range tombstone extends past the grandparent file boundary used to limit the +# size of future compactions. Verify the range tombstone is split at that file +# boundary. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.SET.3:v +L2 + a.RANGEDEL.2:e +L3 + a.SET.0:v + b.SET.0:v +L3 + c.SET.0:v + d.SET.0:v +---- +1: + 000004:[a-a] +2: + 000005:[a-e] +3: + 000006:[a-b] + 000007:[c-d] + +wait-pending-table-stats +000005 +---- +num-entries: 1 +num-deletions: 1 +point-deletions-bytes-estimate: 0 +range-deletions-bytes-estimate: 1552 + +compact a-e L1 +---- +2: + 000008:[a#3,SETWITHDEL-b#72057594037927935,RANGEDEL] + 000009:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] + 000010:[d#2,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000006:[a#0,SET-b#0,SET] + 000007:[c#0,SET-d#0,SET] + +wait-pending-table-stats +000008 +---- +num-entries: 2 +num-deletions: 1 +point-deletions-bytes-estimate: 0 +range-deletions-bytes-estimate: 776 + +# Same as above, except range tombstone covers multiple grandparent file boundaries. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.SET.3:v +L2 + a.RANGEDEL.2:g +L3 + a.SET.0:v + b.SET.0:v +L3 + c.SET.0:v + d.SET.0:v +L3 + e.SET.0:v + f.SET.1:v +L3 + f.SET.0:v + g.SET.0:v +---- +1: + 000004:[a-a] +2: + 000005:[a-g] +3: + 000006:[a-b] + 000007:[c-d] + 000008:[e-f] + 000009:[f-g] + +compact a-e L1 +---- +2: + 000010:[a#3,SETWITHDEL-b#72057594037927935,RANGEDEL] + 000011:[b#2,RANGEDEL-d#72057594037927935,RANGEDEL] + 000012:[d#2,RANGEDEL-f#72057594037927935,RANGEDEL] + 000013:[f#2,RANGEDEL-g#72057594037927935,RANGEDEL] +3: + 000006:[a#0,SET-b#0,SET] + 000007:[c#0,SET-d#0,SET] + 000008:[e#0,SET-f#1,SET] + 000009:[f#0,SET-g#0,SET] + +# A range tombstone covers multiple grandparent file boundaries between point keys, +# rather than after all point keys. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.SET.3:v + h.SET.3:v +L2 + a.RANGEDEL.2:g +L3 + a.SET.0:v + b.SET.0:v +L3 + c.SET.0:v + d.SET.0:v +L3 + e.SET.0:v + f.SET.1:v +---- +1: + 000004:[a-h] +2: + 000005:[a-g] +3: + 000006:[a-b] + 000007:[c-d] + 000008:[e-f] + +compact a-e L1 +---- +2: + 000009:[a#3,SETWITHDEL-g#72057594037927935,RANGEDEL] + 000010:[h#3,SET-h#3,SET] +3: + 000006:[a#0,SET-b#0,SET] + 000007:[c#0,SET-d#0,SET] + 000008:[e#0,SET-f#1,SET] + +# A range tombstone is the first and only item output by a compaction, and it +# extends past the grandparent file boundary used to limit the size of future +# compactions. Verify the range tombstone is split at that file boundary. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.RANGEDEL.3:e +L2 + a.SET.2:v +L3 + a.SET.0:v + b.SET.0:v +L3 + c.SET.0:v + d.SET.0:v +---- +1: + 000004:[a-e] +2: + 000005:[a-a] +3: + 000006:[a-b] + 000007:[c-d] + +compact a-e L1 +---- +2: + 000008:[a#3,RANGEDEL-b#72057594037927935,RANGEDEL] + 000009:[b#3,RANGEDEL-d#72057594037927935,RANGEDEL] + 000010:[d#3,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000006:[a#0,SET-b#0,SET] + 000007:[c#0,SET-d#0,SET] + +# An elided range tombstone is the first item encountered by a compaction, +# and the grandparent limit set by it extends to the next item, also a range +# tombstone. The first item should be elided, and the second item should +# reset the grandparent limit. + +define target-file-sizes=(100, 100, 100, 100) +L1 + a.RANGEDEL.4:d +L1 + grandparent.RANGEDEL.2:z + h.SET.3:v +L2 + grandparent.SET.1:v +L3 + grandparent.SET.0:v +L3 + m.SET.0:v +---- +1: + 000004:[a-d] + 000005:[grandparent-z] +2: + 000006:[grandparent-grandparent] +3: + 000007:[grandparent-grandparent] + 000008:[m-m] + +compact a-h L1 +---- +2: + 000009:[grandparent#2,RANGEDEL-m#72057594037927935,RANGEDEL] + 000010:[m#2,RANGEDEL-z#72057594037927935,RANGEDEL] +3: + 000007:[grandparent#0,SET-grandparent#0,SET] + 000008:[m#0,SET-m#0,SET] + +# Setup such that grandparent overlap limit is exceeded multiple times at the same user key ("b"). +# Ensures the compaction output files are non-overlapping. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.SET.2:v + c.SET.2:v +L2 + a.RANGEDEL.3:c +L3 + b.SET.2:v +L3 + b.SET.1:v +L3 + b.SET.0:v +---- +1: + 000004:[a-c] +2: + 000005:[a-c] +3: + 000006:[b-b] + 000007:[b-b] + 000008:[b-b] + +compact a-c L1 +---- +2: + 000009:[a#3,RANGEDEL-c#72057594037927935,RANGEDEL] + 000010:[c#2,SET-c#2,SET] +3: + 000006:[b#2,SET-b#2,SET] + 000007:[b#1,SET-b#1,SET] + 000008:[b#0,SET-b#0,SET] + +# Regression test for a bug where compaction would stop process range +# tombstones for an input level upon finding an sstable in the input +# level with no range tombstones. In the scenario below, sstable 6 +# does not contain any range tombstones while sstable 7 does. Both are +# compacted together with sstable 5. + +reset +---- + +batch +set a 1 +set b 1 +set c 1 +set d 1 +set z 1 +---- + +compact a-z +---- +6: + 000005:[a#1,SET-z#5,SET] + +build ext1 +set a 2 +---- + +build ext2 +set b 2 +del-range c z +---- + +ingest ext1 ext2 +---- +0.0: + 000006:[a#6,SET-a#6,SET] + 000007:[b#7,SET-z#72057594037927935,RANGEDEL] +6: + 000005:[a#1,SET-z#5,SET] + +iter +first +next +next +next +---- +a:2 +b:2 +z:1 +. + +compact a-z +---- +6: + 000008:[a#0,SET-z#0,SET] + +iter +first +next +next +next +---- +a:2 +b:2 +z:1 +. + +# Regresion test for a bug in sstable smallest boundary generation +# where the smallest key for an sstable was set to a key "larger" than +# the start key of the first range tombstone. This in turn fouled up +# the processing logic of range tombstones used by mergingIter which +# allowed stepping out of an sstable even though it contained a range +# tombstone that covered keys in lower levels. + +define target-file-sizes=(1, 1, 1, 1) +L0 + c.SET.4:4 +L1 + a.SET.3:3 +L2 + a.RANGEDEL.2:e +L3 + b.SET.1:1 +---- +0.0: + 000004:[c-c] +1: + 000005:[a-a] +2: + 000006:[a-e] +3: + 000007:[b-b] + +compact a-e L1 +---- +0.0: + 000004:[c#4,SET-c#4,SET] +2: + 000008:[a#3,SETWITHDEL-b#72057594037927935,RANGEDEL] + 000009:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000007:[b#1,SET-b#1,SET] + +# We should only see a:3 and c:4 at this point. + +iter +first +next +next +---- +a:3 +c:4 +. + +# The bug allowed seeing b:1 during reverse iteration. + +iter +last +prev +prev +---- +c:4 +a:3 +. + +# This is a similar scenario to the one above, but we've arranged for +# the compaction to produce an sstable (9) for which the smallest +# range tombstone overlaps in key space with the largest key in the +# previous sstable (8). This necessitates adjusting the seqnum of the +# range tombstone. There was no actual iteration bug from earlier +# behavior of setting the seqnum to 0, but now we set the seqnum to +# prev.LargestKey.SeqNum-1. +# +# Note the target-file-size of 26 is specially tailored to get the +# desired compaction output. + +define target-file-sizes=(26, 26, 26, 26) snapshots=(1, 2, 3) +L1 + a.SET.4:4 +L1 + b.SET.2:2 + b.RANGEDEL.3:e +L3 + b.SET.1:1 +---- +1: + 000004:[a-a] + 000005:[b-e] +3: + 000006:[b-b] + +compact a-e L1 +---- +2: + 000007:[a#4,SET-b#2,SET] + 000008:[b#1,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000006:[b#1,SET-b#1,SET] + +iter +first +next +last +prev +---- +a:4 +. +a:4 +. + +# Similar to the preceding scenario, except the range tombstone has +# the same seqnum as the largest key in the preceding file. + +define target-file-sizes=(26, 26, 26, 26) snapshots=(1, 2, 3) +L1 + a.SET.4:4 +L1 + b.SET.3:3 + b.RANGEDEL.3:e +L3 + b.SET.1:1 +---- +1: + 000004:[a-a] + 000005:[b-e] +3: + 000006:[b-b] + +compact a-e L1 +---- +2: + 000007:[a#4,SET-b#3,SET] + 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000006:[b#1,SET-b#1,SET] + +iter +first +next +next +last +prev +prev +---- +a:4 +b:3 +. +b:3 +a:4 +. + +# Similar to the preceding scenario, except the range tombstone has +# a smaller seqnum than the largest key in the preceding file. + +define target-file-sizes=(26, 26, 26, 26) snapshots=(1, 2, 3) +L1 + a.SET.4:4 +L1 + b.SET.4:4 + b.RANGEDEL.2:e +L3 + b.SET.1:1 +---- +1: + 000004:[a-a] + 000005:[b-e] +3: + 000006:[b-b] + +compact a-e L1 +---- +2: + 000007:[a#4,SET-b#4,SET] + 000008:[b#2,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000006:[b#1,SET-b#1,SET] + +iter +first +next +next +last +prev +prev +---- +a:4 +b:4 +. +b:4 +a:4 +. + +# Test a scenario where the last point key in an sstable has a seqnum +# of 0, and the smallest range tombstone in the next sstable starts +# with the same key. We have to tweak the kind of the boundary key in +# sstable 8 to be a DEL rather than a RANGEDEL so that the two tables +# are disjoint in the key space. + +define target-file-sizes=(1, 1, 26) snapshots=(2) +L1 + a.SET.3:3 + b.RANGEDEL.3:e + b.SET.0:0 +L3 + a.RANGEDEL.2:b +L3 + c.SET.0:0 + d.SET.0:0 +---- +1: + 000004:[a-e] +3: + 000005:[a-b] + 000006:[c-d] + +iter +last +prev +---- +a:3 +. + +compact a-e L1 +---- +2: + 000007:[a#3,SET-e#72057594037927935,RANGEDEL] +3: + 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL] + 000006:[c#0,SET-d#0,SET] + +iter +last +prev +---- +a:3 +. + +# Test a scenario where the last point key in an sstable before the +# grandparent limit is reached has a seqnum of 0. We want to cut the +# sstable after the next point key is added, rather than continuing to +# add keys indefinitely (or till the size limit is reached). + +define target-file-sizes=(100, 1, 52) snapshots=(2) +L1 + a.SET.3:3 + b.RANGEDEL.3:e + b.SET.0:0 + c.SET.1:1 + d.SET.1:1 +L3 + a.RANGEDEL.2:b +---- +1: + 000004:[a-e] +3: + 000005:[a-b] + +compact a-f L1 +---- +2: + 000006:[a#3,SET-c#72057594037927935,RANGEDEL] + 000007:[c#3,RANGEDEL-e#72057594037927935,RANGEDEL] +3: + 000005:[a#2,RANGEDEL-b#72057594037927935,RANGEDEL] + +# Test a scenario where we the last point key in an sstable has a +# seqnum of 0, but there is another range tombstone later in the +# compaction. This scenario was previously triggering an assertion due +# to the rangedel.Fragmenter being finished prematurely. + +define target-file-sizes=(1, 1, 1) +L1 + a.SET.0:0 + c.RANGEDEL.1:d +L3 + b.SET.0:0 +---- +1: + 000004:[a-d] +3: + 000005:[b-b] + +compact a-e L1 +---- +2: + 000006:[a#0,SET-a#0,SET] +3: + 000005:[b#0,SET-b#0,SET] + +define target-file-sizes=(1, 1, 1, 1) +L0 + b.SET.1:v +L0 + a.SET.2:v +---- +0.0: + 000005:[a-a] + 000004:[b-b] + +add-ongoing-compaction startLevel=0 outputLevel=1 +---- + +async-compact a-b L0 +---- +manual compaction blocked until ongoing finished +1: + 000006:[a#0,SET-a#0,SET] + 000007:[b#0,SET-b#0,SET] + +compact a-b L1 +---- +2: + 000008:[a#0,SET-a#0,SET] + 000009:[b#0,SET-b#0,SET] + +add-ongoing-compaction startLevel=0 outputLevel=1 +---- + +async-compact a-b L2 +---- +manual compaction blocked until ongoing finished +3: + 000010:[a#0,SET-a#0,SET] + 000011:[b#0,SET-b#0,SET] + +add-ongoing-compaction startLevel=0 outputLevel=1 +---- + +set-concurrent-compactions num=2 +---- + +async-compact a-b L3 +---- +manual compaction did not block for ongoing +4: + 000012:[a#0,SET-a#0,SET] + 000013:[b#0,SET-b#0,SET] + +remove-ongoing-compaction +---- + +add-ongoing-compaction startLevel=5 outputLevel=6 +---- + +async-compact a-b L4 +---- +manual compaction blocked until ongoing finished +5: + 000014:[a#0,SET-a#0,SET] + 000015:[b#0,SET-b#0,SET] + +# Test of a scenario where consecutive elided range tombstones and grandparent +# boundaries could result in an invariant violation in the rangedel fragmenter. + +define target-file-sizes=(1, 1, 1, 1) +L1 + a.RANGEDEL.4:b + c.RANGEDEL.4:d + e.RANGEDEL.4:f +L1 + g.RANGEDEL.6:h + i.RANGEDEL.4:j +L1 + k.RANGEDEL.5:q + m.RANGEDEL.4:q +L2 + a.SET.2:foo +L3 + a.SET.1:foo + c.SET.1:foo +L3 + ff.SET.1:v +L3 + k.SET.1:foo +---- +1: + 000004:[a-f] + 000005:[g-j] + 000006:[k-q] +2: + 000007:[a-a] +3: + 000008:[a-c] + 000009:[ff-ff] + 000010:[k-k] + +compact a-q L1 +---- +2: + 000011:[a#4,RANGEDEL-d#72057594037927935,RANGEDEL] + 000012:[k#5,RANGEDEL-m#72057594037927935,RANGEDEL] +3: + 000008:[a#1,SET-c#1,SET] + 000009:[ff#1,SET-ff#1,SET] + 000010:[k#1,SET-k#1,SET] + +# Test a case where a new output file is started, there are no previous output +# files, there are no additional keys (key = nil) and the rangedel fragmenter +# is non-empty. +define target-file-sizes=(1, 1, 1) +L1 + a.RANGEDEL.10:b + d.RANGEDEL.9:e + q.RANGEDEL.8:r +L2 + g.RANGEDEL.7:h +L3 + q.SET.6:6 +---- +1: + 000004:[a-r] +2: + 000005:[g-h] +3: + 000006:[q-q] + +compact a-r L1 +---- +2: + 000007:[q#8,RANGEDEL-r#72057594037927935,RANGEDEL] +3: + 000006:[q#6,SET-q#6,SET] + +define target-file-sizes=(100, 100, 100) +L1 + a.RANGEDEL.10:b + b.SET.0:foo + d.RANGEDEL.0:e + j.SET.10:foo +L2 + f.RANGEDEL.7:g +L3 + c.SET.6:6 +L3 + c.SET.5:5 +L3 + c.SET.4:4 +L4 + a.SET.0:0 + f.SET.0:0 +---- +1: + 000004:[a-j] +2: + 000005:[f-g] +3: + 000006:[c-c] + 000007:[c-c] + 000008:[c-c] +4: + 000009:[a-f] + +compact a-r L1 +---- +2: + 000010:[a#10,RANGEDEL-b#0,SET] + 000011:[d#0,RANGEDEL-j#10,SET] +3: + 000006:[c#6,SET-c#6,SET] + 000007:[c#5,SET-c#5,SET] + 000008:[c#4,SET-c#4,SET] +4: + 000009:[a#0,SET-f#0,SET] + +# Test a snapshot that separates a range deletion from all the data that it +# deletes. Ensure that we respect the target-file-size and split into multiple +# outputs. + +define target-file-sizes=(1, 1, 1) snapshots=(14) +L1 + a.RANGEDEL.15:z + b.SET.11:foo + c.SET.11:foo +L2 + c.SET.0:foo + d.SET.0:foo +---- +1: + 000004:[a-z] +2: + 000005:[c-d] + +compact a-z L1 +---- +2: + 000006:[a#15,RANGEDEL-c#72057594037927935,RANGEDEL] + 000007:[c#15,RANGEDEL-d#72057594037927935,RANGEDEL] + 000008:[d#15,RANGEDEL-z#72057594037927935,RANGEDEL] + +# Test an interaction between a range deletion that will be elided with +# output splitting. Ensure that the output is still split (previous versions +# of the code did not, because of intricacies around preventing a zero +# sequence number in an output's largest key). + +define target-file-sizes=(1, 1, 1) +L1 + a.RANGEDEL.10:z + b.SET.11:foo + c.SET.11:foo +L2 + c.SET.0:foo + d.SET.0:foo +---- +1: + 000004:[a-z] +2: + 000005:[c-d] + +compact a-z L1 +---- +2: + 000006:[b#0,SET-b#0,SET] + 000007:[c#0,SET-c#0,SET] diff --git a/testdata/singledel_manual_compaction_set_with_del b/testdata/singledel_manual_compaction_set_with_del new file mode 100644 index 0000000000..7d7e1053d5 --- /dev/null +++ b/testdata/singledel_manual_compaction_set_with_del @@ -0,0 +1,78 @@ +# This is not actually a manual compaction test, and simply uses manual +# compaction to demonstrate single delete semantics when used with +# set-with-delete. + +# Define a sequence of SET=>SET=>DEL=>SET=>SINGLEDEL. +define target-file-sizes=(1, 1, 1, 1, 1) +L1 + a.SINGLEDEL.10: +L2 + a.SET.9:v3 +L3 + a.DEL.8: +L4 + a.SET.7:v2 +L5 + a.SET.6:v1 +---- +1: + 000004:[a-a] +2: + 000005:[a-a] +3: + 000006:[a-a] +4: + 000007:[a-a] +5: + 000008:[a-a] + +# No data. +iter +first +---- +. + +# Compact away the DEL. +compact a-b L2 +---- +1: + 000004:[a#10,SINGLEDEL-a#10,SINGLEDEL] +3: + 000009:[a#9,SETWITHDEL-a#9,SETWITHDEL] +4: + 000007:[a#7,SET-a#7,SET] +5: + 000008:[a#6,SET-a#6,SET] + +# No data. +iter +first +---- +. + +# Do two compactions to compact away the SINGLEDEL and 1 SET. +compact a-b L1 +---- +2: + 000010:[a#10,SINGLEDEL-a#10,SINGLEDEL] +3: + 000009:[a#9,SETWITHDEL-a#9,SETWITHDEL] +4: + 000007:[a#7,SET-a#7,SET] +5: + 000008:[a#6,SET-a#6,SET] + +compact a-b L2 +---- +3: + 000011:[a#10,DEL-a#10,DEL] +4: + 000007:[a#7,SET-a#7,SET] +5: + 000008:[a#6,SET-a#6,SET] + +# Deleted data is not resurrected. +iter +first +---- +. diff --git a/tool/find.go b/tool/find.go index 214f9a256f..8714a5b757 100644 --- a/tool/find.go +++ b/tool/find.go @@ -345,7 +345,8 @@ func (f *findT) searchLogs(searchKey []byte, refs []findRef) []findRef { case base.InternalKeyKindDelete, base.InternalKeyKindSet, base.InternalKeyKindMerge, - base.InternalKeyKindSingleDelete: + base.InternalKeyKindSingleDelete, + base.InternalKeyKindSetWithDelete: if cmp(searchKey, ikey.UserKey) != 0 { continue } diff --git a/tool/wal.go b/tool/wal.go index 28092eef42..d701174bba 100644 --- a/tool/wal.go +++ b/tool/wal.go @@ -138,6 +138,8 @@ func (w *walT) runDump(cmd *cobra.Command, args []string) { fmt.Fprintf(stdout, "<%d>", len(value)) case base.InternalKeyKindSingleDelete: fmt.Fprintf(stdout, "%s", w.fmtKey.fn(ukey)) + case base.InternalKeyKindSetWithDelete: + fmt.Fprintf(stdout, "%s", w.fmtKey.fn(ukey)) case base.InternalKeyKindRangeDelete: fmt.Fprintf(stdout, "%s,%s", w.fmtKey.fn(ukey), w.fmtKey.fn(value)) }