Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
76983: sql/schemachanger: enable add column by default in declarative schema changer r=fqazi a=fqazi

These changes will enable add column by default in the declarative schema changer and the following fixes:

1. Add support for retrying schema changes when retriable errors are encountered
2. Add support for refreshing stats during declarative schema changes
3. Block add column when cross-database references are observed in the declarative schema changer
4. Block implicit record types inside expressions for the declarative schema changer
5. Add schema change-related event log entries
6. Properly combine multiple declarative schema change operations inside CTEs, which is required for support geometry related builtins that add columns
7. Enforce schema usage related privileges in the declarative schema changer
8. Add add column related telemetry
9. Generate appropriate errors when a newly added column conflicts with a system column
10. Support empty column family names
11. Add columns in the user-specified order inside the declarative schema changer
12. Enforce constraints when adding new columns, specifically detecting if default values will violate them

Along with the following known issues and limitations (tracked by issue: #80545)
1. Legacy schema changer will be used whenever serial and generated columns are observed.
2. Regional by row tables will default to the legacy schema changer since we are missing proper support for updating zone configs
3. Columns with the unique constraint will default to the legacy schema changer
4. Default expressions using sequences will default to the legacy schema changer
5. A number of tests cases depending on the MVCC backfiller are using the legacy schema changer for now

81077: ui: update dates to 24h UTC r=maryliag a=maryliag

This commit updates all dates to use a 24h format in UTC

Fixes #78442

Release note (ui change): Update all dates to use 24h format in UTC

81371: tree: extend disallow_imports_test r=ajwerner a=otan

Now blocking roachpb and util/log.

Release note: None

81381: bazel: augment `disallowed_imports_test` to allow disallowing `c-deps` r=ajwerner a=rickystewart

Due to bitrot some of these assertions are already failing. I've filed
the follow-up bugs #81375, #81378, and #81380 for these cases.

I also replaced the `VerifyNoImports` test used by a few packages with
`disallowed_imports_test`.

Release note: None

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
5 people committed May 17, 2022
5 parents e851f9f + ada7157 + 8027298 + 9ca7c59 + dde568a commit 9e64c79
Show file tree
Hide file tree
Showing 145 changed files with 7,891 additions and 1,133 deletions.
10 changes: 9 additions & 1 deletion pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ ALL_TESTS = [
"//pkg/cli/clisqlshell:clisqlshell_test",
"//pkg/cli/democluster:democluster_test",
"//pkg/cli/exit:exit_test",
"//pkg/cli:cli_disallowed_imports_test",
"//pkg/cli:cli_test",
"//pkg/cloud/amazon:amazon_test",
"//pkg/cloud/azure:azure_test",
Expand All @@ -82,7 +83,7 @@ ALL_TESTS = [
"//pkg/clusterversion:clusterversion_test",
"//pkg/cmd/bazci:bazci_test",
"//pkg/cmd/cmpconn:cmpconn_test",
"//pkg/cmd/cockroach-oss:cockroach-oss_test",
"//pkg/cmd/cockroach-oss:cockroach-oss_disallowed_imports_test",
"//pkg/cmd/dev:dev_test",
"//pkg/cmd/docgen/extract:extract_test",
"//pkg/cmd/docs-issue-generation:docs-issue-generation_test",
Expand Down Expand Up @@ -132,6 +133,7 @@ ALL_TESTS = [
"//pkg/jobs:jobs_test",
"//pkg/keys:keys_test",
"//pkg/kv/bulk:bulk_test",
"//pkg/kv/kvclient/kvcoord:kvcoord_disallowed_imports_test",
"//pkg/kv/kvclient/kvcoord:kvcoord_test",
"//pkg/kv/kvclient/kvstreamer:kvstreamer_test",
"//pkg/kv/kvclient/rangecache:rangecache_test",
Expand Down Expand Up @@ -178,6 +180,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/uncertainty:uncertainty_test",
"//pkg/kv/kvserver:kvserver_test",
"//pkg/kv:kv_test",
"//pkg/roachpb:roachpb_disallowed_imports_test",
"//pkg/roachpb:roachpb_test",
"//pkg/roachpb:string_test",
"//pkg/roachprod/cloud:cloud_test",
Expand All @@ -196,6 +199,7 @@ ALL_TESTS = [
"//pkg/server/debug/goroutineui:goroutineui_test",
"//pkg/server/debug/pprofui:pprofui_test",
"//pkg/server/debug:debug_test",
"//pkg/server/diagnostics:diagnostics_disallowed_imports_test",
"//pkg/server/diagnostics:diagnostics_test",
"//pkg/server/dumpstore:dumpstore_test",
"//pkg/server/goroutinedumper:goroutinedumper_test",
Expand Down Expand Up @@ -242,6 +246,7 @@ ALL_TESTS = [
"//pkg/sql/catalog/systemschema_test:systemschema_test_test",
"//pkg/sql/catalog/tabledesc:tabledesc_test",
"//pkg/sql/catalog/typedesc:typedesc_test",
"//pkg/sql/catalog:catalog_disallowed_imports_test",
"//pkg/sql/catalog:catalog_test",
"//pkg/sql/colcontainer:colcontainer_test",
"//pkg/sql/colconv:colconv_disallowed_imports_test",
Expand All @@ -267,6 +272,7 @@ ALL_TESTS = [
"//pkg/sql/colexec/colexecspan:colexecspan_disallowed_imports_test",
"//pkg/sql/colexec/colexecspan:colexecspan_test",
"//pkg/sql/colexec/colexectestutils:colexectestutils_test",
"//pkg/sql/colexec/colexecutils:colexecutils_disallowed_imports_test",
"//pkg/sql/colexec/colexecutils:colexecutils_test",
"//pkg/sql/colexec/colexecwindow:colexecwindow_disallowed_imports_test",
"//pkg/sql/colexec/colexecwindow:colexecwindow_test",
Expand Down Expand Up @@ -342,6 +348,7 @@ ALL_TESTS = [
"//pkg/sql/opt/testutils:testutils_test",
"//pkg/sql/opt/xform:xform_test",
"//pkg/sql/opt:opt_test",
"//pkg/sql/parser:parser_disallowed_imports_test",
"//pkg/sql/parser:parser_test",
"//pkg/sql/pgwire/hba:hba_test",
"//pkg/sql/pgwire/identmap:identmap_test",
Expand Down Expand Up @@ -406,6 +413,7 @@ ALL_TESTS = [
"//pkg/sql/ttl/ttljob:ttljob_test",
"//pkg/sql/types:types_disallowed_imports_test",
"//pkg/sql/types:types_test",
"//pkg/sql:sql_disallowed_imports_test",
"//pkg/sql:sql_test",
"//pkg/startupmigrations/leasemanager:leasemanager_test",
"//pkg/startupmigrations:startupmigrations_test",
Expand Down
6 changes: 3 additions & 3 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -5,9 +5,9 @@ exp,benchmark
13,AlterTableAddCheckConstraint/alter_table_add_1_check_constraint
13,AlterTableAddCheckConstraint/alter_table_add_2_check_constraints
13,AlterTableAddCheckConstraint/alter_table_add_3_check_constraints
13,AlterTableAddColumn/alter_table_add_1_column
13,AlterTableAddColumn/alter_table_add_2_columns
13,AlterTableAddColumn/alter_table_add_3_columns
15,AlterTableAddColumn/alter_table_add_1_column
15,AlterTableAddColumn/alter_table_add_2_columns
15,AlterTableAddColumn/alter_table_add_3_columns
17,AlterTableAddForeignKey/alter_table_add_1_foreign_key
21,AlterTableAddForeignKey/alter_table_add_2_foreign_keys
25,AlterTableAddForeignKey/alter_table_add_3_foreign_keys
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/changefeedccl/alter_changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -305,7 +305,7 @@ func TestAlterChangefeedErrors(t *testing.T) {

sqlDB.Exec(t, `ALTER TABLE bar ADD COLUMN b INT`)
var alterTableJobID jobspb.JobID
sqlDB.QueryRow(t, `SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE'`).Scan(&alterTableJobID)
sqlDB.QueryRow(t, `SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'NEW SCHEMA CHANGE'`).Scan(&alterTableJobID)
sqlDB.ExpectErr(t,
fmt.Sprintf(`job %d is not changefeed job`, alterTableJobID),
fmt.Sprintf(`ALTER CHANGEFEED %d ADD bar`, alterTableJobID),
Expand Down Expand Up @@ -984,6 +984,7 @@ func TestAlterChangefeedAddTargetsDuringSchemaChangeError(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

knobs := f.Server().(*server.TestServer).Cfg.TestingKnobs.
DistSQL.(*execinfra.TestingKnobs).
Expand Down
8 changes: 8 additions & 0 deletions pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -896,6 +896,7 @@ func TestChangefeedUserDefinedTypes(t *testing.T) {
defer leaktest.AfterTest(t)()
testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)
// Set up a type and table.
sqlDB.Exec(t, `CREATE TYPE t AS ENUM ('hello', 'howdy', 'hi')`)
sqlDB.Exec(t, `CREATE TABLE tt (x INT PRIMARY KEY, y t)`)
Expand Down Expand Up @@ -1011,6 +1012,7 @@ func TestChangefeedSchemaChangeNoBackfill(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

// Schema changes that predate the changefeed.
t.Run(`historical`, func(t *testing.T) {
Expand Down Expand Up @@ -1340,6 +1342,7 @@ func TestChangefeedSchemaChangeBackfillCheckpoint(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

knobs := f.Server().(*server.TestServer).Cfg.TestingKnobs.
DistSQL.(*execinfra.TestingKnobs).
Expand Down Expand Up @@ -1577,6 +1580,7 @@ func TestChangefeedSchemaChangeAllowBackfill(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

// Expected semantics:
//
Expand Down Expand Up @@ -1766,6 +1770,7 @@ func TestChangefeedSchemaChangeBackfillScope(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

t.Run(`add column with default`, func(t *testing.T) {
sqlDB.Exec(t, `CREATE TABLE add_column_def (a INT PRIMARY KEY)`)
Expand Down Expand Up @@ -2050,6 +2055,7 @@ func TestChangefeedSingleColumnFamilySchemaChanges(t *testing.T) {
testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {

sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

// Table with 2 column families.
sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING, c STRING, FAMILY most (a,b), FAMILY rest (c))`)
Expand Down Expand Up @@ -2091,6 +2097,7 @@ func TestChangefeedEachColumnFamilySchemaChanges(t *testing.T) {

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)

// Table with 2 column families.
sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING, c STRING, FAMILY f1 (a,b), FAMILY f2 (c))`)
Expand Down Expand Up @@ -2737,6 +2744,7 @@ func TestChangefeedNoBackfill(t *testing.T) {
skip.UnderShort(t)
testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)
// Shorten the intervals so this test doesn't take so long. We need to wait
// for timestamps to get resolved.
sqlDB.Exec(t, "SET CLUSTER SETTING changefeed.experimental_poll_interval = '200ms'")
Expand Down
9 changes: 8 additions & 1 deletion pkg/ccl/changefeedccl/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,18 @@ import (

var testSinkFlushFrequency = 100 * time.Millisecond

// disableDeclarativeSchemaChangesForTest tests that are disabled due to differences
// in changefeed behaviour and are tracked by issue #80545.
func disableDeclarativeSchemaChangesForTest(t testing.TB, sqlDB *sqlutils.SQLRunner) {
sqlDB.Exec(t, "SET use_declarative_schema_changer='off'")
sqlDB.Exec(t, "SET CLUSTER SETTING sql.defaults.use_declarative_schema_changer='off'")
}

func waitForSchemaChange(
t testing.TB, sqlDB *sqlutils.SQLRunner, stmt string, arguments ...interface{},
) {
sqlDB.Exec(t, stmt, arguments...)
row := sqlDB.QueryRow(t, "SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE' ORDER BY created DESC LIMIT 1")
row := sqlDB.QueryRow(t, "SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'NEW SCHEMA CHANGE' OR job_type ='SCHEMA CHANGE' ORDER BY created DESC LIMIT 1")
var jobID string
row.Scan(&jobID)

Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/changefeedccl/nemeses_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)
Expand All @@ -27,6 +28,8 @@ func TestChangefeedNemeses(t *testing.T) {
skip.UnderRace(t, "takes >1 min under race")

testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) {
sqlDB := sqlutils.MakeSQLRunner(db)
disableDeclarativeSchemaChangesForTest(t, sqlDB)
// TODO(dan): Ugly hack to disable `eventPause` in sinkless feeds. See comment in
// `RunNemesis` for details.
isSinkless := strings.Contains(t.Name(), "sinkless")
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/schemafeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/sql/catalog/lease",
"//pkg/sql/catalog/typedesc",
"//pkg/sql/execinfra",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
Expand Down
11 changes: 11 additions & 0 deletions pkg/ccl/changefeedccl/schemafeed/table_event_filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -175,6 +176,16 @@ func pkChangeMutationExists(desc catalog.TableDescriptor) bool {
return true
}
}
// For declarative schema changer check if we are trying to add
// a primary index.
if desc.GetDeclarativeSchemaChangerState() != nil {
for idx, target := range desc.GetDeclarativeSchemaChangerState().Targets {
if target.PrimaryIndex != nil &&
desc.GetDeclarativeSchemaChangerState().CurrentStatuses[idx] != scpb.Status_PUBLIC {
return true
}
}
}
return false
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/cliccl/debug_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -498,9 +498,9 @@ func TestExportDataWithMultipleRanges(t *testing.T) {
sqlDB.Exec(t, fmt.Sprintf(`BACKUP TABLE fooTable TO $1 AS OF SYSTEM TIME '%s'`, ts.AsOfSystemTime()), backupPath)

sqlDB.QueryRow(t, `SELECT count(*) from [SHOW RANGES from TABLE fooTable]`).Scan(&rangeNum)
require.Equal(t, 4, rangeNum)
require.Equal(t, 1, rangeNum)
sqlDB.QueryRow(t, `SELECT count(*) from [SHOW BACKUP FILES $1]`, backupPath).Scan(&rangeNum)
require.Equal(t, 8, rangeNum)
require.Equal(t, 5, rangeNum)

t.Run("export-data-with-multiple-ranges", func(t *testing.T) {
setDebugContextDefault()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,27 +185,35 @@ query idx=2 wait-until-follower-read
SELECT * FROM t AS OF SYSTEM TIME with_max_staleness('10s', true) WHERE pk = 1
----
1
events (7 found):
events (11 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
* event 3: colbatchscan trace on node_idx 2: local read
* event 4: transaction retry on node_idx: 2
* event 5: colbatchscan trace on node_idx 2: local read
* event 6: transaction retry on node_idx: 2
* event 7: colbatchscan trace on node_idx 2: local follower read
* event 7: colbatchscan trace on node_idx 2: local read
* event 8: transaction retry on node_idx: 2
* event 9: colbatchscan trace on node_idx 2: local read
* event 10: transaction retry on node_idx: 2
* event 11: colbatchscan trace on node_idx 2: local follower read

query idx=2
SELECT * FROM t AS OF SYSTEM TIME with_min_timestamp(now() - '10s', true) WHERE pk = 1
----
1
events (7 found):
events (11 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
* event 3: colbatchscan trace on node_idx 2: local read
* event 4: transaction retry on node_idx: 2
* event 5: colbatchscan trace on node_idx 2: local read
* event 6: transaction retry on node_idx: 2
* event 7: colbatchscan trace on node_idx 2: local follower read
* event 7: colbatchscan trace on node_idx 2: local read
* event 8: transaction retry on node_idx: 2
* event 9: colbatchscan trace on node_idx 2: local read
* event 10: transaction retry on node_idx: 2
* event 11: colbatchscan trace on node_idx 2: local follower read

# When creating a new table, ensure when nearest_only=True, we correctly error
# with the schema not existing if none of the followers have caught up.
Expand All @@ -225,24 +233,32 @@ query idx=2
SELECT * FROM t2 AS OF SYSTEM TIME with_min_timestamp(now() - '10s', true) WHERE pk = 2
----
pq: referenced descriptor ID 105: descriptor not found
events (7 found):
events (11 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
* event 3: colbatchscan trace on node_idx 2: local read
* event 4: transaction retry on node_idx: 2
* event 5: colbatchscan trace on node_idx 2: local read
* event 6: transaction retry on node_idx: 2
* event 7: colbatchscan trace on node_idx 2: local read
* event 8: transaction retry on node_idx: 2
* event 9: colbatchscan trace on node_idx 2: local read
* event 10: transaction retry on node_idx: 2
* event 11: colbatchscan trace on node_idx 2: local read

query idx=2
SELECT * FROM t2 AS OF SYSTEM TIME with_min_timestamp(now() - '10s', true) WHERE pk = 2
----
pq: referenced descriptor ID 105: descriptor not found
events (7 found):
events (11 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
* event 3: colbatchscan trace on node_idx 2: local read
* event 4: transaction retry on node_idx: 2
* event 5: colbatchscan trace on node_idx 2: local read
* event 6: transaction retry on node_idx: 2
* event 7: colbatchscan trace on node_idx 2: local read
* event 8: transaction retry on node_idx: 2
* event 9: colbatchscan trace on node_idx 2: local read
* event 10: transaction retry on node_idx: 2
* event 11: colbatchscan trace on node_idx 2: local read
1 change: 0 additions & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/tenant_settings
Original file line number Diff line number Diff line change
Expand Up @@ -199,4 +199,3 @@ query I
SELECT count(*) FROM system.tenant_settings WHERE tenant_id = 1234
----
0

7 changes: 6 additions & 1 deletion pkg/ccl/multiregionccl/regional_by_row_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,7 +394,12 @@ USE t;
if err := sqltestutils.BulkInsertIntoTable(sqlDB, maxValue); err != nil {
t.Fatal(err)
}

// Disable declarative schema changer for the add column, since
// the backfill callbacks required are not currently used (they
// need the MVCC compliant backfiller).
if _, err := sqlDB.Exec(`SET use_declarative_schema_changer='off'`); err != nil {
t.Fatal(err)
}
// We add the "cr" column, which can be used for REGIONAL BY ROW AS.
if _, err := sqlDB.Exec(`
ALTER TABLE t.test ADD COLUMN cr t.crdb_internal_region
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/schemachangerccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ go_test(
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/schemachanger/scrun",
"//pkg/sql/schemachanger/scexec",
"//pkg/sql/schemachanger/sctest",
"//pkg/testutils",
"//pkg/testutils/serverutils",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/schemachangerccl/schemachanger_ccl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/build/bazel"
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/sctest"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func newCluster(t *testing.T, knobs *scrun.TestingKnobs) (*gosql.DB, func()) {
func newCluster(t *testing.T, knobs *scexec.TestingKnobs) (*gosql.DB, func()) {
_, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(
t, 3 /* numServers */, base.TestingKnobs{
SQLDeclarativeSchemaChanger: knobs,
Expand Down
Loading

0 comments on commit 9e64c79

Please sign in to comment.