diff --git a/batch.go b/batch.go index d2c8c62ff5..9bd4d007ed 100644 --- a/batch.go +++ b/batch.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/private" + "github.com/cockroachdb/pebble/internal/rangekey" "github.com/cockroachdb/pebble/internal/rawalloc" ) @@ -72,10 +73,11 @@ func (d DeferredBatchOp) Finish() { } } -// A Batch is a sequence of Sets, Merges, Deletes, and/or DeleteRanges that are -// applied atomically. Batch implements the Reader interface, but only an -// indexed batch supports reading (without error) via Get or NewIter. A -// non-indexed batch will return ErrNotIndexed when read from . +// A Batch is a sequence of Sets, Merges, Deletes, DeleteRanges, RangeKeySets, +// RangeKeyUnsets, and/or RangeKeyDeletes that are applied atomically. Batch +// implements the Reader interface, but only an indexed batch supports reading +// (without error) via Get or NewIter. A non-indexed batch will return +// ErrNotIndexed when read from . // // Indexing // @@ -86,8 +88,8 @@ func (d DeferredBatchOp) Finish() { // in the LSM where every entry in the batch is considered newer than any entry // in the underlying database (batch entries have the InternalKeySeqNumBatch // bit set). By treating the batch as an additional layer in the LSM, iteration -// supports all batch operations (i.e. Set, Merge, Delete, and DeleteRange) -// with minimal effort. +// supports all batch operations (i.e. Set, Merge, Delete, DeleteRange, +// RangeKeySet, RangeKeyUnset, RangeKeyDelete) with minimal effort. // // The same key can be operated on multiple times in a batch, though only the // latest operation will be visible. For example, Put("a", "b"), Delete("a") @@ -155,14 +157,21 @@ func (d DeferredBatchOp) Finish() { // exactly those specified by InternalKeyKind. The following table shows the // format for records of each kind: // -// InternalKeyKindDelete varstring -// InternalKeyKindLogData varstring -// InternalKeyKindSet varstring varstring -// InternalKeyKindMerge varstring varstring -// InternalKeyKindRangeDelete varstring varstring +// InternalKeyKindDelete varstring +// InternalKeyKindLogData varstring +// InternalKeyKindSet varstring varstring +// InternalKeyKindMerge varstring varstring +// InternalKeyKindRangeDelete varstring varstring +// InternalKeyKindRangeKeySet varstring varstring +// InternalKeyKindRangeKeyUnset varstring varstring +// InternalKeyKindRangeKeyDelete varstring varstring // -// The intuitive understanding here are that the arguments to Delete(), Set(), -// Merge(), and DeleteRange() are encoded into the batch. +// The intuitive understanding here are that the arguments to Delete, Set, +// Merge, DeleteRange and RangeKeyDelete are encoded into the batch. The +// RangeKeySet and RangeKeyUnset operations are slightly more complicated, +// encoding their end key, suffix and value [in the case of RangeKeySet] within +// the Value varstring. For more information on the value encoding for +// RangeKeySet and RangeKeyUnset, see the internal/rangekey package. // // The internal batch representation is the on disk format for a batch in the // WAL, and thus stable. New record kinds may be added, but the existing ones @@ -208,6 +217,13 @@ type Batch struct { // deletion is added. countRangeDels uint64 + // The count of range key sets, unsets and deletes in the batch. Updated + // every time a RANGEKEYSET, RANGEKEYUNSET or RANGEKEYDEL key is added. + // TODO(jackson): This likely won't be necessary long-term, but it's useful + // for the in-memory only implementation in which these keys require special + // handling. + countRangeKeys uint64 + // A deferredOp struct, stored in the Batch so that a pointer can be returned // from the *Deferred() methods rather than a value. deferredOp DeferredBatchOp @@ -215,12 +231,18 @@ type Batch struct { // An optional skiplist keyed by offset into data of the entry. index *batchskl.Skiplist rangeDelIndex *batchskl.Skiplist + rangeKeyIndex *batchskl.Skiplist // Fragmented range deletion tombstones. Cached the first time a range // deletion iterator is requested. The cache is invalidated whenever a new // range deletion is added to the batch. tombstones []keyspan.Span + // Fragmented range key spans. Cached the first time a range key iterator is + // requested. The cache is invalidated whenever a new range key (either Set + // or Unset) is added to the batch. + rangeKeys []keyspan.Span + // The flushableBatch wrapper if the batch is too large to fit in the // memtable. flushable *flushableBatch @@ -290,7 +312,7 @@ func (b *Batch) release() { if b.index == nil { batchPool.Put(b) } else { - b.index, b.rangeDelIndex = nil, nil + b.index, b.rangeDelIndex, b.rangeKeyIndex = nil, nil, nil indexedBatchPool.Put((*indexedBatch)(unsafe.Pointer(b))) } } @@ -302,14 +324,18 @@ func (b *Batch) refreshMemTableSize() { } b.countRangeDels = 0 + b.countRangeKeys = 0 for r := b.Reader(); ; { kind, key, value, ok := r.Next() if !ok { break } b.memTableSize += memTableEntrySize(len(key), len(value)) - if kind == InternalKeyKindRangeDelete { + switch kind { + case InternalKeyKindRangeDelete: b.countRangeDels++ + case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + b.countRangeKeys++ } } } @@ -343,18 +369,28 @@ func (b *Batch) Apply(batch *Batch, _ *WriteOptions) error { if !ok { break } - if kind == InternalKeyKindRangeDelete { + switch kind { + case InternalKeyKindRangeDelete: b.countRangeDels++ + case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + b.countRangeKeys++ } if b.index != nil { var err error - if kind == InternalKeyKindRangeDelete { + switch kind { + case InternalKeyKindRangeDelete: b.tombstones = nil if b.rangeDelIndex == nil { b.rangeDelIndex = batchskl.NewSkiplist(&b.data, b.cmp, b.abbreviatedKey) } err = b.rangeDelIndex.Add(uint32(offset)) - } else { + case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + b.rangeKeys = nil + if b.rangeKeyIndex == nil { + b.rangeKeyIndex = batchskl.NewSkiplist(&b.data, b.cmp, b.abbreviatedKey) + } + err = b.rangeKeyIndex.Add(uint32(offset)) + default: err = b.index.Add(uint32(offset)) } if err != nil { @@ -578,8 +614,9 @@ func (b *Batch) SingleDeleteDeferred(keyLen int) *DeferredBatchOp { return &b.deferredOp } -// DeleteRange deletes all of the keys (and values) in the range [start,end) -// (inclusive on start, exclusive on end). +// DeleteRange deletes all of the point keys (and values) in the range +// [start,end) (inclusive on start, exclusive on end). DeleteRange does NOT +// delete overlapping range keys (eg, keys set via RangeKeySet). // // It is safe to modify the contents of the arguments after DeleteRange // returns. @@ -618,6 +655,136 @@ func (b *Batch) DeleteRangeDeferred(startLen, endLen int) *DeferredBatchOp { return &b.deferredOp } +// Experimental returns the same batch, with experimental portions of the API +// exposed. +func (b *Batch) Experimental() ExperimentalBatch { + return ExperimentalBatch{b} +} + +// ExperimentalBatch provides access to experimental features of a Batch. +type ExperimentalBatch struct { + *Batch +} + +// RangeKeySet sets a range key mapping the key range [start, end) at the MVCC +// timestamp suffix to value. The suffix is optional. If any portion of the key +// range [start, end) is already set by a range key with the same suffix value, +// RangeKeySet overrides it. +// +// It is safe to modify the contents of the arguments after RangeKeySet returns. +// +// WARNING: This is an experimental feature with limited functionality. +func (b ExperimentalBatch) RangeKeySet(start, end, suffix, value []byte, _ *WriteOptions) error { + suffixValues := [1]rangekey.SuffixValue{{Suffix: suffix, Value: value}} + internalValueLen := rangekey.EncodedSetValueLen(end, suffixValues[:]) + + deferredOp := b.rangeKeySetDeferred(len(start), internalValueLen) + copy(deferredOp.Key, start) + n := rangekey.EncodeSetValue(deferredOp.Value, end, suffixValues[:]) + if n != internalValueLen { + panic("unexpected internal value length mismatch") + } + + // Manually inline DeferredBatchOp.Finish(). + if deferredOp.index != nil { + if err := deferredOp.index.Add(deferredOp.offset); err != nil { + // We never add duplicate entries, so an error should never occur. + panic(err) + } + } + return nil +} + +func (b *Batch) rangeKeySetDeferred(startLen, internalValueLen int) *DeferredBatchOp { + b.prepareDeferredKeyValueRecord(startLen, internalValueLen, InternalKeyKindRangeKeySet) + b.incrementRangeKeysCount() + return &b.deferredOp +} + +func (b *Batch) incrementRangeKeysCount() { + b.countRangeKeys++ + if b.index != nil { + b.rangeKeys = nil + // Range keys are rare, so we lazily allocate the index for them. + if b.rangeKeyIndex == nil { + b.rangeKeyIndex = batchskl.NewSkiplist(&b.data, b.cmp, b.abbreviatedKey) + } + b.deferredOp.index = b.rangeKeyIndex + } +} + +// RangeKeyUnset removes a range key mapping the key range [start, end) at the +// MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed +// range key. RangeKeyUnset only removes portions of range keys that fall within +// the [start, end) key span, and only range keys with suffixes that exactly +// match the unset suffix. +// +// It is safe to modify the contents of the arguments after RangeKeyUnset +// returns. +// +// WARNING: This is an experimental feature with limited functionality. +func (b ExperimentalBatch) RangeKeyUnset(start, end, suffix []byte, _ *WriteOptions) error { + suffixes := [1][]byte{suffix} + internalValueLen := rangekey.EncodedUnsetValueLen(end, suffixes[:]) + + deferredOp := b.rangeKeyUnsetDeferred(len(start), internalValueLen) + copy(deferredOp.Key, start) + n := rangekey.EncodeUnsetValue(deferredOp.Value, end, suffixes[:]) + if n != internalValueLen { + panic("unexpected internal value length mismatch") + } + + // Manually inline DeferredBatchOp.Finish() + if deferredOp.index != nil { + if err := deferredOp.index.Add(deferredOp.offset); err != nil { + // We never add duplicate entries, so an error should never occur. + panic(err) + } + } + return nil +} + +func (b *Batch) rangeKeyUnsetDeferred(startLen, internalValueLen int) *DeferredBatchOp { + b.prepareDeferredKeyValueRecord(startLen, internalValueLen, InternalKeyKindRangeKeyUnset) + b.incrementRangeKeysCount() + return &b.deferredOp +} + +// RangeKeyDelete deletes all of the range keys in the range [start,end) +// (inclusive on start, exclusive on end). It does not delete point keys (for +// that use DeleteRange). RangeKeyDelete removes all range keys within the +// bounds, including those with or without suffixes. +// +// It is safe to modify the contents of the arguments after RangeKeyDelete +// returns. +// +// WARNING: This is an experimental feature with limited functionality. +func (b ExperimentalBatch) RangeKeyDelete(start, end []byte, _ *WriteOptions) error { + deferredOp := b.RangeKeyDeleteDeferred(len(start), len(end)) + copy(deferredOp.Key, start) + copy(deferredOp.Value, end) + // Manually inline DeferredBatchOp.Finish(). + if deferredOp.index != nil { + if err := deferredOp.index.Add(deferredOp.offset); err != nil { + // We never add duplicate entries, so an error should never occur. + panic(err) + } + } + return nil +} + +// RangeKeyDeleteDeferred is similar to RangeKeyDelete in that it adds an +// operation to delete range keys to the batch, except it only takes in key +// lengths instead of complete slices, letting the caller encode into those +// objects and then call Finish() on the returned object. Note that +// DeferredBatchOp.Key should be populated with the start key, and +// DeferredBatchOp.Value should be populated with the end key. +func (b ExperimentalBatch) RangeKeyDeleteDeferred(startLen, endLen int) *DeferredBatchOp { + b.prepareDeferredKeyValueRecord(startLen, endLen, InternalKeyKindRangeKeyDelete) + b.incrementRangeKeysCount() + return &b.deferredOp +} + // LogData adds the specified to the batch. The data will be written to the // WAL, but not added to memtables or sstables. Log data is never indexed, // which makes it useful for testing WAL performance. @@ -913,7 +1080,7 @@ func (r *BatchReader) Next() (kind InternalKeyKind, ukey []byte, value []byte, o return 0, nil, nil, false } switch kind { - case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete: + case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete, InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: *r, value, ok = batchDecodeStr(*r) if !ok { return 0, nil, nil, false diff --git a/batch_test.go b/batch_test.go index 71e4a9bfb2..3a8fcadb7b 100644 --- a/batch_test.go +++ b/batch_test.go @@ -46,6 +46,9 @@ func TestBatch(t *testing.T) { } } + // RangeKeySet and RangeKeyUnset are untested here because they don't expose + // deferred variants. This is a consequence of these keys' more complex + // value encodings. testCases := []testCase{ {InternalKeyKindSet, "roses", "red"}, {InternalKeyKindSet, "violets", "blue"}, @@ -69,6 +72,8 @@ func TestBatch(t *testing.T) { {InternalKeyKindRangeDelete, "", ""}, {InternalKeyKindLogData, "logdata", ""}, {InternalKeyKindLogData, "", ""}, + {InternalKeyKindRangeKeyDelete, "grass", "green"}, + {InternalKeyKindRangeKeyDelete, "", ""}, } var b Batch for _, tc := range testCases { @@ -85,6 +90,8 @@ func TestBatch(t *testing.T) { _ = b.DeleteRange([]byte(tc.key), []byte(tc.value), nil) case InternalKeyKindLogData: _ = b.LogData([]byte(tc.key), nil) + case InternalKeyKindRangeKeyDelete: + _ = b.Experimental().RangeKeyDelete([]byte(tc.key), []byte(tc.value), nil) } } verifyTestCases(&b, testCases) @@ -123,6 +130,11 @@ func TestBatch(t *testing.T) { d.Finish() case InternalKeyKindLogData: _ = b.LogData([]byte(tc.key), nil) + case InternalKeyKindRangeKeyDelete: + d := b.Experimental().RangeKeyDeleteDeferred(len(key), len(value)) + copy(d.Key, key) + copy(d.Value, value) + d.Finish() } } verifyTestCases(&b, testCases) @@ -132,37 +144,26 @@ func TestBatchEmpty(t *testing.T) { var b Batch require.True(t, b.Empty()) - b.Set(nil, nil, nil) - require.False(t, b.Empty()) - b.Reset() - require.True(t, b.Empty()) - // Reset may choose to reuse b.data, so clear it to the zero value in - // order to test the lazy initialization of b.data. - b = Batch{} - - b.Merge(nil, nil, nil) - require.False(t, b.Empty()) - b.Reset() - require.True(t, b.Empty()) - b = Batch{} - - b.Delete(nil, nil) - require.False(t, b.Empty()) - b.Reset() - require.True(t, b.Empty()) - b = Batch{} - - b.DeleteRange(nil, nil, nil) - require.False(t, b.Empty()) - b.Reset() - require.True(t, b.Empty()) - b = Batch{} + ops := []func(*Batch){ + func(b *Batch) { b.Set(nil, nil, nil) }, + func(b *Batch) { b.Merge(nil, nil, nil) }, + func(b *Batch) { b.Delete(nil, nil) }, + func(b *Batch) { b.DeleteRange(nil, nil, nil) }, + func(b *Batch) { b.LogData(nil, nil) }, + func(b *Batch) { b.Experimental().RangeKeySet(nil, nil, nil, nil, nil) }, + func(b *Batch) { b.Experimental().RangeKeyUnset(nil, nil, nil, nil) }, + func(b *Batch) { b.Experimental().RangeKeyDelete(nil, nil, nil) }, + } - b.LogData(nil, nil) - require.False(t, b.Empty()) - b.Reset() - require.True(t, b.Empty()) - b = Batch{} + for _, op := range ops { + op(&b) + require.False(t, b.Empty()) + b.Reset() + require.True(t, b.Empty()) + // Reset may choose to reuse b.data, so clear it to the zero value in + // order to test the lazy initialization of b.data. + b = Batch{} + } _ = b.Reader() require.True(t, b.Empty()) diff --git a/db.go b/db.go index fcd60cb18d..3075373283 100644 --- a/db.go +++ b/db.go @@ -108,10 +108,12 @@ type Writer interface { // It is safe to modify the contents of the arguments after SingleDelete returns. SingleDelete(key []byte, o *WriteOptions) error - // DeleteRange deletes all of the keys (and values) in the range [start,end) - // (inclusive on start, exclusive on end). + // DeleteRange deletes all of the point keys (and values) in the range + // [start,end) (inclusive on start, exclusive on end). DeleteRange does NOT + // delete overlapping range keys (eg, keys set via RangeKeySet). // - // It is safe to modify the contents of the arguments after Delete returns. + // It is safe to modify the contents of the arguments after DeleteRange + // returns. DeleteRange(start, end []byte, o *WriteOptions) error // LogData adds the specified to the batch. The data will be written to the diff --git a/internal.go b/internal.go index 563785ee67..34be0d865e 100644 --- a/internal.go +++ b/internal.go @@ -19,6 +19,9 @@ const ( InternalKeyKindRangeDelete = base.InternalKeyKindRangeDelete InternalKeyKindMax = base.InternalKeyKindMax InternalKeyKindSetWithDelete = base.InternalKeyKindSetWithDelete + InternalKeyKindRangeKeySet = base.InternalKeyKindRangeKeySet + InternalKeyKindRangeKeyUnset = base.InternalKeyKindRangeKeyUnset + InternalKeyKindRangeKeyDelete = base.InternalKeyKindRangeKeyDelete InternalKeyKindInvalid = base.InternalKeyKindInvalid InternalKeySeqNumBatch = base.InternalKeySeqNumBatch InternalKeySeqNumMax = base.InternalKeySeqNumMax diff --git a/internal/base/internal.go b/internal/base/internal.go index c3f90caa4c..0f91befa6f 100644 --- a/internal/base/internal.go +++ b/internal/base/internal.go @@ -51,6 +51,15 @@ const ( // https://github.com/cockroachdb/pebble/issues/1255. InternalKeyKindSetWithDelete InternalKeyKind = 18 + // InternalKeyKindRangeKeySet and InternalKeyKindRangeUnset represent + // keys that set and unset values associated with ranges of key + // space. See the internal/rangekey package for more details. + InternalKeyKindRangeKeySet InternalKeyKind = 19 + InternalKeyKindRangeKeyUnset InternalKeyKind = 20 + // InternalKeyKindRangeKeyDelete removes all range keys within a key range. + // See the internal/rangekey package for more details. + InternalKeyKindRangeKeyDelete InternalKeyKind = 21 + // This maximum value isn't part of the file format. It's unlikely, // but future extensions may increase this value. // @@ -60,7 +69,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 = 18 + InternalKeyKindMax InternalKeyKind = 21 // A marker for an invalid key. InternalKeyKindInvalid InternalKeyKind = 255 @@ -81,15 +90,18 @@ const ( ) var internalKeyKindNames = []string{ - InternalKeyKindDelete: "DEL", - InternalKeyKindSet: "SET", - InternalKeyKindMerge: "MERGE", - InternalKeyKindLogData: "LOGDATA", - InternalKeyKindSingleDelete: "SINGLEDEL", - InternalKeyKindRangeDelete: "RANGEDEL", - InternalKeyKindSeparator: "SEPARATOR", - InternalKeyKindSetWithDelete: "SETWITHDEL", - InternalKeyKindInvalid: "INVALID", + InternalKeyKindDelete: "DEL", + InternalKeyKindSet: "SET", + InternalKeyKindMerge: "MERGE", + InternalKeyKindLogData: "LOGDATA", + InternalKeyKindSingleDelete: "SINGLEDEL", + InternalKeyKindRangeDelete: "RANGEDEL", + InternalKeyKindSeparator: "SEPARATOR", + InternalKeyKindSetWithDelete: "SETWITHDEL", + InternalKeyKindRangeKeySet: "RANGEKEYSET", + InternalKeyKindRangeKeyUnset: "RANGEKEYUNSET", + InternalKeyKindRangeKeyDelete: "RANGEKEYDEL", + InternalKeyKindInvalid: "INVALID", } func (k InternalKeyKind) String() string { @@ -146,14 +158,17 @@ func MakeRangeDeleteSentinelKey(userKey []byte) InternalKey { } var kindsMap = map[string]InternalKeyKind{ - "DEL": InternalKeyKindDelete, - "SINGLEDEL": InternalKeyKindSingleDelete, - "RANGEDEL": InternalKeyKindRangeDelete, - "SET": InternalKeyKindSet, - "MERGE": InternalKeyKindMerge, - "INVALID": InternalKeyKindInvalid, - "SEPARATOR": InternalKeyKindSeparator, - "SETWITHDEL": InternalKeyKindSetWithDelete, + "DEL": InternalKeyKindDelete, + "SINGLEDEL": InternalKeyKindSingleDelete, + "RANGEDEL": InternalKeyKindRangeDelete, + "SET": InternalKeyKindSet, + "MERGE": InternalKeyKindMerge, + "INVALID": InternalKeyKindInvalid, + "SEPARATOR": InternalKeyKindSeparator, + "SETWITHDEL": InternalKeyKindSetWithDelete, + "RANGEKEYSET": InternalKeyKindRangeKeySet, + "RANGEKEYUNSET": InternalKeyKindRangeKeyUnset, + "RANGEKEYDEL": InternalKeyKindRangeKeyDelete, } // 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 d41d060114..d8eee30d86 100644 --- a/internal/base/internal_test.go +++ b/internal/base/internal_test.go @@ -37,14 +37,13 @@ func TestInvalidInternalKey(t *testing.T) { "\x01\x02\x03\x04\x05\x06\x07", "foo", "foo\x08\x07\x06\x05\x04\x03\x02", - "foo\x13\x07\x06\x05\x04\x03\x02\x01", + "foo\x16\x07\x06\x05\x04\x03\x02\x01", } for _, tc := range testCases { k := DecodeInternalKey([]byte(tc)) if k.Valid() { t.Errorf("%q is a valid key, want invalid", tc) } - // Invalid key kind because the key doesn't have an 8 byte trailer. if k.Kind() == InternalKeyKindInvalid && k.UserKey != nil { t.Errorf("expected nil UserKey after decoding encodedKey=%q", tc) diff --git a/internal/rangekey/rangekey.go b/internal/rangekey/rangekey.go new file mode 100644 index 0000000000..c692ff6495 --- /dev/null +++ b/internal/rangekey/rangekey.go @@ -0,0 +1,334 @@ +// Copyright 2021 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +// Package rangekey provides facilities for encoding, decoding and merging range +// keys. +// +// Range keys map a span of keyspan `[start, end)`, at an optional suffix, to a +// value. +// +// # Encoding +// +// Unlike other Pebble keys, range keys encode several fields of information: +// start key, end key, suffix and value. Internally within Pebble and its +// sstables, all keys including range keys are represented as a key-value tuple. +// Range keys map to internal key-value tuples by mapping the start key to the +// key and encoding the remainder of the fields in the value. +// +// ## `RANGEKEYSET` +// +// A `RANGEKEYSET` represents one more range keys set over a single region of +// user key space. Each represented range key must have a unique suffix. A +// `RANGEKEYSET` encapsulates a start key, an end key and a set of SuffixValue +// pairs. +// +// A `RANGEKEYSET` key's user key holds the start key. Its value is a varstring +// end key, followed by a set of SuffixValue pairs. A `RANGEKEYSET` may have +// multiple SuffixValue pairs if the keyspan was set at multiple unique suffix +// values. +// +// ## `RANGEKEYUNSET` +// +// A `RANGEKEYUNSET` represents the removal of range keys at specific suffixes +// over a single region of user key space. A `RANGEKEYUNSET` encapsulates a +// start key, an end key and a set of suffixes. +// +// A `RANGEKEYUNSET` key's user key holds the start key. Its value is a +// varstring end key, followed by a set of suffixes. A `RANGEKEYUNSET` may have +// multiple suffixes if the keyspan was unset at multiple unique suffixes. +// +// ## `RANGEKEYDEL` +// +// A `RANGEKEYDEL` represents the removal of all range keys over a single region +// of user key space, regardless of suffix. A `RANGEKEYDEL` encapsulates a +// start key and an end key. The end key is stored in the value, without any +// varstring length prefixing. +package rangekey + +// TODO(jackson): Document the encoding of RANGEKEYSET and RANGEKEYUNSET values +// once we're confident they're stable. + +import ( + "encoding/binary" + "fmt" + "strings" + + "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/keyspan" +) + +// SuffixValue represents a tuple of a suffix and a corresponding value. A +// physical RANGEKEYSET key may contain many logical RangeKeySets, each +// represented with a separate SuffixValue tuple. +type SuffixValue struct { + Suffix []byte + Value []byte +} + +// EncodedSetValueLen precomputes the length of a RangeKeySet's value when +// encoded. It may be used to construct a buffer of the appropriate size before +// encoding. +func EncodedSetValueLen(endKey []byte, suffixValues []SuffixValue) int { + n := lenVarint(len(endKey)) + n += len(endKey) + for i := 0; i < len(suffixValues); i++ { + n += lenVarint(len(suffixValues[i].Suffix)) + n += len(suffixValues[i].Suffix) + n += lenVarint(len(suffixValues[i].Value)) + n += len(suffixValues[i].Value) + } + + return n +} + +// EncodeSetValue encodes a RangeKeySet's value into dst. The length of dst must +// be greater than or equal to EncodedSetValueLen. EncodeSetValue returns the +// number of bytes written, which should always equal the EncodedSetValueLen +// with the same arguments. +func EncodeSetValue(dst []byte, endKey []byte, suffixValues []SuffixValue) int { + // First encode the end key as a varstring. + n := binary.PutUvarint(dst, uint64(len(endKey))) + n += copy(dst[n:], endKey) + + // Encode the list of (suffix, value-len) tuples. + for i := 0; i < len(suffixValues); i++ { + // Encode the length of the suffix. + n += binary.PutUvarint(dst[n:], uint64(len(suffixValues[i].Suffix))) + + // Encode the suffix itself. + n += copy(dst[n:], suffixValues[i].Suffix) + + // Encode the value length. + n += binary.PutUvarint(dst[n:], uint64(len(suffixValues[i].Value))) + + // Encode the value itself. + n += copy(dst[n:], suffixValues[i].Value) + } + return n +} + +// DecodeEndKey reads the end key from the beginning of a range key (RANGEKEYSET, +// RANGEKEYUNSET or RANGEKEYDEL)'s physical encoded value. Both sets and unsets +// encode the range key, plus additional data in the value. +func DecodeEndKey(kind base.InternalKeyKind, data []byte) (endKey, value []byte, ok bool) { + switch kind { + case base.InternalKeyKindRangeKeyDelete: + // No splitting is necessary for range key deletes. The value is the end + // key, and there is no additional associated value. + return data, nil, true + case base.InternalKeyKindRangeKeySet, base.InternalKeyKindRangeKeyUnset: + v, n := binary.Uvarint(data) + if n <= 0 { + return nil, nil, false + } + endKey, value = data[n:n+int(v)], data[n+int(v):] + return endKey, value, true + default: + panic(errors.Newf("key kind %s is not a range key kind", kind)) + } +} + +// DecodeSuffixValue decodes a single encoded SuffixValue from a RangeKeySet's +// split value. The end key must have already been stripped from the +// RangeKeySet's value (see DecodeEndKey). +func DecodeSuffixValue(data []byte) (sv SuffixValue, rest []byte, ok bool) { + // Decode the suffix. + sv.Suffix, data, ok = decodeVarstring(data) + if !ok { + return SuffixValue{}, nil, false + } + // Decode the value. + sv.Value, data, ok = decodeVarstring(data) + if !ok { + return SuffixValue{}, nil, false + } + return sv, data, true +} + +// EncodedUnsetValueLen precomputes the length of a RangeKeyUnset's value when +// encoded. It may be used to construct a buffer of the appropriate size before +// encoding. +func EncodedUnsetValueLen(endKey []byte, suffixes [][]byte) int { + n := lenVarint(len(endKey)) + n += len(endKey) + + for i := 0; i < len(suffixes); i++ { + n += lenVarint(len(suffixes[i])) + n += len(suffixes[i]) + } + return n +} + +// EncodeUnsetValue encodes a RangeKeyUnset's value into dst. The length of dst +// must be greater than or equal to EncodedUnsetValueLen. EncodeUnsetValue +// returns the number of bytes written, which should always equal the +// EncodedUnsetValueLen with the same arguments. +func EncodeUnsetValue(dst []byte, endKey []byte, suffixes [][]byte) int { + // First encode the end key as a varstring. + n := binary.PutUvarint(dst, uint64(len(endKey))) + n += copy(dst[n:], endKey) + + // Encode the list of suffix varstrings. + for i := 0; i < len(suffixes); i++ { + // Encode the length of the suffix. + n += binary.PutUvarint(dst[n:], uint64(len(suffixes[i]))) + + // Encode the suffix itself. + n += copy(dst[n:], suffixes[i]) + } + return n +} + +// DecodeSuffix decodes a single suffix from the beginning of data. If decoding +// suffixes from a RangeKeyUnset's value, the end key must have already been +// stripped from the RangeKeyUnset's value (see DecodeEndKey). +func DecodeSuffix(data []byte) (suffix, rest []byte, ok bool) { + return decodeVarstring(data) +} + +func decodeVarstring(data []byte) (v, rest []byte, ok bool) { + // Decode the length of the string. + l, n := binary.Uvarint(data) + if n <= 0 { + return nil, nil, ok + } + + // Extract the string itself. + return data[n : n+int(l)], data[n+int(l):], true +} + +// Format returns a formatter for the range key (either a RANGEKEYSET, +// RANGEKEYUNSET or RANGEKEYDEL) represented by s. The formatting returned is +// parseable with Parse. +func Format(formatKey base.FormatKey, s keyspan.Span) fmt.Formatter { + return prettyRangeKeySpan{Span: s, formatKey: formatKey} +} + +type prettyRangeKeySpan struct { + keyspan.Span + formatKey base.FormatKey +} + +func (k prettyRangeKeySpan) Format(s fmt.State, c rune) { + fmt.Fprintf(s, "%s.%s.%d: %s", + k.formatKey(k.Start.UserKey), + k.Start.Kind(), + k.Start.SeqNum(), + k.End) + switch k.Start.Kind() { + case base.InternalKeyKindRangeKeySet: + fmt.Fprint(s, " [") + value := k.Value + for len(value) > 0 { + if len(value) < len(k.Value) { + fmt.Fprint(s, ",") + } + sv, rest, ok := DecodeSuffixValue(value) + if !ok { + panic(base.CorruptionErrorf("corrupt set value: unable to decode suffix-value tuple")) + } + value = rest + fmt.Fprintf(s, "(%s=%s)", k.formatKey(sv.Suffix), sv.Value) + } + fmt.Fprint(s, "]") + case base.InternalKeyKindRangeKeyUnset: + fmt.Fprint(s, " [") + value := k.Value + for len(value) > 0 { + if len(value) < len(k.Value) { + fmt.Fprint(s, ",") + } + suffix, rest, ok := DecodeSuffix(value) + if !ok { + panic(base.CorruptionErrorf("corrupt unset value: unable to decode suffix")) + } + value = rest + fmt.Fprint(s, k.formatKey(suffix)) + } + fmt.Fprint(s, "]") + case base.InternalKeyKindRangeKeyDelete: + if len(k.Value) > 0 { + panic("unexpected value on a RANGEKEYDEL") + } + // No additional value to format. + default: + panic(fmt.Sprintf("%s keys are not range keys", k.Start.Kind())) + } +} + +// Parse parses a string representation of a range key (eg, RANGEKEYSET, +// RANGEKEYUNSET or RANGEKEYDEL). Parse is used in tests and debugging +// facilities. It's exported for use in tests outside of the rangekey package. +// +// Parse expects the input string to be in one of the three formats: +// - start.RANGEKEYSET.seqnum: end [(s1=v1), (s2=v2), (s3=v3)] +// - start.RANGEKEYUNSET.seqnum: end [s1, s2, s3] +// - start.RANGEKEYDEL.seqnum: end +// +// For example: +// - a.RANGEKEYSET.5: c [(@t10=foo), (@t9=bar)] +// - a.RANGEKEYUNSET.5: c [@t10, @t9] +// - a.RANGEKEYDEL.5: c +func Parse(s string) (key base.InternalKey, value []byte) { + sep := strings.IndexByte(s, ':') + if sep == -1 { + panic("range key string representation missing key-value separator :") + } + startKey := base.ParseInternalKey(strings.TrimSpace(s[:sep])) + + switch startKey.Kind() { + case base.InternalKeyKindRangeKeySet: + openBracket := strings.IndexByte(s[sep:], '[') + closeBracket := strings.IndexByte(s[sep:], ']') + endKey := strings.TrimSpace(s[sep+1 : sep+openBracket]) + itemStrs := strings.Split(s[sep+openBracket+1:sep+closeBracket], ",") + + var suffixValues []SuffixValue + for _, itemStr := range itemStrs { + itemStr = strings.Trim(itemStr, "() \n\t") + i := strings.IndexByte(itemStr, '=') + if i == -1 { + panic(fmt.Sprintf("range key string %q missing '=' key,value tuple delim", s)) + } + suffixValues = append(suffixValues, SuffixValue{ + Suffix: []byte(strings.TrimSpace(itemStr[:i])), + Value: []byte(strings.TrimSpace(itemStr[i+1:])), + }) + } + value = make([]byte, EncodedSetValueLen([]byte(endKey), suffixValues)) + EncodeSetValue(value, []byte(endKey), suffixValues) + return startKey, value + + case base.InternalKeyKindRangeKeyUnset: + openBracket := strings.IndexByte(s[sep:], '[') + closeBracket := strings.IndexByte(s[sep:], ']') + endKey := strings.TrimSpace(s[sep+1 : sep+openBracket]) + itemStrs := strings.Split(s[sep+openBracket+1:sep+closeBracket], ",") + + var suffixes [][]byte + for _, itemStr := range itemStrs { + suffixes = append(suffixes, []byte(strings.TrimSpace(itemStr))) + } + value = make([]byte, EncodedUnsetValueLen([]byte(endKey), suffixes)) + EncodeUnsetValue(value, []byte(endKey), suffixes) + return startKey, value + + case base.InternalKeyKindRangeKeyDelete: + return startKey, []byte(strings.TrimSpace(s[sep+1:])) + + default: + panic(fmt.Sprintf("key kind %q not a range key", startKey.Kind())) + } +} + +func lenVarint(v int) (n int) { + x := uint32(v) + n++ + for x >= 0x80 { + x >>= 7 + n++ + } + return n +} diff --git a/internal/rangekey/rangekey_test.go b/internal/rangekey/rangekey_test.go new file mode 100644 index 0000000000..066ef78c61 --- /dev/null +++ b/internal/rangekey/rangekey_test.go @@ -0,0 +1,148 @@ +package rangekey + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/keyspan" + "github.com/cockroachdb/pebble/internal/testkeys" + "github.com/stretchr/testify/require" +) + +func TestSetValue_Roundtrip(t *testing.T) { + testCases := []struct { + endKey []byte + suffixValues []SuffixValue + }{ + { + endKey: []byte("hello"), + suffixValues: []SuffixValue{ + {Suffix: []byte{}, Value: []byte("world")}, + }, + }, + { + endKey: []byte("hello world"), + suffixValues: []SuffixValue{ + {Suffix: []byte("foo"), Value: []byte("bar")}, + }, + }, + { + endKey: []byte("hello world"), + suffixValues: []SuffixValue{ + {Suffix: []byte(""), Value: []byte("boop")}, + {Suffix: []byte("foo"), Value: []byte("beep")}, + {Suffix: []byte("bar"), Value: []byte("bop")}, + {Suffix: []byte("bax"), Value: []byte("boink")}, + {Suffix: []byte("zoop"), Value: []byte("zoink")}, + }, + }, + } + + var b []byte + for _, tc := range testCases { + l := EncodedSetValueLen(tc.endKey, tc.suffixValues) + + if l <= cap(b) { + b = b[:l] + } else { + b = make([]byte, l) + } + + n := EncodeSetValue(b, tc.endKey, tc.suffixValues) + require.Equal(t, l, n) + + var endKey, rest []byte + var ok bool + endKey, rest, ok = DecodeEndKey(base.InternalKeyKindRangeKeySet, b[:n]) + require.True(t, ok) + + var suffixValues []SuffixValue + for len(rest) > 0 { + var sv SuffixValue + var ok bool + sv, rest, ok = DecodeSuffixValue(rest) + require.True(t, ok) + suffixValues = append(suffixValues, sv) + } + require.Equal(t, tc.endKey, endKey) + require.Equal(t, tc.suffixValues, suffixValues) + } +} + +func TestUnsetValue_Roundtrip(t *testing.T) { + testCases := []struct { + endKey []byte + suffixes [][]byte + }{ + { + endKey: []byte("hello"), + suffixes: [][]byte{{}}, + }, + { + endKey: []byte("hello world"), + suffixes: [][]byte{[]byte("foo")}, + }, + { + endKey: []byte("hello world"), + suffixes: [][]byte{ + []byte{}, + []byte("foo"), + []byte("bar"), + []byte("bax"), + []byte("zoop"), + }, + }, + } + + var b []byte + for _, tc := range testCases { + l := EncodedUnsetValueLen(tc.endKey, tc.suffixes) + + if l <= cap(b) { + b = b[:l] + } else { + b = make([]byte, l) + } + + n := EncodeUnsetValue(b, tc.endKey, tc.suffixes) + require.Equal(t, l, n) + + var ok bool + var endKey, rest []byte + endKey, rest, ok = DecodeEndKey(base.InternalKeyKindRangeKeyUnset, b[:n]) + require.True(t, ok) + var suffixes [][]byte + for len(rest) > 0 { + var ok bool + var suffix []byte + suffix, rest, ok = DecodeSuffix(rest) + require.True(t, ok) + suffixes = append(suffixes, suffix) + } + require.Equal(t, tc.endKey, endKey) + require.Equal(t, tc.suffixes, suffixes) + } +} + +func TestParseFormatRoundtrip(t *testing.T) { + testCases := []string{ + "a.RANGEKEYSET.100: c [(@t22=foo),(@t1=bar)]", + "apples.RANGEKEYSET.5: bananas [(@t1=bar)]", + "cat.RANGEKEYUNSET.5: catatonic [@t9,@t8,@t7,@t6,@t5]", + "a.RANGEKEYDEL.5: catatonic", + } + for _, in := range testCases { + k, v := Parse(in) + endKey, restValue, ok := DecodeEndKey(k.Kind(), v) + require.True(t, ok) + got := fmt.Sprintf("%s", Format(testkeys.Comparer.FormatKey, keyspan.Span{ + Start: k, + End: endKey, + Value: restValue, + })) + if got != in { + t.Errorf("Format(Parse(%q)) = %q, want %q", in, got, in) + } + } +} diff --git a/mem_table.go b/mem_table.go index 50af703a2c..a7bcdd5107 100644 --- a/mem_table.go +++ b/mem_table.go @@ -193,6 +193,8 @@ func (m *memTable) apply(batch *Batch, seqNum uint64) error { // Don't increment seqNum for LogData, since these are not applied // to the memtable. seqNum-- + case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + // TODO(jackson): Implement. default: err = ins.Add(&m.skl, ikey, value) } diff --git a/range_keys_test.go b/range_keys_test.go new file mode 100644 index 0000000000..43ea6e255d --- /dev/null +++ b/range_keys_test.go @@ -0,0 +1,34 @@ +// Copyright 2021 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. +package pebble + +import ( + "testing" + + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" +) + +func TestRangeKeys(t *testing.T) { + d, err := Open("", &Options{FS: vfs.NewMem()}) + require.NoError(t, err) + defer d.Close() + + b := d.NewBatch() + b.Experimental().RangeKeySet([]byte("a"), []byte("c"), []byte("@t10"), []byte("hello world"), nil) + b.Experimental().RangeKeySet([]byte("b"), []byte("f"), []byte("@t20"), []byte("hello monde"), nil) + require.NoError(t, b.Commit(nil)) + + b = d.NewBatch() + b.Experimental().RangeKeySet([]byte("h"), []byte("q"), []byte("@t30"), []byte("foo"), nil) + require.NoError(t, b.Commit(nil)) + + b = d.NewBatch() + b.Experimental().RangeKeyUnset([]byte("e"), []byte("j"), []byte("@t20"), nil) + b.Experimental().RangeKeyUnset([]byte("e"), []byte("j"), []byte("@t10"), nil) + b.Experimental().RangeKeyUnset([]byte("e"), []byte("j"), []byte("@t30"), nil) + require.NoError(t, b.Commit(nil)) + + // TODO(jackson): Fill out when implemented. +}