Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
65142: sql/*: remove catalog.ResolvedSchema, generalize SchemaDescriptor r=postamar a=ajwerner

The `ResolvedSchema` was a blemish on the descriptor resolution APIs. It made
it very hard to create a reasonable abstraction for descriptor resolution
injection. This commit creates new implementations of `SchemaDescriptor` to
represent virtual, public, and temporary schemas.

There is likely more cleanup that could be done along the way to make the
different kinds of schemas more uniform, but that is left for later or never.

Relates to #64089.

Release note: None

65780: sql: Add unit tests for planning inside the new schema changer r=ajwerner a=fqazi

Previously, the new schema changer did not have any unit
tests covering the planning capability. This was inadequate,
because we had no way of detect if plans were regressing with
changes or enhancements. To address this, this patch adds
basic tests to see if the operators/dependencies for a given
command are sane.

Release note: None

66051: logcrash: update non-release Sentry URL r=mgartner a=mgartner

The documentation for creating Sentry reports in non-release builds
directed developers to set the COCKROACH_CRASH_REPORTS env var to an
incorrect URL. Sentry reports were not created when using this URL. The
documentation has been updated to refer to the correct URL.

Release note: None

66070: sql: fix very slow TestSchemaChangeRetryOnVersionChange r=fqazi a=ajwerner

See [here](https://teamcity.cockroachdb.com/project.html?projectId=Cockroach_UnitTests&testNameId=6532733632782929224&tab=testDetails).
This test was taking 10 minutes every time. This was happening because of the
change to wait for the version's leases to expire in #63725.

After:
```
--- PASS: TestSchemaChangeRetryOnVersionChange (1.70s)
```

Release note: None

66074: sql: fix statement diagnostics for EXECUTE r=RaduBerinde a=RaduBerinde

Previously, prepared statements ran through the EXECUTE statements
could not trigger collection of statement diagnostics. This is because
we consider the fingerprint of the EXECUTE statement instead of the
one from the prepared statement.

The fix is to move up the special handling code in the executor, and
replace the AST and fingerprint before setting up the instrumentation
helper.

Release note (bug fix): queries ran through the EXECUTE statement can
now generate statement diagnostic bundles as expected.

Fixes #66048.

Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
5 people committed Jun 4, 2021
6 parents 6fa99f5 + 55ad7eb + a6ecdea + e002229 + d8684e4 + d3cbcfd commit 02830c2
Show file tree
Hide file tree
Showing 84 changed files with 1,867 additions and 551 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,7 @@ ALL_TESTS = [
"//pkg/sql/schemachanger/scbuild:scbuild_test",
"//pkg/sql/schemachanger/scexec:scexec_test",
"//pkg/sql/schemachanger/scpb:scpb_test",
"//pkg/sql/schemachanger/scplan:scplan_test",
"//pkg/sql/schemachanger:schemachanger_test",
"//pkg/sql/sem/builtins:builtins_test",
"//pkg/sql/sem/tree/eval_test:eval_test_test",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ go_library(
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
Expand All @@ -30,7 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/protoreflect"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -269,7 +269,7 @@ func backupShowerDefault(
// Map database ID to descriptor name.
dbIDToName := make(map[descpb.ID]string)
schemaIDToName := make(map[descpb.ID]string)
schemaIDToName[keys.PublicSchemaID] = sessiondata.PublicSchemaName
schemaIDToName[keys.PublicSchemaID] = catconstants.PublicSchemaName
for i := range manifest.Descriptors {
_, db, _, schema := descpb.FromDescriptor(&manifest.Descriptors[i])
if db != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/cliccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,13 @@ go_library(
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/row",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/storage",
"//pkg/storage/cloud",
"//pkg/storage/cloud/nodelocal",
Expand Down
8 changes: 4 additions & 4 deletions pkg/ccl/cliccl/debug_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,13 +35,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloud/nodelocal"
Expand Down Expand Up @@ -751,7 +751,7 @@ func (b backupMetaDisplayMsg) MarshalJSON() ([]byte, error) {

dbIDToName := make(map[descpb.ID]string)
schemaIDToFullyQualifiedName := make(map[descpb.ID]string)
schemaIDToFullyQualifiedName[keys.PublicSchemaID] = sessiondata.PublicSchemaName
schemaIDToFullyQualifiedName[keys.PublicSchemaID] = catconstants.PublicSchemaName
typeIDToFullyQualifiedName := make(map[descpb.ID]string)
tableIDToFullyQualifiedName := make(map[descpb.ID]string)

Expand All @@ -767,15 +767,15 @@ func (b backupMetaDisplayMsg) MarshalJSON() ([]byte, error) {
schemaIDToFullyQualifiedName[id] = dbName + "." + schemaName
} else if typeDesc != nil {
parentSchema := schemaIDToFullyQualifiedName[typeDesc.GetParentSchemaID()]
if parentSchema == sessiondata.PublicSchemaName {
if parentSchema == catconstants.PublicSchemaName {
parentSchema = dbIDToName[typeDesc.GetParentID()] + "." + parentSchema
}
typeName := descpb.GetDescriptorName(d)
typeIDToFullyQualifiedName[id] = parentSchema + "." + typeName
} else if tableDesc != nil {
tbDesc := tabledesc.NewBuilder(tableDesc).BuildImmutable()
parentSchema := schemaIDToFullyQualifiedName[tbDesc.GetParentSchemaID()]
if parentSchema == sessiondata.PublicSchemaName {
if parentSchema == catconstants.PublicSchemaName {
parentSchema = dbIDToName[tableDesc.GetParentID()] + "." + parentSchema
}
tableName := descpb.GetDescriptorName(d)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,12 +433,12 @@ func importPlanHook(
}
}
parentID = dbDesc.GetID()
switch schema.Kind {
switch schema.SchemaKind() {
case catalog.SchemaVirtual:
return pgerror.Newf(pgcode.InvalidSchemaName,
"cannot import into schema %q", table.SchemaName)
case catalog.SchemaUserDefined, catalog.SchemaPublic, catalog.SchemaTemporary:
parentSchemaID = schema.ID
parentSchemaID = schema.GetID()
}
} else {
// No target table means we're importing whatever we find into the session
Expand Down
1 change: 0 additions & 1 deletion pkg/server/telemetry/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/server/telemetry",
visibility = ["//visibility:public"],
deps = [
"//pkg/sql/catalog/catconstants",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/util/metric",
Expand Down
7 changes: 5 additions & 2 deletions pkg/server/telemetry/features.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"strings"
"sync/atomic"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -226,6 +225,10 @@ func GetFeatureCounts(quantize QuantizeCounts, reset ResetCounters) map[string]i
return m
}

// ValidationTelemetryKeyPrefix is the prefix of telemetry keys pertaining to
// descriptor validation failures.
const ValidationTelemetryKeyPrefix = "sql.schema.validation_errors."

// RecordError takes an error and increments the corresponding count
// for its error code, and, if it is an unimplemented or internal
// error, the count for that feature or the internal error's shortened
Expand All @@ -251,7 +254,7 @@ func RecordError(err error) {
}
for _, tk := range tkeys {
prefixedTelemetryKey := prefix + tk
if strings.HasPrefix(tk, catconstants.ValidationTelemetryKeyPrefix) {
if strings.HasPrefix(tk, ValidationTelemetryKeyPrefix) {
// Descriptor validation errors already have their own prefixing scheme.
prefixedTelemetryKey = tk
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func (p *planner) AlterDatabaseOwner(
return nil, err
}

_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
tree.DatabaseLookupFlags{Required: true})
if err != nil {
return nil, err
Expand Down Expand Up @@ -133,7 +133,7 @@ func (p *planner) AlterDatabaseAddRegion(
return nil, err
}

_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
tree.DatabaseLookupFlags{Required: true},
)
if err != nil {
Expand Down Expand Up @@ -292,7 +292,7 @@ func (p *planner) AlterDatabaseDropRegion(
return nil, err
}

_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
tree.DatabaseLookupFlags{Required: true})
if err != nil {
return nil, err
Expand Down Expand Up @@ -643,7 +643,7 @@ func (p *planner) AlterDatabasePrimaryRegion(
return nil, err
}

_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
tree.DatabaseLookupFlags{Required: true},
)
if err != nil {
Expand Down Expand Up @@ -957,7 +957,7 @@ func (p *planner) AlterDatabaseSurvivalGoal(
return nil, err
}

_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name),
tree.DatabaseLookupFlags{Required: true},
)
if err != nil {
Expand Down
11 changes: 4 additions & 7 deletions pkg/sql/alter_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,23 +53,20 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod
if n.Schema.ExplicitCatalog {
dbName = n.Schema.Catalog()
}
_, db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName,
db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName,
tree.DatabaseLookupFlags{Required: true})
if err != nil {
return nil, err
}
found, schema, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, string(n.Schema.SchemaName), true /* required */)
schema, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, string(n.Schema.SchemaName), true /* required */)
if err != nil {
return nil, err
}
if !found {
return nil, pgerror.Newf(pgcode.InvalidSchemaName, "schema %q does not exist", n.Schema.String())
}
switch schema.Kind {
switch schema.SchemaKind() {
case catalog.SchemaPublic, catalog.SchemaVirtual, catalog.SchemaTemporary:
return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot modify schema %q", n.Schema.String())
case catalog.SchemaUserDefined:
desc := schema.Desc.(*schemadesc.Mutable)
desc := schema.(*schemadesc.Mutable)
// The user must be a superuser or the owner of the schema to modify it.
hasAdmin, err := p.HasAdminRole(ctx)
if err != nil {
Expand Down
28 changes: 14 additions & 14 deletions pkg/sql/authorization.go
Original file line number Diff line number Diff line change
Expand Up @@ -513,13 +513,13 @@ func (p *planner) canCreateOnSchema(
user security.SQLUsername,
checkPublicSchema shouldCheckPublicSchema,
) error {
resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID(
scDesc, err := p.Descriptors().GetImmutableSchemaByID(
ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{})
if err != nil {
return err
}

switch resolvedSchema.Kind {
switch kind := scDesc.SchemaKind(); kind {
case catalog.SchemaPublic:
// The public schema is valid to create in if the parent database is.
if !checkPublicSchema {
Expand All @@ -537,11 +537,11 @@ func (p *planner) canCreateOnSchema(
return nil
case catalog.SchemaVirtual:
return pgerror.Newf(pgcode.InsufficientPrivilege,
"cannot CREATE on schema %s", resolvedSchema.Name)
"cannot CREATE on schema %s", scDesc.GetName())
case catalog.SchemaUserDefined:
return p.CheckPrivilegeForUser(ctx, resolvedSchema.Desc, privilege.CREATE, user)
return p.CheckPrivilegeForUser(ctx, scDesc, privilege.CREATE, user)
default:
panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind))
panic(errors.AssertionFailedf("unknown schema kind %d", kind))
}
}

Expand All @@ -554,20 +554,20 @@ func (p *planner) canResolveDescUnderSchema(
if tbl, ok := desc.(catalog.TableDescriptor); ok && tbl.IsTemporary() {
return nil
}
resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID(
scDesc, err := p.Descriptors().GetImmutableSchemaByID(
ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{})
if err != nil {
return err
}

switch resolvedSchema.Kind {
switch kind := scDesc.SchemaKind(); kind {
case catalog.SchemaPublic, catalog.SchemaTemporary, catalog.SchemaVirtual:
// Anyone can resolve under temporary, public or virtual schemas.
return nil
case catalog.SchemaUserDefined:
return p.CheckPrivilegeForUser(ctx, resolvedSchema.Desc, privilege.USAGE, p.User())
return p.CheckPrivilegeForUser(ctx, scDesc, privilege.USAGE, p.User())
default:
panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind))
panic(errors.AssertionFailedf("unknown schema kind %d", kind))
}
}

Expand Down Expand Up @@ -643,15 +643,15 @@ func (p *planner) HasOwnershipOnSchema(
// Only the node user has ownership over the system database.
return p.User().IsNodeUser(), nil
}
resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID(
scDesc, err := p.Descriptors().GetImmutableSchemaByID(
ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{},
)
if err != nil {
return false, err
}

hasOwnership := false
switch resolvedSchema.Kind {
switch kind := scDesc.SchemaKind(); kind {
case catalog.SchemaPublic:
// admin is the owner of the public schema.
hasOwnership, err = p.UserHasAdminRole(ctx, p.User())
Expand All @@ -663,14 +663,14 @@ func (p *planner) HasOwnershipOnSchema(
case catalog.SchemaTemporary:
// The user owns all the temporary schemas that they created in the session.
hasOwnership = p.SessionData() != nil &&
p.SessionData().IsTemporarySchemaID(uint32(resolvedSchema.ID))
p.SessionData().IsTemporarySchemaID(uint32(scDesc.GetID()))
case catalog.SchemaUserDefined:
hasOwnership, err = p.HasOwnership(ctx, resolvedSchema.Desc)
hasOwnership, err = p.HasOwnership(ctx, scDesc)
if err != nil {
return false, err
}
default:
panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind))
panic(errors.AssertionFailedf("unknown schema kind %d", kind))
}

return hasOwnership, nil
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"descriptor.go",
"descriptor_id_set.go",
"errors.go",
"schema.go",
"table_col_map.go",
"table_col_set.go",
"table_elements.go",
Expand All @@ -21,7 +22,7 @@ go_library(
"//pkg/keys",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/sql/catalog/catconstants",
"//pkg/server/telemetry",
"//pkg/sql/catalog/descpb",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/catalog/accessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,11 +40,11 @@ type Accessor interface {
ctx context.Context, txn *kv.Txn, dbName string, flags tree.DatabaseLookupFlags,
) (DatabaseDescriptor, error)

// GetSchema returns true and a ResolvedSchema object if the target schema
// GetSchemaByName returns true and a SchemaDescriptor object if the target schema
// exists under the target database.
GetSchemaByName(
ctx context.Context, txn *kv.Txn, dbID descpb.ID, scName string, flags tree.SchemaLookupFlags,
) (bool, ResolvedSchema, error)
) (SchemaDescriptor, error)

// GetObjectNamesAndIDs returns the list of all objects in the given
// database and schema.
Expand Down
Loading

0 comments on commit 02830c2

Please sign in to comment.