diff --git a/README.md b/README.md index 005a4f38c9cc..8d398cd89646 100644 --- a/README.md +++ b/README.md @@ -104,6 +104,14 @@ Guide](https://www.cockroachlabs.com/docs/stable/architecture/overview.html). For the original design motivation, see our [design doc](https://github.com/cockroachdb/cockroach/blob/master/docs/design.md). +## Licensing + +Current CockroachDB code is released under a combination of two licenses, the [Business Source License (BSL)](https://www.cockroachlabs.com/docs/stable/licensing.html#bsl) and the [Cockroach Community License (CCL)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-free). + +When contributing to a CockroachDB feature, you can find the relevant license in the comments at the top of each file. + +For more information, see the [Licensing FAQs](https://www.cockroachlabs.com/docs/stable/licensing-faqs.html). + ## Comparison with Other Databases To see how key features of CockroachDB stack up against other databases, diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index fd830a090539..529a7dcc8745 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2638,6 +2638,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)

crdb_internal.num_inverted_index_entries(val: jsonb, version: int) → int

This function is used only by CockroachDB’s developers for testing purposes.

+crdb_internal.payloads_for_span(span ID: int) → jsonb

Returns the payload(s) of the span whose ID is passed in the argument.

+
crdb_internal.pretty_key(raw_key: bytes, skip_fields: int) → string

This function is used only by CockroachDB’s developers for testing purposes.

crdb_internal.range_stats(key: bytes) → jsonb

This function is used to retrieve range statistics information as a JSON object.

diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 74381ec046d9..2737bfcc126c 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -19,6 +19,7 @@ ALL_TESTS = [ "//pkg/ccl/kvccl/kvfollowerreadsccl:kvfollowerreadsccl_test", "//pkg/ccl/kvccl/kvtenantccl:kvtenantccl_test", "//pkg/ccl/logictestccl:logictestccl_test", + "//pkg/ccl/multiregionccl:multiregionccl_test", "//pkg/ccl/oidcccl:oidcccl_test", "//pkg/ccl/partitionccl:partitionccl_test", "//pkg/ccl/serverccl:serverccl_test", diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel index a7e53be1281f..c802b884a91c 100644 --- a/pkg/ccl/BUILD.bazel +++ b/pkg/ccl/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/ccl/gssapiccl", "//pkg/ccl/importccl", "//pkg/ccl/kvccl", + "//pkg/ccl/multiregionccl", "//pkg/ccl/oidcccl", "//pkg/ccl/partitionccl", "//pkg/ccl/storageccl", diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index d7ba8059a94e..a6a16f42a213 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -426,7 +426,7 @@ func WriteDescriptors( } for _, db := range databases { - if err := db.Validate(); err != nil { + if err := db.Validate(ctx, dg); err != nil { return errors.Wrapf(err, "validate database %d", errors.Safe(db.GetID())) } diff --git a/pkg/ccl/ccl_init.go b/pkg/ccl/ccl_init.go index e5a03cc061f3..50ccee16324a 100644 --- a/pkg/ccl/ccl_init.go +++ b/pkg/ccl/ccl_init.go @@ -21,6 +21,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/gssapiccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/importccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/oidcccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 143ea539011a..0ffac4e65b9a 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -59,7 +59,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) { if err != nil { return nil, err } - return mutDesc, mutDesc.ValidateTable(ctx) + return mutDesc, mutDesc.ValidateSelf(ctx) } func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.EncDatumRow, error) { diff --git a/pkg/ccl/logictestccl/testdata/logic_test/alter_table_locality b/pkg/ccl/logictestccl/testdata/logic_test/alter_table_locality index 02cd9db3f722..b00d11f73981 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/alter_table_locality +++ b/pkg/ccl/logictestccl/testdata/logic_test/alter_table_locality @@ -1749,27 +1749,66 @@ DATABASE alter_locality_test ALTER DATABASE alter_locality_test CONFIGURE ZONE voter_constraints = '[+region=ca-central-1]', lease_preferences = '[[+region=ca-central-1]]' -# TODO(#60606): this is flaky because REGIONAL BY ROW CREATE TABLE statements -# do not round trip. - -#statement ok -#CREATE TABLE regional_by_row_to_regional_by_row_as ( -# pk INT PRIMARY KEY, -# i INT, -# cr crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2', -# INDEX(i), -# FAMILY (pk, i) -#) LOCALITY REGIONAL BY ROW; -#INSERT INTO regional_by_row_to_regional_by_row_as (pk, i) VALUES (1, 1); -#ALTER TABLE regional_by_row_to_regional_by_row_as SET LOCALITY REGIONAL BY ROW AS "cr" - -#query TT -#SHOW CREATE TABLE regional_by_row_to_regional_by_row_as -#---- - -#query TT -#SHOW ZONE CONFIGURATION FOR TABLE regional_by_row_to_regional_by_row_as -#---- +statement ok +CREATE TABLE regional_by_row_to_regional_by_row_as ( + pk INT PRIMARY KEY, + i INT, + cr crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2', + INDEX(i), + FAMILY (pk, i) +) LOCALITY REGIONAL BY ROW; +INSERT INTO regional_by_row_to_regional_by_row_as (pk, i) VALUES (1, 1); +ALTER TABLE regional_by_row_to_regional_by_row_as SET LOCALITY REGIONAL BY ROW AS "cr" + +query TT +SHOW CREATE TABLE regional_by_row_to_regional_by_row_as +---- +regional_by_row_to_regional_by_row_as CREATE TABLE public.regional_by_row_to_regional_by_row_as ( + pk INT8 NOT NULL, + i INT8 NULL, + cr public.crdb_internal_region NOT NULL DEFAULT 'ap-southeast-2':::public.crdb_internal_region, + crdb_region public.crdb_internal_region NOT VISIBLE NOT NULL DEFAULT default_to_database_primary_region(gateway_region())::public.crdb_internal_region, + CONSTRAINT "primary" PRIMARY KEY (pk ASC), + INDEX regional_by_row_to_regional_by_row_as_i_idx (i ASC), + FAMILY fam_0_pk_i_cr_crdb_region (pk, i, cr, crdb_region) +) LOCALITY REGIONAL BY ROW AS cr; +ALTER PARTITION "ap-southeast-2" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=ap-southeast-2]', + lease_preferences = '[[+region=ap-southeast-2]]'; +ALTER PARTITION "ap-southeast-2" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=ap-southeast-2]', + lease_preferences = '[[+region=ap-southeast-2]]'; +ALTER PARTITION "ca-central-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=ca-central-1]', + lease_preferences = '[[+region=ca-central-1]]'; +ALTER PARTITION "ca-central-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=ca-central-1]', + lease_preferences = '[[+region=ca-central-1]]'; +ALTER PARTITION "us-east-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@primary CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]'; +ALTER PARTITION "us-east-1" OF INDEX alter_locality_test.public.regional_by_row_to_regional_by_row_as@regional_by_row_to_regional_by_row_as_i_idx CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + +query TT +SHOW ZONE CONFIGURATION FOR TABLE regional_by_row_to_regional_by_row_as +---- +DATABASE alter_locality_test ALTER DATABASE alter_locality_test CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 5, + num_voters = 3, + constraints = '{+region=ap-southeast-2: 1, +region=ca-central-1: 1, +region=us-east-1: 1}', + voter_constraints = '[+region=ca-central-1]', + lease_preferences = '[[+region=ca-central-1]]' # Altering from REGIONAL BY ROW AS diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row index 0c2ad06a0d51..750904afb215 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row @@ -16,14 +16,6 @@ CREATE TABLE regional_by_row_table ( PARTITION BY LIST (pk) (PARTITION one VALUES IN ((1))) LOCALITY REGIONAL BY ROW -statement error cannot specify crdb_region column in REGIONAL BY ROW table as the column is implicitly created by the system -CREATE TABLE regional_by_row_table ( - pk int, - a int, - crdb_region crdb_internal_region -) -LOCALITY REGIONAL BY ROW - statement error REGIONAL BY ROW on a TABLE containing PARTITION BY is not supported CREATE TABLE regional_by_row_table ( pk int @@ -72,6 +64,46 @@ CREATE TABLE regional_by_row_table ( ) LOCALITY REGIONAL BY ROW +statement error cannot use column crdb_region which has type INT8 in REGIONAL BY ROW\nDETAIL: Column crdb_internal_region must be of type crdb_internal_region +CREATE TABLE regional_by_row_table ( + pk int, + a int, + crdb_region int +) +LOCALITY REGIONAL BY ROW + +statement ok +CREATE TABLE regional_by_row_table_explicit_crdb_region_column ( + pk int PRIMARY KEY, + a int, + crdb_region crdb_internal_region, + FAMILY (pk, a, crdb_region) +) +LOCALITY REGIONAL BY ROW + +query T +SELECT create_statement FROM [SHOW CREATE TABLE regional_by_row_table_explicit_crdb_region_column] +---- +CREATE TABLE public.regional_by_row_table_explicit_crdb_region_column ( + pk INT8 NOT NULL, + a INT8 NULL, + crdb_region public.crdb_internal_region NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (pk ASC), + FAMILY fam_0_pk_a_crdb_region (pk, a, crdb_region) +) LOCALITY REGIONAL BY ROW; +ALTER PARTITION "ap-southeast-2" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING + num_voters = 5, + voter_constraints = '{+region=ap-southeast-2: 2}', + lease_preferences = '[[+region=ap-southeast-2]]'; +ALTER PARTITION "ca-central-1" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING + num_voters = 5, + voter_constraints = '{+region=ca-central-1: 2}', + lease_preferences = '[[+region=ca-central-1]]'; +ALTER PARTITION "us-east-1" OF INDEX multi_region_test_db.public.regional_by_row_table_explicit_crdb_region_column@primary CONFIGURE ZONE USING + num_voters = 5, + voter_constraints = '{+region=us-east-1: 2}', + lease_preferences = '[[+region=us-east-1]]' + statement ok CREATE TABLE regional_by_row_table ( pk int PRIMARY KEY, @@ -169,8 +201,9 @@ regional_by_row_table_j_idx j false query TTTTIT colnames SHOW TABLES ---- -schema_name table_name type owner estimated_row_count locality -public regional_by_row_table table root 0 REGIONAL BY ROW +schema_name table_name type owner estimated_row_count locality +public regional_by_row_table table root 0 REGIONAL BY ROW +public regional_by_row_table_explicit_crdb_region_column table root 0 REGIONAL BY ROW query TI INSERT INTO regional_by_row_table (pk, pk2, a, b, j) VALUES @@ -843,7 +876,7 @@ ALTER PARTITION "us-east-1" OF INDEX multi_region_test_db.public.regional_by_row lease_preferences = '[[+region=us-east-1]]' # Tests for REGIONAL BY TABLE AS -statement error cannot use column crdb_region_col which has type INT8 in REGIONAL BY ROW AS\nDETAIL:\s+REGIONAL BY ROW AS must reference a column of type crdb_internal_region. +statement error cannot use column crdb_region_col which has type INT8 in REGIONAL BY ROW\nDETAIL: REGIONAL BY ROW AS must reference a column of type crdb_internal_region CREATE TABLE regional_by_row_table_as ( pk int PRIMARY KEY, crdb_region_col int diff --git a/pkg/ccl/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel new file mode 100644 index 000000000000..f6d4006e8373 --- /dev/null +++ b/pkg/ccl/multiregionccl/BUILD.bazel @@ -0,0 +1,42 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "multiregionccl", + srcs = ["multiregion.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl", + visibility = ["//visibility:public"], +) + +go_test( + name = "multiregionccl_test", + srcs = [ + "main_test.go", + "regional_by_row_test.go", + "show_test.go", + ], + deps = [ + "//pkg/base", + "//pkg/ccl/partitionccl", + "//pkg/ccl/utilccl", + "//pkg/jobs", + "//pkg/keys", + "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/catalogkv", + "//pkg/sql/execinfra", + "//pkg/sql/sqltestutils", + "//pkg/sql/tests", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/ccl/multiregionccl/main_test.go b/pkg/ccl/multiregionccl/main_test.go new file mode 100644 index 000000000000..3074c894b061 --- /dev/null +++ b/pkg/ccl/multiregionccl/main_test.go @@ -0,0 +1,33 @@ +// 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 multiregionccl_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + defer utilccl.TestingEnableEnterprise()() + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/ccl/multiregionccl/multiregion.go b/pkg/ccl/multiregionccl/multiregion.go new file mode 100644 index 000000000000..723467949137 --- /dev/null +++ b/pkg/ccl/multiregionccl/multiregion.go @@ -0,0 +1,9 @@ +// 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 multiregionccl diff --git a/pkg/ccl/partitionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go similarity index 99% rename from pkg/ccl/partitionccl/regional_by_row_test.go rename to pkg/ccl/multiregionccl/regional_by_row_test.go index 1c5d96a3156a..0fdf86aafd63 100644 --- a/pkg/ccl/partitionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package partitionccl_test +package multiregionccl_test import ( "context" @@ -32,7 +32,7 @@ import ( "github.com/stretchr/testify/require" ) -// REGIONAL BY ROW tests are defined in partitionccl as REGIONAL BY ROW +// REGIONAL BY ROW tests are defined in multiregionccl as REGIONAL BY ROW // requires CCL to operate. // TestAlterTableLocalityRegionalByRowError tests an alteration involving diff --git a/pkg/ccl/multiregionccl/show_test.go b/pkg/ccl/multiregionccl/show_test.go new file mode 100644 index 000000000000..8e5ec639061a --- /dev/null +++ b/pkg/ccl/multiregionccl/show_test.go @@ -0,0 +1,114 @@ +// 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 multiregionccl_test + +import ( + "testing" + + // Blank import partitionccl to install CreatePartitioning hook. + _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" + "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestShowCreateTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testCases := []sqltestutils.ShowCreateTableTestCase{ + // Check GLOBAL tables are round trippable. + { + CreateStatement: `CREATE TABLE %s ( + a INT + ) LOCALITY GLOBAL`, + Expect: `CREATE TABLE public.%s ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + FAMILY "primary" (a, rowid) +) LOCALITY GLOBAL`, + Database: "mrdb", + }, + // Check REGIONAL BY TABLE tables are round trippable. + { + CreateStatement: `CREATE TABLE %s ( + a INT + ) LOCALITY REGIONAL BY TABLE`, + Expect: `CREATE TABLE public.%s ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + FAMILY "primary" (a, rowid) +) LOCALITY REGIONAL BY TABLE IN PRIMARY REGION`, + Database: "mrdb", + }, + { + CreateStatement: `CREATE TABLE %s ( + a INT + ) LOCALITY REGIONAL BY TABLE IN "us-west1"`, + Expect: `CREATE TABLE public.%s ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + FAMILY "primary" (a, rowid) +) LOCALITY REGIONAL BY TABLE IN "us-west1"`, + Database: "mrdb", + }, + // Check REGIONAL BY ROW tests are round trippable. + { + CreateStatement: `SET experimental_enable_implicit_column_partitioning = true; CREATE TABLE %s ( + a INT, + INDEX a_idx (a) + ) LOCALITY REGIONAL BY ROW`, + Expect: `CREATE TABLE public.%[1]s ( + a INT8 NULL, + crdb_region public.crdb_internal_region NOT VISIBLE NOT NULL DEFAULT default_to_database_primary_region(gateway_region())::public.crdb_internal_region, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + INDEX a_idx (a ASC), + FAMILY "primary" (a, crdb_region, rowid) +) LOCALITY REGIONAL BY ROW; +ALTER PARTITION "us-west1" OF INDEX mrdb.public.%[1]s@a_idx CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-west1]', + lease_preferences = '[[+region=us-west1]]'; +ALTER PARTITION "us-west1" OF INDEX mrdb.public.%[1]s@primary CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-west1]', + lease_preferences = '[[+region=us-west1]]'`, + Database: "mrdb", + }, + { + CreateStatement: `SET experimental_enable_implicit_column_partitioning = true; CREATE TABLE %s ( + a INT, + crdb_region_col crdb_internal_region, + INDEX a_idx (a) + ) LOCALITY REGIONAL BY ROW AS crdb_region_col`, + Expect: `CREATE TABLE public.%[1]s ( + a INT8 NULL, + crdb_region_col public.crdb_internal_region NOT NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + INDEX a_idx (a ASC), + FAMILY "primary" (a, crdb_region_col, rowid) +) LOCALITY REGIONAL BY ROW AS crdb_region_col; +ALTER PARTITION "us-west1" OF INDEX mrdb.public.%[1]s@a_idx CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-west1]', + lease_preferences = '[[+region=us-west1]]'; +ALTER PARTITION "us-west1" OF INDEX mrdb.public.%[1]s@primary CONFIGURE ZONE USING + num_voters = 3, + voter_constraints = '[+region=us-west1]', + lease_preferences = '[[+region=us-west1]]'`, + Database: "mrdb", + }, + } + sqltestutils.ShowCreateTableTest(t, testCases) +} diff --git a/pkg/ccl/partitionccl/BUILD.bazel b/pkg/ccl/partitionccl/BUILD.bazel index 145fc80ed620..b3cce6124459 100644 --- a/pkg/ccl/partitionccl/BUILD.bazel +++ b/pkg/ccl/partitionccl/BUILD.bazel @@ -32,7 +32,6 @@ go_test( "drop_test.go", "main_test.go", "partition_test.go", - "regional_by_row_test.go", "zone_test.go", ], embed = [":partitionccl"], @@ -43,7 +42,6 @@ go_test( "//pkg/ccl/utilccl", "//pkg/config", "//pkg/config/zonepb", - "//pkg/jobs", "//pkg/keys", "//pkg/kv/kvserver", "//pkg/roachpb", @@ -55,12 +53,10 @@ go_test( "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", - "//pkg/sql/execinfra", "//pkg/sql/gcjob", "//pkg/sql/parser", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", - "//pkg/sql/sqltestutils", "//pkg/sql/tests", "//pkg/sql/types", "//pkg/testutils", @@ -68,7 +64,6 @@ go_test( "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", - "//pkg/util", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/leaktest", @@ -78,7 +73,6 @@ go_test( "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", - "@com_github_stretchr_testify//require", "@in_gopkg_yaml_v2//:yaml_v2", ], ) diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 80673478c7c1..b3daa6209ea5 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -139,7 +139,7 @@ func (pt *partitioningTest) parse() error { return err } pt.parsed.tableDesc = mutDesc - if err := pt.parsed.tableDesc.ValidateTable(ctx); err != nil { + if err := pt.parsed.tableDesc.ValidateSelf(ctx); err != nil { return err } } diff --git a/pkg/sql/catalog/catalogkv/catalogkv.go b/pkg/sql/catalog/catalogkv/catalogkv.go index e1c6c455bdb2..94cd53859a6f 100644 --- a/pkg/sql/catalog/catalogkv/catalogkv.go +++ b/pkg/sql/catalog/catalogkv/catalogkv.go @@ -280,21 +280,6 @@ func (t *oneLevelUncachedDescGetter) GetDescs( var _ catalog.DescGetter = (*oneLevelUncachedDescGetter)(nil) -func validateDescriptor(ctx context.Context, dg catalog.DescGetter, desc catalog.Descriptor) error { - switch desc := desc.(type) { - case catalog.TableDescriptor: - return desc.Validate(ctx, dg) - case catalog.DatabaseDescriptor: - return desc.Validate() - case catalog.TypeDescriptor: - return desc.Validate(ctx, dg) - case catalog.SchemaDescriptor: - return nil - default: - return errors.AssertionFailedf("unknown descriptor type %T", desc) - } -} - // unwrapDescriptor takes a descriptor retrieved using a transaction and unwraps // it into an immutable implementation of Descriptor. It ensures that // the ModificationTime is set properly and will validate the descriptor if @@ -327,7 +312,7 @@ func unwrapDescriptor( return nil, nil } if validate { - if err := validateDescriptor(ctx, dg, unwrapped); err != nil { + if err := unwrapped.Validate(ctx, dg); err != nil { return nil, err } } @@ -350,13 +335,13 @@ func unwrapDescriptorMutable( if err != nil { return nil, err } - if err := mutTable.ValidateTable(ctx); err != nil { + if err := mutTable.ValidateSelf(ctx); err != nil { return nil, err } return mutTable, nil case database != nil: dbDesc := dbdesc.NewExistingMutable(*database) - if err := dbDesc.Validate(); err != nil { + if err := dbDesc.Validate(ctx, dg); err != nil { return nil, err } return dbDesc, nil @@ -432,7 +417,7 @@ func GetAllDescriptors( dg[desc.GetID()] = desc } for _, desc := range descs { - if err := validateDescriptor(ctx, dg, desc); err != nil { + if err := desc.Validate(ctx, dg); err != nil { return nil, err } } @@ -603,6 +588,7 @@ func getDescriptorsFromIDs( if err := txn.Run(ctx, b); err != nil { return nil, err } + dg := NewOneLevelUncachedDescGetter(txn, codec) results := make([]catalog.Descriptor, 0, len(ids)) for i := range b.Results { result := &b.Results[i] @@ -624,7 +610,7 @@ func getDescriptorsFromIDs( var catalogDesc catalog.Descriptor if desc.Union != nil { var err error - catalogDesc, err = unwrapDescriptor(ctx, nil /* descGetter */, result.Rows[0].Value.Timestamp, desc, true) + catalogDesc, err = unwrapDescriptor(ctx, dg, result.Rows[0].Value.Timestamp, desc, true) if err != nil { return nil, err } diff --git a/pkg/sql/catalog/catalogkv/unwrap_validation_test.go b/pkg/sql/catalog/catalogkv/unwrap_validation_test.go index 4f4a53d9adea..1cde4832356b 100644 --- a/pkg/sql/catalog/catalogkv/unwrap_validation_test.go +++ b/pkg/sql/catalog/catalogkv/unwrap_validation_test.go @@ -87,20 +87,6 @@ func (o oneLevelMapDescGetter) GetDesc( return unwrapDescriptorMutable(ctx, nil, mt, &desc) } -func (o oneLevelMapDescGetter) GetDescs( - ctx context.Context, reqs []descpb.ID, -) ([]catalog.Descriptor, error) { - resps := make([]catalog.Descriptor, len(reqs)) - for i, r := range reqs { - var err error - resps[i], err = o.GetDesc(ctx, r) - if err != nil { - return nil, err - } - } - return resps, nil -} - func decodeDescriptorDSV(t *testing.T, descriptorCSVPath string) oneLevelMapDescGetter { f, err := os.Open(descriptorCSVPath) require.NoError(t, err) diff --git a/pkg/sql/catalog/dbdesc/BUILD.bazel b/pkg/sql/catalog/dbdesc/BUILD.bazel index d081c8bbea18..7a8dfb7fb5ec 100644 --- a/pkg/sql/catalog/dbdesc/BUILD.bazel +++ b/pkg/sql/catalog/dbdesc/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/privilege", "//pkg/util/hlc", + "//pkg/util/iterutil", "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/sql/catalog/dbdesc/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go index e9c5b9958a4f..1d93197052e3 100644 --- a/pkg/sql/catalog/dbdesc/database_desc.go +++ b/pkg/sql/catalog/dbdesc/database_desc.go @@ -13,6 +13,7 @@ package dbdesc import ( + "context" "fmt" "github.com/cockroachdb/cockroach/pkg/keys" @@ -21,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -250,10 +252,26 @@ func (desc *Mutable) SetName(name string) { desc.Name = name } -// Validate validates that the database descriptor is well formed. +// ForEachSchemaInfo iterates f over each schema info mapping in the descriptor. +// iterutil.StopIteration is supported. +func (desc *Immutable) ForEachSchemaInfo( + f func(id descpb.ID, name string, isDropped bool) error, +) error { + for name, info := range desc.Schemas { + if err := f(info.ID, name, info.Dropped); err != nil { + if iterutil.Done(err) { + return nil + } + return err + } + } + return nil +} + +// ValidateSelf validates that the database descriptor is well formed. // Checks include validate the database name, and verifying that there // is at least one read and write user. -func (desc *Immutable) Validate() error { +func (desc *Immutable) ValidateSelf(_ context.Context) error { if err := catalog.ValidateName(desc.GetName(), "descriptor"); err != nil { return err } @@ -294,6 +312,16 @@ func (desc *Immutable) Validate() error { return desc.Privileges.Validate(desc.GetID(), privilege.Database) } +// Validate punts to ValidateSelf. +func (desc *Immutable) Validate(ctx context.Context, _ catalog.DescGetter) error { + return desc.ValidateSelf(ctx) +} + +// ValidateTxnCommit punts to Validate. +func (desc *Immutable) ValidateTxnCommit(ctx context.Context, descGetter catalog.DescGetter) error { + return desc.Validate(ctx, descGetter) +} + // SchemaMeta implements the tree.SchemaMeta interface. // TODO (rohany): I don't want to keep this here, but it seems to be used // by backup only for the fake resolution that occurs in backup. Is it possible diff --git a/pkg/sql/catalog/desc_getter.go b/pkg/sql/catalog/desc_getter.go index 1973da85ffbf..00d550082de5 100644 --- a/pkg/sql/catalog/desc_getter.go +++ b/pkg/sql/catalog/desc_getter.go @@ -20,9 +20,32 @@ import ( // is used to look up other descriptors during validation. type DescGetter interface { GetDesc(ctx context.Context, id descpb.ID) (Descriptor, error) +} + +// BatchDescGetter is like DescGetter but retrieves batches of descriptors, +// which for some implementation may make more sense performance-wise. +type BatchDescGetter interface { GetDescs(ctx context.Context, reqs []descpb.ID) ([]Descriptor, error) } +// GetDescs retrieves multiple descriptors using a DescGetter. +// If the latter is also a BatchDescGetter, it will delegate to its GetDescs +// method. +func GetDescs(ctx context.Context, descGetter DescGetter, reqs []descpb.ID) ([]Descriptor, error) { + if bdg, ok := descGetter.(BatchDescGetter); ok { + return bdg.GetDescs(ctx, reqs) + } + ret := make([]Descriptor, len(reqs)) + for i, id := range reqs { + desc, err := descGetter.GetDesc(ctx, id) + if err != nil { + return nil, err + } + ret[i] = desc + } + return ret, nil +} + // GetTypeDescFromID retrieves the type descriptor for the type ID passed // in using an existing descGetter. It returns an error if the descriptor // doesn't exist or if it exists and is not a type descriptor. @@ -62,12 +85,3 @@ func (m MapDescGetter) GetDesc(ctx context.Context, id descpb.ID) (Descriptor, e desc := m[id] return desc, nil } - -// GetDescs implements the catalog.DescGetter interface. -func (m MapDescGetter) GetDescs(ctx context.Context, ids []descpb.ID) ([]Descriptor, error) { - ret := make([]Descriptor, len(ids)) - for i, id := range ids { - ret[i], _ = m.GetDesc(ctx, id) - } - return ret, nil -} diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 1172dd990706..fb414d7c1f50 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -72,6 +72,15 @@ type Descriptor interface { // DescriptorProto prepares this descriptor for serialization. DescriptorProto() *descpb.Descriptor + + // ValidateSelf checks the internal consistency of the descriptor. + ValidateSelf(ctx context.Context) error + + // Validate is like ValidateSelf but with additional cross-reference checks. + Validate(ctx context.Context, descGetter DescGetter) error + + // ValidateTxnCommit is like Validate but with additional pre-commit checks. + ValidateTxnCommit(ctx context.Context, descGetter DescGetter) error } // DatabaseDescriptor will eventually be called dbdesc.Descriptor. @@ -90,8 +99,8 @@ type DatabaseDescriptor interface { RegionNames() (descpb.RegionNames, error) IsMultiRegion() bool PrimaryRegionName() (descpb.RegionName, error) - Validate() error MultiRegionEnumID() (descpb.ID, error) + ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error) error } // SchemaDescriptor will eventually be called schemadesc.Descriptor. @@ -245,8 +254,6 @@ type TableDescriptor interface { databaseDesc DatabaseDescriptor, getType func(descpb.ID) (TypeDescriptor, error), ) (descpb.IDs, error) - Validate(ctx context.Context, txn DescGetter) error - ForeachDependedOnBy(f func(dep *descpb.TableDescriptor_Reference) error) error GetDependsOn() []descpb.ID GetConstraintInfoWithLookup(fn TableLookupFn) (map[string]descpb.ConstraintDetail, error) @@ -493,7 +500,6 @@ type TypeDescriptor interface { PrimaryRegionName() (descpb.RegionName, error) RegionNames() (descpb.RegionNames, error) - Validate(ctx context.Context, dg DescGetter) error } // TypeDescriptorResolver is an interface used during hydration of type diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index cc3535989933..5dccdb65b64a 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", @@ -65,6 +66,7 @@ go_test( "//pkg/sql/catalog/tabledesc", "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", + "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index c12a2d31006a..7678d47935c9 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -1342,13 +1343,25 @@ func (tc *Collection) addUncommittedDescriptor( return ud, nil } +// validateOnWriteEnabled is the cluster setting used to enable or disable +// validating descriptors prior to writing. +var validateOnWriteEnabled = settings.RegisterBoolSetting( + "sql.catalog.descs.validate_on_write.enabled", + "set to true to validate descriptors prior to writing, false to disable; default is true", + true, /* defaultValue */ +) + // WriteDescToBatch calls MaybeIncrementVersion, adds the descriptor to the // collection as an uncommitted descriptor, and writes it into b. func (tc *Collection) WriteDescToBatch( ctx context.Context, kvTrace bool, desc catalog.MutableDescriptor, b *kv.Batch, ) error { desc.MaybeIncrementVersion() - // TODO(ajwerner): Add validation here. + if validateOnWriteEnabled.Get(&tc.settings.SV) { + if err := desc.ValidateSelf(ctx); err != nil { + return err + } + } if err := tc.AddUncommittedDescriptor(desc); err != nil { return err } @@ -1392,6 +1405,43 @@ func (tc *Collection) GetUncommittedTables() (tables []catalog.TableDescriptor) return tables } +type collectionDescGetter struct { + tc *Collection + txn *kv.Txn +} + +var _ catalog.DescGetter = collectionDescGetter{} + +func (cdg collectionDescGetter) GetDesc( + ctx context.Context, id descpb.ID, +) (catalog.Descriptor, error) { + flags := tree.CommonLookupFlags{ + Required: true, + // Include everything, we want to cast the net as wide as we can. + IncludeOffline: true, + IncludeDropped: true, + // Avoid leased descriptors, if we're leasing the previous version then this + // older version may be returned and this may cause validation to fail. + AvoidCached: true, + } + return cdg.tc.getDescriptorByID(ctx, cdg.txn, id, flags, false /* mutable */) +} + +// ValidateUncommittedDescriptors validates all uncommitted descriptors +func (tc *Collection) ValidateUncommittedDescriptors(ctx context.Context, txn *kv.Txn) error { + if !validateOnWriteEnabled.Get(&tc.settings.SV) { + return nil + } + cdg := collectionDescGetter{tc: tc, txn: txn} + for i, n := 0, len(tc.uncommittedDescriptors); i < n; i++ { + desc := tc.uncommittedDescriptors[i].immutable + if err := desc.ValidateTxnCommit(ctx, cdg); err != nil { + return err + } + } + return nil +} + // User defined type accessors. // GetMutableTypeVersionByID is the equivalent of GetMutableTableDescriptorByID diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index dcd976c2adc8..5c60529d742e 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -65,11 +67,36 @@ func TestCollectionWriteDescToBatch(t *testing.T) { // We want to create some descriptors and then ensure that writing them to a // batch works as expected. newTable := tabledesc.NewCreatedMutable(descpb.TableDescriptor{ - ID: 42, + ID: 142, Name: "table2", Version: 1, ParentID: mut.GetParentID(), UnexposedParentSchemaID: mut.GetParentSchemaID(), + Columns: []descpb.ColumnDescriptor{ + {ID: 1, Name: "a", Type: types.Int}, + }, + Families: []descpb.ColumnFamilyDescriptor{ + { + ID: 0, + Name: "primary", + ColumnNames: []string{"a"}, + ColumnIDs: []descpb.ColumnID{1}, + DefaultColumnID: 1, + }, + }, + PrimaryIndex: descpb.IndexDescriptor{ + ID: 1, + Name: "pk", + ColumnIDs: []descpb.ColumnID{1}, + ColumnNames: []string{"a"}, + ColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + }, + Privileges: descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()), + NextColumnID: 2, + NextFamilyID: 1, + NextIndexID: 2, + NextMutationID: 1, + FormatVersion: descpb.FamilyFormatVersion, }) b := txn.NewBatch() diff --git a/pkg/sql/catalog/descs/txn.go b/pkg/sql/catalog/descs/txn.go index a0102c12c949..13d561cb5e2a 100644 --- a/pkg/sql/catalog/descs/txn.go +++ b/pkg/sql/catalog/descs/txn.go @@ -52,6 +52,9 @@ func Txn( if err := f(ctx, txn, descsCol); err != nil { return err } + if err := descsCol.ValidateUncommittedDescriptors(ctx, txn); err != nil { + return err + } retryErr, err := CheckTwoVersionInvariant( ctx, db.Clock(), ie, descsCol, txn, nil /* onRetryBackoff */) if retryErr { diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 1309f60cede3..9977661bfe7b 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -229,7 +229,7 @@ func (s storage) acquire( } // TODO (lucy): Previously this called getTableDescFromID followed by a call - // to ValidateTable() instead of Validate(), to avoid the cross-table + // to ValidateSelf() instead of Validate(), to avoid the cross-table // checks. Does this actually matter? We already potentially do cross-table // checks when populating pre-19.2 foreign keys. desc, err := catalogkv.GetDescriptorByID(ctx, txn, s.codec, id, catalogkv.Immutable, diff --git a/pkg/sql/catalog/schemadesc/BUILD.bazel b/pkg/sql/catalog/schemadesc/BUILD.bazel index fc8394d636b2..e730a535b450 100644 --- a/pkg/sql/catalog/schemadesc/BUILD.bazel +++ b/pkg/sql/catalog/schemadesc/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/privilege", "//pkg/sql/sessiondata", "//pkg/util/hlc", "//pkg/util/protoutil", diff --git a/pkg/sql/catalog/schemadesc/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go index 8f2154f1db20..499599ad865b 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc.go +++ b/pkg/sql/catalog/schemadesc/schema_desc.go @@ -11,6 +11,8 @@ package schemadesc import ( + "context" + "fmt" "strings" "github.com/cockroachdb/cockroach/pkg/keys" @@ -18,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -165,6 +168,77 @@ func (desc *Immutable) DescriptorProto() *descpb.Descriptor { } } +// ValidateSelf implements the catalog.Descriptor interface. +func (desc *Immutable) ValidateSelf(_ context.Context) error { + if err := catalog.ValidateName(desc.GetName(), "descriptor"); err != nil { + return err + } + if desc.GetID() == 0 { + return fmt.Errorf("invalid schema ID %d", desc.GetID()) + } + // Validate the privilege descriptor. + return desc.Privileges.Validate(desc.GetID(), privilege.Schema) +} + +// Validate implements the catalog.Descriptor interface. +func (desc *Immutable) Validate(ctx context.Context, descGetter catalog.DescGetter) error { + if err := desc.ValidateSelf(ctx); err != nil { + return err + } + // Don't validate cross-references for dropped schemas. + if desc.Dropped() || descGetter == nil { + return nil + } + + // Check schema parent reference. + foundDesc, err := descGetter.GetDesc(ctx, desc.GetParentID()) + if err != nil { + return err + } + db, isDB := foundDesc.(catalog.DatabaseDescriptor) + if !isDB { + return errors.AssertionFailedf("parent database ID %d does not exist", errors.Safe(desc.GetParentID())) + } + + // Check that parent has correct entry in schemas mapping. + isInDBSchemas := false + err = db.ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error { + if id == desc.GetID() { + if isDropped { + if name == desc.GetName() { + return errors.AssertionFailedf("present in parent database [%d] schemas mapping but marked as dropped", + errors.Safe(desc.GetParentID())) + } + return nil + } + if name != desc.GetName() { + return errors.AssertionFailedf("present in parent database [%d] schemas mapping but under name %q", + errors.Safe(desc.GetParentID()), errors.Safe(name)) + } + isInDBSchemas = true + return nil + } + if !isDropped && name == desc.GetName() { + return errors.AssertionFailedf("present in parent database [%d] schemas mapping but name maps to other schema [%d]", + errors.Safe(desc.GetParentID()), errors.Safe(id)) + } + return nil + }) + if err != nil { + return err + } + if !isInDBSchemas { + return errors.AssertionFailedf("not present in parent database [%d] schemas mapping", + errors.Safe(desc.GetParentID())) + } + return nil +} + +// ValidateTxnCommit punts to Validate. +func (desc *Immutable) ValidateTxnCommit(ctx context.Context, descGetter catalog.DescGetter) error { + return desc.Validate(ctx, descGetter) +} + // NameResolutionResult implements the ObjectDescriptor interface. func (desc *Immutable) NameResolutionResult() {} diff --git a/pkg/sql/catalog/tabledesc/helpers_test.go b/pkg/sql/catalog/tabledesc/helpers_test.go index 024f573dfcef..ccc39c92522c 100644 --- a/pkg/sql/catalog/tabledesc/helpers_test.go +++ b/pkg/sql/catalog/tabledesc/helpers_test.go @@ -22,7 +22,7 @@ func ValidateTable(ctx context.Context, immI catalog.TableDescriptor) error { if !ok { return errors.Errorf("expected immutable descriptor") } - return imm.ValidateTable(ctx) + return imm.ValidateSelf(ctx) } func ValidateCrossReferences( diff --git a/pkg/sql/catalog/tabledesc/safe_format_test.go b/pkg/sql/catalog/tabledesc/safe_format_test.go index 4f841a072d3a..b83870f25d07 100644 --- a/pkg/sql/catalog/tabledesc/safe_format_test.go +++ b/pkg/sql/catalog/tabledesc/safe_format_test.go @@ -265,7 +265,7 @@ func TestSafeMessage(t *testing.T) { td = desc } redacted := string(redact.Sprint(td).Redact()) - require.NoError(t, desc.ValidateTable(ctx)) + require.NoError(t, desc.ValidateSelf(ctx)) require.Equal(t, tc.exp, redacted) var m map[string]interface{} require.NoError(t, yaml.UnmarshalStrict([]byte(redacted), &m), redacted) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 8c45360d39db..f312fd02a2f8 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -945,7 +945,7 @@ func (desc *Mutable) AllocateIDs(ctx context.Context) error { if desc.ID == 0 { desc.ID = keys.MinUserDescID } - err := desc.ValidateTable(ctx) + err := desc.ValidateSelf(ctx) desc.ID = savedID return err } @@ -1258,17 +1258,39 @@ type testingDescriptorValidation bool // ensure testing specific descriptor validation happens. var PerformTestingDescriptorValidation testingDescriptorValidation = true -// Validate validates that the table descriptor is well formed. Checks include -// both single table and cross table invariants. -func (desc *wrapper) Validate(ctx context.Context, dg catalog.DescGetter) error { - err := desc.ValidateTable(ctx) - if err != nil { +// Validate performs ValidateSelf and then validates that +// each reference to another table is resolvable and that the necessary back +// references exist. +func (desc *wrapper) Validate(ctx context.Context, descGetter catalog.DescGetter) error { + if err := desc.ValidateSelf(ctx); err != nil { + return err + } + if desc.Dropped() || descGetter == nil { + return nil + } + + return errors.Wrapf(desc.validateCrossReferences(ctx, descGetter), "desc %d", desc.GetID()) +} + +// ValidateTxnCommit performs Validate and then performs additional +// pre-transaction-commit checks. +func (desc *wrapper) ValidateTxnCommit(ctx context.Context, descGetter catalog.DescGetter) error { + if err := desc.Validate(ctx, descGetter); err != nil { return err } if desc.Dropped() { return nil } - return errors.Wrapf(desc.validateCrossReferences(ctx, dg), "desc %d", desc.GetID()) + // Pre-transaction commit table validations. + + // Check that primary key exists. + if !desc.HasPrimaryKey() { + return unimplemented.NewWithIssuef(48026, + "primary key of table %s dropped without subsequent addition of new primary key", + desc.GetName()) + } + + return nil } // validateTableIfTesting is similar to validateTable, except it is only invoked @@ -1823,13 +1845,13 @@ func (desc *wrapper) ValidateIndexNameIsUnique(indexName string) error { return nil } -// ValidateTable validates that the table descriptor is well formed. Checks +// ValidateSelf validates that the table descriptor is well formed. Checks // include validating the table, column and index names, verifying that column // names and index names are unique and verifying that column IDs and index IDs // are consistent. Use Validate to validate that cross-table references are // correct. // If version is supplied, the descriptor is checked for version incompatibilities. -func (desc *wrapper) ValidateTable(ctx context.Context) error { +func (desc *wrapper) ValidateSelf(ctx context.Context) error { if err := catalog.ValidateName(desc.Name, "table"); err != nil { return err } diff --git a/pkg/sql/catalog/tabledesc/structured_test.go b/pkg/sql/catalog/tabledesc/structured_test.go index 3b0de3a118b2..27e9ca6cc9f1 100644 --- a/pkg/sql/catalog/tabledesc/structured_test.go +++ b/pkg/sql/catalog/tabledesc/structured_test.go @@ -138,6 +138,7 @@ func TestAllocateIDs(t *testing.T) { } func TestValidateDatabaseDesc(t *testing.T) { defer leaktest.AfterTest(t)() + ctx := context.Background() testData := []struct { err string @@ -195,7 +196,7 @@ func TestValidateDatabaseDesc(t *testing.T) { } for i, d := range testData { t.Run(d.err, func(t *testing.T) { - if err := d.desc.Validate(); err == nil { + if err := d.desc.Validate(ctx, nil /* descGetter */); err == nil { t.Errorf("%d: expected \"%s\", but found success: %+v", i, d.err, d.desc) } else if d.err != err.Error() && "internal error: "+d.err != err.Error() { t.Errorf("%d: expected \"%s\", but found \"%+v\"", i, d.err, err) diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 88ed46ef5379..ce3da9a2aeb4 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -434,8 +434,8 @@ func isBeingDropped(member *descpb.TypeDescriptor_EnumMember) bool { member.Direction == descpb.TypeDescriptor_EnumMember_REMOVE } -// Validate performs validation on the TypeDescriptor. -func (desc *Immutable) Validate(ctx context.Context, dg catalog.DescGetter) error { +// ValidateSelf performs validation on the TypeDescriptor. +func (desc *Immutable) ValidateSelf(_ context.Context) error { // Validate local properties of the descriptor. if err := catalog.ValidateName(desc.Name, "type"); err != nil { return err @@ -524,8 +524,18 @@ func (desc *Immutable) Validate(ctx context.Context, dg catalog.DescGetter) erro } } + return nil +} + +// Validate performs ValidateSelf followed by +// cross reference checks on the descriptor. +func (desc *Immutable) Validate(ctx context.Context, descGetter catalog.DescGetter) error { + if err := desc.ValidateSelf(ctx); err != nil { + return err + } + // Don't validate cross-references for dropped descriptors. - if desc.Dropped() { + if desc.Dropped() || descGetter == nil { return nil } @@ -663,14 +673,13 @@ func (desc *Immutable) Validate(ctx context.Context, dg catalog.DescGetter) erro } } if !desc.Dropped() { - for _, id := range desc.ReferencingDescriptorIDs { reqs = append(reqs, id) checks = append(checks, tableExists(id)) } } - descs, err := dg.GetDescs(ctx, reqs) + descs, err := catalog.GetDescs(ctx, descGetter, reqs) if err != nil { return err } @@ -685,6 +694,11 @@ func (desc *Immutable) Validate(ctx context.Context, dg catalog.DescGetter) erro return nil } +// ValidateTxnCommit punts to Validate. +func (desc *Immutable) ValidateTxnCommit(ctx context.Context, descGetter catalog.DescGetter) error { + return desc.Validate(ctx, descGetter) +} + // TypeLookupFunc is a type alias for a function that looks up a type by ID. type TypeLookupFunc func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index af1bd1544032..d54eeaf6d254 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -38,7 +38,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/duration" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -749,7 +748,7 @@ func (ex *connExecutor) commitSQLTransactionInternal( } } - if err := validatePrimaryKeys(&ex.extraTxnState.descCollection); err != nil { + if err := ex.extraTxnState.descCollection.ValidateUncommittedDescriptors(ctx, ex.state.mu.txn); err != nil { return err } @@ -770,22 +769,6 @@ func (ex *connExecutor) commitSQLTransactionInternal( return nil } -// validatePrimaryKeys verifies that all tables modified in the transaction have -// an enabled primary key after potentially undergoing DROP PRIMARY KEY, which -// is required to be followed by ADD PRIMARY KEY. -func validatePrimaryKeys(tc *descs.Collection) error { - tables := tc.GetUncommittedTables() - for _, table := range tables { - if !table.HasPrimaryKey() { - return unimplemented.NewWithIssuef(48026, - "primary key of table %s dropped without subsequent addition of new primary key", - table.GetName(), - ) - } - } - return nil -} - // rollbackSQLTransaction executes a ROLLBACK statement: the KV transaction is // rolled-back and an event is produced. func (ex *connExecutor) rollbackSQLTransaction(ctx context.Context) (fsm.Event, fsm.EventPayload) { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 040584710734..439e90b1acc8 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -66,7 +66,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" - ptypes "github.com/gogo/protobuf/types" ) // CrdbInternalName is the name of the crdb_internal schema. @@ -1149,7 +1148,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( duration INTERVAL, -- The span's duration, measured from start to Finish(). -- A span whose recording is collected before it's finished will -- have the duration set as the "time of collection - start time". - num_payloads INT NOT NULL, -- The number of structured payloads in this span. operation STRING NULL -- The span's operation. )`, populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { @@ -1177,11 +1175,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( spanDuration := rec.Duration operation := rec.Operation - var numStructured int - rec.Structured(func(any *ptypes.Any) { - numStructured++ - }) - if err := addRow( // TODO(angelapwen): we're casting uint64s to int64 here, // is that ok? @@ -1195,7 +1188,6 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( duration.MakeDuration(spanDuration.Nanoseconds(), 0, 0), types.DefaultIntervalTypeMetadata, ), - tree.NewDInt(tree.DInt(numStructured)), tree.NewDString(operation), ); err != nil { return err diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index b8dc2c0050a0..2fe3b3ed610c 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -128,7 +128,7 @@ func doCreateSequence( } // makeSequenceTableDesc already validates the table. No call to - // desc.ValidateTable() needed here. + // desc.ValidateSelf() needed here. key := catalogkv.MakeObjectNameKey( params.ctx, @@ -231,7 +231,7 @@ func NewSequenceTableDesc( // immediately. desc.State = descpb.DescriptorState_PUBLIC - if err := desc.ValidateTable(ctx); err != nil { + if err := desc.ValidateSelf(ctx); err != nil { return nil, err } return &desc, nil diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 199e787027ce..15876424efe3 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1543,7 +1543,7 @@ func NewTableDesc( if t.Oid() != typedesc.TypeIDToOID(dbDesc.RegionConfig.RegionEnumID) { err = pgerror.Newf( pgcode.InvalidTableDefinition, - "cannot use column %s which has type %s in REGIONAL BY ROW AS", + "cannot use column %s which has type %s in REGIONAL BY ROW", d.Name, t.SQLString(), ) @@ -1551,11 +1551,24 @@ func NewTableDesc( ctx, typedesc.TypeIDToOID(dbDesc.RegionConfig.RegionEnumID), ); terr == nil { - err = errors.WithDetailf( - err, - "REGIONAL BY ROW AS must reference a column of type %s.", - t.Name(), - ) + if n.Locality.RegionalByRowColumn != tree.RegionalByRowRegionNotSpecifiedName { + // In this case, someone used REGIONAL BY ROW AS where + // col has a non crdb_internal_region type. + err = errors.WithDetailf( + err, + "REGIONAL BY ROW AS must reference a column of type %s", + t.Name(), + ) + } else { + // In this case, someone used REGIONAL BY ROW but also specified + // a crdb_region column that does not have a crdb_internal_region type. + err = errors.WithDetailf( + err, + "Column %s must be of type %s", + t.Name(), + tree.RegionEnum, + ) + } } return nil, err } @@ -1577,16 +1590,11 @@ func NewTableDesc( } } - if n.Locality.RegionalByRowColumn == tree.RegionalByRowRegionNotSpecifiedName { - // Implicitly create REGIONAL BY ROW column if no AS ... was defined. - if regionalByRowColExists { - return nil, errors.WithHintf( - pgerror.Newf( - pgcode.InvalidTableDefinition, - `cannot specify %s column in REGIONAL BY ROW table as the column is implicitly created by the system`, - regionalByRowCol.String(), - ), - "Use LOCALITY REGIONAL BY ROW AS %s instead.", + if !regionalByRowColExists { + if n.Locality.RegionalByRowColumn != tree.RegionalByRowRegionNotSpecifiedName { + return nil, pgerror.Newf( + pgcode.UndefinedColumn, + "column %s in REGIONAL BY ROW AS does not exist", regionalByRowCol.String(), ) } @@ -1596,12 +1604,6 @@ func NewTableDesc( regionalByRowDefaultColDef(oid, regionalByRowGatewayRegionDefaultExpr(oid)), ) columnDefaultExprs = append(columnDefaultExprs, nil) - } else if !regionalByRowColExists { - return nil, pgerror.Newf( - pgcode.UndefinedColumn, - "column %s in REGIONAL BY ROW AS does not exist", - regionalByRowCol.String(), - ) } // Construct the partitioning for the PARTITION ALL BY. @@ -1946,7 +1948,17 @@ func NewTableDesc( if err := idx.FillColumns(d.Columns); err != nil { return nil, err } - if d.PartitionByIndex.ContainsPartitioningClause() || desc.PartitionAllBy { + // Specifying a partitioning on a PRIMARY KEY constraint should be disallowed by the + // syntax, but do a sanity check. + if d.PrimaryKey && d.PartitionByIndex.ContainsPartitioningClause() { + return nil, errors.AssertionFailedf( + "PRIMARY KEY partitioning should be defined at table level", + ) + } + // We should only do partitioning of non-primary indexes at this point - + // the PRIMARY KEY CreatePartitioning is done at the of CreateTable, so + // avoid the duplicate work. + if !d.PrimaryKey && (d.PartitionByIndex.ContainsPartitioningClause() || desc.PartitionAllBy) { partitionBy := partitionAllBy if !desc.PartitionAllBy { if d.PartitionByIndex.ContainsPartitions() { @@ -2093,7 +2105,7 @@ func NewTableDesc( return nil, err } // During CreatePartitioning, implicitly partitioned columns may be - // created. AllocateIDs which allocates ExtraColumnIDs to each index + // created. AllocateIDs which allocates column IDs to each index // needs to be called before CreatePartitioning as CreatePartitioning // requires IDs to be allocated. // diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 49f7bfa231dc..2df5b7ff7c3f 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -87,9 +87,6 @@ func (p *planner) writeNonDropDatabaseChange( func (p *planner) writeDatabaseChangeToBatch( ctx context.Context, desc *dbdesc.Mutable, b *kv.Batch, ) error { - if err := desc.Validate(); err != nil { - return err - } return p.Descriptors().WriteDescToBatch( ctx, p.extendedEvalCtx.Tracing.KVTracingEnabled(), diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index deded748f3dc..64dedf27913b 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -186,20 +186,23 @@ func (p *planner) createDescriptorWithID( } case *tabledesc.Mutable: isTable = true - if err := desc.ValidateTable(ctx); err != nil { + if err := desc.ValidateSelf(ctx); err != nil { return err } if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { return err } case *dbdesc.Mutable: - if err := desc.Validate(); err != nil { + if err := desc.ValidateSelf(ctx); err != nil { return err } if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { return err } case *schemadesc.Mutable: + if err := desc.ValidateSelf(ctx); err != nil { + return err + } if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { return err } diff --git a/pkg/sql/descriptor_mutation_test.go b/pkg/sql/descriptor_mutation_test.go index eb74fb4129af..4c391806d265 100644 --- a/pkg/sql/descriptor_mutation_test.go +++ b/pkg/sql/descriptor_mutation_test.go @@ -85,7 +85,7 @@ func (mt mutationTest) makeMutationsActive(ctx context.Context) { } mt.tableDesc.Mutations = nil mt.tableDesc.Version++ - if err := mt.tableDesc.ValidateTable(ctx); err != nil { + if err := mt.tableDesc.ValidateSelf(ctx); err != nil { mt.Fatal(err) } if err := mt.kvDB.Put( @@ -145,7 +145,7 @@ func (mt mutationTest) writeMutation(ctx context.Context, m descpb.DescriptorMut } mt.tableDesc.Mutations = append(mt.tableDesc.Mutations, m) mt.tableDesc.Version++ - if err := mt.tableDesc.ValidateTable(ctx); err != nil { + if err := mt.tableDesc.ValidateSelf(ctx); err != nil { mt.Fatal(err) } if err := mt.kvDB.Put( @@ -455,21 +455,21 @@ CREATE INDEX allidx ON t.test (k, v); // Check that a mutation can only be inserted with an explicit mutation state, and direction. tableDesc = mTest.tableDesc tableDesc.Mutations = []descpb.DescriptorMutation{{}} - if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, and no column/index descriptor") { + if err := tableDesc.ValidateSelf(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, and no column/index descriptor") { t.Fatal(err) } tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Column{Column: &tableDesc.Columns[len(tableDesc.Columns)-1]}}} tableDesc.Columns = tableDesc.Columns[:len(tableDesc.Columns)-1] - if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, `mutation in state UNKNOWN, direction NONE, col "i", id 3`) { + if err := tableDesc.ValidateSelf(ctx); !testutils.IsError(err, `mutation in state UNKNOWN, direction NONE, col "i", id 3`) { t.Fatal(err) } tableDesc.Mutations[0].State = descpb.DescriptorMutation_DELETE_ONLY - if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, `mutation in state DELETE_ONLY, direction NONE, col "i", id 3`) { + if err := tableDesc.ValidateSelf(ctx); !testutils.IsError(err, `mutation in state DELETE_ONLY, direction NONE, col "i", id 3`) { t.Fatal(err) } tableDesc.Mutations[0].State = descpb.DescriptorMutation_UNKNOWN tableDesc.Mutations[0].Direction = descpb.DescriptorMutation_DROP - if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, `mutation in state UNKNOWN, direction DROP, col "i", id 3`) { + if err := tableDesc.ValidateSelf(ctx); !testutils.IsError(err, `mutation in state UNKNOWN, direction DROP, col "i", id 3`) { t.Fatal(err) } } @@ -645,7 +645,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR, INDEX foo (v)); index := tableDesc.PublicNonPrimaryIndexes()[len(tableDesc.PublicNonPrimaryIndexes())-1] tableDesc.Mutations = []descpb.DescriptorMutation{{Descriptor_: &descpb.DescriptorMutation_Index{Index: index.IndexDesc()}}} tableDesc.RemovePublicNonPrimaryIndex(index.Ordinal()) - if err := tableDesc.ValidateTable(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, index foo, id 2") { + if err := tableDesc.ValidateSelf(ctx); !testutils.IsError(err, "mutation in state UNKNOWN, direction NONE, index foo, id 2") { t.Fatal(err) } } diff --git a/pkg/sql/logictest/testdata/logic_test/contention_event b/pkg/sql/logictest/testdata/logic_test/contention_event index ede473134a10..42f6c904223e 100644 --- a/pkg/sql/logictest/testdata/logic_test/contention_event +++ b/pkg/sql/logictest/testdata/logic_test/contention_event @@ -41,18 +41,24 @@ ROLLBACK user root -# Check that the number of payloads in the open trace is at least 1. -# TODO(angelapwen): when we have a way to pull particular payloads for -# a trace, we should verify that we're seeing the right contention event. -# As is, the payloads could be something else (though we verified manually -# and there is a contention event). +# Check that there is at least 1 contention event payload in all spans in the +# open trace. # # NB: the contention event is not in our trace span but in one of its # children, so it wouldn't be found if we filtered by the trace span ID. # # NB: this needs the 5node-pretend59315 config because otherwise the span is not # tracked. +# query B -SELECT count(num_payloads) > 0 FROM crdb_internal.node_inflight_trace_spans WHERE trace_id = crdb_internal.trace_id(); +WITH spans AS ( + SELECT span_id FROM crdb_internal.node_inflight_trace_spans + WHERE trace_id = crdb_internal.trace_id() +), payload_types AS ( + SELECT jsonb_array_elements(crdb_internal.payloads_for_span(span_id))->>'@type' AS payload_type + FROM spans +) SELECT count(*) > 0 + FROM payload_types + WHERE payload_type = 'type.googleapis.com/cockroach.roachpb.ContentionEvent'; ---- true diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index d6c36a2eba62..2540e27c4068 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -250,10 +250,10 @@ SELECT * FROM crdb_internal.zones WHERE false zone_id subzone_id target range_name database_name table_name index_name partition_name raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql -query IIIIBTTIT colnames +query IIIIBTIT colnames SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- -trace_id parent_span_id span_id goroutine_id finished start_time duration num_payloads operation +trace_id parent_span_id span_id goroutine_id finished start_time duration operation query ITTTTITTTTTTTTTI colnames SELECT * FROM crdb_internal.ranges WHERE range_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index c54eda79e560..6b4dfbe29fe7 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -262,10 +262,10 @@ SELECT * FROM crdb_internal.zones WHERE false zone_id subzone_id target range_name database_name table_name index_name partition_name raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql -query IIIIBTTIT colnames +query IIIIBTIT colnames SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- -trace_id parent_span_id span_id goroutine_id finished start_time duration num_payloads operation +trace_id parent_span_id span_id goroutine_id finished start_time duration operation statement error not fully contained in tenant keyspace SELECT * FROM crdb_internal.ranges WHERE range_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/virtual_columns b/pkg/sql/logictest/testdata/logic_test/virtual_columns index 21da631c6411..fce43a8d709c 100644 --- a/pkg/sql/logictest/testdata/logic_test/virtual_columns +++ b/pkg/sql/logictest/testdata/logic_test/virtual_columns @@ -772,3 +772,108 @@ ALTER TABLE sc ADD COLUMN w INT AS (a*b) VIRTUAL CHECK (w <= 100) statement error failed to satisfy CHECK constraint INSERT INTO sc VALUES (20, 20) + +# Test inverted indexes on virtual columns. +subtest InvertedIndexes + +statement ok +CREATE TABLE inv ( + k INT PRIMARY KEY, + i INT, + j JSON, + iv INT AS (i + 10) VIRTUAL, + jv JSON AS (j->'a') VIRTUAL, + INVERTED INDEX jv_idx (jv), + INVERTED INDEX i_jv_idx (i, jv), + INVERTED INDEX iv_j_idx (iv, j), + INVERTED INDEX iv_jv_idx (iv, jv) +) + +statement ok +INSERT INTO inv VALUES + (1, 10, NULL), + (2, 10, '1'), + (3, 10, '"a"'), + (4, 10, 'true'), + (5, 10, 'null'), + (6, 10, '{}'), + (7, 10, '[]'), + (8, 10, '{"a": "b"}'), + (9, 10, '{"a": "b", "c": "d"}'), + (10, 10, '{"a": {}, "b": "c"}'), + (11, 10, '{"a": {"b": "c"}, "d": "e"}'), + (12, 10, '{"a": {"b": "c", "d": "e"}}'), + (13, 10, '{"a": [], "d": "e"}'), + (14, 10, '{"a": ["b", "c"], "d": "e"}'), + (15, 10, '["a"]'), + (16, 10, '["a", "b", "c"]'), + (17, 10, '[{"a": "b"}, "c"]') + +statement ok +INSERT INTO inv +SELECT k+17, 20, j FROM inv + +query IT +SELECT k, jv FROM inv@jv_idx WHERE jv @> '{"b": "c"}' ORDER BY k +---- +11 {"b": "c"} +12 {"b": "c", "d": "e"} +28 {"b": "c"} +29 {"b": "c", "d": "e"} + +query IT +SELECT k, jv FROM inv@jv_idx WHERE jv->'b' = '"c"' ORDER BY k +---- +11 {"b": "c"} +12 {"b": "c", "d": "e"} +28 {"b": "c"} +29 {"b": "c", "d": "e"} + +query IT +SELECT k, jv FROM inv@jv_idx WHERE jv @> '"b"' ORDER BY k +---- +8 "b" +9 "b" +14 ["b", "c"] +25 "b" +26 "b" +31 ["b", "c"] + +query IIT +SELECT k, i, jv FROM inv@i_jv_idx WHERE i IN (10, 20, 30) AND jv @> '{"b": "c"}' ORDER BY k +---- +11 10 {"b": "c"} +12 10 {"b": "c", "d": "e"} +28 20 {"b": "c"} +29 20 {"b": "c", "d": "e"} + +query IIT +SELECT k, i, jv FROM inv@i_jv_idx WHERE i = 20 AND jv @> '{"b": "c"}' ORDER BY k +---- +28 20 {"b": "c"} +29 20 {"b": "c", "d": "e"} + +query IIT +SELECT k, iv, j FROM inv@iv_j_idx WHERE iv IN (10, 20, 30) AND j @> '{"b": "c"}' ORDER BY k +---- +10 20 {"a": {}, "b": "c"} +27 30 {"a": {}, "b": "c"} + +query IIT +SELECT k, iv, j FROM inv@iv_j_idx WHERE iv = 20 AND j @> '{"b": "c"}' ORDER BY k +---- +10 20 {"a": {}, "b": "c"} + +query IIT +SELECT k, iv, jv FROM inv@iv_jv_idx WHERE iv IN (10, 20, 30) AND jv @> '{"b": "c"}' ORDER BY k +---- +11 20 {"b": "c"} +12 20 {"b": "c", "d": "e"} +28 30 {"b": "c"} +29 30 {"b": "c", "d": "e"} + +query IIT +SELECT k, iv, jv FROM inv@iv_jv_idx WHERE iv = 20 AND jv @> '{"b": "c"}' ORDER BY k +---- +11 20 {"b": "c"} +12 20 {"b": "c", "d": "e"} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/virtual_columns b/pkg/sql/opt/exec/execbuilder/testdata/virtual_columns index 69ca0af3a1ba..b2e0d2f73eeb 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/virtual_columns +++ b/pkg/sql/opt/exec/execbuilder/testdata/virtual_columns @@ -1365,3 +1365,70 @@ vectorized: true estimated row count: 60 (missing stats) table: checks@checks_v_b_idx spans: /0/10-/0/16 /1/10-/1/16 /2/10-/2/16 /3/10-/3/16 + +subtest InvertedIndexes + +statement ok +CREATE TABLE inv ( + k INT PRIMARY KEY, + i INT, + j JSON, + iv INT AS (i + 10) VIRTUAL, + jv JSON AS (j->'a') VIRTUAL, + INVERTED INDEX jv_idx (jv), + INVERTED INDEX i_jv_idx (i, jv), + INVERTED INDEX iv_j_idx (iv, j), + INVERTED INDEX iv_jv_idx (iv, jv) +) + +# Verify that we use jv_idx. +query T +EXPLAIN (VERBOSE) SELECT k FROM inv WHERE jv @> '{"a": "b"}' +---- +distribution: local +vectorized: true +· +• scan + columns: (k) + estimated row count: 111 (missing stats) + table: inv@jv_idx + spans: /"a"/"b"-/"a"/"b"/PrefixEnd + +# Verify that we use i_jv_idx. +query T +EXPLAIN (VERBOSE) SELECT k FROM inv WHERE i IN (10, 20, 30) AND jv @> '{"a": "b"}' +---- +distribution: local +vectorized: true +· +• scan + columns: (k) + estimated row count: 3 (missing stats) + table: inv@i_jv_idx + spans: /10/"a"/"b"-/10/"a"/"b"/PrefixEnd /20/"a"/"b"-/20/"a"/"b"/PrefixEnd /30/"a"/"b"-/30/"a"/"b"/PrefixEnd + +# Verify that we use iv_j_idx. +query T +EXPLAIN (VERBOSE) SELECT k FROM inv WHERE iv IN (10, 20, 30) AND j @> '{"a": "b"}' +---- +distribution: local +vectorized: true +· +• scan + columns: (k) + estimated row count: 3 (missing stats) + table: inv@iv_j_idx + spans: /10/"a"/"b"-/10/"a"/"b"/PrefixEnd /20/"a"/"b"-/20/"a"/"b"/PrefixEnd /30/"a"/"b"-/30/"a"/"b"/PrefixEnd + +# Verify that we use iv_jv_idx. +query T +EXPLAIN (VERBOSE) SELECT k FROM inv WHERE iv IN (10, 20, 30) AND jv @> '{"a": "b"}' +---- +distribution: local +vectorized: true +· +• scan + columns: (k) + estimated row count: 3 (missing stats) + table: inv@iv_jv_idx + spans: /10/"a"/"b"-/10/"a"/"b"/PrefixEnd /20/"a"/"b"-/20/"a"/"b"/PrefixEnd /30/"a"/"b"-/30/"a"/"b"/PrefixEnd diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index e4b9d3c5c610..436fb9277dc8 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -49,15 +49,14 @@ import ( // write to different descriptors. It will also validate the structure of the // descriptor but not its references. // -// TODO(ajwerner): It is critical that we not validate all of the relevant -// descriptors during statement execution as it may be the case that more than -// one descriptor is corrupt. Instead, we should validate all of the relevant -// descriptors just prior to committing the transaction. This would bring the -// requirement that if a descriptor is upserted, that it leave the database in -// a valid state, at least in terms of that descriptor and its references. -// Perhaps transactions which do end up using this should also end up validating -// all descriptors at the end of the transaction to ensure that this operation -// didn't break a reference to this descriptor. +// It is critical that we not validate all of the relevant descriptors during +// statement execution as it may be the case that more than one descriptor is +// corrupt. Instead, we rely on ValidateTxnCommit which runs just prior to +// committing any transaction. This brings the requirement that if a descriptor +// is to be upserted, it must leave the database in a valid state, at least in +// terms of that descriptor and its references. This validation can be disabled +// via the `sql.catalog.descs.validate_on_write.enabled` cluster setting if need +// be, even though such a need is rather not obvious to foresee. func (p *planner) UnsafeUpsertDescriptor( ctx context.Context, descID int64, encodedDesc []byte, force bool, ) error { @@ -242,7 +241,9 @@ func comparePrivileges( cur := curUserMap[username] granted, revoked := computePrivilegeChanges(prev, cur) delete(curUserMap, username) - + if granted == nil && revoked == nil { + continue + } // Log events. if err := logPrivilegeEvents( ctx, p, existing, granted, revoked, username, @@ -256,6 +257,9 @@ func comparePrivileges( username := curUserPrivileges[i].User().Normalized() if _, ok := curUserMap[username]; ok { granted := privilege.ListFromBitField(curUserPrivileges[i].Privileges, objectType).SortedNames() + if granted == nil { + continue + } if err := logPrivilegeEvents( ctx, p, existing, granted, nil, username, ); err != nil { @@ -552,6 +556,8 @@ func (p *planner) UnsafeDeleteNamespaceEntry( // This method will perform very minimal validation. An error will be returned // if no such descriptor exists. This method can very easily introduce // corruption, beware. +// +// See UnsafeUpsertDescriptor for additional details, and warnings. func (p *planner) UnsafeDeleteDescriptor(ctx context.Context, descID int64, force bool) error { const method = "crdb_internal.unsafe_delete_descriptor()" if err := checkPlannerStateForRepairFunctions(ctx, p, method); err != nil { diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 95131fc532d6..16132ece9c21 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -816,20 +816,6 @@ func (l *internalLookupCtx) GetDesc(ctx context.Context, id descpb.ID) (catalog. return nil, nil } -func (l *internalLookupCtx) GetDescs( - ctx context.Context, reqs []descpb.ID, -) ([]catalog.Descriptor, error) { - ret := make([]catalog.Descriptor, len(reqs)) - for i := 0; i < len(reqs); i++ { - var err error - ret[i], err = l.GetDesc(ctx, reqs[i]) - if err != nil { - return nil, err - } - } - return ret, nil -} - // tableLookupFn can be used to retrieve a table descriptor and its corresponding // database descriptor using the table's ID. type tableLookupFn = *internalLookupCtx diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index a5ebf361e423..9b9f69983c46 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1038,11 +1038,6 @@ func WaitToUpdateLeasesMultiple( // It also kicks off GC jobs as needed. func (sc *SchemaChanger) done(ctx context.Context) error { - // Get the other tables whose foreign key backreferences need to be removed. - // We also have to handle the situation to add Foreign Key backreferences. - var fksByBackrefTable map[descpb.ID][]*descpb.ConstraintToUpdate - var interleaveParents map[descpb.ID]struct{} - var referencedTypeIDs []descpb.ID // Jobs (for GC, etc.) that need to be started immediately after the table // descriptor updates are published. var childJobs []*jobs.StartableJob @@ -1051,8 +1046,6 @@ func (sc *SchemaChanger) done(ctx context.Context) error { ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ) error { childJobs = nil - fksByBackrefTable = make(map[descpb.ID][]*descpb.ConstraintToUpdate) - interleaveParents = make(map[descpb.ID]struct{}) scTable, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) if err != nil { @@ -1068,7 +1061,7 @@ func (sc *SchemaChanger) done(ctx context.Context) error { if err != nil { return err } - referencedTypeIDs, err = scTable.GetAllReferencedTypeIDs(dbDesc, + referencedTypeIDs, err := scTable.GetAllReferencedTypeIDs(dbDesc, func(id descpb.ID) (catalog.TypeDescriptor, error) { desc, err := descsCol.GetImmutableTypeByID(ctx, txn, id, tree.ObjectLookupFlags{}) if err != nil { @@ -1080,39 +1073,6 @@ func (sc *SchemaChanger) done(ctx context.Context) error { return err } b := txn.NewBatch() - for _, mutation := range scTable.Mutations { - if mutation.MutationID != sc.mutationID { - break - } - if constraint := mutation.GetConstraint(); constraint != nil && - constraint.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && - mutation.Direction == descpb.DescriptorMutation_ADD && - constraint.ForeignKey.Validity == descpb.ConstraintValidity_Unvalidated { - // Add backref table to referenced table with an unvalidated foreign key constraint - fk := &constraint.ForeignKey - if fk.ReferencedTableID != scTable.ID { - fksByBackrefTable[constraint.ForeignKey.ReferencedTableID] = append(fksByBackrefTable[constraint.ForeignKey.ReferencedTableID], constraint) - } - } else if swap := mutation.GetPrimaryKeySwap(); swap != nil { - // If any old indexes (including the old primary index) being rewritten are interleaved - // children, we will have to update their parents as well. - for _, idxID := range append([]descpb.IndexID{swap.OldPrimaryIndexId}, swap.OldIndexes...) { - oldIndex, err := scTable.FindIndexWithID(idxID) - if err != nil { - return err - } - if oldIndex.NumInterleaveAncestors() != 0 { - ancestor := oldIndex.GetInterleaveAncestor(oldIndex.NumInterleaveAncestors() - 1) - if ancestor.TableID != scTable.ID { - interleaveParents[ancestor.TableID] = struct{}{} - } - } - } - // Because we are not currently supporting primary key changes on tables/indexes - // that are interleaved parents, we don't check oldPrimaryIndex.InterleavedBy. - } - } - const kvTrace = true var i int // set to determine whether there is a mutation diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index b699dd2b53ba..bf7c5a630f20 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -84,6 +84,7 @@ go_library( "//pkg/util/uuid", "@com_github_cockroachdb_apd_v2//:apd", "@com_github_cockroachdb_errors//:errors", + "@com_github_gogo_protobuf//types", "@com_github_golang_geo//s1", "@com_github_knz_strtime//:strtime", "@com_github_lib_pq//oid", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 31f371c294a0..b25ea2527c24 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -73,6 +73,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/unaccent" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" + pbtypes "github.com/gogo/protobuf/types" "github.com/knz/strtime" ) @@ -3586,6 +3587,15 @@ may increase either contention or retry errors, or both.`, Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.Int), Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + // The user must be an admin to use this builtin. + isAdmin, err := ctx.SessionAccessor.HasAdminRole(ctx.Context) + if err != nil { + return nil, err + } + if !isAdmin { + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "user needs the admin role to view trace ID") + } + sp := tracing.SpanFromContext(ctx.Context) if sp == nil { return tree.DNull, nil @@ -3598,6 +3608,50 @@ may increase either contention or retry errors, or both.`, }, ), + "crdb_internal.payloads_for_span": makeBuiltin( + tree.FunctionProperties{Category: categorySystemInfo}, + tree.Overload{ + Types: tree.ArgTypes{{"span ID", types.Int}}, + ReturnType: tree.FixedReturnType(types.Jsonb), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + // The user must be an admin to use this builtin. + isAdmin, err := ctx.SessionAccessor.HasAdminRole(ctx.Context) + if err != nil { + return nil, err + } + if !isAdmin { + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "user needs the admin role to view payloads") + } + + builder := json.NewArrayBuilder(len(args)) + + spanID := uint64(*(args[0].(*tree.DInt))) + span, found := ctx.Settings.Tracer.GetActiveSpanFromID(spanID) + // A span may not be found if its ID was surfaced previously but its + // corresponding trace has ended by the time this builtin was run. + if !found { + // Returns an empty JSON array. + return tree.NewDJSON(builder.Build()), nil + } + + for _, rec := range span.GetRecording() { + rec.Structured(func(item *pbtypes.Any) { + payload, err := protoreflect.MessageToJSON(item, true /* emitDefaults */) + if err != nil { + return + } + if payload != nil { + builder.Add(payload) + } + }) + } + return tree.NewDJSON(builder.Build()), nil + }, + Info: "Returns the payload(s) of the span whose ID is passed in the argument.", + Volatility: tree.VolatilityVolatile, + }, + ), + "crdb_internal.locality_value": makeBuiltin( tree.FunctionProperties{Category: categorySystemInfo}, tree.Overload{ diff --git a/pkg/sql/show_test.go b/pkg/sql/show_test.go index c66ee9782167..795b74b83b44 100644 --- a/pkg/sql/show_test.go +++ b/pkg/sql/show_test.go @@ -21,56 +21,26 @@ import ( "unicode/utf8" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" - "github.com/stretchr/testify/require" ) func TestShowCreateTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() - params.Locality.Tiers = []roachpb.Tier{ - {Key: "region", Value: "us-west1"}, - } - s, sqlDB, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) - - if _, err := sqlDB.Exec(` - SET CLUSTER SETTING sql.cross_db_fks.enabled = TRUE; - SET experimental_enable_hash_sharded_indexes = TRUE; - CREATE DATABASE d; - CREATE DATABASE mrdb PRIMARY REGION = "us-west1"; - USE d; - CREATE TABLE items ( - a int8, - b int8, - c int8 unique, - primary key (a, b) - ); - CREATE DATABASE o; - CREATE TABLE o.foo(x int primary key); - `); err != nil { - t.Fatal(err) - } - - tests := []struct { - stmt string - expect string // empty means identical to stmt - database string // empty means to use database "d" - }{ + testCases := []sqltestutils.ShowCreateTableTestCase{ { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i INT8, s STRING NULL, v FLOAT NOT NULL, @@ -79,7 +49,7 @@ func TestShowCreateTable(t *testing.T) { FAMILY "primary" (i, v, t, rowid), FAMILY fam_1_s (s) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, s STRING NULL, v FLOAT8 NOT NULL, @@ -92,7 +62,7 @@ func TestShowCreateTable(t *testing.T) { )`, }, { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i INT8 CHECK (i > 0), s STRING NULL, v FLOAT NOT NULL, @@ -100,7 +70,7 @@ func TestShowCreateTable(t *testing.T) { FAMILY "primary" (i, v, t, rowid), FAMILY fam_1_s (s) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, s STRING NULL, v FLOAT8 NOT NULL, @@ -113,14 +83,14 @@ func TestShowCreateTable(t *testing.T) { )`, }, { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i INT8 NULL, s STRING NULL, CONSTRAINT ck CHECK (i > 0), FAMILY "primary" (i, rowid), FAMILY fam_1_s (s) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, s STRING NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), @@ -131,24 +101,24 @@ func TestShowCreateTable(t *testing.T) { )`, }, { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i INT8 PRIMARY KEY )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (i ASC), FAMILY "primary" (i) )`, }, { - stmt: ` + CreateStatement: ` CREATE TABLE %s (i INT8, f FLOAT, s STRING, d DATE, FAMILY "primary" (i, f, d, rowid), FAMILY fam_1_s (s)); CREATE INDEX idx_if on %[1]s (f, i) STORING (s, d); CREATE UNIQUE INDEX on %[1]s (d); `, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, f FLOAT8 NULL, s STRING NULL, @@ -162,24 +132,24 @@ func TestShowCreateTable(t *testing.T) { )`, }, { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( "te""st" INT8 NOT NULL, CONSTRAINT "pri""mary" PRIMARY KEY ("te""st" ASC), FAMILY "primary" ("te""st") )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( "te""st" INT8 NOT NULL, CONSTRAINT "pri""mary" PRIMARY KEY ("te""st" ASC), FAMILY "primary" ("te""st") )`, }, { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( a int8, b int8, index c(a asc, b desc) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( a INT8 NULL, b INT8 NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), @@ -191,13 +161,13 @@ func TestShowCreateTable(t *testing.T) { // Check that FK dependencies inside the current database // have their db name omitted. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i int8, j int8, FOREIGN KEY (i, j) REFERENCES items (a, b), k int REFERENCES items (c) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, j INT8 NULL, k INT8 NULL, @@ -211,13 +181,13 @@ func TestShowCreateTable(t *testing.T) { // Check that FK dependencies using MATCH FULL on a non-composite key still // show { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i int8, j int8, k int REFERENCES items (c) MATCH FULL, FOREIGN KEY (i, j) REFERENCES items (a, b) MATCH FULL )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL, j INT8 NULL, k INT8 NULL, @@ -231,11 +201,11 @@ func TestShowCreateTable(t *testing.T) { // Check that FK dependencies outside of the current database // have their db name prefixed. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( x INT8, CONSTRAINT fk_ref FOREIGN KEY (x) REFERENCES o.foo (x) )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( x INT8 NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT "primary" PRIMARY KEY (rowid ASC), @@ -246,13 +216,13 @@ func TestShowCreateTable(t *testing.T) { // Check that FK dependencies using SET NULL or SET DEFAULT // are pretty-printed properly. Regression test for #32529. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i int8 DEFAULT 123, j int8 DEFAULT 123, FOREIGN KEY (i, j) REFERENCES items (a, b) ON DELETE SET DEFAULT, k int8 REFERENCES items (c) ON DELETE SET NULL )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL DEFAULT 123:::INT8, j INT8 NULL DEFAULT 123:::INT8, k INT8 NULL, @@ -266,12 +236,12 @@ func TestShowCreateTable(t *testing.T) { // Check that INTERLEAVE dependencies inside the current database // have their db name omitted. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( a INT8, b INT8, PRIMARY KEY (a, b) ) INTERLEAVE IN PARENT items (a, b)`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( a INT8 NOT NULL, b INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (a ASC, b ASC), @@ -281,10 +251,10 @@ func TestShowCreateTable(t *testing.T) { // Check that INTERLEAVE dependencies outside of the current // database are prefixed by their db name. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( x INT8 PRIMARY KEY ) INTERLEAVE IN PARENT o.foo (x)`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( x INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (x ASC), FAMILY "primary" (x) @@ -293,7 +263,7 @@ func TestShowCreateTable(t *testing.T) { // Check that FK dependencies using MATCH FULL and MATCH SIMPLE are both // pretty-printed properly. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( i int DEFAULT 1, j int DEFAULT 2, k int DEFAULT 3, @@ -301,7 +271,7 @@ func TestShowCreateTable(t *testing.T) { FOREIGN KEY (i, j) REFERENCES items (a, b) MATCH SIMPLE ON DELETE SET DEFAULT, FOREIGN KEY (k, l) REFERENCES items (a, b) MATCH FULL ON UPDATE CASCADE )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( i INT8 NULL DEFAULT 1:::INT8, j INT8 NULL DEFAULT 2:::INT8, k INT8 NULL DEFAULT 3:::INT8, @@ -315,11 +285,11 @@ func TestShowCreateTable(t *testing.T) { }, // Check hash sharded indexes are round trippable. { - stmt: `CREATE TABLE %s ( + CreateStatement: `CREATE TABLE %s ( a INT, INDEX (a) USING HASH WITH BUCKET_COUNT = 8 )`, - expect: `CREATE TABLE public.%s ( + Expect: `CREATE TABLE public.%s ( a INT8 NULL, crdb_internal_a_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(COALESCE(CAST(a AS STRING), '':::STRING)), 8:::INT8)) STORED, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), @@ -328,94 +298,8 @@ func TestShowCreateTable(t *testing.T) { FAMILY "primary" (a, crdb_internal_a_shard_8, rowid) )`, }, - // Check GLOBAL tables are round trippable. - { - stmt: `CREATE TABLE %s ( - a INT - ) LOCALITY GLOBAL`, - expect: `CREATE TABLE public.%s ( - a INT8 NULL, - rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), - CONSTRAINT "primary" PRIMARY KEY (rowid ASC), - FAMILY "primary" (a, rowid) -) LOCALITY GLOBAL`, - database: "mrdb", - }, - // Check REGIONAL BY TABLE tables are round trippable. - { - stmt: `CREATE TABLE %s ( - a INT - ) LOCALITY REGIONAL BY TABLE`, - expect: `CREATE TABLE public.%s ( - a INT8 NULL, - rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), - CONSTRAINT "primary" PRIMARY KEY (rowid ASC), - FAMILY "primary" (a, rowid) -) LOCALITY REGIONAL BY TABLE IN PRIMARY REGION`, - database: "mrdb", - }, - { - stmt: `CREATE TABLE %s ( - a INT - ) LOCALITY REGIONAL BY TABLE IN "us-west1"`, - expect: `CREATE TABLE public.%s ( - a INT8 NULL, - rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), - CONSTRAINT "primary" PRIMARY KEY (rowid ASC), - FAMILY "primary" (a, rowid) -) LOCALITY REGIONAL BY TABLE IN "us-west1"`, - database: "mrdb", - }, - } - for i, test := range tests { - name := fmt.Sprintf("t%d", i) - t.Run(name, func(t *testing.T) { - if test.expect == "" { - test.expect = test.stmt - } - db := test.database - if db == "" { - db = "d" - } - _, err := sqlDB.Exec("USE $1", db) - require.NoError(t, err) - stmt := fmt.Sprintf(test.stmt, name) - expect := fmt.Sprintf(test.expect, name) - if _, err := sqlDB.Exec(stmt); err != nil { - t.Fatal(err) - } - row := sqlDB.QueryRow(fmt.Sprintf("SHOW CREATE TABLE %s", name)) - var scanName, create string - if err := row.Scan(&scanName, &create); err != nil { - t.Fatal(err) - } - if scanName != name { - t.Fatalf("expected table name %s, got %s", name, scanName) - } - if create != expect { - t.Fatalf("statement: %s\ngot: %s\nexpected: %s", stmt, create, expect) - } - if _, err := sqlDB.Exec(fmt.Sprintf("DROP TABLE %s", name)); err != nil { - t.Fatal(err) - } - // Re-insert to make sure it's round-trippable. - name += "_2" - expect = fmt.Sprintf(test.expect, name) - if _, err := sqlDB.Exec(expect); err != nil { - t.Fatalf("reinsert failure: %s: %s", expect, err) - } - row = sqlDB.QueryRow(fmt.Sprintf("SHOW CREATE TABLE %s", name)) - if err := row.Scan(&scanName, &create); err != nil { - t.Fatal(err) - } - if create != expect { - t.Fatalf("round trip statement: %s\ngot: %s", expect, create) - } - if _, err := sqlDB.Exec(fmt.Sprintf("DROP TABLE %s", name)); err != nil { - t.Fatal(err) - } - }) } + sqltestutils.ShowCreateTableTest(t, testCases) } func TestShowCreateView(t *testing.T) { diff --git a/pkg/sql/sqltestutils/BUILD.bazel b/pkg/sql/sqltestutils/BUILD.bazel index 0d4f8f76278b..32f513b19974 100644 --- a/pkg/sql/sqltestutils/BUILD.bazel +++ b/pkg/sql/sqltestutils/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "sqltestutils", - srcs = ["sql_test_utils.go"], + srcs = [ + "show_create_table.go", + "sql_test_utils.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils", visibility = ["//visibility:public"], deps = [ @@ -10,8 +13,11 @@ go_library( "//pkg/jobs", "//pkg/keys", "//pkg/kv", + "//pkg/roachpb", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", + "//pkg/sql/tests", + "//pkg/testutils/serverutils", "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", diff --git a/pkg/sql/sqltestutils/show_create_table.go b/pkg/sql/sqltestutils/show_create_table.go new file mode 100644 index 000000000000..b0d83968fe27 --- /dev/null +++ b/pkg/sql/sqltestutils/show_create_table.go @@ -0,0 +1,116 @@ +// 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 sqltestutils + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/stretchr/testify/require" +) + +// ShowCreateTableTestCase is a test case for ShowCreateTableTest. +type ShowCreateTableTestCase struct { + // CreateStatement is the statement used to create the table. + // A %s may be used to signify the table name. + CreateStatement string + // Expect is the statement that is expected from SHOW CREATE TABLE. + // A %s may be used to signify the table name. + Expect string + // Database is the database to execute on. + // Execute on "d" by default. + Database string +} + +// ShowCreateTableTest tests the output for SHOW CREATE TABLE matches +// the expect values. Furthermore, it round trips SHOW CREATE TABLE +// statements to ensure they produces an identical SHOW CREATE TABLE. +func ShowCreateTableTest(t *testing.T, testCases []ShowCreateTableTestCase) { + params, _ := tests.CreateTestServerParams() + params.Locality.Tiers = []roachpb.Tier{ + {Key: "region", Value: "us-west1"}, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(context.Background()) + + if _, err := sqlDB.Exec(` + SET CLUSTER SETTING sql.cross_db_fks.enabled = TRUE; + SET experimental_enable_hash_sharded_indexes = TRUE; + CREATE DATABASE d; + CREATE DATABASE mrdb PRIMARY REGION = "us-west1"; + USE d; + -- Create a table we can point FKs to. + CREATE TABLE items ( + a int8, + b int8, + c int8 unique, + primary key (a, b) + ); + -- Create a database we can cross reference. + CREATE DATABASE o; + CREATE TABLE o.foo(x int primary key); + `); err != nil { + t.Fatal(err) + } + for i, test := range testCases { + name := fmt.Sprintf("t%d", i) + t.Run(name, func(t *testing.T) { + if test.Expect == "" { + test.Expect = test.CreateStatement + } + db := test.Database + if db == "" { + db = "d" + } + _, err := sqlDB.Exec("USE $1", db) + require.NoError(t, err) + stmt := fmt.Sprintf(test.CreateStatement, name) + expect := fmt.Sprintf(test.Expect, name) + if _, err := sqlDB.Exec(stmt); err != nil { + t.Fatal(err) + } + row := sqlDB.QueryRow(fmt.Sprintf("SHOW CREATE TABLE %s", name)) + var scanName, create string + if err := row.Scan(&scanName, &create); err != nil { + t.Fatal(err) + } + if scanName != name { + t.Fatalf("expected table name %s, got %s", name, scanName) + } + if create != expect { + t.Fatalf("statement: %s\ngot: %s\nexpected: %s", stmt, create, expect) + } + if _, err := sqlDB.Exec(fmt.Sprintf("DROP TABLE %s", name)); err != nil { + t.Fatal(err) + } + // Re-insert to make sure it's round-trippable. + name += "_roundtrip" + expect = fmt.Sprintf(test.Expect, name) + if _, err := sqlDB.Exec(expect); err != nil { + t.Fatalf("reinsert failure: %s: %s", expect, err) + } + row = sqlDB.QueryRow(fmt.Sprintf("SHOW CREATE TABLE %s", name)) + if err := row.Scan(&scanName, &create); err != nil { + t.Fatal(err) + } + if create != expect { + t.Fatalf("round trip statement: %s\ngot: %s", expect, create) + } + if _, err := sqlDB.Exec(fmt.Sprintf("DROP TABLE %s", name)); err != nil { + t.Fatal(err) + } + }) + } +} diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 4a9021c978cf..20efe908ce5b 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -279,7 +279,7 @@ func (p *planner) writeTableDescToBatch( } } - if err := tableDesc.ValidateTable(ctx); err != nil { + if err := tableDesc.ValidateSelf(ctx); err != nil { return errors.AssertionFailedf("table descriptor is not valid: %s\n%v", err, tableDesc) } diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index 69a5f1f5deb9..c879ea3096e2 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -393,7 +393,7 @@ func TestPrimaryKeyUnspecified(t *testing.T) { } desc.SetPrimaryIndex(descpb.IndexDescriptor{}) - err = desc.ValidateTable(ctx) + err = desc.ValidateSelf(ctx) if !testutils.IsError(err, tabledesc.ErrMissingPrimaryKey.Error()) { t.Fatalf("unexpected error: %v", err) } diff --git a/pkg/sql/tests/repair_test.go b/pkg/sql/tests/repair_test.go index 3a4711ea0057..399a5bdd7f87 100644 --- a/pkg/sql/tests/repair_test.go +++ b/pkg/sql/tests/repair_test.go @@ -40,6 +40,8 @@ func TestDescriptorRepairOrphanedDescriptors(t *testing.T) { ctx := context.Background() setup := func(t *testing.T) (serverutils.TestServerInterface, *gosql.DB, func()) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + _, err := db.Exec(`SET CLUSTER SETTING sql.catalog.descs.validate_on_write.enabled = false`) + require.NoError(t, err) return s, db, func() { s.Stopper().Stop(ctx) } @@ -67,7 +69,7 @@ func TestDescriptorRepairOrphanedDescriptors(t *testing.T) { descID = 53 tableName = "foo" ) - // This test will inject the table an demonstrate + // This test will inject the table and demonstrate // that there are problems. It will then repair it by just dropping the // descriptor and namespace entry. This would normally be unsafe because // it would leave table data around. @@ -86,10 +88,14 @@ func TestDescriptorRepairOrphanedDescriptors(t *testing.T) { return err })) + // Now that we've finished setting up the test, we can restore validations. + _, err := db.Exec(`SET CLUSTER SETTING sql.catalog.descs.validate_on_write.enabled = true`) + require.NoError(t, err) + // Ideally we should be able to query `crdb_internal.invalid_object` but it // does not do enough validation. Instead we'll just observe the issue that // the parent descriptor cannot be found. - _, err := db.Exec( + _, err = db.Exec( "SELECT count(*) FROM \"\".crdb_internal.tables WHERE table_id = $1", descID) require.Regexp(t, "internal error: desc 53: parentID 52 does not exist", err) @@ -135,10 +141,14 @@ func TestDescriptorRepairOrphanedDescriptors(t *testing.T) { return err })) + // Now that we've finished setting up the test, we can restore validations. + _, err := db.Exec(`SET CLUSTER SETTING sql.catalog.descs.validate_on_write.enabled = true`) + require.NoError(t, err) + // Ideally we should be able to query `crdb_internal.invalid_objects` but it // does not do enough validation. Instead we'll just observe the issue that // the parent descriptor cannot be found. - _, err := db.Exec( + _, err = db.Exec( "SELECT count(*) FROM \"\".crdb_internal.tables WHERE table_id = $1", descID) require.Regexp(t, "internal error: desc 53: parentID 52 does not exist", err) @@ -242,6 +252,8 @@ func TestDescriptorRepair(t *testing.T) { ctx := context.Background() setup := func(t *testing.T) (serverutils.TestServerInterface, *gosql.DB, func()) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + _, err := db.Exec(`SET CLUSTER SETTING sql.catalog.descs.validate_on_write.enabled = false`) + require.NoError(t, err) return s, db, func() { s.Stopper().Stop(ctx) } @@ -258,6 +270,9 @@ func TestDescriptorRepair(t *testing.T) { after []string }{ { + before: []string{ + `CREATE DATABASE test`, + }, op: upsertRepair, expEventLogEntries: []eventLogPattern{ { @@ -276,10 +291,19 @@ func TestDescriptorRepair(t *testing.T) { typ: "change_table_privilege", info: `"DescriptorID":59,"Grantee":"admin","GrantedPrivileges":\["ALL"\]`, }, + { + typ: "change_table_privilege", + info: `"DescriptorID":59,"Grantee":"newuser1","GrantedPrivileges":\["ALL"\]`, + }, + { + typ: "change_table_privilege", + info: `"DescriptorID":59,"Grantee":"newuser2","GrantedPrivileges":\["ALL"\]`, + }, }, }, { before: []string{ + `CREATE DATABASE test`, upsertRepair, }, op: upsertUpdatePrivileges, @@ -290,15 +314,11 @@ func TestDescriptorRepair(t *testing.T) { }, { typ: "change_table_privilege", - info: `"DescriptorID":59,"Grantee":"root","GrantedPrivileges":\["DROP"\],"RevokedPrivileges":\["ALL"\]`, - }, - { - typ: "change_table_privilege", - info: `"DescriptorID":59,"Grantee":"newuser","GrantedPrivileges":\["CREATE"\]`, + info: `"DescriptorID":59,"Grantee":"newuser1","GrantedPrivileges":\["DROP"\],"RevokedPrivileges":\["ALL"\]`, }, { typ: "change_table_privilege", - info: `"DescriptorID":59,"Grantee":"admin","RevokedPrivileges":\["ALL"\]`, + info: `"DescriptorID":59,"Grantee":"newuser2","RevokedPrivileges":\["ALL"\]`, }, }, }, @@ -423,7 +443,9 @@ SELECT crdb_internal.unsafe_delete_namespace_entry("parentID", 0, 'foo', id) for _, op := range tc.before { tdb.Exec(t, op) } - _, err := db.Exec(tc.op) + _, err := db.Exec(`SET CLUSTER SETTING sql.catalog.descs.validate_on_write.enabled = true`) + require.NoError(t, err) + _, err = db.Exec(tc.op) if tc.expErrRE == "" { require.NoError(t, err) } else { @@ -586,7 +608,7 @@ SELECT crdb_internal.unsafe_upsert_descriptor(52, descriptor, true) SELECT crdb_internal.unsafe_upsert_descriptor(59, crdb_internal.json_to_pb('cockroach.sql.sqlbase.Descriptor', '{ "table": { - "columns": [ { "id": 1, "name": "i" } ], + "columns": [ { "id": 1, "name": "i", "type": { "family": "IntFamily", "oid": 20, "width": 64 } } ], "families": [ { "columnIds": [ 1 ], @@ -616,7 +638,12 @@ SELECT crdb_internal.unsafe_upsert_descriptor(59, crdb_internal.json_to_pb('cock }, "privileges": { "owner_proto": "root", - "users": [ { "privileges": 2, "user_proto": "admin" }, { "privileges": 2, "user_proto": "root" } ], + "users": [ + { "privileges": 2, "user_proto": "admin" }, + { "privileges": 2, "user_proto": "root" }, + { "privileges": 2, "user_proto": "newuser1" }, + { "privileges": 2, "user_proto": "newuser2" } + ], "version": 1 }, "state": "PUBLIC", @@ -634,7 +661,7 @@ SELECT crdb_internal.unsafe_upsert_descriptor(59, crdb_internal.json_to_pb('cock SELECT crdb_internal.unsafe_upsert_descriptor(59, crdb_internal.json_to_pb('cockroach.sql.sqlbase.Descriptor', '{ "table": { - "columns": [ { "id": 1, "name": "i" } ], + "columns": [ { "id": 1, "name": "i", "type": { "family": "IntFamily", "oid": 20, "width": 64 } } ], "families": [ { "columnIds": [ 1 ], @@ -664,7 +691,11 @@ SELECT crdb_internal.unsafe_upsert_descriptor(59, crdb_internal.json_to_pb('cock }, "privileges": { "owner_proto": "admin", - "users": [ { "privileges": 5, "user_proto": "newuser" }, { "privileges": 8, "user_proto": "root" } ], + "users": [ + { "privileges": 2, "user_proto": "admin" }, + { "privileges": 2, "user_proto": "root" }, + { "privileges": 8, "user_proto": "newuser1" } + ], "version": 1 }, "state": "PUBLIC", diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 4b1c4db1abda..47519c93cba7 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -204,7 +204,7 @@ func TestSystemTableLiterals(t *testing.T) { if err != nil { t.Fatalf("test: %+v, err: %v", test, err) } - require.NoError(t, gen.ValidateTable(ctx)) + require.NoError(t, gen.ValidateSelf(ctx)) if !test.pkg.TableDesc().Equal(gen.TableDesc()) { diff := strings.Join(pretty.Diff(test.pkg.TableDesc(), gen.TableDesc()), "\n") diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 42cbbdb309f2..8f5ac49d3c51 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -381,10 +381,6 @@ func (t *typeSchemaChanger) cleanupEnumValues(ctx context.Context) error { if err != nil { return err } - if err := dbDesc.Validate(); err != nil { - return errors.Wrapf(err, "could not re-add region to the database descriptor") - } - if err := descsCol.WriteDescToBatch(ctx, true /* kvTrace */, dbDesc, b); err != nil { return err } diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index d9151aad73cd..bbba9588b9a7 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -155,7 +155,8 @@ type Tracer struct { // In normal operation, a local root Span is inserted on creation and // removed on .Finish(). // - // The map can be introspected by `Tracer.VisitSpans`. + // The map can be introspected by `Tracer.VisitSpans`. A Span can also be + // retrieved from its ID by `Tracer.GetActiveSpanFromID`. activeSpans struct { // NB: it might be tempting to use a sync.Map here, but // this incurs an allocation per Span (sync.Map does @@ -678,6 +679,14 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (*SpanMeta, error) { }, nil } +// GetActiveSpanFromID retrieves any active span given its span ID. +func (t *Tracer) GetActiveSpanFromID(spanID uint64) (*Span, bool) { + t.activeSpans.Lock() + span, found := t.activeSpans.m[spanID] + t.activeSpans.Unlock() + return span, found +} + // VisitSpans invokes the visitor with all active Spans. The function will // gracefully exit if the visitor returns iterutil.StopIteration(). func (t *Tracer) VisitSpans(visitor func(*Span) error) error {