From 767419b488a1122290e8900d16a973320d31fd05 Mon Sep 17 00:00:00 2001
From: Gemma Shay
Date: Wed, 27 Jan 2021 14:50:49 -0500
Subject: [PATCH 01/12] add licensing info to README
---
README.md | 16 ++++++++++++++++
1 file changed, 16 insertions(+)
diff --git a/README.md b/README.md
index 005a4f38c9cc..c20a7c10c42b 100644
--- a/README.md
+++ b/README.md
@@ -104,6 +104,22 @@ 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 licensed in three ways:
+
+- [Business Source License (BSL)](https://www.cockroachlabs.com/docs/stable/licensing.html#bsl)
+- [Cockroach Community License (Free)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-free)
+- [Cockroach Community License (Paid)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-paid)
+
+Core features released up to and including version 19.1 are licensed under [Apache 2.0](https://www.cockroachlabs.com/docs/stable/licensing.html#apache). After version 19.1, core features are licensed under the BSL for three years before converting to the Apache 2.0 license.
+
+BSL features are free to use and the source code is available, but users may not use CockroachDB as a service without an agreement with Cockroach Labs. Outside of this usage, BSL features will continue to be free to use and source code will continue to be available.
+
+[Certain core features](https://www.cockroachlabs.com/docs/stable/licensing.html#core-feature-licensing) and all enterprise features are licensed under the Cockroach Community License (CCL).
+
+For more information, see [Licensing](https://www.cockroachlabs.com/docs/stable/licensing.html).
+
## Comparison with Other Databases
To see how key features of CockroachDB stack up against other databases,
From 9b554a4ec938f9376230f75f03db88e7e63c012d Mon Sep 17 00:00:00 2001
From: Gemma Shay
Date: Wed, 27 Jan 2021 17:32:16 -0500
Subject: [PATCH 02/12] removed summary and added info for contributors
---
README.md | 12 ++----------
1 file changed, 2 insertions(+), 10 deletions(-)
diff --git a/README.md b/README.md
index c20a7c10c42b..1f4c546ce61d 100644
--- a/README.md
+++ b/README.md
@@ -106,17 +106,9 @@ doc](https://github.com/cockroachdb/cockroach/blob/master/docs/design.md).
## Licensing
-Current CockroachDB code is licensed in three ways:
+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).
-- [Business Source License (BSL)](https://www.cockroachlabs.com/docs/stable/licensing.html#bsl)
-- [Cockroach Community License (Free)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-free)
-- [Cockroach Community License (Paid)](https://www.cockroachlabs.com/docs/stable/licensing.html#ccl-paid)
-
-Core features released up to and including version 19.1 are licensed under [Apache 2.0](https://www.cockroachlabs.com/docs/stable/licensing.html#apache). After version 19.1, core features are licensed under the BSL for three years before converting to the Apache 2.0 license.
-
-BSL features are free to use and the source code is available, but users may not use CockroachDB as a service without an agreement with Cockroach Labs. Outside of this usage, BSL features will continue to be free to use and source code will continue to be available.
-
-[Certain core features](https://www.cockroachlabs.com/docs/stable/licensing.html#core-feature-licensing) and all enterprise features are licensed under the Cockroach Community License (CCL).
+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 [Licensing](https://www.cockroachlabs.com/docs/stable/licensing.html).
From bfad1cfb692b64b8cc6a7d2fd32cff9dbd378392 Mon Sep 17 00:00:00 2001
From: Marius Posta
Date: Thu, 11 Feb 2021 12:41:24 -0500
Subject: [PATCH 03/12] sql: add descriptor validation on write
Previously, we didn't systematically validate descriptors when they were
written. Furthermore, there existed no common method to validate
descriptors across all descriptor subtypes
This commit adds three methods to the catalog.Descriptor interface:
1. ValidateSelf ( context.Context ) error
2. Validate ( context.Context, catalog.DescGetter ) error
3. ValidateTxnCommit ( context.Context, catalog.DescGetter) error
Each performs a subset the checks performed by the next. ValidateSelf
contains all checks which can be performed in isolation, Validate also
performs all those involving DescGetters (i.e. cross-reference checks)
and ValidateTxnCommit also performs checks which should only be done at
commit-time. An example of the latter is checking that a table has
a primary key: dropping the PK is allowed within a transaction as long
as a new PK is subsequently provided before committing.
This commit adds new validation calls when writing descriptors:
1. ValidateSelf is called prior to Collection adding a descriptor Put
to a kv.Batch. At this point, we want descriptors to be at least
internally-consistent, both to catch validation errors early and
because it's not possible to do cross-reference checking at this
point (changes on FKs for instance involve multiple descriptors).
2. ValidateTxnCommit is called on the descs.Collection's uncommitted
descriptors when the corresponding txn is about to commit, just
prior to the two-version-invariant check. The Collection itself is
used as a catalog.DescGetter.
These validations may be disabled using a new cluster setting:
sql.catalog.descs.validate_on_write.enabled
Setting this to false makes it possible to corrupt the descriptor state
using the crdb_internal.unsafe_* functions.
Release note: None
---
pkg/ccl/backupccl/restore_job.go | 2 +-
pkg/ccl/changefeedccl/avro_test.go | 2 +-
pkg/ccl/partitionccl/partition_test.go | 2 +-
pkg/sql/catalog/catalogkv/catalogkv.go | 26 ++------
.../catalogkv/unwrap_validation_test.go | 14 -----
pkg/sql/catalog/dbdesc/database_desc.go | 15 ++++-
pkg/sql/catalog/desc_getter.go | 32 +++++++---
pkg/sql/catalog/descriptor.go | 13 ++--
pkg/sql/catalog/descs/BUILD.bazel | 2 +
pkg/sql/catalog/descs/collection.go | 52 +++++++++++++++-
pkg/sql/catalog/descs/collection_test.go | 29 ++++++++-
pkg/sql/catalog/descs/txn.go | 3 +
pkg/sql/catalog/lease/lease.go | 2 +-
pkg/sql/catalog/schemadesc/schema_desc.go | 16 +++++
pkg/sql/catalog/tabledesc/helpers_test.go | 2 +-
pkg/sql/catalog/tabledesc/safe_format_test.go | 2 +-
pkg/sql/catalog/tabledesc/structured.go | 40 ++++++++++---
pkg/sql/catalog/tabledesc/structured_test.go | 3 +-
pkg/sql/catalog/typedesc/type_desc.go | 24 ++++++--
pkg/sql/conn_executor_exec.go | 19 +-----
pkg/sql/create_sequence.go | 4 +-
pkg/sql/database.go | 3 -
pkg/sql/descriptor.go | 7 ++-
pkg/sql/descriptor_mutation_test.go | 14 ++---
pkg/sql/repair.go | 26 ++++----
pkg/sql/resolver.go | 14 -----
pkg/sql/table.go | 2 +-
pkg/sql/table_test.go | 2 +-
pkg/sql/tests/repair_test.go | 59 ++++++++++++++-----
pkg/sql/tests/system_table_test.go | 2 +-
pkg/sql/type_change.go | 4 --
31 files changed, 288 insertions(+), 149 deletions(-)
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/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/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/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go
index e9c5b9958a4f..a580c786a9f3 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"
@@ -250,10 +251,10 @@ func (desc *Mutable) SetName(name string) {
desc.Name = name
}
-// Validate validates that the database descriptor is well formed.
+// 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 +295,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..a37b8e41b27b 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,7 +99,6 @@ type DatabaseDescriptor interface {
RegionNames() (descpb.RegionNames, error)
IsMultiRegion() bool
PrimaryRegionName() (descpb.RegionName, error)
- Validate() error
MultiRegionEnumID() (descpb.ID, error)
}
@@ -245,8 +253,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 +499,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/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go
index 8f2154f1db20..2be444f96a38 100644
--- a/pkg/sql/catalog/schemadesc/schema_desc.go
+++ b/pkg/sql/catalog/schemadesc/schema_desc.go
@@ -11,6 +11,7 @@
package schemadesc
import (
+ "context"
"strings"
"github.com/cockroachdb/cockroach/pkg/keys"
@@ -165,6 +166,21 @@ func (desc *Immutable) DescriptorProto() *descpb.Descriptor {
}
}
+// ValidateSelf implements the catalog.Descriptor interface.
+func (desc *Immutable) ValidateSelf(_ context.Context) error {
+ return nil
+}
+
+// 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)
+}
+
// 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/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/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/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 5c7a1b3ce220..0d85c07470bd 100644
--- a/pkg/sql/resolver.go
+++ b/pkg/sql/resolver.go
@@ -797,20 +797,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/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
}
From 9065522e674ce613a3bbfc62e6fef14391f85e5f Mon Sep 17 00:00:00 2001
From: Marius Posta
Date: Mon, 15 Feb 2021 13:29:30 -0500
Subject: [PATCH 04/12] sql: add schema descriptor validation
Previously, the Validate() method on catalog.SchemaDescriptor did
nothing. This commit adds validation checks that verify that the
parentID exists and that that database descriptor contains the correct
entry in its `schemas` mapping.
Fixes #53439.
Release note: None
---
pkg/sql/catalog/dbdesc/BUILD.bazel | 1 +
pkg/sql/catalog/dbdesc/database_desc.go | 17 ++++++
pkg/sql/catalog/descriptor.go | 1 +
pkg/sql/catalog/schemadesc/BUILD.bazel | 1 +
pkg/sql/catalog/schemadesc/schema_desc.go | 66 +++++++++++++++++++++--
5 files changed, 82 insertions(+), 4 deletions(-)
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 a580c786a9f3..1d93197052e3 100644
--- a/pkg/sql/catalog/dbdesc/database_desc.go
+++ b/pkg/sql/catalog/dbdesc/database_desc.go
@@ -22,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"
@@ -251,6 +252,22 @@ func (desc *Mutable) SetName(name string) {
desc.Name = name
}
+// 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.
diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go
index a37b8e41b27b..fb414d7c1f50 100644
--- a/pkg/sql/catalog/descriptor.go
+++ b/pkg/sql/catalog/descriptor.go
@@ -100,6 +100,7 @@ type DatabaseDescriptor interface {
IsMultiRegion() bool
PrimaryRegionName() (descpb.RegionName, error)
MultiRegionEnumID() (descpb.ID, error)
+ ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error) error
}
// SchemaDescriptor will eventually be called schemadesc.Descriptor.
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 2be444f96a38..499599ad865b 100644
--- a/pkg/sql/catalog/schemadesc/schema_desc.go
+++ b/pkg/sql/catalog/schemadesc/schema_desc.go
@@ -12,6 +12,7 @@ package schemadesc
import (
"context"
+ "fmt"
"strings"
"github.com/cockroachdb/cockroach/pkg/keys"
@@ -19,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"
@@ -168,12 +170,68 @@ func (desc *Immutable) DescriptorProto() *descpb.Descriptor {
// ValidateSelf implements the catalog.Descriptor interface.
func (desc *Immutable) ValidateSelf(_ context.Context) error {
- return nil
+ 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 punts to ValidateSelf.
-func (desc *Immutable) Validate(ctx context.Context, _ catalog.DescGetter) error {
- return desc.ValidateSelf(ctx)
+// 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.
From bd0e202de74bf2c2d891162708bc1e5e3b9fba6e Mon Sep 17 00:00:00 2001
From: Marius Posta
Date: Tue, 16 Feb 2021 15:01:28 -0500
Subject: [PATCH 05/12] sql: remove dead code in SchemaChanger done() method
Previously this method populated a couple of maps but didn't actually do
anything with them besides that.
Release note: None
---
pkg/sql/schema_changer.go | 42 +--------------------------------------
1 file changed, 1 insertion(+), 41 deletions(-)
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
From 96b24aec189d25c28ebcf5e23a3572b143a27e19 Mon Sep 17 00:00:00 2001
From: angelapwen
Date: Thu, 11 Feb 2021 12:55:46 +0100
Subject: [PATCH 06/12] builtins: add builtin to retrieve the payload(s) for a
span.
The `crdb_internal.payloads_for_span` builtin retrieves all
payloads for a given span ID, given that the span is part of an
active trace. The payloads are returned in JSONB format. If the
span is not found, or if the span does not have any payloads, the
builtin returns an empty JSON object.
With the appropriate usage of this builtin and the
`crdb_internal.trace_id` builtin as shown in the `contention_event`
logic test, all payloads for the current trace may be surfaced.
Release note (sql change): add `payloads_for_span` builtin that
takes in a span ID and returns its paylods in JSONB format. If
the span is not found, or if the span does not have any payloads,
the builtin returns an empty JSON object.
---
docs/generated/sql/functions.md | 2 +
pkg/sql/crdb_internal.go | 8 ---
.../testdata/logic_test/contention_event | 18 ++++---
.../testdata/logic_test/crdb_internal | 4 +-
.../testdata/logic_test/crdb_internal_tenant | 4 +-
pkg/sql/sem/builtins/BUILD.bazel | 1 +
pkg/sql/sem/builtins/builtins.go | 54 +++++++++++++++++++
pkg/util/tracing/tracer.go | 11 +++-
8 files changed, 83 insertions(+), 19 deletions(-)
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/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/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/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/util/tracing/tracer.go b/pkg/util/tracing/tracer.go
index 3ed10b929521..ac1165f8835b 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
@@ -676,6 +677,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 {
From d6c76a84967da38ddd037c4fbf3b48b951e57bb1 Mon Sep 17 00:00:00 2001
From: Marcus Gartner
Date: Wed, 17 Feb 2021 10:44:08 -0800
Subject: [PATCH 07/12] sql: add tests for inverted indexes on virtual columns
No code changes were needed to support inverted indexes on virtual
columns.
Release note: None
---
.../testdata/logic_test/virtual_columns | 105 ++++++++++++++++++
.../exec/execbuilder/testdata/virtual_columns | 67 +++++++++++
2 files changed, 172 insertions(+)
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
From c99c17c9b15876a4b4f60aa6aa10226b74c54ce4 Mon Sep 17 00:00:00 2001
From: Gemma Shay
Date: Thu, 18 Feb 2021 13:53:29 -0500
Subject: [PATCH 08/12] change licensing back to faqs
---
README.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/README.md b/README.md
index 1f4c546ce61d..8d398cd89646 100644
--- a/README.md
+++ b/README.md
@@ -110,7 +110,7 @@ Current CockroachDB code is released under a combination of two licenses, the [B
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 [Licensing](https://www.cockroachlabs.com/docs/stable/licensing.html).
+For more information, see the [Licensing FAQs](https://www.cockroachlabs.com/docs/stable/licensing-faqs.html).
## Comparison with Other Databases
From 0e5b0d914fccd4ab283e5efcfb22de914259e4dc Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Thu, 18 Feb 2021 11:36:45 +1100
Subject: [PATCH 09/12] sql: allow crdb_region in REGIONAL BY ROW definition
Allow CREATE TABLE ... REGIONAL BY ROW if there is a crdb_region column
with the correct crdb_internal_region type in it.
Release note: None
---
.../testdata/logic_test/regional_by_row | 56 +++++++++++++++----
pkg/sql/create_table.go | 46 +++++++--------
2 files changed, 69 insertions(+), 33 deletions(-)
diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row
index 0c2ad06a0d51..4e0b9cbfc53a 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,47 @@ 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,
+ 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 NULL,
+ a INT8 NULL,
+ crdb_region public.crdb_internal_region NOT NULL,
+ rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
+ CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
+ FAMILY fam_0_pk_a_crdb_region_rowid (pk, a, crdb_region, rowid)
+) 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 +202,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 +877,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/sql/create_table.go b/pkg/sql/create_table.go
index 480e9da3267c..b333fcd9c152 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.
From 9ff1503e6ae33ba3ed5a595e4b2d70b82012ebc7 Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Fri, 12 Feb 2021 10:35:21 +1100
Subject: [PATCH 10/12] sql: fix case where PRIMARY KEY as a constraint gets
partitioned twice
If PRIMARY KEY is defined as a constraint, it gets CreatePartitioning
called on twice which is redunant. This gets fixed in this commit.
Release note: None
---
pkg/sql/create_table.go | 14 ++++++++++++--
1 file changed, 12 insertions(+), 2 deletions(-)
diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go
index b333fcd9c152..530895500450 100644
--- a/pkg/sql/create_table.go
+++ b/pkg/sql/create_table.go
@@ -1948,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() {
@@ -2095,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.
//
From aea08fb932701a43a7953c706707a8032f6f7ff0 Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Thu, 18 Feb 2021 11:24:57 +1100
Subject: [PATCH 11/12] sql: ensure REGIONAL BY ROW statements roundtrip in
tests
* Move SHOW CREATE TABLE round trip tests to a common library, so that we
can access the tests in the CCL directory.
* Move multiregion related SHOW CREATE TABLE round trip tests to
multiregionccl test package.
Release note: None
---
pkg/ccl/BUILD.bazel | 1 +
pkg/ccl/ccl_init.go | 1 +
.../testdata/logic_test/alter_table_locality | 81 ++++++--
.../testdata/logic_test/regional_by_row | 9 +-
pkg/ccl/multiregionccl/BUILD.bazel | 29 +++
pkg/ccl/multiregionccl/main_test.go | 33 ++++
pkg/ccl/multiregionccl/multiregion.go | 9 +
pkg/ccl/multiregionccl/show_test.go | 114 +++++++++++
pkg/sql/show_test.go | 182 ++++--------------
pkg/sql/sqltestutils/BUILD.bazel | 8 +-
pkg/sql/sqltestutils/show_create_table.go | 116 +++++++++++
11 files changed, 407 insertions(+), 176 deletions(-)
create mode 100644 pkg/ccl/multiregionccl/BUILD.bazel
create mode 100644 pkg/ccl/multiregionccl/main_test.go
create mode 100644 pkg/ccl/multiregionccl/multiregion.go
create mode 100644 pkg/ccl/multiregionccl/show_test.go
create mode 100644 pkg/sql/sqltestutils/show_create_table.go
diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel
index f6190ee414b8..ceb5fc163971 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/ccl_init.go b/pkg/ccl/ccl_init.go
index 224be65c492d..161f92190512 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/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 4e0b9cbfc53a..750904afb215 100644
--- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row
+++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row
@@ -74,7 +74,7 @@ LOCALITY REGIONAL BY ROW
statement ok
CREATE TABLE regional_by_row_table_explicit_crdb_region_column (
- pk int,
+ pk int PRIMARY KEY,
a int,
crdb_region crdb_internal_region,
FAMILY (pk, a, crdb_region)
@@ -85,12 +85,11 @@ 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 NULL,
+ pk INT8 NOT NULL,
a INT8 NULL,
crdb_region public.crdb_internal_region NOT NULL,
- rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
- CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
- FAMILY fam_0_pk_a_crdb_region_rowid (pk, a, crdb_region, rowid)
+ 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,
diff --git a/pkg/ccl/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel
new file mode 100644
index 000000000000..6a6d5314fa83
--- /dev/null
+++ b/pkg/ccl/multiregionccl/BUILD.bazel
@@ -0,0 +1,29 @@
+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",
+ "show_test.go",
+ ],
+ deps = [
+ "//pkg/ccl/partitionccl",
+ "//pkg/ccl/utilccl",
+ "//pkg/security",
+ "//pkg/security/securitytest",
+ "//pkg/server",
+ "//pkg/sql/sqltestutils",
+ "//pkg/testutils/serverutils",
+ "//pkg/testutils/testcluster",
+ "//pkg/util/leaktest",
+ "//pkg/util/log",
+ "//pkg/util/randutil",
+ ],
+)
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/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/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)
+ }
+ })
+ }
+}
From 9a6b6be60dc388013f15781bd7bc2d0c4119ac7f Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Thu, 18 Feb 2021 21:25:08 +1100
Subject: [PATCH 12/12] multiregionccl: move regional_by_row_test to
multiregionccl
This previously lived in partitionccl, but we have a new home for it
now.
Release note: None
---
pkg/BUILD.bazel | 1 +
pkg/ccl/multiregionccl/BUILD.bazel | 13 +++++++++++++
.../regional_by_row_test.go | 4 ++--
pkg/ccl/partitionccl/BUILD.bazel | 6 ------
4 files changed, 16 insertions(+), 8 deletions(-)
rename pkg/ccl/{partitionccl => multiregionccl}/regional_by_row_test.go (99%)
diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index a8fc4f6ccddb..a0ec37b55e0b 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/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel
index 6a6d5314fa83..f6d4006e8373 100644
--- a/pkg/ccl/multiregionccl/BUILD.bazel
+++ b/pkg/ccl/multiregionccl/BUILD.bazel
@@ -11,19 +11,32 @@ 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/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/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",
],
)
|