From 064418abca194a5758e12003a12038120ab79cb8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 12 Sep 2023 16:53:19 -0700 Subject: [PATCH 1/2] row: adjust TestRowFetcherMVCCMetadata to work with secondary tenants The problem was that we previously iterated over the whole key space (beyond the tenant's boundaries) which then we couldn't correctly decode using the row fetcher. Release note: None --- pkg/sql/catalog/bootstrap/metadata.go | 16 +++++++++++++--- pkg/sql/row/fetcher_mvcc_test.go | 24 ++++++++++++------------ 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 80f97e01ae56..35ff883e2bf7 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -624,11 +624,15 @@ func addSystemTenantEntry(target *MetadataSchema) { target.otherKV = append(target.otherKV, kvs...) } +func testingMinUserDescID(codec keys.SQLCodec) uint32 { + ms := MakeMetadataSchema(codec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) + return uint32(ms.FirstNonSystemDescriptorID()) +} + // TestingMinUserDescID returns the smallest user-created descriptor ID in a // bootstrapped cluster. func TestingMinUserDescID() uint32 { - ms := MakeMetadataSchema(keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef()) - return uint32(ms.FirstNonSystemDescriptorID()) + return testingMinUserDescID(keys.SystemSQLCodec) } // TestingMinNonDefaultUserDescID returns the smallest user-creatable descriptor @@ -647,5 +651,11 @@ func TestingUserDescID(offset uint32) uint32 { // TestingUserTableDataMin is a convenience function which returns the first // user table data key in a simple unit test setting. func TestingUserTableDataMin() roachpb.Key { - return keys.SystemSQLCodec.TablePrefix(TestingUserDescID(0)) + return keys.SystemSQLCodec.TablePrefix(TestingMinUserDescID()) +} + +// TestingUserTableDataMinEx is the same as TestingUserTableDataMin but supports +// secondary tenants. +func TestingUserTableDataMinEx(codec keys.SQLCodec) roachpb.Key { + return codec.TablePrefix(testingMinUserDescID(codec)) } diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 430b6946dfbc..9bc543e35748 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -34,7 +35,7 @@ import ( "github.com/cockroachdb/errors" ) -func slurpUserDataKVs(t testing.TB, e storage.Engine) []roachpb.KeyValue { +func slurpUserDataKVs(t testing.TB, e storage.Engine, codec keys.SQLCodec) []roachpb.KeyValue { t.Helper() // Scan meta keys directly from engine. We put this in a retry loop @@ -43,12 +44,12 @@ func slurpUserDataKVs(t testing.TB, e storage.Engine) []roachpb.KeyValue { var kvs []roachpb.KeyValue testutils.SucceedsSoon(t, func() error { kvs = nil - it, err := e.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) + it, err := e.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: codec.TenantEndKey()}) if err != nil { t.Fatal(err) } defer it.Close() - for it.SeekGE(storage.MVCCKey{Key: bootstrap.TestingUserTableDataMin()}); ; it.NextKey() { + for it.SeekGE(storage.MVCCKey{Key: bootstrap.TestingUserTableDataMinEx(codec)}); ; it.NextKey() { ok, err := it.Valid() if err != nil { t.Fatal(err) @@ -78,12 +79,11 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ - DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(109396), - }) + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) defer srv.Stopper().Stop(ctx) s := srv.ApplicationLayer() - store, _ := srv.GetStores().(*kvserver.Stores).GetStore(srv.GetFirstStoreID()) + codec := s.Codec() + store, _ := srv.StorageLayer().GetStores().(*kvserver.Stores).GetStore(srv.GetFirstStoreID()) sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `CREATE DATABASE d`) @@ -92,10 +92,10 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { a STRING PRIMARY KEY, b STRING, c STRING, d STRING, FAMILY (a, b, c), FAMILY (d) )`) - desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), `d`, `parent`) + desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, `d`, `parent`) var spec fetchpb.IndexFetchSpec if err := rowenc.InitIndexFetchSpec( - &spec, s.Codec(), desc, desc.GetPrimaryIndex(), desc.PublicColumnIDs(), + &spec, codec, desc, desc.GetPrimaryIndex(), desc.PublicColumnIDs(), ); err != nil { t.Fatal(err) } @@ -155,7 +155,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { SELECT cluster_logical_timestamp(); END;`).Scan(&ts1) - if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine())), []rowWithMVCCMetadata{ + if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine(), codec)), []rowWithMVCCMetadata{ {[]string{`1`, `a`, `a`, `a`}, false, ts1}, {[]string{`2`, `b`, `b`, `b`}, false, ts1}, }; !reflect.DeepEqual(expected, actual) { @@ -169,7 +169,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { SELECT cluster_logical_timestamp(); END;`).Scan(&ts2) - if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine())), []rowWithMVCCMetadata{ + if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine(), codec)), []rowWithMVCCMetadata{ {[]string{`1`, `NULL`, `NULL`, `NULL`}, false, ts2}, {[]string{`2`, `b`, `b`, `NULL`}, false, ts2}, }; !reflect.DeepEqual(expected, actual) { @@ -181,7 +181,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { DELETE FROM parent WHERE a = '1'; SELECT cluster_logical_timestamp(); END;`).Scan(&ts3) - if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine())), []rowWithMVCCMetadata{ + if actual, expected := kvsToRows(slurpUserDataKVs(t, store.TODOEngine(), codec)), []rowWithMVCCMetadata{ {[]string{`1`, `NULL`, `NULL`, `NULL`}, true, ts3}, {[]string{`2`, `b`, `b`, `NULL`}, false, ts2}, }; !reflect.DeepEqual(expected, actual) { From d83d5c50e3272f3a4ee8e7941d72dad02e8e1312 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 12 Sep 2023 17:19:17 -0700 Subject: [PATCH 2/2] catalog/bootstrap: adjust TestingUserTableDataMin to take in codec This commit adjusts existing `TestingUserTableDataMin` to take in codec as an argument as well as adjusts a few tests to pass the correct codec (rather than hard-coding system tenant's one). This should make it easier to spot issues if we ever decide to convert the remaining usages to run against the test tenant. While working on this I also realized that some CDC tests explicitly opt out of default test tenant mode via `feedTestNoTenants` test option (meta issue 76378 has been adjusted accordingly). Release note: None --- pkg/ccl/changefeedccl/protected_timestamps_test.go | 2 +- .../kvclient/kvcoord/dist_sender_ambiguous_test.go | 3 ++- pkg/kv/kvserver/batcheval/cmd_export_test.go | 4 ++-- pkg/kv/kvserver/client_lease_test.go | 10 +++++----- pkg/kv/kvserver/client_replica_test.go | 4 ++-- pkg/kv/kvserver/client_split_test.go | 6 +++--- pkg/sql/catalog/bootstrap/metadata.go | 8 +------- pkg/sql/revert_test.go | 9 ++++----- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/upsert_test.go | 13 ++++++++++--- pkg/storage/external_helpers_test.go | 1 - pkg/storage/mvcc_test.go | 3 --- 12 files changed, 31 insertions(+), 34 deletions(-) diff --git a/pkg/ccl/changefeedccl/protected_timestamps_test.go b/pkg/ccl/changefeedccl/protected_timestamps_test.go index 6c0189bfdd00..ae4833af63e6 100644 --- a/pkg/ccl/changefeedccl/protected_timestamps_test.go +++ b/pkg/ccl/changefeedccl/protected_timestamps_test.go @@ -159,7 +159,7 @@ func TestChangefeedProtectedTimestamps(t *testing.T) { var ( ctx = context.Background() userSpan = roachpb.Span{ - Key: bootstrap.TestingUserTableDataMin(), + Key: bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec), EndKey: keys.TableDataMax, } done = make(chan struct{}) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go index 2841d4e6fd67..b2bb7af058d7 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" @@ -280,7 +281,7 @@ func TestTransactionUnexpectedlyCommitted(t *testing.T) { } // Key constants. - tablePrefix := bootstrap.TestingUserTableDataMin() + tablePrefix := bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec) tableSpan := roachpb.Span{Key: tablePrefix, EndKey: tablePrefix.PrefixEnd()} keyA := roachpb.Key(encoding.EncodeBytesAscending(tablePrefix.Clone(), []byte("a"))) keyB := roachpb.Key(encoding.EncodeBytesAscending(tablePrefix.Clone(), []byte("b"))) diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index fe0bcc12a7d8..be62f1921b78 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -501,8 +501,8 @@ func TestExportGCThreshold(t *testing.T) { defer srv.Stopper().Stop(ctx) ts := srv.ApplicationLayer() - startKey := append(ts.Codec().TenantPrefix(), bootstrap.TestingUserTableDataMin()...) - endKey := append(ts.Codec().TenantPrefix(), keys.MaxKey...) + startKey := bootstrap.TestingUserTableDataMin(ts.Codec()) + endKey := ts.Codec().TenantEndKey() req := &kvpb.ExportRequest{ RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index b782f7c04201..beeec61221bc 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -739,7 +739,7 @@ func TestLeasePreferencesRebalance(t *testing.T) { }) defer tc.Stopper().Stop(ctx) - key := bootstrap.TestingUserTableDataMin() + key := bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec) tc.SplitRangeOrFatal(t, key) tc.AddVotersOrFatal(t, key, tc.Targets(1, 2)...) require.NoError(t, tc.WaitForVoters(key, tc.Targets(1, 2)...)) @@ -826,7 +826,7 @@ func TestLeaseholderRelocate(t *testing.T) { }) defer tc.Stopper().Stop(ctx) - _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin()) + _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) // We start with having the range under test on (1,2,3). tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2)...) @@ -1013,7 +1013,7 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { defer tc.Stopper().Stop(ctx) - key := bootstrap.TestingUserTableDataMin() + key := bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec) tc.SplitRangeOrFatal(t, key) tc.AddVotersOrFatal(t, key, tc.Targets(1, 3)...) repl := tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey(key)) @@ -1199,7 +1199,7 @@ func TestLeasesDontThrashWhenNodeBecomesSuspect(t *testing.T) { _, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.allocator.load_based_lease_rebalancing.enabled = 'false'`) require.NoError(t, err) - _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin()) + _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2, 3)...) tc.RemoveLeaseHolderOrFatal(t, rhsDesc, tc.Target(0), tc.Target(1)) @@ -1354,7 +1354,7 @@ func TestAlterRangeRelocate(t *testing.T) { ) defer tc.Stopper().Stop(ctx) - _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin()) + _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2)...) // We start with having the range under test on (1,2,3). diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index d9d864eac8cf..16da71505fad 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -2391,7 +2391,7 @@ func TestRemoveLeaseholder(t *testing.T) { ReplicationMode: base.ReplicationManual, }) defer tc.Stopper().Stop(context.Background()) - _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin()) + _, rhsDesc := tc.SplitRangeOrFatal(t, bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) // We start with having the range under test on (1,2,3). tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2)...) @@ -4700,7 +4700,7 @@ func TestTenantID(t *testing.T) { t.Run("(1) initial set", func(t *testing.T) { // Ensure that a normal range has the system tenant. { - _, repl := getFirstStoreReplica(t, tc.Server(0), bootstrap.TestingUserTableDataMin()) + _, repl := getFirstStoreReplica(t, tc.Server(0), bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) ri := repl.State(ctx) require.Equal(t, roachpb.SystemTenantID.ToUint64(), ri.TenantID, "%v", repl) } diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 889cc1eec16c..b2f440781278 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -1231,7 +1231,7 @@ func TestStoreRangeSplitBackpressureWrites(t *testing.T) { } t.Run(name, func(t *testing.T) { var activateSplitFilter int32 - splitKey := roachpb.RKey(bootstrap.TestingUserTableDataMin()) + splitKey := roachpb.RKey(bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec)) splitPending, blockSplits := make(chan struct{}), make(chan struct{}) // Set maxBytes to something small so we can exceed the maximum split @@ -2453,13 +2453,13 @@ func TestStoreTxnWaitQueueEnabledOnSplit(t *testing.T) { store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) require.NoError(t, err) - key := bootstrap.TestingUserTableDataMin() + key := bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec) args := adminSplitArgs(key) if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil { t.Fatalf("%q: split unexpected error: %s", key, pErr) } - rhsRepl := store.LookupReplica(roachpb.RKey(bootstrap.TestingUserTableDataMin())) + rhsRepl := store.LookupReplica(roachpb.RKey(bootstrap.TestingUserTableDataMin(keys.SystemSQLCodec))) if !rhsRepl.GetConcurrencyManager().TestingTxnWaitQueue().IsEnabled() { t.Errorf("expected RHS replica's push txn queue to be enabled post-split") } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 35ff883e2bf7..5aca65ef1b97 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -650,12 +650,6 @@ func TestingUserDescID(offset uint32) uint32 { // TestingUserTableDataMin is a convenience function which returns the first // user table data key in a simple unit test setting. -func TestingUserTableDataMin() roachpb.Key { - return keys.SystemSQLCodec.TablePrefix(TestingMinUserDescID()) -} - -// TestingUserTableDataMinEx is the same as TestingUserTableDataMin but supports -// secondary tenants. -func TestingUserTableDataMinEx(codec keys.SQLCodec) roachpb.Key { +func TestingUserTableDataMin(codec keys.SQLCodec) roachpb.Key { return codec.TablePrefix(testingMinUserDescID(codec)) } diff --git a/pkg/sql/revert_test.go b/pkg/sql/revert_test.go index 91695c8df350..03fabef4f01b 100644 --- a/pkg/sql/revert_test.go +++ b/pkg/sql/revert_test.go @@ -18,7 +18,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -317,12 +316,12 @@ func TestRevertGCThreshold(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) - kvDB := tc.Server(0).DB() + srv, _, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + codec := srv.ApplicationLayer().Codec() req := &kvpb.RevertRangeRequest{ - RequestHeader: kvpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, + RequestHeader: kvpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(codec), EndKey: codec.TenantEndKey()}, TargetTime: hlc.Timestamp{WallTime: -1}, } _, pErr := kv.SendWrapped(ctx, kvDB.NonTransactionalSender(), req) diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 9bc543e35748..abd4767723eb 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -49,7 +49,7 @@ func slurpUserDataKVs(t testing.TB, e storage.Engine, codec keys.SQLCodec) []roa t.Fatal(err) } defer it.Close() - for it.SeekGE(storage.MVCCKey{Key: bootstrap.TestingUserTableDataMinEx(codec)}); ; it.NextKey() { + for it.SeekGE(storage.MVCCKey{Key: bootstrap.TestingUserTableDataMin(codec)}); ; it.NextKey() { ok, err := it.Valid() if err != nil { t.Fatal(err) diff --git a/pkg/sql/upsert_test.go b/pkg/sql/upsert_test.go index e3b66997e398..b4c116928308 100644 --- a/pkg/sql/upsert_test.go +++ b/pkg/sql/upsert_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -50,8 +51,13 @@ func TestUpsertFastPath(t *testing.T) { var gets uint64 var scans uint64 var endTxn uint64 + var codecValue atomic.Value filter := func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { - if bytes.Compare(filterArgs.Req.Header().Key, bootstrap.TestingUserTableDataMin()) >= 0 { + codec := codecValue.Load() + if codec == nil { + return nil + } + if bytes.Compare(filterArgs.Req.Header().Key, bootstrap.TestingUserTableDataMin(codec.(keys.SQLCodec))) >= 0 { switch filterArgs.Req.Method() { case kvpb.Scan: atomic.AddUint64(&scans, 1) @@ -68,14 +74,15 @@ func TestUpsertFastPath(t *testing.T) { return nil } - s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ + srv, conn, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ TestingEvalFilter: filter, }, }}, }) - defer s.Stopper().Stop(context.Background()) + defer srv.Stopper().Stop(context.Background()) + codecValue.Store(srv.ApplicationLayer().Codec()) sqlDB := sqlutils.MakeSQLRunner(conn) sqlDB.Exec(t, `CREATE DATABASE d`) sqlDB.Exec(t, `CREATE TABLE d.kv (k INT PRIMARY KEY, v INT)`) diff --git a/pkg/storage/external_helpers_test.go b/pkg/storage/external_helpers_test.go index 71c05dfcb610..8a57685668ad 100644 --- a/pkg/storage/external_helpers_test.go +++ b/pkg/storage/external_helpers_test.go @@ -17,5 +17,4 @@ import ( func init() { storage.TestingUserDescID = bootstrap.TestingUserDescID - storage.TestingUserTableDataMin = bootstrap.TestingUserTableDataMin } diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 6d9de11338cc..61f0ee2f9672 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -5036,9 +5036,6 @@ func (it *seekLTTrackingIterator) SeekLT(k MVCCKey) { it.MVCCIterator.SeekLT(k) } -// Injected via `external_helpers_test.go`. -var TestingUserTableDataMin func() roachpb.Key - // TestMVCCGarbageCollectUsesSeekLTAppropriately ensures that the garbage // collection only utilizes SeekLT if there are enough undeleted versions. func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) {