From 3771e3320fcb79e482ac0339faef109170d6e19c Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Thu, 20 Jul 2023 14:28:23 -0400 Subject: [PATCH 1/2] storage: Update Engine/Reader/Writer interfaces for ScanInternal This change updates pkg/storage interfaces and implementations to allow the use of ScanInternal in skip-shared iteration mode as well as writing/reading of internal point keys, range dels and range keys. Replication / snapshot code will soon rely on these changes to be able to replicate internal keys in higher levels plus metadata of shared sstables in lower levels, as opposed to just observed user keys. Part of #103028 Epic: none Release note: None --- pkg/kv/kvserver/spanset/BUILD.bazel | 1 + pkg/kv/kvserver/spanset/batch.go | 36 +++++++++++++++++ pkg/storage/batch.go | 32 +++++++++++++-- pkg/storage/engine.go | 54 +++++++++++++++++++++++++- pkg/storage/open.go | 5 +++ pkg/storage/pebble.go | 60 ++++++++++++++++++++++++++++- pkg/storage/pebble_batch.go | 40 +++++++++++++++++++ pkg/storage/sst_writer.go | 53 +++++++++++++++++++++++++ 8 files changed, 275 insertions(+), 6 deletions(-) diff --git a/pkg/kv/kvserver/spanset/BUILD.bazel b/pkg/kv/kvserver/spanset/BUILD.bazel index 42c79003d8fa..5afeede69aef 100644 --- a/pkg/kv/kvserver/spanset/BUILD.bazel +++ b/pkg/kv/kvserver/spanset/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_pebble//:pebble", + "@com_github_cockroachdb_pebble//rangekey", ], ) diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 2b49b71c388e..8138cc2ccb1e 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/rangekey" ) // MVCCIterator wraps an storage.MVCCIterator and ensures that it can @@ -445,6 +446,17 @@ type spanSetReader struct { var _ storage.Reader = spanSetReader{} +func (s spanSetReader) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + return s.r.ScanInternal(ctx, lower, upper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile) +} + func (s spanSetReader) Close() { s.r.Close() } @@ -762,6 +774,18 @@ type spanSetBatch struct { var _ storage.Batch = spanSetBatch{} +func (s spanSetBatch) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + // Only used on Engine. + panic("unimplemented") +} + func (s spanSetBatch) Commit(sync bool) error { return s.b.Commit(sync) } @@ -794,6 +818,18 @@ func (s spanSetBatch) CommitStats() storage.BatchCommitStats { return s.b.CommitStats() } +func (s spanSetBatch) PutInternalRangeKey(start, end []byte, key rangekey.Key) error { + return s.b.PutInternalRangeKey(start, end, key) +} + +func (s spanSetBatch) PutInternalPointKey(key *pebble.InternalKey, value []byte) error { + return s.b.PutInternalPointKey(key, value) +} + +func (s spanSetBatch) ClearRawEncodedRange(start, end []byte) error { + return s.b.ClearRawEncodedRange(start, end) +} + // NewBatch returns a storage.Batch that asserts access of the underlying // Batch against the given SpanSet. We only consider span boundaries, associated // timestamps are not considered. diff --git a/pkg/storage/batch.go b/pkg/storage/batch.go index 40a38fe6b2d4..35a8281d70ca 100644 --- a/pkg/storage/batch.go +++ b/pkg/storage/batch.go @@ -150,8 +150,8 @@ func (r *BatchReader) Value() []byte { } } -// EngineEndKey returns the engine end key of the current ranged batch entry. -func (r *BatchReader) EngineEndKey() (EngineKey, error) { +// EndKey returns the raw end key of the current ranged batch entry. +func (r *BatchReader) EndKey() ([]byte, error) { var rawKey []byte switch r.kind { case pebble.InternalKeyKindRangeDelete: @@ -160,14 +160,23 @@ func (r *BatchReader) EngineEndKey() (EngineKey, error) { case pebble.InternalKeyKindRangeKeySet, pebble.InternalKeyKindRangeKeyUnset, pebble.InternalKeyKindRangeKeyDelete: rangeKeys, err := r.rangeKeys() if err != nil { - return EngineKey{}, err + return nil, err } rawKey = rangeKeys.End default: - return EngineKey{}, errors.AssertionFailedf( + return nil, errors.AssertionFailedf( "can only ask for EndKey on a ranged entry, got %v", r.kind) } + return rawKey, nil +} + +// EngineEndKey returns the engine end key of the current ranged batch entry. +func (r *BatchReader) EngineEndKey() (EngineKey, error) { + rawKey, err := r.EndKey() + if err != nil { + return EngineKey{}, err + } key, ok := DecodeEngineKey(rawKey) if !ok { @@ -176,6 +185,21 @@ func (r *BatchReader) EngineEndKey() (EngineKey, error) { return key, nil } +// RawRangeKeys returns the raw range key values at the current entry. +func (r *BatchReader) RawRangeKeys() ([]rangekey.Key, error) { + switch r.kind { + case pebble.InternalKeyKindRangeKeySet, pebble.InternalKeyKindRangeKeyUnset: + default: + return nil, errors.AssertionFailedf( + "can only ask for range keys on a range key entry, got %v", r.kind) + } + rangeKeys, err := r.rangeKeys() + if err != nil { + return nil, err + } + return rangeKeys.Keys, nil +} + // EngineRangeKeys returns the engine range key values at the current entry. func (r *BatchReader) EngineRangeKeys() ([]EngineRangeKeyValue, error) { switch r.kind { diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index be393d0e1355..97056747c635 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/rangekey" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/redact" prometheusgo "github.com/prometheus/client_model/go" @@ -598,6 +599,26 @@ type Reader interface { // with the iterator to free resources. The caller can change IterOptions // after this function returns. NewEngineIterator(opts IterOptions) EngineIterator + // ScanInternal allows a caller to inspect the underlying engine's InternalKeys + // using a visitor pattern, while also allowing for keys in shared files to be + // skipped if a visitor is provided for visitSharedFiles. Useful for + // fast-replicating state from one Reader to another. Point keys are collapsed + // such that only one internal key per user key is exposed, and rangedels and + // range keys are collapsed and defragmented with each span being surfaced + // exactly once, alongside the highest seqnum for a rangedel on that span + // (for rangedels) or all coalesced rangekey.Keys in that span (for range + // keys). A point key deleted by a rangedel will not be exposed, but the + // rangedel would be exposed. + // + // Note that ScanInternal does not obey the guarantees indicated by + // ConsistentIterators. + ScanInternal( + ctx context.Context, lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start, end []byte, seqNum uint64) error, + visitRangeKey func(start, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, + ) error // ConsistentIterators returns true if the Reader implementation guarantees // that the different iterators constructed by this Reader will see the same // underlying Engine state. This is not true about Batch writes: new iterators @@ -854,6 +875,32 @@ type Writer interface { BufferedSize() int } +// InternalWriter is an extension of Writer that supports additional low-level +// methods to operate on internal keys in Pebble. These additional methods +// should only be used sparingly, when one of the high-level methods cannot +// achieve the same ends. +type InternalWriter interface { + Writer + // ClearRawEncodedRange is similar to ClearRawRange, except it takes pre-encoded + // start, end keys and bypasses the EngineKey encoding step. It also only + // operates on point keys; for range keys, use ClearEngineRangeKey or + // PutInternalRangeKey. + // + // It is safe to modify the contents of the arguments after it returns. + ClearRawEncodedRange(start, end []byte) error + + // PutInternalRangeKey adds an InternalRangeKey to this batch. This is a very + // low-level method that should be used sparingly. + // + // It is safe to modify the contents of the arguments after it returns. + PutInternalRangeKey(start, end []byte, key rangekey.Key) error + // PutInternalPointKey adds a point InternalKey to this batch. This is a very + // low-level method that should be used sparingly. + // + // It is safe to modify the contents of the arguments after it returns. + PutInternalPointKey(key *pebble.InternalKey, value []byte) error +} + // ClearOptions holds optional parameters to methods that clear keys from the // storage engine. type ClearOptions struct { @@ -984,6 +1031,11 @@ type Engine interface { // additionally returns ingestion stats. IngestExternalFilesWithStats( ctx context.Context, paths []string) (pebble.IngestOperationStats, error) + // IngestAndExciseExternalFiles is a variant of IngestExternalFilesWithStats + // that excises an ExciseSpan, and ingests either local or shared sstables or + // both. + IngestAndExciseExternalFiles( + ctx context.Context, paths []string, shared []pebble.SharedSSTMeta, exciseSpan roachpb.Span) (pebble.IngestOperationStats, error) // PreIngestDelay offers an engine the chance to backpressure ingestions. // When called, it may choose to block if the engine determines that it is in // or approaching a state where further ingestions may risk its health. @@ -1048,7 +1100,7 @@ type Batch interface { // WriteBatch is the interface for write batch specific operations. type WriteBatch interface { - Writer + InternalWriter // Close closes the batch, freeing up any outstanding resources. Close() // Commit atomically applies any batched updates to the underlying engine. If diff --git a/pkg/storage/open.go b/pkg/storage/open.go index e6083b69cf55..a329be6c6622 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -181,6 +181,11 @@ func BallastSize(size int64) ConfigOption { func SharedStorage(sharedStorage cloud.ExternalStorage) ConfigOption { return func(cfg *engineConfig) error { cfg.SharedStorage = sharedStorage + // TODO(bilal): Do the format major version ratchet while accounting for + // version upgrade finalization. However, seeing as shared storage is + // an experimental feature and upgrading from existing stores is not + // supported, this is fine. + cfg.Opts.FormatMajorVersion = pebble.ExperimentalFormatVirtualSSTables return nil } } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 63705f2264d1..2ab380055ba3 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1025,7 +1025,13 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { ValueBlocksEnabled.Get(&cfg.Settings.SV) } opts.Experimental.DisableIngestAsFlushable = func() bool { - return !IngestAsFlushable.Get(&cfg.Settings.SV) + // Disable flushable ingests if shared storage is enabled. This is because + // flushable ingests currently do not support Excise operations. + // + // TODO(bilal): Remove the first part of this || statement when + // https://github.com/cockroachdb/pebble/issues/2676 is completed, or when + // Pebble has better guards against this. + return cfg.SharedStorage != nil || !IngestAsFlushable.Get(&cfg.Settings.SV) } auxDir := opts.FS.PathJoin(cfg.Dir, base.AuxiliaryDir) @@ -1475,6 +1481,20 @@ func (p *Pebble) NewEngineIterator(opts IterOptions) EngineIterator { return newPebbleIterator(p.db, opts, StandardDurability, p) } +// ScanInternal implements the Engine interface. +func (p *Pebble) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + rawLower := EngineKey{Key: lower}.Encode() + rawUpper := EngineKey{Key: upper}.Encode() + return p.db.ScanInternal(ctx, rawLower, rawUpper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile) +} + // ConsistentIterators implements the Engine interface. func (p *Pebble) ConsistentIterators() bool { return false @@ -2033,6 +2053,17 @@ func (p *Pebble) IngestExternalFilesWithStats( return p.db.IngestWithStats(paths) } +// IngestAndExciseExternalFiles implements the Engine interface. +func (p *Pebble) IngestAndExciseExternalFiles( + ctx context.Context, paths []string, shared []pebble.SharedSSTMeta, exciseSpan roachpb.Span, +) (pebble.IngestOperationStats, error) { + rawSpan := pebble.KeyRange{ + Start: EngineKey{Key: exciseSpan.Key}.Encode(), + End: EngineKey{Key: exciseSpan.EndKey}.Encode(), + } + return p.db.IngestAndExcise(paths, shared, rawSpan) +} + // PreIngestDelay implements the Engine interface. func (p *Pebble) PreIngestDelay(ctx context.Context) { preIngestDelay(ctx, p, p.settings) @@ -2444,10 +2475,23 @@ func (p *pebbleReadOnly) PinEngineStateForIterators() error { return nil } +// ScanInternal implements the Reader interface. +func (p *pebbleReadOnly) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + return p.parent.ScanInternal(ctx, lower, upper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile) +} + // Writer methods are not implemented for pebbleReadOnly. Ideally, the code // could be refactored so that a Reader could be supplied to evaluateBatch // Writer is the write interface to an engine's data. + func (p *pebbleReadOnly) ApplyBatchRepr(repr []byte, sync bool) error { panic("not implemented") } @@ -2621,6 +2665,20 @@ func (p *pebbleSnapshot) PinEngineStateForIterators() error { return nil } +// ScanInternal implements the Reader interface. +func (p *pebbleSnapshot) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + rawLower := EngineKey{Key: lower}.Encode() + rawUpper := EngineKey{Key: upper}.Encode() + return p.snapshot.ScanInternal(ctx, rawLower, rawUpper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile) +} + // ExceedMaxSizeError is the error returned when an export request // fails due the export size exceeding the budget. This can be caused // by large KVs that have many revisions. diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 36d45def93c1..3dfc6022172d 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/rangekey" ) // Wrapper struct around a pebble.Batch. @@ -257,6 +258,23 @@ func (p *pebbleBatch) NewEngineIterator(opts IterOptions) EngineIterator { return iter } +// ScanInternal implements the Reader interface. +func (p *pebbleBatch) ScanInternal( + ctx context.Context, + lower, upper roachpb.Key, + visitPointKey func(key *pebble.InternalKey, value pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start []byte, end []byte, seqNum uint64) error, + visitRangeKey func(start []byte, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + panic("ScanInternal only supported on Engine and Snapshot.") +} + +// ClearRawEncodedRange implements the InternalWriter interface. +func (p *pebbleBatch) ClearRawEncodedRange(start, end []byte) error { + return p.batch.DeleteRange(start, end, pebble.Sync) +} + // ConsistentIterators implements the Batch interface. func (p *pebbleBatch) ConsistentIterators() bool { return true @@ -482,6 +500,20 @@ func (p *pebbleBatch) PutEngineRangeKey(start, end roachpb.Key, suffix, value [] EngineKey{Key: start}.Encode(), EngineKey{Key: end}.Encode(), suffix, value, nil) } +// PutInternalRangeKey implements the InternalWriter interface. +func (p *pebbleBatch) PutInternalRangeKey(start, end []byte, key rangekey.Key) error { + switch key.Kind() { + case pebble.InternalKeyKindRangeKeyUnset: + return p.batch.RangeKeyUnset(start, end, key.Suffix, nil /* writeOptions */) + case pebble.InternalKeyKindRangeKeySet: + return p.batch.RangeKeySet(start, end, key.Suffix, key.Value, nil /* writeOptions */) + case pebble.InternalKeyKindRangeKeyDelete: + return p.batch.RangeKeyDelete(start, end, nil /* writeOptions */) + default: + panic("unexpected range key kind") + } +} + // ClearEngineRangeKey implements the Engine interface. func (p *pebbleBatch) ClearEngineRangeKey(start, end roachpb.Key, suffix []byte) error { return p.batch.RangeKeyUnset( @@ -542,6 +574,14 @@ func (p *pebbleBatch) PutEngineKey(key EngineKey, value []byte) error { return p.batch.Set(p.buf, value, nil) } +// PutInternalPointKey implements the WriteBatch interface. +func (p *pebbleBatch) PutInternalPointKey(key *pebble.InternalKey, value []byte) error { + if len(key.UserKey) == 0 { + return emptyKeyError() + } + return p.batch.AddInternalKey(key, value, nil /* writeOptions */) +} + func (p *pebbleBatch) put(key MVCCKey, value []byte) error { if len(key.Key) == 0 { return emptyKeyError() diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 2def95a54895..01538caeafa5 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -20,7 +20,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/objstorage" + "github.com/cockroachdb/pebble/rangekey" "github.com/cockroachdb/pebble/sstable" ) @@ -37,6 +39,7 @@ type SSTWriter struct { var _ Writer = &SSTWriter{} var _ ExportWriter = &SSTWriter{} +var _ InternalWriter = &SSTWriter{} // noopFinishAbort is used to wrap io.Writers for sstable.Writer. type noopFinishAbort struct { @@ -228,6 +231,56 @@ func (fw *SSTWriter) ClearEngineRangeKey(start, end roachpb.Key, suffix []byte) return fw.fw.RangeKeyUnset(EngineKey{Key: start}.Encode(), EngineKey{Key: end}.Encode(), suffix) } +// ClearRawEncodedRange implements the InternalWriter interface. +func (fw *SSTWriter) ClearRawEncodedRange(start, end []byte) error { + startEngine, ok := DecodeEngineKey(start) + if !ok { + return errors.New("cannot decode start engine key") + } + endEngine, ok := DecodeEngineKey(end) + if !ok { + return errors.New("cannot decode end engine key") + } + fw.DataSize += int64(len(startEngine.Key)) + int64(len(endEngine.Key)) + return fw.fw.DeleteRange(start, end) +} + +// PutInternalRangeKey implements the InternalWriter interface. +func (fw *SSTWriter) PutInternalRangeKey(start, end []byte, key rangekey.Key) error { + if !fw.supportsRangeKeys { + return errors.New("range keys not supported by SST writer") + } + startEngine, ok := DecodeEngineKey(start) + if !ok { + return errors.New("cannot decode engine key") + } + endEngine, ok := DecodeEngineKey(end) + if !ok { + return errors.New("cannot decode engine key") + } + fw.DataSize += int64(len(startEngine.Key)) + int64(len(endEngine.Key)) + int64(len(key.Value)) + switch key.Kind() { + case pebble.InternalKeyKindRangeKeyUnset: + return fw.fw.RangeKeyUnset(start, end, key.Suffix) + case pebble.InternalKeyKindRangeKeySet: + return fw.fw.RangeKeySet(start, end, key.Suffix, key.Value) + case pebble.InternalKeyKindRangeKeyDelete: + return fw.fw.RangeKeyDelete(start, end) + default: + panic("unexpected range key kind") + } +} + +// PutInternalPointKey implements the InternalWriter interface. +func (fw *SSTWriter) PutInternalPointKey(key *pebble.InternalKey, value []byte) error { + ek, ok := DecodeEngineKey(key.UserKey) + if !ok { + return errors.New("cannot decode engine key") + } + fw.DataSize += int64(len(ek.Key)) + int64(len(value)) + return fw.fw.Add(*key, value) +} + // clearRange clears all point keys in the given range by dropping a Pebble // range tombstone. // From e714d52bddc9606fbac8117955af8de7db4d4a29 Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Wed, 19 Jul 2023 13:04:53 -0400 Subject: [PATCH 2/2] kvserver: Add ability to filter replicated spans in Select/Iterate This change adds the ability to select for just the replicated span in rditer.Select and rditer.IterateReplicaKeySpans. Also adds a new rditer.IterateReplicaKeySpansShared that does a ScanInternal on just the user key span, to be able to collect metadata of shared sstables as well as any internal keys above them. We only use skip-shared iteration for the replicated user key span of a range, and in practice, only if it's a non-system range. Part of #103028. Epic: none Release note: None --- pkg/cli/debug.go | 2 +- pkg/kv/kvserver/client_merge_test.go | 2 +- pkg/kv/kvserver/rditer/BUILD.bazel | 2 + pkg/kv/kvserver/rditer/replica_data_iter.go | 47 ++++++++++++++- .../kvserver/rditer/replica_data_iter_test.go | 51 +++++++++++----- pkg/kv/kvserver/rditer/select.go | 59 +++++++++++++------ pkg/kv/kvserver/rditer/select_test.go | 23 +++++++- .../r1/all/{ => all}/output | 0 .../r1/all/exclude-user/output | 24 ++++++++ .../r1/all/user-only/output | 21 +++++++ .../r1/replicatedOnly/{ => all}/output | 0 .../r1/replicatedOnly/exclude-user/output | 18 ++++++ .../r1/replicatedOnly/user-only/output | 15 +++++ .../r2/all/{ => all}/output | 0 .../r2/all/exclude-user/output | 24 ++++++++ .../r2/all/user-only/output | 21 +++++++ .../r2/replicatedOnly/{ => all}/output | 0 .../r2/replicatedOnly/exclude-user/output | 18 ++++++ .../r2/replicatedOnly/user-only/output | 15 +++++ .../r3/all/{ => all}/output | 0 .../r3/all/exclude-user/output | 24 ++++++++ .../r3/all/user-only/output | 21 +++++++ .../r3/replicatedOnly/{ => all}/output | 0 .../r3/replicatedOnly/exclude-user/output | 18 ++++++ .../r3/replicatedOnly/user-only/output | 15 +++++ .../rditer/testdata/TestSelect/no_span | 8 +-- pkg/kv/kvserver/rditer/testdata/TestSelect/r1 | 8 +-- pkg/kv/kvserver/rditer/testdata/TestSelect/r2 | 8 +-- .../rditer/testdata/TestSelect/r2_excludeuser | 22 +++++++ .../rditer/testdata/TestSelect/r2_useronly | 14 +++++ pkg/kv/kvserver/rditer/testdata/TestSelect/r3 | 8 +-- pkg/kv/kvserver/replica_learner_test.go | 2 +- pkg/kv/kvserver/replica_test.go | 2 +- pkg/kv/kvserver/store_snapshot.go | 2 +- 34 files changed, 434 insertions(+), 60 deletions(-) rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/user-only/output rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/user-only/output rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/user-only/output rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/user-only/output rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/user-only/output rename pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/{ => all}/output (100%) create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/exclude-user/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/user-only/output create mode 100644 pkg/kv/kvserver/rditer/testdata/TestSelect/r2_excludeuser create mode 100644 pkg/kv/kvserver/rditer/testdata/TestSelect/r2_useronly diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 864f59f8716d..ead75e464a67 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -472,7 +472,7 @@ func runDebugRangeData(cmd *cobra.Command, args []string) error { defer snapshot.Close() var results int - return rditer.IterateReplicaKeySpans(&desc, snapshot, debugCtx.replicated, + return rditer.IterateReplicaKeySpans(&desc, snapshot, debugCtx.replicated, rditer.ReplicatedSpansAll, func(iter storage.EngineIterator, _ roachpb.Span, keyType storage.IterKeyType) error { for ok := true; ok && err == nil; ok, err = iter.NextEngineKey() { switch keyType { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 85f9044aad33..37a0615fe618 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -3812,7 +3812,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { } } - err := rditer.IterateReplicaKeySpans(inSnap.Desc, sendingEngSnapshot, true, /* replicatedOnly */ + err := rditer.IterateReplicaKeySpans(inSnap.Desc, sendingEngSnapshot, true /* replicatedOnly */, rditer.ReplicatedSpansAll, func(iter storage.EngineIterator, span roachpb.Span, keyType storage.IterKeyType) error { fw, ok := sstFileWriters[string(span.Key)] if !ok || !fw.span.Equal(span) { diff --git a/pkg/kv/kvserver/rditer/BUILD.bazel b/pkg/kv/kvserver/rditer/BUILD.bazel index 4b75c8494b8d..2e2277b48599 100644 --- a/pkg/kv/kvserver/rditer/BUILD.bazel +++ b/pkg/kv/kvserver/rditer/BUILD.bazel @@ -15,6 +15,8 @@ go_library( "//pkg/storage", "//pkg/storage/enginepb", "//pkg/util/iterutil", + "@com_github_cockroachdb_pebble//:pebble", + "@com_github_cockroachdb_pebble//rangekey", ], ) diff --git a/pkg/kv/kvserver/rditer/replica_data_iter.go b/pkg/kv/kvserver/rditer/replica_data_iter.go index 926b31658189..6bf86136e838 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter.go @@ -11,10 +11,14 @@ package rditer import ( + "context" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/rangekey" ) // ReplicaDataIteratorOptions defines ReplicaMVCCDataIterator creation options. @@ -315,6 +319,7 @@ func IterateReplicaKeySpans( desc *roachpb.RangeDescriptor, reader storage.Reader, replicatedOnly bool, + replicatedSpansFilter ReplicatedSpansFilter, visitor func(storage.EngineIterator, roachpb.Span, storage.IterKeyType) error, ) error { if !reader.ConsistentIterators() { @@ -322,9 +327,18 @@ func IterateReplicaKeySpans( } var spans []roachpb.Span if replicatedOnly { - spans = MakeReplicatedKeySpans(desc) + spans = Select(desc.RangeID, SelectOpts{ + ReplicatedSpansFilter: replicatedSpansFilter, + ReplicatedBySpan: desc.RSpan(), + ReplicatedByRangeID: true, + }) } else { - spans = makeAllKeySpans(desc) + spans = Select(desc.RangeID, SelectOpts{ + ReplicatedBySpan: desc.RSpan(), + ReplicatedSpansFilter: replicatedSpansFilter, + ReplicatedByRangeID: true, + UnreplicatedByRangeID: true, + }) } keyTypes := []storage.IterKeyType{storage.IterKeyTypePointsOnly, storage.IterKeyTypeRangesOnly} for _, span := range spans { @@ -350,6 +364,35 @@ func IterateReplicaKeySpans( return nil } +// IterateReplicaKeySpansShared iterates over the range's user key span, +// skipping any keys present in shared files. It calls the appropriate visitor +// function for the type of key visited, namely, point keys, range deletes and +// range keys. Shared files that are skipped during this iteration are also +// surfaced through a dedicated visitor. Note that this method only iterates +// over a range's user key span; IterateReplicaKeySpans must be called to +// iterate over the other key spans. +// +// Must use a reader with consistent iterators. +func IterateReplicaKeySpansShared( + ctx context.Context, + desc *roachpb.RangeDescriptor, + reader storage.Reader, + visitPoint func(key *pebble.InternalKey, val pebble.LazyValue, info pebble.IteratorLevel) error, + visitRangeDel func(start, end []byte, seqNum uint64) error, + visitRangeKey func(start, end []byte, keys []rangekey.Key) error, + visitSharedFile func(sst *pebble.SharedSSTMeta) error, +) error { + if !reader.ConsistentIterators() { + panic("reader must provide consistent iterators") + } + spans := Select(desc.RangeID, SelectOpts{ + ReplicatedSpansFilter: ReplicatedSpansUserOnly, + ReplicatedBySpan: desc.RSpan(), + }) + span := spans[0] + return reader.ScanInternal(ctx, span.Key, span.EndKey, visitPoint, visitRangeDel, visitRangeKey, visitSharedFile) +} + // IterateOptions instructs how points and ranges should be presented to visitor // and if iterators should be visited in forward or reverse order. // Reverse iterator are also positioned at the end of the range prior to being diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index d533b8ea564e..87a4964fe76a 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -148,6 +148,7 @@ func verifyIterateReplicaKeySpans( desc *roachpb.RangeDescriptor, eng storage.Engine, replicatedOnly bool, + replicatedSpansFilter ReplicatedSpansFilter, ) { readWriter := eng.NewSnapshot() defer readWriter.Close() @@ -161,7 +162,7 @@ func verifyIterateReplicaKeySpans( "pretty", }) - require.NoError(t, IterateReplicaKeySpans(desc, readWriter, replicatedOnly, + require.NoError(t, IterateReplicaKeySpans(desc, readWriter, replicatedOnly, replicatedSpansFilter, func(iter storage.EngineIterator, span roachpb.Span, keyType storage.IterKeyType) error { var err error for ok := true; ok && err == nil; ok, err = iter.NextEngineKey() { @@ -180,12 +181,18 @@ func verifyIterateReplicaKeySpans( var err error mvccKey, err = key.ToMVCCKey() require.NoError(t, err) + if replicatedSpansFilter == ReplicatedSpansExcludeUser && desc.KeySpan().AsRawSpanWithNoLocals().ContainsKey(key.Key) { + t.Fatalf("unexpected user key when user key are expected to be skipped: %s", mvccKey) + } } else { // lock key ltk, err := key.ToLockTableKey() require.NoError(t, err) mvccKey = storage.MVCCKey{ Key: ltk.Key, } + if replicatedSpansFilter == ReplicatedSpansUserOnly { + t.Fatalf("unexpected lock table key when only table keys requested: %s", ltk.Key) + } } tbl.Append([]string{ span.String(), @@ -271,21 +278,33 @@ func TestReplicaDataIterator(t *testing.T) { parName := fmt.Sprintf("r%d", tc.desc.RangeID) t.Run(parName, func(t *testing.T) { testutils.RunTrueAndFalse(t, "replicatedOnly", func(t *testing.T, replicatedOnly bool) { - name := "all" - if replicatedOnly { - name = "replicatedOnly" - } - w := echotest.NewWalker(t, filepath.Join(path, parName, name)) + replicatedSpans := []ReplicatedSpansFilter{ReplicatedSpansAll, ReplicatedSpansExcludeUser, ReplicatedSpansUserOnly} + for i := range replicatedSpans { + replicatedKeysName := "all" + switch replicatedSpans[i] { + case ReplicatedSpansExcludeUser: + replicatedKeysName = "exclude-user" + case ReplicatedSpansUserOnly: + replicatedKeysName = "user-only" + } + t.Run(fmt.Sprintf("replicatedSpans=%v", replicatedKeysName), func(t *testing.T) { + name := "all" + if replicatedOnly { + name = "replicatedOnly" + } + w := echotest.NewWalker(t, filepath.Join(path, parName, name, replicatedKeysName)) - w.Run(t, "output", func(t *testing.T) string { - var innerBuf strings.Builder - tbl := tablewriter.NewWriter(&innerBuf) - // Print contents of the Replica according to the iterator. - verifyIterateReplicaKeySpans(t, tbl, &tc.desc, eng, replicatedOnly) + w.Run(t, "output", func(t *testing.T) string { + var innerBuf strings.Builder + tbl := tablewriter.NewWriter(&innerBuf) + // Print contents of the Replica according to the iterator. + verifyIterateReplicaKeySpans(t, tbl, &tc.desc, eng, replicatedOnly, replicatedSpans[i]) - tbl.Render() - return innerBuf.String() - })(t) + tbl.Render() + return innerBuf.String() + })(t) + }) + } }) }) } @@ -449,7 +468,7 @@ func TestReplicaDataIteratorGlobalRangeKey(t *testing.T) { } var actualSpans []roachpb.Span - require.NoError(t, IterateReplicaKeySpans(&desc, snapshot, replicatedOnly, + require.NoError(t, IterateReplicaKeySpans(&desc, snapshot, replicatedOnly, ReplicatedSpansAll, func(iter storage.EngineIterator, span roachpb.Span, keyType storage.IterKeyType) error { // We should never see any point keys. require.Equal(t, storage.IterKeyTypeRangesOnly, keyType) @@ -556,7 +575,7 @@ func benchReplicaEngineDataIterator(b *testing.B, numRanges, numKeysPerRange, va for i := 0; i < b.N; i++ { for _, desc := range descs { - err := IterateReplicaKeySpans(&desc, snapshot, false, /* replicatedOnly */ + err := IterateReplicaKeySpans(&desc, snapshot, false /* replicatedOnly */, ReplicatedSpansAll, func(iter storage.EngineIterator, _ roachpb.Span, _ storage.IterKeyType) error { var err error for ok := true; ok && err == nil; ok, err = iter.NextEngineKey() { diff --git a/pkg/kv/kvserver/rditer/select.go b/pkg/kv/kvserver/rditer/select.go index 87f2f310f944..bf32356ab981 100644 --- a/pkg/kv/kvserver/rditer/select.go +++ b/pkg/kv/kvserver/rditer/select.go @@ -15,6 +15,19 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" ) +type ReplicatedSpansFilter int + +const ( + // ReplicatedSpansAll includes all replicated spans, including user keys, + // range descriptors, and lock keys. + ReplicatedSpansAll ReplicatedSpansFilter = iota + // ReplicatedSpansExcludeUser includes all replicated spans except for user keys. + ReplicatedSpansExcludeUser + // ReplicatedSpansUserOnly includes just user keys, and no other replicated + // spans. + ReplicatedSpansUserOnly +) + // SelectOpts configures which spans for a Replica to return from Select. // A Replica comprises replicated (i.e. belonging to the state machine) spans // and unreplicated spans, and depending on external circumstances one may want @@ -24,6 +37,10 @@ type SelectOpts struct { // key. This includes user keys, range descriptors, and locks (separated // intents). ReplicatedBySpan roachpb.RSpan + // ReplicatedSpansFilter specifies which of the replicated spans indicated by + // ReplicatedBySpan should be returned or excluded. The zero value, + // ReplicatedSpansAll, returns all replicated spans. + ReplicatedSpansFilter ReplicatedSpansFilter // ReplicatedByRangeID selects all RangeID-keyed replicated keys. An example // of a key that falls into this Span is the GCThresholdKey. ReplicatedByRangeID bool @@ -60,27 +77,31 @@ func Select(rangeID roachpb.RangeID, opts SelectOpts) []roachpb.Span { // See also the comment on KeySpan. in := opts.ReplicatedBySpan adjustedIn := in.KeySpan() - sl = append(sl, makeRangeLocalKeySpan(in)) + if opts.ReplicatedSpansFilter != ReplicatedSpansUserOnly { + sl = append(sl, makeRangeLocalKeySpan(in)) - // Lock table. - { - // Handle doubly-local lock table keys since range descriptor key - // is a range local key that can have a replicated lock acquired on it. - startRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(in.Key), nil) - endRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(in.EndKey), nil) - // Need adjusted start key to avoid overlapping with the local lock span right above. - startGlobal, _ := keys.LockTableSingleKey(adjustedIn.Key.AsRawKey(), nil) - endGlobal, _ := keys.LockTableSingleKey(adjustedIn.EndKey.AsRawKey(), nil) - sl = append(sl, roachpb.Span{ - Key: startRangeLocal, - EndKey: endRangeLocal, - }, roachpb.Span{ - Key: startGlobal, - EndKey: endGlobal, - }) + // Lock table. + { + // Handle doubly-local lock table keys since range descriptor key + // is a range local key that can have a replicated lock acquired on it. + startRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(in.Key), nil) + endRangeLocal, _ := keys.LockTableSingleKey(keys.MakeRangeKeyPrefix(in.EndKey), nil) + // Need adjusted start key to avoid overlapping with the local lock span right above. + startGlobal, _ := keys.LockTableSingleKey(adjustedIn.Key.AsRawKey(), nil) + endGlobal, _ := keys.LockTableSingleKey(adjustedIn.EndKey.AsRawKey(), nil) + sl = append(sl, roachpb.Span{ + Key: startRangeLocal, + EndKey: endRangeLocal, + }, roachpb.Span{ + Key: startGlobal, + EndKey: endGlobal, + }) + } + } + if opts.ReplicatedSpansFilter != ReplicatedSpansExcludeUser { + // Adjusted span because r1's "normal" keyspace starts only at LocalMax, not RKeyMin. + sl = append(sl, adjustedIn.AsRawSpanWithNoLocals()) } - // Adjusted span because r1's "normal" keyspace starts only at LocalMax, not RKeyMin. - sl = append(sl, adjustedIn.AsRawSpanWithNoLocals()) } return sl } diff --git a/pkg/kv/kvserver/rditer/select_test.go b/pkg/kv/kvserver/rditer/select_test.go index 4eb08942e3a0..c47a7bddcb80 100644 --- a/pkg/kv/kvserver/rditer/select_test.go +++ b/pkg/kv/kvserver/rditer/select_test.go @@ -29,8 +29,9 @@ func TestSelect(t *testing.T) { w := echotest.NewWalker(t, datapathutils.TestDataPath(t, t.Name())) for _, tc := range []struct { - name string - sp roachpb.RSpan + name string + sp roachpb.RSpan + filter ReplicatedSpansFilter }{ { name: "no_span", @@ -49,6 +50,23 @@ func TestSelect(t *testing.T) { Key: roachpb.RKey("a"), EndKey: roachpb.RKey("c"), }, + filter: ReplicatedSpansAll, + }, + { + name: "r2_excludeuser", + sp: roachpb.RSpan{ + Key: roachpb.RKey("a"), + EndKey: roachpb.RKey("c"), + }, + filter: ReplicatedSpansExcludeUser, + }, + { + name: "r2_useronly", + sp: roachpb.RSpan{ + Key: roachpb.RKey("a"), + EndKey: roachpb.RKey("c"), + }, + filter: ReplicatedSpansUserOnly, }, { name: "r3", @@ -64,6 +82,7 @@ func TestSelect(t *testing.T) { for _, unreplicatedByRangeID := range []bool{false, true} { opts := SelectOpts{ ReplicatedBySpan: tc.sp, + ReplicatedSpansFilter: tc.filter, ReplicatedByRangeID: replicatedByRangeID, UnreplicatedByRangeID: unreplicatedByRangeID, } diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/exclude-user/output new file mode 100644 index 000000000000..e5d07cf7e213 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/exclude-user/output @@ -0,0 +1,24 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/1/{r""-s""} | 016989726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/1/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/1/{r""-s""} | 016989726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/1/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/1/{r""-s""} | 016989726c67632d | | | /Local/RangeID/1/r/RangeGCThreshold | +| /Local/RangeID/1/{r""-s""} | 016989727261736b | | | /Local/RangeID/1/r/RangeAppliedState | +| /Local/RangeID/1/{r""-s""} | 01698972726c6c2d | | | /Local/RangeID/1/r/RangeLease | +| /Local/RangeID/1/{r""-s""} | 016989723a61 | 016989723a78 | 000000000000000109 | /Local/RangeID/1/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/1/{u""-v""} | 0169897572667462 | | | /Local/RangeID/1/u/RangeTombstone | +| /Local/RangeID/1/{u""-v""} | 0169897572667468 | | | /Local/RangeID/1/u/RaftHardState | +| /Local/RangeID/1/{u""-v""} | 016989757266746c0000000000000001 | | | /Local/RangeID/1/u/RaftLog/logIndex:1 | +| /Local/RangeID/1/{u""-v""} | 016989757266746c0000000000000002 | | | /Local/RangeID/1/u/RaftLog/logIndex:2 | +| /Local/RangeID/1/{u""-v""} | 01698975726c7274 | | | /Local/RangeID/1/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/1/{u""-v""} | 016989753a61 | 016989753a78 | 000000000000000109 | /Local/RangeID/1/u":{a"-x"}/0.000000001,0 | +| /Local/Range"{a"-b"} | 016b1261000172647363 | | 0000000000000001 | /Local/Range"a"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{a"-b"} | 016b1261000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"a"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{a"-b"} | 016b126100ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"a\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{a"-b"} | 016b1261ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"a\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{a"-b"} | 017a6b12016b126100ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"a"/RangeDescriptor | +| /Local/Lock/Intent"{a"-b"} | 017a6b12610001 | | 030ce61c175eb445878c36dcf4062ada4c | "a" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/user-only/output new file mode 100644 index 000000000000..21d8f0604597 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/all/user-only/output @@ -0,0 +1,21 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/1/{r""-s""} | 016989726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/1/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/1/{r""-s""} | 016989726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/1/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/1/{r""-s""} | 016989726c67632d | | | /Local/RangeID/1/r/RangeGCThreshold | +| /Local/RangeID/1/{r""-s""} | 016989727261736b | | | /Local/RangeID/1/r/RangeAppliedState | +| /Local/RangeID/1/{r""-s""} | 01698972726c6c2d | | | /Local/RangeID/1/r/RangeLease | +| /Local/RangeID/1/{r""-s""} | 016989723a61 | 016989723a78 | 000000000000000109 | /Local/RangeID/1/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/1/{u""-v""} | 0169897572667462 | | | /Local/RangeID/1/u/RangeTombstone | +| /Local/RangeID/1/{u""-v""} | 0169897572667468 | | | /Local/RangeID/1/u/RaftHardState | +| /Local/RangeID/1/{u""-v""} | 016989757266746c0000000000000001 | | | /Local/RangeID/1/u/RaftLog/logIndex:1 | +| /Local/RangeID/1/{u""-v""} | 016989757266746c0000000000000002 | | | /Local/RangeID/1/u/RaftLog/logIndex:2 | +| /Local/RangeID/1/{u""-v""} | 01698975726c7274 | | | /Local/RangeID/1/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/1/{u""-v""} | 016989753a61 | 016989753a78 | 000000000000000109 | /Local/RangeID/1/u":{a"-x"}/0.000000001,0 | +| {a-b} | 61 | | 0000000000000001 | "a"/0.000000001,0 | +| {a-b} | 61ffffffff | | 0000000000000001 | "a\xff\xff\xff\xff"/0.000000001,0 | +| {a-b} | 61 | 62 | 000000000000000109 | {a-b}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/exclude-user/output new file mode 100644 index 000000000000..78b748b3379b --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/exclude-user/output @@ -0,0 +1,18 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/1/{r""-s""} | 016989726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/1/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/1/{r""-s""} | 016989726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/1/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/1/{r""-s""} | 016989726c67632d | | | /Local/RangeID/1/r/RangeGCThreshold | +| /Local/RangeID/1/{r""-s""} | 016989727261736b | | | /Local/RangeID/1/r/RangeAppliedState | +| /Local/RangeID/1/{r""-s""} | 01698972726c6c2d | | | /Local/RangeID/1/r/RangeLease | +| /Local/RangeID/1/{r""-s""} | 016989723a61 | 016989723a78 | 000000000000000109 | /Local/RangeID/1/r":{a"-x"}/0.000000001,0 | +| /Local/Range"{a"-b"} | 016b1261000172647363 | | 0000000000000001 | /Local/Range"a"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{a"-b"} | 016b1261000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"a"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{a"-b"} | 016b126100ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"a\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{a"-b"} | 016b1261ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"a\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{a"-b"} | 017a6b12016b126100ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"a"/RangeDescriptor | +| /Local/Lock/Intent"{a"-b"} | 017a6b12610001 | | 030ce61c175eb445878c36dcf4062ada4c | "a" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/user-only/output new file mode 100644 index 000000000000..0d6abd2a4650 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r1/replicatedOnly/user-only/output @@ -0,0 +1,15 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/1/{r""-s""} | 016989726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/1/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/1/{r""-s""} | 016989726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/1/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/1/{r""-s""} | 016989726c67632d | | | /Local/RangeID/1/r/RangeGCThreshold | +| /Local/RangeID/1/{r""-s""} | 016989727261736b | | | /Local/RangeID/1/r/RangeAppliedState | +| /Local/RangeID/1/{r""-s""} | 01698972726c6c2d | | | /Local/RangeID/1/r/RangeLease | +| /Local/RangeID/1/{r""-s""} | 016989723a61 | 016989723a78 | 000000000000000109 | /Local/RangeID/1/r":{a"-x"}/0.000000001,0 | +| {a-b} | 61 | | 0000000000000001 | "a"/0.000000001,0 | +| {a-b} | 61ffffffff | | 0000000000000001 | "a\xff\xff\xff\xff"/0.000000001,0 | +| {a-b} | 61 | 62 | 000000000000000109 | {a-b}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/exclude-user/output new file mode 100644 index 000000000000..892908511c47 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/exclude-user/output @@ -0,0 +1,24 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/2/{r""-s""} | 01698a726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/2/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/2/{r""-s""} | 01698a726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/2/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/2/{r""-s""} | 01698a726c67632d | | | /Local/RangeID/2/r/RangeGCThreshold | +| /Local/RangeID/2/{r""-s""} | 01698a727261736b | | | /Local/RangeID/2/r/RangeAppliedState | +| /Local/RangeID/2/{r""-s""} | 01698a72726c6c2d | | | /Local/RangeID/2/r/RangeLease | +| /Local/RangeID/2/{r""-s""} | 01698a723a61 | 01698a723a78 | 000000000000000109 | /Local/RangeID/2/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/2/{u""-v""} | 01698a7572667462 | | | /Local/RangeID/2/u/RangeTombstone | +| /Local/RangeID/2/{u""-v""} | 01698a7572667468 | | | /Local/RangeID/2/u/RaftHardState | +| /Local/RangeID/2/{u""-v""} | 01698a757266746c0000000000000001 | | | /Local/RangeID/2/u/RaftLog/logIndex:1 | +| /Local/RangeID/2/{u""-v""} | 01698a757266746c0000000000000002 | | | /Local/RangeID/2/u/RaftLog/logIndex:2 | +| /Local/RangeID/2/{u""-v""} | 01698a75726c7274 | | | /Local/RangeID/2/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/2/{u""-v""} | 01698a753a61 | 01698a753a78 | 000000000000000109 | /Local/RangeID/2/u":{a"-x"}/0.000000001,0 | +| /Local/Range"{b"-c"} | 016b1262000172647363 | | 0000000000000001 | /Local/Range"b"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{b"-c"} | 016b1262000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"b"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{b"-c"} | 016b126200ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"b\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{b"-c"} | 016b1262ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"b\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{b"-c"} | 017a6b12016b126200ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"b"/RangeDescriptor | +| /Local/Lock/Intent"{b"-c"} | 017a6b12620001 | | 030ce61c175eb445878c36dcf4062ada4c | "b" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/user-only/output new file mode 100644 index 000000000000..c8f0a593037e --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/all/user-only/output @@ -0,0 +1,21 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/2/{r""-s""} | 01698a726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/2/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/2/{r""-s""} | 01698a726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/2/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/2/{r""-s""} | 01698a726c67632d | | | /Local/RangeID/2/r/RangeGCThreshold | +| /Local/RangeID/2/{r""-s""} | 01698a727261736b | | | /Local/RangeID/2/r/RangeAppliedState | +| /Local/RangeID/2/{r""-s""} | 01698a72726c6c2d | | | /Local/RangeID/2/r/RangeLease | +| /Local/RangeID/2/{r""-s""} | 01698a723a61 | 01698a723a78 | 000000000000000109 | /Local/RangeID/2/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/2/{u""-v""} | 01698a7572667462 | | | /Local/RangeID/2/u/RangeTombstone | +| /Local/RangeID/2/{u""-v""} | 01698a7572667468 | | | /Local/RangeID/2/u/RaftHardState | +| /Local/RangeID/2/{u""-v""} | 01698a757266746c0000000000000001 | | | /Local/RangeID/2/u/RaftLog/logIndex:1 | +| /Local/RangeID/2/{u""-v""} | 01698a757266746c0000000000000002 | | | /Local/RangeID/2/u/RaftLog/logIndex:2 | +| /Local/RangeID/2/{u""-v""} | 01698a75726c7274 | | | /Local/RangeID/2/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/2/{u""-v""} | 01698a753a61 | 01698a753a78 | 000000000000000109 | /Local/RangeID/2/u":{a"-x"}/0.000000001,0 | +| {b-c} | 62 | | 0000000000000001 | "b"/0.000000001,0 | +| {b-c} | 62ffffffff | | 0000000000000001 | "b\xff\xff\xff\xff"/0.000000001,0 | +| {b-c} | 62 | 63 | 000000000000000109 | {b-c}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/exclude-user/output new file mode 100644 index 000000000000..4790320b2a4b --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/exclude-user/output @@ -0,0 +1,18 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/2/{r""-s""} | 01698a726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/2/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/2/{r""-s""} | 01698a726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/2/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/2/{r""-s""} | 01698a726c67632d | | | /Local/RangeID/2/r/RangeGCThreshold | +| /Local/RangeID/2/{r""-s""} | 01698a727261736b | | | /Local/RangeID/2/r/RangeAppliedState | +| /Local/RangeID/2/{r""-s""} | 01698a72726c6c2d | | | /Local/RangeID/2/r/RangeLease | +| /Local/RangeID/2/{r""-s""} | 01698a723a61 | 01698a723a78 | 000000000000000109 | /Local/RangeID/2/r":{a"-x"}/0.000000001,0 | +| /Local/Range"{b"-c"} | 016b1262000172647363 | | 0000000000000001 | /Local/Range"b"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{b"-c"} | 016b1262000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"b"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{b"-c"} | 016b126200ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"b\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{b"-c"} | 016b1262ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"b\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{b"-c"} | 017a6b12016b126200ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"b"/RangeDescriptor | +| /Local/Lock/Intent"{b"-c"} | 017a6b12620001 | | 030ce61c175eb445878c36dcf4062ada4c | "b" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/user-only/output new file mode 100644 index 000000000000..704f2b962503 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r2/replicatedOnly/user-only/output @@ -0,0 +1,15 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/2/{r""-s""} | 01698a726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/2/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/2/{r""-s""} | 01698a726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/2/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/2/{r""-s""} | 01698a726c67632d | | | /Local/RangeID/2/r/RangeGCThreshold | +| /Local/RangeID/2/{r""-s""} | 01698a727261736b | | | /Local/RangeID/2/r/RangeAppliedState | +| /Local/RangeID/2/{r""-s""} | 01698a72726c6c2d | | | /Local/RangeID/2/r/RangeLease | +| /Local/RangeID/2/{r""-s""} | 01698a723a61 | 01698a723a78 | 000000000000000109 | /Local/RangeID/2/r":{a"-x"}/0.000000001,0 | +| {b-c} | 62 | | 0000000000000001 | "b"/0.000000001,0 | +| {b-c} | 62ffffffff | | 0000000000000001 | "b\xff\xff\xff\xff"/0.000000001,0 | +| {b-c} | 62 | 63 | 000000000000000109 | {b-c}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/exclude-user/output new file mode 100644 index 000000000000..e984657f77bd --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/exclude-user/output @@ -0,0 +1,24 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/3/{r""-s""} | 01698b726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/3/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/3/{r""-s""} | 01698b726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/3/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/3/{r""-s""} | 01698b726c67632d | | | /Local/RangeID/3/r/RangeGCThreshold | +| /Local/RangeID/3/{r""-s""} | 01698b727261736b | | | /Local/RangeID/3/r/RangeAppliedState | +| /Local/RangeID/3/{r""-s""} | 01698b72726c6c2d | | | /Local/RangeID/3/r/RangeLease | +| /Local/RangeID/3/{r""-s""} | 01698b723a61 | 01698b723a78 | 000000000000000109 | /Local/RangeID/3/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/3/{u""-v""} | 01698b7572667462 | | | /Local/RangeID/3/u/RangeTombstone | +| /Local/RangeID/3/{u""-v""} | 01698b7572667468 | | | /Local/RangeID/3/u/RaftHardState | +| /Local/RangeID/3/{u""-v""} | 01698b757266746c0000000000000001 | | | /Local/RangeID/3/u/RaftLog/logIndex:1 | +| /Local/RangeID/3/{u""-v""} | 01698b757266746c0000000000000002 | | | /Local/RangeID/3/u/RaftLog/logIndex:2 | +| /Local/RangeID/3/{u""-v""} | 01698b75726c7274 | | | /Local/RangeID/3/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/3/{u""-v""} | 01698b753a61 | 01698b753a78 | 000000000000000109 | /Local/RangeID/3/u":{a"-x"}/0.000000001,0 | +| /Local/Range"{c"-d"} | 016b1263000172647363 | | 0000000000000001 | /Local/Range"c"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{c"-d"} | 016b1263000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"c"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{c"-d"} | 016b126300ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"c\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{c"-d"} | 016b1263ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"c\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{c"-d"} | 017a6b12016b126300ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"c"/RangeDescriptor | +| /Local/Lock/Intent"{c"-d"} | 017a6b12630001 | | 030ce61c175eb445878c36dcf4062ada4c | "c" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/user-only/output new file mode 100644 index 000000000000..e146d6ec8b87 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/all/user-only/output @@ -0,0 +1,21 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/3/{r""-s""} | 01698b726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/3/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/3/{r""-s""} | 01698b726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/3/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/3/{r""-s""} | 01698b726c67632d | | | /Local/RangeID/3/r/RangeGCThreshold | +| /Local/RangeID/3/{r""-s""} | 01698b727261736b | | | /Local/RangeID/3/r/RangeAppliedState | +| /Local/RangeID/3/{r""-s""} | 01698b72726c6c2d | | | /Local/RangeID/3/r/RangeLease | +| /Local/RangeID/3/{r""-s""} | 01698b723a61 | 01698b723a78 | 000000000000000109 | /Local/RangeID/3/r":{a"-x"}/0.000000001,0 | +| /Local/RangeID/3/{u""-v""} | 01698b7572667462 | | | /Local/RangeID/3/u/RangeTombstone | +| /Local/RangeID/3/{u""-v""} | 01698b7572667468 | | | /Local/RangeID/3/u/RaftHardState | +| /Local/RangeID/3/{u""-v""} | 01698b757266746c0000000000000001 | | | /Local/RangeID/3/u/RaftLog/logIndex:1 | +| /Local/RangeID/3/{u""-v""} | 01698b757266746c0000000000000002 | | | /Local/RangeID/3/u/RaftLog/logIndex:2 | +| /Local/RangeID/3/{u""-v""} | 01698b75726c7274 | | | /Local/RangeID/3/u/RangeLastReplicaGCTimestamp | +| /Local/RangeID/3/{u""-v""} | 01698b753a61 | 01698b753a78 | 000000000000000109 | /Local/RangeID/3/u":{a"-x"}/0.000000001,0 | +| {c-d} | 63 | | 0000000000000001 | "c"/0.000000001,0 | +| {c-d} | 63ffffffff | | 0000000000000001 | "c\xff\xff\xff\xff"/0.000000001,0 | +| {c-d} | 63 | 64 | 000000000000000109 | {c-d}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/all/output similarity index 100% rename from pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/output rename to pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/all/output diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/exclude-user/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/exclude-user/output new file mode 100644 index 000000000000..8fcaf8a3c45a --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/exclude-user/output @@ -0,0 +1,18 @@ +echo +---- ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ +| /Local/RangeID/3/{r""-s""} | 01698b726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/3/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/3/{r""-s""} | 01698b726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/3/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/3/{r""-s""} | 01698b726c67632d | | | /Local/RangeID/3/r/RangeGCThreshold | +| /Local/RangeID/3/{r""-s""} | 01698b727261736b | | | /Local/RangeID/3/r/RangeAppliedState | +| /Local/RangeID/3/{r""-s""} | 01698b72726c6c2d | | | /Local/RangeID/3/r/RangeLease | +| /Local/RangeID/3/{r""-s""} | 01698b723a61 | 01698b723a78 | 000000000000000109 | /Local/RangeID/3/r":{a"-x"}/0.000000001,0 | +| /Local/Range"{c"-d"} | 016b1263000172647363 | | 0000000000000001 | /Local/Range"c"/RangeDescriptor/0.000000001,0 | +| /Local/Range"{c"-d"} | 016b1263000174786e2d0ce61c175eb445878c36dcf4062ada4c | | | /Local/Range"c"/Transaction/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/Range"{c"-d"} | 016b126300ff000174786e2d9855a1ef8eb94c06a106cab1dda78a2b | | | /Local/Range"c\x00"/Transaction/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/Range"{c"-d"} | 016b1263ffffffff000174786e2d295e727c8ca9437cbb5e8e2ebbad996f | | | /Local/Range"c\xff\xff\xff\xff"/Transaction/"295e727c-8ca9-437c-bb5e-8e2ebbad996f" | +| /Local/Lock/Intent/Local/Range"{c"-d"} | 017a6b12016b126300ff01726473630001 | | 030ce61c175eb445878c36dcf4062ada4c | /Local/Range"c"/RangeDescriptor | +| /Local/Lock/Intent"{c"-d"} | 017a6b12630001 | | 030ce61c175eb445878c36dcf4062ada4c | "c" | ++----------------------------------------+--------------------------------------------------------------+--------------+------------------------------------+------------------------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/user-only/output b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/user-only/output new file mode 100644 index 000000000000..b5130d147176 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestReplicaDataIterator/r3/replicatedOnly/user-only/output @@ -0,0 +1,15 @@ +echo +---- ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| SPAN | KEY HEX | ENDKEY HEX | VERSION HEX | PRETTY | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ +| /Local/RangeID/3/{r""-s""} | 01698b726162632d120ce61c175eb445878c36dcf4062ada4c0001 | | | /Local/RangeID/3/r/AbortSpan/"0ce61c17-5eb4-4587-8c36-dcf4062ada4c" | +| /Local/RangeID/3/{r""-s""} | 01698b726162632d129855a1ef8eb94c06a106cab1dda78a2b0001 | | | /Local/RangeID/3/r/AbortSpan/"9855a1ef-8eb9-4c06-a106-cab1dda78a2b" | +| /Local/RangeID/3/{r""-s""} | 01698b726c67632d | | | /Local/RangeID/3/r/RangeGCThreshold | +| /Local/RangeID/3/{r""-s""} | 01698b727261736b | | | /Local/RangeID/3/r/RangeAppliedState | +| /Local/RangeID/3/{r""-s""} | 01698b72726c6c2d | | | /Local/RangeID/3/r/RangeLease | +| /Local/RangeID/3/{r""-s""} | 01698b723a61 | 01698b723a78 | 000000000000000109 | /Local/RangeID/3/r":{a"-x"}/0.000000001,0 | +| {c-d} | 63 | | 0000000000000001 | "c"/0.000000001,0 | +| {c-d} | 63ffffffff | | 0000000000000001 | "c\xff\xff\xff\xff"/0.000000001,0 | +| {c-d} | 63 | 64 | 000000000000000109 | {c-d}/0.000000001,0 | ++----------------------------+--------------------------------------------------------+--------------+--------------------+---------------------------------------------------------------------+ diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/no_span b/pkg/kv/kvserver/rditer/testdata/TestSelect/no_span index f8eacf173f16..f06a5a7b21b0 100644 --- a/pkg/kv/kvserver/rditer/testdata/TestSelect/no_span +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/no_span @@ -1,10 +1,10 @@ echo ---- -Select({ReplicatedBySpan:/Min ReplicatedByRangeID:false UnreplicatedByRangeID:false}): -Select({ReplicatedBySpan:/Min ReplicatedByRangeID:false UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:/Min ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:/Min ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): /Local/RangeID/123/{u""-v""} -Select({ReplicatedBySpan:/Min ReplicatedByRangeID:true UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:/Min ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): /Local/RangeID/123/{r""-s""} -Select({ReplicatedBySpan:/Min ReplicatedByRangeID:true UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:/Min ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): /Local/RangeID/123/{r""-s""} /Local/RangeID/123/{u""-v""} diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/r1 b/pkg/kv/kvserver/rditer/testdata/TestSelect/r1 index 7597540eeeaf..3eb72a88dd63 100644 --- a/pkg/kv/kvserver/rditer/testdata/TestSelect/r1 +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/r1 @@ -1,23 +1,23 @@ echo ---- -Select({ReplicatedBySpan:{/Min-c} ReplicatedByRangeID:false UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{/Min-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): /Local/Range{/Min-"c"} /Local/Lock/Intent/Local/Range{/Min-"c"} /Local/Lock/Intent{/Meta1/""-"c"} {/Meta1-c} -Select({ReplicatedBySpan:{/Min-c} ReplicatedByRangeID:false UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{/Min-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): /Local/RangeID/123/{u""-v""} /Local/Range{/Min-"c"} /Local/Lock/Intent/Local/Range{/Min-"c"} /Local/Lock/Intent{/Meta1/""-"c"} {/Meta1-c} -Select({ReplicatedBySpan:{/Min-c} ReplicatedByRangeID:true UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{/Min-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): /Local/RangeID/123/{r""-s""} /Local/Range{/Min-"c"} /Local/Lock/Intent/Local/Range{/Min-"c"} /Local/Lock/Intent{/Meta1/""-"c"} {/Meta1-c} -Select({ReplicatedBySpan:{/Min-c} ReplicatedByRangeID:true UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{/Min-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): /Local/RangeID/123/{r""-s""} /Local/RangeID/123/{u""-v""} /Local/Range{/Min-"c"} diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/r2 b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2 index 35f755e3c75d..1cf0dbb2137f 100644 --- a/pkg/kv/kvserver/rditer/testdata/TestSelect/r2 +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2 @@ -1,23 +1,23 @@ echo ---- -Select({ReplicatedBySpan:{a-c} ReplicatedByRangeID:false UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): /Local/Range"{a"-c"} /Local/Lock/Intent/Local/Range"{a"-c"} /Local/Lock/Intent"{a"-c"} {a-c} -Select({ReplicatedBySpan:{a-c} ReplicatedByRangeID:false UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): /Local/RangeID/123/{u""-v""} /Local/Range"{a"-c"} /Local/Lock/Intent/Local/Range"{a"-c"} /Local/Lock/Intent"{a"-c"} {a-c} -Select({ReplicatedBySpan:{a-c} ReplicatedByRangeID:true UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): /Local/RangeID/123/{r""-s""} /Local/Range"{a"-c"} /Local/Lock/Intent/Local/Range"{a"-c"} /Local/Lock/Intent"{a"-c"} {a-c} -Select({ReplicatedBySpan:{a-c} ReplicatedByRangeID:true UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): /Local/RangeID/123/{r""-s""} /Local/RangeID/123/{u""-v""} /Local/Range"{a"-c"} diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_excludeuser b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_excludeuser new file mode 100644 index 000000000000..af01fa9abc3a --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_excludeuser @@ -0,0 +1,22 @@ +echo +---- +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:1 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): + /Local/Range"{a"-c"} + /Local/Lock/Intent/Local/Range"{a"-c"} + /Local/Lock/Intent"{a"-c"} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:1 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): + /Local/RangeID/123/{u""-v""} + /Local/Range"{a"-c"} + /Local/Lock/Intent/Local/Range"{a"-c"} + /Local/Lock/Intent"{a"-c"} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:1 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): + /Local/RangeID/123/{r""-s""} + /Local/Range"{a"-c"} + /Local/Lock/Intent/Local/Range"{a"-c"} + /Local/Lock/Intent"{a"-c"} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:1 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): + /Local/RangeID/123/{r""-s""} + /Local/RangeID/123/{u""-v""} + /Local/Range"{a"-c"} + /Local/Lock/Intent/Local/Range"{a"-c"} + /Local/Lock/Intent"{a"-c"} diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_useronly b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_useronly new file mode 100644 index 000000000000..253df5668631 --- /dev/null +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/r2_useronly @@ -0,0 +1,14 @@ +echo +---- +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:2 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): + {a-c} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:2 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): + /Local/RangeID/123/{u""-v""} + {a-c} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:2 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): + /Local/RangeID/123/{r""-s""} + {a-c} +Select({ReplicatedBySpan:{a-c} ReplicatedSpansFilter:2 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): + /Local/RangeID/123/{r""-s""} + /Local/RangeID/123/{u""-v""} + {a-c} diff --git a/pkg/kv/kvserver/rditer/testdata/TestSelect/r3 b/pkg/kv/kvserver/rditer/testdata/TestSelect/r3 index 4e3f40dca0c6..235b4b4c5ff6 100644 --- a/pkg/kv/kvserver/rditer/testdata/TestSelect/r3 +++ b/pkg/kv/kvserver/rditer/testdata/TestSelect/r3 @@ -1,23 +1,23 @@ echo ---- -Select({ReplicatedBySpan:{a-/Max} ReplicatedByRangeID:false UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{a-/Max} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:false}): /Local/Range{"a"-/Max} /Local/Lock/Intent/Local/Range{"a"-/Max} /Local/Lock/Intent{"a"-/Max} {a-/Max} -Select({ReplicatedBySpan:{a-/Max} ReplicatedByRangeID:false UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{a-/Max} ReplicatedSpansFilter:0 ReplicatedByRangeID:false UnreplicatedByRangeID:true}): /Local/RangeID/123/{u""-v""} /Local/Range{"a"-/Max} /Local/Lock/Intent/Local/Range{"a"-/Max} /Local/Lock/Intent{"a"-/Max} {a-/Max} -Select({ReplicatedBySpan:{a-/Max} ReplicatedByRangeID:true UnreplicatedByRangeID:false}): +Select({ReplicatedBySpan:{a-/Max} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:false}): /Local/RangeID/123/{r""-s""} /Local/Range{"a"-/Max} /Local/Lock/Intent/Local/Range{"a"-/Max} /Local/Lock/Intent{"a"-/Max} {a-/Max} -Select({ReplicatedBySpan:{a-/Max} ReplicatedByRangeID:true UnreplicatedByRangeID:true}): +Select({ReplicatedBySpan:{a-/Max} ReplicatedSpansFilter:0 ReplicatedByRangeID:true UnreplicatedByRangeID:true}): /Local/RangeID/123/{r""-s""} /Local/RangeID/123/{u""-v""} /Local/Range{"a"-/Max} diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index 87eb20d7ce4d..7ba6ab2c48ee 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -2231,7 +2231,7 @@ func getExpectedSnapshotSizeBytes( b := originStore.TODOEngine().NewWriteBatch() defer b.Close() - err = rditer.IterateReplicaKeySpans(snap.State.Desc, snap.EngineSnap, true, /* replicatedOnly */ + err = rditer.IterateReplicaKeySpans(snap.State.Desc, snap.EngineSnap, true /* replicatedOnly */, rditer.ReplicatedSpansAll, func(iter storage.EngineIterator, _ roachpb.Span, keyType storage.IterKeyType) error { var err error for ok := true; ok && err == nil; ok, err = iter.NextEngineKey() { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index c5bcb46e95a3..3a13a68b3ee7 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -7183,7 +7183,7 @@ func TestReplicaDestroy(t *testing.T) { expectedKeys := []roachpb.Key{keys.RangeTombstoneKey(tc.repl.RangeID)} actualKeys := []roachpb.Key{} - require.NoError(t, rditer.IterateReplicaKeySpans(tc.repl.Desc(), engSnapshot, false, /* replicatedOnly */ + require.NoError(t, rditer.IterateReplicaKeySpans(tc.repl.Desc(), engSnapshot, false /* replicatedOnly */, rditer.ReplicatedSpansAll, func(iter storage.EngineIterator, _ roachpb.Span, keyType storage.IterKeyType) error { require.Equal(t, storage.IterKeyTypePointsOnly, keyType) var err error diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 0dc36e5a34ce..54bb08e74f81 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -579,7 +579,7 @@ func (kvSS *kvBatchSnapshotStrategy) Send( return nil } - err := rditer.IterateReplicaKeySpans(snap.State.Desc, snap.EngineSnap, true, /* replicatedOnly */ + err := rditer.IterateReplicaKeySpans(snap.State.Desc, snap.EngineSnap, true /* replicatedOnly */, rditer.ReplicatedSpansAll, func(iter storage.EngineIterator, _ roachpb.Span, keyType storage.IterKeyType) error { timingTag.start("iter") defer timingTag.stop("iter")