Skip to content

Commit 5d1c0f8

Browse files
craig[bot]shubhamdhamadt
committedFeb 21, 2025·
141604: sql: increase tenant testing coverage r=rafiss,yuzefovich a=shubhamdhama This PR continues the work from #140447, replacing occurrences of `createTestServerParams` with `createTestServerParamsAllowTenants` to enable tenant testing in these tests. Informs: #140446 Epic: CRDB-38970 Release note: None 141833: upgrades: report progress during 25.1 upgrade jobs backfill r=dt a=dt Release note: none. Epic: none. Co-authored-by: Shubham Dhama <[email protected]> Co-authored-by: David Taylor <[email protected]>
3 parents 28a7a3c + 6f2435c + 5a2139b commit 5d1c0f8

20 files changed

+101
-58
lines changed
 

‎pkg/sql/err_count_test.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ func TestErrorCounts(t *testing.T) {
2525

2626
telemetry.GetFeatureCounts(telemetry.Raw, telemetry.ResetCounts)
2727

28-
params, _ := createTestServerParams()
28+
params, _ := createTestServerParamsAllowTenants()
2929
s, db, _ := serverutils.StartServer(t, params)
3030
defer s.Stopper().Stop(context.Background())
3131

@@ -69,7 +69,7 @@ func TestTransactionRetryErrorCounts(t *testing.T) {
6969
// in pgwire (pgwire.convertToErrWithPGCode). Make sure we're
7070
// reporting errors at a level that allows this code to be recorded.
7171

72-
params, _ := createTestServerParams()
72+
params, _ := createTestServerParamsAllowTenants()
7373
s, db, _ := serverutils.StartServer(t, params)
7474
defer s.Stopper().Stop(context.Background())
7575

‎pkg/sql/explain_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -517,7 +517,7 @@ func TestExplainRedact(t *testing.T) {
517517
rng, seed := randutil.NewTestRand()
518518
t.Log("seed:", seed)
519519

520-
params, _ := createTestServerParams()
520+
params, _ := createTestServerParamsAllowTenants()
521521
srv, sqlDB, _ := serverutils.StartServer(t, params)
522522
defer srv.Stopper().Stop(ctx)
523523

‎pkg/sql/materialized_view_test.go

+10-11
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ import (
1010
"testing"
1111

1212
"github.com/cockroachdb/cockroach/pkg/base"
13-
"github.com/cockroachdb/cockroach/pkg/keys"
1413
"github.com/cockroachdb/cockroach/pkg/sql"
1514
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
1615
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
@@ -31,7 +30,7 @@ func TestMaterializedViewClearedAfterRefresh(t *testing.T) {
3130
defer log.Scope(t).Close(t)
3231

3332
ctx := context.Background()
34-
params, _ := createTestServerParams()
33+
params, _ := createTestServerParamsAllowTenants()
3534

3635
s, sqlDB, kvDB := serverutils.StartServer(t, params)
3736
defer s.Stopper().Stop(ctx)
@@ -49,7 +48,7 @@ CREATE MATERIALIZED VIEW t.v AS SELECT x FROM t.t;
4948
t.Fatal(err)
5049
}
5150

52-
descBeforeRefresh := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "v")
51+
descBeforeRefresh := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "v")
5352

5453
// Update the view and refresh it.
5554
if _, err := sqlDB.Exec(`
@@ -80,7 +79,7 @@ REFRESH MATERIALIZED VIEW t.v;
8079

8180
// The data should be deleted.
8281
testutils.SucceedsSoon(t, func() error {
83-
indexPrefix := keys.SystemSQLCodec.IndexPrefix(uint32(descBeforeRefresh.GetID()), uint32(descBeforeRefresh.GetPrimaryIndexID()))
82+
indexPrefix := s.Codec().IndexPrefix(uint32(descBeforeRefresh.GetID()), uint32(descBeforeRefresh.GetPrimaryIndexID()))
8483
indexEnd := indexPrefix.PrefixEnd()
8584
if kvs, err := kvDB.Scan(ctx, indexPrefix, indexEnd, 0); err != nil {
8685
t.Fatal(err)
@@ -98,7 +97,7 @@ func TestMaterializedViewRefreshVisibility(t *testing.T) {
9897
defer log.Scope(t).Close(t)
9998

10099
ctx := context.Background()
101-
params, _ := createTestServerParams()
100+
params, _ := createTestServerParamsAllowTenants()
102101

103102
waitForCommit, waitToProceed, refreshDone := make(chan struct{}), make(chan struct{}), make(chan struct{})
104103
params.Knobs = base.TestingKnobs{
@@ -146,7 +145,7 @@ func TestMaterializedViewCleansUpOnRefreshFailure(t *testing.T) {
146145
defer log.Scope(t).Close(t)
147146

148147
ctx := context.Background()
149-
params, _ := createTestServerParams()
148+
params, _ := createTestServerParamsAllowTenants()
150149

151150
// Protects shouldError
152151
var mu syncutil.Mutex
@@ -182,7 +181,7 @@ CREATE MATERIALIZED VIEW t.v AS SELECT x FROM t.t;
182181
t.Fatal(err)
183182
}
184183

185-
descBeforeRefresh := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "v")
184+
descBeforeRefresh := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "v")
186185

187186
// Add a zone config to delete all table data.
188187
_, err := sqltestutils.AddImmediateGCZoneConfig(sqlDB, descBeforeRefresh.GetID())
@@ -196,7 +195,7 @@ CREATE MATERIALIZED VIEW t.v AS SELECT x FROM t.t;
196195
}
197196

198197
testutils.SucceedsSoon(t, func() error {
199-
tableStart := keys.SystemSQLCodec.TablePrefix(uint32(descBeforeRefresh.GetID()))
198+
tableStart := s.Codec().TablePrefix(uint32(descBeforeRefresh.GetID()))
200199
tableEnd := tableStart.PrefixEnd()
201200
if kvs, err := kvDB.Scan(ctx, tableStart, tableEnd, 0); err != nil {
202201
t.Fatal(err)
@@ -212,7 +211,7 @@ func TestDropMaterializedView(t *testing.T) {
212211
defer log.Scope(t).Close(t)
213212

214213
ctx := context.Background()
215-
params, _ := createTestServerParams()
214+
params, _ := createTestServerParamsAllowTenants()
216215
s, sqlRaw, kvDB := serverutils.StartServer(t, params)
217216
defer s.Stopper().Stop(ctx)
218217

@@ -229,7 +228,7 @@ CREATE TABLE t.t (x INT);
229228
INSERT INTO t.t VALUES (1), (2);
230229
CREATE MATERIALIZED VIEW t.v AS SELECT x FROM t.t;
231230
`)
232-
desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "v")
231+
desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "v")
233232
// Add a zone config to delete all table data.
234233
_, err := sqltestutils.AddImmediateGCZoneConfig(sqlRaw, desc.GetID())
235234
require.NoError(t, err)
@@ -240,7 +239,7 @@ CREATE MATERIALIZED VIEW t.v AS SELECT x FROM t.t;
240239

241240
// All of the table data should be cleaned up.
242241
testutils.SucceedsSoon(t, func() error {
243-
tableStart := keys.SystemSQLCodec.TablePrefix(uint32(desc.GetID()))
242+
tableStart := s.Codec().TablePrefix(uint32(desc.GetID()))
244243
tableEnd := tableStart.PrefixEnd()
245244
if kvs, err := kvDB.Scan(ctx, tableStart, tableEnd, 0); err != nil {
246245
t.Fatal(err)

‎pkg/sql/metric_test.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ func TestQueryCounts(t *testing.T) {
5353
defer leaktest.AfterTest(t)()
5454
defer log.Scope(t).Close(t)
5555

56-
params, _ := createTestServerParams()
56+
params, _ := createTestServerParamsAllowTenants()
5757
params.Knobs = base.TestingKnobs{
5858
SQLLeaseManager: &lease.ManagerTestingKnobs{
5959
// Disable SELECT called for delete orphaned leases to keep
@@ -174,7 +174,7 @@ func TestAbortCountConflictingWrites(t *testing.T) {
174174
defer log.Scope(t).Close(t)
175175

176176
testutils.RunTrueAndFalse(t, "retry loop", func(t *testing.T, retry bool) {
177-
params, cmdFilters := createTestServerParams()
177+
params, cmdFilters := createTestServerParamsAllowTenants()
178178
s, sqlDB, _ := serverutils.StartServer(t, params)
179179
defer s.Stopper().Stop(context.Background())
180180

@@ -279,7 +279,7 @@ func TestAbortCountConflictingWrites(t *testing.T) {
279279
func TestAbortCountErrorDuringTransaction(t *testing.T) {
280280
defer leaktest.AfterTest(t)()
281281
defer log.Scope(t).Close(t)
282-
params, _ := createTestServerParams()
282+
params, _ := createTestServerParamsAllowTenants()
283283
s, sqlDB, _ := serverutils.StartServer(t, params)
284284
defer s.Stopper().Stop(context.Background())
285285

@@ -314,7 +314,7 @@ func TestSavepointMetrics(t *testing.T) {
314314
defer leaktest.AfterTest(t)()
315315
defer log.Scope(t).Close(t)
316316

317-
params, _ := createTestServerParams()
317+
params, _ := createTestServerParamsAllowTenants()
318318
s, sqlDB, _ := serverutils.StartServer(t, params)
319319
defer s.Stopper().Stop(context.Background())
320320

‎pkg/sql/mutation_test.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ func TestConstraintValidationBeforeBuffering(t *testing.T) {
3030
defer leaktest.AfterTest(t)()
3131
defer log.Scope(t).Close(t)
3232

33-
params, _ := createTestServerParams()
33+
params, _ := createTestServerParamsAllowTenants()
3434
s, db, _ := serverutils.StartServer(t, params)
3535
defer s.Stopper().Stop(context.Background())
3636

@@ -133,7 +133,7 @@ func TestReadCommittedImplicitPartitionUpsert(t *testing.T) {
133133
}
134134

135135
ctx := context.Background()
136-
params, _ := createTestServerParams()
136+
params, _ := createTestServerParamsAllowTenants()
137137
// If test is in Ready state, transition to ReadDone and wait for conflict.
138138
params.Knobs = base.TestingKnobs{
139139
SQLExecutor: &sql.ExecutorTestingKnobs{

‎pkg/sql/mvcc_backfiller_test.go

+12-9
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ func TestIndexBackfillMergeRetry(t *testing.T) {
5959

6060
skip.UnderDuress(t, "this test fails under duress")
6161

62-
params, _ := createTestServerParams()
62+
params, _ := createTestServerParamsAllowTenants()
6363

6464
writesPopulated := false
6565
var writesFn func() error
@@ -176,7 +176,7 @@ func TestIndexBackfillFractionTracking(t *testing.T) {
176176
defer leaktest.AfterTest(t)()
177177
defer log.Scope(t).Close(t)
178178

179-
params, _ := createTestServerParams()
179+
params, _ := createTestServerParamsAllowTenants()
180180

181181
const (
182182
rowCount = 2000
@@ -213,20 +213,21 @@ func TestIndexBackfillFractionTracking(t *testing.T) {
213213
lastPercentage = fraction
214214
}
215215

216+
var codec keys.SQLCodec
216217
params.Knobs = base.TestingKnobs{
217218
SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{
218219
BackfillChunkSize: chunkSize,
219220
RunBeforeResume: func(id jobspb.JobID) error {
220221
jobID = id
221-
tableDesc := desctestutils.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "public", "test")
222+
tableDesc := desctestutils.TestingGetTableDescriptor(kvDB, codec, "t", "public", "test")
222223
split(tableDesc, tableDesc.GetPrimaryIndex())
223224
return nil
224225
},
225226
RunBeforeTempIndexMerge: func() {
226227
for i := rowCount + 1; i < (rowCount*2)+1; i++ {
227228
sqlRunner.Exec(t, "INSERT INTO t.test VALUES ($1, $1)", i)
228229
}
229-
tableDesc := desctestutils.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "public", "test")
230+
tableDesc := desctestutils.TestingGetTableDescriptor(kvDB, codec, "t", "public", "test")
230231
tempIdx, err := findCorrespondingTemporaryIndex(tableDesc, "new_idx")
231232
require.NoError(t, err)
232233
split(tableDesc, tempIdx)
@@ -254,6 +255,7 @@ func TestIndexBackfillFractionTracking(t *testing.T) {
254255
})
255256
defer tc.Stopper().Stop(context.Background())
256257
kvDB = tc.Server(0).DB()
258+
codec = tc.Server(0).Codec()
257259
sqlDB := tc.ServerConn(0)
258260
_, err := sqlDB.Exec("SET CLUSTER SETTING sql.defaults.use_declarative_schema_changer='off';")
259261
require.NoError(t, err)
@@ -289,7 +291,7 @@ func TestRaceWithIndexBackfillMerge(t *testing.T) {
289291
maxValue = 200
290292
}
291293

292-
params, _ := createTestServerParams()
294+
params, _ := createTestServerParamsAllowTenants()
293295
initMergeNotification := func() chan struct{} {
294296
mu.Lock()
295297
defer mu.Unlock()
@@ -484,7 +486,7 @@ func TestInvertedIndexMergeEveryStateWrite(t *testing.T) {
484486
var initialRows = 10000
485487
rowIdx := 0
486488

487-
params, _ := createTestServerParams()
489+
params, _ := createTestServerParamsAllowTenants()
488490
var writeMore func() error
489491
params.Knobs = base.TestingKnobs{
490492
SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{
@@ -660,7 +662,8 @@ func splitIndex(
660662
rkts := make(map[roachpb.RangeID]rangeAndKT)
661663
for _, sp := range sps {
662664

663-
pik, err := randgen.TestingMakeSecondaryIndexKey(desc, index, keys.SystemSQLCodec, sp.Vals...)
665+
pik, err := randgen.TestingMakeSecondaryIndexKey(
666+
desc, index, tc.Server(0).Codec(), sp.Vals...)
664667
if err != nil {
665668
return err
666669
}
@@ -741,7 +744,7 @@ func TestIndexMergeEveryChunkWrite(t *testing.T) {
741744
rowIdx := 0
742745
mergeSerializeCh := make(chan struct{}, 1)
743746

744-
params, _ := createTestServerParams()
747+
params, _ := createTestServerParamsAllowTenants()
745748
var writeMore func() error
746749
params.Knobs = base.TestingKnobs{
747750
DistSQL: &execinfra.TestingKnobs{
@@ -847,7 +850,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v int);`
847850
// Wait for the beginning of the Merge step of the schema change.
848851
// Write data to the temp index and split it at the hazardous
849852
// points.
850-
params, _ := createTestServerParams()
853+
params, _ := createTestServerParamsAllowTenants()
851854
params.Knobs = base.TestingKnobs{
852855
SQLDeclarativeSchemaChanger: &scexec.TestingKnobs{
853856
BeforeStage: func(p scplan.Plan, stageIdx int) error {

‎pkg/sql/privileged_accessor_test.go

+2-3
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@ import (
99
"context"
1010
"testing"
1111

12-
"github.com/cockroachdb/cockroach/pkg/keys"
1312
"github.com/cockroachdb/cockroach/pkg/kv"
1413
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
1514
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -26,14 +25,14 @@ func TestLookupNamespaceID(t *testing.T) {
2625
defer log.Scope(t).Close(t)
2726

2827
ctx := context.Background()
29-
params, _ := createTestServerParams()
28+
params, _ := createTestServerParamsAllowTenants()
3029
s, sqlDB, kvDB := serverutils.StartServer(t, params)
3130
defer s.Stopper().Stop(ctx)
3231

3332
err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
3433
return txn.Put(
3534
ctx,
36-
catalogkeys.EncodeNameKey(keys.SystemSQLCodec, &descpb.NameInfo{ParentID: 999, ParentSchemaID: 1000, Name: "bob"}),
35+
catalogkeys.EncodeNameKey(s.Codec(), &descpb.NameInfo{ParentID: 999, ParentSchemaID: 1000, Name: "bob"}),
3736
9999,
3837
)
3938
})

‎pkg/sql/run_control_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -409,7 +409,7 @@ func TestCancelWithSubquery(t *testing.T) {
409409
defer leaktest.AfterTest(t)()
410410
defer log.Scope(t).Close(t)
411411

412-
params, _ := createTestServerParams()
412+
params, _ := createTestServerParamsAllowTenants()
413413
s, conn, _ := serverutils.StartServer(t, params)
414414
defer s.Stopper().Stop(context.Background())
415415

‎pkg/sql/show_create_all_tables_builtin_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ func TestRecreateTables(t *testing.T) {
2323
defer leaktest.AfterTest(t)()
2424
defer log.Scope(t).Close(t)
2525

26-
params, _ := createTestServerParams()
26+
params, _ := createTestServerParamsAllowTenants()
2727
s, sqlDB, _ := serverutils.StartServer(t, params)
2828
defer s.Stopper().Stop(context.Background())
2929

‎pkg/sql/show_test.go

+5-5
Original file line numberDiff line numberDiff line change
@@ -294,7 +294,7 @@ func TestShowCreateView(t *testing.T) {
294294
defer leaktest.AfterTest(t)()
295295
defer log.Scope(t).Close(t)
296296

297-
params, _ := createTestServerParams()
297+
params, _ := createTestServerParamsAllowTenants()
298298
s, sqlDB, _ := serverutils.StartServer(t, params)
299299
defer s.Stopper().Stop(context.Background())
300300

@@ -391,7 +391,7 @@ func TestShowCreateSequence(t *testing.T) {
391391
defer leaktest.AfterTest(t)()
392392
defer log.Scope(t).Close(t)
393393

394-
params, _ := createTestServerParams()
394+
params, _ := createTestServerParamsAllowTenants()
395395
s, sqlDB, _ := serverutils.StartServer(t, params)
396396
defer s.Stopper().Stop(context.Background())
397397

@@ -971,7 +971,7 @@ func TestShowSessionPrivileges(t *testing.T) {
971971
defer leaktest.AfterTest(t)()
972972
defer log.Scope(t).Close(t)
973973

974-
params, _ := createTestServerParams()
974+
params, _ := createTestServerParamsAllowTenants()
975975
params.Insecure = true
976976
s, rawSQLDBroot, _ := serverutils.StartServer(t, params)
977977
sqlDBroot := sqlutils.MakeSQLRunner(rawSQLDBroot)
@@ -1054,7 +1054,7 @@ func TestShowRedactedActiveStatements(t *testing.T) {
10541054
defer leaktest.AfterTest(t)()
10551055
defer log.Scope(t).Close(t)
10561056

1057-
params, _ := createTestServerParams()
1057+
params, _ := createTestServerParamsAllowTenants()
10581058
params.Insecure = true
10591059
ctx, cancel := context.WithCancel(context.Background())
10601060
s, rawSQLDBroot, _ := serverutils.StartServer(t, params)
@@ -1215,7 +1215,7 @@ func TestShowRedactedSessions(t *testing.T) {
12151215
defer leaktest.AfterTest(t)()
12161216
defer log.Scope(t).Close(t)
12171217

1218-
params, _ := createTestServerParams()
1218+
params, _ := createTestServerParamsAllowTenants()
12191219
params.Insecure = true
12201220
ctx, cancel := context.WithCancel(context.Background())
12211221
s, rawSQLDBroot, _ := serverutils.StartServer(t, params)

‎pkg/sql/sort_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ func TestOrderByRandom(t *testing.T) {
1818
defer leaktest.AfterTest(t)()
1919
defer log.Scope(t).Close(t)
2020

21-
params, _ := createTestServerParams()
21+
params, _ := createTestServerParamsAllowTenants()
2222
s, sqlDB, _ := serverutils.StartServer(t, params)
2323
defer s.Stopper().Stop(context.Background())
2424

‎pkg/sql/split_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ func TestSplitAt(t *testing.T) {
2222
defer leaktest.AfterTest(t)()
2323
defer log.Scope(t).Close(t)
2424

25-
params, _ := createTestServerParams()
25+
params, _ := createTestServerParamsAllowTenants()
2626
s, db, _ := serverutils.StartServer(t, params)
2727
defer s.Stopper().Stop(context.Background())
2828

‎pkg/sql/table_ref_test.go

+4-5
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,6 @@ import (
1111
"regexp"
1212
"testing"
1313

14-
"github.com/cockroachdb/cockroach/pkg/keys"
1514
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1615
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
1716
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
@@ -23,7 +22,7 @@ func TestTableRefs(t *testing.T) {
2322
defer leaktest.AfterTest(t)()
2423
defer log.Scope(t).Close(t)
2524

26-
params, _ := createTestServerParams()
25+
params, _ := createTestServerParamsAllowTenants()
2726
s, db, kvDB := serverutils.StartServer(t, params)
2827
defer s.Stopper().Stop(context.Background())
2928

@@ -40,7 +39,7 @@ CREATE INDEX bc ON test.t(b, c);
4039
}
4140

4241
// Retrieve the numeric descriptors.
43-
tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t")
42+
tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "test", "t")
4443
tID := tableDesc.GetID()
4544
var aID, bID, cID descpb.ColumnID
4645
for _, c := range tableDesc.PublicColumns() {
@@ -55,7 +54,7 @@ CREATE INDEX bc ON test.t(b, c);
5554
}
5655

5756
// Retrieve the numeric descriptors.
58-
tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "hidden")
57+
tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "test", "hidden")
5958
tIDHidden := tableDesc.GetID()
6059
var rowIDHidden descpb.ColumnID
6160
for _, c := range tableDesc.PublicColumns() {
@@ -78,7 +77,7 @@ ALTER TABLE test.t DROP COLUMN xx;
7877
t.Fatal(err)
7978
}
8079

81-
tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t")
80+
tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "test", "t")
8281
pkID := tableDesc.GetPrimaryIndexID()
8382
secID := tableDesc.PublicNonPrimaryIndexes()[0].GetID()
8483

‎pkg/sql/type_change_test.go

+7-7
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ func TestTypeSchemaChangeRetriesTransparently(t *testing.T) {
3636
// Protects errorReturned.
3737
var mu syncutil.Mutex
3838
errorReturned := false
39-
params, _ := createTestServerParams()
39+
params, _ := createTestServerParamsAllowTenants()
4040
params.Knobs.SQLTypeSchemaChanger = &sql.TypeSchemaChangerTestingKnobs{
4141
RunBeforeExec: func() error {
4242
mu.Lock()
@@ -81,7 +81,7 @@ func TestFailedTypeSchemaChangeRetriesTransparently(t *testing.T) {
8181
var mu syncutil.Mutex
8282
// Ensures just the first try to cleanup returns a retryable error.
8383
errReturned := false
84-
params, _ := createTestServerParams()
84+
params, _ := createTestServerParamsAllowTenants()
8585
cleanupSuccessfullyFinished := make(chan struct{})
8686
params.Knobs.SQLTypeSchemaChanger = &sql.TypeSchemaChangerTestingKnobs{
8787
RunBeforeExec: func() error {
@@ -141,7 +141,7 @@ func TestFailedTypeSchemaChangeIgnoresDrops(t *testing.T) {
141141
defer log.Scope(t).Close(t)
142142

143143
ctx := context.Background()
144-
params, _ := createTestServerParams()
144+
params, _ := createTestServerParamsAllowTenants()
145145
startDrop := make(chan struct{})
146146
dropFinished := make(chan struct{})
147147
cancelled := atomic.Bool{}
@@ -201,7 +201,7 @@ func TestAddDropValuesInTransaction(t *testing.T) {
201201

202202
ctx := context.Background()
203203

204-
params, _ := createTestServerParams()
204+
params, _ := createTestServerParamsAllowTenants()
205205
// Decrease the adopt loop interval so that retries happen quickly.
206206
params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
207207

@@ -346,7 +346,7 @@ func TestEnumMemberTransitionIsolation(t *testing.T) {
346346
defer leaktest.AfterTest(t)()
347347
defer log.Scope(t).Close(t)
348348

349-
params, _ := createTestServerParams()
349+
params, _ := createTestServerParamsAllowTenants()
350350
// Protects blocker.
351351
var mu syncutil.Mutex
352352
blocker := make(chan struct{})
@@ -493,7 +493,7 @@ func TestTypeChangeJobCancelSemantics(t *testing.T) {
493493
for _, tc := range testCases {
494494
t.Run(tc.desc, func(t *testing.T) {
495495

496-
params, _ := createTestServerParams()
496+
params, _ := createTestServerParamsAllowTenants()
497497

498498
// Wait groups for synchronizing various parts of the test.
499499
typeSchemaChangeStarted := make(chan struct{})
@@ -583,7 +583,7 @@ func TestAddDropEnumValues(t *testing.T) {
583583
defer ccl.TestingEnableEnterprise()()
584584
ctx := context.Background()
585585

586-
params, _ := createTestServerParams()
586+
params, _ := createTestServerParamsAllowTenants()
587587
// Decrease the adopt loop interval so that retries happen quickly.
588588
params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
589589

‎pkg/sql/unsplit_test.go

+12-1
Original file line numberDiff line numberDiff line change
@@ -13,18 +13,20 @@ import (
1313

1414
"github.com/cockroachdb/cockroach/pkg/base"
1515
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
16+
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
1617
"github.com/cockroachdb/cockroach/pkg/roachpb"
1718
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
1819
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
1920
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2021
"github.com/cockroachdb/cockroach/pkg/util/log"
22+
"github.com/stretchr/testify/require"
2123
)
2224

2325
func TestUnsplitAt(t *testing.T) {
2426
defer leaktest.AfterTest(t)()
2527
defer log.Scope(t).Close(t)
2628

27-
params, _ := createTestServerParams()
29+
params, _ := createTestServerParamsAllowTenants()
2830
// TODO(jeffreyxiao): Disable the merge queue due to a race condition. The
2931
// merge queue might issue an AdminMerge and before the actual merge happens,
3032
// the LHS of the merge is manually split and is later merged even though a
@@ -41,6 +43,15 @@ func TestUnsplitAt(t *testing.T) {
4143
s, db, _ := serverutils.StartServer(t, params)
4244
defer s.Stopper().Stop(context.Background())
4345

46+
if s.DeploymentMode().IsExternal() {
47+
require.NoError(t, s.GrantTenantCapabilities(
48+
context.Background(), serverutils.TestTenantID(),
49+
map[tenantcapabilities.ID]string{
50+
tenantcapabilities.CanAdminSplit: "true",
51+
tenantcapabilities.CanAdminUnsplit: "true",
52+
}))
53+
}
54+
4455
r := sqlutils.MakeSQLRunner(db)
4556

4657
r.Exec(t, "CREATE DATABASE d")

‎pkg/sql/zone_test.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -234,7 +234,7 @@ func TestZoneInheritField(t *testing.T) {
234234
defer leaktest.AfterTest(t)()
235235
defer log.Scope(t).Close(t)
236236

237-
params, _ := createTestServerParams()
237+
params, _ := createTestServerParamsAllowTenants()
238238
s, db, _ := serverutils.StartServer(t, params)
239239
defer s.Stopper().Stop(context.Background())
240240

@@ -281,7 +281,7 @@ func TestInvalidSetShowZones(t *testing.T) {
281281
defer leaktest.AfterTest(t)()
282282
defer log.Scope(t).Close(t)
283283

284-
params, _ := createTestServerParams()
284+
params, _ := createTestServerParamsAllowTenants()
285285
s, db, _ := serverutils.StartServer(t, params)
286286
defer s.Stopper().Stop(context.Background())
287287

‎pkg/upgrade/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ go_library(
1414
deps = [
1515
"//pkg/clusterversion",
1616
"//pkg/jobs",
17+
"//pkg/jobs/jobspb",
1718
"//pkg/keys",
1819
"//pkg/keyvisualizer",
1920
"//pkg/kv",

‎pkg/upgrade/tenant_upgrade.go

+4
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@ import (
1111

1212
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1313
"github.com/cockroachdb/cockroach/pkg/jobs"
14+
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
1415
"github.com/cockroachdb/cockroach/pkg/keys"
1516
"github.com/cockroachdb/cockroach/pkg/kv"
1617
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfo"
@@ -49,6 +50,9 @@ type TenantDeps struct {
4950
SchemaResolverConstructor func( // A constructor that returns a schema resolver for `descriptors` in `currDb`.
5051
txn *kv.Txn, descriptors *descs.Collection, currDb string,
5152
) (resolver.SchemaResolver, func(), error)
53+
54+
// OptionalJobID is the job ID for this upgrade if it is running inside a job.
55+
OptionalJobID jobspb.JobID
5256
}
5357

5458
// TenantUpgradeFunc is used to perform sql-level upgrades. It may be run from

‎pkg/upgrade/upgradejob/upgrade_job.go

+1
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,7 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error {
9595
SessionData: execCtx.SessionData(),
9696
ClusterID: execCtx.ExtendedEvalContext().ClusterID,
9797
TenantInfoAccessor: mc.SystemDeps().TenantInfoAccessor,
98+
OptionalJobID: r.j.ID(),
9899
}
99100

100101
tenantDeps.SchemaResolverConstructor = func(

‎pkg/upgrade/upgrades/v25_1_add_jobs_tables.go

+28-2
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ import (
1414
"github.com/cockroachdb/cockroach/pkg/keys"
1515
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
1616
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
17+
"github.com/cockroachdb/cockroach/pkg/sql/isql"
1718
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
1819
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
1920
"github.com/cockroachdb/cockroach/pkg/upgrade"
@@ -92,8 +93,18 @@ var jobsBackfillPageSize = 32
9293
func backfillJobsTablesAndColumns(
9394
ctx context.Context, cv clusterversion.ClusterVersion, d upgrade.TenantDeps,
9495
) error {
96+
totalRowsRow, err := d.DB.Executor().QueryRowEx(ctx, "jobs-backfill-count", nil, sessiondata.NodeUserSessionDataOverride,
97+
`SELECT count(*) FROM system.jobs WHERE owner IS NULL`)
98+
if err != nil {
99+
return err
100+
}
101+
totalRows := int(tree.MustBeDInt(totalRowsRow[0]))
102+
if totalRows == 0 {
103+
return nil
104+
}
105+
95106
every := log.Every(time.Minute)
96-
log.Infof(ctx, "backfilling new jobs tables and columns")
107+
log.Infof(ctx, "backfilling new jobs tables and columns for %d jobs", totalRows)
97108
jobsBackfilled := 0
98109
for {
99110
candidateRows, err := d.DB.Executor().QueryBufferedEx(ctx, "jobs-backfill-find", nil, sessiondata.NodeUserSessionDataOverride,
@@ -192,7 +203,22 @@ func backfillJobsTablesAndColumns(
192203
jobsBackfilled++
193204
}
194205
if every.ShouldLog() {
195-
log.Infof(ctx, "backfilled new columns for %d jobs so far", jobsBackfilled)
206+
log.Infof(ctx, "backfilled new columns for %d of %d jobs so far", jobsBackfilled, totalRows)
207+
if jobID := d.OptionalJobID; jobID != 0 {
208+
frac := float32(jobsBackfilled) / float32(totalRows)
209+
if err := d.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
210+
// It would be nice to just ProgressStorage(jobID).Set() here, but
211+
// the new table is not used by reads until this backfill is marked
212+
// as complete, so we still need to use the old LoadJob/WithTxn API.
213+
j, err := d.JobRegistry.LoadClaimedJob(ctx, jobID)
214+
if err != nil {
215+
return err
216+
}
217+
return j.WithTxn(txn).FractionProgressed(ctx, jobs.FractionUpdater(frac))
218+
}); err != nil {
219+
log.Warningf(ctx, "failed to update progress for job %d: %v", jobID, err)
220+
}
221+
}
196222
}
197223
}
198224
}

0 commit comments

Comments
 (0)
Please sign in to comment.