From 5a71d3ef9d6c95ec4e89fbb4b91d819c6924f632 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 12:52:19 -0700 Subject: [PATCH 01/16] execinfra: allow tenants to disable the streamer Previously, we marked the setting that controls whether the streamer is used as `TenantReadOnly` since we were not sure whether the streamer fit well into the tenant cost model. Recently we revamped the cost model so that it can now correctly predict the usage of the hardware resources by the streamer, so at this point it seems safe to mark the setting `TenantWritable`. Release note: None --- pkg/sql/execinfra/readerbase.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/execinfra/readerbase.go b/pkg/sql/execinfra/readerbase.go index dd1c8ee20c6a..c5352129765e 100644 --- a/pkg/sql/execinfra/readerbase.go +++ b/pkg/sql/execinfra/readerbase.go @@ -229,7 +229,7 @@ func (flowCtx *FlowCtx) UseStreamer() (bool, *kv.Txn, error) { // useStreamerEnabled determines whether the Streamer API should be used. // TODO(yuzefovich): remove this in 23.1. var useStreamerEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.TenantWritable, "sql.distsql.use_streamer.enabled", "determines whether the usage of the Streamer API is allowed. "+ "Enabling this will increase the speed of lookup/index joins "+ From 0d47e12abdf08d6521a8cc81c52d2c25eb3511dc Mon Sep 17 00:00:00 2001 From: Marylia Gutierrez Date: Fri, 15 Jul 2022 18:09:13 -0400 Subject: [PATCH 02/16] ui: fix sorting of explain plans Previously, the sorting on the plans on the Explain Plans tab on Statement Details wasn't working. This commit adds the missing code required to sort that table. Fixes #84079 Release note (bug fix): Sorting on the plans table inside the Statement Details page is now properly working. --- .../planDetails/planDetails.tsx | 20 +++++++++++++++++-- .../src/statementDetails/statementDetails.tsx | 19 +++++++++++++----- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx index e7e04f8aa05b..220cb423d742 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/planDetails/planDetails.tsx @@ -18,12 +18,19 @@ import { } from "./plansTable"; import { Button } from "../../button"; import { SqlBox, SqlBoxSize } from "../../sql"; +import { SortSetting } from "../../sortedtable"; interface PlanDetailsProps { plans: PlanHashStats[]; + sortSetting: SortSetting; + onChangeSortSetting: (ss: SortSetting) => void; } -export function PlanDetails({ plans }: PlanDetailsProps): React.ReactElement { +export function PlanDetails({ + plans, + sortSetting, + onChangeSortSetting, +}: PlanDetailsProps): React.ReactElement { const [plan, setPlan] = useState(null); const handleDetails = (plan: PlanHashStats): void => { setPlan(plan); @@ -35,13 +42,20 @@ export function PlanDetails({ plans }: PlanDetailsProps): React.ReactElement { if (plan) { return renderExplainPlan(plan, backToPlanTable); } else { - return renderPlanTable(plans, handleDetails); + return renderPlanTable( + plans, + handleDetails, + sortSetting, + onChangeSortSetting, + ); } } function renderPlanTable( plans: PlanHashStats[], handleDetails: (plan: PlanHashStats) => void, + sortSetting: SortSetting, + onChangeSortSetting: (ss: SortSetting) => void, ): React.ReactElement { const columns = makeExplainPlanColumns(handleDetails); return ( @@ -49,6 +63,8 @@ function renderPlanTable( columns={columns} data={plans} className="statements-table" + sortSetting={sortSetting} + onChangeSortSetting={onChangeSortSetting} /> ); } diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx index a5c2ecb64c1f..5c493a9a1dc1 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx @@ -88,7 +88,7 @@ export type StatementDetailsProps = StatementDetailsOwnProps & RouteComponentProps<{ implicitTxn: string; statement: string }>; export interface StatementDetailsState { - sortSetting: SortSetting; + plansSortSetting: SortSetting; currentTab?: string; } @@ -210,10 +210,9 @@ export class StatementDetails extends React.Component< super(props); const searchParams = new URLSearchParams(props.history.location.search); this.state = { - sortSetting: { - // Latency + plansSortSetting: { ascending: false, - columnTitle: "statementTime", + columnTitle: "lastExecTime", }, currentTab: searchParams.get("tab") || "overview", }; @@ -347,6 +346,12 @@ export class StatementDetails extends React.Component< } }; + onChangePlansSortSetting = (ss: SortSetting): void => { + this.setState({ + plansSortSetting: ss, + }); + }; + render(): React.ReactElement { const { refreshStatementDiagnosticsRequests, @@ -733,7 +738,11 @@ export class StatementDetails extends React.Component<

- + ); From 3c0dc6f07d4edf5aeb169d7f01a4963945fc1815 Mon Sep 17 00:00:00 2001 From: Oleg Afanasyev Date: Wed, 1 Jun 2022 11:22:28 +0100 Subject: [PATCH 03/16] gc: take range tombstones into account in mvcc gc Previously GC didn't take range tombstones into account when removing old point data. This patch adds support for removal of data hidden by range tombstones. This change doesn't change the protocol that GC is using but will look on range tombstones when processing replica in GC queue. Point keys that are covered by range tombstones will be included in the GCRequest same way as keys covered by newer versions or point tombstones. Release note: None --- pkg/kv/kvserver/gc/BUILD.bazel | 1 + pkg/kv/kvserver/gc/data_distribution_test.go | 201 ++++++++- pkg/kv/kvserver/gc/gc.go | 71 ++- pkg/kv/kvserver/gc/gc_iterator.go | 152 +++++-- pkg/kv/kvserver/gc/gc_iterator_test.go | 156 +++++-- pkg/kv/kvserver/gc/gc_old_test.go | 5 +- pkg/kv/kvserver/gc/gc_random_test.go | 129 +++++- pkg/kv/kvserver/gc/gc_test.go | 407 +++++++++++++++--- pkg/kv/kvserver/mvcc_gc_queue.go | 4 +- pkg/kv/kvserver/rditer/replica_data_iter.go | 20 + .../kvserver/rditer/replica_data_iter_test.go | 56 ++- pkg/roachpb/api.proto | 8 + pkg/storage/mvcc.go | 116 ++++- pkg/storage/mvcc_test.go | 73 +++- 14 files changed, 1166 insertions(+), 233 deletions(-) diff --git a/pkg/kv/kvserver/gc/BUILD.bazel b/pkg/kv/kvserver/gc/BUILD.bazel index a5a5f7fdf7b3..4481965fe46a 100644 --- a/pkg/kv/kvserver/gc/BUILD.bazel +++ b/pkg/kv/kvserver/gc/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/settings", "//pkg/storage", "//pkg/storage/enginepb", + "//pkg/util", "//pkg/util/bufalloc", "//pkg/util/contextutil", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index e5b1e5738315..669aa2a9767c 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -32,10 +32,16 @@ import ( ) // dataDistribution is an abstraction for testing that represents a stream of -// MVCCKeyValues. The stream may indicate that a value is an intent by returning +// MVCCKeyValues and MVCCRangeKeyValues. Each call would return either point +// value or range tombstone, but not both. +// The stream may indicate that a point value is an intent by returning // a non-nil transaction. If an intent is returned it must have a higher // timestamp than any other version written for the key. -type dataDistribution func() (storage.MVCCKeyValue, *roachpb.Transaction, bool) +// Range key values could only be tombstones and can't have transaction or +// intent placed on them. +type dataDistribution func() ( + storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool, +) // setupTest writes the data from this distribution into eng. All data should // be a part of the range represented by desc. @@ -46,11 +52,19 @@ func (ds dataDistribution) setupTest( var maxTs hlc.Timestamp var ms enginepb.MVCCStats for { - kv, txn, ok := ds() + kv, rangeKV, txn, ok := ds() if !ok { break } - if txn == nil { + if rangeKey := rangeKV.RangeKey; len(rangeKey.StartKey) > 0 { + require.Nil(t, txn, "invalid test data, range can't use transaction") + require.Zero(t, len(kv.Key.Key), + "invalid test data, range can't be used together with value: key=%s, rangeKey=%s", + kv.Key.String(), rangeKey.String()) + err := storage.MVCCDeleteRangeUsingTombstone(ctx, eng, &ms, rangeKey.StartKey, + rangeKey.EndKey, rangeKey.Timestamp, hlc.ClockTimestamp{}, nil, nil, 1) + require.NoError(t, err, "failed to put delete range") + } else if txn == nil { if kv.Key.Timestamp.IsEmpty() { require.NoError(t, eng.PutUnversioned(kv.Key.Key, kv.Value)) } else { @@ -73,6 +87,9 @@ func (ds dataDistribution) setupTest( if !kv.Key.Timestamp.Less(maxTs) { maxTs = kv.Key.Timestamp } + if ts := rangeKV.RangeKey.Timestamp; !ts.Less(maxTs) { + maxTs = ts + } } require.NoError(t, eng.Flush()) snap := eng.NewSnapshot() @@ -82,6 +99,100 @@ func (ds dataDistribution) setupTest( return ms } +type dataFeedItem struct { + kv storage.MVCCKeyValue + rkv storage.MVCCRangeKeyValue + txn *roachpb.Transaction +} + +func (i *dataFeedItem) String() string { + if i.txn != nil { + return fmt.Sprintf("%s ! %s", i.kv.Key.String(), i.txn.ID.String()) + } + if len(i.kv.Key.Key) > 0 { + return i.kv.Key.String() + } + return i.rkv.RangeKey.String() +} + +// sortedDistribution consume provided distribution fully and produce a +// distribution with data ordered by timestamp. This distribution is helpful +// for range key tombstones as they must be placed on top of multiple existing +// point keys. +func sortedDistribution(dist dataDistribution) dataDistribution { + var allData []dataFeedItem + for { + kv, rkv, txn, ok := dist() + if !ok { + break + } + allData = append(allData, dataFeedItem{kv: kv, rkv: rkv, txn: txn}) + } + isPoint := func(d dataFeedItem) bool { + return len(d.kv.Key.Key) > 0 + } + meta := func(i int) (roachpb.Key, hlc.Timestamp, bool) { + if !isPoint(allData[i]) { + return allData[i].rkv.RangeKey.StartKey, allData[i].rkv.RangeKey.Timestamp, false + } + return allData[i].kv.Key.Key, allData[i].kv.Key.Timestamp, true + } + sort.Slice(allData, func(i, j int) bool { + ki, ti, pi := meta(i) + kj, tj, _ := meta(j) + switch ti.Compare(tj) { + case -1: + return true + case 1: + return false + } + switch ki.Compare(kj) { + case -1: + return true + case 1: + return false + } + return pi + }) + + var lastTs hlc.Timestamp + var lastIsPoint = true + for i, v := range allData { + switch { + case isPoint(v) && !lastIsPoint && v.kv.Key.Timestamp.LessEq(lastTs): + lastTs.WallTime++ + allData[i].kv.Key.Timestamp = lastTs + lastIsPoint = true + case isPoint(v) && lastIsPoint && v.kv.Key.Timestamp.Less(lastTs): + allData[i].kv.Key.Timestamp = lastTs + case !isPoint(v) && !lastIsPoint && v.rkv.RangeKey.Timestamp.LessEq(lastTs): + lastTs.WallTime++ + allData[i].rkv.RangeKey.Timestamp = lastTs + case !isPoint(v) && lastIsPoint && v.rkv.RangeKey.Timestamp.LessEq(lastTs): + lastTs.WallTime++ + allData[i].rkv.RangeKey.Timestamp = lastTs + lastIsPoint = false + default: + lastIsPoint = isPoint(v) + if lastIsPoint { + lastTs = v.kv.Key.Timestamp + } else { + lastTs = v.rkv.RangeKey.Timestamp + } + } + } + + return func() (storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool) { + if len(allData) == 0 { + return storage.MVCCKeyValue{}, storage.MVCCRangeKeyValue{}, nil, false + } + defer func() { + allData = allData[1:] + }() + return allData[0].kv, allData[0].rkv, allData[0].txn, true + } +} + // maxRetriesAllowed is limiting how many times we could retry when generating // keys and timestamps for objects that are restricted by some criteria (e.g. // keys are unique, timestamps shouldn't be duplicate in history, intents @@ -101,17 +212,16 @@ func newDataDistribution( versionsPerKey func() int, intentFrac float64, oldIntentFrac float64, // within intents(!) + rangeKeyFrac float64, totalKeys int, rng *rand.Rand, ) dataDistribution { - // TODO(ajwerner): provide a mechanism to control the rate of expired intents - // or the intent age. Such a knob would likely require decoupling intents from - // other keys. + rangeKeyDist := rangeKeyDistribution(keyDist) var ( // Remaining values (all versions of all keys together with intents). remaining = totalKeys // Key for the objects currently emitted (if versions are not empty). - key roachpb.Key + key, endKey roachpb.Key // Set of key.String() to avoid generating data for the same key multiple // times. seen = map[string]struct{}{} @@ -121,7 +231,7 @@ func newDataDistribution( hasIntent bool ) - generatePointKey := func() (nextKey roachpb.Key, keyTimestamps []hlc.Timestamp, hasIntent bool) { + generatePointKey := func() (nextKey, unusedEndKey roachpb.Key, keyTimestamps []hlc.Timestamp, hasIntent bool) { hasIntent = rng.Float64() < intentFrac oldIntent := hasIntent && rng.Float64() < oldIntentFrac for retries := 0; len(keyTimestamps) == 0; retries++ { @@ -186,14 +296,27 @@ func newDataDistribution( } retries = 0 } - return nextKey, keyTimestamps, hasIntent + return nextKey, nil, keyTimestamps, hasIntent + } + + generateRangeKey := func() (startKey, endKey roachpb.Key, timestamps []hlc.Timestamp, hasIntent bool) { + var ts hlc.Timestamp + for { + ts = tsDist() + if ts.Less(minIntentTs) { + break + } + } + timestamps = []hlc.Timestamp{ts} + startKey, endKey = rangeKeyDist() + return startKey, endKey, timestamps, false } - return func() (storage.MVCCKeyValue, *roachpb.Transaction, bool) { + return func() (storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool) { if remaining == 0 { // Throw away temp key data, because we reached the end of sequence. seen = nil - return storage.MVCCKeyValue{}, nil, false + return storage.MVCCKeyValue{}, storage.MVCCRangeKeyValue{}, nil, false } defer func() { remaining-- }() @@ -201,12 +324,28 @@ func newDataDistribution( // Loop because we can have duplicate keys or unacceptable values, in that // case we retry key from scratch. for len(timestamps) == 0 { - key, timestamps, hasIntent = generatePointKey() + if rng.Float64() < rangeKeyFrac { + key, endKey, timestamps, hasIntent = generateRangeKey() + } else { + key, endKey, timestamps, hasIntent = generatePointKey() + } } seen[string(key)] = struct{}{} } ts := timestamps[0] timestamps = timestamps[1:] + + if len(endKey) > 0 { + return storage.MVCCKeyValue{}, + storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: key, + EndKey: endKey, + Timestamp: ts, + }, + }, nil, true + } + var txn *roachpb.Transaction // On the last version, we generate a transaction as needed. if len(timestamps) == 0 && hasIntent { @@ -222,7 +361,7 @@ func newDataDistribution( return storage.MVCCKeyValue{ Key: storage.MVCCKey{Key: key, Timestamp: ts}, Value: valueDist().RawBytes, - }, txn, true + }, storage.MVCCRangeKeyValue{}, txn, true } } @@ -243,19 +382,30 @@ type uniformDistSpec struct { deleteFrac float64 keysPerValueMin, keysPerValueMax int intentFrac, oldIntentFrac float64 + rangeKeyFrac float64 } var _ distSpec = uniformDistSpec{} func (ds uniformDistSpec) dist(maxRows int, rng *rand.Rand) dataDistribution { + if ds.tsSecMinIntent <= ds.tsSecFrom && ds.rangeKeyFrac > 0 { + panic("min intent ts should be set if range key generation is needed") + } + if ds.tsSecOldIntentTo <= ds.tsSecMinIntent && ds.oldIntentFrac > 0 { + panic("old intent ts must be lower than min intent ts if old intents are enabled") + } return newDataDistribution( - uniformTimestampDistribution(ds.tsSecFrom*time.Second.Nanoseconds(), ds.tsSecTo*time.Second.Nanoseconds(), rng), + uniformTimestampDistribution(ds.tsSecFrom*time.Second.Nanoseconds(), + ds.tsSecTo*time.Second.Nanoseconds(), rng), hlc.Timestamp{WallTime: ds.tsSecMinIntent * time.Second.Nanoseconds()}, hlc.Timestamp{WallTime: ds.tsSecOldIntentTo * time.Second.Nanoseconds()}, - uniformTableStringKeyDistribution(ds.desc().StartKey.AsRawKey(), ds.keySuffixMin, ds.keySuffixMax, rng), + uniformTableStringKeyDistribution(ds.desc().StartKey.AsRawKey(), ds.keySuffixMin, + ds.keySuffixMax, rng), uniformValueStringDistribution(ds.valueLenMin, ds.valueLenMax, ds.deleteFrac, rng), uniformValuesPerKey(ds.keysPerValueMin, ds.keysPerValueMax, rng), - ds.intentFrac, ds.oldIntentFrac, + ds.intentFrac, + ds.oldIntentFrac, + ds.rangeKeyFrac, maxRows, rng, ) @@ -275,12 +425,12 @@ func (ds uniformDistSpec) String() string { "keySuffix=[%d,%d],"+ "valueLen=[%d,%d],"+ "keysPerValue=[%d,%d],"+ - "deleteFrac=%f,intentFrac=%f", + "deleteFrac=%f,intentFrac=%f,oldIntentFrac=%f,rangeFrac=%f", ds.tsSecFrom, ds.tsSecTo, ds.keySuffixMin, ds.keySuffixMax, ds.valueLenMin, ds.valueLenMax, ds.keysPerValueMin, ds.keysPerValueMax, - ds.deleteFrac, ds.intentFrac) + ds.deleteFrac, ds.intentFrac, ds.oldIntentFrac, ds.rangeKeyFrac) } // uniformTimestamp returns an hlc timestamp distribution with a wall time @@ -374,3 +524,16 @@ func uniformTableStringKeyDistribution( return encoding.EncodeBytesAscending(prefix[0:len(prefix):len(prefix)], []byte(key)) } } + +func rangeKeyDistribution(keyDist func() roachpb.Key) func() (roachpb.Key, roachpb.Key) { + return func() (roachpb.Key, roachpb.Key) { + k1 := keyDist() + k2 := keyDist() + for ; k1.Equal(k2); k2 = keyDist() { + } + if k1.Compare(k2) > 0 { + return k2, k1 + } + return k1, k2 + } +} diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index fc9b42d63291..151f6c756ad8 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -120,7 +121,7 @@ type Thresholder interface { // PureGCer is part of the GCer interface. type PureGCer interface { - GC(context.Context, []roachpb.GCRequest_GCKey) error + GC(context.Context, []roachpb.GCRequest_GCKey, []roachpb.GCRequest_GCRangeKey) error } // A GCer is an abstraction used by the MVCC GC queue to carry out chunked deletions. @@ -138,7 +139,11 @@ var _ GCer = NoopGCer{} func (NoopGCer) SetGCThreshold(context.Context, Threshold) error { return nil } // GC implements storage.GCer. -func (NoopGCer) GC(context.Context, []roachpb.GCRequest_GCKey) error { return nil } +func (NoopGCer) GC( + context.Context, []roachpb.GCRequest_GCKey, []roachpb.GCRequest_GCRangeKey, +) error { + return nil +} // Threshold holds the key and txn span GC thresholds, respectively. type Threshold struct { @@ -309,8 +314,10 @@ func processReplicatedKeyRange( // Compute intent expiration (intent age at which we attempt to resolve). intentExp := now.Add(-intentAgeThreshold.Nanoseconds(), 0) - batcher := newIntentBatcher(cleanupIntentsFn, options, info) + intentBatcher := newIntentBatcher(cleanupIntentsFn, options, info) + // handleIntent will deserialize transaction info and if intent is older than + // threshold enqueue it to batcher, otherwise ignore it. handleIntent := func(keyValue *storage.MVCCKeyValue) error { meta := &enginepb.MVCCMetadata{} if err := protoutil.Unmarshal(keyValue.Value, meta); err != nil { @@ -322,7 +329,7 @@ func processReplicatedKeyRange( // expiration threshold. if meta.Timestamp.ToTimestamp().Less(intentExp) { info.IntentsConsidered++ - if err := batcher.addAndMaybeFlushIntents(ctx, keyValue.Key.Key, meta); err != nil { + if err := intentBatcher.addAndMaybeFlushIntents(ctx, keyValue.Key.Key, meta); err != nil { if errors.Is(err, ctx.Err()) { return err } @@ -349,7 +356,7 @@ func processReplicatedKeyRange( gcTimestampForThisKey hlc.Timestamp sentBatchForThisKey bool ) - it := makeGCIterator(desc, snap) + it := makeGCIterator(desc, snap, threshold) defer it.close() for ; ; it.step() { s, ok := it.state() @@ -368,8 +375,9 @@ func processReplicatedKeyRange( } continue } - isNewest := s.curIsNewest() - if isGarbage(threshold, s.cur, s.next, isNewest) { + // No more values in buffer or next value has different key. + isNewestPoint := s.curIsNewest() + if isGarbage(threshold, s.cur, s.next, isNewestPoint, s.firstRangeTombstoneTsAtOrBelowGC) { keyBytes := int64(s.cur.Key.EncodedSize()) batchGCKeysBytes += keyBytes haveGarbageForThisKey = true @@ -377,7 +385,10 @@ func processReplicatedKeyRange( info.AffectedVersionsKeyBytes += keyBytes info.AffectedVersionsValBytes += int64(len(s.cur.Value)) } - if affected := isNewest && (sentBatchForThisKey || haveGarbageForThisKey); affected { + // We bump how many keys were processed when we reach newest key and looking + // if key has garbage or if garbage for this key was included in previous + // batch. + if affected := isNewestPoint && (sentBatchForThisKey || haveGarbageForThisKey); affected { info.NumKeysAffected++ // If we reached newest timestamp for the key then we should reset sent // batch to ensure subsequent keys are not included in affected keys if @@ -385,7 +396,7 @@ func processReplicatedKeyRange( sentBatchForThisKey = false } shouldSendBatch := batchGCKeysBytes >= KeyVersionChunkBytes - if shouldSendBatch || isNewest && haveGarbageForThisKey { + if shouldSendBatch || isNewestPoint && haveGarbageForThisKey { alloc, s.cur.Key.Key = alloc.Copy(s.cur.Key.Key, 0) batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{ Key: s.cur.Key.Key, @@ -398,10 +409,11 @@ func processReplicatedKeyRange( // even if it turns out that there's no more garbage for this key. // We want to count a key as affected once even if we paginate the // deletion of its versions. - sentBatchForThisKey = shouldSendBatch && !isNewest + sentBatchForThisKey = shouldSendBatch && !isNewestPoint } + // If limit was reached, delegate to GC'r to remove collected batch. if shouldSendBatch { - if err := gcer.GC(ctx, batchGCKeys); err != nil { + if err := gcer.GC(ctx, batchGCKeys, nil); err != nil { if errors.Is(err, ctx.Err()) { return err } @@ -417,14 +429,14 @@ func processReplicatedKeyRange( } } // We need to send out last intent cleanup batch. - if err := batcher.maybeFlushPendingIntents(ctx); err != nil { + if err := intentBatcher.maybeFlushPendingIntents(ctx); err != nil { if errors.Is(err, ctx.Err()) { return err } log.Warningf(ctx, "failed to cleanup intents batch: %v", err) } if len(batchGCKeys) > 0 { - if err := gcer.GC(ctx, batchGCKeys); err != nil { + if err := gcer.GC(ctx, batchGCKeys, nil); err != nil { return err } } @@ -551,10 +563,13 @@ func (b *intentBatcher) maybeFlushPendingIntents(ctx context.Context) error { return err } -// isGarbage makes a determination whether a key ('cur') is garbage. If 'next' -// is non-nil, it should be the chronologically newer version of the same key -// (or the metadata KV if cur is an intent). If isNewest is false, next must be -// non-nil. isNewest implies that this is the highest timestamp committed +// isGarbage makes a determination whether a key ('cur') is garbage. +// If its timestamp is below firstRangeTombstoneTsAtOrBelowGC then all versions +// were deleted by the range key regardless if they are isNewestPoint or not. +// +// If 'next' is non-nil, it should be the chronologically newer version of the +// same key (or the metadata KV if cur is an intent). If isNewest is false, next +// must be non-nil. isNewest implies that this is the highest timestamp committed // version for this key. If isNewest is true and next is non-nil, it is an // intent. Conservatively we have to assume that the intent will get aborted, // so we will be able to GC just the values that we could remove if there @@ -565,19 +580,33 @@ func (b *intentBatcher) maybeFlushPendingIntents(ctx context.Context) error { // guaranteed as described above. However if this were the only rule, then if // the most recent write was a delete, it would never be removed. Thus, when a // deleted value is the most recent before expiration, it can be deleted. -func isGarbage(threshold hlc.Timestamp, cur, next *storage.MVCCKeyValue, isNewest bool) bool { +func isGarbage( + threshold hlc.Timestamp, + cur, next *storage.MVCCKeyValue, + isNewestPoint bool, + firstRangeTombstoneTsAtOrBelowGC hlc.Timestamp, +) bool { // If the value is not at or below the threshold then it's not garbage. if belowThreshold := cur.Key.Timestamp.LessEq(threshold); !belowThreshold { return false } + if cur.Key.Timestamp.Less(firstRangeTombstoneTsAtOrBelowGC) { + if util.RaceEnabled { + if threshold.Less(firstRangeTombstoneTsAtOrBelowGC) { + panic(fmt.Sprintf("gc attempt to remove key: using range tombstone %s above gc threshold %s", + firstRangeTombstoneTsAtOrBelowGC.String(), threshold.String())) + } + } + return true + } isDelete := len(cur.Value) == 0 - if isNewest && !isDelete { + if isNewestPoint && !isDelete { return false } // If this value is not a delete, then we need to make sure that the next // value is also at or below the threshold. // NB: This doesn't need to check whether next is nil because we know - // isNewest is false when evaluating rhs of the or below. + // isNewestPoint is false when evaluating rhs of the or below. if !isDelete && next == nil { panic("huh") } @@ -736,7 +765,7 @@ func (b *batchingInlineGCer) FlushingAdd(ctx context.Context, key roachpb.Key) { } func (b *batchingInlineGCer) Flush(ctx context.Context) { - err := b.gcer.GC(ctx, b.gcKeys) + err := b.gcer.GC(ctx, b.gcKeys, nil) b.gcKeys = nil b.size = 0 if err != nil { diff --git a/pkg/kv/kvserver/gc/gc_iterator.go b/pkg/kv/kvserver/gc/gc_iterator.go index 8cabe12627d6..e5c60e38bdd7 100644 --- a/pkg/kv/kvserver/gc/gc_iterator.go +++ b/pkg/kv/kvserver/gc/gc_iterator.go @@ -11,29 +11,46 @@ package gc import ( + "sort" + "strings" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // gcIterator wraps an rditer.ReplicaMVCCDataIterator which it reverse iterates for // the purpose of discovering gc-able replicated data. type gcIterator struct { - it *rditer.ReplicaMVCCDataIterator - done bool - err error - buf gcIteratorRingBuf + it *rditer.ReplicaMVCCDataIterator + threshold hlc.Timestamp + done bool + err error + buf gcIteratorRingBuf + + // Range tombstone timestamp caching to avoid recomputing timestamp for every + // object covered by current range key. + cachedRangeTombstoneTS hlc.Timestamp + cachedRangeTombstoneKey roachpb.Key } -func makeGCIterator(desc *roachpb.RangeDescriptor, snap storage.Reader) gcIterator { +func makeGCIterator( + desc *roachpb.RangeDescriptor, snap storage.Reader, threshold hlc.Timestamp, +) gcIterator { return gcIterator{ - it: rditer.NewReplicaMVCCDataIterator(desc, snap, true /* seekEnd */), + it: rditer.NewReplicaMVCCDataIterator(desc, snap, true /* seekEnd */), + threshold: threshold, } } type gcIteratorState struct { + // Sequential elements in iteration order (oldest to newest). cur, next, afterNext *storage.MVCCKeyValue + // Optional timestamp of the first available range tombstone at or below the + // GC threshold for the cur key. + firstRangeTombstoneTsAtOrBelowGC hlc.Timestamp } // curIsNewest returns true if the current MVCCKeyValue in the gcIteratorState @@ -57,8 +74,42 @@ func (s *gcIteratorState) curIsIntent() bool { return s.next != nil && !s.next.Key.IsValue() } +func kVString(v *storage.MVCCKeyValue) string { + b := strings.Builder{} + if v != nil { + b.WriteString(v.Key.String()) + if len(v.Value) == 0 { + b.WriteString(" del") + } + } else { + b.WriteString("") + } + return b.String() +} + +// String implements Stringer for debugging purposes. +func (s *gcIteratorState) String() string { + b := strings.Builder{} + add := func(v *storage.MVCCKeyValue, last bool) { + b.WriteString(kVString(v)) + if !last { + b.WriteString(", ") + } + } + add(s.cur, false) + add(s.next, false) + add(s.afterNext, true) + if ts := s.firstRangeTombstoneTsAtOrBelowGC; !ts.IsEmpty() { + b.WriteString(" rts@") + b.WriteString(ts.String()) + } + return b.String() +} + // state returns the current state of the iterator. The state contains the -// current and the two following versions of the current key if they exist. +// current and the two following versions of the current key if they exist +// as well as an optional timestamp of the first range tombstone covering +// current key at or below GC threshold. // // If ok is false, further iteration is unsafe; either the end of iteration has // been reached or an error has occurred. Callers should check it.err to @@ -70,11 +121,11 @@ func (it *gcIterator) state() (s gcIteratorState, ok bool) { // The current key is the newest if the key which comes next is different or // the key which comes after the current key is an intent or this is the first // key in the range. - s.cur, ok = it.peekAt(0) + s.cur, s.firstRangeTombstoneTsAtOrBelowGC, ok = it.peekAt(0) if !ok { return gcIteratorState{}, false } - next, ok := it.peekAt(1) + next, _, ok := it.peekAt(1) if !ok && it.err != nil { // cur is the first key in the range return gcIteratorState{}, false } @@ -82,7 +133,7 @@ func (it *gcIterator) state() (s gcIteratorState, ok bool) { return s, true } s.next = next - afterNext, ok := it.peekAt(2) + afterNext, _, ok := it.peekAt(2) if !ok && it.err != nil { // cur is the first key in the range return gcIteratorState{}, false } @@ -97,13 +148,16 @@ func (it *gcIterator) step() { it.buf.removeFront() } -func (it *gcIterator) peekAt(i int) (*storage.MVCCKeyValue, bool) { +// peekAt returns key value and a ts of first range tombstone less or equal +// to gc threshold. +func (it *gcIterator) peekAt(i int) (*storage.MVCCKeyValue, hlc.Timestamp, bool) { if it.buf.len <= i { if !it.fillTo(i + 1) { - return nil, false + return nil, hlc.Timestamp{}, false } } - return it.buf.at(i), true + kv, rangeTs := it.buf.at(i) + return kv, rangeTs, true } func (it *gcIterator) fillTo(targetLen int) (ok bool) { @@ -112,12 +166,41 @@ func (it *gcIterator) fillTo(targetLen int) (ok bool) { it.err, it.done = err, err == nil return false } - it.buf.pushBack(it.it) + if hasPoint, hasRange := it.it.HasPointAndRange(); hasPoint { + ts := hlc.Timestamp{} + if hasRange { + ts = it.currentRangeTS() + } + it.buf.pushBack(it.it.UnsafeKey(), it.it.UnsafeValue(), ts) + } it.it.Prev() } return true } +// currentRangeTS returns timestamp of the first range tombstone at or below +// gc threshold for current key. it also updates cached value to avoid +// recomputation for every key and version by checking current range bounds +// start key. +// Note: should only be called if HasPointAndRange() indicated that we have +// a range key. +func (it *gcIterator) currentRangeTS() hlc.Timestamp { + rangeTombstoneStartKey := it.it.RangeBounds().Key + if rangeTombstoneStartKey.Equal(it.cachedRangeTombstoneKey) { + return it.cachedRangeTombstoneTS + } + + it.cachedRangeTombstoneTS = hlc.Timestamp{} + rangeKeys := it.it.RangeKeys() + if idx := sort.Search(len(rangeKeys), func(i int) bool { + return rangeKeys[i].RangeKey.Timestamp.LessEq(it.threshold) + }); idx < len(rangeKeys) { + it.cachedRangeTombstoneTS = rangeKeys[idx].RangeKey.Timestamp + } + it.cachedRangeTombstoneKey = append(it.cachedRangeTombstoneKey[:0], rangeTombstoneStartKey...) + return it.cachedRangeTombstoneTS +} + func (it *gcIterator) close() { it.it.Close() it.it = nil @@ -130,15 +213,37 @@ const gcIteratorRingBufSize = 3 type gcIteratorRingBuf struct { allocs [gcIteratorRingBufSize]bufalloc.ByteAllocator buf [gcIteratorRingBufSize]storage.MVCCKeyValue - len int - head int + // If there are any range tombstones available for the key, this buffer will + // contain ts of first range at or below gc threshold. Otherwise, it'll be an + // empty timestamp. + firstRangeTombstoneAtOrBelowGCTss [gcIteratorRingBufSize]hlc.Timestamp + len int + head int +} + +func (b *gcIteratorRingBuf) String() string { + sb := strings.Builder{} + ptr := b.head + for i := 0; i < b.len; i++ { + sb.WriteString(kVString(&b.buf[ptr])) + if ts := b.firstRangeTombstoneAtOrBelowGCTss[ptr]; !ts.IsEmpty() { + sb.WriteString(" trs@") + sb.WriteString(b.firstRangeTombstoneAtOrBelowGCTss[ptr].String()) + } + if i < b.len-1 { + sb.WriteString(", ") + } + ptr = (ptr + 1) % gcIteratorRingBufSize + } + return sb.String() } -func (b *gcIteratorRingBuf) at(i int) *storage.MVCCKeyValue { +func (b *gcIteratorRingBuf) at(i int) (*storage.MVCCKeyValue, hlc.Timestamp) { if i >= b.len { panic("index out of range") } - return &b.buf[(b.head+i)%gcIteratorRingBufSize] + idx := (b.head + i) % gcIteratorRingBufSize + return &b.buf[idx], b.firstRangeTombstoneAtOrBelowGCTss[idx] } func (b *gcIteratorRingBuf) removeFront() { @@ -146,28 +251,23 @@ func (b *gcIteratorRingBuf) removeFront() { panic("cannot remove from empty gcIteratorRingBuf") } b.buf[b.head] = storage.MVCCKeyValue{} + b.firstRangeTombstoneAtOrBelowGCTss[b.head] = hlc.Timestamp{} b.head = (b.head + 1) % gcIteratorRingBufSize b.len-- } -type iterator interface { - UnsafeKey() storage.MVCCKey - UnsafeValue() []byte -} - -func (b *gcIteratorRingBuf) pushBack(it iterator) { +func (b *gcIteratorRingBuf) pushBack(k storage.MVCCKey, v []byte, rangeTS hlc.Timestamp) { if b.len == gcIteratorRingBufSize { panic("cannot add to full gcIteratorRingBuf") } i := (b.head + b.len) % gcIteratorRingBufSize b.allocs[i] = b.allocs[i].Truncate() - k := it.UnsafeKey() - v := it.UnsafeValue() b.allocs[i], k.Key = b.allocs[i].Copy(k.Key, len(v)) b.allocs[i], v = b.allocs[i].Copy(v, 0) b.buf[i] = storage.MVCCKeyValue{ Key: k, Value: v, } + b.firstRangeTombstoneAtOrBelowGCTss[i] = rangeTS b.len++ } diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index 4f450e22e88a..77f049ba8da0 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -11,7 +11,6 @@ package gc import ( - "fmt" "math/rand" "testing" "time" @@ -32,32 +31,47 @@ func TestGCIterator(t *testing.T) { // dataItem represents a version in the storage engine and optionally a // corresponding transaction which will make the MVCCKeyValue an intent. type dataItem struct { - storage.MVCCKeyValue + kv storage.MVCCKeyValue txn *roachpb.Transaction + rkv storage.MVCCRangeKeyValue + } + makeTS := func(ts int64) hlc.Timestamp { + return hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()} } // makeDataItem is a shorthand to construct dataItems. makeDataItem := func(k roachpb.Key, val roachpb.Value, ts int64, txn *roachpb.Transaction) dataItem { return dataItem{ - MVCCKeyValue: storage.MVCCKeyValue{ + kv: storage.MVCCKeyValue{ Key: storage.MVCCKey{ Key: k, - Timestamp: hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()}, + Timestamp: makeTS(ts), }, Value: val.RawBytes, }, txn: txn, } } + makeRangeTombstone := func(start, end roachpb.Key, ts int64) dataItem { + return dataItem{ + rkv: storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: start, + EndKey: end, + Timestamp: makeTS(ts), + }, + }, + } + } // makeLiteralDistribution adapts dataItems for use with the data distribution // infrastructure. makeLiteralDataDistribution := func(items ...dataItem) dataDistribution { - return func() (storage.MVCCKeyValue, *roachpb.Transaction, bool) { + return func() (storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool) { if len(items) == 0 { - return storage.MVCCKeyValue{}, nil, false + return storage.MVCCKeyValue{}, storage.MVCCRangeKeyValue{}, nil, false } item := items[0] defer func() { items = items[1:] }() - return item.MVCCKeyValue, item.txn, true + return item.kv, item.rkv, item.txn, true } } // stateExpectations are expectations about the state of the iterator. @@ -66,6 +80,7 @@ func TestGCIterator(t *testing.T) { isNewest bool isIntent bool isNotValue bool + rangeTombstoneTS hlc.Timestamp } // notation to mark that an iterator state element as either nil or metadata. const ( @@ -73,17 +88,23 @@ func TestGCIterator(t *testing.T) { isMD = -2 ) // exp is a shorthand to construct state expectations. - exp := func(cur, next, afterNext int, isNewest, isIntent, isNotValue bool) stateExpectations { + exp := func(cur, next, afterNext int, isNewest, isIntent, isNotValue bool, + tombstoneTS hlc.Timestamp, + ) stateExpectations { return stateExpectations{ cur: cur, next: next, afterNext: afterNext, - isNewest: isNewest, - isIntent: isIntent, - isNotValue: isNotValue, + isNewest: isNewest, + isIntent: isIntent, + isNotValue: isNotValue, + rangeTombstoneTS: tombstoneTS, } } vals := uniformValueDistribution(3, 5, 0, rand.New(rand.NewSource(1))) tablePrefix := keys.SystemSQLCodec.TablePrefix(42) - desc := roachpb.RangeDescriptor{StartKey: roachpb.RKey(tablePrefix), EndKey: roachpb.RKey(tablePrefix.PrefixEnd())} + desc := roachpb.RangeDescriptor{ + StartKey: roachpb.RKey(tablePrefix), + EndKey: roachpb.RKey(tablePrefix.PrefixEnd()), + } keyA := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'a') keyB := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'b') keyC := append(tablePrefix[0:len(tablePrefix):len(tablePrefix)], 'c') @@ -99,27 +120,29 @@ func TestGCIterator(t *testing.T) { name string data []dataItem expectations []stateExpectations + gcThreshold hlc.Timestamp } // checkExpectations tests whether the state of the iterator matches the // expectation. checkExpectations := func( - t *testing.T, data []dataItem, ex stateExpectations, s gcIteratorState, + t *testing.T, step int, data []dataItem, ex stateExpectations, s gcIteratorState, ) { - check := func(ex int, kv *storage.MVCCKeyValue) { + check := func(ex int, role string, kv *storage.MVCCKeyValue) { switch { case ex >= 0: - require.EqualValues(t, &data[ex].MVCCKeyValue, kv) + require.EqualValues(t, &data[ex].kv, kv, "step %d: unexpected data for %s at index %d", step, role, ex) case ex == isNil: - require.Nil(t, kv) + require.Nil(t, kv, "step %d", step) case ex == isMD: - require.False(t, kv.Key.IsValue()) + require.False(t, kv.Key.IsValue(), "step %d: expected metadata, found value", step) } } - check(ex.cur, s.cur) - check(ex.next, s.next) - check(ex.afterNext, s.afterNext) - require.Equal(t, ex.isNewest, s.curIsNewest()) - require.Equal(t, ex.isIntent, s.curIsIntent()) + check(ex.cur, "cur", s.cur) + check(ex.next, "next", s.next) + check(ex.afterNext, "after", s.afterNext) + require.Equal(t, ex.rangeTombstoneTS, s.firstRangeTombstoneTsAtOrBelowGC, "step %d: unexpected last tombstone timestamp", step) + require.Equal(t, ex.isNewest, s.curIsNewest(), "step %d: is newest", step) + require.Equal(t, ex.isIntent, s.curIsIntent(), "step %d: is intent", step) } makeTest := func(tc testCase) func(t *testing.T) { return func(t *testing.T) { @@ -129,20 +152,21 @@ func TestGCIterator(t *testing.T) { ds.setupTest(t, eng, desc) snap := eng.NewSnapshot() defer snap.Close() - it := makeGCIterator(&desc, snap) + it := makeGCIterator(&desc, snap, tc.gcThreshold) defer it.close() expectations := tc.expectations for i, ex := range expectations { - t.Run(fmt.Sprint(i), func(t *testing.T) { - s, ok := it.state() - require.True(t, ok) - checkExpectations(t, tc.data, ex, s) - }) + s, ok := it.state() + require.True(t, ok, "failed to get next state on step %d", i) + checkExpectations(t, i, tc.data, ex, s) it.step() } } } - di := makeDataItem // shorthand for convenient notation + // Shorthands for convenient notation. + noTS := hlc.Timestamp{} + di := makeDataItem + rts := makeRangeTombstone for _, tc := range []testCase{ { name: "basic", @@ -154,13 +178,75 @@ func TestGCIterator(t *testing.T) { di(keyC, vals(), 7, makeTxn()), }, expectations: []stateExpectations{ - exp(4, isMD, isNil, false, true, false), - exp(isMD, isNil, isNil, false, false, true), - exp(3, isNil, isNil, true, false, false), - exp(0, 1, 2, false, false, false), - exp(1, 2, isNil, false, false, false), - exp(2, isNil, isNil, true, false, false), + exp(4, isMD, isNil, false, true, false, noTS), + exp(isMD, isNil, isNil, false, false, true, noTS), + exp(3, isNil, isNil, true, false, false, noTS), + exp(0, 1, 2, false, false, false, noTS), + exp(1, 2, isNil, false, false, false, noTS), + exp(2, isNil, isNil, true, false, false, noTS), + }, + }, + { + name: "range tombstones range ts in future", + data: []dataItem{ + di(keyA, vals(), 2, nil), // 0 + di(keyB, vals(), 3, nil), // 1 + di(keyC, vals(), 7, makeTxn()), // 2 + rts(keyA, keyB, 10), // - + di(keyA, vals(), 11, nil), // 4 + di(keyA, vals(), 14, nil), // 5 + }, + expectations: []stateExpectations{ + exp(2, isMD, isNil, false, true, false, noTS), + exp(isMD, isNil, isNil, false, false, true, noTS), + exp(1, isNil, isNil, true, false, false, noTS), + exp(0, 4, 5, false, false, false, noTS), + exp(4, 5, isNil, false, false, false, noTS), + exp(5, isNil, isNil, true, false, false, noTS), + }, + gcThreshold: makeTS(7), + }, + { + name: "range tombstones with ts", + data: []dataItem{ + di(keyA, vals(), 2, nil), // 0 + di(keyB, vals(), 3, nil), // 1 + di(keyC, vals(), 7, makeTxn()), // 2 + rts(keyA, keyB, 10), // - + di(keyA, vals(), 11, nil), // 4 + di(keyA, vals(), 14, nil), // 5 + }, + expectations: []stateExpectations{ + exp(2, isMD, isNil, false, true, false, noTS), + exp(isMD, isNil, isNil, false, false, true, noTS), + exp(1, isNil, isNil, true, false, false, noTS), + exp(0, 4, 5, false, false, false, makeTS(10)), + exp(4, 5, isNil, false, false, false, makeTS(10)), + exp(5, isNil, isNil, true, false, false, makeTS(10)), + }, + gcThreshold: makeTS(10), + }, + { + name: "multiple range tombstones", + data: []dataItem{ + rts(keyA, keyB, 1), // - + di(keyA, vals(), 2, nil), // 1 + di(keyB, vals(), 3, nil), // 2 + rts(keyA, keyB, 7), // - + di(keyC, vals(), 7, makeTxn()), // 4 + rts(keyA, keyC, 10), // - + di(keyA, vals(), 11, nil), // 6 + di(keyA, vals(), 14, nil), // 7 + }, + expectations: []stateExpectations{ + exp(4, isMD, isNil, false, true, false, noTS), + exp(isMD, isNil, isNil, false, false, true, noTS), + exp(2, isNil, isNil, true, false, false, noTS), + exp(1, 6, 7, false, false, false, makeTS(7)), + exp(6, 7, isNil, false, false, false, makeTS(7)), + exp(7, isNil, isNil, true, false, false, makeTS(7)), }, + gcThreshold: makeTS(9), }, } { t.Run(tc.name, makeTest(tc)) diff --git a/pkg/kv/kvserver/gc/gc_old_test.go b/pkg/kv/kvserver/gc/gc_old_test.go index 8c5ab7133ad7..d70f38099108 100644 --- a/pkg/kv/kvserver/gc/gc_old_test.go +++ b/pkg/kv/kvserver/gc/gc_old_test.go @@ -147,7 +147,7 @@ func runGCOld( if batchGCKeysBytes >= KeyVersionChunkBytes { batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: keys[i].Timestamp}) - err := gcer.GC(ctx, batchGCKeys) + err := gcer.GC(ctx, batchGCKeys, nil) batchGCKeys = nil batchGCKeysBytes = 0 @@ -206,7 +206,7 @@ func runGCOld( // Handle last collected set of keys/vals. processKeysAndValues() if len(batchGCKeys) > 0 { - if err := gcer.GC(ctx, batchGCKeys); err != nil { + if err := gcer.GC(ctx, batchGCKeys, nil); err != nil { return Info{}, err } } @@ -326,7 +326,6 @@ var ( // TestGarbageCollectorFilter verifies the filter policies for // different sorts of MVCC keys. -// TODO(oleg): Remove once we don't need old GC. func TestGarbageCollectorFilter(t *testing.T) { defer leaktest.AfterTest(t)() gcA := makeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, time.Second) diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 218392005146..ad4a09fa08a8 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -73,6 +73,18 @@ var ( keysPerValueMin: 1, keysPerValueMax: 100, intentFrac: 1, } + someVersionsWithSomeRangeKeys = uniformDistSpec{ + tsSecFrom: 1, tsSecTo: 100, + tsSecMinIntent: 70, tsSecOldIntentTo: 85, + keySuffixMin: 8, keySuffixMax: 8, + valueLenMin: 8, valueLenMax: 16, + deleteFrac: .1, + keysPerValueMin: 1, + keysPerValueMax: 100, + intentFrac: .1, + oldIntentFrac: .1, + rangeKeyFrac: .05, + } ) const intentAgeThreshold = 2 * time.Hour @@ -222,13 +234,22 @@ func TestNewVsInvariants(t *testing.T) { }, ttlSec: 1, }, + { + ds: someVersionsWithSomeRangeKeys, + now: hlc.Timestamp{ + WallTime: 100 * time.Second.Nanoseconds(), + }, + ttlSec: 10, + intentAgeSec: 15, + }, } { t.Run(fmt.Sprintf("%v@%v,ttl=%vsec", tc.ds, tc.now, tc.ttlSec), func(t *testing.T) { rng := rand.New(rand.NewSource(1)) + desc := tc.ds.desc() eng := storage.NewDefaultInMemForTesting() defer eng.Close() - tc.ds.dist(N, rng).setupTest(t, eng, *tc.ds.desc()) + sortedDistribution(tc.ds.dist(N, rng)).setupTest(t, eng, *desc) beforeGC := eng.NewSnapshot() // Run GCer over snapshot. @@ -237,7 +258,7 @@ func TestNewVsInvariants(t *testing.T) { intentThreshold := tc.now.Add(-intentAgeThreshold.Nanoseconds(), 0) gcer := makeFakeGCer() - gcInfoNew, err := Run(ctx, tc.ds.desc(), beforeGC, tc.now, + gcInfoNew, err := Run(ctx, desc, beforeGC, tc.now, gcThreshold, RunOptions{IntentAgeThreshold: intentAgeThreshold}, ttl, &gcer, gcer.resolveIntents, @@ -258,12 +279,17 @@ func TestNewVsInvariants(t *testing.T) { require.NoError(t, err, "failed to resolve intent") } - assertLiveData(t, eng, beforeGC, *tc.ds.desc(), tc.now, gcThreshold, intentThreshold, ttl, + assertLiveData(t, eng, beforeGC, *desc, tc.now, gcThreshold, intentThreshold, ttl, gcInfoNew) }) } } +type historyItem struct { + storage.MVCCKeyValue + isRangeDel bool +} + // assertLiveData will create a stream of expected values based on full data // set contained in provided "before" reader and compare it with the "after" // reader that contains data after applying GC request. @@ -297,7 +323,7 @@ func assertLiveData( storage.IterOptions{ LowerBound: desc.StartKey.AsRawKey(), UpperBound: desc.EndKey.AsRawKey(), - KeyTypes: storage.IterKeyTypePointsOnly, + KeyTypes: storage.IterKeyTypePointsAndRanges, }) defer pointIt.Close() pointIt.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) @@ -363,26 +389,62 @@ func getExpectationsGenerator( // For new key, collect intent and all versions from highest to lowest // to make a decision. var baseKey roachpb.Key - var history []storage.MVCCKeyValue + var history []historyItem for { ok, err := it.Valid() require.NoError(t, err, "failed to read data from unmodified engine") if !ok { break } - k := it.Key() - v := it.Value() - if len(baseKey) == 0 { - baseKey = k.Key - } else if !baseKey.Equal(k.Key) { - break + p, r := it.HasPointAndRange() + if p { + k := it.Key() + v := it.Value() + if len(baseKey) == 0 { + baseKey = k.Key + // We are only interested in range tombstones covering current point, + // so we will add them to history of current key for analysis. + // Bare range tombstones are ignored. + if r { + for _, r := range it.RangeKeys() { + history = append(history, historyItem{ + MVCCKeyValue: storage.MVCCKeyValue{ + Key: storage.MVCCKey{ + Key: r.RangeKey.StartKey, + Timestamp: r.RangeKey.Timestamp, + }, + Value: nil, + }, + isRangeDel: true, + }) + } + } + } else if !baseKey.Equal(k.Key) { + break + } + history = append(history, historyItem{ + MVCCKeyValue: storage.MVCCKeyValue{Key: k, Value: v}, + isRangeDel: false, + }) } - history = append(history, storage.MVCCKeyValue{Key: k, Value: v}) it.Next() } if len(history) == 0 { return storage.MVCCKeyValue{}, false } + // Sort with zero timestamp at the beginning of the list. This could only + // happen if we have a meta record. There could only be single record, + // and it is always at the top of the history so zero timestamp is + // prioritized. + sort.Slice(history, func(i, j int) bool { + if history[i].Key.Timestamp.IsEmpty() { + return true + } + if history[j].Key.Timestamp.IsEmpty() { + return false + } + return history[j].Key.Timestamp.Less(history[i].Key.Timestamp) + }) // Process key history slice by first filtering intents as needed and then // applying invariant that values on or above gc threshold should remain, @@ -409,8 +471,8 @@ func getExpectationsGenerator( } else { // Intent is not considered as a part of GC removal cycle so we keep // it intact if it doesn't satisfy push age check. - pending = append(pending, history[i]) - pending = append(pending, history[i+1]) + pending = append(pending, history[i].MVCCKeyValue) + pending = append(pending, history[i+1].MVCCKeyValue) } i += 2 continue @@ -420,12 +482,14 @@ func getExpectationsGenerator( switch { case gcThreshold.Less(history[i].Key.Timestamp): // Any value above threshold including intents that have no timestamp. - pending = append(pending, history[i]) + if !history[i].isRangeDel { + pending = append(pending, history[i].MVCCKeyValue) + } i++ case history[i].Key.Timestamp.LessEq(gcThreshold) && len(history[i].Value) > 0: // First value on or under threshold should be preserved, but the rest // of history should be skipped. - pending = append(pending, history[i]) + pending = append(pending, history[i].MVCCKeyValue) i++ stop = true default: @@ -436,10 +500,16 @@ func getExpectationsGenerator( // Remaining part of the history is removed, so accumulate it as gc stats. if i < len(history) { - expInfo.NumKeysAffected++ + removedKeys := false for ; i < len(history); i++ { - expInfo.AffectedVersionsKeyBytes += int64(history[i].Key.EncodedSize()) - expInfo.AffectedVersionsValBytes += int64(len(history[i].Value)) + if !history[i].isRangeDel { + expInfo.AffectedVersionsKeyBytes += int64(history[i].Key.EncodedSize()) + expInfo.AffectedVersionsValBytes += int64(len(history[i].Value)) + removedKeys = true + } + } + if removedKeys { + expInfo.NumKeysAffected++ } } } @@ -452,7 +522,7 @@ func getKeyHistory(t *testing.T, r storage.Reader, key roachpb.Key) string { it := r.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ LowerBound: key, UpperBound: key.Next(), - KeyTypes: storage.IterKeyTypePointsOnly, + KeyTypes: storage.IterKeyTypePointsAndRanges, RangeKeyMaskingBelow: hlc.Timestamp{}, }) defer it.Close() @@ -461,9 +531,22 @@ func getKeyHistory(t *testing.T, r storage.Reader, key roachpb.Key) string { for { ok, err := it.Valid() require.NoError(t, err, "failed to read engine iterator") - if !ok || !it.UnsafeKey().Key.Equal(key) { + if !ok { break } + p, r := it.HasPointAndRange() + if !p { + it.Next() + continue + } + if !it.UnsafeKey().Key.Equal(key) { + break + } + if r && len(result) == 0 { + for _, rk := range it.RangeKeys() { + result = append(result, fmt.Sprintf("R:%s", rk.RangeKey.String())) + } + } result = append(result, fmt.Sprintf("P:%s(%d)", it.UnsafeKey().String(), len(it.UnsafeValue()))) it.Next() } @@ -492,7 +575,9 @@ func (f *fakeGCer) SetGCThreshold(ctx context.Context, t Threshold) error { return nil } -func (f *fakeGCer) GC(ctx context.Context, keys []roachpb.GCRequest_GCKey) error { +func (f *fakeGCer) GC( + ctx context.Context, keys []roachpb.GCRequest_GCKey, rangeKeys []roachpb.GCRequest_GCRangeKey, +) error { for _, k := range keys { f.gcKeys[k.Key.String()] = k } diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 5aadec7e0265..b73a66479f52 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -55,7 +55,9 @@ type collectingGCer struct { keys [][]roachpb.GCRequest_GCKey } -func (c *collectingGCer) GC(_ context.Context, keys []roachpb.GCRequest_GCKey) error { +func (c *collectingGCer) GC( + _ context.Context, keys []roachpb.GCRequest_GCKey, rangeKeys []roachpb.GCRequest_GCRangeKey, +) error { c.keys = append(c.keys, keys) return nil } @@ -564,20 +566,97 @@ var intentsAfterDelete = ` 1 | ` +var deleteRangeData = ` + | a b c d e f g h i j +---+---------------------- + 9 | + 8 | + 7 | + 6 | *- +>5 | *- + 4 | *- + 3 | + 2 | a b C + 1 | +` + +var deleteRangeDataWithNewerValues = ` + | a b c d e f g h i j +---+---------------------- + 9 | + 8 | A C E *--- + 7 | + 6 | *-G +>5 | *- + 4 | *- I + 3 | + 2 | b d F H i + 1 | +` + +var deleteRangeMultipleValues = ` + | a b c d e f g h i j +---+---------------------- + 9 | + 8 | + 7 | + 6 | *--- +>5 | + 4 | *- + 3 | + 2 | a B C + 1 | +` + +var deleteRangeDataWithIntents = ` + | a b c d e f g h i j +---+---------------------- + 9 | + 8 | !A !C !E + 7 | + 6 | *-- +>5 | *-- + 4 | *-- + 3 | + 2 | b d F + 1 | +` + +// This case verifies that if range tombstone stack caching is working correctly +// when different keys have different removal thresholds. +var differentRangeStacksPerPoint = ` + | a bb ccc d +---+--------------- + 9 | +>8 | B3 + 7 | *---------- + 6 | b2 + 5 | *---------- + 4 | a2 b1 + 3 | *---------- + 2 | a1 + 1 | +` + func TestGC(t *testing.T) { for _, d := range []struct { name string data string }{ - {"single", singleValueData}, - {"multiple_newer", multipleValuesNewerData}, - {"multiple_older", multipleValuesOlderData}, - {"delete", deleteData}, - {"delete_with_newer", deleteWithNewerData}, - {"multiple_values", multipleValuesData}, - {"intents", intents}, - {"intents_after_data", intentsAfterData}, - {"intents_after_delete", intentsAfterDelete}, + {name: "single", data: singleValueData}, + {name: "multiple_newer", data: multipleValuesNewerData}, + {name: "multiple_older", data: multipleValuesOlderData}, + {name: "delete", data: deleteData}, + {name: "delete_with_newer", data: deleteWithNewerData}, + {name: "multiple_values", data: multipleValuesData}, + {name: "intents", data: intents}, + {name: "intents_after_data", data: intentsAfterData}, + {name: "intents_after_delete", data: intentsAfterDelete}, + {name: "delete_range_data", data: deleteRangeData}, + {name: "delete_range_data_newer", data: deleteRangeDataWithNewerValues}, + {name: "delete_range_multiple_points", data: deleteRangeMultipleValues}, + {name: "delete_range_with_intents", data: deleteRangeDataWithIntents}, + {name: "delete_with_different_range_stacks", data: differentRangeStacksPerPoint}, } { t.Run(d.name, func(t *testing.T) { runTest(t, d.data) @@ -628,6 +707,10 @@ func runTest(t *testing.T, data string) { } requireEqualReaders(t, ctrlEng, eng, desc) + + // Age stats to the same TS before comparing. + expectedStats.AgeTo(now.WallTime) + stats.AgeTo(now.WallTime) require.Equal(t, expectedStats, stats, "mvcc stats don't match the data") } @@ -676,9 +759,17 @@ func requireEqualReaders( // dataItem is element read from test table containing mvcc key value along with // metadata needed for filtering. type dataItem struct { - value storage.MVCCKeyValue - txn *roachpb.Transaction - live bool + value storage.MVCCKeyValue + txn *roachpb.Transaction + rangeKeyValue storage.MVCCRangeKeyValue + live bool +} + +func (d dataItem) timestamp() hlc.Timestamp { + if !d.value.Key.Timestamp.IsEmpty() { + return d.value.Key.Timestamp + } + return d.rangeKeyValue.RangeKey.Timestamp } type tableData []dataItem @@ -719,14 +810,9 @@ func readTableData( parsePoint := func(val string, i int, ts hlc.Timestamp) int { val = strings.TrimSpace(val) if len(val) > 0 { - value := []byte(val) - // Special meaning for deletions. - if val == "*" || val == "." { - value = nil - } var txn *roachpb.Transaction if val[0] == '!' { - value = value[1:] + val = val[1:] txn = &roachpb.Transaction{ Status: roachpb.PENDING, ReadTimestamp: ts, @@ -737,8 +823,9 @@ func readTableData( txn.Key = txn.ID.GetBytes() } var v roachpb.Value - if value != nil { - v.SetBytes(value) + // Special meaning for deletions. + if val != "*" && val != "." { + v.SetString(val) } kv := storage.MVCCKeyValue{ Key: storage.MVCCKey{ @@ -753,6 +840,36 @@ func readTableData( return i + 1 } + parseRange := func(val, l string, i int, ts hlc.Timestamp) int { + startKey := columnKeys[i] + // Handle range key. We are modifying outer loop index here. + p := columnPositions[i+1] - 1 + // Find where range definition ends. + for ; p < columnPositions[len(columnPositions)-1] && l[p] == '-'; p++ { + } + // Find key following the end of range. + for i++; i < len(columnKeys) && columnPositions[i] < p; i++ { + } + // Extract value from the first element. + value := val[0] + if value != '*' && value != '.' { + panic("test data only supports range deletions") + } + live := value == '*' + // Add range to data. + endKey := columnKeys[i] + items = append(items, dataItem{ + rangeKeyValue: storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: startKey, + EndKey: endKey, + Timestamp: ts, + }, + }, live: live, + }) + return i + } + var gcTS hlc.Timestamp var lastTs int64 = math.MaxInt64 parseTS := func(l string) (ts hlc.Timestamp) { @@ -769,7 +886,7 @@ func readTableData( require.NoError(t, err, "Failed to parse timestamp from %s", l) require.Less(t, tsInt, lastTs, "Timestamps should be decreasing") lastTs = tsInt - return hlc.Timestamp{WallTime: tsInt} + return hlc.Timestamp{WallTime: tsInt * time.Second.Nanoseconds()} } for _, l := range lines { @@ -793,7 +910,11 @@ func readTableData( ts := parseTS(l) for i := 0; i < len(columnKeys); { val := l[columnPositions[i]:columnPositions[i+1]] - i = parsePoint(val, i, ts) + if val[len(val)-1] == '-' { + i = parseRange(val, l, i, ts) + } else { + i = parsePoint(val, i, ts) + } } } @@ -802,19 +923,19 @@ func readTableData( items[i], items[j] = items[j], items[i] } - return items, gcTS, items[len(items)-1].value.Key.Timestamp.Add(1, 0) + return items, gcTS, items[len(items)-1].timestamp().Add(1, 0) } // fullDistribution creates a data distribution that contains all data read from // table. func (d tableData) fullDistribution() dataDistribution { items := d - return func() (storage.MVCCKeyValue, *roachpb.Transaction, bool) { + return func() (storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool) { if len(items) == 0 { - return storage.MVCCKeyValue{}, nil, false + return storage.MVCCKeyValue{}, storage.MVCCRangeKeyValue{}, nil, false } defer func() { items = items[1:] }() - return items[0].value, items[0].txn, true + return items[0].value, items[0].rangeKeyValue, items[0].txn, true } } @@ -822,10 +943,10 @@ func (d tableData) fullDistribution() dataDistribution { // marked as live (see table data format above). func (d tableData) liveDistribution() dataDistribution { items := d - return func() (storage.MVCCKeyValue, *roachpb.Transaction, bool) { + return func() (storage.MVCCKeyValue, storage.MVCCRangeKeyValue, *roachpb.Transaction, bool) { for { if len(items) == 0 { - return storage.MVCCKeyValue{}, nil, false + return storage.MVCCKeyValue{}, storage.MVCCRangeKeyValue{}, nil, false } if items[0].live { break @@ -833,12 +954,89 @@ func (d tableData) liveDistribution() dataDistribution { items = items[1:] } defer func() { items = items[1:] }() - return items[0].value, items[0].txn, true + return items[0].value, items[0].rangeKeyValue, items[0].txn, true } } +// engineData reads all engine data as tableCells in no particular order. func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) []tableCell { var result []tableCell + + rangeIt := r.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypeRangesOnly, + RangeKeyMaskingBelow: hlc.Timestamp{}, + }) + defer rangeIt.Close() + rangeIt.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + makeRangeCells := func(rks []storage.MVCCRangeKey) (tc []tableCell) { + for _, rk := range rks { + tc = append(tc, tableCell{ + key: storage.MVCCKey{ + Key: rk.StartKey, + Timestamp: rk.Timestamp, + }, + endKey: rk.EndKey, + value: ".", + }) + } + return tc + } + var partialRangeKeys []storage.MVCCRangeKey + var lastEnd roachpb.Key + for { + ok, err := rangeIt.Valid() + require.NoError(t, err, "failed to iterate range keys") + if !ok { + break + } + _, r := rangeIt.HasPointAndRange() + if r { + span := rangeIt.RangeBounds() + newKeys := rangeIt.RangeKeys() + if lastEnd.Equal(span.Key) { + // Try merging keys by timestamp. + var newPartial []storage.MVCCRangeKey + i, j := 0, 0 + for i < len(newKeys) && j < len(partialRangeKeys) { + switch newKeys[i].RangeKey.Timestamp.Compare(partialRangeKeys[j].Timestamp) { + case 1: + newPartial = append(newPartial, newKeys[i].RangeKey.Clone()) + i++ + case 0: + newPartial = append(newPartial, storage.MVCCRangeKey{ + StartKey: partialRangeKeys[j].StartKey, + EndKey: newKeys[i].RangeKey.EndKey.Clone(), + Timestamp: partialRangeKeys[j].Timestamp, + }) + i++ + j++ + case -1: + newPartial = append(newPartial, partialRangeKeys[j].Clone()) + j++ + } + } + for ; i < len(newKeys); i++ { + newPartial = append(newPartial, newKeys[i].RangeKey.Clone()) + } + for ; j < len(partialRangeKeys); j++ { + newPartial = append(newPartial, partialRangeKeys[j].Clone()) + } + partialRangeKeys = newPartial + } else { + result = append(result, makeRangeCells(partialRangeKeys)...) + partialRangeKeys = make([]storage.MVCCRangeKey, len(newKeys)) + for i, rk := range newKeys { + partialRangeKeys[i] = rk.RangeKey.Clone() + } + } + lastEnd = span.EndKey.Clone() + } + rangeIt.NextKey() + } + result = append(result, makeRangeCells(partialRangeKeys)...) + it := r.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ LowerBound: desc.StartKey.AsRawKey(), UpperBound: desc.EndKey.AsRawKey(), @@ -878,13 +1076,23 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) [] } type tableCell struct { - key storage.MVCCKey + key storage.MVCCKey + // Optional endKey for range keys. + endKey roachpb.Key + // Formatted key value (could be value, deletion or intent). value string } type columnInfo struct { key string maxValueLength int + position int + formatStr string +} + +type cellValue struct { + value string + endKey string } // formatTable renders table with data. expecting data to be sorted naturally: @@ -892,64 +1100,123 @@ type columnInfo struct { // prefix if provided defines start of the key, that would be stripped from the // keys to avoid clutter. func formatTable(data []tableCell, prefix roachpb.Key) []string { - prefixStr := "" + // Table with no data is a special case. + if len(data) == 0 { + return nil + } + + keyPrefixStr := "" if prefix != nil { - prefixStr = prefix.String() + keyPrefixStr = prefix.String() } keyRe := regexp.MustCompile(`^/"(.*)"$`) - var foundKeys []columnInfo - var lastKey roachpb.Key - rowData := make(map[int64][]string) + columnName := func(key roachpb.Key) string { + keyStr := key.String() + if strings.Index(keyStr, keyPrefixStr) == 0 { + keyStr = keyStr[len(keyPrefixStr):] + if keyRe.FindSubmatch([]byte(keyStr)) != nil { + keyStr = keyStr[2 : len(keyStr)-1] + } + } + return keyStr + } + + // Table data indexed by ts, key. + keyData := make(map[string]columnInfo) + + addKeyColumn := func(key roachpb.Key, columnLen int) { + keyStr := key.String() + if column, ok := keyData[keyStr]; !ok { + title := columnName(key) + if titleLen := len(title); columnLen < len(title) { + columnLen = titleLen + } + keyData[keyStr] = columnInfo{ + key: title, + maxValueLength: columnLen, + } + } else { + if columnLen > column.maxValueLength { + column.maxValueLength = columnLen + keyData[keyStr] = column + } + } + } + + sparseTable := make(map[int64]map[string]cellValue) for _, c := range data { ts := c.key.Timestamp.WallTime key := c.key.Key.String() - if strings.Index(key, prefixStr) == 0 { - key = key[len(prefixStr):] - if keyRe.FindSubmatch([]byte(key)) != nil { - key = key[2 : len(key)-1] - } - } - if !c.key.Key.Equal(lastKey) { - foundKeys = append(foundKeys, columnInfo{ - key: key, - maxValueLength: len(key), - }) - lastKey = c.key.Key + addKeyColumn(c.key.Key, len(c.value)) + if _, ok := sparseTable[ts]; !ok { + sparseTable[ts] = make(map[string]cellValue) } - row := rowData[ts] - for len(row) < len(foundKeys)-1 { - row = append(row, "") + endKey := "" + if len(c.endKey) > 0 { + endKey = c.endKey.String() } - rowData[ts] = append(row, c.value) - valueLen := len(c.value) - if i := len(foundKeys) - 1; valueLen > foundKeys[i].maxValueLength { - foundKeys[i].maxValueLength = valueLen + sparseTable[ts][key] = cellValue{c.value, endKey} + if len(c.endKey) > 0 { + addKeyColumn(c.endKey, 0) } } - var tss []int64 - for ts := range rowData { - tss = append(tss, ts) + + // Build table key positions for ease of iterations. + var keySeq []string + for k := range keyData { + keySeq = append(keySeq, k) + } + sort.Strings(keySeq) + base := 0 + for _, k := range keySeq { + kd := keyData[k] + kd.position = base + kd.formatStr = fmt.Sprintf("%%-%ds", kd.maxValueLength) + base += kd.maxValueLength + 1 + keyData[k] = kd + } + // Build key sequence. + var tsSeq []int64 + for ts := range sparseTable { + tsSeq = append(tsSeq, ts) } - sort.Slice(tss, func(i, j int) bool { - return tss[i] > tss[j] + sort.Slice(tsSeq, func(i, j int) bool { + return tsSeq[i] > tsSeq[j] }) - lsLen := len(fmt.Sprintf("%d", tss[0])) - rowPrefixFmt := fmt.Sprintf(" %%%dd | ", lsLen) - var result []string + lsLen := len(fmt.Sprintf("%d", tsSeq[0]/time.Second.Nanoseconds())) + rowPrefixFmt := fmt.Sprintf(" %%%dd | ", lsLen) firstRow := fmt.Sprintf(" %s | ", strings.Repeat(" ", lsLen)) - for _, colInfo := range foundKeys { - firstRow += fmt.Sprintf(fmt.Sprintf("%%%ds ", colInfo.maxValueLength), colInfo.key) + for _, key := range keySeq { + colInfo := keyData[key] + firstRow += fmt.Sprintf(colInfo.formatStr, colInfo.key) + firstRow += " " } result = append(result, firstRow) result = append(result, strings.Repeat("-", len(firstRow))) - for _, ts := range tss { - row := rowData[ts] - rowStr := fmt.Sprintf(rowPrefixFmt, ts) - for i, v := range row { - rowStr += fmt.Sprintf(fmt.Sprintf("%%%ds ", foundKeys[i].maxValueLength), v) + + for _, ts := range tsSeq { + row := sparseTable[ts] + rowStr := fmt.Sprintf(rowPrefixFmt, ts/time.Second.Nanoseconds()) + curLen := 0 + for _, key := range keySeq { + if v, ok := row[key]; ok { + colInfo := keyData[key] + pos := colInfo.position + gap := pos - curLen + valStr := strings.Repeat(" ", gap) + if len(v.endKey) > 0 { + rangeEnd := keyData[v.endKey].position + rangeLen := rangeEnd - pos - len(v.value) + valStr += v.value + strings.Repeat("-", rangeLen) + } else { + valStr += fmt.Sprintf(colInfo.formatStr, v.value) + } + rowStr += valStr + curLen += len(valStr) + } } result = append(result, rowStr) } diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 42aca229b802..5a42f2cf8303 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -482,7 +482,9 @@ func (r *replicaGCer) SetGCThreshold(ctx context.Context, thresh gc.Threshold) e return r.send(ctx, req) } -func (r *replicaGCer) GC(ctx context.Context, keys []roachpb.GCRequest_GCKey) error { +func (r *replicaGCer) GC( + ctx context.Context, keys []roachpb.GCRequest_GCKey, rangeKeys []roachpb.GCRequest_GCRangeKey, +) error { if len(keys) == 0 { return nil } diff --git a/pkg/kv/kvserver/rditer/replica_data_iter.go b/pkg/kv/kvserver/rditer/replica_data_iter.go index f9d2ee54ae56..72f2ae08c5c1 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter.go @@ -253,6 +253,7 @@ func (ri *ReplicaMVCCDataIterator) tryCloseAndCreateIter() { storage.IterOptions{ LowerBound: ri.ranges[ri.curIndex].Start, UpperBound: ri.ranges[ri.curIndex].End, + KeyTypes: storage.IterKeyTypePointsAndRanges, }) if ri.reverse { ri.it.SeekLT(storage.MakeMVCCMetadataKey(ri.ranges[ri.curIndex].End)) @@ -340,12 +341,31 @@ func (ri *ReplicaMVCCDataIterator) UnsafeKey() storage.MVCCKey { return ri.it.UnsafeKey() } +// RangeBounds returns the range bounds for the current range key, or an +// empty span if there are none. The returned keys are only valid until the +// next iterator call. +func (ri *ReplicaMVCCDataIterator) RangeBounds() roachpb.Span { + return ri.it.RangeBounds() +} + // UnsafeValue returns the same value as Value, but the memory is invalidated on // the next call to {Next,Prev,Close}. func (ri *ReplicaMVCCDataIterator) UnsafeValue() []byte { return ri.it.UnsafeValue() } +// RangeKeys exposes RangeKeys from underlying iterator. See +// storage.SimpleMVCCIterator for details. +func (ri *ReplicaMVCCDataIterator) RangeKeys() []storage.MVCCRangeKeyValue { + return ri.it.RangeKeys() +} + +// HasPointAndRange exposes HasPointAndRange from underlying iterator. See +// storage.SimpleMVCCIterator for details. +func (ri *ReplicaMVCCDataIterator) HasPointAndRange() (bool, bool) { + return ri.it.HasPointAndRange() +} + // NewReplicaEngineDataIterator creates a ReplicaEngineDataIterator for the // given replica. func NewReplicaEngineDataIterator( diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 121f700a92b9..5f5c3f266439 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -122,7 +122,11 @@ func createRangeData( } func verifyRDReplicatedOnlyMVCCIter( - t *testing.T, desc *roachpb.RangeDescriptor, eng storage.Engine, expectedKeys []storage.MVCCKey, + t *testing.T, + desc *roachpb.RangeDescriptor, + eng storage.Engine, + expectedKeys []storage.MVCCKey, + expectedRangeKeys []storage.MVCCRangeKey, ) { t.Helper() verify := func(t *testing.T, useSpanSet, reverse bool) { @@ -146,22 +150,47 @@ func verifyRDReplicatedOnlyMVCCIter( } iter := NewReplicaMVCCDataIterator(desc, readWriter, reverse /* seekEnd */) defer iter.Close() + next := iter.Next + if reverse { + next = iter.Prev + } + var rangeStart roachpb.Key actualKeys := []storage.MVCCKey{} + actualRanges := []storage.MVCCRangeKey{} for { ok, err := iter.Valid() require.NoError(t, err) if !ok { break } - if !reverse { - actualKeys = append(actualKeys, iter.Key()) - iter.Next() - } else { - actualKeys = append([]storage.MVCCKey{iter.Key()}, actualKeys...) - iter.Prev() + p, r := iter.HasPointAndRange() + if p { + if !reverse { + actualKeys = append(actualKeys, iter.Key()) + } else { + actualKeys = append([]storage.MVCCKey{iter.Key()}, actualKeys...) + } + } + if r { + rks := iter.RangeKeys() + if !rks[0].RangeKey.StartKey.Equal(rangeStart) { + if !reverse { + for _, rk := range rks { + actualRanges = append(actualRanges, rk.RangeKey.Clone()) + } + } else { + for i := len(rks) - 1; i >= 0; i-- { + actualRanges = append([]storage.MVCCRangeKey{rks[i].RangeKey.Clone()}, + actualRanges...) + } + } + rangeStart = rks[0].RangeKey.StartKey.Clone() + } } + next() } require.Equal(t, expectedKeys, actualKeys) + require.Equal(t, expectedRangeKeys, actualRanges) } testutils.RunTrueAndFalse(t, "reverse", func(t *testing.T, reverse bool) { testutils.RunTrueAndFalse(t, "spanset", func(t *testing.T, useSpanSet bool) { @@ -230,7 +259,7 @@ func TestReplicaDataIteratorEmptyRange(t *testing.T) { EndKey: roachpb.RKey("z"), } - verifyRDReplicatedOnlyMVCCIter(t, desc, eng, []storage.MVCCKey{}) + verifyRDReplicatedOnlyMVCCIter(t, desc, eng, []storage.MVCCKey{}, []storage.MVCCRangeKey{}) verifyRDEngineIter(t, desc, eng, false, []interface{}{}) verifyRDEngineIter(t, desc, eng, true, []interface{}{}) } @@ -282,16 +311,19 @@ func TestReplicaDataIterator(t *testing.T) { verifyRDEngineIter(t, &tc.desc, eng, true, tc.replicatedKeys) // Verify the replicated MVCC contents. - // - // TODO(erikgrinaker): This currently only supports MVCC point keys, so we - // ignore MVCC range keys for now. var pointKeys []storage.MVCCKey + var rangeKeys []storage.MVCCRangeKey for _, key := range tc.replicatedKeys { if pointKey, ok := key.(storage.MVCCKey); ok { pointKeys = append(pointKeys, pointKey) + } else if rangeKey, ok := key.(storage.MVCCRangeKey); ok { + // TODO(oleg): This is very naive and only works if keys don't overlap. + // Needs some thinking on how could we use this if ranges become + // fragmented. + rangeKeys = append(rangeKeys, rangeKey) } } - verifyRDReplicatedOnlyMVCCIter(t, &tc.desc, eng, pointKeys) + verifyRDReplicatedOnlyMVCCIter(t, &tc.desc, eng, pointKeys, rangeKeys) }) } } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 1083edb7f0eb..32767f65a3dd 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -936,11 +936,19 @@ message HeartbeatTxnResponse { message GCRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + // Point keys message GCKey { bytes key = 1 [(gogoproto.casttype) = "Key"]; util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; } repeated GCKey keys = 3 [(gogoproto.nullable) = false]; + // Range keys + message GCRangeKey { + bytes startKey = 1 [(gogoproto.casttype) = "Key"]; + bytes endKey = 2 [(gogoproto.casttype) = "Key"]; + util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + } + repeated GCRangeKey rangeKeys = 6 [(gogoproto.nullable) = false]; // Threshold is the expiration timestamp. util.hlc.Timestamp threshold = 4 [(gogoproto.nullable) = false]; diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 1d67cd2f062a..756ef7119212 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -990,7 +990,7 @@ func mvccGetMetadata( if !hasPoint || !unsafeKey.Key.Equal(metaKey.Key) { meta.Deleted = true meta.Timestamp = rkTimestamp.ToLegacyTimestamp() - return true, int64(EncodedMVCCKeyPrefixLength(metaKey.Key)), 0, hlc.Timestamp{}, nil + return true, 0, 0, hlc.Timestamp{}, nil } } @@ -4085,8 +4085,6 @@ func MVCCResolveWriteIntentRange( // not a mix of the two. This is to accommodate the implementation below // that creates an iterator with bounds that span from the first to last // key (in sorted order). -// -// TODO(erikgrinaker): This must handle MVCC range tombstones. func MVCCGarbageCollect( ctx context.Context, rw ReadWriter, @@ -4119,32 +4117,78 @@ func MVCCGarbageCollect( iter := rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ LowerBound: keys[0].Key, UpperBound: keys[len(keys)-1].Key.Next(), + KeyTypes: IterKeyTypePointsAndRanges, }) defer iter.Close() supportsPrev := iter.SupportsPrev() + // To update mvcc stats, we need to go through range tombstones to determine + // GCBytesAge. That requires copying current stack of range keys covering + // point since returned range key values are unsafe. We try to cache those + // copies by checking start of range. + var lastRangeTombstoneStart roachpb.Key + var rangeTombstoneTss []hlc.Timestamp + // Iterate through specified GC keys. meta := &enginepb.MVCCMetadata{} for _, gcKey := range keys { encKey := MakeMVCCMetadataKey(gcKey.Key) - ok, metaKeySize, metaValSize, _, err := mvccGetMetadata(iter, encKey, meta) + // TODO(oleg): Results of this call are not obvious and logic to handle + // stats updates for different real and synthesized metadata becomes + // unnecessary complicated. Revisit this to make it cleaner. + ok, metaKeySize, metaValSize, realKeyChanged, err := mvccGetMetadata(iter, encKey, meta) if err != nil { return err } if !ok { continue } + + // If mvccGetMetadata landed on a bare range tombstone for the key it will + // synthesize deletion meta. We need to filter this case out to avoid + // updating key stats as the key doesn't exist. + // Empty realKeyChanged is an indication that there are no values for the + // key present, but it may contain an inlined metadata which we filter out + // by checking that it is not inlined. + // In that case, we can skip to the next gc key as there's nothing to GC. + // As a side effect of this change, we should be positioned on the point + // key or inlined meta at this point and can do further checks for GC + // eligibility. inlinedValue := meta.IsInline() - implicitMeta := iter.UnsafeKey().IsValue() - // First, check whether all values of the key are being deleted. + if realKeyChanged.IsEmpty() && !inlinedValue { + continue + } + + // We are guaranteed now to be positioned at the meta or version key that + // belongs to gcKey history. + + unsafeKey := iter.UnsafeKey() + implicitMeta := unsafeKey.IsValue() + // First check for the case of range tombstone covering keys when no + // metadata is available. // // Note that we naively can't terminate GC'ing keys loop early if we - // enter this branch, as it will update the stats under the provision - // that the (implicit or explicit) meta key (and thus all versions) are - // being removed. We had this faulty functionality at some point; it - // should no longer be necessary since the higher levels already make - // sure each individual GCRequest does bounded work. - if meta.Timestamp.ToTimestamp().LessEq(gcKey.Timestamp) { + // enter any of branches below, as it will update the stats under the + // provision that the (implicit or explicit) meta key (and thus all + // versions) are being removed. We had this faulty functionality at some + // point; it should no longer be necessary since the higher levels already + // make sure each individual GCRequest does bounded work. + if implicitMeta && meta.Deleted && !unsafeKey.Timestamp.Equal(realKeyChanged) { + // If we have implicit deletion meta, and realKeyChanged is not the first + // key in history, that means it is covered by a range tombstone (which + // was used to synthesize meta). + if unsafeKey.Timestamp.LessEq(gcKey.Timestamp) { + // If first object in history is at or below gcKey timestamp then we + // have no explicit meta and all objects are subject to deletion. + if ms != nil { + ms.Add(updateStatsOnGC(gcKey.Key, metaKeySize, metaValSize, meta, + realKeyChanged.WallTime)) + } + } + } else if meta.Timestamp.ToTimestamp().LessEq(gcKey.Timestamp) { + // Then, check whether all values of the key are being deleted in the + // rest of the cases. + // // For version keys, don't allow GC'ing the meta key if it's // not marked deleted. However, for inline values we allow it; // they are internal and GCing them directly saves the extra @@ -4185,11 +4229,15 @@ func MVCCGarbageCollect( // first garbage version. prevNanos := timestamp.WallTime { - + // If true - forward iteration positioned iterator on first garbage + // (key.ts <= gc.ts). var foundPrevNanos bool { // If reverse iteration is supported (supportsPrev), we'll step the // iterator a few time before attempting to seek. + + // True if we found next key while iterating. That means there's no + // garbage for the key. var foundNextKey bool // If there are a large number of versions which are not garbage, @@ -4213,6 +4261,10 @@ func MVCCGarbageCollect( foundNextKey = true break } + if hasPoint, _ := iter.HasPointAndRange(); !hasPoint { + foundNextKey = true + break + } unsafeIterKey := iter.UnsafeKey() if !unsafeIterKey.Key.Equal(encKey.Key) { foundNextKey = true @@ -4254,6 +4306,31 @@ func MVCCGarbageCollect( } } + // At this point iterator is positioned on first garbage version and forward + // iteration will give us all versions to delete up to the next key. + + if ms != nil { + // We need to iterate ranges only to compute GCBytesAge if we are updating + // stats. + if _, hasRange := iter.HasPointAndRange(); hasRange && !lastRangeTombstoneStart.Equal(iter.RangeBounds().Key) { + rangeKeys := iter.RangeKeys() + newLen := len(rangeKeys) + if cap(rangeTombstoneTss) < newLen { + rangeTombstoneTss = make([]hlc.Timestamp, newLen) + } else { + rangeTombstoneTss = rangeTombstoneTss[:newLen] + } + for i, rkv := range rangeKeys { + rangeTombstoneTss[i] = rkv.RangeKey.Timestamp + } + lastRangeTombstoneStart = append(lastRangeTombstoneStart[:0], rangeKeys[0].RangeKey.StartKey...) + } else if !hasRange { + lastRangeTombstoneStart = lastRangeTombstoneStart[:0] + rangeTombstoneTss = rangeTombstoneTss[:0] + } + } + rangeIdx := 0 + // Iterate through the garbage versions, accumulating their stats and // issuing clear operations. for ; ; iter.Next() { @@ -4287,6 +4364,19 @@ func MVCCGarbageCollect( fromNS := prevNanos if unsafeVal.IsTombstone() { fromNS = unsafeIterKey.Timestamp.WallTime + } else { + // For non deletions, we need to find if we had a range tombstone + // between this and next value (prevNanos) to use its timestamp for + // computing GCBytesAge. + for ; rangeIdx < len(rangeTombstoneTss); rangeIdx++ { + rangeTS := rangeTombstoneTss[rangeIdx] + if rangeTombstoneTss[rangeIdx].Less(unsafeIterKey.Timestamp) { + break + } + if rangeTS.WallTime < fromNS { + fromNS = rangeTS.WallTime + } + } } ms.Add(updateStatsOnGC(gcKey.Key, keySize, valSize, nil, fromNS)) diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 05cd64b933cd..6c958bb1f7fd 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -4772,14 +4772,16 @@ func TestMVCCGarbageCollect(t *testing.T) { ms := &enginepb.MVCCStats{} - bytes := []byte("value") + val := []byte("value") ts1 := hlc.Timestamp{WallTime: 1e9} ts2 := hlc.Timestamp{WallTime: 2e9} ts3 := hlc.Timestamp{WallTime: 3e9} - val1 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts1) - val2 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts2) - val3 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts3) - valInline := roachpb.MakeValueFromBytesAndTimestamp(bytes, hlc.Timestamp{}) + ts4 := hlc.Timestamp{WallTime: 4e9} + ts5 := hlc.Timestamp{WallTime: 4e9} + val1 := roachpb.MakeValueFromBytesAndTimestamp(val, ts1) + val2 := roachpb.MakeValueFromBytesAndTimestamp(val, ts2) + val3 := roachpb.MakeValueFromBytesAndTimestamp(val, ts3) + valInline := roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}) testData := []struct { key roachpb.Key @@ -4791,6 +4793,10 @@ func TestMVCCGarbageCollect(t *testing.T) { {roachpb.Key("b"), []roachpb.Value{val1, val2, val3}, false}, {roachpb.Key("b-del"), []roachpb.Value{val1, val2, val3}, true}, {roachpb.Key("inline"), []roachpb.Value{valInline}, false}, + {roachpb.Key("r-2"), []roachpb.Value{val1}, false}, + {roachpb.Key("r-3"), []roachpb.Value{val1}, false}, + {roachpb.Key("r-4"), []roachpb.Value{val1}, false}, + {roachpb.Key("t"), []roachpb.Value{val1}, false}, } for i := 0; i < 3; i++ { @@ -4800,30 +4806,46 @@ func TestMVCCGarbageCollect(t *testing.T) { } for _, val := range test.vals[i : i+1] { if i == len(test.vals)-1 && test.isDeleted { - if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, nil); err != nil { + if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, + nil); err != nil { t.Fatal(err) } continue } valCpy := *protoutil.Clone(&val).(*roachpb.Value) valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, + valCpy, nil); err != nil { t.Fatal(err) } } } } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("r"), + roachpb.Key("r-del").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, 0); err != nil { + t.Fatal(err) + } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), + roachpb.Key("u").Next(), ts2, hlc.ClockTimestamp{}, nil, nil, 0); err != nil { + t.Fatal(err) + } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), + roachpb.Key("u").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, 0); err != nil { + t.Fatal(err) + } if log.V(1) { kvsn, err := Scan(engine, localMax, keyMax, 0) if err != nil { t.Fatal(err) } + log.Info(context.Background(), "before") for i, kv := range kvsn { log.Infof(context.Background(), "%d: %s", i, kv.Key) } } - keys := []roachpb.GCRequest_GCKey{ + gcTime := ts5 + gcKeys := []roachpb.GCRequest_GCKey{ {Key: roachpb.Key("a"), Timestamp: ts1}, {Key: roachpb.Key("a-del"), Timestamp: ts2}, {Key: roachpb.Key("b"), Timestamp: ts1}, @@ -4832,13 +4854,41 @@ func TestMVCCGarbageCollect(t *testing.T) { // Keys that don't exist, which should result in a no-op. {Key: roachpb.Key("a-bad"), Timestamp: ts2}, {Key: roachpb.Key("inline-bad"), Timestamp: hlc.Timestamp{}}, + // Keys that are hidden by range key. + // Non-existing keys that needs to skip gracefully without + // distorting stats. (Checking that following keys doesn't affect it) + {Key: roachpb.Key("r-0"), Timestamp: ts1}, + {Key: roachpb.Key("r-1"), Timestamp: ts4}, + // Request has a timestamp below range key, it will be handled by + // logic processing range tombstones specifically. + {Key: roachpb.Key("r-2"), Timestamp: ts1}, + // Requests has a timestamp at or above range key, it will be handled by + // logic processing synthesized metadata. + {Key: roachpb.Key("r-3"), Timestamp: ts3}, + {Key: roachpb.Key("r-4"), Timestamp: ts4}, + // This is a non-existing key that needs to skip gracefully without + // distorting stats. Checking that absence of next key is handled. + {Key: roachpb.Key("r-5"), Timestamp: ts4}, + + {Key: roachpb.Key("t"), Timestamp: ts4}, } if err := MVCCGarbageCollect( - context.Background(), engine, ms, keys, ts3, + context.Background(), engine, ms, gcKeys, gcTime, ); err != nil { t.Fatal(err) } + if log.V(1) { + kvsn, err := Scan(engine, localMax, keyMax, 0) + if err != nil { + t.Fatal(err) + } + log.Info(context.Background(), "after") + for i, kv := range kvsn { + log.Infof(context.Background(), "%d: %s", i, kv.Key) + } + } + expEncKeys := []MVCCKey{ mvccVersionKey(roachpb.Key("a"), ts2), mvccVersionKey(roachpb.Key("b"), ts3), @@ -4859,11 +4909,12 @@ func TestMVCCGarbageCollect(t *testing.T) { } // Verify aggregated stats match computed stats after GC. - iter := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, + IterOptions{UpperBound: roachpb.KeyMax, KeyTypes: IterKeyTypePointsAndRanges}) defer iter.Close() for _, mvccStatsTest := range mvccStatsTests { t.Run(mvccStatsTest.name, func(t *testing.T) { - expMS, err := mvccStatsTest.fn(iter, localMax, roachpb.KeyMax, ts3.WallTime) + expMS, err := mvccStatsTest.fn(iter, localMax, roachpb.KeyMax, gcTime.WallTime) if err != nil { t.Fatal(err) } From 285aead75f77740df357a370aff37e91e579ff8a Mon Sep 17 00:00:00 2001 From: Oleg Afanasyev Date: Wed, 15 Jun 2022 11:37:43 +0100 Subject: [PATCH 04/16] kvserver/gc: remove range tombstones during GC Previously range tombstones were taken into account when doing point key GC, but were never removed themselves. This PR is adding support for removal of old range keys. This PR is extending GCRequest to include range tombstones. Range tombstones are populated by GC in requests sent after all the point keys under the GC threshold are deleted thus guaranteeing that point keys are not accidentally exposed. When processing GC range tombstone requests, replica does an addtional step to validate these assumptions and fail deletions if there's data still covered by range tombstones. Release note: None --- pkg/kv/kvserver/batcheval/BUILD.bazel | 2 + pkg/kv/kvserver/batcheval/cmd_delete_range.go | 20 - pkg/kv/kvserver/batcheval/cmd_gc.go | 84 ++- pkg/kv/kvserver/batcheval/ranges.go | 33 + pkg/kv/kvserver/batcheval/ranges_test.go | 117 +++ pkg/kv/kvserver/gc/data_distribution_test.go | 21 +- pkg/kv/kvserver/gc/gc.go | 141 +++- pkg/kv/kvserver/gc/gc_iterator.go | 6 +- pkg/kv/kvserver/gc/gc_old_test.go | 6 +- pkg/kv/kvserver/gc/gc_random_test.go | 223 +++++- pkg/kv/kvserver/gc/gc_test.go | 247 ++++++- pkg/kv/kvserver/metrics.go | 8 + pkg/kv/kvserver/mvcc_gc_queue.go | 2 + pkg/kv/kvserver/mvcc_gc_queue_test.go | 199 +++-- pkg/kv/kvserver/rditer/replica_data_iter.go | 45 +- .../kvserver/rditer/replica_data_iter_test.go | 6 +- pkg/roachpb/api.proto | 9 +- pkg/storage/mvcc.go | 331 ++++++++- pkg/storage/mvcc_test.go | 690 +++++++++++++++++- pkg/ts/catalog/chart_catalog.go | 4 + 20 files changed, 2048 insertions(+), 146 deletions(-) create mode 100644 pkg/kv/kvserver/batcheval/ranges.go create mode 100644 pkg/kv/kvserver/batcheval/ranges_test.go diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 1220bafd9e6c..a62c7057fba0 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -47,6 +47,7 @@ go_library( "declare.go", "eval_context.go", "intent.go", + "ranges.go", "split_stats_helper.go", "stateloader.go", "transaction.go", @@ -118,6 +119,7 @@ go_test( "declare_test.go", "intent_test.go", "main_test.go", + "ranges_test.go", "testutils_test.go", "transaction_test.go", ], diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 4c063d2692cf..f0c0c52cd82e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -117,23 +117,3 @@ func DeleteRange( // error is not consumed by the caller because the result will be discarded. return result.FromAcquiredLocks(h.Txn, deleted...), err } - -// rangeTombstonePeekBounds returns the left and right bounds that -// MVCCDeleteRangeUsingTombstone can read in order to detect adjacent range -// tombstones to merge with or fragment. The bounds will be truncated to the -// Raft range bounds if given. -func rangeTombstonePeekBounds( - startKey, endKey, rangeStart, rangeEnd roachpb.Key, -) (roachpb.Key, roachpb.Key) { - leftPeekBound := startKey.Prevish(roachpb.PrevishKeyLength) - if len(rangeStart) > 0 && leftPeekBound.Compare(rangeStart) <= 0 { - leftPeekBound = rangeStart - } - - rightPeekBound := endKey.Next() - if len(rangeEnd) > 0 && rightPeekBound.Compare(rangeEnd) >= 0 { - rightPeekBound = rangeEnd - } - - return leftPeekBound.Clone(), rightPeekBound.Clone() -} diff --git a/pkg/kv/kvserver/batcheval/cmd_gc.go b/pkg/kv/kvserver/batcheval/cmd_gc.go index d09ede767dba..2f9290bf2015 100644 --- a/pkg/kv/kvserver/batcheval/cmd_gc.go +++ b/pkg/kv/kvserver/batcheval/cmd_gc.go @@ -12,6 +12,7 @@ package batcheval import ( "context" + "sort" "time" "github.com/cockroachdb/cockroach/pkg/keys" @@ -44,6 +45,17 @@ func declareKeysGC( latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: key.Key}, header.Timestamp) } } + // Extend the range key latches by feather to ensure MVCC stats + // computations correctly account for adjacent range keys tombstones if they + // need to be split. + // TODO(oleg): These latches are very broad and will be disruptive to read and + // write operations despite only accessing "stale" data. We should think of + // better integrating it with latchless GC approach. + for _, span := range mergeAdjacentSpans(makeLookupBoundariesForGCRanges(rs.GetStartKey().AsRawKey(), + nil, gcr.RangeKeys)) { + latchSpans.AddMVCC(spanset.SpanReadWrite, span, + header.Timestamp) + } // Be smart here about blocking on the threshold keys. The MVCC GC queue can // send an empty request first to bump the thresholds, and then another one // that actually does work but can avoid declaring these keys below. @@ -54,6 +66,25 @@ func declareKeysGC( latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) } +// Create latches and merge adjacent. +func mergeAdjacentSpans(spans []roachpb.Span) []roachpb.Span { + if len(spans) == 0 { + return nil + } + sort.Slice(spans, func(i, j int) bool { + return spans[i].Key.Compare(spans[j].Key) < 0 + }) + j := 0 + for i := 1; i < len(spans); i++ { + if spans[i].Key.Compare(spans[j].EndKey) < 0 { + spans[j].EndKey = spans[i].EndKey + } else { + j++ + } + } + return spans[0 : j+1] +} + // GC iterates through the list of keys to garbage collect // specified in the arguments. MVCCGarbageCollect is invoked on each // listed key along with the expiration timestamp. The GC metadata @@ -85,7 +116,8 @@ func GC( // 2. the read could be served off a follower, which could be applying the // GC request's effect from the raft log. Latches held on the leaseholder // would have no impact on a follower read. - if !args.Threshold.IsEmpty() && len(args.Keys) != 0 && + if !args.Threshold.IsEmpty() && + (len(args.Keys) != 0 || len(args.RangeKeys) != 0) && !cArgs.EvalCtx.EvalKnobs().AllowGCWithNewThresholdAndKeys { return result.Result{}, errors.AssertionFailedf( "GC request can set threshold or it can GC keys, but it is unsafe for it to do both") @@ -119,6 +151,16 @@ func GC( } } + // Garbage collect range keys. Note that we pass latch range boundaries for + // each key as we may need to merge range keys with adjacent ones, but we + // are restricted on how far we are allowed to read. + desc := cArgs.EvalCtx.Desc() + rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), + desc.EndKey.AsRawKey(), args.RangeKeys) + if err := storage.MVCCGarbageCollectRangeKeys(ctx, readWriter, cArgs.Stats, rangeKeys); err != nil { + return result.Result{}, err + } + // Optionally bump the GC threshold timestamp. var res result.Result if !args.Threshold.IsEmpty() { @@ -147,3 +189,43 @@ func GC( return res, nil } + +// makeLookupBoundariesForGCRanges creates spans that could be used for latches +// and iterators when performing range tombstone garbage collection. Each of +// spans includes additional keys to the left and right of the GD'd range to +// ensure merging of range tombstones could be performed and at the same time +// no data is accessed outside of latches. +func makeLookupBoundariesForGCRanges( + rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, +) []roachpb.Span { + spans := make([]roachpb.Span, len(rangeKeys)) + for i := range rangeKeys { + l, r := rangeTombstonePeekBounds(rangeKeys[i].StartKey, rangeKeys[i].EndKey, rangeStart, rangeEnd) + spans[i] = roachpb.Span{ + Key: l, + EndKey: r, + } + } + return spans +} + +// makeCollectableGCRangesFromGCRequests creates GC collectable ranges +// containing ranges to be removed as well as safe iteration boundaries. +// See makeLookupBoundariesForGCRanges for why additional boundaries are used. +func makeCollectableGCRangesFromGCRequests( + rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, +) []storage.CollectableGCRangeKey { + latches := makeLookupBoundariesForGCRanges(rangeStart, rangeEnd, rangeKeys) + collectableKeys := make([]storage.CollectableGCRangeKey, len(rangeKeys)) + for i, rk := range rangeKeys { + collectableKeys[i] = storage.CollectableGCRangeKey{ + MVCCRangeKey: storage.MVCCRangeKey{ + StartKey: rk.StartKey, + EndKey: rk.EndKey, + Timestamp: rk.Timestamp, + }, + LatchSpan: latches[i], + } + } + return collectableKeys +} diff --git a/pkg/kv/kvserver/batcheval/ranges.go b/pkg/kv/kvserver/batcheval/ranges.go new file mode 100644 index 000000000000..967fc552ff19 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/ranges.go @@ -0,0 +1,33 @@ +// Copyright 2022 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 batcheval + +import "github.com/cockroachdb/cockroach/pkg/roachpb" + +// rangeTombstonePeekBounds returns the left and right bounds that +// MVCCDeleteRangeUsingTombstone can read in order to detect adjacent range +// tombstones to merge with or fragment. The bounds will be truncated to the +// Raft range bounds if given. +func rangeTombstonePeekBounds( + startKey, endKey, rangeStart, rangeEnd roachpb.Key, +) (roachpb.Key, roachpb.Key) { + leftPeekBound := startKey.Prevish(roachpb.PrevishKeyLength) + if len(rangeStart) > 0 && leftPeekBound.Compare(rangeStart) <= 0 { + leftPeekBound = rangeStart + } + + rightPeekBound := endKey.Next() + if len(rangeEnd) > 0 && rightPeekBound.Compare(rangeEnd) >= 0 { + rightPeekBound = rangeEnd + } + + return leftPeekBound.Clone(), rightPeekBound.Clone() +} diff --git a/pkg/kv/kvserver/batcheval/ranges_test.go b/pkg/kv/kvserver/batcheval/ranges_test.go new file mode 100644 index 000000000000..f74c97a75985 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/ranges_test.go @@ -0,0 +1,117 @@ +// Copyright 2022 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 batcheval + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestMergeGCRangeBoundaries(t *testing.T) { + defer leaktest.AfterTest(t)() + + gcr := func(start, end roachpb.Key) roachpb.GCRequest_GCRangeKey { + return roachpb.GCRequest_GCRangeKey{ + StartKey: start, + EndKey: end, + } + } + span := func(start, end roachpb.Key) roachpb.Span { + return roachpb.Span{ + Key: start, + EndKey: end, + } + } + key := func(k string) roachpb.Key { + return roachpb.Key(k) + } + preKey := func(k string) roachpb.Key { + l, _ := rangeTombstonePeekBounds(key(k), key(k+"zzzzzzz"), nil, nil) + return l + } + postKey := func(k string) roachpb.Key { + _, r := rangeTombstonePeekBounds(key(""), key(k), nil, nil) + return r + } + + for _, d := range []struct { + name string + rangeStart roachpb.Key + rangeEnd roachpb.Key + rangeKeys []roachpb.GCRequest_GCRangeKey + spans []roachpb.Span + }{ + { + name: "empty", + rangeStart: key("a"), + rangeEnd: key("b"), + rangeKeys: []roachpb.GCRequest_GCRangeKey{}, + spans: nil, + }, + { + name: "full range", + rangeStart: key("a"), + rangeEnd: key("b"), + rangeKeys: []roachpb.GCRequest_GCRangeKey{ + gcr(key("a"), key("b")), + }, + spans: []roachpb.Span{ + span(key("a"), key("b")), + }, + }, + { + name: "sub range", + rangeStart: key("a"), + rangeEnd: key("z"), + rangeKeys: []roachpb.GCRequest_GCRangeKey{ + gcr(key("c"), key("d")), + }, + spans: []roachpb.Span{ + span(preKey("c"), postKey("d")), + }, + }, + { + name: "non adjacent", + rangeStart: key("a"), + rangeEnd: key("z"), + rangeKeys: []roachpb.GCRequest_GCRangeKey{ + gcr(key("c"), key("d")), + gcr(key("e"), key("f")), + }, + spans: []roachpb.Span{ + span(preKey("c"), postKey("d")), + span(preKey("e"), postKey("f")), + }, + }, + { + name: "merge adjacent", + rangeStart: key("a"), + rangeEnd: key("z"), + rangeKeys: []roachpb.GCRequest_GCRangeKey{ + gcr(key("a"), key("b")), + gcr(key("b"), key("c")), + gcr(key("c"), key("d")), + }, + spans: []roachpb.Span{ + span(key("a"), postKey("d")), + }, + }, + } { + t.Run(d.name, func(t *testing.T) { + spans := makeLookupBoundariesForGCRanges(d.rangeStart, d.rangeEnd, d.rangeKeys) + merged := mergeAdjacentSpans(spans) + require.Equal(t, d.spans, merged, "combined spans") + }) + } +} diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index 669aa2a9767c..244ff793b7c3 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -296,7 +296,7 @@ func newDataDistribution( } retries = 0 } - return nextKey, nil, keyTimestamps, hasIntent + return nextKey, unusedEndKey, keyTimestamps, hasIntent } generateRangeKey := func() (startKey, endKey roachpb.Key, timestamps []hlc.Timestamp, hasIntent bool) { @@ -375,14 +375,27 @@ type distSpec interface { // uniformDistSpec is a distSpec which represents uniform distributions over its // various dimensions. type uniformDistSpec struct { - tsSecFrom, tsSecTo int64 // seconds + tsSecFrom, tsSecTo int64 + // Intents are split into two categories with distinct time ranges. + // All intents have lower timestamp bound to ensure they don't overlap with + // range tombstones since we will not be able to put a range tombstone over + // intent. + // Additionally, we have two time thresholds for intents. This is needed to + // ensure that we have certain fraction of intents GC'able since they lay + // below certain threshold. tsSecMinIntent, tsSecOldIntentTo int64 keySuffixMin, keySuffixMax int valueLenMin, valueLenMax int deleteFrac float64 + // keysPerValue parameters determine number of versions for a key. This number + // includes tombstones and intents which may be present on top of the history. keysPerValueMin, keysPerValueMax int - intentFrac, oldIntentFrac float64 - rangeKeyFrac float64 + // Fractions define how likely is that a key will belong to one of categories. + // If we only had a single version for each key, then that would be fraction + // of total number of objects, but if we have many versions, this value would + // roughly be total objects/avg(keysPerValueMin, keysPerValueMax) * frac. + intentFrac, oldIntentFrac float64 + rangeKeyFrac float64 } var _ distSpec = uniformDistSpec{} diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 151f6c756ad8..21bbff07c058 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -19,12 +19,14 @@ import ( "context" "fmt" "math" + "sort" "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" @@ -160,7 +162,7 @@ type Info struct { // Stats about the userspace key-values considered, namely the number of // keys with GC'able data, the number of "old" intents and the number of // associated distinct transactions. - NumKeysAffected, IntentsConsidered, IntentTxns int + NumKeysAffected, NumRangeKeysAffected, IntentsConsidered, IntentTxns int // TransactionSpanTotal is the total number of entries in the transaction span. TransactionSpanTotal int // Summary of transactions which were found GCable (assuming that @@ -194,6 +196,13 @@ type Info struct { // AffectedVersionsValBytes is the number of (fully encoded) bytes deleted from values in the storage engine. // See AffectedVersionsKeyBytes for caveats. AffectedVersionsValBytes int64 + // AffectedVersionsRangeKeyBytes is the number of (fully encoded) bytes deleted from range keys. + // For this counter, we don't count start and end key unless all versions are deleted, but we + // do count timestamp size for each version. + AffectedVersionsRangeKeyBytes int64 + // AffectedVersionsRangeValBytes is the number of (fully encoded) bytes deleted from values that + // belong to removed range keys. + AffectedVersionsRangeValBytes int64 } // RunOptions contains collection of limits that GC run applies when performing operations @@ -268,6 +277,10 @@ func Run( if err != nil { return Info{}, err } + err = processReplicatedRangeTombstones(ctx, desc, snap, now, newThreshold, gcer, &info) + if err != nil { + return Info{}, err + } // From now on, all keys processed are range-local and inline (zero timestamp). @@ -739,6 +752,132 @@ func processAbortSpan( }) } +type rangeKeyBatcher struct { + gcer GCer + batchSize int64 + + pending []storage.MVCCRangeKey + pendingSize int64 +} + +// addAndMaybeFlushRangeFragment will try to add fragment to existing batch +// and flush it if batch is full. +// unsafeRangeKeyValues contains all range key values with the same key range +// that has to be GCd. +// To ensure the resulting batch is not too large, we need to account for all +// removed versions. This method will try to include versions from oldest to +// newest and will stop if we either reach batch size or reach the newest +// provided version. Only the last version of this iteration will be flushed. +// If more versions remained after flush, process would be resumed. +func (b *rangeKeyBatcher) addAndMaybeFlushRangeFragment( + ctx context.Context, unsafeRangeKeyValues []storage.MVCCRangeKeyValue, +) error { + maxKey := len(unsafeRangeKeyValues) - 1 + for i := maxKey; i >= 0; i-- { + rk := unsafeRangeKeyValues[i].RangeKey + rangeSize := int64(len(rk.StartKey)) + int64(len(rk.EndKey)) + storage.MVCCVersionTimestampSize + hasData := len(b.pending) > 0 || i < maxKey + if hasData && (b.pendingSize+rangeSize) >= b.batchSize { + // If we need to send a batch, add previous key from history that we + // already accounted for and flush pending. + if i < maxKey { + b.addRangeKey(unsafeRangeKeyValues[i+1].RangeKey) + } + if err := b.flushPendingFragments(ctx); err != nil { + return err + } + } + b.pendingSize += rangeSize + } + b.addRangeKey(unsafeRangeKeyValues[0].RangeKey) + return nil +} + +func (b *rangeKeyBatcher) addRangeKey(unsafeRk storage.MVCCRangeKey) { + if len(b.pending) == 0 { + b.pending = append(b.pending, unsafeRk.Clone()) + return + } + lastFragment := b.pending[len(b.pending)-1] + // If new fragment is adjacent to previous one and has the same timestamp, + // merge fragments. + if lastFragment.EndKey.Equal(unsafeRk.StartKey) && + lastFragment.Timestamp.Equal(unsafeRk.Timestamp) { + lastFragment.EndKey = unsafeRk.EndKey.Clone() + b.pending[len(b.pending)-1] = lastFragment + } else { + b.pending = append(b.pending, unsafeRk.Clone()) + } +} + +func (b *rangeKeyBatcher) flushPendingFragments(ctx context.Context) error { + if pendingCount := len(b.pending); pendingCount > 0 { + toSend := make([]roachpb.GCRequest_GCRangeKey, pendingCount) + for i, rk := range b.pending { + toSend[i] = roachpb.GCRequest_GCRangeKey{ + StartKey: rk.StartKey, + EndKey: rk.EndKey, + Timestamp: rk.Timestamp, + } + } + b.pending = b.pending[:0] + b.pendingSize = 0 + return b.gcer.GC(ctx, nil, toSend) + } + return nil +} + +func processReplicatedRangeTombstones( + ctx context.Context, + desc *roachpb.RangeDescriptor, + snap storage.Reader, + now hlc.Timestamp, + gcThreshold hlc.Timestamp, + gcer GCer, + info *Info, +) error { + iter := rditer.NewReplicaMVCCDataIterator(desc, snap, rditer.ReplicaDataIteratorOptions{ + Reverse: false, + IterKind: storage.MVCCKeyIterKind, + KeyTypes: storage.IterKeyTypeRangesOnly, + }) + defer iter.Close() + + b := rangeKeyBatcher{ + gcer: gcer, + batchSize: KeyVersionChunkBytes, + } + for { + ok, err := iter.Valid() + if err != nil { + return err + } + if !ok { + break + } + rangeKeys := iter.RangeKeys() + + if idx := sort.Search(len(rangeKeys), func(i int) bool { + return rangeKeys[i].RangeKey.Timestamp.LessEq(gcThreshold) + }); idx < len(rangeKeys) { + if err = b.addAndMaybeFlushRangeFragment(ctx, rangeKeys[idx:]); err != nil { + return err + } + info.NumRangeKeysAffected++ + keyBytes := storage.MVCCVersionTimestampSize * int64(len(rangeKeys)-idx) + if idx == 0 { + keyBytes += int64(len(rangeKeys[0].RangeKey.StartKey) + len(rangeKeys[0].RangeKey.EndKey)) + } + info.AffectedVersionsRangeKeyBytes += keyBytes + for _, v := range rangeKeys[idx:] { + info.AffectedVersionsRangeValBytes += int64(len(v.Value)) + } + } + iter.Next() + } + return b.flushPendingFragments(ctx) +} + // batchingInlineGCer is a helper to paginate the GC of inline (i.e. zero // timestamp keys). After creation, keys are added via FlushingAdd(). A // final call to Flush() empties out the buffer when all keys were added. diff --git a/pkg/kv/kvserver/gc/gc_iterator.go b/pkg/kv/kvserver/gc/gc_iterator.go index e5c60e38bdd7..c657033dcfed 100644 --- a/pkg/kv/kvserver/gc/gc_iterator.go +++ b/pkg/kv/kvserver/gc/gc_iterator.go @@ -40,7 +40,11 @@ func makeGCIterator( desc *roachpb.RangeDescriptor, snap storage.Reader, threshold hlc.Timestamp, ) gcIterator { return gcIterator{ - it: rditer.NewReplicaMVCCDataIterator(desc, snap, true /* seekEnd */), + it: rditer.NewReplicaMVCCDataIterator(desc, snap, rditer.ReplicaDataIteratorOptions{ + Reverse: true, + IterKind: storage.MVCCKeyAndIntentsIterKind, + KeyTypes: storage.IterKeyTypePointsAndRanges, + }), threshold: threshold, } } diff --git a/pkg/kv/kvserver/gc/gc_old_test.go b/pkg/kv/kvserver/gc/gc_old_test.go index d70f38099108..1aa8a63562d7 100644 --- a/pkg/kv/kvserver/gc/gc_old_test.go +++ b/pkg/kv/kvserver/gc/gc_old_test.go @@ -51,7 +51,11 @@ func runGCOld( cleanupTxnIntentsAsyncFn CleanupTxnIntentsAsyncFunc, ) (Info, error) { - iter := rditer.NewReplicaMVCCDataIterator(desc, snap, false /* seekEnd */) + iter := rditer.NewReplicaMVCCDataIterator(desc, snap, rditer.ReplicaDataIteratorOptions{ + Reverse: false, + IterKind: storage.MVCCKeyAndIntentsIterKind, + KeyTypes: storage.IterKeyTypePointsOnly, + }) defer iter.Close() // Compute intent expiration (intent age at which we attempt to resolve). diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index ad4a09fa08a8..07be1a9a3341 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -83,7 +84,7 @@ var ( keysPerValueMax: 100, intentFrac: .1, oldIntentFrac: .1, - rangeKeyFrac: .05, + rangeKeyFrac: .1, } ) @@ -242,6 +243,15 @@ func TestNewVsInvariants(t *testing.T) { ttlSec: 10, intentAgeSec: 15, }, + { + ds: someVersionsWithSomeRangeKeys, + now: hlc.Timestamp{ + WallTime: 100 * time.Second.Nanoseconds(), + }, + // Higher TTL means range tombstones between 70 sec and 50 sec are + // not removed. + ttlSec: 50, + }, } { t.Run(fmt.Sprintf("%v@%v,ttl=%vsec", tc.ds, tc.now, tc.ttlSec), func(t *testing.T) { rng := rand.New(rand.NewSource(1)) @@ -268,7 +278,7 @@ func TestNewVsInvariants(t *testing.T) { // Handle GC + resolve intents. var stats enginepb.MVCCStats require.NoError(t, - storage.MVCCGarbageCollect(ctx, eng, &stats, gcer.requests(), gcThreshold)) + storage.MVCCGarbageCollect(ctx, eng, &stats, gcer.pointKeys(), gcThreshold)) for _, i := range gcer.intents { l := roachpb.LockUpdate{ Span: roachpb.Span{Key: i.Key}, @@ -278,6 +288,12 @@ func TestNewVsInvariants(t *testing.T) { _, err := storage.MVCCResolveWriteIntent(ctx, eng, &stats, l) require.NoError(t, err, "failed to resolve intent") } + for _, batch := range gcer.rangeKeyBatches() { + rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), + desc.EndKey.AsRawKey(), batch) + require.NoError(t, + storage.MVCCGarbageCollectRangeKeys(ctx, eng, &stats, rangeKeys)) + } assertLiveData(t, eng, beforeGC, *desc, tc.now, gcThreshold, intentThreshold, ttl, gcInfoNew) @@ -293,9 +309,13 @@ type historyItem struct { // assertLiveData will create a stream of expected values based on full data // set contained in provided "before" reader and compare it with the "after" // reader that contains data after applying GC request. +// Same process is then repeated with range tombstones. +// For range tombstones, we merge all the fragments before asserting to avoid +// any dependency on how key splitting is done inside pebble. // Generated expected values are produces by simulating GC in a naive way where // each value is considered live if: // - it is a value or tombstone and its timestamp is higher than gc threshold +// - it is a range tombstone and its timestamp is higher than gc threshold // - it is a first value at or below gc threshold and there are no deletions // between gc threshold and the value func assertLiveData( @@ -330,8 +350,19 @@ func assertLiveData( pointExpectationsGenerator := getExpectationsGenerator(t, pointIt, gcThreshold, intentThreshold, &expInfo) + rangeIt := before.NewMVCCIterator(storage.MVCCKeyIterKind, + storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypeRangesOnly, + }) + defer rangeIt.Close() + rangeIt.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + expectedRanges := mergeRanges(filterRangeFragments(rangeFragmentsFromIt(t, rangeIt), gcThreshold, + &expInfo)) + // Loop over engine data after applying GCer requests and compare with - // expected ranges. + // expected point keys. itAfter := after.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ LowerBound: desc.StartKey.AsRawKey(), UpperBound: desc.EndKey.AsRawKey(), @@ -369,6 +400,28 @@ func assertLiveData( } } + rangeItAfter := after.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypeRangesOnly, + RangeKeyMaskingBelow: hlc.Timestamp{}, + }) + defer rangeItAfter.Close() + rangeItAfter.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + actualRanges := mergeRanges(rangeFragmentsFromIt(t, rangeItAfter)) + + // Be careful when enabling logging on tests with default large N, + // 1000 elements is ok, but 10k or 100k entries might become unreadable. + if log.V(1) { + ctx := context.Background() + log.Info(ctx, "Expected data:") + for _, l := range formatTable(engineData(t, before, desc), desc.StartKey.AsRawKey()) { + log.Infof(ctx, "%s", l) + } + } + + require.EqualValues(t, expectedRanges, actualRanges, "GC'd range tombstones") + require.EqualValues(t, expInfo, gcInfo, "collected gc info mismatch") } @@ -554,12 +607,121 @@ func getKeyHistory(t *testing.T, r storage.Reader, key roachpb.Key) string { return strings.Join(result, ", ") } +func rangeFragmentsFromIt(t *testing.T, it storage.MVCCIterator) [][]storage.MVCCRangeKeyValue { + var result [][]storage.MVCCRangeKeyValue + for { + ok, err := it.Valid() + require.NoError(t, err, "failed to read range tombstones from iterator") + if !ok { + break + } + _, r := it.HasPointAndRange() + if r { + fragments := make([]storage.MVCCRangeKeyValue, len(it.RangeKeys())) + for i, r := range it.RangeKeys() { + fragments[i] = r.Clone() + } + result = append(result, fragments) + } + it.NextKey() + } + return result +} + +// Filter all fragments that match GC criteria and update gcinfo accordingly. +func filterRangeFragments( + fragments [][]storage.MVCCRangeKeyValue, gcThreshold hlc.Timestamp, expInfo *Info, +) [][]storage.MVCCRangeKeyValue { + var result [][]storage.MVCCRangeKeyValue + for _, stack := range fragments { + var newStack []storage.MVCCRangeKeyValue + for i, r := range stack { + if r.RangeKey.Timestamp.LessEq(gcThreshold) { + // Update expectations: + // On lowest range timestamp bump range counter. + if i == len(stack)-1 { + expInfo.NumRangeKeysAffected++ + } + // If all fragments are deleted then keys bytes are accounted for. + if i == 0 { + expInfo.AffectedVersionsRangeKeyBytes += int64(len(r.RangeKey.StartKey) + len(r.RangeKey.EndKey)) + } + // Count timestamps for all versions of range keys. + expInfo.AffectedVersionsRangeKeyBytes += storage.MVCCVersionTimestampSize + expInfo.AffectedVersionsRangeValBytes += int64(len(r.Value)) + } else { + newStack = append(newStack, r) + } + } + if len(newStack) > 0 { + result = append(result, newStack) + } + } + return result +} + +func mergeRanges(fragments [][]storage.MVCCRangeKeyValue) []storage.MVCCRangeKeyValue { + var result []storage.MVCCRangeKeyValue + var partialRangeKeys []storage.MVCCRangeKeyValue + var lastEnd roachpb.Key + for _, stack := range fragments { + start, end := stack[0].RangeKey.StartKey, stack[0].RangeKey.EndKey + if lastEnd.Equal(start) { + // Try merging keys by timestamp. + var newPartial []storage.MVCCRangeKeyValue + i, j := 0, 0 + for i < len(stack) && j < len(partialRangeKeys) { + switch stack[i].RangeKey.Timestamp.Compare(partialRangeKeys[j].RangeKey.Timestamp) { + case 1: + newPartial = append(newPartial, stack[i]) + i++ + case 0: + // We don't compare range values here as it would complicate things + // too much and not worth for this test as we don't expect mixed + // tombstone types. + newPartial = append(newPartial, storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: partialRangeKeys[j].RangeKey.StartKey, + EndKey: stack[i].RangeKey.EndKey, + Timestamp: partialRangeKeys[j].RangeKey.Timestamp, + }, + Value: partialRangeKeys[j].Value, + }) + i++ + j++ + case -1: + newPartial = append(newPartial, partialRangeKeys[j]) + j++ + } + } + for ; i < len(stack); i++ { + newPartial = append(newPartial, stack[i]) + } + for ; j < len(partialRangeKeys); j++ { + newPartial = append(newPartial, partialRangeKeys[j]) + } + partialRangeKeys = newPartial + } else { + result = append(result, partialRangeKeys...) + partialRangeKeys = make([]storage.MVCCRangeKeyValue, 0, len(stack)) + partialRangeKeys = append(partialRangeKeys, stack...) + } + lastEnd = end + } + result = append(result, partialRangeKeys...) + return result +} + type fakeGCer struct { - gcKeys map[string]roachpb.GCRequest_GCKey - threshold Threshold - intents []roachpb.Intent - batches [][]roachpb.Intent - txnIntents []txnIntents + gcKeys map[string]roachpb.GCRequest_GCKey + // fake GCer stores range key batches as it since we need to be able to + // feed them into MVCCGarbageCollectRangeKeys and ranges argument should be + // non-overlapping. + gcRangeKeyBatches [][]roachpb.GCRequest_GCRangeKey + threshold Threshold + intents []roachpb.Intent + batches [][]roachpb.Intent + txnIntents []txnIntents } func makeFakeGCer() fakeGCer { @@ -581,6 +743,7 @@ func (f *fakeGCer) GC( for _, k := range keys { f.gcKeys[k.Key.String()] = k } + f.gcRangeKeyBatches = append(f.gcRangeKeyBatches, rangeKeys) return nil } @@ -609,7 +772,7 @@ func (f *fakeGCer) normalize() { f.batches = nil } -func (f *fakeGCer) requests() []roachpb.GCRequest_GCKey { +func (f *fakeGCer) pointKeys() []roachpb.GCRequest_GCKey { var reqs []roachpb.GCRequest_GCKey for _, r := range f.gcKeys { reqs = append(reqs, r) @@ -617,6 +780,18 @@ func (f *fakeGCer) requests() []roachpb.GCRequest_GCKey { return reqs } +func (f *fakeGCer) rangeKeyBatches() [][]roachpb.GCRequest_GCRangeKey { + return f.gcRangeKeyBatches +} + +func (f *fakeGCer) rangeKeys() []roachpb.GCRequest_GCRangeKey { + var reqs []roachpb.GCRequest_GCRangeKey + for _, r := range f.gcRangeKeyBatches { + reqs = append(reqs, r...) + } + return reqs +} + func intentLess(a, b *roachpb.Intent) bool { cmp := a.Key.Compare(b.Key) switch { @@ -633,3 +808,33 @@ type txnIntents struct { txn *roachpb.Transaction intents []roachpb.LockUpdate } + +// makeCollectableGCRangesFromGCRequests mirrors +// MakeCollectableGCRangesFromGCRequests to break cyclic dependecies. +func makeCollectableGCRangesFromGCRequests( + rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, +) []storage.CollectableGCRangeKey { + collectableKeys := make([]storage.CollectableGCRangeKey, len(rangeKeys)) + for i, rk := range rangeKeys { + leftPeekBound := rk.StartKey.Prevish(roachpb.PrevishKeyLength) + if len(rangeStart) > 0 && leftPeekBound.Compare(rangeStart) <= 0 { + leftPeekBound = rangeStart + } + rightPeekBound := rk.EndKey.Next() + if len(rangeEnd) > 0 && rightPeekBound.Compare(rangeEnd) >= 0 { + rightPeekBound = rangeEnd + } + collectableKeys[i] = storage.CollectableGCRangeKey{ + MVCCRangeKey: storage.MVCCRangeKey{ + StartKey: rk.StartKey, + EndKey: rk.EndKey, + Timestamp: rk.Timestamp, + }, + LatchSpan: roachpb.Span{ + Key: leftPeekBound, + EndKey: rightPeekBound, + }, + } + } + return collectableKeys +} diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index b73a66479f52..29b5145b0def 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -573,8 +573,8 @@ var deleteRangeData = ` 8 | 7 | 6 | *- ->5 | *- - 4 | *- +>5 | .- + 4 | .- 3 | 2 | a b C 1 | @@ -587,8 +587,8 @@ var deleteRangeDataWithNewerValues = ` 8 | A C E *--- 7 | 6 | *-G ->5 | *- - 4 | *- I +>5 | .- + 4 | .- I 3 | 2 | b d F H i 1 | @@ -602,7 +602,7 @@ var deleteRangeMultipleValues = ` 7 | 6 | *--- >5 | - 4 | *- + 4 | .- 3 | 2 | a B C 1 | @@ -615,8 +615,8 @@ var deleteRangeDataWithIntents = ` 8 | !A !C !E 7 | 6 | *-- ->5 | *-- - 4 | *-- +>5 | .-- + 4 | .-- 3 | 2 | b d F 1 | @@ -629,16 +629,59 @@ var differentRangeStacksPerPoint = ` ---+--------------- 9 | >8 | B3 - 7 | *---------- + 7 | .---------- 6 | b2 - 5 | *---------- + 5 | .---------- 4 | a2 b1 - 3 | *---------- + 3 | .---------- 2 | a1 1 | ` +var deleteFragmentedRanges = ` + | a b c d e f g h i j +---+---------------------- + 9 | + 8 | A C F + 7 | + 6 | +>5 | .-- + 4 | d + 3 | .-------- + 2 | b f g + 1 | +` + +var deleteMergesRanges = ` + | a bb ccc d +---+--------------- + 9 | + 8 | A B F + 7 | *---------- + 6 | *---------- +>5 | .-- + 4 | + 3 | c + 2 | + 1 | +` + +var avoidMergingDifferentTs = ` + | a bb ccc d e +---+--------------- + 9 | + 8 | *-- + 7 | *-- + 6 | +>5 | .----- + 4 | + 3 | + 2 | + 1 | +` + func TestGC(t *testing.T) { + defer leaktest.AfterTest(t)() for _, d := range []struct { name string data string @@ -657,6 +700,9 @@ func TestGC(t *testing.T) { {name: "delete_range_multiple_points", data: deleteRangeMultipleValues}, {name: "delete_range_with_intents", data: deleteRangeDataWithIntents}, {name: "delete_with_different_range_stacks", data: differentRangeStacksPerPoint}, + {name: "delete_fragments_ranges", data: deleteFragmentedRanges}, + {name: "delete_merges_rages", data: deleteMergesRanges}, + {name: "avoid_merging_different_ts", data: avoidMergingDifferentTs}, } { t.Run(d.name, func(t *testing.T) { runTest(t, d.data) @@ -688,10 +734,18 @@ func runTest(t *testing.T, data string) { gcer.resolveIntents, gcer.resolveIntentsAsync) require.NoError(t, err) require.Empty(t, gcer.intents, "expecting no intents") - require.NoError(t, storage.MVCCGarbageCollect(ctx, eng, &stats, gcer.requests(), gcTS)) + require.NoError(t, + storage.MVCCGarbageCollect(ctx, eng, &stats, gcer.pointKeys(), gcTS)) + + for _, batch := range gcer.rangeKeyBatches() { + rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), + desc.EndKey.AsRawKey(), batch) + require.NoError(t, + storage.MVCCGarbageCollectRangeKeys(ctx, eng, &stats, rangeKeys)) + } ctrlEng := storage.NewDefaultInMemForTesting() - defer eng.Close() + defer ctrlEng.Close() expectedStats := dataItems.liveDistribution().setupTest(t, ctrlEng, desc) if log.V(1) { @@ -718,6 +772,8 @@ func runTest(t *testing.T, data string) { func requireEqualReaders( t *testing.T, exected storage.Reader, actual storage.Reader, desc roachpb.RangeDescriptor, ) { + // First compare only points. We assert points and ranges separately for + // simplicity. itExp := exected.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ LowerBound: desc.StartKey.AsRawKey(), UpperBound: desc.EndKey.AsRawKey(), @@ -725,6 +781,7 @@ func requireEqualReaders( RangeKeyMaskingBelow: hlc.Timestamp{}, }) defer itExp.Close() + itExp.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) itActual := actual.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ LowerBound: desc.StartKey.AsRawKey(), @@ -733,8 +790,8 @@ func requireEqualReaders( RangeKeyMaskingBelow: hlc.Timestamp{}, }) defer itActual.Close() - itExp.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) itActual.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + for { okExp, err := itExp.Valid() require.NoError(t, err, "failed to iterate values") @@ -750,10 +807,43 @@ func requireEqualReaders( itActual.UnsafeKey()) require.True(t, bytes.Equal(itExp.UnsafeValue(), itActual.UnsafeValue()), "expected value not equal to actual for key %s", itExp.UnsafeKey()) - itExp.Next() itActual.Next() } + + // Compare only ranges. + itExpRanges := exected.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypeRangesOnly, + RangeKeyMaskingBelow: hlc.Timestamp{}, + }) + defer itExpRanges.Close() + itExpRanges.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + + itActualRanges := actual.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypeRangesOnly, + RangeKeyMaskingBelow: hlc.Timestamp{}, + }) + defer itActualRanges.Close() + itActualRanges.SeekGE(storage.MVCCKey{Key: desc.StartKey.AsRawKey()}) + + for { + okExp, err := itExpRanges.Valid() + require.NoError(t, err, "failed to iterate ranges") + okAct, err := itActualRanges.Valid() + require.NoError(t, err, "failed to iterate ranges") + if !okExp && !okAct { + break + } + + require.Equal(t, okExp, okAct, "range iterators have different number of elements") + require.EqualValues(t, itExpRanges.RangeKeys(), itActualRanges.RangeKeys(), "range keys") + itExpRanges.Next() + itActualRanges.Next() + } } // dataItem is element read from test table containing mvcc key value along with @@ -1222,3 +1312,132 @@ func formatTable(data []tableCell, prefix roachpb.Key) []string { } return result } + +func TestRangeKeyBatching(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + + mkKey := func(key string) roachpb.Key { + var k roachpb.Key + k = append(k, keys.SystemSQLCodec.TablePrefix(42)...) + k = append(k, key...) + return k + } + + mkKvs := func(start, end string, tss ...int) []storage.MVCCRangeKeyValue { + var result []storage.MVCCRangeKeyValue + for _, ts := range tss { + result = append(result, storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: mkKey(start), + EndKey: mkKey(end), + Timestamp: hlc.Timestamp{ + WallTime: int64(ts) * time.Second.Nanoseconds(), + }, + }, + }) + } + return result + } + + mkGCr := func(start, end string, ts int) roachpb.GCRequest_GCRangeKey { + return roachpb.GCRequest_GCRangeKey{ + StartKey: mkKey(start), + EndKey: mkKey(end), + Timestamp: hlc.Timestamp{ + WallTime: int64(ts) * time.Second.Nanoseconds(), + }, + } + } + + for _, data := range []struct { + name string + data [][]storage.MVCCRangeKeyValue + batchSize int64 + expect []roachpb.GCRequest_GCRangeKey + }{ + { + name: "single batch", + data: [][]storage.MVCCRangeKeyValue{ + mkKvs("a", "b", 5, 3, 1), + mkKvs("c", "d", 5, 2, 1), + }, + batchSize: 99999, + expect: []roachpb.GCRequest_GCRangeKey{ + mkGCr("a", "b", 5), + mkGCr("c", "d", 5), + }, + }, + { + name: "merge adjacent", + data: [][]storage.MVCCRangeKeyValue{ + mkKvs("a", "b", 5, 3, 1), + mkKvs("b", "c", 5, 2), + mkKvs("c", "d", 3, 2), + }, + batchSize: 99999, + expect: []roachpb.GCRequest_GCRangeKey{ + mkGCr("a", "c", 5), + mkGCr("c", "d", 3), + }, + }, + { + name: "batch split stack", + data: [][]storage.MVCCRangeKeyValue{ + mkKvs("a", "b", 5, 3, 1), + mkKvs("b", "c", 5, 2), + mkKvs("c", "d", 3, 2), + }, + batchSize: 40, // We could only fit 2 keys in a batch. + expect: []roachpb.GCRequest_GCRangeKey{ + mkGCr("a", "b", 3), + mkGCr("a", "b", 5), + mkGCr("b", "c", 2), + mkGCr("b", "c", 5), + mkGCr("c", "d", 2), + mkGCr("c", "d", 3), + }, + }, + { + name: "batch split keys", + data: [][]storage.MVCCRangeKeyValue{ + mkKvs("a", "b", 5, 3, 1), + mkKvs("b", "c", 5, 2, 1), + mkKvs("c", "d", 3, 2), + }, + batchSize: 50, // We could only fit 3 keys in a batch. + expect: []roachpb.GCRequest_GCRangeKey{ + mkGCr("a", "b", 5), + mkGCr("b", "c", 5), + mkGCr("c", "d", 3), + }, + }, + { + name: "batch split and merge", + data: [][]storage.MVCCRangeKeyValue{ + mkKvs("a", "b", 5, 3), + mkKvs("b", "c", 5, 2), + mkKvs("c", "d", 5, 1), + }, + batchSize: 85, // We could only fit 5 keys in a batch. + expect: []roachpb.GCRequest_GCRangeKey{ + mkGCr("a", "c", 5), + mkGCr("c", "d", 1), + mkGCr("c", "d", 5), + }, + }, + } { + t.Run(data.name, func(t *testing.T) { + gcer := makeFakeGCer() + b := rangeKeyBatcher{ + gcer: &gcer, + batchSize: data.batchSize, + } + for _, d := range data.data { + require.NoError(t, b.addAndMaybeFlushRangeFragment(ctx, d), "failed to gc ranges") + } + require.NoError(t, b.flushPendingFragments(ctx), "failed to gc ranges") + require.EqualValues(t, data.expect, gcer.rangeKeys()) + }) + } +} diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 9adfaf4e4020..13da637c04e8 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -1144,6 +1144,12 @@ difficult to meaningfully interpret this metric.`, Measurement: "Keys", Unit: metric.Unit_COUNT, } + metaGCNumRangeKeysAffected = metric.Metadata{ + Name: "queue.gc.info.numrangekeysaffected", + Help: "Number of range keys GC'able", + Measurement: "Range Keys", + Unit: metric.Unit_COUNT, + } metaGCIntentsConsidered = metric.Metadata{ Name: "queue.gc.info.intentsconsidered", Help: "Number of 'old' intents", @@ -1658,6 +1664,7 @@ type StoreMetrics struct { // GCInfo cumulative totals. GCNumKeysAffected *metric.Counter + GCNumRangeKeysAffected *metric.Counter GCIntentsConsidered *metric.Counter GCIntentTxns *metric.Counter GCTransactionSpanScanned *metric.Counter @@ -2136,6 +2143,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { // GCInfo cumulative totals. GCNumKeysAffected: metric.NewCounter(metaGCNumKeysAffected), + GCNumRangeKeysAffected: metric.NewCounter(metaGCNumRangeKeysAffected), GCIntentsConsidered: metric.NewCounter(metaGCIntentsConsidered), GCIntentTxns: metric.NewCounter(metaGCIntentTxns), GCTransactionSpanScanned: metric.NewCounter(metaGCTransactionSpanScanned), diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 5a42f2cf8303..23b3756db078 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -490,6 +490,7 @@ func (r *replicaGCer) GC( } req := r.template() req.Keys = keys + req.RangeKeys = rangeKeys return r.send(ctx, req) } @@ -622,6 +623,7 @@ func (mgcq *mvccGCQueue) process( func updateStoreMetricsWithGCInfo(metrics *StoreMetrics, info gc.Info) { metrics.GCNumKeysAffected.Inc(int64(info.NumKeysAffected)) + metrics.GCNumRangeKeysAffected.Inc(int64(info.NumRangeKeysAffected)) metrics.GCIntentsConsidered.Inc(int64(info.IntentsConsidered)) metrics.GCIntentTxns.Inc(int64(info.IntentTxns)) metrics.GCTransactionSpanScanned.Inc(int64(info.TransactionSpanTotal)) diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 9e8ca614c722..85b63c70b7b6 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -15,6 +15,7 @@ import ( "fmt" "math/rand" "reflect" + "sort" "sync/atomic" "testing" "testing/quick" @@ -500,55 +501,99 @@ func TestMVCCGCQueueProcess(t *testing.T) { key9 := mkKey("i") key10 := mkKey("j") key11 := mkKey("k") + key12 := mkKey("l") + key13 := mkKey("m") + key14 := mkKey("n") + key15 := mkKey("o") + + type kvData struct { + key roachpb.Key + endKey roachpb.Key + ts hlc.Timestamp + del bool + txn bool + } - data := []struct { - key roachpb.Key - ts hlc.Timestamp - del bool - txn bool - }{ + mkVal := func(key roachpb.Key, ts hlc.Timestamp) kvData { + return kvData{key: key, ts: ts} + } + mkDel := func(key roachpb.Key, ts hlc.Timestamp) kvData { + return kvData{key: key, ts: ts, del: true} + } + mkTxn := func(data kvData) kvData { + data.txn = true + return data + } + mkRng := func(key, endKey roachpb.Key, ts hlc.Timestamp) kvData { + return kvData{key: key, endKey: endKey, ts: ts, del: true} + } + + data := []kvData{ // For key1, we expect first value to GC. - {key1, ts1, false, false}, - {key1, ts2, false, false}, - {key1, ts5, false, false}, + mkVal(key1, ts1), + mkVal(key1, ts2), + mkVal(key1, ts5), // For key2, we expect values to GC, even though most recent is deletion. - {key2, ts1, false, false}, - {key2, ts2m1, false, false}, // use a value < the GC time to verify it's kept - {key2, ts5, true, false}, + mkVal(key2, ts1), + mkVal(key2, ts2m1), // use a value < the GC time to verify it's kept + mkDel(key2, ts5), // For key3, we expect just ts1 to GC, because most recent deletion is intent. - {key3, ts1, false, false}, - {key3, ts2, false, false}, - {key3, ts5, true, true}, + mkVal(key3, ts1), + mkVal(key3, ts2), + mkTxn(mkDel(key3, ts5)), // For key4, expect oldest value to GC. - {key4, ts1, false, false}, - {key4, ts2, false, false}, + mkVal(key4, ts1), + mkVal(key4, ts2), // For key5, expect all values to GC (most recent value deleted). - {key5, ts1, false, false}, - {key5, ts2, true, false}, // deleted, so GC + mkVal(key5, ts1), + mkDel(key5, ts2), // deleted, so GC // For key6, expect no values to GC because most recent value is intent. - {key6, ts1, false, false}, - {key6, ts5, false, true}, + mkVal(key6, ts1), + mkTxn(mkVal(key6, ts5)), // For key7, expect no values to GC because intent is exactly 2h old. - {key7, ts2, false, false}, - {key7, ts4, false, true}, + mkVal(key7, ts2), + mkTxn(mkVal(key7, ts4)), // For key8, expect most recent value to resolve by aborting, which will clean it up. - {key8, ts2, false, false}, - {key8, ts3, true, true}, + mkVal(key8, ts2), + mkTxn(mkDel(key8, ts3)), // For key9, resolve naked intent with no remaining values. - {key9, ts3, false, true}, + mkTxn(mkVal(key9, ts3)), // For key10, GC ts1 because it's a delete but not ts3 because it's above the threshold. - {key10, ts1, true, false}, - {key10, ts3, true, false}, - {key10, ts4, false, false}, - {key10, ts5, false, false}, + mkDel(key10, ts1), + mkDel(key10, ts3), + mkVal(key10, ts4), + mkVal(key10, ts5), // For key11, we can't GC anything because ts1 isn't a delete. - {key11, ts1, false, false}, - {key11, ts3, true, false}, - {key11, ts4, true, false}, - {key11, ts5, true, false}, + mkVal(key11, ts1), + mkDel(key11, ts3), + mkDel(key11, ts4), + mkDel(key11, ts5), + // key12 has its older version covered by range tombstone and should be GCd + mkVal(key12, ts1), + mkVal(key12, ts5), + // key13 has all versions covered by range tombstone + mkVal(key13, ts1), + // This is old range tombstone below gc threshold + mkRng(key12, key14, ts2), + // This is newer range tombstone above gc threshold + mkRng(key13, key15, ts3), } + sort.Slice(data, func(i, j int) bool { + return data[i].ts.Less(data[j].ts) + }) + for i, datum := range data { + if len(datum.endKey) > 0 { + drArgs := deleteRangeArgs(datum.key, datum.endKey) + drArgs.UseRangeTombstone = true + if _, err := tc.SendWrappedWith(roachpb.Header{ + Timestamp: datum.ts, + }, &drArgs); err != nil { + t.Fatalf("%d: could not delete data: %+v", i, err) + } + continue + } if datum.del { dArgs := deleteArgs(datum.key) var txn *roachpb.Transaction @@ -566,23 +611,23 @@ func TestMVCCGCQueueProcess(t *testing.T) { }, &dArgs); err != nil { t.Fatalf("%d: could not delete data: %+v", i, err) } - } else { - pArgs := putArgs(datum.key, []byte("value")) - var txn *roachpb.Transaction - if datum.txn { - txn = newTransaction("test", datum.key, 1, tc.Clock()) - // Overwrite the timestamps set by newTransaction(). - txn.ReadTimestamp = datum.ts - txn.WriteTimestamp = datum.ts - txn.MinTimestamp = datum.ts - assignSeqNumsForReqs(txn, &pArgs) - } - if _, err := tc.SendWrappedWith(roachpb.Header{ - Timestamp: datum.ts, - Txn: txn, - }, &pArgs); err != nil { - t.Fatalf("%d: could not put data: %+v", i, err) - } + continue + } + pArgs := putArgs(datum.key, []byte("value")) + var txn *roachpb.Transaction + if datum.txn { + txn = newTransaction("test", datum.key, 1, tc.Clock()) + // Overwrite the timestamps set by newTransaction(). + txn.ReadTimestamp = datum.ts + txn.WriteTimestamp = datum.ts + txn.MinTimestamp = datum.ts + assignSeqNumsForReqs(txn, &pArgs) + } + if _, err := tc.SendWrappedWith(roachpb.Header{ + Timestamp: datum.ts, + Txn: txn, + }, &pArgs); err != nil { + t.Fatalf("%d: could not put data: %+v", i, err) } } @@ -591,18 +636,30 @@ func TestMVCCGCQueueProcess(t *testing.T) { t.Fatal(err) } - // The total size of the GC'able versions of the keys and values in Info. - // Key size: len(scratch+"a") + MVCCVersionTimestampSize (13 bytes) = 15 bytes. - // Value size: len("value") + headerSize (5 bytes) = 10 bytes. - // key1 at ts1 (15 bytes) => "value" (10 bytes) - // key2 at ts1 (15 bytes) => "value" (10 bytes) - // key3 at ts1 (15 bytes) => "value" (10 bytes) - // key4 at ts1 (15 bytes) => "value" (10 bytes) - // key5 at ts1 (15 bytes) => "value" (10 bytes) - // key5 at ts2 (15 bytes) => delete (0 bytes) - // key10 at ts1 (15 bytes) => delete (0 bytes) - var expectedVersionsKeyBytes int64 = 7 * 15 - var expectedVersionsValBytes int64 = 5 * 10 + // TODO: following computations should take care of new local timestamp + // for tombstones as they can be non-zero in size. + var ( + // The total size of the GC'able versions of the keys and values in Info. + // Key size: len(scratch+"a") + 1 + MVCCVersionTimestampSize (12 bytes) = 15 bytes. + // Value size: len("value") + headerSize (5 bytes) = 10 bytes. + // key1 at ts1 (15 bytes) => "value" (10 bytes) + // key2 at ts1 (15 bytes) => "value" (10 bytes) + // key3 at ts1 (15 bytes) => "value" (10 bytes) + // key4 at ts1 (15 bytes) => "value" (10 bytes) + // key5 at ts1 (15 bytes) => "value" (10 bytes) + // key5 at ts2 (15 bytes) => delete (0 bytes) + // key10 at ts1 (15 bytes) => delete (0 bytes) + // key12 at ts1 (15 bytes) => "value" (10 bytes) + // key13 at ts1 (15 bytes) => "value" (10 bytes) + expectedVersionsKeyBytes int64 = 9 * 15 + expectedVersionsValBytes int64 = 7 * 10 + // Range Key size: len(scratch + "x") * 2 + timestamp (12) + // Range Value size: 0 for deletion + // key13, key14 at ts1 (12) => delete (0 bytes) + // key14, key15 at ts1 (16) => delete (0 bytes) + expectedVersionsRangeKeyBytes int64 = 12 + 16 + expectedVersionsRangeValBytes int64 = 0 + ) // Call Run with dummy functions to get current Info. gcInfo, err := func() (gc.Info, error) { @@ -630,10 +687,20 @@ func TestMVCCGCQueueProcess(t *testing.T) { t.Fatal(err) } if gcInfo.AffectedVersionsKeyBytes != expectedVersionsKeyBytes { - t.Errorf("expected total keys size: %d bytes; got %d bytes", expectedVersionsKeyBytes, gcInfo.AffectedVersionsKeyBytes) + t.Errorf("expected total keys size: %d bytes; got %d bytes", expectedVersionsKeyBytes, + gcInfo.AffectedVersionsKeyBytes) } if gcInfo.AffectedVersionsValBytes != expectedVersionsValBytes { - t.Errorf("expected total values size: %d bytes; got %d bytes", expectedVersionsValBytes, gcInfo.AffectedVersionsValBytes) + t.Errorf("expected total values size: %d bytes; got %d bytes", expectedVersionsValBytes, + gcInfo.AffectedVersionsValBytes) + } + if gcInfo.AffectedVersionsRangeKeyBytes != expectedVersionsRangeKeyBytes { + t.Errorf("expected total range key size: %d bytes; got %d bytes", expectedVersionsRangeKeyBytes, + gcInfo.AffectedVersionsRangeKeyBytes) + } + if gcInfo.AffectedVersionsRangeValBytes != expectedVersionsRangeValBytes { + t.Errorf("expected total range value size: %d bytes; got %d bytes", expectedVersionsRangeValBytes, + gcInfo.AffectedVersionsRangeValBytes) } // Process through a scan queue. @@ -670,6 +737,7 @@ func TestMVCCGCQueueProcess(t *testing.T) { {key11, ts4}, {key11, ts3}, {key11, ts1}, + {key12, ts5}, } // Read data directly from engine to avoid intent errors from MVCC. // However, because the GC processing pushes transactions and @@ -694,6 +762,7 @@ func TestMVCCGCQueueProcess(t *testing.T) { } log.VEventf(ctx, 2, "%d: %s", i, kv.Key) } + t.Log("success") return nil }) } diff --git a/pkg/kv/kvserver/rditer/replica_data_iter.go b/pkg/kv/kvserver/rditer/replica_data_iter.go index 72f2ae08c5c1..c199cf6864ae 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter.go @@ -22,6 +22,16 @@ type KeyRange struct { Start, End roachpb.Key } +// ReplicaDataIteratorOptions defines ReplicaMVCCDataIterator creation options. +type ReplicaDataIteratorOptions struct { + // See NewReplicaMVCCDataIterator for details. + Reverse bool + // IterKind is passed to underlying iterator to select desired value types. + IterKind storage.MVCCIterKind + // KeyTypes is passed to underlying iterator to select desired key types. + KeyTypes storage.IterKeyType +} + // ReplicaMVCCDataIterator provides a complete iteration over MVCC or unversioned // (which can be made to look like an MVCCKey) key / value // rows in a range, including system-local metadata and user data. @@ -36,14 +46,15 @@ type KeyRange struct { // TODO(sumeer): merge with ReplicaEngineDataIterator. We can use an EngineIterator // for MVCC key ranges and convert from EngineKey to MVCCKey. type ReplicaMVCCDataIterator struct { + ReplicaDataIteratorOptions + reader storage.Reader curIndex int ranges []KeyRange // When it is non-nil, it represents the iterator for curIndex. // A non-nil it is valid, else it is either done, or err != nil. - it storage.MVCCIterator - err error - reverse bool + it storage.MVCCIterator + err error } // ReplicaEngineDataIterator provides a complete iteration over all data in a @@ -208,9 +219,9 @@ func MakeUserKeyRange(d *roachpb.RangeDescriptor) KeyRange { // replica. It iterates over the replicated key ranges excluding the lock // table key range. Separated locks are made to appear as interleaved. The // iterator can do one of reverse or forward iteration, based on whether -// seekEnd is true or false, respectively. With reverse iteration, it is -// initially positioned at the end of the last range, else it is initially -// positioned at the start of the first range. +// Reverse is true or false in ReplicaDataIteratorOptions, respectively. +// With reverse iteration, it is initially positioned at the end of the last +// range, else it is initially positioned at the start of the first range. // // The iterator requires the reader.ConsistentIterators is true, since it // creates a different iterator for each replicated key range. This is because @@ -220,17 +231,17 @@ func MakeUserKeyRange(d *roachpb.RangeDescriptor) KeyRange { // TODO(erikgrinaker): ReplicaMVCCDataIterator does not support MVCC range keys. // This should be deprecated in favor of e.g. ReplicaEngineDataIterator. func NewReplicaMVCCDataIterator( - d *roachpb.RangeDescriptor, reader storage.Reader, seekEnd bool, + d *roachpb.RangeDescriptor, reader storage.Reader, opts ReplicaDataIteratorOptions, ) *ReplicaMVCCDataIterator { if !reader.ConsistentIterators() { panic("ReplicaMVCCDataIterator needs a Reader that provides ConsistentIterators") } ri := &ReplicaMVCCDataIterator{ - reader: reader, - ranges: MakeReplicatedKeyRangesExceptLockTable(d), - reverse: seekEnd, + ReplicaDataIteratorOptions: opts, + reader: reader, + ranges: MakeReplicatedKeyRangesExceptLockTable(d), } - if ri.reverse { + if ri.Reverse { ri.curIndex = len(ri.ranges) - 1 } else { ri.curIndex = 0 @@ -249,13 +260,13 @@ func (ri *ReplicaMVCCDataIterator) tryCloseAndCreateIter() { return } ri.it = ri.reader.NewMVCCIterator( - storage.MVCCKeyAndIntentsIterKind, + ri.IterKind, storage.IterOptions{ LowerBound: ri.ranges[ri.curIndex].Start, UpperBound: ri.ranges[ri.curIndex].End, - KeyTypes: storage.IterKeyTypePointsAndRanges, + KeyTypes: ri.KeyTypes, }) - if ri.reverse { + if ri.Reverse { ri.it.SeekLT(storage.MakeMVCCMetadataKey(ri.ranges[ri.curIndex].End)) } else { ri.it.SeekGE(storage.MakeMVCCMetadataKey(ri.ranges[ri.curIndex].Start)) @@ -264,7 +275,7 @@ func (ri *ReplicaMVCCDataIterator) tryCloseAndCreateIter() { ri.err = err return } - if ri.reverse { + if ri.Reverse { ri.curIndex-- } else { ri.curIndex++ @@ -282,7 +293,7 @@ func (ri *ReplicaMVCCDataIterator) Close() { // Next advances to the next key in the iteration. func (ri *ReplicaMVCCDataIterator) Next() { - if ri.reverse { + if ri.Reverse { panic("Next called on reverse iterator") } ri.it.Next() @@ -299,7 +310,7 @@ func (ri *ReplicaMVCCDataIterator) Next() { // Prev advances the iterator one key backwards. func (ri *ReplicaMVCCDataIterator) Prev() { - if !ri.reverse { + if !ri.Reverse { panic("Prev called on forward iterator") } ri.it.Prev() diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 5f5c3f266439..bfea55947089 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -148,7 +148,11 @@ func verifyRDReplicatedOnlyMVCCIter( }, hlc.Timestamp{WallTime: 42}) readWriter = spanset.NewReadWriterAt(readWriter, &spans, hlc.Timestamp{WallTime: 42}) } - iter := NewReplicaMVCCDataIterator(desc, readWriter, reverse /* seekEnd */) + iter := NewReplicaMVCCDataIterator(desc, readWriter, ReplicaDataIteratorOptions{ + Reverse: reverse, + IterKind: storage.MVCCKeyAndIntentsIterKind, + KeyTypes: storage.IterKeyTypePointsAndRanges, + }) defer iter.Close() next := iter.Next if reverse { diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 32767f65a3dd..15558ade9add 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -936,19 +936,18 @@ message HeartbeatTxnResponse { message GCRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // Point keys message GCKey { bytes key = 1 [(gogoproto.casttype) = "Key"]; util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; } repeated GCKey keys = 3 [(gogoproto.nullable) = false]; - // Range keys + message GCRangeKey { - bytes startKey = 1 [(gogoproto.casttype) = "Key"]; - bytes endKey = 2 [(gogoproto.casttype) = "Key"]; + bytes start_key = 1 [(gogoproto.casttype) = "Key"]; + bytes end_key = 2 [(gogoproto.casttype) = "Key"]; util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; } - repeated GCRangeKey rangeKeys = 6 [(gogoproto.nullable) = false]; + repeated GCRangeKey range_keys = 6 [(gogoproto.nullable) = false]; // Threshold is the expiration timestamp. util.hlc.Timestamp threshold = 4 [(gogoproto.nullable) = false]; diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 756ef7119212..fb6d70c1afa3 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4296,12 +4296,14 @@ func MVCCGarbageCollect( if ok, err := iter.Valid(); err != nil { return err } else if ok { - // Use the previous version's timestamp if it's for this key. - if iter.UnsafeKey().Key.Equal(gcKey.Key) { - prevNanos = iter.UnsafeKey().Timestamp.WallTime + if hasPoint, _ := iter.HasPointAndRange(); hasPoint { + // Use the previous version's timestamp if it's for this key. + if iter.UnsafeKey().Key.Equal(gcKey.Key) { + prevNanos = iter.UnsafeKey().Timestamp.WallTime + } + // Seek to the first version for deletion. + iter.Next() } - // Seek to the first version for deletion. - iter.Next() } } } @@ -4392,6 +4394,325 @@ func MVCCGarbageCollect( return nil } +// tsVal encapsulate info about previous range key stack with actual keys +// stripped for efficiency. +type tsVal struct { + ts hlc.Timestamp + val []byte +} + +// rangeKeyMergeTracker is tracking potential merges of range key fragments +// when some of the versions are removed. Tracker stores minimal information +// internally that is required to decide if range key fragments will merge. +type rangeKeyMergeTracker struct { + prevEndKey roachpb.Key + prevTsValues []tsVal + ms *enginepb.MVCCStats +} + +// update updates MVCCStats if previous range key fragments are adjacent to +// current ones and remaining ones in current stack of fragments (indicated by +// removed flag), match exactly with the previous stack (by their timestamps +// and values). +func (t *rangeKeyMergeTracker) update( + startKey, endKey roachpb.Key, unsafeRangeKeys []MVCCRangeKeyValue, +) { + if t.ms == nil { + return + } + if keyCount := len(unsafeRangeKeys); keyCount > 0 && keyCount == len(t.prevTsValues) && t.prevEndKey.Equal(startKey) { + matching := true + for i, pts := range t.prevTsValues { + // We compare bytes directly since we are not interested in value + // equivalence, but identical values for merging keys. e.g. different + // value headers should not merge. + if rkv := unsafeRangeKeys[i]; !pts.ts.Equal(rkv.RangeKey.Timestamp) || !bytes.Equal(pts.val, rkv.Value) { + matching = false + break + } + } + if matching { + // All timestamps in range tombstone history matched with remaining + // timestamp in current history. Range tombstones would merge. + t.ms.Add(updateStatsOnRangeTombstoneMerge(unsafeRangeKeys)) + } + } + t.prevTsValues = t.prevTsValues[:0] + for _, rk := range unsafeRangeKeys { + t.prevTsValues = append(t.prevTsValues, tsVal{ + ts: rk.RangeKey.Timestamp, + val: append([]byte(nil), rk.Value...), + }) + } + t.prevEndKey = endKey.Clone() +} + +// CollectableGCRangeKey is a struct containing range key as well as span +// boundaries locked for particular range key. +// Range GC needs a latch span as it needs to expand iteration beyond the +// range key itself to find adjacent ranges and those ranges should be safe to +// read. +type CollectableGCRangeKey struct { + MVCCRangeKey + LatchSpan roachpb.Span +} + +// MVCCGarbageCollectRangeKeys is similar in functionality to MVCCGarbageCollect but +// operates on range keys. It does sanity checks that no values exist below +// range tombstones so that no values are exposed in case point values GC was +// not performed correctly by the level above. +func MVCCGarbageCollectRangeKeys( + ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, rks []CollectableGCRangeKey, +) error { + + var count int64 + defer func(begin time.Time) { + // TODO(oleg): this could be misleading if GC fails, but this function still + // reports how many keys were GC'd. The approach is identical to what point + // key GC does for consistency, but both places could be improved. + log.Eventf(ctx, + "done with GC evaluation for %d range keys at %.2f keys/sec. Deleted %d entries", + len(rks), float64(len(rks))*1e9/float64(timeutil.Since(begin)), count) + }(timeutil.Now()) + + if len(rks) == 0 { + return nil + } + + // Validate range keys are well formed. + for _, rk := range rks { + if err := rk.Validate(); err != nil { + return errors.Wrap(err, "failed to validate gc range keys in mvcc gc") + } + } + + sort.Slice(rks, func(i, j int) bool { + return rks[i].Compare(rks[j].MVCCRangeKey) < 0 + }) + + // Validate that keys are non-overlapping. + for i := 1; i < len(rks); i++ { + if rks[i].StartKey.Compare(rks[i-1].EndKey) < 0 { + return errors.Errorf("range keys in gc request should be non-overlapping: %s vs %s", + rks[i-1].String(), rks[i].String()) + } + } + + var iter MVCCIterator + var ptIter *MVCCIncrementalIterator + + defer func() { + if iter != nil { + iter.Close() + } + if ptIter != nil { + ptIter.Close() + } + }() + + for _, gcKey := range rks { + mergeTracker := rangeKeyMergeTracker{ms: ms} + + // Bound the iterator appropriately for the set of keys we'll be garbage + // collecting. We are using latch bounds to collect info about adjacent + // range fragments for correct MVCCStats updates. + iter = rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + LowerBound: gcKey.LatchSpan.Key, + UpperBound: gcKey.LatchSpan.EndKey, + KeyTypes: IterKeyTypeRangesOnly, + }) + + iter.SeekGE(MVCCKey{Key: gcKey.LatchSpan.Key}) + + for ; ; iter.Next() { + if ok, err := iter.Valid(); err != nil { + return err + } else if !ok { + break + } + + bounds := iter.RangeBounds() + unsafeRangeKeys := iter.RangeKeys() + + // Check if preceding range tombstone is adjacent to GC'd one. If we + // started iterating too early, just skip to next key. If boundaries + // match, then we capture start of the range and timestamps for later. + switch bounds.EndKey.Compare(gcKey.StartKey) { + case -1: + continue + case 0: + mergeTracker.update(bounds.Key, bounds.EndKey, unsafeRangeKeys) + continue + } + + // Terminate loop once we've reached a range tombstone past the right + // GC range key boundary. + if cmp := bounds.Key.Compare(gcKey.EndKey); cmp >= 0 { + mergeTracker.update(bounds.Key, bounds.EndKey, unsafeRangeKeys) + break + } + + // Check if we have a partial overlap between range tombstone and + // requested GCd range. This shouldn't happen in most cases, but we can + // have a range merge between GC run and cmd_gc execution or + // erroneous GC request and mvcc stats should be updated correctly. + // In those cases gcKey boundaries will not match underlying range + // tombstone boundaries and need to be adjusted. + trimLeft, trimRight := false, false + gcedRangeStartKey := bounds.Key + if gcKey.StartKey.Compare(gcedRangeStartKey) > 0 { + gcedRangeStartKey = gcKey.StartKey + trimLeft = true + } + gcedRangeEndKey := bounds.EndKey + if gcKey.EndKey.Compare(gcedRangeEndKey) < 0 { + gcedRangeEndKey = gcKey.EndKey + trimRight = true + } + + gcedRange := MVCCRangeKeyValue{ + RangeKey: MVCCRangeKey{ + StartKey: gcedRangeStartKey, + EndKey: gcedRangeEndKey, + }, + } + remaining := len(unsafeRangeKeys) + for i, rkv := range unsafeRangeKeys { + gcedRange.RangeKey.Timestamp = rkv.RangeKey.Timestamp + gcedRange.Value = rkv.Value + remove := rkv.RangeKey.Timestamp.LessEq(gcKey.Timestamp) + if remove { + if err := rw.ClearMVCCRangeKey(gcedRange.RangeKey); err != nil { + return err + } + remaining-- + count++ + } + if ms != nil { + topRangeKey := i == 0 + if remove { + ms.Add(updateStatsOnRangeTombstoneGC(gcedRange, topRangeKey)) + } + if trimLeft { + ms.Add(updateStatsOnRangeTombstoneSplit(gcedRange, gcedRange.RangeKey.StartKey, topRangeKey)) + } + if trimRight { + ms.Add(updateStatsOnRangeTombstoneSplit(gcedRange, gcedRange.RangeKey.EndKey, topRangeKey)) + } + } + } + + mergeTracker.update(gcedRangeStartKey, gcedRangeEndKey, unsafeRangeKeys[0:remaining]) + + // If we didn't find any removable fragments, shortcut without checking + // underlying keys. + if remaining == len(unsafeRangeKeys) { + continue + } + + // Verify that there are no remaining data under the deleted range using + // time bound iterator. + ptIter = NewMVCCIncrementalIterator(rw, MVCCIncrementalIterOptions{ + KeyTypes: IterKeyTypePointsOnly, + StartKey: gcedRangeStartKey, + EndKey: gcedRangeEndKey, + EndTime: gcKey.Timestamp, + IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, + }) + ptIter.SeekGE(MVCCKey{Key: gcedRangeStartKey}) + for ; ; ptIter.Next() { + if ok, err := ptIter.Valid(); err != nil { + return err + } else if !ok { + break + } + // Disallow any value under the range key. We only skip intents as they + // must have a provisional value with appropriate timestamp. + if pointKey := ptIter.UnsafeKey(); pointKey.IsValue() { + return errors.Errorf("attempt to delete range tombstone %q hiding key at %q", + gcKey, pointKey) + } + } + ptIter.Close() + ptIter = nil + } + + iter.Close() + iter = nil + } + + return nil +} + +// updateStatsOnRangeTombstoneGC updates stats for removed range keys. +// If all range keys are removed for the interval then this method should be +// called once with removeKeys == true. This would ensure we account for keys +// being removed. Then it should be called without removeKeys for all subsequent +// range keys in stack to account for remaining timestamps and values. +func updateStatsOnRangeTombstoneGC( + removedRange MVCCRangeKeyValue, removeKeys bool, +) (ms enginepb.MVCCStats) { + ms.AgeTo(removedRange.RangeKey.Timestamp.WallTime) + + if removeKeys { + leftKeySize := int64(EncodedMVCCKeyPrefixLength(removedRange.RangeKey.StartKey)) + rightKeySize := int64(EncodedMVCCKeyPrefixLength(removedRange.RangeKey.EndKey)) + ms.RangeKeyBytes -= leftKeySize + rightKeySize + ms.RangeKeyCount-- + } + tsSize := int64(EncodedMVCCTimestampSuffixLength(removedRange.RangeKey.Timestamp)) + valueSize := int64(len(removedRange.Value)) + ms.RangeKeyBytes -= tsSize + ms.RangeValBytes -= valueSize + ms.RangeValCount-- + return ms +} + +// updateStatsOnRangeTombstoneSplit updates stats for ranges that are being +// split during GC operation. +// If splitKeys is true, this method will account for the key sizes added by +// split. If false then only timestamp and value duplication is added. +// When splitting range, this method should be called once with splitKeys +// when processing top range key and then with false for remaining range keys. +func updateStatsOnRangeTombstoneSplit( + rangeKey MVCCRangeKeyValue, splitKey roachpb.Key, splitKeys bool, +) (ms enginepb.MVCCStats) { + ms.AgeTo(rangeKey.RangeKey.Timestamp.WallTime) + + // Mind that we add key contributions twice since they are added as the + // end key and start key for ranges. If trimmed range is removed at current + // timestamp extra value will be removed above when handling the deletion. + if splitKeys { + keySize := int64(EncodedMVCCKeyPrefixLength(splitKey)) + ms.RangeKeyBytes += keySize * 2 + ms.RangeKeyCount++ + } + + tsSize := int64(EncodedMVCCTimestampSuffixLength(rangeKey.RangeKey.Timestamp)) + valueSize := int64(len(rangeKey.Value)) + ms.RangeKeyBytes += tsSize + ms.RangeValBytes += valueSize + ms.RangeValCount++ + return ms +} + +// updateStatsOnRangeTombstoneMerge updates MVCCStats for the case where all +// range tombstone fragments merge to the left. i.e. start key is eliminated +// twice at the top of history and all versions are removed as a timestamp and +// a value. +func updateStatsOnRangeTombstoneMerge(rangeKeys []MVCCRangeKeyValue) (ms enginepb.MVCCStats) { + ms.AgeTo(rangeKeys[0].RangeKey.Timestamp.WallTime) + ms.RangeKeyBytes -= int64(EncodedMVCCKeyPrefixLength(rangeKeys[0].RangeKey.StartKey)) * 2 + ms.RangeKeyCount-- + for _, rk := range rangeKeys { + ms.AgeTo(rk.RangeKey.Timestamp.WallTime) + ms.RangeKeyBytes -= int64(EncodedMVCCTimestampSuffixLength(rk.RangeKey.Timestamp)) + ms.RangeValCount-- + ms.RangeValBytes -= int64(len(rk.Value)) + } + return ms +} + // MVCCFindSplitKey finds a key from the given span such that the left side of // the split is roughly targetSize bytes. The returned key will never be chosen // from the key ranges listed in keys.NoSplitSpans. diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 6c958bb1f7fd..59900718a7f7 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -4834,11 +4834,11 @@ func TestMVCCGarbageCollect(t *testing.T) { t.Fatal(err) } if log.V(1) { + log.Info(context.Background(), "Engine content before GC") kvsn, err := Scan(engine, localMax, keyMax, 0) if err != nil { t.Fatal(err) } - log.Info(context.Background(), "before") for i, kv := range kvsn { log.Infof(context.Background(), "%d: %s", i, kv.Key) } @@ -4879,11 +4879,11 @@ func TestMVCCGarbageCollect(t *testing.T) { } if log.V(1) { + log.Info(context.Background(), "Engine content after GC") kvsn, err := Scan(engine, localMax, keyMax, 0) if err != nil { t.Fatal(err) } - log.Info(context.Background(), "after") for i, kv := range kvsn { log.Infof(context.Background(), "%d: %s", i, kv.Key) } @@ -5236,6 +5236,692 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { } } +type rangeTestDataItem struct { + point MVCCKeyValue + txn *roachpb.Transaction + rangeTombstone MVCCRangeKey +} + +type rangeTestData []rangeTestDataItem + +func (d rangeTestData) populateEngine( + t *testing.T, engine ReadWriter, ms *enginepb.MVCCStats, +) hlc.Timestamp { + ctx := context.Background() + var ts hlc.Timestamp + for _, v := range d { + if v.rangeTombstone.Timestamp.IsEmpty() { + if v.point.Value != nil { + require.NoError(t, MVCCPut(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, + hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes(v.point.Value), v.txn), + "failed to insert test value into engine (%s)", v.point.Key.String()) + } else { + require.NoError(t, MVCCDelete(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, + hlc.ClockTimestamp{}, v.txn), + "failed to insert tombstone value into engine (%s)", v.point.Key.String()) + } + ts = v.point.Key.Timestamp + } else { + require.NoError(t, MVCCDeleteRangeUsingTombstone(ctx, engine, ms, v.rangeTombstone.StartKey, + v.rangeTombstone.EndKey, v.rangeTombstone.Timestamp, hlc.ClockTimestamp{}, nil, nil, 0), + "failed to insert range tombstone into engine (%s)", v.rangeTombstone.String()) + ts = v.rangeTombstone.Timestamp + } + } + return ts +} + +// pt creates a point update for key with default value. +func pt(key roachpb.Key, ts hlc.Timestamp) rangeTestDataItem { + val := roachpb.MakeValueFromString("testval").RawBytes + return rangeTestDataItem{point: MVCCKeyValue{Key: mvccVersionKey(key, ts), Value: val}} +} + +// txn wraps point update and adds transaction to it for intent creation. +func txn(d rangeTestDataItem) rangeTestDataItem { + ts := d.point.Key.Timestamp + d.txn = &roachpb.Transaction{ + Status: roachpb.PENDING, + ReadTimestamp: ts, + GlobalUncertaintyLimit: ts.Next().Next(), + } + d.txn.ID = uuid.MakeV4() + d.txn.WriteTimestamp = ts + d.txn.Key = roachpb.Key([]byte{0, 1}) + return d +} + +// rng creates range tombstone update. +func rng(start, end roachpb.Key, ts hlc.Timestamp) rangeTestDataItem { + return rangeTestDataItem{rangeTombstone: MVCCRangeKey{StartKey: start, EndKey: end, Timestamp: ts}} +} + +func TestMVCCGarbageCollectRanges(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + mkKey := func(k string) roachpb.Key { + return append(keys.SystemSQLCodec.TablePrefix(42), k...) + } + rangeStart := mkKey("") + rangeEnd := rangeStart.PrefixEnd() + + // Note we use keys of different lengths so that stats accounting errors + // would not obviously cancel out if right and left bounds are used + // incorrectly. + keyA := mkKey("a") + keyB := mkKey("bb") + keyC := mkKey("ccc") + keyD := mkKey("dddd") + keyE := mkKey("eeeee") + keyF := mkKey("ffffff") + + mkTs := func(wallTimeSec int64) hlc.Timestamp { + return hlc.Timestamp{WallTime: time.Second.Nanoseconds() * wallTimeSec} + } + + ts1 := mkTs(1) + ts2 := mkTs(2) + ts3 := mkTs(3) + ts4 := mkTs(4) + tsMax := mkTs(9) + + testData := []struct { + name string + // Note that range test data should be in ascending order (valid writes). + before rangeTestData + request []roachpb.GCRequest_GCRangeKey + // Note that expectations should be in timestamp descending order + // (forward iteration). + after []MVCCRangeKey + // Optional start and end range for tests that want to restrict default + // key range. + rangeStart roachpb.Key + rangeEnd roachpb.Key + }{ + { + name: "signle range", + before: rangeTestData{ + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "multiple contiguous fragments", + before: rangeTestData{ + rng(keyA, keyD, ts2), + rng(keyB, keyC, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts4}, + }, + }, + { + name: "multiple non-contiguous fragments", + before: rangeTestData{ + rng(keyA, keyB, ts2), + rng(keyC, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "multiple non-overlapping fragments", + before: rangeTestData{ + rng(keyA, keyB, ts2), + rng(keyC, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + }, + { + name: "overlapping [A--[B--B]--A]", + before: rangeTestData{ + rng(keyB, keyC, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [A--[B--A]--B]", + before: rangeTestData{ + rng(keyB, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyC, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + }, + { + name: "overlapping [B--[A--B]--A]", + before: rangeTestData{ + rng(keyA, keyC, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + }, + }, + { + name: "overlapping [B--[A--A]--B]", + before: rangeTestData{ + rng(keyA, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + }, + { + name: "overlapping [[AB--A]--B]", + before: rangeTestData{ + rng(keyA, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + }, + { + name: "overlapping [[AB--B]--A]", + before: rangeTestData{ + rng(keyA, keyB, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [B--[A--AB]]", + before: rangeTestData{ + rng(keyA, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + }, + }, + { + name: "overlapping [A--[B--AB]]", + before: rangeTestData{ + rng(keyB, keyD, ts2), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [B--[A--AB]] point before", + before: rangeTestData{ + rng(keyB, keyD, ts2), + pt(keyA, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, + }, + }, + { + name: "overlapping [B--[A--AB]] point at range start", + before: rangeTestData{ + rng(keyA, keyD, ts2), + pt(keyA, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + }, + }, + { + name: "overlapping [B--[A--AB]] point between", + before: rangeTestData{ + rng(keyA, keyD, ts2), + pt(keyB, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyC, Timestamp: ts2}, + }, + }, + { + name: "overlapping [B--[A--AB]] point at gc start", + before: rangeTestData{ + rng(keyA, keyD, ts2), + pt(keyB, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, + }, + }, + { + name: "overlapping [A--[B--AB]] point before", + before: rangeTestData{ + rng(keyC, keyD, ts2), + pt(keyA, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [A--[B--AB]] point at gc start", + before: rangeTestData{ + rng(keyB, keyD, ts2), + pt(keyA, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [A--[B--AB]] point between", + before: rangeTestData{ + rng(keyC, keyD, ts2), + pt(keyB, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "overlapping [A--[B--AB]] point at range start", + before: rangeTestData{ + rng(keyB, keyD, ts2), + pt(keyB, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "range under intent", + before: rangeTestData{ + rng(keyA, keyD, ts2), + txn(pt(keyA, ts4)), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "stacked range fragments", + before: rangeTestData{ + rng(keyB, keyC, ts2), + rng(keyA, keyD, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts4}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "old value before range", + before: rangeTestData{ + pt(keyA, ts2), + rng(keyB, keyC, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "old value at range end", + before: rangeTestData{ + pt(keyC, ts2), + rng(keyB, keyC, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, + }, + after: []MVCCRangeKey{}, + }, + { + name: "range partially overlap gc request", + before: rangeTestData{ + rng(keyA, keyD, ts1), + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts3}, + {StartKey: keyA, EndKey: keyB, Timestamp: ts1}, + {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts3}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts1}, + }, + }, + { + name: "range merges sides", + before: rangeTestData{ + rng(keyB, keyC, ts1), + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + }, + }, + { + name: "range merges next", + before: rangeTestData{ + rng(keyB, keyC, ts1), + rng(keyA, keyC, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyC, Timestamp: ts3}, + }, + }, + { + name: "range merges previous", + before: rangeTestData{ + rng(keyA, keyB, ts1), + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyB, Timestamp: ts1}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + }, + }, + { + name: "range merges chain", + before: rangeTestData{ + rng(keyB, keyC, ts1), + rng(keyD, keyE, ts2), + rng(keyA, keyF, ts3), + rng(keyA, keyF, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, + {StartKey: keyD, EndKey: keyE, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyF, Timestamp: ts4}, + {StartKey: keyA, EndKey: keyF, Timestamp: ts3}, + }, + }, + { + name: "range merges sequential", + before: rangeTestData{ + rng(keyC, keyD, ts1), + rng(keyB, keyD, ts2), + rng(keyA, keyE, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, + {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, + }, + after: []MVCCRangeKey{ + {StartKey: keyA, EndKey: keyE, Timestamp: ts3}, + }, + }, + { + name: "don't merge outside range", + before: rangeTestData{ + rng(keyB, keyC, ts1), + // Tombstone spanning multiple ranges. + rng(keyA, keyD, ts4), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, + }, + after: []MVCCRangeKey{ + // We only iterate data within range, so range keys would be + // truncated. + {StartKey: keyB, EndKey: keyC, Timestamp: ts4}, + }, + rangeStart: keyB, + rangeEnd: keyC, + }, + } + + for _, engineImpl := range mvccEngineImpls { + t.Run(engineImpl.name, func(t *testing.T) { + for _, d := range testData { + t.Run(d.name, func(t *testing.T) { + engine := engineImpl.create() + defer engine.Close() + + // Populate range descriptor defaults. + if len(d.rangeStart) == 0 { + d.rangeStart = rangeStart + } + if len(d.rangeEnd) == 0 { + d.rangeEnd = rangeEnd + } + + var ms enginepb.MVCCStats + d.before.populateEngine(t, engine, &ms) + + rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) + require.NoError(t, MVCCGarbageCollectRangeKeys(ctx, engine, &ms, rangeKeys), + "failed to run mvcc range tombstone garbage collect") + + it := engine.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + KeyTypes: IterKeyTypeRangesOnly, + LowerBound: d.rangeStart, + UpperBound: d.rangeEnd, + }) + defer it.Close() + it.SeekGE(MVCCKey{Key: d.rangeStart}) + expectIndex := 0 + for ; ; it.Next() { + ok, err := it.Valid() + require.NoError(t, err, "failed to iterate engine") + if !ok { + break + } + for _, rkv := range it.RangeKeys() { + require.Less(t, expectIndex, len(d.after), "not enough expectations; at unexpected range:", rkv.RangeKey.String()) + require.EqualValues(t, d.after[expectIndex], rkv.RangeKey, "range key is not equal") + expectIndex++ + } + } + require.Equal(t, len(d.after), expectIndex, + "not all range tombstone expectations were consumed") + + ms.AgeTo(tsMax.WallTime) + it = engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: d.rangeStart, + UpperBound: d.rangeEnd, + }) + expMs, err := ComputeStatsForRange(it, rangeStart, rangeEnd, tsMax.WallTime) + require.NoError(t, err, "failed to compute stats for range") + require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") + }) + } + }) + } +} + +func rangesFromRequests( + rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, +) []CollectableGCRangeKey { + collectableKeys := make([]CollectableGCRangeKey, len(rangeKeys)) + for i, rk := range rangeKeys { + leftPeekBound := rk.StartKey.Prevish(roachpb.PrevishKeyLength) + if leftPeekBound.Compare(rangeStart) <= 0 { + leftPeekBound = rangeStart + } + rightPeekBound := rk.EndKey.Next() + if rightPeekBound.Compare(rangeEnd) >= 0 { + rightPeekBound = rangeEnd + } + collectableKeys[i] = CollectableGCRangeKey{ + MVCCRangeKey: MVCCRangeKey{ + StartKey: rk.StartKey, + EndKey: rk.EndKey, + Timestamp: rk.Timestamp, + }, + LatchSpan: roachpb.Span{Key: leftPeekBound, EndKey: rightPeekBound}, + } + } + return collectableKeys +} + +func TestMVCCGarbageCollectRangesFailures(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + mkKey := func(k string) roachpb.Key { + return append(keys.SystemSQLCodec.TablePrefix(42), k...) + } + rangeStart := mkKey("") + rangeEnd := rangeStart.PrefixEnd() + + keyA := mkKey("a") + keyB := mkKey("b") + keyC := mkKey("c") + keyD := mkKey("d") + + mkTs := func(wallTimeSec int64) hlc.Timestamp { + return hlc.Timestamp{WallTime: time.Second.Nanoseconds() * wallTimeSec} + } + + ts1 := mkTs(1) + ts2 := mkTs(2) + ts3 := mkTs(3) + ts4 := mkTs(4) + ts5 := mkTs(5) + ts6 := mkTs(6) + ts7 := mkTs(7) + ts8 := mkTs(8) + + testData := []struct { + name string + before rangeTestData + request []roachpb.GCRequest_GCRangeKey + error string + }{ + { + name: "request overlap", + before: rangeTestData{ + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyC, Timestamp: ts3}, + {StartKey: keyB, EndKey: keyD, Timestamp: ts3}, + }, + error: "range keys in gc request should be non-overlapping", + }, + { + name: "delete range above value", + before: rangeTestData{ + pt(keyB, ts2), + rng(keyA, keyD, ts3), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, + }, + error: "attempt to delete range tombstone .* hiding key at .*", + }, + { + // Note that this test is a bit contrived as we can't put intent + // under the range tombstone, but we test that if you try to delete + // tombstone above intents even if it doesn't exist, we would reject + // the attempt as it is an indication of inconsistency. + // This might be relaxed to ignore any points which are not covered. + name: "delete range above intent", + before: rangeTestData{ + rng(keyA, keyD, ts2), + txn(pt(keyB, ts3)), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts4}, + }, + error: "attempt to delete range tombstone .* hiding key at .*", + }, + { + name: "delete range above tail of long history", + before: rangeTestData{ + pt(keyB, ts1), + rng(keyA, keyD, ts2), + pt(keyB, ts3), + pt(keyB, ts4), + pt(keyB, ts5), + pt(keyB, ts6), + pt(keyB, ts7), + pt(keyB, ts8), + }, + request: []roachpb.GCRequest_GCRangeKey{ + {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, + }, + error: "attempt to delete range tombstone .* hiding key at .*", + }, + } + + ctx := context.Background() + for _, engineImpl := range mvccEngineImpls { + t.Run(engineImpl.name, func(t *testing.T) { + for _, d := range testData { + t.Run(d.name, func(t *testing.T) { + engine := engineImpl.create() + defer engine.Close() + d.before.populateEngine(t, engine, nil) + rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) + err := MVCCGarbageCollectRangeKeys(ctx, engine, nil, rangeKeys) + require.Errorf(t, err, "expected error '%s' but found none", d.error) + require.True(t, testutils.IsError(err, d.error), + "expected error '%s' found '%s'", d.error, err) + }) + } + }) + } +} + // TestResolveIntentWithLowerEpoch verifies that trying to resolve // an intent at an epoch that is lower than the epoch of the intent // leaves the intent untouched. diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 30b8e9278272..2698dbd2abcf 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -853,6 +853,10 @@ var charts = []sectionDescription{ Title: "Keys with GC'able Data", Metrics: []string{"queue.gc.info.numkeysaffected"}, }, + { + Title: "Range Keys with GC'able Data", + Metrics: []string{"queue.gc.info.numrangekeysaffected"}, + }, { Title: "Old Intents", Metrics: []string{"queue.gc.info.intentsconsidered"}, From 4134359cfa36a08915880abc803d738fe022eb76 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 12 Jul 2022 14:50:29 -0700 Subject: [PATCH 05/16] kv: remove TargetBytesAvoidExcess version gate Release note: None --- pkg/clusterversion/cockroach_versions.go | 9 - pkg/clusterversion/key_string.go | 127 +++++----- pkg/kv/kvserver/batcheval/cmd_get_test.go | 19 +- pkg/kv/kvserver/batcheval/cmd_reverse_scan.go | 28 +-- pkg/kv/kvserver/batcheval/cmd_scan.go | 30 +-- pkg/kv/kvserver/batcheval/cmd_scan_test.go | 61 ++--- pkg/roachpb/api.proto | 10 +- pkg/storage/metamorphic/operations.go | 60 +++-- pkg/storage/mvcc.go | 43 ++-- pkg/storage/mvcc_history_test.go | 5 +- pkg/storage/pebble_mvcc_scanner.go | 11 +- .../testdata/mvcc_histories/target_bytes | 236 +----------------- 12 files changed, 176 insertions(+), 463 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 0a397552733a..69382e9d18e0 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,11 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // TargetBytesAvoidExcess prevents exceeding BatchRequest.Header.TargetBytes - // except when there is a single value in the response. 21.2 DistSender logic - // requires the limit to always be overshot in order to properly enforce - // limits when splitting requests. - TargetBytesAvoidExcess // TraceIDDoesntImplyStructuredRecording changes the contract about the kind // of span that RPCs get on the server depending on the tracing context. TraceIDDoesntImplyStructuredRecording @@ -420,10 +415,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: TargetBytesAvoidExcess, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 4}, - }, { Key: TraceIDDoesntImplyStructuredRecording, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 10}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 60c8c5dc758a..1572caa60a3f 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,73 +10,72 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[TargetBytesAvoidExcess-2] - _ = x[TraceIDDoesntImplyStructuredRecording-3] - _ = x[AlterSystemTableStatisticsAddAvgSizeCol-4] - _ = x[MVCCAddSSTable-5] - _ = x[InsertPublicSchemaNamespaceEntryOnRestore-6] - _ = x[UnsplitRangesInAsyncGCJobs-7] - _ = x[ValidateGrantOption-8] - _ = x[PebbleFormatBlockPropertyCollector-9] - _ = x[ProbeRequest-10] - _ = x[SelectRPCsTakeTracingInfoInband-11] - _ = x[PreSeedTenantSpanConfigs-12] - _ = x[SeedTenantSpanConfigs-13] - _ = x[PublicSchemasWithDescriptors-14] - _ = x[EnsureSpanConfigReconciliation-15] - _ = x[EnsureSpanConfigSubscription-16] - _ = x[EnableSpanConfigStore-17] - _ = x[ScanWholeRows-18] - _ = x[SCRAMAuthentication-19] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-20] - _ = x[AlterSystemProtectedTimestampAddColumn-21] - _ = x[EnableProtectedTimestampsForTenant-22] - _ = x[DeleteCommentsWithDroppedIndexes-23] - _ = x[RemoveIncompatibleDatabasePrivileges-24] - _ = x[AddRaftAppliedIndexTermMigration-25] - _ = x[PostAddRaftAppliedIndexTermMigration-26] - _ = x[DontProposeWriteTimestampForLeaseTransfers-27] - _ = x[EnablePebbleFormatVersionBlockProperties-28] - _ = x[DisableSystemConfigGossipTrigger-29] - _ = x[MVCCIndexBackfiller-30] - _ = x[EnableLeaseHolderRemoval-31] - _ = x[BackupResolutionInJob-32] - _ = x[LooselyCoupledRaftLogTruncation-33] - _ = x[ChangefeedIdleness-34] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-35] - _ = x[EnableDeclarativeSchemaChanger-36] - _ = x[RowLevelTTL-37] - _ = x[PebbleFormatSplitUserKeysMarked-38] - _ = x[IncrementalBackupSubdir-39] - _ = x[DateStyleIntervalStyleCastRewrite-40] - _ = x[EnableNewStoreRebalancer-41] - _ = x[ClusterLocksVirtualTable-42] - _ = x[AutoStatsTableSettings-43] - _ = x[ForecastStats-44] - _ = x[SuperRegions-45] - _ = x[EnableNewChangefeedOptions-46] - _ = x[SpanCountTable-47] - _ = x[PreSeedSpanCountTable-48] - _ = x[SeedSpanCountTable-49] - _ = x[V22_1-50] - _ = x[Start22_2-51] - _ = x[LocalTimestamps-52] - _ = x[EnsurePebbleFormatVersionRangeKeys-53] - _ = x[EnablePebbleFormatVersionRangeKeys-54] - _ = x[TrigramInvertedIndexes-55] - _ = x[RemoveGrantPrivilege-56] - _ = x[MVCCRangeTombstones-57] - _ = x[UpgradeSequenceToBeReferencedByID-58] - _ = x[SampledStmtDiagReqs-59] - _ = x[AddSSTableTombstones-60] - _ = x[SystemPrivilegesTable-61] - _ = x[EnablePredicateProjectionChangefeed-62] - _ = x[AlterSystemSQLInstancesAddLocality-63] + _ = x[TraceIDDoesntImplyStructuredRecording-2] + _ = x[AlterSystemTableStatisticsAddAvgSizeCol-3] + _ = x[MVCCAddSSTable-4] + _ = x[InsertPublicSchemaNamespaceEntryOnRestore-5] + _ = x[UnsplitRangesInAsyncGCJobs-6] + _ = x[ValidateGrantOption-7] + _ = x[PebbleFormatBlockPropertyCollector-8] + _ = x[ProbeRequest-9] + _ = x[SelectRPCsTakeTracingInfoInband-10] + _ = x[PreSeedTenantSpanConfigs-11] + _ = x[SeedTenantSpanConfigs-12] + _ = x[PublicSchemasWithDescriptors-13] + _ = x[EnsureSpanConfigReconciliation-14] + _ = x[EnsureSpanConfigSubscription-15] + _ = x[EnableSpanConfigStore-16] + _ = x[ScanWholeRows-17] + _ = x[SCRAMAuthentication-18] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-19] + _ = x[AlterSystemProtectedTimestampAddColumn-20] + _ = x[EnableProtectedTimestampsForTenant-21] + _ = x[DeleteCommentsWithDroppedIndexes-22] + _ = x[RemoveIncompatibleDatabasePrivileges-23] + _ = x[AddRaftAppliedIndexTermMigration-24] + _ = x[PostAddRaftAppliedIndexTermMigration-25] + _ = x[DontProposeWriteTimestampForLeaseTransfers-26] + _ = x[EnablePebbleFormatVersionBlockProperties-27] + _ = x[DisableSystemConfigGossipTrigger-28] + _ = x[MVCCIndexBackfiller-29] + _ = x[EnableLeaseHolderRemoval-30] + _ = x[BackupResolutionInJob-31] + _ = x[LooselyCoupledRaftLogTruncation-32] + _ = x[ChangefeedIdleness-33] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-34] + _ = x[EnableDeclarativeSchemaChanger-35] + _ = x[RowLevelTTL-36] + _ = x[PebbleFormatSplitUserKeysMarked-37] + _ = x[IncrementalBackupSubdir-38] + _ = x[DateStyleIntervalStyleCastRewrite-39] + _ = x[EnableNewStoreRebalancer-40] + _ = x[ClusterLocksVirtualTable-41] + _ = x[AutoStatsTableSettings-42] + _ = x[ForecastStats-43] + _ = x[SuperRegions-44] + _ = x[EnableNewChangefeedOptions-45] + _ = x[SpanCountTable-46] + _ = x[PreSeedSpanCountTable-47] + _ = x[SeedSpanCountTable-48] + _ = x[V22_1-49] + _ = x[Start22_2-50] + _ = x[LocalTimestamps-51] + _ = x[EnsurePebbleFormatVersionRangeKeys-52] + _ = x[EnablePebbleFormatVersionRangeKeys-53] + _ = x[TrigramInvertedIndexes-54] + _ = x[RemoveGrantPrivilege-55] + _ = x[MVCCRangeTombstones-56] + _ = x[UpgradeSequenceToBeReferencedByID-57] + _ = x[SampledStmtDiagReqs-58] + _ = x[AddSSTableTombstones-59] + _ = x[SystemPrivilegesTable-60] + _ = x[EnablePredicateProjectionChangefeed-61] + _ = x[AlterSystemSQLInstancesAddLocality-62] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1TraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 36, 73, 112, 126, 167, 193, 212, 246, 258, 289, 313, 334, 362, 392, 420, 441, 454, 473, 507, 545, 579, 611, 647, 679, 715, 757, 797, 829, 848, 872, 893, 924, 942, 983, 1013, 1024, 1055, 1078, 1111, 1135, 1159, 1181, 1194, 1206, 1232, 1246, 1267, 1285, 1290, 1299, 1314, 1348, 1382, 1404, 1424, 1443, 1476, 1495, 1515, 1536, 1571, 1605} +var _Key_index = [...]uint16{0, 5, 14, 51, 90, 104, 145, 171, 190, 224, 236, 267, 291, 312, 340, 370, 398, 419, 432, 451, 485, 523, 557, 589, 625, 657, 693, 735, 775, 807, 826, 850, 871, 902, 920, 961, 991, 1002, 1033, 1056, 1089, 1113, 1137, 1159, 1172, 1184, 1210, 1224, 1245, 1263, 1268, 1277, 1292, 1326, 1360, 1382, 1402, 1421, 1454, 1473, 1493, 1514, 1549, 1583} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/kvserver/batcheval/cmd_get_test.go b/pkg/kv/kvserver/batcheval/cmd_get_test.go index b0ddc170244d..07d6f8ac6064 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_get_test.go @@ -15,7 +15,6 @@ import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -52,7 +51,6 @@ func TestGetResumeSpan(t *testing.T) { maxKeys int64 targetBytes int64 allowEmpty bool - avoidExcess bool expectResume bool expectReason roachpb.ResumeReason expectNextBytes int64 @@ -67,27 +65,18 @@ func TestGetResumeSpan(t *testing.T) { {targetBytes: 1, expectResume: false}, {targetBytes: 11, expectResume: false}, {targetBytes: 12, expectResume: false}, - // allowEmpty takes precedence over avoidExcess at the RPC level, since - // callers have no control over avoidExcess. - {targetBytes: 1, allowEmpty: true, avoidExcess: false, expectResume: true, expectReason: roachpb.RESUME_BYTE_LIMIT, expectNextBytes: 11}, + {targetBytes: 1, allowEmpty: true, expectResume: true, expectReason: roachpb.RESUME_BYTE_LIMIT, expectNextBytes: 11}, {targetBytes: 11, allowEmpty: true, expectResume: false}, {targetBytes: 12, allowEmpty: true, expectResume: false}, - {targetBytes: 1, allowEmpty: true, avoidExcess: true, expectResume: true, expectReason: roachpb.RESUME_BYTE_LIMIT, expectNextBytes: 11}, - {targetBytes: 11, allowEmpty: true, avoidExcess: true, expectResume: false}, - {targetBytes: 12, allowEmpty: true, avoidExcess: true, expectResume: false}, {maxKeys: -1, targetBytes: -1, expectResume: true, expectReason: roachpb.RESUME_KEY_LIMIT, expectNextBytes: 0}, {maxKeys: 10, targetBytes: 100, expectResume: false}, } for _, tc := range testCases { - name := fmt.Sprintf("maxKeys=%d targetBytes=%d allowEmpty=%t avoidExcess=%t", - tc.maxKeys, tc.targetBytes, tc.allowEmpty, tc.avoidExcess) + name := fmt.Sprintf("maxKeys=%d targetBytes=%d allowEmpty=%t", + tc.maxKeys, tc.targetBytes, tc.allowEmpty) t.Run(name, func(t *testing.T) { - version := clusterversion.TestingBinaryVersion - if !tc.avoidExcess { - version = clusterversion.ByKey(clusterversion.TargetBytesAvoidExcess - 1) - } - settings := cluster.MakeTestingClusterSettingsWithVersions(version, clusterversion.TestingBinaryMinSupportedVersion, true) + settings := cluster.MakeTestingClusterSettings() resp := roachpb.GetResponse{} _, err := Get(ctx, db, CommandArgs{ diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index 30b3a08746ab..cb83a622c4f5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -14,7 +14,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -40,22 +39,19 @@ func ReverseScan( var scanRes storage.MVCCScanResult var err error - avoidExcess := cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, - clusterversion.TargetBytesAvoidExcess) opts := storage.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, - Txn: h.Txn, - MaxKeys: h.MaxSpanRequestKeys, - MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV), - TargetBytes: h.TargetBytes, - TargetBytesAvoidExcess: h.AllowEmpty || avoidExcess, // AllowEmpty takes precedence - AllowEmpty: h.AllowEmpty, - WholeRowsOfSize: h.WholeRowsOfSize, - FailOnMoreRecent: args.KeyLocking != lock.None, - Reverse: true, - MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), - LockTable: cArgs.Concurrency, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, + Txn: h.Txn, + MaxKeys: h.MaxSpanRequestKeys, + MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV), + TargetBytes: h.TargetBytes, + AllowEmpty: h.AllowEmpty, + WholeRowsOfSize: h.WholeRowsOfSize, + FailOnMoreRecent: args.KeyLocking != lock.None, + Reverse: true, + MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), + LockTable: cArgs.Concurrency, } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index 23e95d5235bb..ffac770c5232 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -14,7 +14,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -40,23 +39,20 @@ func Scan( var scanRes storage.MVCCScanResult var err error - avoidExcess := cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, - clusterversion.TargetBytesAvoidExcess) opts := storage.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, - Txn: h.Txn, - Uncertainty: cArgs.Uncertainty, - MaxKeys: h.MaxSpanRequestKeys, - MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV), - TargetBytes: h.TargetBytes, - TargetBytesAvoidExcess: h.AllowEmpty || avoidExcess, // AllowEmpty takes precedence - AllowEmpty: h.AllowEmpty, - WholeRowsOfSize: h.WholeRowsOfSize, - FailOnMoreRecent: args.KeyLocking != lock.None, - Reverse: false, - MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), - LockTable: cArgs.Concurrency, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, + Txn: h.Txn, + Uncertainty: cArgs.Uncertainty, + MaxKeys: h.MaxSpanRequestKeys, + MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV), + TargetBytes: h.TargetBytes, + AllowEmpty: h.AllowEmpty, + WholeRowsOfSize: h.WholeRowsOfSize, + FailOnMoreRecent: args.KeyLocking != lock.None, + Reverse: false, + MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(), + LockTable: cArgs.Concurrency, } switch args.ScanFormat { diff --git a/pkg/kv/kvserver/batcheval/cmd_scan_test.go b/pkg/kv/kvserver/batcheval/cmd_scan_test.go index 640b372d3bfb..0031d4aed50a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan_test.go @@ -15,7 +15,6 @@ import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -41,50 +40,40 @@ func TestScanReverseScanTargetBytes(t *testing.T) { tbNeg = -1 // hard limit, should return no kv pairs tbNone = 0 // no limit, i.e. should return all kv pairs tbOne = 1 // one byte = return first key only - tbMid = 50 // between first and second key, don't return second if avoidExcess + tbMid = 50 // between first and second key, don't return second tbLots = 100000 // de facto ditto tbNone ) testutils.RunTrueAndFalse(t, "reverse", func(t *testing.T, reverse bool) { - testutils.RunTrueAndFalse(t, "avoidExcess", func(t *testing.T, avoidExcess bool) { - testutils.RunTrueAndFalse(t, "allowEmpty", func(t *testing.T, allowEmpty bool) { - testutils.RunTrueAndFalse(t, "requireNextBytes", func(t *testing.T, requireNextBytes bool) { - for _, tb := range []int64{tbNeg, tbNone, tbOne, tbMid, tbLots} { - t.Run(fmt.Sprintf("targetBytes=%d", tb), func(t *testing.T) { - // allowEmpty takes precedence over avoidExcess at the RPC - // level, since callers have no control over avoidExcess. - expN := 2 - if tb == tbNeg { + testutils.RunTrueAndFalse(t, "allowEmpty", func(t *testing.T, allowEmpty bool) { + testutils.RunTrueAndFalse(t, "requireNextBytes", func(t *testing.T, requireNextBytes bool) { + for _, tb := range []int64{tbNeg, tbNone, tbOne, tbMid, tbLots} { + t.Run(fmt.Sprintf("targetBytes=%d", tb), func(t *testing.T) { + expN := 2 + if tb == tbNeg { + expN = 0 + } else if tb == tbOne { + if allowEmpty { expN = 0 - } else if tb == tbOne { - if allowEmpty { - expN = 0 - } else { - expN = 1 - } - } else if tb == tbMid && (allowEmpty || avoidExcess) { + } else { expN = 1 } - for _, sf := range []roachpb.ScanFormat{roachpb.KEY_VALUES, roachpb.BATCH_RESPONSE} { - t.Run(fmt.Sprintf("format=%s", sf), func(t *testing.T) { - testScanReverseScanInner(t, tb, sf, reverse, avoidExcess, allowEmpty, expN) - }) - } - }) - } - }) + } else if tb == tbMid { + expN = 1 + } + for _, sf := range []roachpb.ScanFormat{roachpb.KEY_VALUES, roachpb.BATCH_RESPONSE} { + t.Run(fmt.Sprintf("format=%s", sf), func(t *testing.T) { + testScanReverseScanInner(t, tb, sf, reverse, allowEmpty, expN) + }) + } + }) + } }) }) }) } func testScanReverseScanInner( - t *testing.T, - tb int64, - sf roachpb.ScanFormat, - reverse bool, - avoidExcess bool, - allowEmpty bool, - expN int, + t *testing.T, tb int64, sf roachpb.ScanFormat, reverse bool, allowEmpty bool, expN int, ) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -113,11 +102,7 @@ func testScanReverseScanInner( } req.SetHeader(roachpb.RequestHeader{Key: k1, EndKey: roachpb.KeyMax}) - version := clusterversion.TestingBinaryVersion - if !avoidExcess { - version = clusterversion.ByKey(clusterversion.TargetBytesAvoidExcess - 1) - } - settings := cluster.MakeTestingClusterSettingsWithVersions(version, clusterversion.TestingBinaryMinSupportedVersion, true) + settings := cluster.MakeTestingClusterSettings() cArgs := CommandArgs{ Args: req, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 1083edb7f0eb..90b8fcc81c75 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2435,10 +2435,8 @@ message Header { int64 max_span_request_keys = 8; // If set to a non-zero value, sets a target (in bytes) for how large the // response may grow. This is only supported for (forward and reverse) scans - // and limits the number of rows scanned (and returned). For cluster versions - // 21.2 and below, the target will be overshot. For 22.1 and above, the - // target will only be overshot when the first result is larger than the - // target (see TargetBytesAvoidExcess cluster version), unless + // and limits the number of rows scanned (and returned). The target will only + // be overshot when the first result is larger than the target, unless // target_bytes_allow_empty is set. A suitable resume span will be returned. // // The semantics around overlapping requests, unordered requests, and @@ -2446,9 +2444,7 @@ message Header { // option as well. int64 target_bytes = 15; // If true, allow returning an empty result when the first result exceeds a - // limit (e.g. TargetBytes). Only effective on 22.1 clusters with - // TargetBytesAvoidExcess version gate enabled. Only supported by Get, Scan, - // and ReverseScan. + // limit (e.g. TargetBytes). Only supported by Get, Scan, and ReverseScan. bool allow_empty = 23; // If positive, Scan and ReverseScan requests with limits (MaxSpanRequestKeys // or TargetBytes) will not return results with partial SQL rows at the end diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index d5ef20bc477c..0e905b8caf61 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -126,20 +126,18 @@ func generateMVCCScan( } maxKeys := int64(m.floatGenerator.parse(args[3]) * 32) targetBytes := int64(m.floatGenerator.parse(args[4]) * (1 << 20)) - targetBytesAvoidExcess := m.boolGenerator.parse(args[5]) - allowEmpty := m.boolGenerator.parse(args[6]) + allowEmpty := m.boolGenerator.parse(args[5]) return &mvccScanOp{ - m: m, - key: key.Key, - endKey: endKey.Key, - ts: ts, - txn: txn, - inconsistent: inconsistent, - reverse: reverse, - maxKeys: maxKeys, - targetBytes: targetBytes, - targetBytesAvoidExcess: targetBytesAvoidExcess, - allowEmpty: allowEmpty, + m: m, + key: key.Key, + endKey: endKey.Key, + ts: ts, + txn: txn, + inconsistent: inconsistent, + reverse: reverse, + maxKeys: maxKeys, + targetBytes: targetBytes, + allowEmpty: allowEmpty, } } @@ -408,17 +406,16 @@ func (m mvccFindSplitKeyOp) run(ctx context.Context) string { } type mvccScanOp struct { - m *metaTestRunner - key roachpb.Key - endKey roachpb.Key - ts hlc.Timestamp - txn txnID - inconsistent bool - reverse bool - maxKeys int64 - targetBytes int64 - targetBytesAvoidExcess bool - allowEmpty bool + m *metaTestRunner + key roachpb.Key + endKey roachpb.Key + ts hlc.Timestamp + txn txnID + inconsistent bool + reverse bool + maxKeys int64 + targetBytes int64 + allowEmpty bool } func (m mvccScanOp) run(ctx context.Context) string { @@ -433,14 +430,13 @@ func (m mvccScanOp) run(ctx context.Context) string { // we will try MVCCScanning on batches and produce diffs between runs on // different engines that don't point to an actual issue. result, err := storage.MVCCScan(ctx, m.m.engine, m.key, m.endKey, m.ts, storage.MVCCScanOptions{ - Inconsistent: m.inconsistent, - Tombstones: true, - Reverse: m.reverse, - Txn: txn, - MaxKeys: m.maxKeys, - TargetBytes: m.targetBytes, - TargetBytesAvoidExcess: m.targetBytesAvoidExcess, - AllowEmpty: m.allowEmpty, + Inconsistent: m.inconsistent, + Tombstones: true, + Reverse: m.reverse, + Txn: txn, + MaxKeys: m.maxKeys, + TargetBytes: m.targetBytes, + AllowEmpty: m.allowEmpty, }) if err != nil { return fmt.Sprintf("error: %s", err) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 1d67cd2f062a..da039f6fb98e 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -2832,24 +2832,23 @@ func mvccScanToBytes( defer mvccScanner.release() *mvccScanner = pebbleMVCCScanner{ - parent: iter, - memAccount: opts.MemoryAccount, - lockTable: opts.LockTable, - reverse: opts.Reverse, - start: key, - end: endKey, - ts: timestamp, - maxKeys: opts.MaxKeys, - targetBytes: opts.TargetBytes, - targetBytesAvoidExcess: opts.TargetBytesAvoidExcess, - allowEmpty: opts.AllowEmpty, - wholeRows: opts.WholeRowsOfSize > 1, // single-KV rows don't need processing - maxIntents: opts.MaxIntents, - inconsistent: opts.Inconsistent, - skipLocked: opts.SkipLocked, - tombstones: opts.Tombstones, - failOnMoreRecent: opts.FailOnMoreRecent, - keyBuf: mvccScanner.keyBuf, + parent: iter, + memAccount: opts.MemoryAccount, + lockTable: opts.LockTable, + reverse: opts.Reverse, + start: key, + end: endKey, + ts: timestamp, + maxKeys: opts.MaxKeys, + targetBytes: opts.TargetBytes, + allowEmpty: opts.AllowEmpty, + wholeRows: opts.WholeRowsOfSize > 1, // single-KV rows don't need processing + maxIntents: opts.MaxIntents, + inconsistent: opts.Inconsistent, + skipLocked: opts.SkipLocked, + tombstones: opts.Tombstones, + failOnMoreRecent: opts.FailOnMoreRecent, + keyBuf: mvccScanner.keyBuf, } var trackLastOffsets int @@ -2971,14 +2970,8 @@ type MVCCScanOptions struct { // // The zero value indicates no limit. TargetBytes int64 - // TargetBytesAvoidExcess will prevent TargetBytes from being exceeded - // unless only a single key/value pair is returned. - // - // TODO(erikgrinaker): This option exists for backwards compatibility with - // 21.2 RPC clients, in 22.2 it should always be enabled. - TargetBytesAvoidExcess bool // AllowEmpty will return an empty result if the first kv pair exceeds the - // TargetBytes limit and TargetBytesAvoidExcess is set. + // TargetBytes limit. AllowEmpty bool // WholeRowsOfSize will prevent returning partial rows when limits (MaxKeys or // TargetBytes) are set. The value indicates the max number of keys per row. diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 15e52ec37beb..17b04a8e8f02 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -79,7 +79,7 @@ var sstIterVerify = util.ConstantWithMetamorphicTestBool("mvcc-histories-sst-ite // put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] // put_rangekey ts=[,] [localTs=[,]] k= end= // get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [skipLocked] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] -// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [skipLocked] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [allowEmpty] // export [k=] [end=] [ts=[,]] [kTs=[,]] [startTs=[,]] [maxIntents=] [allRevisions] [targetSize=] [maxSize=] [stopMidKey] // // iter_new [k=] [end=] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [pointSynthesis [emitOnSeekGE]] [maskBelow=[,]] @@ -1242,9 +1242,6 @@ func cmdScan(e *evalCtx) error { e.scanArg(key, &tb) opts.TargetBytes = int64(tb) } - if e.hasArg("avoidExcess") { - opts.TargetBytesAvoidExcess = true - } if e.hasArg("allowEmpty") { opts.AllowEmpty = true } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index feee2f6a45dc..577d52ec06fc 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -311,13 +311,7 @@ type pebbleMVCCScanner struct { // Stop adding keys once p.result.bytes matches or exceeds this threshold, // if nonzero. targetBytes int64 - // If true, don't exceed targetBytes except for the first kv pair. - // - // TODO(erikgrinaker): This option exists for backwards compatibility with - // 21.2 RPC clients, in 22.1 it should always be enabled. - targetBytesAvoidExcess bool - // If true, return an empty result if the first result exceeds targetBytes - // and targetBytesAvoidExcess is true. + // If true, return an empty result if the first result exceeds targetBytes. allowEmpty bool // If set, don't return partial SQL rows (spanning multiple KV pairs) when // hitting a limit. Partial rows at the end of the result will be trimmed. If @@ -972,8 +966,7 @@ func (p *pebbleMVCCScanner) addAndAdvance( } // Check if adding the key would exceed a limit. - if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess && - p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) { + if p.targetBytes > 0 && p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes { p.resumeReason = roachpb.RESUME_BYTE_LIMIT p.resumeNextBytes = int64(p.results.sizeOf(len(rawKey), len(rawValue))) diff --git a/pkg/storage/testdata/mvcc_histories/target_bytes b/pkg/storage/testdata/mvcc_histories/target_bytes index 705c95b8b192..0b42dfcb7cd8 100644 --- a/pkg/storage/testdata/mvcc_histories/target_bytes +++ b/pkg/storage/testdata/mvcc_histories/target_bytes @@ -46,19 +46,11 @@ data: /Table/1/1/"row3"/1/1/123.000000000,45 -> /BYTES/r3b run ok with ts=300,0 k=a end=z scan - scan avoidExcess scan reverse=true - scan reverse=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "a" -> /BYTES/abcdef @123.000000000,45 @@ -66,19 +58,11 @@ scan: "a" -> /BYTES/abcdef @123.000000000,45 run ok with ts=300,0 k=a end=z targetbytes=0 scan - scan avoidExcess scan reverse=true - scan reverse=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "a" -> /BYTES/abcdef @123.000000000,45 @@ -86,22 +70,12 @@ scan: "a" -> /BYTES/abcdef @123.000000000,45 run ok with ts=300,0 k=a end=z targetbytes=10000000 scan - scan avoidExcess scan reverse=true - scan reverse=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: 108 bytes (target 10000000) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: 108 bytes (target 10000000) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: 108 bytes (target 10000000) scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "a" -> /BYTES/abcdef @123.000000000,45 @@ -111,13 +85,9 @@ scan: 108 bytes (target 10000000) run ok with ts=300,0 k=a end=z targetbytes=-1 scan - scan avoidExcess scan allowEmpty - scan avoidExcess allowEmpty scan reverse=true - scan reverse=true avoidExcess scan reverse=true allowEmpty - scan reverse=true avoidExcess allowEmpty ---- scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 scan: "a"-"z" -> @@ -127,14 +97,6 @@ scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 scan: "a"-"z" -> scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 scan: "a"-"z" -> -scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 -scan: "a"-"z" -> -scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 -scan: "a"-"z" -> -scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 -scan: "a"-"z" -> -scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=0 -scan: "a"-"z" -> run ok # Target size one byte returns one result (overshooting instead of returning @@ -153,18 +115,10 @@ run ok scan k=a end=z ts=300,0 targetbytes=1 scan k=a end=z ts=300,0 targetbytes=1 allowEmpty -scan k=a end=z ts=300,0 targetbytes=1 avoidExcess -scan k=a end=z ts=300,0 targetbytes=1 avoidExcess allowEmpty ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 1) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 1) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 1) scan: resume span ["a","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 0 bytes (target 1) scan: "a"-"z" -> @@ -172,8 +126,6 @@ scan: "a"-"z" -> run ok scan k=a end=z ts=300,0 targetbytes=34 scan k=a end=z ts=300,0 targetbytes=34 allowEmpty -scan k=a end=z ts=300,0 targetbytes=34 avoidExcess -scan k=a end=z ts=300,0 targetbytes=34 avoidExcess allowEmpty ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 @@ -181,28 +133,12 @@ scan: 34 bytes (target 34) scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 34) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 34) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 scan k=a end=z ts=300,0 targetbytes=35 allowEmpty -scan k=a end=z ts=300,0 targetbytes=35 avoidExcess -scan k=a end=z ts=300,0 targetbytes=35 avoidExcess allowEmpty ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 35) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 35) -scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 35) scan: "a" -> /BYTES/abcdef @123.000000000,45 @@ -211,59 +147,36 @@ scan: 34 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=74 -scan k=a end=z ts=300,0 targetbytes=74 avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 74 bytes (target 74) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 74) run ok scan k=a end=z ts=300,0 targetbytes=75 -scan k=a end=z ts=300,0 targetbytes=75 avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: 108 bytes (target 75) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 74 bytes (target 75) run ok scan k=a end=z ts=300,0 targetbytes=108 -scan k=a end=z ts=300,0 targetbytes=108 avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: 108 bytes (target 108) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: 108 bytes (target 108) # Works just the same when not starting on an existing key. run ok scan k=b end=z ts=300 targetbytes=1 scan k=b end=z ts=300 targetbytes=1 allowEmpty -scan k=b end=z ts=300 targetbytes=1 avoidExcess -scan k=b end=z ts=300 targetbytes=1 avoidExcess allowEmpty ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 40 bytes (target 1) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 40 bytes (target 1) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 40 bytes (target 1) scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 0 bytes (target 1) scan: "b"-"z" -> @@ -271,16 +184,8 @@ scan: "b"-"z" -> run ok scan k=b end=z ts=300,0 targetbytes=41 scan k=b end=z ts=300,0 targetbytes=41 allowEmpty -scan k=b end=z ts=300,0 targetbytes=41 avoidExcess -scan k=b end=z ts=300,0 targetbytes=41 avoidExcess allowEmpty ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: 74 bytes (target 41) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: 74 bytes (target 41) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 40 bytes (target 41) scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 @@ -292,18 +197,10 @@ scan: 40 bytes (target 41) run ok scan k=a end=z ts=300,0 targetbytes=1 reverse=true scan k=a end=z ts=300,0 targetbytes=1 reverse=true allowEmpty -scan k=a end=z ts=300,0 targetbytes=1 reverse=true avoidExcess -scan k=a end=z ts=300,0 targetbytes=1 reverse=true avoidExcess allowEmpty ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 1) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 1) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 1) scan: resume span ["a","e\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 0 bytes (target 1) scan: "a"-"z" -> @@ -311,8 +208,6 @@ scan: "a"-"z" -> run ok scan k=a end=z ts=300,0 targetbytes=34 reverse=true scan k=a end=z ts=300,0 targetbytes=34 reverse=true allowEmpty -scan k=a end=z ts=300,0 targetbytes=34 reverse=true avoidExcess -scan k=a end=z ts=300,0 targetbytes=34 reverse=true avoidExcess allowEmpty ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 @@ -320,28 +215,12 @@ scan: 34 bytes (target 34) scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 34) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 34) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 -scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 reverse=true scan k=a end=z ts=300,0 targetbytes=35 reverse=true allowEmpty -scan k=a end=z ts=300,0 targetbytes=35 reverse=true avoidExcess -scan k=a end=z ts=300,0 targetbytes=35 reverse=true avoidExcess allowEmpty ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 35) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 35) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") RESUME_BYTE_LIMIT nextBytes=40 scan: 34 bytes (target 35) scan: "e" -> /BYTES/mnopqr @123.000000000,45 @@ -350,42 +229,27 @@ scan: 34 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=74 reverse=true -scan k=a end=z ts=300,0 targetbytes=74 reverse=true avoidExcess ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 74 bytes (target 74) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 74 bytes (target 74) run ok scan k=a end=z ts=300,0 targetbytes=75 reverse=true -scan k=a end=z ts=300,0 targetbytes=75 reverse=true avoidExcess ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: 108 bytes (target 75) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 74 bytes (target 75) run ok scan k=a end=z ts=300,0 targetbytes=108 reverse=true -scan k=a end=z ts=300,0 targetbytes=108 reverse=true avoidExcess ---- scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: 108 bytes (target 108) -scan: "e" -> /BYTES/mnopqr @123.000000000,45 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: 108 bytes (target 108) run ok with k=a end=z ts=300,0 targetbytes=108 reverse=true @@ -408,102 +272,62 @@ scan: 108 bytes (target 108) run ok scan k=aa end=z ts=300,0 targetbytes=1 tombstones=true scan k=aa end=z ts=300,0 targetbytes=1 tombstones=true allowEmpty -scan k=aa end=z ts=300,0 targetbytes=1 tombstones=true avoidExcess -scan k=aa end=z ts=300,0 targetbytes=1 tombstones=true avoidExcess allowEmpty ---- scan: "aa" -> / @250.000000000,1 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 24 bytes (target 1) -scan: "aa" -> / @250.000000000,1 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 24 bytes (target 1) -scan: "aa" -> / @250.000000000,1 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 24 bytes (target 1) scan: resume span ["aa","z") RESUME_BYTE_LIMIT nextBytes=24 scan: 0 bytes (target 1) scan: "aa"-"z" -> run ok scan k=a end=z ts=300,0 targetbytes=34 tombstones=true -scan k=a end=z ts=300,0 targetbytes=34 tombstones=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["aa","z") RESUME_BYTE_LIMIT nextBytes=24 scan: 34 bytes (target 34) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: resume span ["aa","z") RESUME_BYTE_LIMIT nextBytes=24 -scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 tombstones=true -scan k=a end=z ts=300,0 targetbytes=35 tombstones=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 58 bytes (target 35) -scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["aa","z") RESUME_BYTE_LIMIT nextBytes=24 scan: 34 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=58 tombstones=true -scan k=a end=z ts=300,0 targetbytes=58 tombstones=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "aa" -> / @250.000000000,1 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 58 bytes (target 58) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 -scan: 58 bytes (target 58) run ok scan k=a end=z ts=300,0 targetbytes=59 tombstones=true -scan k=a end=z ts=300,0 targetbytes=59 tombstones=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "aa" -> / @250.000000000,1 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 98 bytes (target 59) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "aa" -> / @250.000000000,1 scan: resume span ["c","z") RESUME_BYTE_LIMIT nextBytes=40 scan: 58 bytes (target 59) run ok scan k=a end=z ts=300,0 targetbytes=98 tombstones=true -scan k=a end=z ts=300,0 targetbytes=98 tombstones=true avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "aa" -> / @250.000000000,1 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 98 bytes (target 98) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 98 bytes (target 98) run ok with k=a end=z ts=300,0 targetbytes=98 tombstones=true scan - scan avoidExcess ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "aa" -> / @250.000000000,1 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 scan: 98 bytes (target 98) -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["e","z") RESUME_BYTE_LIMIT nextBytes=34 -scan: 98 bytes (target 98) # ... and similarly in reverse. @@ -511,87 +335,55 @@ scan: 98 bytes (target 98) run ok scan k=a end=b ts=300,0 targetbytes=1 tombstones=true reverse=true scan k=a end=b ts=300,0 targetbytes=1 tombstones=true reverse=true allowEmpty -scan k=a end=b ts=300,0 targetbytes=1 tombstones=true reverse=true avoidExcess -scan k=a end=b ts=300,0 targetbytes=1 tombstones=true reverse=true avoidExcess allowEmpty ---- scan: "aa" -> / @250.000000000,1 scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 24 bytes (target 1) -scan: "aa" -> / @250.000000000,1 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 24 bytes (target 1) -scan: "aa" -> / @250.000000000,1 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 24 bytes (target 1) scan: resume span ["a","aa\x00") RESUME_BYTE_LIMIT nextBytes=24 scan: 0 bytes (target 1) scan: "a"-"b" -> run ok scan k=a end=d ts=300,0 targetbytes=40 reverse=true tombstones=true -scan k=a end=d ts=300,0 targetbytes=40 reverse=true tombstones=true avoidExcess ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","aa\x00") RESUME_BYTE_LIMIT nextBytes=24 scan: 40 bytes (target 40) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: resume span ["a","aa\x00") RESUME_BYTE_LIMIT nextBytes=24 -scan: 40 bytes (target 40) run ok scan k=a end=d ts=300,0 targetbytes=41 reverse=true tombstones=true -scan k=a end=d ts=300,0 targetbytes=41 reverse=true tombstones=true avoidExcess ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 64 bytes (target 41) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","aa\x00") RESUME_BYTE_LIMIT nextBytes=24 scan: 40 bytes (target 41) run ok scan k=a end=d ts=300,0 targetbytes=64 reverse=true tombstones=true -scan k=a end=d ts=300,0 targetbytes=64 reverse=true tombstones=true avoidExcess ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "aa" -> / @250.000000000,1 scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 64 bytes (target 64) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 -scan: 64 bytes (target 64) run ok scan k=a end=d ts=300,0 targetbytes=65 reverse=true tombstones=true -scan k=a end=d ts=300,0 targetbytes=65 reverse=true tombstones=true avoidExcess ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "aa" -> / @250.000000000,1 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: 98 bytes (target 65) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "aa" -> / @250.000000000,1 scan: resume span ["a","a\x00") RESUME_BYTE_LIMIT nextBytes=34 scan: 64 bytes (target 65) run ok scan k=a end=d ts=300,0 targetbytes=98 reverse=true tombstones=true -scan k=a end=d ts=300,0 targetbytes=98 reverse=true tombstones=true avoidExcess ---- scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: "aa" -> / @250.000000000,1 scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: 98 bytes (target 98) -scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 -scan: "aa" -> / @250.000000000,1 -scan: "a" -> /BYTES/abcdef @123.000000000,45 -scan: 98 bytes (target 98) # Whole SQL rows. run ok -with ts=300,0 avoidExcess +with ts=300,0 scan k=/ end=/z targetbytes=1 scan k=/ end=/z targetbytes=1 wholeRows scan k=/ end=/z targetbytes=1 wholeRows allowEmpty @@ -609,7 +401,7 @@ scan: 0 bytes (target 1) scan: /Table/1/1/""/0-/Table/1/1/"z"/0 -> run ok -with ts=300,0 avoidExcess allowEmpty wholeRows +with ts=300,0 allowEmpty wholeRows scan k=/ end=/z targetbytes=121 scan k=/ end=/z targetbytes=122 scan k=/ end=/z targetbytes=123 @@ -629,7 +421,7 @@ scan: resume span [/Table/1/1/"row2"/0,/Table/1/1/"z"/0) RESUME_BYTE_LIMIT nextB scan: 122 bytes (target 123) run ok -with ts=300,0 avoidExcess allowEmpty wholeRows +with ts=300,0 allowEmpty wholeRows scan k=/ end=/z targetbytes=161 scan k=/ end=/z targetbytes=162 scan k=/ end=/z targetbytes=163 @@ -653,7 +445,7 @@ scan: resume span [/Table/1/1/"row3"/0,/Table/1/1/"z"/0) RESUME_BYTE_LIMIT nextB scan: 162 bytes (target 163) run ok -with ts=300,0 avoidExcess allowEmpty wholeRows +with ts=300,0 allowEmpty wholeRows scan k=/ end=/z targetbytes=242 scan k=/ end=/z targetbytes=243 ---- @@ -673,7 +465,7 @@ scan: 243 bytes (target 243) # Whole SQL rows in reverse. run ok -with ts=300,0 avoidExcess +with ts=300,0 scan k=/ end=/z targetbytes=1 reverse scan k=/ end=/z targetbytes=1 reverse wholeRows scan k=/ end=/z targetbytes=1 reverse wholeRows allowEmpty @@ -690,7 +482,7 @@ scan: 0 bytes (target 1) scan: /Table/1/1/""/0-/Table/1/1/"z"/0 -> run ok -with ts=300,0 reverse avoidExcess allowEmpty wholeRows +with ts=300,0 reverse allowEmpty wholeRows scan k=/ end=/z targetbytes=80 scan k=/ end=/z targetbytes=81 scan k=/ end=/z targetbytes=82 @@ -708,7 +500,7 @@ scan: resume span [/Table/1/1/""/0,/Table/1/1/"row2"/0/NULL) RESUME_BYTE_LIMIT n scan: 81 bytes (target 82) run ok -with ts=300,0 reverse avoidExcess allowEmpty wholeRows +with ts=300,0 reverse allowEmpty wholeRows scan k=/ end=/z targetbytes=120 scan k=/ end=/z targetbytes=121 scan k=/ end=/z targetbytes=122 @@ -729,7 +521,7 @@ scan: resume span [/Table/1/1/""/0,/Table/1/1/"row1"/4/1/NULL) RESUME_BYTE_LIMIT scan: 121 bytes (target 122) run ok -with ts=300,0 reverse avoidExcess allowEmpty wholeRows +with ts=300,0 reverse allowEmpty wholeRows scan k=/ end=/z targetbytes=242 scan k=/ end=/z targetbytes=243 ---- @@ -749,7 +541,7 @@ scan: 243 bytes (target 243) # WholeRows on non-SQL row data. run ok with ts=300,0 - scan k=a end=/z targetbytes=110 avoidExcess wholeRows + scan k=a end=/z targetbytes=110 wholeRows ---- scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 @@ -780,22 +572,12 @@ with t=A ts=11,0 targetbytes=32 put k=n v=c txn_step seq=20 scan k=k end=o - scan k=k end=o avoidExcess scan k=k end=o reverse=true - scan k=k end=o reverse=true avoidExcess ---- scan: "k" -> /BYTES/b @11.000000000,0 -scan: "l" -> /BYTES/b @11.000000000,0 -scan: resume span ["m","o") RESUME_BYTE_LIMIT nextBytes=25 -scan: 50 bytes (target 32) -scan: "k" -> /BYTES/b @11.000000000,0 scan: resume span ["l","o") RESUME_BYTE_LIMIT nextBytes=25 scan: 25 bytes (target 32) scan: "n" -> /BYTES/b @11.000000000,0 -scan: "m" -> /BYTES/b @11.000000000,0 -scan: resume span ["k","l\x00") RESUME_BYTE_LIMIT nextBytes=25 -scan: 50 bytes (target 32) -scan: "n" -> /BYTES/b @11.000000000,0 scan: resume span ["k","m\x00") RESUME_BYTE_LIMIT nextBytes=25 scan: 25 bytes (target 32) >> at end: From 935f12aadda5d0941ddffe15cfb0914332bfe522 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 16:41:37 -0700 Subject: [PATCH 06/16] util/tracing: remove TraceIDDoesntImplyStructuredRecording version gate Release note: None --- pkg/clusterversion/cockroach_versions.go | 7 -- pkg/clusterversion/key_string.go | 125 +++++++++++------------ pkg/server/config.go | 6 -- pkg/upgrade/upgrades/builtins_test.go | 12 +-- pkg/util/tracing/tracer.go | 43 -------- 5 files changed, 68 insertions(+), 125 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 69382e9d18e0..3250b375d14a 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // TraceIDDoesntImplyStructuredRecording changes the contract about the kind - // of span that RPCs get on the server depending on the tracing context. - TraceIDDoesntImplyStructuredRecording // AlterSystemTableStatisticsAddAvgSizeCol adds the column avgSize to the // table system.table_statistics that contains a new statistic. AlterSystemTableStatisticsAddAvgSizeCol @@ -415,10 +412,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: TraceIDDoesntImplyStructuredRecording, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 10}, - }, { Key: AlterSystemTableStatisticsAddAvgSizeCol, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 12}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 1572caa60a3f..8c8eab89c51d 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,72 +10,71 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[TraceIDDoesntImplyStructuredRecording-2] - _ = x[AlterSystemTableStatisticsAddAvgSizeCol-3] - _ = x[MVCCAddSSTable-4] - _ = x[InsertPublicSchemaNamespaceEntryOnRestore-5] - _ = x[UnsplitRangesInAsyncGCJobs-6] - _ = x[ValidateGrantOption-7] - _ = x[PebbleFormatBlockPropertyCollector-8] - _ = x[ProbeRequest-9] - _ = x[SelectRPCsTakeTracingInfoInband-10] - _ = x[PreSeedTenantSpanConfigs-11] - _ = x[SeedTenantSpanConfigs-12] - _ = x[PublicSchemasWithDescriptors-13] - _ = x[EnsureSpanConfigReconciliation-14] - _ = x[EnsureSpanConfigSubscription-15] - _ = x[EnableSpanConfigStore-16] - _ = x[ScanWholeRows-17] - _ = x[SCRAMAuthentication-18] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-19] - _ = x[AlterSystemProtectedTimestampAddColumn-20] - _ = x[EnableProtectedTimestampsForTenant-21] - _ = x[DeleteCommentsWithDroppedIndexes-22] - _ = x[RemoveIncompatibleDatabasePrivileges-23] - _ = x[AddRaftAppliedIndexTermMigration-24] - _ = x[PostAddRaftAppliedIndexTermMigration-25] - _ = x[DontProposeWriteTimestampForLeaseTransfers-26] - _ = x[EnablePebbleFormatVersionBlockProperties-27] - _ = x[DisableSystemConfigGossipTrigger-28] - _ = x[MVCCIndexBackfiller-29] - _ = x[EnableLeaseHolderRemoval-30] - _ = x[BackupResolutionInJob-31] - _ = x[LooselyCoupledRaftLogTruncation-32] - _ = x[ChangefeedIdleness-33] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-34] - _ = x[EnableDeclarativeSchemaChanger-35] - _ = x[RowLevelTTL-36] - _ = x[PebbleFormatSplitUserKeysMarked-37] - _ = x[IncrementalBackupSubdir-38] - _ = x[DateStyleIntervalStyleCastRewrite-39] - _ = x[EnableNewStoreRebalancer-40] - _ = x[ClusterLocksVirtualTable-41] - _ = x[AutoStatsTableSettings-42] - _ = x[ForecastStats-43] - _ = x[SuperRegions-44] - _ = x[EnableNewChangefeedOptions-45] - _ = x[SpanCountTable-46] - _ = x[PreSeedSpanCountTable-47] - _ = x[SeedSpanCountTable-48] - _ = x[V22_1-49] - _ = x[Start22_2-50] - _ = x[LocalTimestamps-51] - _ = x[EnsurePebbleFormatVersionRangeKeys-52] - _ = x[EnablePebbleFormatVersionRangeKeys-53] - _ = x[TrigramInvertedIndexes-54] - _ = x[RemoveGrantPrivilege-55] - _ = x[MVCCRangeTombstones-56] - _ = x[UpgradeSequenceToBeReferencedByID-57] - _ = x[SampledStmtDiagReqs-58] - _ = x[AddSSTableTombstones-59] - _ = x[SystemPrivilegesTable-60] - _ = x[EnablePredicateProjectionChangefeed-61] - _ = x[AlterSystemSQLInstancesAddLocality-62] + _ = x[AlterSystemTableStatisticsAddAvgSizeCol-2] + _ = x[MVCCAddSSTable-3] + _ = x[InsertPublicSchemaNamespaceEntryOnRestore-4] + _ = x[UnsplitRangesInAsyncGCJobs-5] + _ = x[ValidateGrantOption-6] + _ = x[PebbleFormatBlockPropertyCollector-7] + _ = x[ProbeRequest-8] + _ = x[SelectRPCsTakeTracingInfoInband-9] + _ = x[PreSeedTenantSpanConfigs-10] + _ = x[SeedTenantSpanConfigs-11] + _ = x[PublicSchemasWithDescriptors-12] + _ = x[EnsureSpanConfigReconciliation-13] + _ = x[EnsureSpanConfigSubscription-14] + _ = x[EnableSpanConfigStore-15] + _ = x[ScanWholeRows-16] + _ = x[SCRAMAuthentication-17] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-18] + _ = x[AlterSystemProtectedTimestampAddColumn-19] + _ = x[EnableProtectedTimestampsForTenant-20] + _ = x[DeleteCommentsWithDroppedIndexes-21] + _ = x[RemoveIncompatibleDatabasePrivileges-22] + _ = x[AddRaftAppliedIndexTermMigration-23] + _ = x[PostAddRaftAppliedIndexTermMigration-24] + _ = x[DontProposeWriteTimestampForLeaseTransfers-25] + _ = x[EnablePebbleFormatVersionBlockProperties-26] + _ = x[DisableSystemConfigGossipTrigger-27] + _ = x[MVCCIndexBackfiller-28] + _ = x[EnableLeaseHolderRemoval-29] + _ = x[BackupResolutionInJob-30] + _ = x[LooselyCoupledRaftLogTruncation-31] + _ = x[ChangefeedIdleness-32] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-33] + _ = x[EnableDeclarativeSchemaChanger-34] + _ = x[RowLevelTTL-35] + _ = x[PebbleFormatSplitUserKeysMarked-36] + _ = x[IncrementalBackupSubdir-37] + _ = x[DateStyleIntervalStyleCastRewrite-38] + _ = x[EnableNewStoreRebalancer-39] + _ = x[ClusterLocksVirtualTable-40] + _ = x[AutoStatsTableSettings-41] + _ = x[ForecastStats-42] + _ = x[SuperRegions-43] + _ = x[EnableNewChangefeedOptions-44] + _ = x[SpanCountTable-45] + _ = x[PreSeedSpanCountTable-46] + _ = x[SeedSpanCountTable-47] + _ = x[V22_1-48] + _ = x[Start22_2-49] + _ = x[LocalTimestamps-50] + _ = x[EnsurePebbleFormatVersionRangeKeys-51] + _ = x[EnablePebbleFormatVersionRangeKeys-52] + _ = x[TrigramInvertedIndexes-53] + _ = x[RemoveGrantPrivilege-54] + _ = x[MVCCRangeTombstones-55] + _ = x[UpgradeSequenceToBeReferencedByID-56] + _ = x[SampledStmtDiagReqs-57] + _ = x[AddSSTableTombstones-58] + _ = x[SystemPrivilegesTable-59] + _ = x[EnablePredicateProjectionChangefeed-60] + _ = x[AlterSystemSQLInstancesAddLocality-61] } -const _Key_name = "V21_2Start22_1TraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1AlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 51, 90, 104, 145, 171, 190, 224, 236, 267, 291, 312, 340, 370, 398, 419, 432, 451, 485, 523, 557, 589, 625, 657, 693, 735, 775, 807, 826, 850, 871, 902, 920, 961, 991, 1002, 1033, 1056, 1089, 1113, 1137, 1159, 1172, 1184, 1210, 1224, 1245, 1263, 1268, 1277, 1292, 1326, 1360, 1382, 1402, 1421, 1454, 1473, 1493, 1514, 1549, 1583} +var _Key_index = [...]uint16{0, 5, 14, 53, 67, 108, 134, 153, 187, 199, 230, 254, 275, 303, 333, 361, 382, 395, 414, 448, 486, 520, 552, 588, 620, 656, 698, 738, 770, 789, 813, 834, 865, 883, 924, 954, 965, 996, 1019, 1052, 1076, 1100, 1122, 1135, 1147, 1173, 1187, 1208, 1226, 1231, 1240, 1255, 1289, 1323, 1345, 1365, 1384, 1417, 1436, 1456, 1477, 1512, 1546} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/server/config.go b/pkg/server/config.go index 291a657df1f7..dfaf0e991667 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -23,7 +23,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -446,11 +445,6 @@ func MakeConfig(ctx context.Context, st *cluster.Settings) Config { sqlCfg := MakeSQLConfig(roachpb.SystemTenantID, tempStorageCfg) tr := tracing.NewTracerWithOpt(ctx, tracing.WithClusterSettings(&st.SV)) - // NB: The OnChange callback will be called on server startup when the version - // is initialized. - st.Version.SetOnChange(func(ctx context.Context, newVersion clusterversion.ClusterVersion) { - tr.SetBackwardsCompatibilityWith211(!newVersion.IsActive(clusterversion.TraceIDDoesntImplyStructuredRecording)) - }) baseCfg := MakeBaseConfig(st, tr) kvCfg := MakeKVConfig(storeSpec) diff --git a/pkg/upgrade/upgrades/builtins_test.go b/pkg/upgrade/upgrades/builtins_test.go index 015400305988..a3bfa3cff216 100644 --- a/pkg/upgrade/upgrades/builtins_test.go +++ b/pkg/upgrade/upgrades/builtins_test.go @@ -32,7 +32,7 @@ func TestIsAtLeastVersionBuiltin(t *testing.T) { Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey(clusterversion.V21_2), + BinaryVersionOverride: clusterversion.ByKey(clusterversion.V22_1), }, }, }, @@ -46,13 +46,13 @@ func TestIsAtLeastVersionBuiltin(t *testing.T) { ) defer tc.Stopper().Stop(ctx) - // Check that the builtin returns false when comparing against 21.2 version - // because we are still on 21.1. - sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('21.2-10')", [][]string{{"false"}}) + // Check that the builtin returns false when comparing against 22.1-2 + // version because we are still on 22.1-0. + sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"false"}}) // Run the upgrade. - sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", clusterversion.ByKey(clusterversion.TraceIDDoesntImplyStructuredRecording).String()) + sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", clusterversion.ByKey(clusterversion.Start22_2).String()) // It should now return true. - sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('21.2-10')", [][]string{{"true"}}) + sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"true"}}) } diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index ea300787866f..f174a2bf263a 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -82,15 +82,6 @@ const ( fieldNameOtelTraceID = prefixTracerState + "otel_traceid" fieldNameOtelSpanID = prefixTracerState + "otel_spanid" - // fieldNameDeprecatedVerboseTracing is the carrier key indicating that the trace - // has verbose recording enabled. It means that a) spans derived from this one - // will not be no-op spans and b) they will start recording. - // - // The key is named the way it is for backwards compatibility reasons. - // TODO(andrei): remove in 22.2, once we no longer need to set this key for - // compatibility with 21.2. - fieldNameDeprecatedVerboseTracing = "crdb-baggage-sb" - spanKindTagKey = "span.kind" ) @@ -277,12 +268,6 @@ type Tracer struct { noopSpan *Span sterileNoopSpan *Span - // backardsCompatibilityWith211, if set, makes the Tracer - // work with 21.1 remote nodes. - // - // Accessed atomically. - backwardsCompatibilityWith211 int64 - // True if tracing to the debug/requests endpoint. Accessed via t.useNetTrace(). _useNetTrace int32 // updated atomically @@ -1292,23 +1277,9 @@ func (t *Tracer) InjectMetaInto(sm SpanMeta, carrier Carrier) { carrier.Set(fieldNameOtelSpanID, sm.otelCtx.SpanID().String()) } - compatMode := atomic.LoadInt64(&t.backwardsCompatibilityWith211) == 1 - - // For compatibility with 21.1, we don't want to propagate the traceID when - // we're not recording. A 21.1 node interprets a traceID as wanting structured - // recording (or verbose recording if fieldNameDeprecatedVerboseTracing is also - // set). - if compatMode && sm.recordingType == tracingpb.RecordingOff { - return - } - carrier.Set(fieldNameTraceID, strconv.FormatUint(uint64(sm.traceID), 16)) carrier.Set(fieldNameSpanID, strconv.FormatUint(uint64(sm.spanID), 16)) carrier.Set(fieldNameRecordingType, sm.recordingType.ToCarrierValue()) - - if compatMode && sm.recordingType == tracingpb.RecordingVerbose { - carrier.Set(fieldNameDeprecatedVerboseTracing, "1") - } } var noopSpanMeta = SpanMeta{} @@ -1355,11 +1326,6 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (SpanMeta, error) { case fieldNameRecordingType: recordingTypeExplicit = true recordingType = tracingpb.RecordingTypeFromCarrierValue(v) - case fieldNameDeprecatedVerboseTracing: - // Compatibility with 21.2. - if !recordingTypeExplicit { - recordingType = tracingpb.RecordingVerbose - } } return nil } @@ -1474,15 +1440,6 @@ func (t *Tracer) ShouldRecordAsyncSpans() bool { return t.testingRecordAsyncSpans } -// SetBackwardsCompatibilityWith211 toggles the compatibility mode. -func (t *Tracer) SetBackwardsCompatibilityWith211(to bool) { - if to { - atomic.StoreInt64(&t.backwardsCompatibilityWith211, 1) - } else { - atomic.StoreInt64(&t.backwardsCompatibilityWith211, 0) - } -} - // PanicOnUseAfterFinish returns true if the Tracer is configured to crash when // a Span is used after it was previously Finish()ed. Crashing is supposed to be // best-effort as, in the future, reliably detecting use-after-Finish might not From f5a7050ca2641250f72950141d2c3c4a911c067b Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 16:52:44 -0700 Subject: [PATCH 07/16] sql/stats: remove AlterSystemTableStatisticsAddAvgSizeCol version gate Release note: None --- pkg/clusterversion/cockroach_versions.go | 7 - pkg/clusterversion/key_string.go | 123 +++++++------- pkg/sql/alter_table.go | 33 +--- pkg/sql/show_stats.go | 97 +++-------- pkg/sql/stats/BUILD.bazel | 1 - pkg/sql/stats/new_stat.go | 23 --- pkg/sql/stats/stats_cache.go | 55 ++---- pkg/upgrade/upgrades/BUILD.bazel | 2 - .../alter_table_statistics_avg_size.go | 42 ----- .../alter_table_statistics_avg_size_test.go | 159 ------------------ pkg/upgrade/upgrades/upgrades.go | 6 - 11 files changed, 102 insertions(+), 446 deletions(-) delete mode 100644 pkg/upgrade/upgrades/alter_table_statistics_avg_size.go delete mode 100644 pkg/upgrade/upgrades/alter_table_statistics_avg_size_test.go diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 3250b375d14a..e0186ff2354c 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // AlterSystemTableStatisticsAddAvgSizeCol adds the column avgSize to the - // table system.table_statistics that contains a new statistic. - AlterSystemTableStatisticsAddAvgSizeCol // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new // SSTTimestampToRequestTimestamp and DisallowConflicts parameters. MVCCAddSSTable @@ -412,10 +409,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: AlterSystemTableStatisticsAddAvgSizeCol, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 12}, - }, { Key: MVCCAddSSTable, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 16}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 8c8eab89c51d..a14d0badd802 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,71 +10,70 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[AlterSystemTableStatisticsAddAvgSizeCol-2] - _ = x[MVCCAddSSTable-3] - _ = x[InsertPublicSchemaNamespaceEntryOnRestore-4] - _ = x[UnsplitRangesInAsyncGCJobs-5] - _ = x[ValidateGrantOption-6] - _ = x[PebbleFormatBlockPropertyCollector-7] - _ = x[ProbeRequest-8] - _ = x[SelectRPCsTakeTracingInfoInband-9] - _ = x[PreSeedTenantSpanConfigs-10] - _ = x[SeedTenantSpanConfigs-11] - _ = x[PublicSchemasWithDescriptors-12] - _ = x[EnsureSpanConfigReconciliation-13] - _ = x[EnsureSpanConfigSubscription-14] - _ = x[EnableSpanConfigStore-15] - _ = x[ScanWholeRows-16] - _ = x[SCRAMAuthentication-17] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-18] - _ = x[AlterSystemProtectedTimestampAddColumn-19] - _ = x[EnableProtectedTimestampsForTenant-20] - _ = x[DeleteCommentsWithDroppedIndexes-21] - _ = x[RemoveIncompatibleDatabasePrivileges-22] - _ = x[AddRaftAppliedIndexTermMigration-23] - _ = x[PostAddRaftAppliedIndexTermMigration-24] - _ = x[DontProposeWriteTimestampForLeaseTransfers-25] - _ = x[EnablePebbleFormatVersionBlockProperties-26] - _ = x[DisableSystemConfigGossipTrigger-27] - _ = x[MVCCIndexBackfiller-28] - _ = x[EnableLeaseHolderRemoval-29] - _ = x[BackupResolutionInJob-30] - _ = x[LooselyCoupledRaftLogTruncation-31] - _ = x[ChangefeedIdleness-32] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-33] - _ = x[EnableDeclarativeSchemaChanger-34] - _ = x[RowLevelTTL-35] - _ = x[PebbleFormatSplitUserKeysMarked-36] - _ = x[IncrementalBackupSubdir-37] - _ = x[DateStyleIntervalStyleCastRewrite-38] - _ = x[EnableNewStoreRebalancer-39] - _ = x[ClusterLocksVirtualTable-40] - _ = x[AutoStatsTableSettings-41] - _ = x[ForecastStats-42] - _ = x[SuperRegions-43] - _ = x[EnableNewChangefeedOptions-44] - _ = x[SpanCountTable-45] - _ = x[PreSeedSpanCountTable-46] - _ = x[SeedSpanCountTable-47] - _ = x[V22_1-48] - _ = x[Start22_2-49] - _ = x[LocalTimestamps-50] - _ = x[EnsurePebbleFormatVersionRangeKeys-51] - _ = x[EnablePebbleFormatVersionRangeKeys-52] - _ = x[TrigramInvertedIndexes-53] - _ = x[RemoveGrantPrivilege-54] - _ = x[MVCCRangeTombstones-55] - _ = x[UpgradeSequenceToBeReferencedByID-56] - _ = x[SampledStmtDiagReqs-57] - _ = x[AddSSTableTombstones-58] - _ = x[SystemPrivilegesTable-59] - _ = x[EnablePredicateProjectionChangefeed-60] - _ = x[AlterSystemSQLInstancesAddLocality-61] + _ = x[MVCCAddSSTable-2] + _ = x[InsertPublicSchemaNamespaceEntryOnRestore-3] + _ = x[UnsplitRangesInAsyncGCJobs-4] + _ = x[ValidateGrantOption-5] + _ = x[PebbleFormatBlockPropertyCollector-6] + _ = x[ProbeRequest-7] + _ = x[SelectRPCsTakeTracingInfoInband-8] + _ = x[PreSeedTenantSpanConfigs-9] + _ = x[SeedTenantSpanConfigs-10] + _ = x[PublicSchemasWithDescriptors-11] + _ = x[EnsureSpanConfigReconciliation-12] + _ = x[EnsureSpanConfigSubscription-13] + _ = x[EnableSpanConfigStore-14] + _ = x[ScanWholeRows-15] + _ = x[SCRAMAuthentication-16] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-17] + _ = x[AlterSystemProtectedTimestampAddColumn-18] + _ = x[EnableProtectedTimestampsForTenant-19] + _ = x[DeleteCommentsWithDroppedIndexes-20] + _ = x[RemoveIncompatibleDatabasePrivileges-21] + _ = x[AddRaftAppliedIndexTermMigration-22] + _ = x[PostAddRaftAppliedIndexTermMigration-23] + _ = x[DontProposeWriteTimestampForLeaseTransfers-24] + _ = x[EnablePebbleFormatVersionBlockProperties-25] + _ = x[DisableSystemConfigGossipTrigger-26] + _ = x[MVCCIndexBackfiller-27] + _ = x[EnableLeaseHolderRemoval-28] + _ = x[BackupResolutionInJob-29] + _ = x[LooselyCoupledRaftLogTruncation-30] + _ = x[ChangefeedIdleness-31] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-32] + _ = x[EnableDeclarativeSchemaChanger-33] + _ = x[RowLevelTTL-34] + _ = x[PebbleFormatSplitUserKeysMarked-35] + _ = x[IncrementalBackupSubdir-36] + _ = x[DateStyleIntervalStyleCastRewrite-37] + _ = x[EnableNewStoreRebalancer-38] + _ = x[ClusterLocksVirtualTable-39] + _ = x[AutoStatsTableSettings-40] + _ = x[ForecastStats-41] + _ = x[SuperRegions-42] + _ = x[EnableNewChangefeedOptions-43] + _ = x[SpanCountTable-44] + _ = x[PreSeedSpanCountTable-45] + _ = x[SeedSpanCountTable-46] + _ = x[V22_1-47] + _ = x[Start22_2-48] + _ = x[LocalTimestamps-49] + _ = x[EnsurePebbleFormatVersionRangeKeys-50] + _ = x[EnablePebbleFormatVersionRangeKeys-51] + _ = x[TrigramInvertedIndexes-52] + _ = x[RemoveGrantPrivilege-53] + _ = x[MVCCRangeTombstones-54] + _ = x[UpgradeSequenceToBeReferencedByID-55] + _ = x[SampledStmtDiagReqs-56] + _ = x[AddSSTableTombstones-57] + _ = x[SystemPrivilegesTable-58] + _ = x[EnablePredicateProjectionChangefeed-59] + _ = x[AlterSystemSQLInstancesAddLocality-60] } -const _Key_name = "V21_2Start22_1AlterSystemTableStatisticsAddAvgSizeColMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1MVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 53, 67, 108, 134, 153, 187, 199, 230, 254, 275, 303, 333, 361, 382, 395, 414, 448, 486, 520, 552, 588, 620, 656, 698, 738, 770, 789, 813, 834, 865, 883, 924, 954, 965, 996, 1019, 1052, 1076, 1100, 1122, 1135, 1147, 1173, 1187, 1208, 1226, 1231, 1240, 1255, 1289, 1323, 1345, 1365, 1384, 1417, 1436, 1456, 1477, 1512, 1546} +var _Key_index = [...]uint16{0, 5, 14, 28, 69, 95, 114, 148, 160, 191, 215, 236, 264, 294, 322, 343, 356, 375, 409, 447, 481, 513, 549, 581, 617, 659, 699, 731, 750, 774, 795, 826, 844, 885, 915, 926, 957, 980, 1013, 1037, 1061, 1083, 1096, 1108, 1134, 1148, 1169, 1187, 1192, 1201, 1216, 1250, 1284, 1306, 1326, 1345, 1378, 1397, 1417, 1438, 1473, 1507} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 5a443dc1ef3b..c9356bf8922b 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -18,7 +18,6 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -1322,10 +1321,9 @@ func insertJSONStatistic( histogram interface{}, ) error { var ( - ctx = params.ctx - ie = params.ExecCfg().InternalExecutor - txn = params.p.Txn() - settings = params.ExecCfg().Settings + ctx = params.ctx + ie = params.ExecCfg().InternalExecutor + txn = params.p.Txn() ) var name interface{} @@ -1333,31 +1331,6 @@ func insertJSONStatistic( name = s.Name } - if !settings.Version.IsActive(params.ctx, clusterversion.AlterSystemTableStatisticsAddAvgSizeCol) { - _ /* rows */, err := ie.Exec( - ctx, - "insert-stats", - txn, - `INSERT INTO system.table_statistics ( - "tableID", - "name", - "columnIDs", - "createdAt", - "rowCount", - "distinctCount", - "nullCount", - histogram - ) VALUES ($1, $2, $3, $4, $5, $6, $7, $8)`, - tableID, - name, - columnIDs, - s.CreatedAt, - s.RowCount, - s.DistinctCount, - s.NullCount, - histogram) - return err - } _ /* rows */, err := ie.Exec( ctx, "insert-stats", diff --git a/pkg/sql/show_stats.go b/pkg/sql/show_stats.go index 60e73223d5d9..2e4e54fda9b2 100644 --- a/pkg/sql/show_stats.go +++ b/pkg/sql/show_stats.go @@ -13,9 +13,7 @@ package sql import ( "context" encjson "encoding/json" - "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -28,19 +26,7 @@ import ( "github.com/cockroachdb/errors" ) -// TODO(harding): Remove use of showTableStatsColumns in 22.2 when AvgSizeColVer -// is fully integrated. var showTableStatsColumns = colinfo.ResultColumns{ - {Name: "statistics_name", Typ: types.String}, - {Name: "column_names", Typ: types.StringArray}, - {Name: "created", Typ: types.Timestamp}, - {Name: "row_count", Typ: types.Int}, - {Name: "distinct_count", Typ: types.Int}, - {Name: "null_count", Typ: types.Int}, - {Name: "histogram_id", Typ: types.Int}, -} - -var showTableStatsColumnsAvgSizeVer = colinfo.ResultColumns{ {Name: "statistics_name", Typ: types.String}, {Name: "column_names", Typ: types.StringArray}, {Name: "created", Typ: types.Timestamp}, @@ -67,11 +53,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p if err := p.CheckAnyPrivilege(ctx, desc); err != nil { return nil, err } - avgSizeColVerActive := p.ExtendedEvalContext().ExecCfg.Settings.Version.IsActive(ctx, clusterversion.AlterSystemTableStatisticsAddAvgSizeCol) - columns := showTableStatsColumnsAvgSizeVer - if !avgSizeColVerActive { - columns = showTableStatsColumns - } + columns := showTableStatsColumns if n.UsingJSON { columns = showTableStatsJSONColumns } @@ -86,23 +68,18 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p // "handle" which can be used with SHOW HISTOGRAM. // TODO(yuzefovich): refactor the code to use the iterator API // (currently it is not possible due to a panic-catcher below). - var avgSize string - if avgSizeColVerActive { - avgSize = ` - "avgSize",` - } - stmt := fmt.Sprintf(`SELECT "statisticID", - name, - "columnIDs", - "createdAt", - "rowCount", - "distinctCount", - "nullCount", - %s - histogram - FROM system.table_statistics - WHERE "tableID" = $1 - ORDER BY "createdAt"`, avgSize) + stmt := `SELECT "statisticID", + name, + "columnIDs", + "createdAt", + "rowCount", + "distinctCount", + "nullCount", + "avgSize", + histogram + FROM system.table_statistics + WHERE "tableID" = $1 + ORDER BY "createdAt"` rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryBuffered( ctx, "read-table-stats", @@ -127,13 +104,6 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p numCols ) - histIdx := histogramIdx - nCols := numCols - if !avgSizeColVerActive { - histIdx = histogramIdx - 1 - nCols = numCols - 1 - } - // Guard against crashes in the code below (e.g. #56356). defer func() { if r := recover(); r != nil { @@ -173,13 +143,11 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p statsRow.RowCount = (uint64)(*r[rowCountIdx].(*tree.DInt)) statsRow.DistinctCount = (uint64)(*r[distinctCountIdx].(*tree.DInt)) statsRow.NullCount = (uint64)(*r[nullCountIdx].(*tree.DInt)) - if avgSizeColVerActive { - statsRow.AvgSize = (uint64)(*r[avgSizeIdx].(*tree.DInt)) - } + statsRow.AvgSize = (uint64)(*r[avgSizeIdx].(*tree.DInt)) if r[nameIdx] != tree.DNull { statsRow.Name = string(*r[nameIdx].(*tree.DString)) } - if err := statsRow.DecodeAndSetHistogram(ctx, &p.semaCtx, r[histIdx]); err != nil { + if err := statsRow.DecodeAndSetHistogram(ctx, &p.semaCtx, r[histogramIdx]); err != nil { v.Close(ctx) return nil, err } @@ -203,7 +171,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p } for _, r := range rows { - if len(r) != nCols { + if len(r) != numCols { v.Close(ctx) return nil, errors.Errorf("incorrect columns from internal query") } @@ -226,32 +194,19 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p } histogramID := tree.DNull - if r[histIdx] != tree.DNull { + if r[histogramIdx] != tree.DNull { histogramID = r[statIDIdx] } - var res tree.Datums - if avgSizeColVerActive { - res = tree.Datums{ - r[nameIdx], - colNames, - r[createdAtIdx], - r[rowCountIdx], - r[distinctCountIdx], - r[nullCountIdx], - r[avgSizeIdx], - histogramID, - } - } else { - res = tree.Datums{ - r[nameIdx], - colNames, - r[createdAtIdx], - r[rowCountIdx], - r[distinctCountIdx], - r[nullCountIdx], - histogramID, - } + res := tree.Datums{ + r[nameIdx], + colNames, + r[createdAtIdx], + r[rowCountIdx], + r[distinctCountIdx], + r[nullCountIdx], + r[avgSizeIdx], + histogramID, } if _, err := v.rows.AddRow(ctx, res); err != nil { diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index 8a507fb78473..c318b0b73c2e 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -19,7 +19,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/stats", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", diff --git a/pkg/sql/stats/new_stat.go b/pkg/sql/stats/new_stat.go index 0d1834658036..d46e1abac290 100644 --- a/pkg/sql/stats/new_stat.go +++ b/pkg/sql/stats/new_stat.go @@ -13,7 +13,6 @@ package stats import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -89,28 +88,6 @@ func InsertNewStat( return err } } - if !settings.Version.IsActive(ctx, clusterversion.AlterSystemTableStatisticsAddAvgSizeCol) { - _, err := executor.Exec( - ctx, "insert-statistic", txn, - `INSERT INTO system.table_statistics ( - "tableID", - "name", - "columnIDs", - "rowCount", - "distinctCount", - "nullCount", - histogram - ) VALUES ($1, $2, $3, $4, $5, $6, $7)`, - tableID, - nameVal, - columnIDsVal, - rowCount, - distinctCount, - nullCount, - histogramVal, - ) - return err - } _, err := executor.Exec( ctx, "insert-statistic", txn, `INSERT INTO system.table_statistics ( diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index cd4090b6a9cf..ddeb7aea4705 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -12,10 +12,8 @@ package stats import ( "context" - "fmt" "sync" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" @@ -491,22 +489,15 @@ const ( // parseStats converts the given datums to a TableStatistic object. It might // need to run a query to get user defined type metadata. func (sc *TableStatisticsCache) parseStats( - ctx context.Context, datums tree.Datums, avgSizeColVerActive bool, + ctx context.Context, datums tree.Datums, ) (*TableStatistic, error) { if datums == nil || datums.Len() == 0 { return nil, nil } - hgIndex := histogramIndex - numStats := statsLen - if !avgSizeColVerActive { - hgIndex = histogramIndex - 1 - numStats = statsLen - 1 - } - // Validate the input length. - if datums.Len() != numStats { - return nil, errors.Errorf("%d values returned from table statistics lookup. Expected %d", datums.Len(), numStats) + if datums.Len() != statsLen { + return nil, errors.Errorf("%d values returned from table statistics lookup. Expected %d", datums.Len(), statsLen) } // Validate the input types. @@ -524,22 +515,8 @@ func (sc *TableStatisticsCache) parseStats( {"rowCount", rowCountIndex, types.Int, false}, {"distinctCount", distinctCountIndex, types.Int, false}, {"nullCount", nullCountIndex, types.Int, false}, - {"histogram", hgIndex, types.Bytes, true}, - } - // It's ok for expectedTypes to be in a different order than the input datums - // since we don't rely on a precise order of expectedTypes when we check them - // below. - if avgSizeColVerActive { - expectedTypes = append(expectedTypes, - struct { - fieldName string - fieldIndex int - expectedType *types.T - nullable bool - }{ - "avgSize", avgSizeIndex, types.Int, false, - }, - ) + {"avgSize", avgSizeIndex, types.Int, false}, + {"histogram", histogramIndex, types.Bytes, true}, } for _, v := range expectedTypes { if !datums[v.fieldIndex].ResolvedType().Equivalent(v.expectedType) && @@ -558,11 +535,9 @@ func (sc *TableStatisticsCache) parseStats( RowCount: (uint64)(*datums[rowCountIndex].(*tree.DInt)), DistinctCount: (uint64)(*datums[distinctCountIndex].(*tree.DInt)), NullCount: (uint64)(*datums[nullCountIndex].(*tree.DInt)), + AvgSize: (uint64)(*datums[avgSizeIndex].(*tree.DInt)), }, } - if avgSizeColVerActive { - res.AvgSize = (uint64)(*datums[avgSizeIndex].(*tree.DInt)) - } columnIDs := datums[columnIDsIndex].(*tree.DArray) res.ColumnIDs = make([]descpb.ColumnID, len(columnIDs.Array)) for i, d := range columnIDs.Array { @@ -571,10 +546,10 @@ func (sc *TableStatisticsCache) parseStats( if datums[nameIndex] != tree.DNull { res.Name = string(*datums[nameIndex].(*tree.DString)) } - if datums[hgIndex] != tree.DNull { + if datums[histogramIndex] != tree.DNull { res.HistogramData = &HistogramData{} if err := protoutil.Unmarshal( - []byte(*datums[hgIndex].(*tree.DBytes)), + []byte(*datums[histogramIndex].(*tree.DBytes)), res.HistogramData, ); err != nil { return nil, err @@ -661,13 +636,7 @@ func DecodeHistogramBuckets(tabStat *TableStatistic) error { func (sc *TableStatisticsCache) getTableStatsFromDB( ctx context.Context, tableID descpb.ID, ) ([]*TableStatistic, error) { - avgSizeColVerActive := sc.Settings.Version.IsActive(ctx, clusterversion.AlterSystemTableStatisticsAddAvgSizeCol) - var avgSize string - if avgSizeColVerActive { - avgSize = ` - "avgSize",` - } - getTableStatisticsStmt := fmt.Sprintf(` + getTableStatisticsStmt := ` SELECT "tableID", "statisticID", @@ -677,12 +646,12 @@ SELECT "rowCount", "distinctCount", "nullCount", - %s + "avgSize", histogram FROM system.table_statistics WHERE "tableID" = $1 ORDER BY "createdAt" DESC -`, avgSize) +` it, err := sc.SQLExecutor.QueryIterator( ctx, "get-table-statistics", nil /* txn */, getTableStatisticsStmt, tableID, @@ -694,7 +663,7 @@ ORDER BY "createdAt" DESC var statsList []*TableStatistic var ok bool for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - stats, err := sc.parseStats(ctx, it.Cur(), avgSizeColVerActive) + stats, err := sc.parseStats(ctx, it.Cur()) if err != nil { log.Warningf(ctx, "could not decode statistic for table %d: %v", tableID, err) continue diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 481a0d54dd50..cbc8091c4022 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -6,7 +6,6 @@ go_library( srcs = [ "alter_sql_instances_locality.go", "alter_table_protected_timestamp_records.go", - "alter_table_statistics_avg_size.go", "comment_on_index_migration.go", "descriptor_utils.go", "insert_missing_public_schema_namespace_entry.go", @@ -71,7 +70,6 @@ go_test( srcs = [ "alter_sql_instances_locality_test.go", "alter_table_protected_timestamp_records_test.go", - "alter_table_statistics_avg_size_test.go", "builtins_test.go", "comment_on_index_migration_external_test.go", "descriptor_utils_test.go", diff --git a/pkg/upgrade/upgrades/alter_table_statistics_avg_size.go b/pkg/upgrade/upgrades/alter_table_statistics_avg_size.go deleted file mode 100644 index bd70ff7159a3..000000000000 --- a/pkg/upgrade/upgrades/alter_table_statistics_avg_size.go +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2021 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 upgrades - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/upgrade" -) - -const addAvgSizeCol = ` -ALTER TABLE system.table_statistics -ADD COLUMN IF NOT EXISTS "avgSize" INT8 NOT NULL DEFAULT (INT8 '0') -FAMILY "fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram" -` - -func alterSystemTableStatisticsAddAvgSize( - ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, -) error { - op := operation{ - name: "add-table-statistics-avgSize-col", - schemaList: []string{"total_consumption"}, - query: addAvgSizeCol, - schemaExistsFn: hasColumn, - } - if err := migrateTable(ctx, cs, d, op, keys.TableStatisticsTableID, systemschema.TableStatisticsTable); err != nil { - return err - } - return nil -} diff --git a/pkg/upgrade/upgrades/alter_table_statistics_avg_size_test.go b/pkg/upgrade/upgrades/alter_table_statistics_avg_size_test.go deleted file mode 100644 index 456748defed3..000000000000 --- a/pkg/upgrade/upgrades/alter_table_statistics_avg_size_test.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2021 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 upgrades_test - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" -) - -func TestAlterSystemTableStatisticsTable(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - clusterArgs := base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.AlterSystemTableStatisticsAddAvgSizeCol - 1), - }, - }, - }, - } - - var ( - ctx = context.Background() - - tc = testcluster.StartTestCluster(t, 1, clusterArgs) - s = tc.Server(0) - sqlDB = tc.ServerConn(0) - ) - defer tc.Stopper().Stop(ctx) - - var ( - validationSchemas = []upgrades.Schema{ - {Name: "avgSize", ValidationFn: upgrades.HasColumn}, - {Name: "fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram", - ValidationFn: upgrades.HasColumnFamily}, - } - ) - - // Inject the old copy of the descriptor. - upgrades.InjectLegacyTable(ctx, t, s, systemschema.TableStatisticsTable, getDeprecatedTableStatisticsDescriptor) - // Validate that the table statistics table has the old schema. - upgrades.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.TableStatisticsTableID, - systemschema.TableStatisticsTable, - []string{}, - validationSchemas, - false, /* expectExists */ - ) - // Run the upgrade. - upgrades.Upgrade( - t, - sqlDB, - clusterversion.AlterSystemTableStatisticsAddAvgSizeCol, - nil, /* done */ - false, /* expectError */ - ) - // Validate that the table has new schema. - upgrades.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.TableStatisticsTableID, - systemschema.TableStatisticsTable, - []string{}, - validationSchemas, - true, /* expectExists */ - ) -} - -// getDeprecatedTableStatisticsDescriptor returns the system.table_statistics -// table descriptor that was being used before adding a new column in the -// current version. -func getDeprecatedTableStatisticsDescriptor() *descpb.TableDescriptor { - uniqueRowIDString := "unique_rowid()" - nowString := "now():::TIMESTAMP" - - return &descpb.TableDescriptor{ - Name: "table_statistics", - ID: keys.TableStatisticsTableID, - ParentID: keys.SystemDatabaseID, - UnexposedParentSchemaID: keys.PublicSchemaID, - Version: 1, - Columns: []descpb.ColumnDescriptor{ - {Name: "tableID", ID: 1, Type: types.Int}, - {Name: "statisticID", ID: 2, Type: types.Int, DefaultExpr: &uniqueRowIDString}, - {Name: "name", ID: 3, Type: types.String, Nullable: true}, - {Name: "columnIDs", ID: 4, Type: types.IntArray}, - {Name: "createdAt", ID: 5, Type: types.Timestamp, DefaultExpr: &nowString}, - {Name: "rowCount", ID: 6, Type: types.Int}, - {Name: "distinctCount", ID: 7, Type: types.Int}, - {Name: "nullCount", ID: 8, Type: types.Int}, - {Name: "histogram", ID: 9, Type: types.Bytes, Nullable: true}, - }, - NextColumnID: 10, - Families: []descpb.ColumnFamilyDescriptor{ - { - Name: "fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram", - ID: 0, - ColumnNames: []string{ - "tableID", - "statisticID", - "name", - "columnIDs", - "createdAt", - "rowCount", - "distinctCount", - "nullCount", - "histogram", - }, - ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8, 9}, - }, - }, - NextFamilyID: 1, - PrimaryIndex: descpb.IndexDescriptor{ - Name: "primary", - ID: 1, - Unique: true, - KeyColumnNames: []string{"tableID", "statisticID"}, - KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC, catpb.IndexColumn_ASC}, - KeyColumnIDs: []descpb.ColumnID{1, 2}, - }, - NextIndexID: 2, - Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), - NextMutationID: 1, - FormatVersion: 3, - } -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 273b4b27fbdc..d6c564f9fbdd 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -54,12 +54,6 @@ var upgrades = []upgrade.Upgrade{ preconditionBeforeStartingAnUpgrade, NoTenantUpgradeFunc, ), - upgrade.NewTenantUpgrade( - "add column avgSize to table system.table_statistics", - toCV(clusterversion.AlterSystemTableStatisticsAddAvgSizeCol), - NoPrecondition, - alterSystemTableStatisticsAddAvgSize, - ), upgrade.NewTenantUpgrade( "seed system.span_configurations with configs for existing tenants", toCV(clusterversion.SeedTenantSpanConfigs), From 04a41bfa96918900c3494709787d13efb4dc252e Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 16:56:55 -0700 Subject: [PATCH 08/16] clusterversion: remove MVCCAddSSTable version Release note: None --- pkg/ccl/backupccl/restore_data_processor.go | 6 - pkg/clusterversion/cockroach_versions.go | 7 -- pkg/clusterversion/key_string.go | 121 +++++++++---------- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 4 +- pkg/sql/importer/import_processor.go | 4 - 5 files changed, 61 insertions(+), 81 deletions(-) diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index 0c9ca8de4e06..50d69fca0ac2 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/bulk" @@ -409,11 +408,6 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( defer sst.cleanup() writeAtBatchTS := restoreAtNow.Get(&evalCtx.Settings.SV) - if writeAtBatchTS && !evalCtx.Settings.Version.IsActive(ctx, clusterversion.MVCCAddSSTable) { - return roachpb.BulkOpSummary{}, errors.Newf( - "cannot use %s until version %s", restoreAtNow.Key(), clusterversion.MVCCAddSSTable.String(), - ) - } // If the system tenant is restoring a guest tenant span, we don't want to // forward all the restored data to now, as there may be importing tables in diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index e0186ff2354c..cdf1d1329ccb 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new - // SSTTimestampToRequestTimestamp and DisallowConflicts parameters. - MVCCAddSSTable // InsertPublicSchemaNamespaceEntryOnRestore ensures all public schemas // have an entry in system.namespace upon being restored. InsertPublicSchemaNamespaceEntryOnRestore @@ -409,10 +406,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: MVCCAddSSTable, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 16}, - }, { Key: InsertPublicSchemaNamespaceEntryOnRestore, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 18}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index a14d0badd802..51a0d0fc1ca9 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,70 +10,69 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[MVCCAddSSTable-2] - _ = x[InsertPublicSchemaNamespaceEntryOnRestore-3] - _ = x[UnsplitRangesInAsyncGCJobs-4] - _ = x[ValidateGrantOption-5] - _ = x[PebbleFormatBlockPropertyCollector-6] - _ = x[ProbeRequest-7] - _ = x[SelectRPCsTakeTracingInfoInband-8] - _ = x[PreSeedTenantSpanConfigs-9] - _ = x[SeedTenantSpanConfigs-10] - _ = x[PublicSchemasWithDescriptors-11] - _ = x[EnsureSpanConfigReconciliation-12] - _ = x[EnsureSpanConfigSubscription-13] - _ = x[EnableSpanConfigStore-14] - _ = x[ScanWholeRows-15] - _ = x[SCRAMAuthentication-16] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-17] - _ = x[AlterSystemProtectedTimestampAddColumn-18] - _ = x[EnableProtectedTimestampsForTenant-19] - _ = x[DeleteCommentsWithDroppedIndexes-20] - _ = x[RemoveIncompatibleDatabasePrivileges-21] - _ = x[AddRaftAppliedIndexTermMigration-22] - _ = x[PostAddRaftAppliedIndexTermMigration-23] - _ = x[DontProposeWriteTimestampForLeaseTransfers-24] - _ = x[EnablePebbleFormatVersionBlockProperties-25] - _ = x[DisableSystemConfigGossipTrigger-26] - _ = x[MVCCIndexBackfiller-27] - _ = x[EnableLeaseHolderRemoval-28] - _ = x[BackupResolutionInJob-29] - _ = x[LooselyCoupledRaftLogTruncation-30] - _ = x[ChangefeedIdleness-31] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-32] - _ = x[EnableDeclarativeSchemaChanger-33] - _ = x[RowLevelTTL-34] - _ = x[PebbleFormatSplitUserKeysMarked-35] - _ = x[IncrementalBackupSubdir-36] - _ = x[DateStyleIntervalStyleCastRewrite-37] - _ = x[EnableNewStoreRebalancer-38] - _ = x[ClusterLocksVirtualTable-39] - _ = x[AutoStatsTableSettings-40] - _ = x[ForecastStats-41] - _ = x[SuperRegions-42] - _ = x[EnableNewChangefeedOptions-43] - _ = x[SpanCountTable-44] - _ = x[PreSeedSpanCountTable-45] - _ = x[SeedSpanCountTable-46] - _ = x[V22_1-47] - _ = x[Start22_2-48] - _ = x[LocalTimestamps-49] - _ = x[EnsurePebbleFormatVersionRangeKeys-50] - _ = x[EnablePebbleFormatVersionRangeKeys-51] - _ = x[TrigramInvertedIndexes-52] - _ = x[RemoveGrantPrivilege-53] - _ = x[MVCCRangeTombstones-54] - _ = x[UpgradeSequenceToBeReferencedByID-55] - _ = x[SampledStmtDiagReqs-56] - _ = x[AddSSTableTombstones-57] - _ = x[SystemPrivilegesTable-58] - _ = x[EnablePredicateProjectionChangefeed-59] - _ = x[AlterSystemSQLInstancesAddLocality-60] + _ = x[InsertPublicSchemaNamespaceEntryOnRestore-2] + _ = x[UnsplitRangesInAsyncGCJobs-3] + _ = x[ValidateGrantOption-4] + _ = x[PebbleFormatBlockPropertyCollector-5] + _ = x[ProbeRequest-6] + _ = x[SelectRPCsTakeTracingInfoInband-7] + _ = x[PreSeedTenantSpanConfigs-8] + _ = x[SeedTenantSpanConfigs-9] + _ = x[PublicSchemasWithDescriptors-10] + _ = x[EnsureSpanConfigReconciliation-11] + _ = x[EnsureSpanConfigSubscription-12] + _ = x[EnableSpanConfigStore-13] + _ = x[ScanWholeRows-14] + _ = x[SCRAMAuthentication-15] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-16] + _ = x[AlterSystemProtectedTimestampAddColumn-17] + _ = x[EnableProtectedTimestampsForTenant-18] + _ = x[DeleteCommentsWithDroppedIndexes-19] + _ = x[RemoveIncompatibleDatabasePrivileges-20] + _ = x[AddRaftAppliedIndexTermMigration-21] + _ = x[PostAddRaftAppliedIndexTermMigration-22] + _ = x[DontProposeWriteTimestampForLeaseTransfers-23] + _ = x[EnablePebbleFormatVersionBlockProperties-24] + _ = x[DisableSystemConfigGossipTrigger-25] + _ = x[MVCCIndexBackfiller-26] + _ = x[EnableLeaseHolderRemoval-27] + _ = x[BackupResolutionInJob-28] + _ = x[LooselyCoupledRaftLogTruncation-29] + _ = x[ChangefeedIdleness-30] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-31] + _ = x[EnableDeclarativeSchemaChanger-32] + _ = x[RowLevelTTL-33] + _ = x[PebbleFormatSplitUserKeysMarked-34] + _ = x[IncrementalBackupSubdir-35] + _ = x[DateStyleIntervalStyleCastRewrite-36] + _ = x[EnableNewStoreRebalancer-37] + _ = x[ClusterLocksVirtualTable-38] + _ = x[AutoStatsTableSettings-39] + _ = x[ForecastStats-40] + _ = x[SuperRegions-41] + _ = x[EnableNewChangefeedOptions-42] + _ = x[SpanCountTable-43] + _ = x[PreSeedSpanCountTable-44] + _ = x[SeedSpanCountTable-45] + _ = x[V22_1-46] + _ = x[Start22_2-47] + _ = x[LocalTimestamps-48] + _ = x[EnsurePebbleFormatVersionRangeKeys-49] + _ = x[EnablePebbleFormatVersionRangeKeys-50] + _ = x[TrigramInvertedIndexes-51] + _ = x[RemoveGrantPrivilege-52] + _ = x[MVCCRangeTombstones-53] + _ = x[UpgradeSequenceToBeReferencedByID-54] + _ = x[SampledStmtDiagReqs-55] + _ = x[AddSSTableTombstones-56] + _ = x[SystemPrivilegesTable-57] + _ = x[EnablePredicateProjectionChangefeed-58] + _ = x[AlterSystemSQLInstancesAddLocality-59] } -const _Key_name = "V21_2Start22_1MVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1InsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 28, 69, 95, 114, 148, 160, 191, 215, 236, 264, 294, 322, 343, 356, 375, 409, 447, 481, 513, 549, 581, 617, 659, 699, 731, 750, 774, 795, 826, 844, 885, 915, 926, 957, 980, 1013, 1037, 1061, 1083, 1096, 1108, 1134, 1148, 1169, 1187, 1192, 1201, 1216, 1250, 1284, 1306, 1326, 1345, 1378, 1397, 1417, 1438, 1473, 1507} +var _Key_index = [...]uint16{0, 5, 14, 55, 81, 100, 134, 146, 177, 201, 222, 250, 280, 308, 329, 342, 361, 395, 433, 467, 499, 535, 567, 603, 645, 685, 717, 736, 760, 781, 812, 830, 871, 901, 912, 943, 966, 999, 1023, 1047, 1069, 1082, 1094, 1120, 1134, 1155, 1173, 1178, 1187, 1202, 1236, 1270, 1292, 1312, 1331, 1364, 1383, 1403, 1424, 1459, 1493} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index dfcd135dd9e4..52a3105c0ca5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -15,7 +15,6 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -137,8 +136,7 @@ func EvalAddSSTable( } // Reject AddSSTable requests not writing at the request timestamp if requested. - if cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.MVCCAddSSTable) && - AddSSTableRequireAtRequestTimestamp.Get(&cArgs.EvalCtx.ClusterSettings().SV) && + if AddSSTableRequireAtRequestTimestamp.Get(&cArgs.EvalCtx.ClusterSettings().SV) && sstToReqTS.IsEmpty() { return result.Result{}, errors.AssertionFailedf( "AddSSTable requests must set SSTTimestampToRequestTimestamp") diff --git a/pkg/sql/importer/import_processor.go b/pkg/sql/importer/import_processor.go index c8952e6d22f3..c3cc6af4a9df 100644 --- a/pkg/sql/importer/import_processor.go +++ b/pkg/sql/importer/import_processor.go @@ -18,7 +18,6 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -355,9 +354,6 @@ func ingestKvs( if !importAtNow.Get(&flowCtx.Cfg.Settings.SV) { log.Warningf(ctx, "ingesting import data with raw timestamps due to cluster setting") writeAtBatchTimestamp = false - } else if !flowCtx.Cfg.Settings.Version.IsActive(ctx, clusterversion.MVCCAddSSTable) { - log.Warningf(ctx, "ingesting import data with raw timestamps due to cluster version") - writeAtBatchTimestamp = false } var pkAdderName, indexAdderName = "rows", "indexes" From 56c480f3c88230cfbbb32100b3fb01007b3258e8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 17:01:10 -0700 Subject: [PATCH 09/16] upgrades: remove InsertPublicSchemaNamespaceEntryOnRestore migration Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 - ...lic_schema_namespace_entry_restore_test.go | 96 -------------- pkg/clusterversion/cockroach_versions.go | 7 -- pkg/clusterversion/key_string.go | 119 +++++++++--------- pkg/upgrade/upgrades/BUILD.bazel | 1 - ...t_missing_public_schema_namespace_entry.go | 82 ------------ pkg/upgrade/upgrades/upgrades.go | 5 - 7 files changed, 59 insertions(+), 252 deletions(-) delete mode 100644 pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go delete mode 100644 pkg/upgrade/upgrades/insert_missing_public_schema_namespace_entry.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index e008ea79f760..0ee3cdc71fd4 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -151,7 +151,6 @@ go_test( "create_scheduled_backup_test.go", "datadriven_test.go", "full_cluster_backup_restore_test.go", - "insert_missing_public_schema_namespace_entry_restore_test.go", "key_rewriter_test.go", "main_test.go", "partitioned_backup_test.go", diff --git a/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go b/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go deleted file mode 100644 index 9f3072c7ea94..000000000000 --- a/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package backupccl_test - -import ( - "context" - "fmt" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/stretchr/testify/require" -) - -func TestInsertMissingPublicSchemaNamespaceEntry(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - dir, cleanup := testutils.TempDir(t) - defer cleanup() - tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - // Disabling the default test tenant due to test failures. More - // investigation is required. Tracked with #76378. - DisableDefaultTestTenant: true, - ExternalIODir: dir, - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore - 1), - }, - }, - }, - }) - defer tc.Stopper().Stop(ctx) - - db := tc.ServerConn(0) - defer db.Close() - sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - - // Mimic a restore where the public schema system.namespace entries are - // missing. - sqlDB.Exec(t, `CREATE DATABASE db1`) - sqlDB.Exec(t, `CREATE TABLE db1.t()`) - sqlDB.Exec(t, `CREATE SCHEMA db1.s`) - sqlDB.Exec(t, `CREATE DATABASE db2`) - sqlDB.Exec(t, `CREATE TABLE db2.t(x INT)`) - sqlDB.Exec(t, `INSERT INTO db2.t VALUES (1), (2)`) - sqlDB.Exec(t, `CREATE SCHEMA db2.s`) - sqlDB.Exec(t, `CREATE TABLE db2.s.t(x INT)`) - sqlDB.Exec(t, `INSERT INTO db2.s.t VALUES (1), (2)`) - - var db1ID, db2ID descpb.ID - row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'db1'`) - row.Scan(&db1ID) - row = sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'db2'`) - row.Scan(&db2ID) - - // Remove system.namespace entries for the public schema for the two - // databases. - err := tc.Servers[0].DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - codec := keys.SystemSQLCodec - b := txn.NewBatch() - b.Del(catalogkeys.MakeSchemaNameKey(codec, db1ID, `public`)) - b.Del(catalogkeys.MakeSchemaNameKey(codec, db2ID, `public`)) - return txn.Run(ctx, b) - }) - require.NoError(t, err) - - // Verify that there are no system.namespace entries for the public schema for - // the two databases. - sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db1ID), [][]string{}) - sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db2ID), [][]string{}) - - // Kick off migration by upgrading to the new version. - _ = sqlDB.Exec(t, `SET CLUSTER SETTING version = $1`, - clusterversion.ByKey(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore).String()) - - sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db1ID), [][]string{{"29"}}) - sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db2ID), [][]string{{"29"}}) - -} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index cdf1d1329ccb..c96b97e0c8d3 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // InsertPublicSchemaNamespaceEntryOnRestore ensures all public schemas - // have an entry in system.namespace upon being restored. - InsertPublicSchemaNamespaceEntryOnRestore // UnsplitRangesInAsyncGCJobs moves ranges unsplitting from transaction of // "drop table"/"truncate table" to async gc jobs UnsplitRangesInAsyncGCJobs @@ -406,10 +403,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: InsertPublicSchemaNamespaceEntryOnRestore, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 18}, - }, { Key: UnsplitRangesInAsyncGCJobs, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 20}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 51a0d0fc1ca9..005ddecda3a1 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,69 +10,68 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[InsertPublicSchemaNamespaceEntryOnRestore-2] - _ = x[UnsplitRangesInAsyncGCJobs-3] - _ = x[ValidateGrantOption-4] - _ = x[PebbleFormatBlockPropertyCollector-5] - _ = x[ProbeRequest-6] - _ = x[SelectRPCsTakeTracingInfoInband-7] - _ = x[PreSeedTenantSpanConfigs-8] - _ = x[SeedTenantSpanConfigs-9] - _ = x[PublicSchemasWithDescriptors-10] - _ = x[EnsureSpanConfigReconciliation-11] - _ = x[EnsureSpanConfigSubscription-12] - _ = x[EnableSpanConfigStore-13] - _ = x[ScanWholeRows-14] - _ = x[SCRAMAuthentication-15] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-16] - _ = x[AlterSystemProtectedTimestampAddColumn-17] - _ = x[EnableProtectedTimestampsForTenant-18] - _ = x[DeleteCommentsWithDroppedIndexes-19] - _ = x[RemoveIncompatibleDatabasePrivileges-20] - _ = x[AddRaftAppliedIndexTermMigration-21] - _ = x[PostAddRaftAppliedIndexTermMigration-22] - _ = x[DontProposeWriteTimestampForLeaseTransfers-23] - _ = x[EnablePebbleFormatVersionBlockProperties-24] - _ = x[DisableSystemConfigGossipTrigger-25] - _ = x[MVCCIndexBackfiller-26] - _ = x[EnableLeaseHolderRemoval-27] - _ = x[BackupResolutionInJob-28] - _ = x[LooselyCoupledRaftLogTruncation-29] - _ = x[ChangefeedIdleness-30] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-31] - _ = x[EnableDeclarativeSchemaChanger-32] - _ = x[RowLevelTTL-33] - _ = x[PebbleFormatSplitUserKeysMarked-34] - _ = x[IncrementalBackupSubdir-35] - _ = x[DateStyleIntervalStyleCastRewrite-36] - _ = x[EnableNewStoreRebalancer-37] - _ = x[ClusterLocksVirtualTable-38] - _ = x[AutoStatsTableSettings-39] - _ = x[ForecastStats-40] - _ = x[SuperRegions-41] - _ = x[EnableNewChangefeedOptions-42] - _ = x[SpanCountTable-43] - _ = x[PreSeedSpanCountTable-44] - _ = x[SeedSpanCountTable-45] - _ = x[V22_1-46] - _ = x[Start22_2-47] - _ = x[LocalTimestamps-48] - _ = x[EnsurePebbleFormatVersionRangeKeys-49] - _ = x[EnablePebbleFormatVersionRangeKeys-50] - _ = x[TrigramInvertedIndexes-51] - _ = x[RemoveGrantPrivilege-52] - _ = x[MVCCRangeTombstones-53] - _ = x[UpgradeSequenceToBeReferencedByID-54] - _ = x[SampledStmtDiagReqs-55] - _ = x[AddSSTableTombstones-56] - _ = x[SystemPrivilegesTable-57] - _ = x[EnablePredicateProjectionChangefeed-58] - _ = x[AlterSystemSQLInstancesAddLocality-59] + _ = x[UnsplitRangesInAsyncGCJobs-2] + _ = x[ValidateGrantOption-3] + _ = x[PebbleFormatBlockPropertyCollector-4] + _ = x[ProbeRequest-5] + _ = x[SelectRPCsTakeTracingInfoInband-6] + _ = x[PreSeedTenantSpanConfigs-7] + _ = x[SeedTenantSpanConfigs-8] + _ = x[PublicSchemasWithDescriptors-9] + _ = x[EnsureSpanConfigReconciliation-10] + _ = x[EnsureSpanConfigSubscription-11] + _ = x[EnableSpanConfigStore-12] + _ = x[ScanWholeRows-13] + _ = x[SCRAMAuthentication-14] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-15] + _ = x[AlterSystemProtectedTimestampAddColumn-16] + _ = x[EnableProtectedTimestampsForTenant-17] + _ = x[DeleteCommentsWithDroppedIndexes-18] + _ = x[RemoveIncompatibleDatabasePrivileges-19] + _ = x[AddRaftAppliedIndexTermMigration-20] + _ = x[PostAddRaftAppliedIndexTermMigration-21] + _ = x[DontProposeWriteTimestampForLeaseTransfers-22] + _ = x[EnablePebbleFormatVersionBlockProperties-23] + _ = x[DisableSystemConfigGossipTrigger-24] + _ = x[MVCCIndexBackfiller-25] + _ = x[EnableLeaseHolderRemoval-26] + _ = x[BackupResolutionInJob-27] + _ = x[LooselyCoupledRaftLogTruncation-28] + _ = x[ChangefeedIdleness-29] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-30] + _ = x[EnableDeclarativeSchemaChanger-31] + _ = x[RowLevelTTL-32] + _ = x[PebbleFormatSplitUserKeysMarked-33] + _ = x[IncrementalBackupSubdir-34] + _ = x[DateStyleIntervalStyleCastRewrite-35] + _ = x[EnableNewStoreRebalancer-36] + _ = x[ClusterLocksVirtualTable-37] + _ = x[AutoStatsTableSettings-38] + _ = x[ForecastStats-39] + _ = x[SuperRegions-40] + _ = x[EnableNewChangefeedOptions-41] + _ = x[SpanCountTable-42] + _ = x[PreSeedSpanCountTable-43] + _ = x[SeedSpanCountTable-44] + _ = x[V22_1-45] + _ = x[Start22_2-46] + _ = x[LocalTimestamps-47] + _ = x[EnsurePebbleFormatVersionRangeKeys-48] + _ = x[EnablePebbleFormatVersionRangeKeys-49] + _ = x[TrigramInvertedIndexes-50] + _ = x[RemoveGrantPrivilege-51] + _ = x[MVCCRangeTombstones-52] + _ = x[UpgradeSequenceToBeReferencedByID-53] + _ = x[SampledStmtDiagReqs-54] + _ = x[AddSSTableTombstones-55] + _ = x[SystemPrivilegesTable-56] + _ = x[EnablePredicateProjectionChangefeed-57] + _ = x[AlterSystemSQLInstancesAddLocality-58] } -const _Key_name = "V21_2Start22_1InsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1UnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 55, 81, 100, 134, 146, 177, 201, 222, 250, 280, 308, 329, 342, 361, 395, 433, 467, 499, 535, 567, 603, 645, 685, 717, 736, 760, 781, 812, 830, 871, 901, 912, 943, 966, 999, 1023, 1047, 1069, 1082, 1094, 1120, 1134, 1155, 1173, 1178, 1187, 1202, 1236, 1270, 1292, 1312, 1331, 1364, 1383, 1403, 1424, 1459, 1493} +var _Key_index = [...]uint16{0, 5, 14, 40, 59, 93, 105, 136, 160, 181, 209, 239, 267, 288, 301, 320, 354, 392, 426, 458, 494, 526, 562, 604, 644, 676, 695, 719, 740, 771, 789, 830, 860, 871, 902, 925, 958, 982, 1006, 1028, 1041, 1053, 1079, 1093, 1114, 1132, 1137, 1146, 1161, 1195, 1229, 1251, 1271, 1290, 1323, 1342, 1362, 1383, 1418, 1452} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index cbc8091c4022..002d285f667e 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -8,7 +8,6 @@ go_library( "alter_table_protected_timestamp_records.go", "comment_on_index_migration.go", "descriptor_utils.go", - "insert_missing_public_schema_namespace_entry.go", "migrate_span_configs.go", "precondition_before_starting_an_upgrade.go", "public_schema_migration.go", diff --git a/pkg/upgrade/upgrades/insert_missing_public_schema_namespace_entry.go b/pkg/upgrade/upgrades/insert_missing_public_schema_namespace_entry.go deleted file mode 100644 index b069e7e45437..000000000000 --- a/pkg/upgrade/upgrades/insert_missing_public_schema_namespace_entry.go +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright 2021 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 upgrades - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/upgrade" -) - -// insertMissingPublicSchemaNamespaceEntry creates a system.namespace entries -// for public schemas that are missing a system.namespace entry. -// This arises from restore where we mistakenly did not create system.namespace -// entries for public schemas when restoring databases. -func insertMissingPublicSchemaNamespaceEntry( - ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, -) error { - // Get the ID of all databases where we're missing a public schema namespace - // entry for. - query := ` - SELECT id - FROM system.namespace - WHERE id - NOT IN ( - SELECT ns_db.id - FROM system.namespace AS ns_db - INNER JOIN system.namespace - AS ns_sc ON ( - ns_db.id - = ns_sc."parentID" - ) - WHERE ns_db."parentSchemaID" = 0 - AND ns_db."parentID" = 0 - AND ns_sc."parentSchemaID" = 0 - AND ns_sc.name = 'public' - AND ns_sc.id = 29 - ) - AND "parentID" = 0 -ORDER BY id ASC; -` - rows, err := d.InternalExecutor.QueryIterator( - ctx, "get_databases_without_public_schema_namespace_entry", nil /* txn */, query, - ) - if err != nil { - return err - } - var databaseIDs []descpb.ID - for ok, err := rows.Next(ctx); ok; ok, err = rows.Next(ctx) { - if err != nil { - return err - } - id := descpb.ID(tree.MustBeDInt(rows.Cur()[0])) - databaseIDs = append(databaseIDs, id) - } - - return d.CollectionFactory.Txn(ctx, d.InternalExecutor, d.DB, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, - ) error { - b := txn.NewBatch() - for _, dbID := range databaseIDs { - publicSchemaKey := catalogkeys.MakeSchemaNameKey(d.Codec, dbID, tree.PublicSchema) - b.Put(publicSchemaKey, keys.PublicSchemaID) - } - return txn.Run(ctx, b) - }) -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index d6c564f9fbdd..12dc3856e3ea 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -60,11 +60,6 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, seedTenantSpanConfigsMigration, ), - upgrade.NewTenantUpgrade("insert missing system.namespace entries for public schemas", - toCV(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore), - NoPrecondition, - insertMissingPublicSchemaNamespaceEntry, - ), upgrade.NewTenantUpgrade( "add column target to system.protected_ts_records", toCV(clusterversion.AlterSystemProtectedTimestampAddColumn), From ec5c4ce05fb048bf5f0ddde115545050d7a136a8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 17:09:00 -0700 Subject: [PATCH 10/16] sql: remove UnsplitRangesInAsyncGCJobs version gate Release note: None --- pkg/clusterversion/cockroach_versions.go | 7 -- pkg/clusterversion/key_string.go | 117 +++++++++++------------ pkg/sql/drop_index.go | 38 -------- pkg/sql/drop_table.go | 58 ----------- pkg/sql/gcjob/BUILD.bazel | 1 + pkg/sql/gcjob/gc_job.go | 37 ++++++- pkg/sql/pgwire/testdata/pgtest/notice | 2 +- pkg/sql/truncate.go | 11 --- pkg/sql/unsplit_range_test.go | 63 +----------- 9 files changed, 99 insertions(+), 235 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index c96b97e0c8d3..9e3f3b7abc6c 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // UnsplitRangesInAsyncGCJobs moves ranges unsplitting from transaction of - // "drop table"/"truncate table" to async gc jobs - UnsplitRangesInAsyncGCJobs // ValidateGrantOption checks whether the current user granting privileges to // another user holds the grant option for those privileges ValidateGrantOption @@ -403,10 +400,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: UnsplitRangesInAsyncGCJobs, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 20}, - }, { Key: ValidateGrantOption, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 22}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 005ddecda3a1..67652caf2eee 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,68 +10,67 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[UnsplitRangesInAsyncGCJobs-2] - _ = x[ValidateGrantOption-3] - _ = x[PebbleFormatBlockPropertyCollector-4] - _ = x[ProbeRequest-5] - _ = x[SelectRPCsTakeTracingInfoInband-6] - _ = x[PreSeedTenantSpanConfigs-7] - _ = x[SeedTenantSpanConfigs-8] - _ = x[PublicSchemasWithDescriptors-9] - _ = x[EnsureSpanConfigReconciliation-10] - _ = x[EnsureSpanConfigSubscription-11] - _ = x[EnableSpanConfigStore-12] - _ = x[ScanWholeRows-13] - _ = x[SCRAMAuthentication-14] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-15] - _ = x[AlterSystemProtectedTimestampAddColumn-16] - _ = x[EnableProtectedTimestampsForTenant-17] - _ = x[DeleteCommentsWithDroppedIndexes-18] - _ = x[RemoveIncompatibleDatabasePrivileges-19] - _ = x[AddRaftAppliedIndexTermMigration-20] - _ = x[PostAddRaftAppliedIndexTermMigration-21] - _ = x[DontProposeWriteTimestampForLeaseTransfers-22] - _ = x[EnablePebbleFormatVersionBlockProperties-23] - _ = x[DisableSystemConfigGossipTrigger-24] - _ = x[MVCCIndexBackfiller-25] - _ = x[EnableLeaseHolderRemoval-26] - _ = x[BackupResolutionInJob-27] - _ = x[LooselyCoupledRaftLogTruncation-28] - _ = x[ChangefeedIdleness-29] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-30] - _ = x[EnableDeclarativeSchemaChanger-31] - _ = x[RowLevelTTL-32] - _ = x[PebbleFormatSplitUserKeysMarked-33] - _ = x[IncrementalBackupSubdir-34] - _ = x[DateStyleIntervalStyleCastRewrite-35] - _ = x[EnableNewStoreRebalancer-36] - _ = x[ClusterLocksVirtualTable-37] - _ = x[AutoStatsTableSettings-38] - _ = x[ForecastStats-39] - _ = x[SuperRegions-40] - _ = x[EnableNewChangefeedOptions-41] - _ = x[SpanCountTable-42] - _ = x[PreSeedSpanCountTable-43] - _ = x[SeedSpanCountTable-44] - _ = x[V22_1-45] - _ = x[Start22_2-46] - _ = x[LocalTimestamps-47] - _ = x[EnsurePebbleFormatVersionRangeKeys-48] - _ = x[EnablePebbleFormatVersionRangeKeys-49] - _ = x[TrigramInvertedIndexes-50] - _ = x[RemoveGrantPrivilege-51] - _ = x[MVCCRangeTombstones-52] - _ = x[UpgradeSequenceToBeReferencedByID-53] - _ = x[SampledStmtDiagReqs-54] - _ = x[AddSSTableTombstones-55] - _ = x[SystemPrivilegesTable-56] - _ = x[EnablePredicateProjectionChangefeed-57] - _ = x[AlterSystemSQLInstancesAddLocality-58] + _ = x[ValidateGrantOption-2] + _ = x[PebbleFormatBlockPropertyCollector-3] + _ = x[ProbeRequest-4] + _ = x[SelectRPCsTakeTracingInfoInband-5] + _ = x[PreSeedTenantSpanConfigs-6] + _ = x[SeedTenantSpanConfigs-7] + _ = x[PublicSchemasWithDescriptors-8] + _ = x[EnsureSpanConfigReconciliation-9] + _ = x[EnsureSpanConfigSubscription-10] + _ = x[EnableSpanConfigStore-11] + _ = x[ScanWholeRows-12] + _ = x[SCRAMAuthentication-13] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-14] + _ = x[AlterSystemProtectedTimestampAddColumn-15] + _ = x[EnableProtectedTimestampsForTenant-16] + _ = x[DeleteCommentsWithDroppedIndexes-17] + _ = x[RemoveIncompatibleDatabasePrivileges-18] + _ = x[AddRaftAppliedIndexTermMigration-19] + _ = x[PostAddRaftAppliedIndexTermMigration-20] + _ = x[DontProposeWriteTimestampForLeaseTransfers-21] + _ = x[EnablePebbleFormatVersionBlockProperties-22] + _ = x[DisableSystemConfigGossipTrigger-23] + _ = x[MVCCIndexBackfiller-24] + _ = x[EnableLeaseHolderRemoval-25] + _ = x[BackupResolutionInJob-26] + _ = x[LooselyCoupledRaftLogTruncation-27] + _ = x[ChangefeedIdleness-28] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-29] + _ = x[EnableDeclarativeSchemaChanger-30] + _ = x[RowLevelTTL-31] + _ = x[PebbleFormatSplitUserKeysMarked-32] + _ = x[IncrementalBackupSubdir-33] + _ = x[DateStyleIntervalStyleCastRewrite-34] + _ = x[EnableNewStoreRebalancer-35] + _ = x[ClusterLocksVirtualTable-36] + _ = x[AutoStatsTableSettings-37] + _ = x[ForecastStats-38] + _ = x[SuperRegions-39] + _ = x[EnableNewChangefeedOptions-40] + _ = x[SpanCountTable-41] + _ = x[PreSeedSpanCountTable-42] + _ = x[SeedSpanCountTable-43] + _ = x[V22_1-44] + _ = x[Start22_2-45] + _ = x[LocalTimestamps-46] + _ = x[EnsurePebbleFormatVersionRangeKeys-47] + _ = x[EnablePebbleFormatVersionRangeKeys-48] + _ = x[TrigramInvertedIndexes-49] + _ = x[RemoveGrantPrivilege-50] + _ = x[MVCCRangeTombstones-51] + _ = x[UpgradeSequenceToBeReferencedByID-52] + _ = x[SampledStmtDiagReqs-53] + _ = x[AddSSTableTombstones-54] + _ = x[SystemPrivilegesTable-55] + _ = x[EnablePredicateProjectionChangefeed-56] + _ = x[AlterSystemSQLInstancesAddLocality-57] } -const _Key_name = "V21_2Start22_1UnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1ValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 40, 59, 93, 105, 136, 160, 181, 209, 239, 267, 288, 301, 320, 354, 392, 426, 458, 494, 526, 562, 604, 644, 676, 695, 719, 740, 771, 789, 830, 860, 871, 902, 925, 958, 982, 1006, 1028, 1041, 1053, 1079, 1093, 1114, 1132, 1137, 1146, 1161, 1195, 1229, 1251, 1271, 1290, 1323, 1342, 1362, 1383, 1418, 1452} +var _Key_index = [...]uint16{0, 5, 14, 33, 67, 79, 110, 134, 155, 183, 213, 241, 262, 275, 294, 328, 366, 400, 432, 468, 500, 536, 578, 618, 650, 669, 693, 714, 745, 763, 804, 834, 845, 876, 899, 932, 956, 980, 1002, 1015, 1027, 1053, 1067, 1088, 1106, 1111, 1120, 1135, 1169, 1203, 1225, 1245, 1264, 1297, 1316, 1336, 1357, 1392, 1426} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index a13692b26be5..fcca9742c84a 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -13,12 +13,8 @@ package sql import ( "context" "fmt" - "strings" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvclient" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -506,40 +502,6 @@ func (p *planner) dropIndexByName( idxEntry := *foundIndex.IndexDesc() idxOrdinal := foundIndex.Ordinal() - // Unsplit all manually split ranges in the index so they can be - // automatically merged by the merge queue. Gate this on being the - // system tenant because secondary tenants aren't allowed to scan - // the meta ranges directly. - // TODO(Chengxiong): Remove this range unsplitting in 22.2 - st := p.EvalContext().Settings - if p.ExecCfg().Codec.ForSystemTenant() && - !st.Version.IsActive(ctx, clusterversion.UnsplitRangesInAsyncGCJobs) { - - span := tableDesc.IndexSpan(p.ExecCfg().Codec, idxEntry.ID) - txn := p.ExecCfg().DB.NewTxn(ctx, "scan-ranges-for-index-drop") - ranges, err := kvclient.ScanMetaKVs(ctx, txn, span) - if err != nil { - return err - } - for _, r := range ranges { - var desc roachpb.RangeDescriptor - if err := r.ValueProto(&desc); err != nil { - return err - } - // We have to explicitly check that the range descriptor's start key - // lies within the span of the index since ScanMetaKVs returns all - // intersecting spans. - if !desc.GetStickyBit().IsEmpty() && span.Key.Compare(desc.StartKey.AsRawKey()) <= 0 { - // Swallow "key is not the start of a range" errors because it would - // mean that the sticky bit was removed and merged concurrently. DROP - // INDEX should not fail because of this. - if err := p.ExecCfg().DB.AdminUnsplit(ctx, desc.StartKey); err != nil && !strings.Contains(err.Error(), "is not the start of a range") { - return err - } - } - } - } - // the idx we picked up with FindIndexByID at the top may not // contain the same field any more due to other schema changes // intervening since the initial lookup. So we send the recent diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 5e35525680b4..431f8df3c09e 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -13,13 +13,9 @@ package sql import ( "context" "fmt" - "strings" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/kv/kvclient" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -326,50 +322,6 @@ func (p *planner) dropTableImpl( return droppedViews, err } -// UnsplitRangesInSpan unsplist any manually split ranges within a span. -// TODO(Chengxiong): move this function to gc_job.go in 22.2 -func UnsplitRangesInSpan(ctx context.Context, kvDB *kv.DB, span roachpb.Span) error { - ranges, err := kvclient.ScanMetaKVs(ctx, kvDB.NewTxn(ctx, "unsplit-ranges-in-span"), span) - if err != nil { - return err - } - for _, r := range ranges { - var desc roachpb.RangeDescriptor - if err := r.ValueProto(&desc); err != nil { - return err - } - - if !span.ContainsKey(desc.StartKey.AsRawKey()) { - continue - } - - if !desc.GetStickyBit().IsEmpty() { - // Swallow "key is not the start of a range" errors because it would mean - // that the sticky bit was removed and merged concurrently. DROP TABLE - // should not fail because of this. - if err := kvDB.AdminUnsplit(ctx, desc.StartKey); err != nil && - !strings.Contains(err.Error(), "is not the start of a range") { - return err - } - } - } - - return nil -} - -// unsplitRangesForTable unsplit any manually split ranges within the tablespan. -// TODO(Chengxiong): Remove this function in 22.2 -func (p *planner) unsplitRangesForTable(ctx context.Context, tableDesc *tabledesc.Mutable) error { - // Gate this on being the system tenant because secondary tenants aren't - // allowed to scan the meta ranges directly. - if !p.ExecCfg().Codec.ForSystemTenant() { - return nil - } - - span := tableDesc.TableSpan(p.ExecCfg().Codec) - return UnsplitRangesInSpan(ctx, p.execCfg.DB, span) -} - // drainName when set implies that the name needs to go through the draining // names process. This parameter is always passed in as true except from // TRUNCATE which directly deletes the old name to id map and doesn't need @@ -394,16 +346,6 @@ func (p *planner) initiateDropTable( tableDesc.DropTime = timeutil.Now().UnixNano() } - // TODO(Chengxiong): Remove this range unsplitting in 22.2 - st := p.EvalContext().Settings - if !st.Version.IsActive(ctx, clusterversion.UnsplitRangesInAsyncGCJobs) { - // Unsplit all manually split ranges in the table so they can be - // automatically merged by the merge queue. - if err := p.unsplitRangesForTable(ctx, tableDesc); err != nil { - return err - } - } - // Actually mark table descriptor as dropped. tableDesc.SetDropped() diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index b19673345fcc..8b48edd7a472 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index 7098efd62726..85f128f41c35 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -13,10 +13,13 @@ package gcjob import ( "context" "math" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -93,6 +96,36 @@ func performGC( return nil } +// unsplitRangesInSpan unsplits any manually splits ranges within a span. +func unsplitRangesInSpan(ctx context.Context, kvDB *kv.DB, span roachpb.Span) error { + ranges, err := kvclient.ScanMetaKVs(ctx, kvDB.NewTxn(ctx, "unsplit-ranges-in-span"), span) + if err != nil { + return err + } + for _, r := range ranges { + var desc roachpb.RangeDescriptor + if err := r.ValueProto(&desc); err != nil { + return err + } + + if !span.ContainsKey(desc.StartKey.AsRawKey()) { + continue + } + + if !desc.GetStickyBit().IsEmpty() { + // Swallow "key is not the start of a range" errors because it would mean + // that the sticky bit was removed and merged concurrently. DROP TABLE + // should not fail because of this. + if err := kvDB.AdminUnsplit(ctx, desc.StartKey); err != nil && + !strings.Contains(err.Error(), "is not the start of a range") { + return err + } + } + } + + return nil +} + func unsplitRangesForTables( ctx context.Context, execCfg *sql.ExecutorConfig, @@ -108,7 +141,7 @@ func unsplitRangesForTables( Key: startKey, EndKey: startKey.PrefixEnd(), } - if err := sql.UnsplitRangesInSpan(ctx, execCfg.DB, span); err != nil { + if err := unsplitRangesInSpan(ctx, execCfg.DB, span); err != nil { return err } } @@ -134,7 +167,7 @@ func unsplitRangesForIndexes( EndKey: startKey.PrefixEnd(), } - if err := sql.UnsplitRangesInSpan(ctx, execCfg.DB, idxSpan); err != nil { + if err := unsplitRangesInSpan(ctx, execCfg.DB, idxSpan); err != nil { return err } } diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index 6ecf80dc8568..870757f8043c 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -55,7 +55,7 @@ Query {"String": "DROP INDEX t_x_idx"} until crdb_only CommandComplete ---- -{"Severity":"NOTICE","SeverityUnlocalized":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":576,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","SeverityUnlocalized":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":538,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 0be5cc7eb950..fc8a5970f119 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -14,7 +14,6 @@ import ( "context" "math/rand" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -253,16 +252,6 @@ func (p *planner) truncateTable(ctx context.Context, id descpb.ID, jobDesc strin return err } - // TODO(Chengxiong): remove this block in 22.2 - st := p.EvalContext().Settings - if !st.Version.IsActive(ctx, clusterversion.UnsplitRangesInAsyncGCJobs) { - // Unsplit all manually split ranges in the table so they can be - // automatically merged by the merge queue. - if err := p.unsplitRangesForTable(ctx, tableDesc); err != nil { - return err - } - } - oldIndexIDs := make([]descpb.IndexID, len(oldIndexes)) for i := range oldIndexIDs { oldIndexIDs[i] = oldIndexes[i].ID diff --git a/pkg/sql/unsplit_range_test.go b/pkg/sql/unsplit_range_test.go index 69897da2a171..e380bf635ab9 100644 --- a/pkg/sql/unsplit_range_test.go +++ b/pkg/sql/unsplit_range_test.go @@ -17,13 +17,11 @@ import ( "errors" "testing" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" @@ -157,12 +155,7 @@ func rangeIsManuallySplit( } // Test that manually split ranges get unsplit when dropping a -// table/database/index or truncating a table. It verifies that the logic is -// working on both the old (before version `UnsplitRangesInAsyncGCJobs`) and new -// (from versiom `UnsplitRangesInAsyncGCJobs`) pathes. -// TODO(Chengxiong): remove test for test cases with binary version -// "clusterversion.UnsplitRangesInAsyncGCJobs - 1" and update this comment in -// 22.2. +// table/database/index or truncating a table. func TestUnsplitRanges(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -175,9 +168,7 @@ func TestUnsplitRanges(t *testing.T) { // // For each testcase we execute a statement represented by the testcase's // query string to either drop the table/database/index or truncate the table. - // The Unsplit logic is then triggered through either the new or old code - // path, determined by the "binaryVersion" assigned. For the new code path, - // ranges are not unsplit until the gc job is kicked off. So we wait until it + // Ranges are not unsplit until the gc job is kicked off. So we wait until it // succeeds. // // In the end, expected results are verified based on the testcase's settings. @@ -185,9 +176,8 @@ func TestUnsplitRanges(t *testing.T) { // Then we checked if there are still split ranges start with key "splitKey" // and on the table in overall. type testCase struct { - name string - query string - binaryVersion clusterversion.Key + name string + query string // allKeyCntAfterGC is the expected keys count of the whole table. // For example, we expect it to be 0 when dropping a table because all data // should be gone, while 2*numRows is expected if only index "foo" is being @@ -241,28 +231,9 @@ func TestUnsplitRanges(t *testing.T) { } testCases := []testCase{ - { - name: "drop-table-unsplit-sync", - query: "DROP TABLE t.test1", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs - 1, - allKeyCntAfterGC: 0, - hasSplitOnTableAfterGC: false, - hasSplitOnKeyAfterGC: false, - gcSucceedFunc: tableDropSucceed, - }, { name: "drop-table-unsplit-async", query: "DROP TABLE t.test1", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs, - allKeyCntAfterGC: 0, - hasSplitOnTableAfterGC: false, - hasSplitOnKeyAfterGC: false, - gcSucceedFunc: tableDropSucceed, - }, - { - name: "drop-database-unsplit-sync", - query: "DROP DATABASE t", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs - 1, allKeyCntAfterGC: 0, hasSplitOnTableAfterGC: false, hasSplitOnKeyAfterGC: false, @@ -271,43 +242,22 @@ func TestUnsplitRanges(t *testing.T) { { name: "drop-database-unsplit-async", query: "DROP DATABASE t", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs, allKeyCntAfterGC: 0, hasSplitOnTableAfterGC: false, hasSplitOnKeyAfterGC: false, gcSucceedFunc: tableDropSucceed, }, - { - name: "truncate-table-unsplit-sync", - query: "TRUNCATE TABLE t.test1", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs - 1, - allKeyCntAfterGC: 0, - hasSplitOnTableAfterGC: true, // It's true since we copy split points. - hasSplitOnKeyAfterGC: false, - gcSucceedFunc: tableTruncateSucceed, - }, { name: "truncate-table-unsplit-async", query: "TRUNCATE TABLE t.test1", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs, allKeyCntAfterGC: 0, hasSplitOnTableAfterGC: true, // It's true since we copy split points. hasSplitOnKeyAfterGC: false, gcSucceedFunc: tableTruncateSucceed, }, - { - name: "drop-index-unsplit-sync", - query: "DROP INDEX t.test1@foo", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs - 1, - allKeyCntAfterGC: numRows * 2, - hasSplitOnTableAfterGC: true, // It's true since we only unsplit ranges of index foo - hasSplitOnKeyAfterGC: true, - gcSucceedFunc: indexDropSucceed, - }, { name: "drop-index-unsplit-async", query: "DROP INDEX t.test1@foo", - binaryVersion: clusterversion.UnsplitRangesInAsyncGCJobs, allKeyCntAfterGC: numRows * 2, hasSplitOnTableAfterGC: true, // It's true since we only unsplit ranges of index foo hasSplitOnKeyAfterGC: true, @@ -318,11 +268,6 @@ func TestUnsplitRanges(t *testing.T) { ctx := context.Background() run := func(t *testing.T, tc testCase) { params, _ := tests.CreateTestServerParams() - // Override binary version to be older. - params.Knobs.Server = &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey(tc.binaryVersion), - } params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() defer gcjob.SetSmallMaxGCIntervalForTest()() From 01f4daac9ecd93543752d6caf96893af9b82e022 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 17:15:18 -0700 Subject: [PATCH 11/16] clusterversion: remove noop version gates This commit removes the following version gates that are noops (meaning no code actually uses them): - ValidateGrantOption - ScanWholeRows - DisableSystemConfigGossipTrigger - BackupResolutionInJob - DateStyleIntervalStyleCastRewrite - ForecastStats. Release note: None --- pkg/clusterversion/cockroach_versions.go | 42 --------- pkg/clusterversion/key_string.go | 110 +++++++++++------------ pkg/storage/min_version_test.go | 2 +- 3 files changed, 53 insertions(+), 101 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 9e3f3b7abc6c..70678bd66907 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -164,9 +164,6 @@ const ( // Start22_1 demarcates work towards CockroachDB v22.1. Start22_1 - // ValidateGrantOption checks whether the current user granting privileges to - // another user holds the grant option for those privileges - ValidateGrantOption // PebbleFormatBlockPropertyCollector switches to a backwards incompatible // Pebble version that provides block property collectors that can be used // for fine-grained time bound iteration. See @@ -205,9 +202,6 @@ const ( // EnableSpanConfigStore enables the use of the span configs infrastructure // in KV. EnableSpanConfigStore - // ScanWholeRows is the version at which the Header.WholeRowsOfSize parameter - // was introduced, preventing limited scans from returning partial rows. - ScanWholeRows // SCRAM authentication is available. SCRAMAuthentication // UnsafeLossOfQuorumRecoveryRangeLog adds a new value to RangeLogEventReason @@ -247,10 +241,6 @@ const ( // engine running at the required format major version, as do all other nodes // in the cluster. EnablePebbleFormatVersionBlockProperties - // DisableSystemConfigGossipTrigger is a follow-up to EnableSpanConfigStore - // to disable the data propagation mechanism it and the entire spanconfig - // infrastructure obviates. - DisableSystemConfigGossipTrigger // MVCCIndexBackfiller supports MVCC-compliant index // backfillers via a new BACKFILLING index state, delete // preserving temporary indexes, and a post-backfill merging @@ -259,9 +249,6 @@ const ( // EnableLeaseHolderRemoval enables removing a leaseholder and transferring the lease // during joint configuration, including to VOTER_INCOMING replicas. EnableLeaseHolderRemoval - // BackupResolutionInJob defaults to resolving backup destinations during the - // execution of a backup job rather than during planning. - BackupResolutionInJob // LooselyCoupledRaftLogTruncation allows the cluster to reduce the coupling // for raft log truncation, by allowing each replica to treat a truncation // proposal as an upper bound on what should be truncated. @@ -284,9 +271,6 @@ const ( // IncrementalBackupSubdir enables backing up new incremental backups to a // dedicated subdirectory, to make it easier to apply a different ttl. IncrementalBackupSubdir - // DateStyleIntervalStyleCastRewrite rewrites cast that cause inconsistencies - // when DateStyle/IntervalStyle is enabled. - DateStyleIntervalStyleCastRewrite // EnableNewStoreRebalancer enables the new store rebalancer introduced in // 22.1. EnableNewStoreRebalancer @@ -296,8 +280,6 @@ const ( // AutoStatsTableSettings is the version where we allow auto stats related // table settings. AutoStatsTableSettings - // ForecastStats enables statistics forecasting per table. - ForecastStats // SuperRegions enables the usage on super regions. SuperRegions // EnableNewChangefeedOptions enables the usage of new changefeed options @@ -400,10 +382,6 @@ var versionsSingleton = keyedVersions{ Key: Start22_1, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2}, }, - { - Key: ValidateGrantOption, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 22}, - }, { Key: PebbleFormatBlockPropertyCollector, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 24}, @@ -440,10 +418,6 @@ var versionsSingleton = keyedVersions{ Key: EnableSpanConfigStore, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 40}, }, - { - Key: ScanWholeRows, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 42}, - }, { Key: SCRAMAuthentication, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 44}, @@ -484,10 +458,6 @@ var versionsSingleton = keyedVersions{ Key: EnablePebbleFormatVersionBlockProperties, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 64}, }, - { - Key: DisableSystemConfigGossipTrigger, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 66}, - }, { Key: MVCCIndexBackfiller, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 68}, @@ -498,10 +468,6 @@ var versionsSingleton = keyedVersions{ }, // Internal: 72 was reverted (EnsurePebbleFormatVersionRangeKeys) // Internal: 74 was reverted (EnablePebbleFormatVersionRangeKeys) - { - Key: BackupResolutionInJob, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 76}, - }, // Internal: 78 was reverted (ExperimentalMVCCRangeTombstones) { Key: LooselyCoupledRaftLogTruncation, @@ -531,10 +497,6 @@ var versionsSingleton = keyedVersions{ Key: IncrementalBackupSubdir, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 92}, }, - { - Key: DateStyleIntervalStyleCastRewrite, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 94}, - }, { Key: EnableNewStoreRebalancer, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 96}, @@ -547,10 +509,6 @@ var versionsSingleton = keyedVersions{ Key: AutoStatsTableSettings, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 100}, }, - { - Key: ForecastStats, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 102}, - }, { Key: SuperRegions, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 104}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 67652caf2eee..2a7364dd1a37 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -10,67 +10,61 @@ func _() { var x [1]struct{} _ = x[V21_2-0] _ = x[Start22_1-1] - _ = x[ValidateGrantOption-2] - _ = x[PebbleFormatBlockPropertyCollector-3] - _ = x[ProbeRequest-4] - _ = x[SelectRPCsTakeTracingInfoInband-5] - _ = x[PreSeedTenantSpanConfigs-6] - _ = x[SeedTenantSpanConfigs-7] - _ = x[PublicSchemasWithDescriptors-8] - _ = x[EnsureSpanConfigReconciliation-9] - _ = x[EnsureSpanConfigSubscription-10] - _ = x[EnableSpanConfigStore-11] - _ = x[ScanWholeRows-12] - _ = x[SCRAMAuthentication-13] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-14] - _ = x[AlterSystemProtectedTimestampAddColumn-15] - _ = x[EnableProtectedTimestampsForTenant-16] - _ = x[DeleteCommentsWithDroppedIndexes-17] - _ = x[RemoveIncompatibleDatabasePrivileges-18] - _ = x[AddRaftAppliedIndexTermMigration-19] - _ = x[PostAddRaftAppliedIndexTermMigration-20] - _ = x[DontProposeWriteTimestampForLeaseTransfers-21] - _ = x[EnablePebbleFormatVersionBlockProperties-22] - _ = x[DisableSystemConfigGossipTrigger-23] - _ = x[MVCCIndexBackfiller-24] - _ = x[EnableLeaseHolderRemoval-25] - _ = x[BackupResolutionInJob-26] - _ = x[LooselyCoupledRaftLogTruncation-27] - _ = x[ChangefeedIdleness-28] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-29] - _ = x[EnableDeclarativeSchemaChanger-30] - _ = x[RowLevelTTL-31] - _ = x[PebbleFormatSplitUserKeysMarked-32] - _ = x[IncrementalBackupSubdir-33] - _ = x[DateStyleIntervalStyleCastRewrite-34] - _ = x[EnableNewStoreRebalancer-35] - _ = x[ClusterLocksVirtualTable-36] - _ = x[AutoStatsTableSettings-37] - _ = x[ForecastStats-38] - _ = x[SuperRegions-39] - _ = x[EnableNewChangefeedOptions-40] - _ = x[SpanCountTable-41] - _ = x[PreSeedSpanCountTable-42] - _ = x[SeedSpanCountTable-43] - _ = x[V22_1-44] - _ = x[Start22_2-45] - _ = x[LocalTimestamps-46] - _ = x[EnsurePebbleFormatVersionRangeKeys-47] - _ = x[EnablePebbleFormatVersionRangeKeys-48] - _ = x[TrigramInvertedIndexes-49] - _ = x[RemoveGrantPrivilege-50] - _ = x[MVCCRangeTombstones-51] - _ = x[UpgradeSequenceToBeReferencedByID-52] - _ = x[SampledStmtDiagReqs-53] - _ = x[AddSSTableTombstones-54] - _ = x[SystemPrivilegesTable-55] - _ = x[EnablePredicateProjectionChangefeed-56] - _ = x[AlterSystemSQLInstancesAddLocality-57] + _ = x[PebbleFormatBlockPropertyCollector-2] + _ = x[ProbeRequest-3] + _ = x[SelectRPCsTakeTracingInfoInband-4] + _ = x[PreSeedTenantSpanConfigs-5] + _ = x[SeedTenantSpanConfigs-6] + _ = x[PublicSchemasWithDescriptors-7] + _ = x[EnsureSpanConfigReconciliation-8] + _ = x[EnsureSpanConfigSubscription-9] + _ = x[EnableSpanConfigStore-10] + _ = x[SCRAMAuthentication-11] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-12] + _ = x[AlterSystemProtectedTimestampAddColumn-13] + _ = x[EnableProtectedTimestampsForTenant-14] + _ = x[DeleteCommentsWithDroppedIndexes-15] + _ = x[RemoveIncompatibleDatabasePrivileges-16] + _ = x[AddRaftAppliedIndexTermMigration-17] + _ = x[PostAddRaftAppliedIndexTermMigration-18] + _ = x[DontProposeWriteTimestampForLeaseTransfers-19] + _ = x[EnablePebbleFormatVersionBlockProperties-20] + _ = x[MVCCIndexBackfiller-21] + _ = x[EnableLeaseHolderRemoval-22] + _ = x[LooselyCoupledRaftLogTruncation-23] + _ = x[ChangefeedIdleness-24] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-25] + _ = x[EnableDeclarativeSchemaChanger-26] + _ = x[RowLevelTTL-27] + _ = x[PebbleFormatSplitUserKeysMarked-28] + _ = x[IncrementalBackupSubdir-29] + _ = x[EnableNewStoreRebalancer-30] + _ = x[ClusterLocksVirtualTable-31] + _ = x[AutoStatsTableSettings-32] + _ = x[SuperRegions-33] + _ = x[EnableNewChangefeedOptions-34] + _ = x[SpanCountTable-35] + _ = x[PreSeedSpanCountTable-36] + _ = x[SeedSpanCountTable-37] + _ = x[V22_1-38] + _ = x[Start22_2-39] + _ = x[LocalTimestamps-40] + _ = x[EnsurePebbleFormatVersionRangeKeys-41] + _ = x[EnablePebbleFormatVersionRangeKeys-42] + _ = x[TrigramInvertedIndexes-43] + _ = x[RemoveGrantPrivilege-44] + _ = x[MVCCRangeTombstones-45] + _ = x[UpgradeSequenceToBeReferencedByID-46] + _ = x[SampledStmtDiagReqs-47] + _ = x[AddSSTableTombstones-48] + _ = x[SystemPrivilegesTable-49] + _ = x[EnablePredicateProjectionChangefeed-50] + _ = x[AlterSystemSQLInstancesAddLocality-51] } -const _Key_name = "V21_2Start22_1ValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 33, 67, 79, 110, 134, 155, 183, 213, 241, 262, 275, 294, 328, 366, 400, 432, 468, 500, 536, 578, 618, 650, 669, 693, 714, 745, 763, 804, 834, 845, 876, 899, 932, 956, 980, 1002, 1015, 1027, 1053, 1067, 1088, 1106, 1111, 1120, 1135, 1169, 1203, 1225, 1245, 1264, 1297, 1316, 1336, 1357, 1392, 1426} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 91, 115, 136, 164, 194, 222, 243, 262, 296, 334, 368, 400, 436, 468, 504, 546, 586, 605, 629, 660, 678, 719, 749, 760, 791, 814, 838, 862, 884, 896, 922, 936, 957, 975, 980, 989, 1004, 1038, 1072, 1094, 1114, 1133, 1166, 1185, 1205, 1226, 1261, 1295} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/storage/min_version_test.go b/pkg/storage/min_version_test.go index 9697dc30feb1..a92e2fcb0924 100644 --- a/pkg/storage/min_version_test.go +++ b/pkg/storage/min_version_test.go @@ -109,7 +109,7 @@ func TestSetMinVersion(t *testing.T) { // Advancing the store cluster version to another cluster version // that does not advance the Pebble format major version should // leave the format major version unchanged. - err = p.SetMinVersion(clusterversion.ByKey(clusterversion.ValidateGrantOption)) + err = p.SetMinVersion(clusterversion.ByKey(clusterversion.Start22_1)) require.NoError(t, err) require.Equal(t, pebble.FormatSetWithDelete, p.db.FormatMajorVersion()) From 2edb4fa3564a3f0fc5499eac52d6b4310d26c80e Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 17:55:31 -0700 Subject: [PATCH 12/16] rpc: remove SelectRPCsTakeTracingInfoInband version gate Release note: None --- pkg/clusterversion/cockroach_versions.go | 12 --- pkg/clusterversion/key_string.go | 99 +++++++++---------- pkg/rpc/context.go | 7 +- .../grpcinterceptor/grpc_interceptor.go | 18 +--- .../grpcinterceptor/grpc_interceptor_test.go | 4 +- 5 files changed, 56 insertions(+), 84 deletions(-) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 70678bd66907..f81f74242515 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -173,14 +173,6 @@ const ( // This version must be active before any ProbeRequest is issued on the // cluster. ProbeRequest - // SelectRPCsTakeTracingInfoInband switches the way tracing works for a couple - // of common RPCs. Tracing information for these select RPCs is no longer - // marshaled from the client to the server as gRPC metadata, and the gRPC - // server interceptor is no longer in charge of transparently creating server - // spans. Instead, trace information is carried by the respective request - // protos (the client is responsible for filling it in explicitly), and the - // server-side handler is responsible for opening a span manually. - SelectRPCsTakeTracingInfoInband // PreSeedTenantSpanConfigs precedes SeedTenantSpanConfigs, and enables the // creation of initial span config records for newly created tenants. PreSeedTenantSpanConfigs @@ -390,10 +382,6 @@ var versionsSingleton = keyedVersions{ Key: ProbeRequest, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 26}, }, - { - Key: SelectRPCsTakeTracingInfoInband, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 28}, - }, { Key: PreSeedTenantSpanConfigs, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 30}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 2a7364dd1a37..e5787f5b92c3 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -12,59 +12,58 @@ func _() { _ = x[Start22_1-1] _ = x[PebbleFormatBlockPropertyCollector-2] _ = x[ProbeRequest-3] - _ = x[SelectRPCsTakeTracingInfoInband-4] - _ = x[PreSeedTenantSpanConfigs-5] - _ = x[SeedTenantSpanConfigs-6] - _ = x[PublicSchemasWithDescriptors-7] - _ = x[EnsureSpanConfigReconciliation-8] - _ = x[EnsureSpanConfigSubscription-9] - _ = x[EnableSpanConfigStore-10] - _ = x[SCRAMAuthentication-11] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-12] - _ = x[AlterSystemProtectedTimestampAddColumn-13] - _ = x[EnableProtectedTimestampsForTenant-14] - _ = x[DeleteCommentsWithDroppedIndexes-15] - _ = x[RemoveIncompatibleDatabasePrivileges-16] - _ = x[AddRaftAppliedIndexTermMigration-17] - _ = x[PostAddRaftAppliedIndexTermMigration-18] - _ = x[DontProposeWriteTimestampForLeaseTransfers-19] - _ = x[EnablePebbleFormatVersionBlockProperties-20] - _ = x[MVCCIndexBackfiller-21] - _ = x[EnableLeaseHolderRemoval-22] - _ = x[LooselyCoupledRaftLogTruncation-23] - _ = x[ChangefeedIdleness-24] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-25] - _ = x[EnableDeclarativeSchemaChanger-26] - _ = x[RowLevelTTL-27] - _ = x[PebbleFormatSplitUserKeysMarked-28] - _ = x[IncrementalBackupSubdir-29] - _ = x[EnableNewStoreRebalancer-30] - _ = x[ClusterLocksVirtualTable-31] - _ = x[AutoStatsTableSettings-32] - _ = x[SuperRegions-33] - _ = x[EnableNewChangefeedOptions-34] - _ = x[SpanCountTable-35] - _ = x[PreSeedSpanCountTable-36] - _ = x[SeedSpanCountTable-37] - _ = x[V22_1-38] - _ = x[Start22_2-39] - _ = x[LocalTimestamps-40] - _ = x[EnsurePebbleFormatVersionRangeKeys-41] - _ = x[EnablePebbleFormatVersionRangeKeys-42] - _ = x[TrigramInvertedIndexes-43] - _ = x[RemoveGrantPrivilege-44] - _ = x[MVCCRangeTombstones-45] - _ = x[UpgradeSequenceToBeReferencedByID-46] - _ = x[SampledStmtDiagReqs-47] - _ = x[AddSSTableTombstones-48] - _ = x[SystemPrivilegesTable-49] - _ = x[EnablePredicateProjectionChangefeed-50] - _ = x[AlterSystemSQLInstancesAddLocality-51] + _ = x[PreSeedTenantSpanConfigs-4] + _ = x[SeedTenantSpanConfigs-5] + _ = x[PublicSchemasWithDescriptors-6] + _ = x[EnsureSpanConfigReconciliation-7] + _ = x[EnsureSpanConfigSubscription-8] + _ = x[EnableSpanConfigStore-9] + _ = x[SCRAMAuthentication-10] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-11] + _ = x[AlterSystemProtectedTimestampAddColumn-12] + _ = x[EnableProtectedTimestampsForTenant-13] + _ = x[DeleteCommentsWithDroppedIndexes-14] + _ = x[RemoveIncompatibleDatabasePrivileges-15] + _ = x[AddRaftAppliedIndexTermMigration-16] + _ = x[PostAddRaftAppliedIndexTermMigration-17] + _ = x[DontProposeWriteTimestampForLeaseTransfers-18] + _ = x[EnablePebbleFormatVersionBlockProperties-19] + _ = x[MVCCIndexBackfiller-20] + _ = x[EnableLeaseHolderRemoval-21] + _ = x[LooselyCoupledRaftLogTruncation-22] + _ = x[ChangefeedIdleness-23] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-24] + _ = x[EnableDeclarativeSchemaChanger-25] + _ = x[RowLevelTTL-26] + _ = x[PebbleFormatSplitUserKeysMarked-27] + _ = x[IncrementalBackupSubdir-28] + _ = x[EnableNewStoreRebalancer-29] + _ = x[ClusterLocksVirtualTable-30] + _ = x[AutoStatsTableSettings-31] + _ = x[SuperRegions-32] + _ = x[EnableNewChangefeedOptions-33] + _ = x[SpanCountTable-34] + _ = x[PreSeedSpanCountTable-35] + _ = x[SeedSpanCountTable-36] + _ = x[V22_1-37] + _ = x[Start22_2-38] + _ = x[LocalTimestamps-39] + _ = x[EnsurePebbleFormatVersionRangeKeys-40] + _ = x[EnablePebbleFormatVersionRangeKeys-41] + _ = x[TrigramInvertedIndexes-42] + _ = x[RemoveGrantPrivilege-43] + _ = x[MVCCRangeTombstones-44] + _ = x[UpgradeSequenceToBeReferencedByID-45] + _ = x[SampledStmtDiagReqs-46] + _ = x[AddSSTableTombstones-47] + _ = x[SystemPrivilegesTable-48] + _ = x[EnablePredicateProjectionChangefeed-49] + _ = x[AlterSystemSQLInstancesAddLocality-50] } -const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 48, 60, 91, 115, 136, 164, 194, 222, 243, 262, 296, 334, 368, 400, 436, 468, 504, 546, 586, 605, 629, 660, 678, 719, 749, 760, 791, 814, 838, 862, 884, 896, 922, 936, 957, 975, 980, 989, 1004, 1038, 1072, 1094, 1114, 1133, 1166, 1185, 1205, 1226, 1261, 1295} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 84, 105, 133, 163, 191, 212, 231, 265, 303, 337, 369, 405, 437, 473, 515, 555, 574, 598, 629, 647, 688, 718, 729, 760, 783, 807, 831, 853, 865, 891, 905, 926, 944, 949, 958, 973, 1007, 1041, 1063, 1083, 1102, 1135, 1154, 1174, 1195, 1230, 1264} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 7a4e45c57926..26afd43e82b4 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -26,7 +26,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -629,19 +628,15 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { tagger := func(span *tracing.Span) { span.SetTag("node", attribute.IntValue(int(rpcCtx.NodeID.Get()))) } - compatMode := func(reqCtx context.Context) bool { - return !rpcCtx.ContextOptions.Settings.Version.IsActive(reqCtx, clusterversion.SelectRPCsTakeTracingInfoInband) - } if rpcCtx.ClientOnly { // client-only RPC contexts don't have a node ID to report nor a // cluster version to check against. tagger = func(span *tracing.Span) {} - compatMode = func(_ context.Context) bool { return false } } rpcCtx.clientUnaryInterceptors = append(rpcCtx.clientUnaryInterceptors, - grpcinterceptor.ClientInterceptor(tracer, tagger, compatMode)) + grpcinterceptor.ClientInterceptor(tracer, tagger)) rpcCtx.clientStreamInterceptors = append(rpcCtx.clientStreamInterceptors, grpcinterceptor.StreamClientInterceptor(tracer, tagger)) } diff --git a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go index 0c913ef57635..0804cba5674c 100644 --- a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go +++ b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go @@ -49,8 +49,8 @@ func setGRPCErrorTag(sp *tracing.Span, err error) { // BatchMethodName is the method name of Internal.Batch RPC. const BatchMethodName = "/cockroach.roachpb.Internal/Batch" -// SendKVBatchMethodName is the method name for adminServer.SendKVBatch. -const SendKVBatchMethodName = "/cockroach.server.serverpb.Admin/SendKVBatch" +// sendKVBatchMethodName is the method name for adminServer.SendKVBatch. +const sendKVBatchMethodName = "/cockroach.server.serverpb.Admin/SendKVBatch" // SetupFlowMethodName is the method name of DistSQL.SetupFlow RPC. const SetupFlowMethodName = "/cockroach.sql.distsqlrun.DistSQL/SetupFlow" @@ -64,7 +64,7 @@ const flowStreamMethodName = "/cockroach.sql.distsqlrun.DistSQL/FlowStream" // tracing because it's not worth it. func methodExcludedFromTracing(method string) bool { return method == BatchMethodName || - method == SendKVBatchMethodName || + method == sendKVBatchMethodName || method == SetupFlowMethodName || method == flowStreamMethodName } @@ -205,16 +205,8 @@ func injectSpanMeta( // metadata; they will also look in the context.Context for an active // in-process parent Span and establish a ChildOf relationship if such a parent // Span could be found. -// -// compatibilityMode is a callback that will be used to check whether the node -// (still) needs compatibility with 21.2. If it doesn't, then a more performant -// trace propagation mechanism is used. The compatibility check is built as a -// callback rather than directly checking the cluster version because this -// tracing package cannot use cluster settings. func ClientInterceptor( - tracer *tracing.Tracer, - init func(*tracing.Span), - compatibilityMode func(ctx context.Context) bool, + tracer *tracing.Tracer, init func(*tracing.Span), ) grpc.UnaryClientInterceptor { if init == nil { init = func(*tracing.Span) {} @@ -247,7 +239,7 @@ func ClientInterceptor( // For most RPCs we pass along tracing info as gRPC metadata. Some select // RPCs carry the tracing in the request protos, which is more efficient. - if compatibilityMode(ctx) || !methodExcludedFromTracing(method) { + if !methodExcludedFromTracing(method) { ctx = injectSpanMeta(ctx, tracer, clientSpan) } var err error diff --git a/pkg/util/tracing/grpcinterceptor/grpc_interceptor_test.go b/pkg/util/tracing/grpcinterceptor/grpc_interceptor_test.go index f695f3be3de3..b225a8b8d6d1 100644 --- a/pkg/util/tracing/grpcinterceptor/grpc_interceptor_test.go +++ b/pkg/util/tracing/grpcinterceptor/grpc_interceptor_test.go @@ -234,9 +234,7 @@ func TestGRPCInterceptors(t *testing.T) { conn, err := grpc.DialContext(bgCtx, ln.Addr().String(), //lint:ignore SA1019 grpc.WithInsecure is deprecated grpc.WithInsecure(), - grpc.WithUnaryInterceptor(grpcinterceptor.ClientInterceptor(tr, nil, /* init */ - func(_ context.Context) bool { return false }, /* compatibilityMode */ - )), + grpc.WithUnaryInterceptor(grpcinterceptor.ClientInterceptor(tr, nil /* init */)), grpc.WithStreamInterceptor(grpcinterceptor.StreamClientInterceptor(tr, nil /* init */)), ) require.NoError(t, err) From a2ce495ee932c1a9cd5fcfd1e1336daf4c5a6694 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Jul 2022 18:00:25 -0700 Subject: [PATCH 13/16] upgrades: remove seed tenant span config migration Release note: None --- pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel | 2 - .../seed_span_counts_external_test.go | 1 + .../seed_tenant_span_configs_external_test.go | 220 ------------------ pkg/clusterversion/cockroach_versions.go | 17 -- pkg/clusterversion/key_string.go | 96 ++++---- pkg/sql/tenant.go | 8 - pkg/upgrade/upgrades/BUILD.bazel | 2 - .../upgrades/seed_tenant_span_configs.go | 97 -------- pkg/upgrade/upgrades/upgrades.go | 6 - 9 files changed, 48 insertions(+), 401 deletions(-) delete mode 100644 pkg/ccl/upgradeccl/upgradessccl/seed_tenant_span_configs_external_test.go delete mode 100644 pkg/upgrade/upgrades/seed_tenant_span_configs.go diff --git a/pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel b/pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel index 22c18a0ab9ef..51d0a7d18122 100644 --- a/pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel +++ b/pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel @@ -6,13 +6,11 @@ go_test( srcs = [ "main_test.go", "seed_span_counts_external_test.go", - "seed_tenant_span_configs_external_test.go", ], deps = [ "//pkg/base", "//pkg/ccl/kvccl/kvtenantccl", "//pkg/clusterversion", - "//pkg/keys", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/upgradeccl/upgradessccl/seed_span_counts_external_test.go b/pkg/ccl/upgradeccl/upgradessccl/seed_span_counts_external_test.go index 38dc65b15f72..bfc49ae63ce1 100644 --- a/pkg/ccl/upgradeccl/upgradessccl/seed_span_counts_external_test.go +++ b/pkg/ccl/upgradeccl/upgradessccl/seed_span_counts_external_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" diff --git a/pkg/ccl/upgradeccl/upgradessccl/seed_tenant_span_configs_external_test.go b/pkg/ccl/upgradeccl/upgradessccl/seed_tenant_span_configs_external_test.go deleted file mode 100644 index 73dc0f285f4d..000000000000 --- a/pkg/ccl/upgradeccl/upgradessccl/seed_tenant_span_configs_external_test.go +++ /dev/null @@ -1,220 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package upgradessccl_test - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -// TestPreSeedSpanConfigsWrittenWhenActive tests that seed span configs are -// written to for fresh tenants if the cluster version that introduced it is -// active. -func TestPreSeedSpanConfigsWrittenWhenActive(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.PreSeedTenantSpanConfigs, - ), - }, - }, - }, - }) - - defer tc.Stopper().Stop(ctx) - ts := tc.Server(0) - - tenantID := roachpb.MakeTenantID(10) - _, err := ts.StartTenant(ctx, base.TestTenantArgs{ - TenantID: tenantID, - TestingKnobs: base.TestingKnobs{ - SpanConfig: &spanconfig.TestingKnobs{ - // Disable the tenant's span config reconciliation process, - // it'll muck with the tenant's span configs that we check - // below. - ManagerDisableJobCreation: true, - }, - }, - }) - require.NoError(t, err) - - scKVAccessor := ts.SpanConfigKVAccessor().(spanconfig.KVAccessor) - tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()} - tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} - - { - records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ - spanconfig.MakeTargetFromSpan(tenantSpan), - }) - require.NoError(t, err) - require.Len(t, records, 1) - require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan) - } -} - -// TestSeedTenantSpanConfigs tests that the upgrade installs relevant seed -// span configs for existing secondary tenants. -func TestSeedTenantSpanConfigs(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.PreSeedTenantSpanConfigs - 1, - ), - }, - }, - }, - }) - - defer tc.Stopper().Stop(ctx) - ts := tc.Server(0) - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - scKVAccessor := ts.SpanConfigKVAccessor().(spanconfig.KVAccessor) - - tenantID := roachpb.MakeTenantID(10) - tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantTarget := spanconfig.MakeTargetFromSpan( - roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()}, - ) - tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} - { - _, err := ts.StartTenant(ctx, base.TestTenantArgs{ - TenantID: tenantID, - TestingKnobs: base.TestingKnobs{ - SpanConfig: &spanconfig.TestingKnobs{ - // Disable the tenant's span config reconciliation process, - // it'll muck with the tenant's span configs that we check - // below. - ManagerDisableJobCreation: true, - }, - }, - }) - require.NoError(t, err) - } - - { // Ensure that no span config records are to be found - records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ - tenantTarget, - }) - require.NoError(t, err) - require.Empty(t, records) - } - - tdb.Exec(t, - "SET CLUSTER SETTING version = $1", - clusterversion.ByKey(clusterversion.SeedTenantSpanConfigs).String(), - ) - - { // Ensure that the tenant now has a span config record. - records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ - tenantTarget, - }) - require.NoError(t, err) - require.Len(t, records, 1) - require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan) - } -} - -// TestSeedTenantSpanConfigsWithExistingEntry tests that the upgrade ignores -// tenants with existing span config records. -func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.PreSeedTenantSpanConfigs, - ), - }, - }, - }, - }) - - defer tc.Stopper().Stop(ctx) - ts := tc.Server(0) - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - scKVAccessor := ts.SpanConfigKVAccessor().(spanconfig.KVAccessor) - - tenantID := roachpb.MakeTenantID(10) - tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantTarget := spanconfig.MakeTargetFromSpan( - roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.PrefixEnd()}, - ) - tenantSeedSpan := roachpb.Span{Key: tenantPrefix, EndKey: tenantPrefix.Next()} - { - _, err := ts.StartTenant(ctx, base.TestTenantArgs{ - TenantID: tenantID, - TestingKnobs: base.TestingKnobs{ - SpanConfig: &spanconfig.TestingKnobs{ - // Disable the tenant's span config reconciliation process, - // it'll muck with the tenant's span configs that we check - // below. - ManagerDisableJobCreation: true, - }, - }, - }) - require.NoError(t, err) - } - - { // Ensure that the tenant already has a span config record. - records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ - tenantTarget, - }) - require.NoError(t, err) - require.Len(t, records, 1) - require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan) - } - - // Ensure the cluster version bump goes through successfully. - tdb.Exec(t, - "SET CLUSTER SETTING version = $1", - clusterversion.ByKey(clusterversion.SeedTenantSpanConfigs).String(), - ) - - { // Ensure that the tenant's span config record stay as it was. - records, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{ - tenantTarget, - }) - require.NoError(t, err) - require.Len(t, records, 1) - require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan) - } -} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index f81f74242515..4db9b95a8ccb 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -173,15 +173,6 @@ const ( // This version must be active before any ProbeRequest is issued on the // cluster. ProbeRequest - // PreSeedTenantSpanConfigs precedes SeedTenantSpanConfigs, and enables the - // creation of initial span config records for newly created tenants. - PreSeedTenantSpanConfigs - // SeedTenantSpanConfigs populates system.span_configurations with seed - // data for secondary tenants. This state is what ensures that we always - // split on tenant boundaries when using the span configs infrastructure. - // This version comes with a migration to populate the same seed data - // for existing tenants. - SeedTenantSpanConfigs // PublicSchemasWithDescriptors backs public schemas with descriptors. PublicSchemasWithDescriptors // EnsureSpanConfigReconciliation ensures that the host tenant has run its @@ -382,14 +373,6 @@ var versionsSingleton = keyedVersions{ Key: ProbeRequest, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 26}, }, - { - Key: PreSeedTenantSpanConfigs, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 30}, - }, - { - Key: SeedTenantSpanConfigs, - Version: roachpb.Version{Major: 21, Minor: 2, Internal: 32}, - }, { Key: PublicSchemasWithDescriptors, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 34}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index e5787f5b92c3..70e586b01728 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -12,58 +12,56 @@ func _() { _ = x[Start22_1-1] _ = x[PebbleFormatBlockPropertyCollector-2] _ = x[ProbeRequest-3] - _ = x[PreSeedTenantSpanConfigs-4] - _ = x[SeedTenantSpanConfigs-5] - _ = x[PublicSchemasWithDescriptors-6] - _ = x[EnsureSpanConfigReconciliation-7] - _ = x[EnsureSpanConfigSubscription-8] - _ = x[EnableSpanConfigStore-9] - _ = x[SCRAMAuthentication-10] - _ = x[UnsafeLossOfQuorumRecoveryRangeLog-11] - _ = x[AlterSystemProtectedTimestampAddColumn-12] - _ = x[EnableProtectedTimestampsForTenant-13] - _ = x[DeleteCommentsWithDroppedIndexes-14] - _ = x[RemoveIncompatibleDatabasePrivileges-15] - _ = x[AddRaftAppliedIndexTermMigration-16] - _ = x[PostAddRaftAppliedIndexTermMigration-17] - _ = x[DontProposeWriteTimestampForLeaseTransfers-18] - _ = x[EnablePebbleFormatVersionBlockProperties-19] - _ = x[MVCCIndexBackfiller-20] - _ = x[EnableLeaseHolderRemoval-21] - _ = x[LooselyCoupledRaftLogTruncation-22] - _ = x[ChangefeedIdleness-23] - _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-24] - _ = x[EnableDeclarativeSchemaChanger-25] - _ = x[RowLevelTTL-26] - _ = x[PebbleFormatSplitUserKeysMarked-27] - _ = x[IncrementalBackupSubdir-28] - _ = x[EnableNewStoreRebalancer-29] - _ = x[ClusterLocksVirtualTable-30] - _ = x[AutoStatsTableSettings-31] - _ = x[SuperRegions-32] - _ = x[EnableNewChangefeedOptions-33] - _ = x[SpanCountTable-34] - _ = x[PreSeedSpanCountTable-35] - _ = x[SeedSpanCountTable-36] - _ = x[V22_1-37] - _ = x[Start22_2-38] - _ = x[LocalTimestamps-39] - _ = x[EnsurePebbleFormatVersionRangeKeys-40] - _ = x[EnablePebbleFormatVersionRangeKeys-41] - _ = x[TrigramInvertedIndexes-42] - _ = x[RemoveGrantPrivilege-43] - _ = x[MVCCRangeTombstones-44] - _ = x[UpgradeSequenceToBeReferencedByID-45] - _ = x[SampledStmtDiagReqs-46] - _ = x[AddSSTableTombstones-47] - _ = x[SystemPrivilegesTable-48] - _ = x[EnablePredicateProjectionChangefeed-49] - _ = x[AlterSystemSQLInstancesAddLocality-50] + _ = x[PublicSchemasWithDescriptors-4] + _ = x[EnsureSpanConfigReconciliation-5] + _ = x[EnsureSpanConfigSubscription-6] + _ = x[EnableSpanConfigStore-7] + _ = x[SCRAMAuthentication-8] + _ = x[UnsafeLossOfQuorumRecoveryRangeLog-9] + _ = x[AlterSystemProtectedTimestampAddColumn-10] + _ = x[EnableProtectedTimestampsForTenant-11] + _ = x[DeleteCommentsWithDroppedIndexes-12] + _ = x[RemoveIncompatibleDatabasePrivileges-13] + _ = x[AddRaftAppliedIndexTermMigration-14] + _ = x[PostAddRaftAppliedIndexTermMigration-15] + _ = x[DontProposeWriteTimestampForLeaseTransfers-16] + _ = x[EnablePebbleFormatVersionBlockProperties-17] + _ = x[MVCCIndexBackfiller-18] + _ = x[EnableLeaseHolderRemoval-19] + _ = x[LooselyCoupledRaftLogTruncation-20] + _ = x[ChangefeedIdleness-21] + _ = x[BackupDoesNotOverwriteLatestAndCheckpoint-22] + _ = x[EnableDeclarativeSchemaChanger-23] + _ = x[RowLevelTTL-24] + _ = x[PebbleFormatSplitUserKeysMarked-25] + _ = x[IncrementalBackupSubdir-26] + _ = x[EnableNewStoreRebalancer-27] + _ = x[ClusterLocksVirtualTable-28] + _ = x[AutoStatsTableSettings-29] + _ = x[SuperRegions-30] + _ = x[EnableNewChangefeedOptions-31] + _ = x[SpanCountTable-32] + _ = x[PreSeedSpanCountTable-33] + _ = x[SeedSpanCountTable-34] + _ = x[V22_1-35] + _ = x[Start22_2-36] + _ = x[LocalTimestamps-37] + _ = x[EnsurePebbleFormatVersionRangeKeys-38] + _ = x[EnablePebbleFormatVersionRangeKeys-39] + _ = x[TrigramInvertedIndexes-40] + _ = x[RemoveGrantPrivilege-41] + _ = x[MVCCRangeTombstones-42] + _ = x[UpgradeSequenceToBeReferencedByID-43] + _ = x[SampledStmtDiagReqs-44] + _ = x[AddSSTableTombstones-45] + _ = x[SystemPrivilegesTable-46] + _ = x[EnablePredicateProjectionChangefeed-47] + _ = x[AlterSystemSQLInstancesAddLocality-48] } -const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" -var _Key_index = [...]uint16{0, 5, 14, 48, 60, 84, 105, 133, 163, 191, 212, 231, 265, 303, 337, 369, 405, 437, 473, 515, 555, 574, 598, 629, 647, 688, 718, 729, 760, 783, 807, 831, 853, 865, 891, 905, 926, 944, 949, 958, 973, 1007, 1041, 1063, 1083, 1102, 1135, 1154, 1174, 1195, 1230, 1264} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 643, 673, 684, 715, 738, 762, 786, 808, 820, 846, 860, 881, 899, 904, 913, 928, 962, 996, 1018, 1038, 1057, 1090, 1109, 1129, 1150, 1185, 1219} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/tenant.go b/pkg/sql/tenant.go index 98b837f9a6f9..1a25630f6d78 100644 --- a/pkg/sql/tenant.go +++ b/pkg/sql/tenant.go @@ -126,10 +126,6 @@ func CreateTenantRecord( } } - if !execCfg.Settings.Version.IsActive(ctx, clusterversion.PreSeedTenantSpanConfigs) { - return nil - } - // Install a single key[1] span config at the start of tenant's keyspace; // elsewhere this ensures that we split on the tenant boundary. The subset // of entries with spans in the tenant keyspace are, henceforth, governed @@ -473,10 +469,6 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten return errors.Wrapf(err, "deleting tenant %d settings", info.ID) } - if !execCfg.Settings.Version.IsActive(ctx, clusterversion.PreSeedTenantSpanConfigs) { - return nil - } - // Clear out all span config records left over by the tenant. tenID := roachpb.MakeTenantID(info.ID) tenantPrefix := keys.MakeTenantPrefix(tenID) diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 002d285f667e..f49d6b4aa375 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -15,7 +15,6 @@ go_library( "remove_grant_migration.go", "sampled_stmt_diagnostics_requests.go", "schema_changes.go", - "seed_tenant_span_configs.go", "span_count_table.go", "system_privileges.go", "upgrade_sequence_to_be_referenced_by_ID.go", @@ -32,7 +31,6 @@ go_library( "//pkg/roachpb", "//pkg/security/username", "//pkg/server/serverpb", - "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/upgrade/upgrades/seed_tenant_span_configs.go b/pkg/upgrade/upgrades/seed_tenant_span_configs.go deleted file mode 100644 index 824d5f7f6977..000000000000 --- a/pkg/upgrade/upgrades/seed_tenant_span_configs.go +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2021 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 upgrades - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/upgrade" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" -) - -func seedTenantSpanConfigsMigration( - ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, -) error { - if !d.Codec.ForSystemTenant() { - return nil - } - - return d.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - const getTenantIDsQuery = `SELECT id from system.tenants` - it, err := d.InternalExecutor.QueryIteratorEx(ctx, "get-tenant-ids", txn, - sessiondata.NodeUserSessionDataOverride, getTenantIDsQuery, - ) - if err != nil { - return errors.Wrap(err, "unable to fetch existing tenant IDs") - } - - var tenantIDs []roachpb.TenantID - var ok bool - for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - row := it.Cur() - tenantID := roachpb.MakeTenantID(uint64(tree.MustBeDInt(row[0]))) - tenantIDs = append(tenantIDs, tenantID) - } - if err != nil { - return err - } - - scKVAccessor := d.SpanConfig.KVAccessor.WithTxn(ctx, txn) - for _, tenantID := range tenantIDs { - // Install a single key span config at the start of tenant's - // keyspace; elsewhere this ensures that we split on the tenant - // boundary. Look towards CreateTenantRecord for more details. - tenantSpanConfig := d.SpanConfig.Default - tenantPrefix := keys.MakeTenantPrefix(tenantID) - tenantTarget := spanconfig.MakeTargetFromSpan(roachpb.Span{ - Key: tenantPrefix, - EndKey: tenantPrefix.PrefixEnd(), - }) - tenantSeedSpan := roachpb.Span{ - Key: tenantPrefix, - EndKey: tenantPrefix.Next(), - } - record, err := spanconfig.MakeRecord(spanconfig.MakeTargetFromSpan(tenantSeedSpan), - tenantSpanConfig) - if err != nil { - return err - } - toUpsert := []spanconfig.Record{record} - scRecords, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{tenantTarget}) - if err != nil { - return err - } - if len(scRecords) != 0 { - // This tenant already has span config records. It was either - // already migrated (upgrades need to be idempotent) or it was - // created after PreSeedTenantSpanConfigs was activated. There's - // nothing left to do here. - continue - } - if err := scKVAccessor.UpdateSpanConfigRecords( - ctx, nil /* toDelete */, toUpsert, hlc.MinTimestamp, hlc.MaxTimestamp, - ); err != nil { - return errors.Wrapf(err, "failed to seed span config for tenant %d", tenantID) - } - } - - return nil - }) -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 12dc3856e3ea..fd30653f7090 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -54,12 +54,6 @@ var upgrades = []upgrade.Upgrade{ preconditionBeforeStartingAnUpgrade, NoTenantUpgradeFunc, ), - upgrade.NewTenantUpgrade( - "seed system.span_configurations with configs for existing tenants", - toCV(clusterversion.SeedTenantSpanConfigs), - NoPrecondition, - seedTenantSpanConfigsMigration, - ), upgrade.NewTenantUpgrade( "add column target to system.protected_ts_records", toCV(clusterversion.AlterSystemProtectedTimestampAddColumn), From c8c5149ee55edd94e2302f861ec40f1fc750c932 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 15 Jul 2022 12:03:06 -0400 Subject: [PATCH 14/16] sql: add parsing support for SHOW CREATE FUNCTION statement This commit adds a `SHOW CREATE FUNCTION` statement to the SQL grammar. This statement is not yet implemented and executing it results in an error. Release note: None --- pkg/sql/delegate/BUILD.bazel | 1 + pkg/sql/delegate/delegate.go | 3 +++ pkg/sql/delegate/show_function.go | 20 ++++++++++++++++ .../logictest/testdata/logic_test/show_create | 5 ++++ pkg/sql/parser/sql.y | 14 +++++++++-- pkg/sql/parser/testdata/show | 24 +++++++++++++++++++ pkg/sql/sem/tree/show.go | 13 ++++++++++ pkg/sql/sem/tree/stmt.go | 16 ++++++++++--- 8 files changed, 91 insertions(+), 5 deletions(-) create mode 100644 pkg/sql/delegate/show_function.go diff --git a/pkg/sql/delegate/BUILD.bazel b/pkg/sql/delegate/BUILD.bazel index 6fea5a16e127..784079de690a 100644 --- a/pkg/sql/delegate/BUILD.bazel +++ b/pkg/sql/delegate/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "show_default_privileges.go", "show_enums.go", "show_full_table_scans.go", + "show_function.go", "show_grants.go", "show_jobs.go", "show_partitions.go", diff --git a/pkg/sql/delegate/delegate.go b/pkg/sql/delegate/delegate.go index 6734984cb6ce..cfdb5fe95da4 100644 --- a/pkg/sql/delegate/delegate.go +++ b/pkg/sql/delegate/delegate.go @@ -58,6 +58,9 @@ func TryDelegate( case *tree.ShowCreate: return d.delegateShowCreate(t) + case *tree.ShowCreateFunction: + return d.delegateShowCreateFunction(t) + case *tree.ShowCreateAllSchemas: return d.delegateShowCreateAllSchemas() diff --git a/pkg/sql/delegate/show_function.go b/pkg/sql/delegate/show_function.go new file mode 100644 index 000000000000..7fc125eba8a3 --- /dev/null +++ b/pkg/sql/delegate/show_function.go @@ -0,0 +1,20 @@ +// Copyright 2022 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 delegate + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" +) + +func (d *delegator) delegateShowCreateFunction(n *tree.ShowCreateFunction) (tree.Statement, error) { + return nil, unimplemented.New("SHOW CREATE FUNCTION", "this statement is not yet supported") +} diff --git a/pkg/sql/logictest/testdata/logic_test/show_create b/pkg/sql/logictest/testdata/logic_test/show_create index 0e4d21073240..4c2c1bc44b52 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_create +++ b/pkg/sql/logictest/testdata/logic_test/show_create @@ -140,3 +140,8 @@ CREATE TABLE public.t ( CONSTRAINT t_pkey PRIMARY KEY (rowid ASC) ); COMMENT ON COLUMN public.t.c IS 'first comment' + +subtest show_create_function + +statement error this statement is not yet supported +SHOW CREATE FUNCTION lower diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index be076bb675e8..64636eea28dd 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -5679,8 +5679,9 @@ zone_value: // %Category: Group // %Text: // SHOW BACKUP, SHOW CLUSTER SETTING, SHOW COLUMNS, SHOW CONSTRAINTS, -// SHOW CREATE, SHOW CREATE SCHEDULES, SHOW DATABASES, SHOW ENUMS, SHOW HISTOGRAM, SHOW INDEXES, SHOW -// PARTITIONS, SHOW JOBS, SHOW STATEMENTS, SHOW RANGE, SHOW RANGES, SHOW REGIONS, SHOW SURVIVAL GOAL, +// SHOW CREATE, SHOW CREATE SCHEDULES, SHOW DATABASES, SHOW ENUMS, SHOW +// FUNCTION, SHOW HISTOGRAM, SHOW INDEXES, SHOW PARTITIONS, SHOW JOBS, SHOW +// STATEMENTS, SHOW RANGE, SHOW RANGES, SHOW REGIONS, SHOW SURVIVAL GOAL, // SHOW ROLES, SHOW SCHEMAS, SHOW SEQUENCES, SHOW SESSION, SHOW SESSIONS, // SHOW STATISTICS, SHOW SYNTAX, SHOW TABLES, SHOW TRACE, SHOW TRANSACTION, // SHOW TRANSACTIONS, SHOW TRANSFER, SHOW TYPES, SHOW USERS, SHOW LAST QUERY STATISTICS, @@ -6760,6 +6761,15 @@ show_create_stmt: /* SKIP DOC */ $$.val = &tree.ShowCreate{Mode: tree.ShowCreateModeDatabase, Name: $4.unresolvedObjectName()} } +| SHOW CREATE FUNCTION db_object_name + { + /* SKIP DOC */ + $$.val = &tree.ShowCreateFunction{ + Name: tree.ResolvableFunctionReference{ + FunctionReference: $4.unresolvedObjectName().ToUnresolvedName(), + }, + } + } | SHOW CREATE ALL SCHEMAS { $$.val = &tree.ShowCreateAllSchemas{} diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index 88a8f0d849aa..64d9eb6beac1 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -1754,3 +1754,27 @@ SHOW client_encoding -- normalized! SHOW client_encoding -- fully parenthesized SHOW client_encoding -- literals removed SHOW client_encoding -- identifiers removed + +parse +SHOW CREATE FUNCTION foo +---- +SHOW CREATE FUNCTION foo +SHOW CREATE FUNCTION foo -- fully parenthesized +SHOW CREATE FUNCTION foo -- literals removed +SHOW CREATE FUNCTION _ -- identifiers removed + +parse +SHOW CREATE FUNCTION db.foo +---- +SHOW CREATE FUNCTION db.foo +SHOW CREATE FUNCTION db.foo -- fully parenthesized +SHOW CREATE FUNCTION db.foo -- literals removed +SHOW CREATE FUNCTION _._ -- identifiers removed + +parse +SHOW CREATE FUNCTION db.sch.foo +---- +SHOW CREATE FUNCTION db.sch.foo +SHOW CREATE FUNCTION db.sch.foo -- fully parenthesized +SHOW CREATE FUNCTION db.sch.foo -- literals removed +SHOW CREATE FUNCTION _._._ -- identifiers removed diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index cb9d8a42cb28..a4a726869b4f 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -944,3 +944,16 @@ func (s ShowCompletions) Format(ctx *FmtCtx) { } var _ Statement = &ShowCompletions{} + +// ShowCreateFunction represents a SHOW CREATE FUNCTION statement. +type ShowCreateFunction struct { + Name ResolvableFunctionReference +} + +// Format implements the NodeFormatter interface. +func (node *ShowCreateFunction) Format(ctx *FmtCtx) { + ctx.WriteString("SHOW CREATE FUNCTION ") + ctx.FormatNode(&node.Name) +} + +var _ Statement = &ShowCreateFunction{} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 155f49c6cd58..94adc13827c7 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -1714,6 +1714,15 @@ func (*ShowCompletions) observerStatement() {} func (*ShowCompletions) hiddenFromShowQueries() {} +// StatementReturnType implements the Statement interface. +func (*ShowCreateFunction) StatementReturnType() StatementReturnType { return Rows } + +// StatementType implements the Statement interface. +func (*ShowCreateFunction) StatementType() StatementType { return TypeDML } + +// StatementTag returns a short string identifying the type of statement. +func (*ShowCreateFunction) StatementTag() string { return "SHOW CREATE FUNCTION" } + // StatementReturnType implements the Statement interface. func (*Split) StatementReturnType() StatementReturnType { return Rows } @@ -1926,14 +1935,15 @@ func (n *ShowTenantClusterSettingList) String() string { return AsString(n) } func (n *ShowColumns) String() string { return AsString(n) } func (n *ShowConstraints) String() string { return AsString(n) } func (n *ShowCreate) String() string { return AsString(n) } -func (node *ShowCreateAllSchemas) String() string { return AsString(node) } -func (node *ShowCreateAllTables) String() string { return AsString(node) } -func (node *ShowCreateAllTypes) String() string { return AsString(node) } +func (n *ShowCreateAllSchemas) String() string { return AsString(n) } +func (n *ShowCreateAllTables) String() string { return AsString(n) } +func (n *ShowCreateAllTypes) String() string { return AsString(n) } func (n *ShowCreateSchedules) String() string { return AsString(n) } func (n *ShowDatabases) String() string { return AsString(n) } func (n *ShowDatabaseIndexes) String() string { return AsString(n) } func (n *ShowEnums) String() string { return AsString(n) } func (n *ShowFullTableScans) String() string { return AsString(n) } +func (n *ShowCreateFunction) String() string { return AsString(n) } func (n *ShowGrants) String() string { return AsString(n) } func (n *ShowHistogram) String() string { return AsString(n) } func (n *ShowSchedules) String() string { return AsString(n) } From 1f04334cf6b6e12812bc30b60191f64a93cbb6a8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 18 Jul 2022 12:30:32 -0700 Subject: [PATCH 15/16] bench: add a benchmark of index join with ordering Release note: None --- pkg/bench/bench_test.go | 44 +++++++++++++++++++++++++++++------------ 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/pkg/bench/bench_test.go b/pkg/bench/bench_test.go index 08bb244ba6d4..6b3b92fcfbce 100644 --- a/pkg/bench/bench_test.go +++ b/pkg/bench/bench_test.go @@ -1136,13 +1136,10 @@ func BenchmarkPlanning(b *testing.B) { }) } -// BenchmarkIndexJoin measure an index-join with 1000 rows. -func BenchmarkIndexJoin(b *testing.B) { - defer log.Scope(b).Close(b) - ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) { - // The table will have an extra column not contained in the index to force a - // join with the PK. - create := ` +func setupIndexJoinBenchmark(b *testing.B, db *sqlutils.SQLRunner) { + // The table will have an extra column not contained in the index to force a + // join with the PK. + create := ` CREATE TABLE tidx ( k INT NOT NULL, v INT NULL, @@ -1152,13 +1149,20 @@ func BenchmarkIndexJoin(b *testing.B) { FAMILY "primary" (k, v, extra) ) ` - // We'll insert 1000 rows with random values below 1000 in the index. - // We'll then force scanning of the secondary index which will require - // performing an index join to get 'extra' column. - insert := "insert into tidx(k,v) select generate_series(1,1000), (random()*1000)::int" + // We'll insert 1000 rows with random values below 1000 in the index. + // We'll then force scanning of the secondary index which will require + // performing an index join to get 'extra' column. + insert := "insert into tidx(k,v) select generate_series(1,1000), (random()*1000)::int" - db.Exec(b, create) - db.Exec(b, insert) + db.Exec(b, create) + db.Exec(b, insert) +} + +// BenchmarkIndexJoin measure an index-join with 1000 rows. +func BenchmarkIndexJoin(b *testing.B) { + defer log.Scope(b).Close(b) + ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) { + setupIndexJoinBenchmark(b, db) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -1167,6 +1171,20 @@ func BenchmarkIndexJoin(b *testing.B) { }) } +// BenchmarkIndexJoinOrdering is the same as BenchmarkIndexJoin when the +// ordering needs to be maintained. +func BenchmarkIndexJoinOrdering(b *testing.B) { + defer log.Scope(b).Close(b) + ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) { + setupIndexJoinBenchmark(b, db) + b.ResetTimer() + + for i := 0; i < b.N; i++ { + db.Exec(b, "select * from bench.tidx@idx where v < 1000 order by v") + } + }) +} + // BenchmarkIndexJoinColumnFamilies is the same as BenchmarkIndexJoin, only with // the table having two column families. func BenchmarkIndexJoinColumnFamilies(b *testing.B) { From b9088e9971c5977524bd956ba8884480f82b5290 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 18 Jul 2022 13:28:55 -0700 Subject: [PATCH 16/16] rowexec: use OutOfOrder mode of streamer for lookup joins with ordering MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Currently, the join reader always restores the required order for lookup joins on its own since all looked up rows are buffered before any output row is emitted. This observation allows us to use the OutOfOrder mode of the streamer in such scenarios, so this commit makes such a change. Previously, we would effectively maintain the order twice - both in the streamer and in the join reader, and the former is redundant. This will change in the future, but for now we can use the more-efficient mode. ``` name old time/op new time/op delta LookupJoinEqColsAreKeyOrdering/Cockroach-24 6.64ms ± 1% 6.48ms ± 1% -2.34% (p=0.000 n=10+10) LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24 7.89ms ± 1% 7.75ms ± 1% -1.80% (p=0.000 n=10+10) LookupJoinOrdering/Cockroach-24 9.01ms ± 3% 8.88ms ± 4% ~ (p=0.218 n=10+10) LookupJoinOrdering/MultinodeCockroach-24 12.1ms ± 4% 12.0ms ± 3% ~ (p=0.393 n=10+10) name old alloc/op new alloc/op delta LookupJoinEqColsAreKeyOrdering/Cockroach-24 1.68MB ± 1% 1.60MB ± 1% -4.93% (p=0.000 n=10+10) LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24 2.37MB ± 2% 2.29MB ± 2% -3.11% (p=0.000 n=10+10) LookupJoinOrdering/Cockroach-24 1.75MB ± 1% 1.66MB ± 1% -5.01% (p=0.000 n=10+9) LookupJoinOrdering/MultinodeCockroach-24 2.36MB ± 1% 2.25MB ± 1% -4.68% (p=0.000 n=8+10) name old allocs/op new allocs/op delta LookupJoinEqColsAreKeyOrdering/Cockroach-24 10.0k ± 1% 10.0k ± 1% ~ (p=0.278 n=10+9) LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24 14.3k ± 1% 14.3k ± 1% ~ (p=0.470 n=10+10) LookupJoinOrdering/Cockroach-24 12.4k ± 1% 12.5k ± 1% ~ (p=0.780 n=10+10) LookupJoinOrdering/MultinodeCockroach-24 17.1k ± 1% 17.0k ± 1% ~ (p=0.494 n=10+10) ``` Release note: None --- pkg/kv/kvclient/kvstreamer/streamer_test.go | 5 ++++- pkg/sql/rowexec/joinreader.go | 19 +++++++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvclient/kvstreamer/streamer_test.go b/pkg/kv/kvclient/kvstreamer/streamer_test.go index b9c7b229ca4c..c853b82e49e6 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer_test.go +++ b/pkg/kv/kvclient/kvstreamer/streamer_test.go @@ -504,7 +504,10 @@ func TestStreamerMultiRangeScan(t *testing.T) { // The crux of the test - run a query that performs a lookup join when // ordering needs to be maintained and then confirm that the results of the // parallel lookups are served in the right order. - r := db.QueryRow("SELECT array_agg(s) FROM small INNER LOOKUP JOIN large ON small.n = large.n GROUP BY small.n ORDER BY small.n") + // TODO(yuzefovich): remove ORDER BY clause inside array_agg when the lookup + // joins use the InOrder mode of the streamer when ordering needs to be + // maintained. + r := db.QueryRow("SELECT array_agg(s ORDER BY s) FROM small INNER LOOKUP JOIN large ON small.n = large.n GROUP BY small.n ORDER BY small.n") var result string err = r.Scan(&result) require.NoError(t, err) diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 00f3c1b94f43..4cd3712b530e 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -128,6 +128,16 @@ type joinReader struct { streamerInfo struct { unlimitedMemMonitor *mon.BytesMonitor budgetAcc mon.BoundAccount + // maintainOrdering indicates whether the ordering of the input stream + // needs to be maintained AND that we rely on the streamer for that. + // + // Currently this is only the case when joinReader.maintainOrdering is + // true and we are performing an index join. Due to implementation + // details, we don't rely on the streamer for maintaining the ordering + // for lookup joins at the moment (since we still buffer all looked up + // rows and restore the ordering explicitly via the + // joinReaderOrderingStrategy). + maintainOrdering bool diskMonitor *mon.BytesMonitor txnKVStreamerMemAcc mon.BoundAccount } @@ -467,9 +477,10 @@ func newJoinReader( jr.streamerInfo.unlimitedMemMonitor.StartNoReserved(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Mon) jr.streamerInfo.budgetAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount() jr.streamerInfo.txnKVStreamerMemAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount() + jr.streamerInfo.maintainOrdering = jr.maintainOrdering && readerType == indexJoinReaderType var diskBuffer kvstreamer.ResultDiskBuffer - if jr.maintainOrdering { + if jr.streamerInfo.maintainOrdering { jr.streamerInfo.diskMonitor = execinfra.NewMonitor( flowCtx.EvalCtx.Ctx(), jr.FlowCtx.DiskMonitor, "streamer-disk", /* name */ ) @@ -487,7 +498,7 @@ func newJoinReader( spec.LockingStrength, streamerBudgetLimit, &jr.streamerInfo.budgetAcc, - spec.MaintainOrdering, + jr.streamerInfo.maintainOrdering, singleRowLookup, int(spec.FetchSpec.MaxKeysPerRow), diskBuffer, @@ -938,7 +949,7 @@ func (jr *joinReader) readInput() ( return jrStateUnknown, nil, jr.DrainHelper() } } else { - if !jr.usesStreamer || jr.maintainOrdering { + if !jr.usesStreamer || jr.streamerInfo.maintainOrdering { // We don't want to sort the spans here if we're using the Streamer, // and it will perform the sort on its own - currently, this is the // case with OutOfOrder mode. @@ -1010,7 +1021,7 @@ func (jr *joinReader) performLookup() (joinReaderState, *execinfrapb.ProducerMet } if len(spans) != 0 { - if !jr.usesStreamer || jr.maintainOrdering { + if !jr.usesStreamer || jr.streamerInfo.maintainOrdering { // Sort the spans so that we can rely upon the fetcher to limit // the number of results per batch. It's safe to reorder the // spans here because we already restore the original order of