Skip to content

Commit

Permalink
Merge #83031
Browse files Browse the repository at this point in the history
83031: storage: support reading and writing raw engine range keys r=nicktrav a=erikgrinaker

**storage: rename `ClearAllMVCCRangeKeys` to `ClearAllRangeKeys`**

Since `Engine.ExperimentalClearAllMVCCRangeKeys` takes only
`roachpb.Key` bounds, it can be used to clear both MVCC range keys and
more general engine range keys. This patch therefore renames the method
to `ExperimentalClearAllRangeKeys` to make it agnostic to the range key
type.

Release note: None
  
**storage: support range keys in `EngineIterator`**

This patch adds range key support to `EngineIterator`. This is needed
e.g. to process range keys in the Raft machinery, which operates on
arbitrary range data.

No tests are included, as there is no existing test framework for
`EngineIterator`. The logic is a simple wrapper around Pebble, so this
was considered less problematic than it would normally be.

Resolves #82935.

Release note: None
  
**storage: add `Engine.ExperimentalPutEngineRangeKey`**

This patch adds `Engine.ExperimentalPutEngineRangeKey` to write raw
engine range keys. This will be used e.g. when ingesting Raft snapshots,
which is agnostic to the kind of range key and doesn't care about
decoding or encoding them.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Jun 24, 2022
2 parents 4ff65df + fb91e27 commit c09d7fc
Show file tree
Hide file tree
Showing 9 changed files with 286 additions and 90 deletions.
34 changes: 32 additions & 2 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,6 +358,21 @@ func (i *EngineIterator) checkKeyAllowed() (valid bool, err error) {
return true, nil
}

// HasEnginePointAndRange is part of the storage.EngineIterator interface.
func (i *EngineIterator) HasEnginePointAndRange() (bool, bool) {
return i.i.HasEnginePointAndRange()
}

// EngineRangeBounds is part of the storage.EngineIterator interface.
func (i *EngineIterator) EngineRangeBounds() (roachpb.Span, error) {
return i.i.EngineRangeBounds()
}

// EngineRangeKeys is part of the storage.EngineIterator interface.
func (i *EngineIterator) EngineRangeKeys() []storage.EngineRangeKeyValue {
return i.i.EngineRangeKeys()
}

// UnsafeEngineKey is part of the storage.EngineIterator interface.
func (i *EngineIterator) UnsafeEngineKey() (storage.EngineKey, error) {
return i.i.UnsafeEngineKey()
Expand Down Expand Up @@ -610,18 +625,33 @@ func (s spanSetWriter) ExperimentalPutMVCCRangeKey(
return s.w.ExperimentalPutMVCCRangeKey(rangeKey, value)
}

func (s spanSetWriter) ExperimentalPutEngineRangeKey(
start, end roachpb.Key, suffix, value []byte,
) error {
if !s.spansOnly {
panic("cannot do timestamp checking for ExperimentalPutEngineRangeKey")
}
if err := s.checkAllowedRange(start, end); err != nil {
return err
}
return s.w.ExperimentalPutEngineRangeKey(start, end, suffix, value)
}

func (s spanSetWriter) ExperimentalClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error {
if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil {
return err
}
return s.w.ExperimentalClearMVCCRangeKey(rangeKey)
}

func (s spanSetWriter) ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) error {
func (s spanSetWriter) ExperimentalClearAllRangeKeys(start, end roachpb.Key) error {
if !s.spansOnly {
panic("cannot do timestamp checking for ExperimentalClearAllRangeKeys")
}
if err := s.checkAllowedRange(start, end); err != nil {
return err
}
return s.w.ExperimentalClearAllMVCCRangeKeys(start, end)
return s.w.ExperimentalClearAllRangeKeys(start, end)
}

func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error {
Expand Down
62 changes: 39 additions & 23 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,20 @@ type EngineIterator interface {
// the iteration. After this call, valid will be true if the iterator was
// not originally positioned at the first key.
PrevEngineKey() (valid bool, err error)
// HasEnginePointAndRange returns whether the iterator is positioned on a
// point or range key.
//
// TODO(erikgrinaker): Consider renaming this HasPointAndRange and merge with
// the SimpleMVCCIterator implementation. However, HasPointAndRange() needs to
// imply Valid() for MVCC iterators, which in turns prevents it from being
// used e.g. across the lock table. Once we revamp the MVCCIterator interface
// we can probably merge these:
// https://github.com/cockroachdb/cockroach/issues/82589
HasEnginePointAndRange() (bool, bool)
// EngineRangeBounds returns the current range key bounds.
EngineRangeBounds() (roachpb.Span, error)
// EngineRangeKeys returns the engine range keys at the current position.
EngineRangeKeys() []EngineRangeKeyValue
// UnsafeEngineKey returns the same value as EngineKey, but the memory is
// invalidated on the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}.
// REQUIRES: latest positioning function returned valid=true.
Expand Down Expand Up @@ -388,12 +402,6 @@ type IterOptions struct {
// iterator position, and RangeBounds() and RangeKeys() to access range keys.
// Defaults to IterKeyTypePointsOnly. For more details on range keys, see
// comment on SimpleMVCCIterator.
//
// Range keys are only supported for use with MVCCIterators. Enabling them
// for EngineIterators will error.
//
// TODO(erikgrinaker): Consider separating the options structs for
// EngineIterator and MVCCIterator.
KeyTypes IterKeyType
// RangeKeyMaskingBelow enables masking (hiding) of point keys by range keys.
// Any range key with a timestamp at or below RangeKeyMaskingBelow
Expand Down Expand Up @@ -537,7 +545,7 @@ type Reader interface {
// NewEngineIterator returns a new instance of an EngineIterator over this
// engine. The caller must invoke EngineIterator.Close() when finished
// with the iterator to free resources. The caller can change IterOptions
// after this function returns. EngineIterators do not support range keys.
// after this function returns.
NewEngineIterator(opts IterOptions) EngineIterator
// ConsistentIterators returns true if the Reader implementation guarantees
// that the different iterators constructed by this Reader will see the same
Expand Down Expand Up @@ -651,35 +659,36 @@ type Writer interface {
// after it returns.
ClearMVCCIteratorRange(start, end roachpb.Key) error

// ExperimentalClearMVCCRangeKey deletes an MVCC range key from start
// (inclusive) to end (exclusive) at the given timestamp. For any range key
// that straddles the start and end boundaries, only the segments within the
// boundaries will be cleared. Range keys at other timestamps are unaffected.
// Clears are idempotent.
// ExperimentalClearAllRangeKeys deletes all range keys (and all versions)
// from start (inclusive) to end (exclusive). This can be used both for MVCC
// range keys or the more general engine range keys. For any range key that
// straddles the start and end boundaries, only the segments within the
// boundaries will be cleared. Clears are idempotent.
//
// This method is primarily intended for MVCC garbage collection and similar
// internal use.
// internal use. It will do an internal scan across the span first to check
// whether it contains any range keys at all, and clear the smallest single
// span that covers all range keys (if any), to avoid dropping Pebble range
// tombstones across unnecessary spans.
//
// This method is EXPERIMENTAL: range keys are under active development, and
// have severe limitations including being ignored by all KV and MVCC APIs and
// only being stored in memory.
ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error
ExperimentalClearAllRangeKeys(start, end roachpb.Key) error

// ExperimentalClearAllMVCCRangeKeys deletes all MVCC range keys (i.e. all
// versions) from start (inclusive) to end (exclusive). For any range key
// ExperimentalClearMVCCRangeKey deletes an MVCC range key from start
// (inclusive) to end (exclusive) at the given timestamp. For any range key
// that straddles the start and end boundaries, only the segments within the
// boundaries will be cleared. Clears are idempotent.
// boundaries will be cleared. Range keys at other timestamps are unaffected.
// Clears are idempotent.
//
// This method is primarily intended for MVCC garbage collection and similar
// internal use. It will do an internal scan across the span first to check
// whether it contains any range keys at all, and clear the smallest single
// span that covers all range keys (if any), to avoid dropping Pebble range
// tombstones across unnecessary spans.
// internal use.
//
// This method is EXPERIMENTAL: range keys are under active development, and
// have severe limitations including being ignored by all KV and MVCC APIs and
// only being stored in memory.
ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) error
ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error

// ExperimentalPutMVCCRangeKey writes an MVCC range key. It will replace any
// overlapping range keys at the given timestamp (even partial overlap). Only
Expand All @@ -697,6 +706,13 @@ type Writer interface {
// only being stored in memory.
ExperimentalPutMVCCRangeKey(MVCCRangeKey, MVCCValue) error

// ExperimentalPutEngineRangeKey sets the given range key to the values
// provided. This is a general-purpose and low-level method that should be
// used sparingly, only when the other Put* methods are not applicable.
//
// It is safe to modify the contents of the arguments after it returns.
ExperimentalPutEngineRangeKey(start, end roachpb.Key, suffix, value []byte) error

// Merge is a high-performance write operation used for values which are
// accumulated over several writes. Multiple values can be merged
// sequentially into a single key; a subsequent read will return a "merged"
Expand Down Expand Up @@ -1211,7 +1227,7 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke
if err != nil {
return err
}
return writer.ExperimentalClearAllMVCCRangeKeys(start, end)
return writer.ExperimentalClearAllRangeKeys(start, end)
}

var ingestDelayL0Threshold = settings.RegisterIntSetting(
Expand Down
8 changes: 8 additions & 0 deletions pkg/storage/engine_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -289,3 +289,11 @@ func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) {
copy(k.Version[1:], lk.TxnUUID)
return k, buf
}

// EngineRangeKeyValue is a raw value for a general range key as stored in the
// engine. It consists of a version (suffix) and corresponding value. The range
// key bounds are not included, but are surfaced via EngineRangeBounds().
type EngineRangeKeyValue struct {
Version []byte
Value []byte
}
7 changes: 7 additions & 0 deletions pkg/storage/engine_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,3 +186,10 @@ func TestEngineKeyValidate(t *testing.T) {
})
}
}

func engineKey(key string, ts int) EngineKey {
return EngineKey{
Key: roachpb.Key(key),
Version: encodeMVCCTimestamp(wallTS(ts)),
}
}
Loading

0 comments on commit c09d7fc

Please sign in to comment.