Skip to content

Commit

Permalink
Merge #110518
Browse files Browse the repository at this point in the history
110518: row: adjust TestRowFetcherMVCCMetadata to work with secondary tenants r=yuzefovich a=yuzefovich

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.

Fixes: #109396.
Informs: #76378.

Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Sep 14, 2023
2 parents 49573d5 + d83d5c5 commit 0cde11b
Show file tree
Hide file tree
Showing 12 changed files with 49 additions and 42 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/protected_timestamps_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{})
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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")))
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)...))
Expand Down Expand Up @@ -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)...)
Expand Down Expand Up @@ -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))
Expand Down Expand Up @@ -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))

Expand Down Expand Up @@ -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).
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)...)
Expand Down Expand Up @@ -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)
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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")
}
Expand Down
12 changes: 8 additions & 4 deletions pkg/sql/catalog/bootstrap/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -646,6 +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(TestingUserDescID(0))
func TestingUserTableDataMin(codec keys.SQLCodec) roachpb.Key {
return codec.TablePrefix(testingMinUserDescID(codec))
}
9 changes: 4 additions & 5 deletions pkg/sql/revert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/row/fetcher_mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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
Expand All @@ -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.TestingUserTableDataMin(codec)}); ; it.NextKey() {
ok, err := it.Valid()
if err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -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`)
Expand All @@ -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)
}
Expand Down Expand Up @@ -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) {
Expand All @@ -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) {
Expand All @@ -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) {
Expand Down
13 changes: 10 additions & 3 deletions pkg/sql/upsert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand All @@ -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)`)
Expand Down
1 change: 0 additions & 1 deletion pkg/storage/external_helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,5 +17,4 @@ import (

func init() {
storage.TestingUserDescID = bootstrap.TestingUserDescID
storage.TestingUserTableDataMin = bootstrap.TestingUserTableDataMin
}
3 changes: 0 additions & 3 deletions pkg/storage/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down

0 comments on commit 0cde11b

Please sign in to comment.