From 3771e3320fcb79e482ac0339faef109170d6e19c Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Thu, 20 Jul 2023 14:28:23 -0400 Subject: [PATCH 1/9] 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/9] 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") From 38f086c48a4e8c725d45b4318f4e9b3a6c88ae95 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Wed, 9 Aug 2023 18:10:08 +0000 Subject: [PATCH 3/9] cloudccl: allow external connection tests to be run in parallel Currently external connection tests read and write to the same path in cloud storage. Add a random uint64 as part of the path so that test runs have unique paths and can be run in parallel. Fixes: #107407 Release note: None --- pkg/ccl/cloudccl/amazon/BUILD.bazel | 1 + pkg/ccl/cloudccl/amazon/s3_connection_test.go | 21 +++++++----- pkg/ccl/cloudccl/azure/BUILD.bazel | 1 + .../cloudccl/azure/azure_connection_test.go | 10 +++--- pkg/ccl/cloudccl/gcp/gcp_connection_test.go | 34 ++++++++++++------- 5 files changed, 43 insertions(+), 24 deletions(-) diff --git a/pkg/ccl/cloudccl/amazon/BUILD.bazel b/pkg/ccl/cloudccl/amazon/BUILD.bazel index a647aec1ddcd..a3e5fe97260b 100644 --- a/pkg/ccl/cloudccl/amazon/BUILD.bazel +++ b/pkg/ccl/cloudccl/amazon/BUILD.bazel @@ -14,6 +14,7 @@ go_test( "//pkg/cloud", "//pkg/cloud/amazon", "//pkg/cloud/cloudpb", + "//pkg/cloud/cloudtestutils", "//pkg/cloud/externalconn/providers", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/cloudccl/amazon/s3_connection_test.go b/pkg/ccl/cloudccl/amazon/s3_connection_test.go index 1c52461c0eb3..74f80e2a9c1c 100644 --- a/pkg/ccl/cloudccl/amazon/s3_connection_test.go +++ b/pkg/ccl/cloudccl/amazon/s3_connection_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/amazon" "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" + "github.com/cockroachdb/cockroach/pkg/cloud/cloudtestutils" _ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // import External Connection providers. "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -77,6 +78,8 @@ func TestS3ExternalConnection(t *testing.T) { skip.IgnoreLint(t, "AWS_S3_BUCKET env var must be set") } + testID := cloudtestutils.NewTestID() + t.Run("auth-implicit", func(t *testing.T) { // You can create an IAM that can access S3 // in the AWS console, then set it up locally. @@ -93,14 +96,14 @@ func TestS3ExternalConnection(t *testing.T) { params := make(url.Values) params.Add(cloud.AuthParam, cloud.AuthParamImplicit) - s3URI := fmt.Sprintf("s3://%s/backup-ec-test-default?%s", bucket, params.Encode()) + s3URI := fmt.Sprintf("s3://%s/backup-ec-test-default-%d?%s", bucket, testID, params.Encode()) ecName := "auth-implicit-s3" createExternalConnection(ecName, s3URI) backupAndRestoreFromExternalConnection(ecName) }) t.Run("auth-specified", func(t *testing.T) { - s3URI := amazon.S3URI(bucket, "backup-ec-test-default", + s3URI := amazon.S3URI(bucket, fmt.Sprintf("backup-ec-test-default-%d", testID), &cloudpb.ExternalStorage_S3{ AccessKey: creds.AccessKeyID, Secret: creds.SecretAccessKey, @@ -126,7 +129,7 @@ func TestS3ExternalConnection(t *testing.T) { "refer to https://docs.aws.com/cli/latest/userguide/cli-configure-role.html: %s", err) } - s3URI := amazon.S3URI(bucket, "backup-ec-test-sse-256", &cloudpb.ExternalStorage_S3{ + s3URI := amazon.S3URI(bucket, fmt.Sprintf("backup-ec-test-sse-256-%d", testID), &cloudpb.ExternalStorage_S3{ Region: "us-east-1", Auth: cloud.AuthParamImplicit, ServerEncMode: "AES256", @@ -139,7 +142,7 @@ func TestS3ExternalConnection(t *testing.T) { if v == "" { skip.IgnoreLint(t, "AWS_KMS_KEY_ARN env var must be set") } - s3KMSURI := amazon.S3URI(bucket, "backup-ec-test-sse-kms", &cloudpb.ExternalStorage_S3{ + s3KMSURI := amazon.S3URI(bucket, fmt.Sprintf("backup-ec-test-sse-kms-%d", testID), &cloudpb.ExternalStorage_S3{ Region: "us-east-1", Auth: cloud.AuthParamImplicit, ServerEncMode: "aws:kms", @@ -163,7 +166,7 @@ func TestS3ExternalConnection(t *testing.T) { } // Unsupported server side encryption option. - invalidS3URI := amazon.S3URI(bucket, "backup-ec-test-sse-256", &cloudpb.ExternalStorage_S3{ + invalidS3URI := amazon.S3URI(bucket, fmt.Sprintf("backup-ec-test-sse-256-%d", testID), &cloudpb.ExternalStorage_S3{ Region: "us-east-1", Auth: cloud.AuthParamImplicit, ServerEncMode: "unsupported-algorithm", @@ -172,7 +175,7 @@ func TestS3ExternalConnection(t *testing.T) { "unsupported server encryption mode unsupported-algorithm. Supported values are `aws:kms` and `AES256", fmt.Sprintf(`BACKUP DATABASE foo INTO '%s'`, invalidS3URI)) - invalidS3URI = amazon.S3URI(bucket, "backup-ec-test-sse-256", &cloudpb.ExternalStorage_S3{ + invalidS3URI = amazon.S3URI(bucket, fmt.Sprintf("backup-ec-test-sse-256-%d", testID), &cloudpb.ExternalStorage_S3{ Region: "us-east-1", Auth: cloud.AuthParamImplicit, ServerEncMode: "aws:kms", @@ -257,8 +260,9 @@ func TestAWSKMSExternalConnection(t *testing.T) { skip.IgnoreLint(t, "AWS_S3_BUCKET env var must be set") } + testID := cloudtestutils.NewTestID() // Create an external connection where we will write the backup. - backupURI := fmt.Sprintf("s3://%s/backup?%s=%s", bucket, + backupURI := fmt.Sprintf("s3://%s/backup-%d?%s=%s", bucket, testID, cloud.AuthParam, cloud.AuthParamImplicit) backupExternalConnectionName := "backup" createExternalConnection(backupExternalConnectionName, backupURI) @@ -372,8 +376,9 @@ func TestAWSKMSExternalConnectionAssumeRole(t *testing.T) { skip.IgnoreLint(t, "AWS_S3_BUCKET env var must be set") } + testID := cloudtestutils.NewTestID() // Create an external connection where we will write the backup. - backupURI := fmt.Sprintf("s3://%s/backup?%s=%s", bucket, + backupURI := fmt.Sprintf("s3://%s/backup-%d?%s=%s", bucket, testID, cloud.AuthParam, cloud.AuthParamImplicit) backupExternalConnectionName := "backup" createExternalConnection(backupExternalConnectionName, backupURI) diff --git a/pkg/ccl/cloudccl/azure/BUILD.bazel b/pkg/ccl/cloudccl/azure/BUILD.bazel index 002f85c5b68c..ead845f28bb8 100644 --- a/pkg/ccl/cloudccl/azure/BUILD.bazel +++ b/pkg/ccl/cloudccl/azure/BUILD.bazel @@ -13,6 +13,7 @@ go_test( "//pkg/ccl", "//pkg/ccl/kvccl/kvtenantccl", "//pkg/cloud/azure", + "//pkg/cloud/cloudtestutils", "//pkg/cloud/externalconn/providers", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/cloudccl/azure/azure_connection_test.go b/pkg/ccl/cloudccl/azure/azure_connection_test.go index fbf7887d93dd..e1c35eaab26a 100644 --- a/pkg/ccl/cloudccl/azure/azure_connection_test.go +++ b/pkg/ccl/cloudccl/azure/azure_connection_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" _ "github.com/cockroachdb/cockroach/pkg/ccl" "github.com/cockroachdb/cockroach/pkg/cloud/azure" + "github.com/cockroachdb/cockroach/pkg/cloud/cloudtestutils" _ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // import External Connection providers. "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -29,9 +30,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) -func (a azureConfig) URI(file string) string { - return fmt.Sprintf("azure-storage://%s/%s?%s=%s&%s=%s&%s=%s", - a.bucket, file, +func (a azureConfig) URI(file string, testID uint64) string { + return fmt.Sprintf("azure-storage://%s/%s-%d?%s=%s&%s=%s&%s=%s", + a.bucket, file, testID, azure.AzureAccountKeyParam, url.QueryEscape(a.key), azure.AzureAccountNameParam, url.QueryEscape(a.account), azure.AzureEnvironmentKeyParam, url.QueryEscape(a.environment)) @@ -97,7 +98,8 @@ func TestExternalConnections(t *testing.T) { return } + testID := cloudtestutils.NewTestID() ecName := "azure-ec" - createExternalConnection(ecName, cfg.URI("backup-ec")) + createExternalConnection(ecName, cfg.URI("backup-ec", testID)) backupAndRestoreFromExternalConnection(ecName) } diff --git a/pkg/ccl/cloudccl/gcp/gcp_connection_test.go b/pkg/ccl/cloudccl/gcp/gcp_connection_test.go index 705af47b43e2..1f4703725731 100644 --- a/pkg/ccl/cloudccl/gcp/gcp_connection_test.go +++ b/pkg/ccl/cloudccl/gcp/gcp_connection_test.go @@ -89,8 +89,9 @@ func TestGCPKMSExternalConnection(t *testing.T) { skip.IgnoreLint(t, "implicit auth is not configured") } + testID := cloudtestutils.NewTestID() // Create an external connection where we will write the backup. - backupURI := fmt.Sprintf("gs://%s/backup?%s=%s", bucket, + backupURI := fmt.Sprintf("gs://%s/backup-%d?%s=%s", bucket, testID, cloud.AuthParam, cloud.AuthParamImplicit) backupExternalConnectionName := "backup" createExternalConnection(backupExternalConnectionName, backupURI) @@ -229,8 +230,10 @@ func TestGCPKMSExternalConnectionAssumeRole(t *testing.T) { skip.IgnoreLint(t, "implicit auth is not configured") } + testID := cloudtestutils.NewTestID() + // Create an external connection where we will write the backup. - backupURI := fmt.Sprintf("gs://%s/backup?%s=%s", bucket, + backupURI := fmt.Sprintf("gs://%s/backup-%d?%s=%s", bucket, testID, cloud.AuthParam, cloud.AuthParamImplicit) backupExternalConnectionName := "backup" createExternalConnection(backupExternalConnectionName, backupURI) @@ -339,6 +342,7 @@ func TestGCPAssumeRoleExternalConnection(t *testing.T) { skip.IgnoreLint(t, "ASSUME_SERVICE_ACCOUNT env var must be set") } + testID := cloudtestutils.NewTestID() t.Run("ec-assume-role-specified", func(t *testing.T) { ecName := "ec-assume-role-specified" disallowedECName := "ec-assume-role-specified-disallowed" @@ -347,13 +351,14 @@ func TestGCPAssumeRoleExternalConnection(t *testing.T) { skip.IgnoreLint(t, "GOOGLE_CREDENTIALS_JSON env var must be set") } encoded := base64.StdEncoding.EncodeToString([]byte(credentials)) - disallowedURI := fmt.Sprintf("gs://%s/%s?%s=%s", limitedBucket, disallowedECName, + disallowedURI := fmt.Sprintf("gs://%s/%s-%d?%s=%s", limitedBucket, disallowedECName, testID, gcp.CredentialsParam, url.QueryEscape(encoded)) disallowedCreateExternalConnection(t, disallowedECName, disallowedURI) - uri := fmt.Sprintf("gs://%s/%s?%s=%s&%s=%s&%s=%s", + uri := fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s&%s=%s", limitedBucket, ecName, + testID, cloud.AuthParam, cloud.AuthParamSpecified, gcp.AssumeRoleParam, @@ -370,13 +375,14 @@ func TestGCPAssumeRoleExternalConnection(t *testing.T) { } ecName := "ec-assume-role-implicit" disallowedECName := "ec-assume-role-implicit-disallowed" - disallowedURI := fmt.Sprintf("gs://%s/%s?%s=%s", limitedBucket, disallowedECName, + disallowedURI := fmt.Sprintf("gs://%s/%s-%d?%s=%s", limitedBucket, disallowedECName, testID, cloud.AuthParam, cloud.AuthParamImplicit) disallowedCreateExternalConnection(t, disallowedECName, disallowedURI) - uri := fmt.Sprintf("gs://%s/%s?%s=%s&%s=%s", + uri := fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s", limitedBucket, ecName, + testID, cloud.AuthParam, cloud.AuthParamImplicit, gcp.AssumeRoleParam, @@ -418,17 +424,18 @@ func TestGCPAssumeRoleExternalConnection(t *testing.T) { i := i q.Set(gcp.AssumeRoleParam, role) disallowedECName := fmt.Sprintf("ec-assume-role-checking-%d", i) - disallowedBackupURI := fmt.Sprintf("gs://%s/%s?%s", limitedBucket, - disallowedECName, q.Encode()) + disallowedBackupURI := fmt.Sprintf("gs://%s/%s-%d?%s", limitedBucket, + disallowedECName, testID, q.Encode()) disallowedCreateExternalConnection(t, disallowedECName, disallowedBackupURI) } // Finally, check that the chain of roles can be used to access the storage. q.Set(gcp.AssumeRoleParam, roleChainStr) ecName := fmt.Sprintf("ec-assume-role-checking-%s", tc.auth) - uri := fmt.Sprintf("gs://%s/%s?%s", + uri := fmt.Sprintf("gs://%s/%s-%d?%s", limitedBucket, ecName, + testID, q.Encode(), ) createExternalConnection(t, ecName, uri) @@ -477,13 +484,14 @@ func TestGCPExternalConnection(t *testing.T) { skip.IgnoreLint(t, "GOOGLE_BUCKET env var must be set") } + testID := cloudtestutils.NewTestID() t.Run("ec-auth-implicit", func(t *testing.T) { if !cloudtestutils.IsImplicitAuthConfigured() { skip.IgnoreLint(t, "implicit auth is not configured") } ecName := "ec-auth-implicit" - backupURI := fmt.Sprintf("gs://%s/%s?%s=%s", bucket, ecName, cloud.AuthParam, + backupURI := fmt.Sprintf("gs://%s/%s-%d?%s=%s", bucket, ecName, testID, cloud.AuthParam, cloud.AuthParamImplicit) createExternalConnection(ecName, backupURI) backupAndRestoreFromExternalConnection(ecName) @@ -496,9 +504,10 @@ func TestGCPExternalConnection(t *testing.T) { } encoded := base64.StdEncoding.EncodeToString([]byte(credentials)) ecName := "ec-auth-specified" - backupURI := fmt.Sprintf("gs://%s/%s?%s=%s", + backupURI := fmt.Sprintf("gs://%s/%s-%d?%s=%s", bucket, ecName, + testID, gcp.CredentialsParam, url.QueryEscape(encoded), ) @@ -520,9 +529,10 @@ func TestGCPExternalConnection(t *testing.T) { token, err := ts.Token() require.NoError(t, err, "getting token") ecName := "ec-auth-specified-bearer-token" - backupURI := fmt.Sprintf("gs://%s/%s?%s=%s", + backupURI := fmt.Sprintf("gs://%s/%s-%d?%s=%s", bucket, ecName, + testID, gcp.BearerTokenParam, token.AccessToken, ) From 64d2b3fef36a51bbe35716c14b1922080f17b606 Mon Sep 17 00:00:00 2001 From: David Hartunian Date: Wed, 9 Aug 2023 16:58:13 -0400 Subject: [PATCH 4/9] acceptance: stabilize start-single-node in tcl test We've continued to see flakes on this test which contain messages of throttled stores on node startup. The hypothesis is that these are due to leftover data directories from prior startups during the same test. This change clears the `logs/db` data directory for those invocations and also adds the sql memory flag which the common tcl function also uses. Resolves #108405 Epic: None Release note: None --- pkg/cli/interactive_tests/test_missing_log_output.tcl | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/cli/interactive_tests/test_missing_log_output.tcl b/pkg/cli/interactive_tests/test_missing_log_output.tcl index ccc4cb6bcdad..427f552a3ee4 100644 --- a/pkg/cli/interactive_tests/test_missing_log_output.tcl +++ b/pkg/cli/interactive_tests/test_missing_log_output.tcl @@ -63,7 +63,7 @@ eexpect ":/# " stop_server $argv start_test "Check that a server started with --logtostderr logs even info messages to stderr." -send "$argv start-single-node -s=path=logs/db --insecure --logtostderr\r" +send "$argv start-single-node --max-sql-memory=128MB -s=path=logs/db --insecure --logtostderr\r" eexpect "CockroachDB node starting" end_test @@ -71,9 +71,10 @@ end_test interrupt interrupt eexpect ":/# " +system "rm -rf logs/db" start_test "Check that --logtostderr can override the threshold but no error is printed on startup" -send "echo marker; $argv start-single-node -s=path=logs/db --insecure --logtostderr=ERROR 2>&1 | grep -v '^\\*'\r" +send "echo marker; $argv start-single-node --max-sql-memory=128MB -s=path=logs/db --insecure --logtostderr=ERROR 2>&1 | grep -v '^\\*'\r" eexpect "marker\r\nCockroachDB node starting" end_test @@ -81,9 +82,11 @@ end_test interrupt interrupt eexpect ":/# " +system "rm -rf logs/db" + start_test "Check that panic reports are printed to the log even when --logtostderr is specified" -send "$argv start-single-node -s=path=logs/db --insecure --logtostderr\r" +send "$argv start-single-node --max-sql-memory=128MB -s=path=logs/db --insecure --logtostderr\r" eexpect "CockroachDB node starting" system "($argv sql --insecure -e \"select crdb_internal.force_panic('helloworld')\" || true)&" From 990dfdcc1275e3df622eef866d8f43c51f607c88 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 10 Jul 2023 13:37:43 -0700 Subject: [PATCH 5/9] sql: retry all DistSQL runner dial errors This commit marks the error that DistSQL runners produce when dialing remote nodes in a special way that is now always retried-as-local. In particular, this allows us to fix two problematic scenarios that could occur when using secondary tenants: - when attempting to start a pod with stale instance information - the port is in use by an RPC server for the same tenant, but with a new instance id. This commit includes the test from Jeff that exposed the gap in the retry-as-local mechanism. Release note: None --- pkg/ccl/serverccl/BUILD.bazel | 1 + pkg/ccl/serverccl/server_sql_test.go | 58 ++++++++++++++++++++++++++++ pkg/sql/distsql_running.go | 23 ++++++++++- 3 files changed, 81 insertions(+), 1 deletion(-) diff --git a/pkg/ccl/serverccl/BUILD.bazel b/pkg/ccl/serverccl/BUILD.bazel index ac9b87428c2b..0610046dc3f9 100644 --- a/pkg/ccl/serverccl/BUILD.bazel +++ b/pkg/ccl/serverccl/BUILD.bazel @@ -87,6 +87,7 @@ go_test( "//pkg/util/log", "//pkg/util/protoutil", "//pkg/util/randutil", + "//pkg/util/stop", "//pkg/util/timeutil", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/ccl/serverccl/server_sql_test.go b/pkg/ccl/serverccl/server_sql_test.go index c1617a11371a..f6cee0fa0927 100644 --- a/pkg/ccl/serverccl/server_sql_test.go +++ b/pkg/ccl/serverccl/server_sql_test.go @@ -12,6 +12,7 @@ import ( "context" "fmt" "io" + "net" "net/http" "strings" "testing" @@ -35,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/errors" "github.com/lib/pq" "github.com/stretchr/testify/require" @@ -413,3 +415,59 @@ func TestSystemConfigWatcherCache(t *testing.T) { defer leaktest.AfterTest(t)() systemconfigwatchertest.TestSystemConfigWatcher(t, false /* skipSecondary */) } + +// TestStartTenantWithStaleInstance covers the following scenario: +// - a sql server starts up and is assigned port 'a' +// - the sql server shuts down and releases port 'a' +// - something else starts up and claims port 'a'. In the test that is the +// listener. This is important because the listener causes connections to 'a' to +// hang instead of responding with a RESET packet. +// - a different server with stale instance information schedules a distsql +// flow and attempts to dial port 'a'. +func TestStartTenantWithStaleInstance(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s := serverutils.StartServerOnly(t, base.TestServerArgs{ + DefaultTestTenant: base.TestControlsTenantsExplicitly, + }) + defer s.Stopper().Stop(ctx) + + var listener net.Listener + // In rare cases under stress net.Listen call can result in an error that + // the address is already in use (because the stopped tenant hasn't released + // the socket); thus, we allow for some retries to go around that issue. + testutils.SucceedsSoon(t, func() error { + rpcAddr := func() string { + tenantStopper := stop.NewStopper() + defer tenantStopper.Stop(ctx) + server, db := serverutils.StartTenant(t, s, base.TestTenantArgs{ + Stopper: tenantStopper, + TenantID: serverutils.TestTenantID(), + }, + ) + defer db.Close() + return server.RPCAddr() + }() + + var err error + listener, err = net.Listen("tcp", rpcAddr) + return err + }) + defer func() { + _ = listener.Close() + }() + + _, db := serverutils.StartTenant(t, s, base.TestTenantArgs{ + TenantID: serverutils.TestTenantID(), + }) + defer func() { + _ = db.Close() + }() + + // Query a table to make sure the tenant is healthy, doesn't really matter + // which table. + _, err := db.Exec("SELECT count(*) FROM system.sqlliveness") + require.NoError(t, err) +} diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 127cf1ff4d7e..ddefb975fadf 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -100,6 +100,22 @@ type runnerResult struct { err error } +type runnerDialErr struct { + err error +} + +func (e *runnerDialErr) Error() string { + return e.err.Error() +} + +func (e *runnerDialErr) Cause() error { + return e.err +} + +func isDialErr(err error) bool { + return errors.HasType(err, (*runnerDialErr)(nil)) +} + // run executes the request. An error, if encountered, is both sent on the // result channel and returned. func (req runnerRequest) run() error { @@ -111,6 +127,9 @@ func (req runnerRequest) run() error { conn, err := req.podNodeDialer.Dial(req.ctx, roachpb.NodeID(req.sqlInstanceID), rpc.DefaultClass) if err != nil { + // Mark this error as special runnerDialErr so that we could retry this + // distributed query as local. + err = &runnerDialErr{err: err} res.err = err return err } @@ -1979,7 +1998,9 @@ func (dsp *DistSQLPlanner) PlanAndRun( // cancellation has already occurred. return } - if !pgerror.IsSQLRetryableError(distributedErr) && !flowinfra.IsFlowRetryableError(distributedErr) { + if !pgerror.IsSQLRetryableError(distributedErr) && + !flowinfra.IsFlowRetryableError(distributedErr) && + !isDialErr(distributedErr) { // Only re-run the query if we think there is a high chance of a // successful local execution. return From 847722ed9d146c0c1f678f95eceeac9fd30b8591 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 7 Aug 2023 19:59:55 -0700 Subject: [PATCH 6/9] server: minor cleanup of logging around draining This commit moves the "drain succeeded" logging message to be at the very end of the drain process. Also, it removes now stale comment. Release note: None --- pkg/server/drain.go | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 315e28d11df3..61c898be4e68 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -447,10 +447,6 @@ func (s *drainServer) drainClients( // tasks that may issue SQL statements have shut down. s.sqlServer.leaseMgr.SetDraining(ctx, true /* drain */, reporter) - // Mark this phase in the logs to clarify the context of any subsequent - // errors/warnings, if any. - log.Infof(ctx, "SQL server drained successfully; SQL queries cannot execute any more") - session, err := s.sqlServer.sqlLivenessProvider.Release(ctx) if err != nil { return err @@ -464,8 +460,9 @@ func (s *drainServer) drainClients( // Mark the node as fully drained. s.sqlServer.gracefulDrainComplete.Set(true) - - // Done. This executes the defers set above to drain SQL leases. + // Mark this phase in the logs to clarify the context of any subsequent + // errors/warnings, if any. + log.Infof(ctx, "SQL server drained successfully; SQL queries cannot execute any more") return nil } From 5c09eb1152c1d27fee447fa5bb9bad04d6d2e82a Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 7 Aug 2023 20:05:59 -0700 Subject: [PATCH 7/9] sql: retry all "rpc error" distributed errors as local This commit includes all errors that contain `rpc error` substring to be retried-as-local. In particular, this allows us to avoid problems with DistSQL using no-longer-live SQL pod after that pod is shutdown. (This usage of the downed pod is currently expected given that the cache of live instances isn't updated when the pod is shutdown.) Release note: None --- pkg/jobs/joberror/BUILD.bazel | 1 + pkg/jobs/joberror/errors.go | 24 ++---------------------- pkg/sql/distsql_running.go | 4 +++- pkg/sql/sqlerrors/errors.go | 21 +++++++++++++++++++++ 4 files changed, 27 insertions(+), 23 deletions(-) diff --git a/pkg/jobs/joberror/BUILD.bazel b/pkg/jobs/joberror/BUILD.bazel index 4a0cc63b885b..eb754a23ed39 100644 --- a/pkg/jobs/joberror/BUILD.bazel +++ b/pkg/jobs/joberror/BUILD.bazel @@ -8,6 +8,7 @@ go_library( deps = [ "//pkg/kv/kvclient/kvcoord", "//pkg/sql/flowinfra", + "//pkg/sql/sqlerrors", "//pkg/util/circuit", "//pkg/util/grpcutil", "//pkg/util/sysutil", diff --git a/pkg/jobs/joberror/errors.go b/pkg/jobs/joberror/errors.go index 96fb3858a3de..73fe2e5daacc 100644 --- a/pkg/jobs/joberror/errors.go +++ b/pkg/jobs/joberror/errors.go @@ -11,36 +11,16 @@ package joberror import ( - "strings" - circuitbreaker "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/sysutil" "github.com/cockroachdb/errors" ) -// IsDistSQLRetryableError returns true if the supplied error, or any of its parent -// causes is an rpc error. -// This is an unfortunate implementation that should be looking for a more -// specific error. -func IsDistSQLRetryableError(err error) bool { - if err == nil { - return false - } - - // TODO(knz): this is a bad implementation. Make it go away - // by avoiding string comparisons. - - errStr := err.Error() - // When a crdb node dies, any DistSQL flows with processors scheduled on - // it get an error with "rpc error" in the message from the call to - // `(*DistSQLPlanner).Run`. - return strings.Contains(errStr, `rpc error`) -} - // isBreakerOpenError returns true if err is a circuit.ErrBreakerOpen. // // NB: Two packages have ErrBreakerOpen error types. The cicruitbreaker package @@ -57,7 +37,7 @@ func IsPermanentBulkJobError(err error) bool { if err == nil { return false } - return !IsDistSQLRetryableError(err) && + return !sqlerrors.IsDistSQLRetryableError(err) && !grpcutil.IsClosedConnection(err) && !flowinfra.IsFlowRetryableError(err) && !flowinfra.IsNoInboundStreamConnectionError(err) && diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index ddefb975fadf..50cecc9f3426 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/buildutil" @@ -1998,7 +1999,8 @@ func (dsp *DistSQLPlanner) PlanAndRun( // cancellation has already occurred. return } - if !pgerror.IsSQLRetryableError(distributedErr) && + if !sqlerrors.IsDistSQLRetryableError(distributedErr) && + !pgerror.IsSQLRetryableError(distributedErr) && !flowinfra.IsFlowRetryableError(distributedErr) && !isDialErr(distributedErr) { // Only re-run the query if we think there is a high chance of a diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index 21b98bdb1b17..f9ad4ab17c37 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -12,6 +12,8 @@ package sqlerrors import ( + "strings" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -411,3 +413,22 @@ func errHasCode(err error, code ...pgcode.Code) bool { } return false } + +// IsDistSQLRetryableError returns true if the supplied error, or any of its parent +// causes is an rpc error. +// This is an unfortunate implementation that should be looking for a more +// specific error. +func IsDistSQLRetryableError(err error) bool { + if err == nil { + return false + } + + // TODO(knz): this is a bad implementation. Make it go away + // by avoiding string comparisons. + + errStr := err.Error() + // When a crdb node dies, any DistSQL flows with processors scheduled on + // it get an error with "rpc error" in the message from the call to + // `(*DistSQLPlanner).Run`. + return strings.Contains(errStr, `rpc error`) +} From a4005e31c3bb180404451a7b61bf0b97c9ef2020 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 9 Aug 2023 23:27:16 -0400 Subject: [PATCH 8/9] kv: unit test PrepareTransactionForRetry and TransactionRefreshTimestamp Informs #104233. This commit adds a pair of new unit tests to verify the behavior of `PrepareTransactionForRetry` and `TransactionRefreshTimestamp`. These functions will be getting more complex for #104233, so it will be helpful to have these tests in place. The tests also serve as good documentation. Release note: None --- pkg/kv/kvclient/kvcoord/txn_coord_sender.go | 9 +- pkg/kv/kvpb/BUILD.bazel | 2 + pkg/kv/kvpb/data.go | 31 +-- pkg/kv/kvpb/data_test.go | 233 ++++++++++++++++++++ 4 files changed, 257 insertions(+), 18 deletions(-) create mode 100644 pkg/kv/kvpb/data_test.go diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 0b928bde1d8e..57c24ce3e364 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -804,9 +804,7 @@ func (tc *TxnCoordSender) UpdateStateOnRemoteRetryableErr( // not be usable afterwards (in case of TransactionAbortedError). The caller is // expected to check the ID of the resulting transaction. If the TxnCoordSender // can still be used, it will have been prepared for a new epoch. -func (tc *TxnCoordSender) handleRetryableErrLocked( - ctx context.Context, pErr *kvpb.Error, -) *kvpb.TransactionRetryWithProtoRefreshError { +func (tc *TxnCoordSender) handleRetryableErrLocked(ctx context.Context, pErr *kvpb.Error) error { // If the error is a transaction retry error, update metrics to // reflect the reason for the restart. More details about the // different error types are documented above on the metaRestart @@ -842,7 +840,10 @@ func (tc *TxnCoordSender) handleRetryableErrLocked( tc.metrics.RestartsUnknown.Inc() } errTxnID := pErr.GetTxn().ID - newTxn := kvpb.PrepareTransactionForRetry(ctx, pErr, tc.mu.userPriority, tc.clock) + newTxn, assertErr := kvpb.PrepareTransactionForRetry(pErr, tc.mu.userPriority, tc.clock) + if assertErr != nil { + return assertErr + } // We'll pass a TransactionRetryWithProtoRefreshError up to the next layer. retErr := kvpb.NewTransactionRetryWithProtoRefreshError( diff --git a/pkg/kv/kvpb/BUILD.bazel b/pkg/kv/kvpb/BUILD.bazel index c89c653cf675..59f427d58c33 100644 --- a/pkg/kv/kvpb/BUILD.bazel +++ b/pkg/kv/kvpb/BUILD.bazel @@ -55,6 +55,7 @@ go_test( srcs = [ "api_test.go", "batch_test.go", + "data_test.go", "errors_test.go", "node_decommissioned_error_test.go", "replica_unavailable_error_test.go", @@ -74,6 +75,7 @@ go_test( "//pkg/util/buildutil", "//pkg/util/hlc", "//pkg/util/protoutil", + "//pkg/util/timeutil", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/kv/kvpb/data.go b/pkg/kv/kvpb/data.go index eb3471f7e7a6..9df6732ae722 100644 --- a/pkg/kv/kvpb/data.go +++ b/pkg/kv/kvpb/data.go @@ -12,11 +12,9 @@ package kvpb import ( - "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" ) // PrepareTransactionForRetry returns a new Transaction to be used for retrying @@ -34,14 +32,18 @@ import ( // In case retryErr tells us that a new Transaction needs to be created, // isolation and name help initialize this new transaction. func PrepareTransactionForRetry( - ctx context.Context, pErr *Error, pri roachpb.UserPriority, clock *hlc.Clock, -) roachpb.Transaction { + pErr *Error, pri roachpb.UserPriority, clock *hlc.Clock, +) (roachpb.Transaction, error) { + if pErr == nil { + return roachpb.Transaction{}, errors.AssertionFailedf("nil error") + } if pErr.TransactionRestart() == TransactionRestart_NONE { - log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) + return roachpb.Transaction{}, errors.AssertionFailedf( + "invalid retryable error (%T): %s", pErr.GetDetail(), pErr) } - if pErr.GetTxn() == nil { - log.Fatalf(ctx, "missing txn for retryable error: %s", pErr) + return roachpb.Transaction{}, errors.AssertionFailedf( + "missing txn for retryable error: %s", pErr) } txn := *pErr.GetTxn() @@ -108,19 +110,20 @@ func PrepareTransactionForRetry( // IntentMissingErrors are not expected to be handled at this level; // We instead expect the txnPipeliner to transform them into a // TransactionRetryErrors(RETRY_ASYNC_WRITE_FAILURE) error. - log.Fatalf( - ctx, "unexpected intent missing error (%T); should be transformed into retry error", pErr.GetDetail(), - ) + return roachpb.Transaction{}, errors.AssertionFailedf( + "unexpected intent missing error (%T); should be transformed into retry error", pErr.GetDetail()) default: - log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) + return roachpb.Transaction{}, errors.AssertionFailedf( + "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) } if !aborted { if txn.Status.IsFinalized() { - log.Fatalf(ctx, "transaction unexpectedly finalized in (%T): %s", pErr.GetDetail(), pErr) + return roachpb.Transaction{}, errors.AssertionFailedf( + "transaction unexpectedly finalized in (%T): %s", pErr.GetDetail(), pErr) } txn.Restart(pri, txn.Priority, txn.WriteTimestamp) } - return txn + return txn, nil } // TransactionRefreshTimestamp returns whether the supplied error is a retry diff --git a/pkg/kv/kvpb/data_test.go b/pkg/kv/kvpb/data_test.go new file mode 100644 index 000000000000..96eba928871f --- /dev/null +++ b/pkg/kv/kvpb/data_test.go @@ -0,0 +1,233 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvpb + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestPrepareTransactionForRetry(t *testing.T) { + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + tsClock := hlc.Timestamp{WallTime: 3} + txn := roachpb.MakeTransaction("test", nil, isolation.Serializable, -1, ts1, 0, 99) + txn2ID := uuid.MakeV4() // used if txn is aborted + tests := []struct { + name string + err *Error + expTxn roachpb.Transaction + expErr bool + }{ + { + name: "no error", + err: nil, + expErr: true, + }, + { + name: "no txn", + err: NewError(errors.New("random")), + expErr: true, + }, + { + name: "random error", + err: NewErrorWithTxn(errors.New("random"), &txn), + expErr: true, + }, + { + name: "txn aborted error", + err: NewErrorWithTxn(&TransactionAbortedError{}, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.ID = txn2ID + nextTxn.ReadTimestamp = tsClock + nextTxn.WriteTimestamp = tsClock + nextTxn.MinTimestamp = tsClock + nextTxn.LastHeartbeat = tsClock + nextTxn.GlobalUncertaintyLimit = tsClock + return nextTxn + }(), + }, + { + name: "read within uncertainty error", + err: NewErrorWithTxn(&ReadWithinUncertaintyIntervalError{ValueTimestamp: ts2}, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.Epoch++ + nextTxn.ReadTimestamp = ts2.Next() + nextTxn.WriteTimestamp = ts2.Next() + return nextTxn + }(), + }, + { + name: "txn push error", + err: NewErrorWithTxn(&TransactionPushError{ + PusheeTxn: roachpb.Transaction{TxnMeta: enginepb.TxnMeta{WriteTimestamp: ts2, Priority: 3}}, + }, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.Epoch++ + nextTxn.ReadTimestamp = ts2 + nextTxn.WriteTimestamp = ts2 + nextTxn.Priority = 2 + return nextTxn + }(), + }, + { + name: "txn retry error (reason: write too old)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_WRITE_TOO_OLD}, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.Epoch++ + return nextTxn + }(), + }, + { + name: "txn retry error (reason: serializable)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_SERIALIZABLE}, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.Epoch++ + nextTxn.ReadTimestamp = tsClock + nextTxn.WriteTimestamp = tsClock + return nextTxn + }(), + }, + { + name: "write too old error", + err: NewErrorWithTxn(&WriteTooOldError{ActualTimestamp: ts2}, &txn), + expTxn: func() roachpb.Transaction { + nextTxn := txn + nextTxn.Epoch++ + nextTxn.ReadTimestamp = ts2 + nextTxn.WriteTimestamp = ts2 + return nextTxn + }(), + }, + { + name: "intent missing error", + err: NewErrorWithTxn(&IntentMissingError{}, &txn), + expErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, tsClock.WallTime))) + nextTxn, err := PrepareTransactionForRetry(tt.err, -1 /* pri */, clock) + if tt.expErr { + require.Error(t, err) + require.True(t, errors.IsAssertionFailure(err)) + require.Zero(t, nextTxn) + } else { + require.NoError(t, err) + if nextTxn.ID != txn.ID { + // Eliminate randomness from ID generation. + nextTxn.ID = txn2ID + } + require.Equal(t, tt.expTxn, nextTxn) + } + }) + } +} + +func TestTransactionRefreshTimestamp(t *testing.T) { + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + txn := roachpb.MakeTransaction("test", nil, isolation.Serializable, 1, ts1, 0, 99) + tests := []struct { + name string + err *Error + expOk bool + expTs hlc.Timestamp + }{ + { + name: "no error", + err: nil, + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "no txn", + err: NewError(errors.New("random")), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "random error", + err: NewErrorWithTxn(errors.New("random"), &txn), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "txn aborted error", + err: NewErrorWithTxn(&TransactionAbortedError{}, &txn), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "txn retry error (reason: unknown)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_REASON_UNKNOWN}, &txn), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "txn retry error (reason: write too old)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_WRITE_TOO_OLD}, &txn), + expOk: true, + expTs: ts1, + }, + { + name: "txn retry error (reason: serializable)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_SERIALIZABLE}, &txn), + expOk: true, + expTs: ts1, + }, + { + name: "txn retry error (reason: async write failure)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_ASYNC_WRITE_FAILURE}, &txn), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "txn retry error (reason: commit deadline exceeded)", + err: NewErrorWithTxn(&TransactionRetryError{Reason: RETRY_COMMIT_DEADLINE_EXCEEDED}, &txn), + expOk: false, + expTs: hlc.Timestamp{}, + }, + { + name: "write too old error", + err: NewErrorWithTxn(&WriteTooOldError{ActualTimestamp: ts2}, &txn), + expOk: true, + expTs: ts2, + }, + { + name: "read within uncertainty error", + err: NewErrorWithTxn(&ReadWithinUncertaintyIntervalError{ValueTimestamp: ts2}, &txn), + expOk: true, + expTs: ts2.Next(), + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + ok, ts := TransactionRefreshTimestamp(tt.err) + require.Equal(t, tt.expOk, ok) + require.Equal(t, tt.expTs, ts) + }) + } +} From 49374d4f9efa5a98b91c9ab65a12eed43782f41c Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 9 Aug 2023 03:16:26 +0200 Subject: [PATCH 9/9] server,testutils: remove complexity There is a saying (paraphrasing) that it always takes more work removing unwanted complexity than it takes to add it. This is an example of that. Prior to this commit, there was an "interesting" propagation of the flag that decides whether or not to define a test tenant for test servers and clusters. In a nutshell, we had: - an "input" flag in `base.TestServerArgs`, which remained mostly immutable - a boolean decided once by `ShouldStartDefaultTestTenant()` either in: - `serverutils.StartServerOnlyE` - or `testcluster.Start` - that boolean choice was then propagated to `server.testServer` via _another_ boolean config flag in `server.BaseConfig` - both the 2nd boolean and the original input flag were then again checked when the time came to do the work (in `maybeStartDefaultTestTenant`). Additional complexity was then incurred by the need of `TestCluster` to make the determination just once (and not once per server). This commit cuts through all the layers of complexity by simply propagating the choice of `ShouldStartDefaultTestTenant()` back into the `TestServerArgs` and only ever reading from that subsequently. Release note: None --- pkg/base/test_server_args.go | 3 - pkg/ccl/backupccl/backup_test.go | 21 +++-- pkg/ccl/changefeedccl/changefeed_test.go | 26 ++++-- .../boundedstaleness_test.go | 4 +- .../kvfollowerreadsccl/followerreads_test.go | 9 +- .../multiregionccl/cold_start_latency_test.go | 6 +- pkg/ccl/multiregionccl/datadriven_test.go | 14 ++-- .../multiregionccltestutils/testutils.go | 16 ++-- pkg/server/config.go | 3 - pkg/server/multi_store_test.go | 7 +- pkg/server/testserver.go | 43 +++++----- pkg/sql/sqltestutils/telemetry.go | 8 +- pkg/testutils/lint/lint_test.go | 1 + pkg/testutils/serverutils/api.go | 15 +--- pkg/testutils/serverutils/test_server_shim.go | 67 ++++++++------- pkg/testutils/testcluster/testcluster.go | 83 ++++++++++--------- .../upgrademanager/manager_external_test.go | 5 +- 17 files changed, 177 insertions(+), 154 deletions(-) diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 38a5ffee2fd8..8dc4b6c3bc46 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -281,9 +281,6 @@ var ( // InternalNonDefaultDecision is a sentinel value used inside a // mechanism in serverutils. Should not be used by tests directly. - // - // TODO(#76378): Investigate how we can remove the need for this - // sentinel value. InternalNonDefaultDecision = DefaultTestTenantOptions{testBehavior: ttDisabled, allowAdditionalTenants: true} ) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index f9776de5e9cd..22faec4c7da6 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -337,9 +337,11 @@ func TestBackupRestorePartitioned(t *testing.T) { // Disabled to run within tenant as certain MR features are not available to tenants. args := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, + }, ServerArgsPerNode: map[int]base.TestServerArgs{ 0: { - DefaultTestTenant: base.TODOTestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "west"}, // NB: This has the same value as an az in the east region @@ -349,7 +351,6 @@ func TestBackupRestorePartitioned(t *testing.T) { }}, }, 1: { - DefaultTestTenant: base.TODOTestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "east"}, // NB: This has the same value as an az in the west region @@ -359,7 +360,6 @@ func TestBackupRestorePartitioned(t *testing.T) { }}, }, 2: { - DefaultTestTenant: base.TODOTestTenantDisabled, Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "region", Value: "east"}, {Key: "az", Value: "az2"}, @@ -490,34 +490,33 @@ func TestBackupRestoreExecLocality(t *testing.T) { // Disabled to run within tenant as certain MR features are not available to tenants. args := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, + }, ServerArgsPerNode: map[int]base.TestServerArgs{ 0: { - ExternalIODir: "/west0", - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: "/west0", Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "0"}, {Key: "region", Value: "west"}, }}, }, 1: { - ExternalIODir: "/west1", - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: "/west1", Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "1"}, {Key: "region", Value: "west"}, }}, }, 2: { - ExternalIODir: "/east0", - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: "/east0", Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "0"}, {Key: "region", Value: "east"}, }}, }, 3: { - ExternalIODir: "/east1", - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: "/east1", Locality: roachpb.Locality{Tiers: []roachpb.Tier{ {Key: "tier", Value: "1"}, {Key: "region", Value: "east"}, diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 4cdc8e419337..806af670a96f 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -5689,10 +5689,7 @@ func TestChangefeedHandlesRollingRestart(t *testing.T) { nodeDrainChannels[n].Store(make(chan struct{})) return base.TestServerArgs{ - // Test uses SPLIT AT, which isn't currently supported for - // secondary tenants. Tracked with #76378. - DefaultTestTenant: base.TODOTestTenantDisabled, - UseDatabase: "test", + UseDatabase: "test", Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ DrainFast: true, @@ -5770,6 +5767,11 @@ func TestChangefeedHandlesRollingRestart(t *testing.T) { } return perNode }(), + ServerArgs: base.TestServerArgs{ + // Test uses SPLIT AT, which isn't currently supported for + // secondary tenants. Tracked with #76378. + DefaultTestTenant: base.TODOTestTenantDisabled, + }, }) defer tc.Stopper().Stop(context.Background()) @@ -5882,9 +5884,6 @@ func TestChangefeedPropagatesTerminalError(t *testing.T) { perServerKnobs := make(map[int]base.TestServerArgs, numNodes) for i := 0; i < numNodes; i++ { perServerKnobs[i] = base.TestServerArgs{ - // Test uses SPLIT AT, which isn't currently supported for - // secondary tenants. Tracked with #76378. - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ DrainFast: true, @@ -5901,6 +5900,11 @@ func TestChangefeedPropagatesTerminalError(t *testing.T) { base.TestClusterArgs{ ServerArgsPerNode: perServerKnobs, ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + // Test uses SPLIT AT, which isn't currently supported for + // secondary tenants. Tracked with #76378. + DefaultTestTenant: base.TODOTestTenantDisabled, + }, }) defer tc.Stopper().Stop(context.Background()) @@ -8241,13 +8245,17 @@ func TestChangefeedExecLocality(t *testing.T) { str := strconv.Itoa const nodes = 4 - args := base.TestClusterArgs{ServerArgsPerNode: map[int]base.TestServerArgs{}} + args := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, // need nodelocal and splits. + }, + ServerArgsPerNode: map[int]base.TestServerArgs{}, + } for i := 0; i < nodes; i++ { args.ServerArgsPerNode[i] = base.TestServerArgs{ ExternalIODir: path.Join(dir, str(i)), Locality: roachpb.Locality{ Tiers: []roachpb.Tier{{Key: "x", Value: str(i / 2)}, {Key: "y", Value: str(i % 2)}}}, - DefaultTestTenant: base.TODOTestTenantDisabled, // need nodelocal and splits. } } diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index b0a06668a1a8..78d89de8d93c 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -268,6 +268,9 @@ func TestBoundedStalenessDataDriven(t *testing.T) { ctx := context.Background() clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, + }, ServerArgsPerNode: map[int]base.TestServerArgs{}, } const numNodes = 3 @@ -275,7 +278,6 @@ func TestBoundedStalenessDataDriven(t *testing.T) { for i := 0; i < numNodes; i++ { i := i clusterArgs.ServerArgsPerNode[i] = base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 91dfd4f6f205..10621aac8d80 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -714,8 +714,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { // Also, we're going to collect a trace of the test's final query. ServerArgsPerNode: map[int]base.TestServerArgs{ 3: { - DefaultTestTenant: base.TODOTestTenantDisabled, - UseDatabase: "t", + UseDatabase: "t", Knobs: base.TestingKnobs{ KVClient: &kvcoord.ClientTestingKnobs{ // Inhibit the checking of connection health done by the @@ -901,13 +900,15 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) { } localities[i] = locality serverArgs[i] = base.TestServerArgs{ - Locality: localities[i], - DefaultTestTenant: base.TODOTestTenantDisabled, // we'll create one ourselves below. + Locality: localities[i], } } tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgsPerNode: serverArgs, + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, // we'll create one ourselves below. + }, }) ctx := context.Background() defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/multiregionccl/cold_start_latency_test.go b/pkg/ccl/multiregionccl/cold_start_latency_test.go index 413602b73266..53a01d0e100f 100644 --- a/pkg/ccl/multiregionccl/cold_start_latency_test.go +++ b/pkg/ccl/multiregionccl/cold_start_latency_test.go @@ -79,8 +79,7 @@ func TestColdStartLatency(t *testing.T) { for i := 0; i < numNodes; i++ { i := i args := base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, - Locality: localities[i], + Locality: localities[i], } signalAfter[i] = make(chan struct{}) serverKnobs := &server.TestingKnobs{ @@ -120,6 +119,9 @@ func TestColdStartLatency(t *testing.T) { tc := testcluster.NewTestCluster(t, numNodes, base.TestClusterArgs{ ParallelStart: true, ServerArgsPerNode: perServerArgs, + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, + }, }) go func() { for _, c := range signalAfter { diff --git a/pkg/ccl/multiregionccl/datadriven_test.go b/pkg/ccl/multiregionccl/datadriven_test.go index 7ce93960ec57..9fd8fd293412 100644 --- a/pkg/ccl/multiregionccl/datadriven_test.go +++ b/pkg/ccl/multiregionccl/datadriven_test.go @@ -153,12 +153,6 @@ func TestMultiRegionDataDriven(t *testing.T) { } serverArgs[i] = base.TestServerArgs{ Locality: localityCfg, - // We need to disable the default test tenant here - // because it appears as though operations like - // "wait-for-zone-config-changes" only work correctly - // when called from the system tenant. More - // investigation is required (tracked with #76378). - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { @@ -175,6 +169,14 @@ func TestMultiRegionDataDriven(t *testing.T) { numServers := len(localityNames) tc := testcluster.StartTestCluster(t, numServers, base.TestClusterArgs{ ServerArgsPerNode: serverArgs, + ServerArgs: base.TestServerArgs{ + // We need to disable the default test tenant here + // because it appears as though operations like + // "wait-for-zone-config-changes" only work correctly + // when called from the system tenant. More + // investigation is required (tracked with #76378). + DefaultTestTenant: base.TODOTestTenantDisabled, + }, }) ds.tc = tc diff --git a/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go b/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go index 2bead565dc49..9ca9ecfcfb18 100644 --- a/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go +++ b/pkg/ccl/multiregionccl/multiregionccltestutils/testutils.go @@ -110,13 +110,6 @@ func TestingCreateMultiRegionClusterWithRegionList( Knobs: knobs, ExternalIODir: params.baseDir, UseDatabase: params.useDatabase, - // Disabling this due to failures in the rtt_analysis tests. Ideally - // we could disable multi-tenancy just for those tests, but this function - // is used to create the MR cluster for all test cases. For - // bonus points, the code to re-enable this should also provide more - // flexibility in disabling the default test tenant by callers of this - // function. Re-enablement is tracked with #76378. - DefaultTestTenant: base.TODOTestTenantDisabled, Locality: roachpb.Locality{ Tiers: []roachpb.Tier{{Key: "region", Value: region}}, }, @@ -128,6 +121,15 @@ func TestingCreateMultiRegionClusterWithRegionList( tc := testcluster.StartTestCluster(t, totalServerCount, base.TestClusterArgs{ ReplicationMode: params.replicationMode, ServerArgsPerNode: serverArgs, + ServerArgs: base.TestServerArgs{ + // Disabling this due to failures in the rtt_analysis tests. Ideally + // we could disable multi-tenancy just for those tests, but this function + // is used to create the MR cluster for all test cases. For + // bonus points, the code to re-enable this should also provide more + // flexibility in disabling the default test tenant by callers of this + // function. Re-enablement is tracked with #76378. + DefaultTestTenant: base.TODOTestTenantDisabled, + }, }) ctx := context.Background() diff --git a/pkg/server/config.go b/pkg/server/config.go index 8a29c24d4510..b918e38fb3f0 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -204,9 +204,6 @@ type BaseConfig struct { // Environment Variable: COCKROACH_DISABLE_SPAN_CONFIGS SpanConfigsDisabled bool - // Disables the default test tenant. - DisableDefaultTestTenant bool - // TestingKnobs is used for internal test controls only. TestingKnobs base.TestingKnobs diff --git a/pkg/server/multi_store_test.go b/pkg/server/multi_store_test.go index a5f5ea1c84e3..596b232dd007 100644 --- a/pkg/server/multi_store_test.go +++ b/pkg/server/multi_store_test.go @@ -62,11 +62,12 @@ func TestAddNewStoresToExistingNodes(t *testing.T) { // again explicitly. ReplicationMode: base.ReplicationAuto, ServerArgsPerNode: map[int]base.TestServerArgs{}, + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TODOTestTenantDisabled, + }, } for srvIdx := 0; srvIdx < numNodes; srvIdx++ { - serverArgs := base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, - } + serverArgs := base.TestServerArgs{} serverArgs.Knobs.Server = &server.TestingKnobs{StickyVFSRegistry: ser} for storeIdx := 0; storeIdx < numStoresPerNode; storeIdx++ { id := fmt.Sprintf("s%d.%d", srvIdx+1, storeIdx+1) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 903ace5744df..fa2ae1ee543b 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -308,9 +308,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { cfg.TempStorageConfig.Settings = st } - // TODO(#76378): Review this assignment to ensure it does not interfere with randomization. - cfg.DisableDefaultTestTenant = params.DefaultTestTenant.TestTenantAlwaysDisabled() - if cfg.TestingKnobs.Store == nil { cfg.TestingKnobs.Store = &kvserver.StoreTestingKnobs{} } @@ -569,16 +566,6 @@ func (ts *testServer) TestTenants() []serverutils.ApplicationLayerInterface { return ts.testTenants } -// DefaultTestTenantDisabled is part of the serverutils.TenantControlInterface. -func (ts *testServer) DefaultTestTenantDisabled() bool { - return ts.cfg.DisableDefaultTestTenant -} - -// DisableDefaultTestTenant is part of the serverutils.TenantControlInterface. -func (ts *testServer) DisableDefaultTestTenant() { - ts.cfg.DisableDefaultTestTenant = true -} - // maybeStartDefaultTestTenant might start a test tenant. This can then be used // for multi-tenant testing, where the default SQL connection will be made to // this tenant instead of to the system tenant. Note that we will @@ -586,9 +573,16 @@ func (ts *testServer) DisableDefaultTestTenant() { // enterprise enabled build. This is due to licensing restrictions on the MT // capabilities. func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { + if !(ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() || + ts.params.DefaultTestTenant.TestTenantAlwaysEnabled()) { + return errors.WithHint( + errors.AssertionFailedf("programming error: no decision taken about the default test tenant"), + "Maybe add the missing call to serverutils.ShouldStartDefaultTestTenant()?") + } + // If the flag has been set to disable the default test tenant, don't start // it here. - if ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() || ts.cfg.DisableDefaultTestTenant { + if ts.params.DefaultTestTenant.TestTenantAlwaysDisabled() { return nil } @@ -597,7 +591,10 @@ func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { log.Shoutf(ctx, severity.WARNING, "test tenant requested by configuration, but code organization prevents start!\n%v", err) // If not enterprise enabled, we won't be able to use SQL Servers so eat // the error and return without creating/starting a SQL server. - ts.cfg.DisableDefaultTestTenant = true + // + // TODO(knz/yahor): Remove this - as we discussed this ought to work + // now even when not enterprise enabled. + ts.params.DefaultTestTenant = base.TODOTestTenantDisabled return nil // nolint:returnerrcheck } @@ -668,7 +665,15 @@ func (ts *testServer) maybeStartDefaultTestTenant(ctx context.Context) error { // testServer.AdvRPCAddr() after Start() for client connections. // Use testServer.Stopper().Stop() to shutdown the server after the test // completes. -func (ts *testServer) Start(ctx context.Context) error { +func (ts *testServer) Start(ctx context.Context) (retErr error) { + defer func() { + if retErr != nil { + // Use a separate context to avoid using an already-cancelled + // context in closers. + ts.Stopper().Stop(context.Background()) + } + }() + if err := ts.topLevelServer.PreStart(ctx); err != nil { return err } @@ -681,16 +686,16 @@ func (ts *testServer) Start(ctx context.Context) error { ); err != nil { return err } + + // Let clients connect. if err := ts.topLevelServer.AcceptClients(ctx); err != nil { return err } if err := ts.maybeStartDefaultTestTenant(ctx); err != nil { - // We're failing the call to this function but we've already started - // the testServer above. Stop it here to avoid leaking the server. - ts.Stopper().Stop(context.Background()) return err } + go func() { // If the server requests a shutdown, do that simply by stopping the // stopper. diff --git a/pkg/sql/sqltestutils/telemetry.go b/pkg/sql/sqltestutils/telemetry.go index 411b109be117..40d5b7631ce1 100644 --- a/pkg/sql/sqltestutils/telemetry.go +++ b/pkg/sql/sqltestutils/telemetry.go @@ -147,7 +147,6 @@ func (tt *telemetryTest) Start(t *testing.T, serverArgs []base.TestServerArgs) { diagSrvURL := tt.diagSrv.URL() mapServerArgs := make(map[int]base.TestServerArgs, len(serverArgs)) for i, v := range serverArgs { - v.DefaultTestTenant = base.TestControlsTenantsExplicitly v.Knobs.Server = &server.TestingKnobs{ DiagnosticsTestingKnobs: diagnostics.TestingKnobs{ OverrideReportingURL: &diagSrvURL, @@ -159,7 +158,12 @@ func (tt *telemetryTest) Start(t *testing.T, serverArgs []base.TestServerArgs) { tt.cluster = serverutils.StartNewTestCluster( tt.t, len(serverArgs), - base.TestClusterArgs{ServerArgsPerNode: mapServerArgs}, + base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestControlsTenantsExplicitly, + }, + ServerArgsPerNode: mapServerArgs, + }, ) tt.server = tt.cluster.Server(0) tt.serverDB = tt.cluster.ServerConn(0) diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 19800bdc43a1..7ef6f766c493 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -2337,6 +2337,7 @@ func TestLint(t *testing.T) { ":!server/storage_api/decommission_test.go", ":!server/storage_api/health_test.go", ":!server/storage_api/rangelog_test.go", + ":!server/testserver.go", ":!sql/catalog/internal/catkv/catalog_reader_test.go", ":!sql/importer/import_processor_test.go", ":!sql/importer/import_stmt_test.go", diff --git a/pkg/testutils/serverutils/api.go b/pkg/testutils/serverutils/api.go index 055d96e1ff80..31033188d2dd 100644 --- a/pkg/testutils/serverutils/api.go +++ b/pkg/testutils/serverutils/api.go @@ -53,6 +53,9 @@ type TestServerInterface interface { // Start runs the server. This is pre-called by StartServer(). // It is provided for tests that use the TestServerFactory directly // (mostly 'cockroach demo'). + // + // For convenience, the caller can assume that Stop() has been called + // already if Start() fails with an error. Start(context.Context) error // Stop stops the server. This must be called at the end of a test @@ -438,18 +441,6 @@ type TenantControlInterface interface { // StartedDefaultTestTenant returns true if the server has started // the service for the default test tenant. StartedDefaultTestTenant() bool - - // DefaultTestTenantDisabled returns true if the server has disabled - // the service for the default test tenant. - // TODO(knz): Verify whether this accessor is needed. This should - // be simplified. - DefaultTestTenantDisabled() bool - - // DisableDefaultTestTenant prevents the server from starting the - // service for the default test tenant. - // TODO(knz): Verify whether this accessor is needed. This should - // be simplified. - DisableDefaultTestTenant() } // StorageLayerInterface defines accessors to the storage layer of a diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 99ce3b9b7bd1..b9f500abe0f5 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -43,9 +43,9 @@ import ( "github.com/cockroachdb/errors" ) -// DefaultTestTenantMessage is a message that is printed when a test is run +// defaultTestTenantMessage is a message that is printed when a test is run // with the default test tenant. This is useful for debugging test failures. -const DefaultTestTenantMessage = ` +const defaultTestTenantMessage = ` Test server was configured to route SQL queries to a secondary tenant (virtual cluster). If you are only seeing a test failure when this message appears, there may be a problem specific to cluster virtualization or multi-tenancy. @@ -59,26 +59,40 @@ var PreventStartTenantError = errors.New("attempting to manually start a server // ShouldStartDefaultTestTenant determines whether a default test tenant // should be started for test servers or clusters, to serve SQL traffic by -// default. -// This can be overridden either via the build tag `metamorphic_disable` +// default. It returns a new base.DefaultTestTenantOptions that reflects +// the decision that was taken. +// +// The decision can be overridden either via the build tag `metamorphic_disable` // or just for test tenants via COCKROACH_TEST_TENANT. -func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) bool { +// +// This function is included in package 'serverutils' instead of 'server.testServer' +// directly so that it only gets linked into test code (and to avoid a linter +// error that 'skip' must only be used in test code). +func ShouldStartDefaultTestTenant( + t TestLogger, baseArg base.DefaultTestTenantOptions, +) (retval base.DefaultTestTenantOptions) { + defer func() { + if !(retval.TestTenantAlwaysEnabled() || retval.TestTenantAlwaysDisabled()) { + panic(errors.AssertionFailedf("programming error: no decision was actually taken")) + } + }() + // Explicit cases for enabling or disabling the default test tenant. - if serverArgs.DefaultTestTenant.TestTenantAlwaysEnabled() { - return true + if baseArg.TestTenantAlwaysEnabled() { + return baseArg } - if serverArgs.DefaultTestTenant.TestTenantAlwaysDisabled() { - if issueNum, label := serverArgs.DefaultTestTenant.IssueRef(); issueNum != 0 { + if baseArg.TestTenantAlwaysDisabled() { + if issueNum, label := baseArg.IssueRef(); issueNum != 0 { t.Logf("cluster virtualization disabled due to issue: #%d (expected label: %s)", issueNum, label) } - return false + return baseArg } if skip.UnderBench() { // Until #83461 is resolved, we want to make sure that we don't use the // multi-tenant setup so that the comparison against old single-tenant // SHAs in the benchmarks is fair. - return false + return base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(83461) } // Obey the env override if present. @@ -87,7 +101,10 @@ func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) if err != nil { panic(err) } - return v + if v { + return base.TestTenantAlwaysEnabled + } + return base.InternalNonDefaultDecision } // Note: we ask the metamorphic framework for a "disable" value, instead @@ -95,9 +112,12 @@ func ShouldStartDefaultTestTenant(t TestLogger, serverArgs base.TestServerArgs) // more often than not and that is what we want. enabled := !util.ConstantWithMetamorphicTestBoolWithoutLogging("disable-test-tenant", false) if enabled && t != nil { - t.Log(DefaultTestTenantMessage) + t.Log(defaultTestTenantMessage) } - return enabled + if enabled { + return base.TestTenantAlwaysEnabled + } + return base.InternalNonDefaultDecision } var srvFactoryImpl TestServerFactory @@ -134,17 +154,9 @@ type TestFataler interface { // server configuration messages. func StartServerOnlyE(t TestLogger, params base.TestServerArgs) (TestServerInterface, error) { allowAdditionalTenants := params.DefaultTestTenant.AllowAdditionalTenants() - // Determine if we should probabilistically start a test tenant - // for this server. - startDefaultSQLServer := ShouldStartDefaultTestTenant(t, params) - if !startDefaultSQLServer { - // If we're told not to start a test tenant, set the - // disable flag explicitly. - // - // TODO(#76378): review the definition of params.DefaultTestTenant - // so we do not need this weird sentinel value. - params.DefaultTestTenant = base.InternalNonDefaultDecision - } + // Update the flags with the actual decision as to whether we should + // start the service for a default test tenant. + params.DefaultTestTenant = ShouldStartDefaultTestTenant(t, params.DefaultTestTenant) s, err := NewServer(params) if err != nil { @@ -154,14 +166,9 @@ func StartServerOnlyE(t TestLogger, params base.TestServerArgs) (TestServerInter ctx := context.Background() if err := s.Start(ctx); err != nil { - s.Stopper().Stop(ctx) return nil, err } - if s.StartedDefaultTestTenant() && t != nil { - t.Log(DefaultTestTenantMessage) - } - if !allowAdditionalTenants { s.DisableStartTenant(PreventStartTenantError) } diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index ff7c6babb0f8..1a659095b8fa 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -73,6 +73,8 @@ type TestCluster struct { serverArgs []base.TestServerArgs clusterArgs base.TestClusterArgs + defaultTestTenantOptions base.DefaultTestTenantOptions + t serverutils.TestFataler } @@ -276,6 +278,23 @@ func NewTestCluster( noLocalities = false } + // Find out how to do the default test tenant. + // The choice should be made by the top-level ServerArgs. + defaultTestTenantOptions := tc.clusterArgs.ServerArgs.DefaultTestTenant + // API check: verify that no non-default choice was made via per-server args, + // and inform the user otherwise. + for i := 0; i < nodes; i++ { + if args, ok := tc.clusterArgs.ServerArgsPerNode[i]; ok && + args.DefaultTestTenant != (base.DefaultTestTenantOptions{}) && + args.DefaultTestTenant != defaultTestTenantOptions { + tc.Stopper().Stop(context.Background()) + t.Fatalf("improper use of DefaultTestTenantOptions in per-server args: %v vs %v\n"+ + "Tip: use the top-level ServerArgs to set the default test tenant options.", + args.DefaultTestTenant, defaultTestTenantOptions) + } + } + tc.defaultTestTenantOptions = serverutils.ShouldStartDefaultTestTenant(t, defaultTestTenantOptions) + var firstListener net.Listener for i := 0; i < nodes; i++ { var serverArgs base.TestServerArgs @@ -359,22 +378,20 @@ func NewTestCluster( // in a separate thread and with ParallelStart enabled (otherwise it'll block // on waiting for init for the first server). func (tc *TestCluster) Start(t serverutils.TestFataler) { + defer func() { + if r := recover(); r != nil { + // Avoid a stopper leak. + tc.Stopper().Stop(context.Background()) + panic(r) + } + }() + nodes := len(tc.Servers) var errCh chan error if tc.clusterArgs.ParallelStart { errCh = make(chan error, nodes) } - // Determine if we should probabilistically start a test tenant for the - // cluster. We key off of the DisableDefaultTestTenant flag of the first - // server in the cluster since they should all be set to the same value - // (validated below). - probabilisticallyStartTestTenant := false - if !tc.Servers[0].DefaultTestTenantDisabled() { - probabilisticallyStartTestTenant = serverutils.ShouldStartDefaultTestTenant(t, tc.serverArgs[0]) - } - - startedTestTenant := true disableLBS := false for i := 0; i < nodes; i++ { // Disable LBS if any server has a very low scan interval. @@ -382,27 +399,13 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { disableLBS = true } - // If we're not probabilistically starting the test tenant, disable - // its start and set the "started" flag accordingly. We need to do this - // with two separate if checks because the DisableDefaultTestTenant flag - // could have been set coming into this function by the caller. - if !probabilisticallyStartTestTenant { - tc.Servers[i].DisableDefaultTestTenant() - } - if tc.Servers[i].DefaultTestTenantDisabled() { - if startedTestTenant && i > 0 { - t.Fatal(errors.Newf("starting only some nodes with a test tenant is not"+ - "currently supported - attempted to disable SQL sever on node %d", i)) - } - startedTestTenant = false - } - if tc.clusterArgs.ParallelStart { go func(i int) { errCh <- tc.startServer(i, tc.serverArgs[i]) }(i) } else { if err := tc.startServer(i, tc.serverArgs[i]); err != nil { + tc.Stopper().Stop(context.Background()) t.Fatal(err) } // We want to wait for stores for each server in order to have predictable @@ -412,16 +415,17 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { } } - if tc.StartedDefaultTestTenant() { - t.Log(serverutils.DefaultTestTenantMessage) - } - if tc.clusterArgs.ParallelStart { + var cerr error for i := 0; i < nodes; i++ { if err := <-errCh; err != nil { - t.Fatal(err) + cerr = errors.CombineErrors(cerr, err) } } + if cerr != nil { + tc.Stopper().Stop(context.Background()) + t.Fatal(cerr) + } tc.WaitForNStores(t, tc.NumServers(), tc.Servers[0].GossipI().(*gossip.Gossip)) } @@ -434,15 +438,13 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { ie := layer.InternalExecutor().(isql.Executor) if _, err := ie.Exec(context.Background(), "set-cluster-version", nil, /* txn */ `SET CLUSTER SETTING version = $1`, v.String()); err != nil { + tc.Stopper().Stop(context.Background()) t.Fatal(err) } } } - // No need to disable the merge queue for SQL servers, as they don't have - // access to that cluster setting (and ALTER TABLE ... SPLIT AT is not - // supported in SQL servers either). - if !startedTestTenant && tc.clusterArgs.ReplicationMode == base.ReplicationManual { + if tc.clusterArgs.ReplicationMode == base.ReplicationManual { // We've already disabled the merge queue via testing knobs above, but ALTER // TABLE ... SPLIT AT will throw an error unless we also disable merges via // the cluster setting. @@ -453,6 +455,7 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { InternalExecutor().(isql.Executor). Exec(context.Background(), "enable-merge-queue", nil, /* txn */ `SET CLUSTER SETTING kv.range_merge.queue_enabled = false`); err != nil { + tc.Stopper().Stop(context.Background()) t.Fatal(err) } } @@ -462,6 +465,7 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { InternalExecutor().(isql.Executor). Exec(context.Background(), "enable-split-by-load", nil, /*txn */ `SET CLUSTER SETTING kv.range_split.by_load_enabled = false`); err != nil { + tc.Stopper().Stop(context.Background()) t.Fatal(err) } } @@ -472,6 +476,7 @@ func (tc *TestCluster) Start(t serverutils.TestFataler) { if tc.clusterArgs.ReplicationMode == base.ReplicationAuto { if err := tc.WaitForFullReplication(); err != nil { + tc.Stopper().Stop(context.Background()) t.Fatal(err) } } @@ -602,17 +607,15 @@ func (tc *TestCluster) AddServer( serverArgs.Addr = serverArgs.Listener.Addr().String() } + // Inject the decision that was made about whether or not to start a + // test tenant server, into this new server's configuration. + serverArgs.DefaultTestTenant = tc.defaultTestTenantOptions + s, err := serverutils.NewServer(serverArgs) if err != nil { return nil, err } - // If we only allowed probabilistic starting of the test tenant, we disable - // starting additional tenants, even if we didn't start the test tenant. - if serverArgs.DefaultTestTenant == base.TestTenantProbabilisticOnly { - s.DisableStartTenant(serverutils.PreventStartTenantError) - } - tc.Servers = append(tc.Servers, s) tc.serverArgs = append(tc.serverArgs, serverArgs) diff --git a/pkg/upgrade/upgrademanager/manager_external_test.go b/pkg/upgrade/upgrademanager/manager_external_test.go index 5392ab8608a2..b61fe0c4c533 100644 --- a/pkg/upgrade/upgrademanager/manager_external_test.go +++ b/pkg/upgrade/upgrademanager/manager_external_test.go @@ -612,8 +612,6 @@ func TestPrecondition(t *testing.T) { ctx := context.Background() args := func() base.TestServerArgs { return base.TestServerArgs{ - DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(107397), - Knobs: knobs, Settings: cluster.MakeTestingClusterSettingsWithVersions( v2, // binaryVersion @@ -623,6 +621,9 @@ func TestPrecondition(t *testing.T) { } } tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(107397), + }, ServerArgsPerNode: map[int]base.TestServerArgs{ 0: args(), 1: args(),