From 0a3137c2d7de89a1e6b242c308e2afe2c54735f5 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Wed, 25 May 2022 15:54:28 -0400 Subject: [PATCH] sql/sem/builtins: remove dependency of builtins on sql/catalog/descs builtins is a major bottleneck in the build graph. Reducing its deps helps move it to be earlier. This removes effectively all dependencies on the subtree under catalog. The introduction of the new package was required to eliminate the very upsetting method to return a descriptor as an opaque interface. Release note: None --- pkg/BUILD.bazel | 3 + pkg/sql/BUILD.bazel | 1 + pkg/sql/conn_executor.go | 2 + pkg/sql/distsql/server.go | 2 + pkg/sql/evalcatalog/BUILD.bazel | 35 ++++ pkg/sql/evalcatalog/encode_table_index_key.go | 138 +++++++++++++++ pkg/sql/evalcatalog/eval_catalog.go | 43 +++++ .../evalcatalog/geo_inverted_index_entries.go | 88 ++++++++++ pkg/sql/evalcatalog/pg_updatable.go | 98 +++++++++++ pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/flow_context.go | 6 + pkg/sql/faketreeeval/evalctx.go | 7 - pkg/sql/planner.go | 5 + pkg/sql/sem/builtins/BUILD.bazel | 10 +- pkg/sql/sem/builtins/builtins.go | 163 +++--------------- pkg/sql/sem/builtins/pg_builtins.go | 77 +-------- pkg/sql/sem/eval/BUILD.bazel | 9 + pkg/sql/sem/eval/context.go | 4 + pkg/sql/sem/eval/deps.go | 51 +++++- 19 files changed, 516 insertions(+), 227 deletions(-) create mode 100644 pkg/sql/evalcatalog/BUILD.bazel create mode 100644 pkg/sql/evalcatalog/encode_table_index_key.go create mode 100644 pkg/sql/evalcatalog/eval_catalog.go create mode 100644 pkg/sql/evalcatalog/geo_inverted_index_entries.go create mode 100644 pkg/sql/evalcatalog/pg_updatable.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 437dc1f2c47f..2cd8882bf6d8 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -298,6 +298,7 @@ ALL_TESTS = [ "//pkg/sql/distsql:distsql_test", "//pkg/sql/doctor:doctor_test", "//pkg/sql/enum:enum_test", + "//pkg/sql/evalcatalog:evalcatalog_disallowed_imports_test", "//pkg/sql/execinfra:execinfra_disallowed_imports_test", "//pkg/sql/execinfra:execinfra_test", "//pkg/sql/execinfrapb:execinfrapb_disallowed_imports_test", @@ -386,11 +387,13 @@ ALL_TESTS = [ "//pkg/sql/schemachanger/screl:screl_test", "//pkg/sql/schemachanger/scrun:scrun_test", "//pkg/sql/schemachanger:schemachanger_test", + "//pkg/sql/sem/builtins:builtins_disallowed_imports_test", "//pkg/sql/sem/builtins:builtins_test", "//pkg/sql/sem/cast:cast_test", "//pkg/sql/sem/catconstants:catconstants_disallowed_imports_test", "//pkg/sql/sem/eval/cast_test:cast_test_test", "//pkg/sql/sem/eval/eval_test:eval_test_test", + "//pkg/sql/sem/eval:eval_disallowed_imports_test", "//pkg/sql/sem/eval:eval_test", "//pkg/sql/sem/normalize:normalize_test", "//pkg/sql/sem/tree:tree_disallowed_imports_test", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index b3791deaef24..9081571fe766 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -329,6 +329,7 @@ go_library( "//pkg/sql/delegate", "//pkg/sql/distsql", "//pkg/sql/enum", + "//pkg/sql/evalcatalog", "//pkg/sql/execinfra", "//pkg/sql/execinfra/execagg", "//pkg/sql/execinfra/execopnode", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 52c8ec1d189a..36e4df5da257 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -2685,6 +2685,7 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo ConsistencyChecker: p.execCfg.ConsistencyChecker, RangeProber: p.execCfg.RangeProber, StmtDiagnosticsRequestInserter: ex.server.cfg.StmtDiagnosticsRecorder.InsertRequest, + CatalogBuiltins: &p.evalCatalogBuiltins, }, Tracing: &ex.sessionTracing, MemMetrics: &ex.memMetrics, @@ -2821,6 +2822,7 @@ func (ex *connExecutor) resetPlanner( p.schemaResolver.txn = txn p.schemaResolver.sessionDataStack = p.EvalContext().SessionDataStack + p.evalCatalogBuiltins.Init(p.execCfg.Codec, txn, p.Descriptors()) p.skipDescriptorCache = false p.typeResolutionDbID = descpb.InvalidID } diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 8860a07ad2e0..9278d316970c 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -478,6 +478,8 @@ func (ds *ServerImpl) newFlowContext( // on flow cleanup. flowCtx.Descriptors = ds.CollectionFactory.NewCollection(ctx, descs.NewTemporarySchemaProvider(evalCtx.SessionDataStack)) flowCtx.IsDescriptorsCleanupRequired = true + flowCtx.EvalCatalogBuiltins.Init(evalCtx.Codec, evalCtx.Txn, flowCtx.Descriptors) + evalCtx.CatalogBuiltins = &flowCtx.EvalCatalogBuiltins } return flowCtx } diff --git a/pkg/sql/evalcatalog/BUILD.bazel b/pkg/sql/evalcatalog/BUILD.bazel new file mode 100644 index 000000000000..21244168183b --- /dev/null +++ b/pkg/sql/evalcatalog/BUILD.bazel @@ -0,0 +1,35 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") + +go_library( + name = "evalcatalog", + srcs = [ + "encode_table_index_key.go", + "eval_catalog.go", + "geo_inverted_index_entries.go", + "pg_updatable.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/evalcatalog", + visibility = ["//visibility:public"], + deps = [ + "//pkg/geo/geoindex", + "//pkg/keys", + "//pkg/kv", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/rowenc", + "//pkg/sql/sem/catid", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlerrors", + "//pkg/sql/types", + "@com_github_cockroachdb_errors//:errors", + ], +) + +disallowed_imports_test( + src = "evalcatalog", + disallowed_list = ["//pkg/sql/sem/builtins"], +) diff --git a/pkg/sql/evalcatalog/encode_table_index_key.go b/pkg/sql/evalcatalog/encode_table_index_key.go new file mode 100644 index 000000000000..2c06b938e0bf --- /dev/null +++ b/pkg/sql/evalcatalog/encode_table_index_key.go @@ -0,0 +1,138 @@ +// Copyright 2022 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 evalcatalog + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "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/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" +) + +// EncodeTableIndexKey is part of eval.CatalogBuiltins. +func (ec *Builtins) EncodeTableIndexKey( + ctx context.Context, + tableID catid.DescID, + indexID catid.IndexID, + rowDatums *tree.DTuple, + performCast func(context.Context, tree.Datum, *types.T) (tree.Datum, error), +) ([]byte, error) { + // Get the referenced table and index. + tableDesc, err := ec.dc.GetImmutableTableByID( + ctx, ec.txn, tableID, tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + return nil, err + } + index, err := tableDesc.FindIndexWithID(indexID) + if err != nil { + return nil, err + } + // Collect the index columns. If the index is a non-unique secondary + // index, it might have some extra key columns. + indexColIDs := make([]descpb.ColumnID, index.NumKeyColumns(), index.NumKeyColumns()+index.NumKeySuffixColumns()) + for i := 0; i < index.NumKeyColumns(); i++ { + indexColIDs[i] = index.GetKeyColumnID(i) + } + if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() { + for i := 0; i < index.NumKeySuffixColumns(); i++ { + indexColIDs = append(indexColIDs, index.GetKeySuffixColumnID(i)) + } + } + + // Ensure that the input tuple length equals the number of index cols. + if len(rowDatums.D) != len(indexColIDs) { + err := pgerror.Newf( + pgcode.InvalidParameterValue, + "number of values must equal number of columns in index %q", + index.GetName(), + ) + // If the index has some extra key columns, then output an error + // message with some extra information to explain the subtlety. + if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() && index.NumKeySuffixColumns() > 0 { + var extraColNames []string + for i := 0; i < index.NumKeySuffixColumns(); i++ { + id := index.GetKeySuffixColumnID(i) + col, colErr := tableDesc.FindColumnWithID(id) + if colErr != nil { + return nil, errors.CombineErrors(err, colErr) + } + extraColNames = append(extraColNames, col.GetName()) + } + var allColNames []string + for _, id := range indexColIDs { + col, colErr := tableDesc.FindColumnWithID(id) + if colErr != nil { + return nil, errors.CombineErrors(err, colErr) + } + allColNames = append(allColNames, col.GetName()) + } + return nil, errors.WithHintf( + err, + "columns %v are implicitly part of index %q's key, include columns %v in this order", + extraColNames, + index.GetName(), + allColNames, + ) + } + return nil, err + } + + // Check that the input datums are typed as the index columns types. + var datums tree.Datums + for i, d := range rowDatums.D { + // We perform a cast here rather than a type check because datums + // already have a fixed type, and not enough information is known at + // typechecking time to ensure that the datums are typed with the + // types of the index columns. So, try to cast the input datums to + // the types of the index columns here. + var newDatum tree.Datum + col, err := tableDesc.FindColumnWithID(indexColIDs[i]) + if err != nil { + return nil, err + } + if d.ResolvedType() == types.Unknown { + if !col.IsNullable() { + return nil, pgerror.Newf(pgcode.NotNullViolation, "NULL provided as a value for a nonnullable column") + } + newDatum = tree.DNull + } else { + expectedTyp := col.GetType() + newDatum, err = performCast(ctx, d, expectedTyp) + if err != nil { + return nil, errors.WithHint(err, "try to explicitly cast each value to the corresponding column type") + } + } + datums = append(datums, newDatum) + } + + // Create a column id to row index map. In this case, each column ID + // just maps to the i'th ordinal. + var colMap catalog.TableColMap + for i, id := range indexColIDs { + colMap.Set(id, i) + } + // Finally, encode the index key using the provided datums. + keyPrefix := rowenc.MakeIndexKeyPrefix(ec.codec, tableDesc.GetID(), index.GetID()) + keyAndSuffixCols := tableDesc.IndexFetchSpecKeyAndSuffixColumns(index) + if len(datums) > len(keyAndSuffixCols) { + return nil, errors.Errorf("encoding too many columns (%d)", len(datums)) + } + res, _, err := rowenc.EncodePartialIndexKey(keyAndSuffixCols[:len(datums)], colMap, datums, keyPrefix) + return res, err +} diff --git a/pkg/sql/evalcatalog/eval_catalog.go b/pkg/sql/evalcatalog/eval_catalog.go new file mode 100644 index 000000000000..8be7c07f9e94 --- /dev/null +++ b/pkg/sql/evalcatalog/eval_catalog.go @@ -0,0 +1,43 @@ +// Copyright 2022 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 evalcatalog provides the concrete implementation of +// eval.CatalogBuiltins. +package evalcatalog + +import ( + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" +) + +// Builtins implements methods to evaluate logic that depends on having +// catalog access. It implements the eval.Catalog interface. Note that it +// importantly is not the planner directly. +// +// NOTE: The hope is that many of the methods of the planner will make their +// way to this object and that this object may subsume awareness of session +// information. +// +// TODO(ajwerner): Extract the sql.schemaResolver and consider unifying with +// this thing or wrapping that thing. +type Builtins struct { + codec keys.SQLCodec + dc *descs.Collection + txn *kv.Txn +} + +// Init initializes the fields of a Builtins. The object should not be used +// before being initialized. +func (ec *Builtins) Init(codec keys.SQLCodec, txn *kv.Txn, descriptors *descs.Collection) { + ec.codec = codec + ec.txn = txn + ec.dc = descriptors +} diff --git a/pkg/sql/evalcatalog/geo_inverted_index_entries.go b/pkg/sql/evalcatalog/geo_inverted_index_entries.go new file mode 100644 index 000000000000..af78820aa1cc --- /dev/null +++ b/pkg/sql/evalcatalog/geo_inverted_index_entries.go @@ -0,0 +1,88 @@ +// Copyright 2022 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 evalcatalog + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// NumGeometryInvertedIndexEntries is part of the eval.CatalogBuiltins +// interface. +func (ec *Builtins) NumGeometryInvertedIndexEntries( + ctx context.Context, tableID catid.DescID, indexID catid.IndexID, g *tree.DGeometry, +) (int, error) { + geoConfig, err := getIndexGeoConfig(ctx, ec.dc, ec.txn, tableID, indexID) + if err != nil { + return 0, err + } + if geoConfig.S2Geometry == nil { + return 0, pgerror.Newf( + pgcode.InvalidParameterValue, + "index_id %d is not a geography inverted index", indexID, + ) + } + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, geoConfig) + if err != nil { + return 0, err + } + return len(keys), nil +} + +// NumGeographyInvertedIndexEntries is part of the eval.CatalogBuiltins +// interface. +func (ec *Builtins) NumGeographyInvertedIndexEntries( + ctx context.Context, tableID catid.DescID, indexID catid.IndexID, g *tree.DGeography, +) (int, error) { + geoConfig, err := getIndexGeoConfig(ctx, ec.dc, ec.txn, tableID, indexID) + if err != nil { + return 0, err + } + if geoConfig.S2Geography == nil { + return 0, pgerror.Newf( + pgcode.InvalidParameterValue, + "index_id %d is not a geography inverted index", indexID, + ) + } + keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, geoConfig) + if err != nil { + return 0, err + } + return len(keys), nil +} + +func getIndexGeoConfig( + ctx context.Context, + dc *descs.Collection, + txn *kv.Txn, + tableID catid.DescID, + indexID catid.IndexID, +) (geoindex.Config, error) { + tableDesc, err := dc.GetImmutableTableByID( + ctx, txn, tableID, tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + return geoindex.Config{}, err + } + index, err := tableDesc.FindIndexWithID(indexID) + if err != nil { + return geoindex.Config{}, err + } + return index.GetGeoConfig(), nil +} diff --git a/pkg/sql/evalcatalog/pg_updatable.go b/pkg/sql/evalcatalog/pg_updatable.go new file mode 100644 index 000000000000..5eabe58cc9dc --- /dev/null +++ b/pkg/sql/evalcatalog/pg_updatable.go @@ -0,0 +1,98 @@ +// Copyright 2022 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 evalcatalog + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" +) + +// UpdatableCommand matches update operations in postgres. +type UpdatableCommand tree.DInt + +// The following constants are the values for UpdatableCommand enumeration. +const ( + UpdateCommand UpdatableCommand = 2 + iota + InsertCommand + DeleteCommand +) + +var ( + nonUpdatableEvents = tree.NewDInt(0) + allUpdatableEvents = tree.NewDInt((1 << UpdateCommand) | (1 << InsertCommand) | (1 << DeleteCommand)) +) + +// PGRelationIsUpdatable is part of the eval.CatalogBuiltins interface. +func (b *Builtins) PGRelationIsUpdatable(ctx context.Context, oid *tree.DOid) (*tree.DInt, error) { + tableDesc, err := b.dc.GetImmutableTableByID( + ctx, b.txn, descpb.ID(oid.DInt), tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + // For postgres compatibility, it is expected that rather returning + // an error this return nonUpdatableEvents (Zero) because there could + // be oid references on deleted tables. + if sqlerrors.IsUndefinedRelationError(err) { + return nonUpdatableEvents, nil + } + return nonUpdatableEvents, err + } + if !tableDesc.IsTable() || tableDesc.IsVirtualTable() { + return nonUpdatableEvents, nil + } + + // pg_relation_is_updatable was created for compatibility. This + // should return the update events the relation supports, but as crdb + // does not support updatable views or foreign tables, right now this + // basically return allEvents or none. + return allUpdatableEvents, nil +} + +// PGColumnIsUpdatable is part of the eval.CatalogBuiltins interface. +func (b *Builtins) PGColumnIsUpdatable( + ctx context.Context, oidArg *tree.DOid, attNumArg tree.DInt, +) (*tree.DBool, error) { + oid := descpb.ID(oidArg.DInt) + if attNumArg < 0 { + // System columns are not updatable. + return tree.DBoolFalse, nil + } + attNum := descpb.PGAttributeNum(attNumArg) + tableDesc, err := b.dc.GetImmutableTableByID(ctx, b.txn, oid, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + if sqlerrors.IsUndefinedRelationError(err) { + // For postgres compatibility, it is expected that rather returning + // an error this return nonUpdatableEvents (Zero) because there could + // be oid references on deleted tables. + return tree.DBoolFalse, nil + } + return nil, err + } + if !tableDesc.IsTable() || tableDesc.IsVirtualTable() { + return tree.DBoolFalse, nil + } + + column, err := tableDesc.FindColumnWithPGAttributeNum(attNum) + if err != nil { + if sqlerrors.IsUndefinedColumnError(err) { + // When column does not exist postgres returns true. + return tree.DBoolTrue, nil + } + return nil, err + } + + // pg_column_is_updatable was created for compatibility. This + // will return true if is a table (not virtual) and column is not + // a computed column. + return tree.MakeDBool(tree.DBool(!column.IsComputed())), nil +} diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 19fab071916c..585c4037d930 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/tabledesc", + "//pkg/sql/evalcatalog", "//pkg/sql/execinfrapb", "//pkg/sql/rowenc", "//pkg/sql/rowenc/valueside", diff --git a/pkg/sql/execinfra/flow_context.go b/pkg/sql/execinfra/flow_context.go index 2b96d7a37140..1cbb2f653d69 100644 --- a/pkg/sql/execinfra/flow_context.go +++ b/pkg/sql/execinfra/flow_context.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/evalcatalog" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -63,6 +64,11 @@ type FlowCtx struct { // this object must be released. Descriptors *descs.Collection + // EvalCatalogBuiltins is initialized if the flow context is remote and the + // above descs.Collection is non-nil. It is referenced in the eval.Context + // in order to provide catalog access to builtins. + EvalCatalogBuiltins evalcatalog.Builtins + // IsDescriptorsCleanupRequired is set if Descriptors needs to release the // leases it acquired after the flow is complete. IsDescriptorsCleanupRequired bool diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 70bce742b10e..e3b359a0807b 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -177,13 +177,6 @@ func (ep *DummyEvalPlanner) UnsafeUpsertDescriptor( return errors.WithStack(errEvalPlanner) } -// GetImmutableTableInterfaceByID is part of the Planner interface. -func (ep *DummyEvalPlanner) GetImmutableTableInterfaceByID( - ctx context.Context, id int, -) (interface{}, error) { - return nil, errors.WithStack(errEvalPlanner) -} - // UnsafeDeleteDescriptor is part of the Planner interface. func (ep *DummyEvalPlanner) UnsafeDeleteDescriptor( ctx context.Context, descID int64, force bool, diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 4be9aa7f90b4..69f8039f2585 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" + "github.com/cockroachdb/cockroach/pkg/sql/evalcatalog" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -237,6 +238,9 @@ type planner struct { noticeSender noticeSender queryCacheSession querycache.Session + + // evalCatalogBuiltins is used as part of the eval.Context. + evalCatalogBuiltins evalcatalog.Builtins } func (evalCtx *extendedEvalContext) setSessionID(sessionID clusterunique.ID) { @@ -412,6 +416,7 @@ func newInternalPlanner( p.schemaResolver.sessionDataStack = sds p.schemaResolver.txn = p.txn p.schemaResolver.authAccessor = p + p.evalCatalogBuiltins.Init(execCfg.Codec, p.txn, p.Descriptors()) return p, func() { // Note that we capture ctx here. This is only valid as long as we create diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 2ec5058fcc06..836b095abaf4 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "builtins", @@ -51,7 +52,6 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/descs", "//pkg/sql/colexecerror", "//pkg/sql/lex", "//pkg/sql/lexbase", @@ -75,7 +75,6 @@ go_library( "//pkg/sql/sem/volatility", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", - "//pkg/sql/sqlerrors", "//pkg/sql/sqlliveness", "//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil", "//pkg/sql/sqltelemetry", @@ -170,3 +169,10 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + src = "builtins", + disallowed_list = [ + "//pkg/sql/catalog/descs", + ], +) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index dde16d4ca852..3d27fcaad6ee 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -13,6 +13,7 @@ package builtins import ( "bytes" "compress/gzip" + "context" "crypto/md5" cryptorand "crypto/rand" "crypto/sha1" @@ -47,10 +48,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "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/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" @@ -63,6 +62,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" "github.com/cockroachdb/cockroach/pkg/sql/sem/asof" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" @@ -4920,8 +4920,8 @@ value if you rely on the HLC for accuracy.`, }, ReturnType: tree.FixedReturnType(types.Bytes), Fn: func(ctx *eval.Context, args tree.Datums) (tree.Datum, error) { - tableID := int(tree.MustBeDInt(args[0])) - indexID := int(tree.MustBeDInt(args[1])) + tableID := catid.DescID(tree.MustBeDInt(args[0])) + indexID := catid.IndexID(tree.MustBeDInt(args[1])) rowDatums, ok := tree.AsDTuple(args[2]) if !ok { return nil, pgerror.Newf( @@ -4930,107 +4930,14 @@ value if you rely on the HLC for accuracy.`, args[2], ) } - - // Get the referenced table and index. - tableDescI, err := ctx.Planner.GetImmutableTableInterfaceByID(ctx.Ctx(), tableID) - if err != nil { - return nil, err - } - tableDesc := tableDescI.(catalog.TableDescriptor) - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) - if err != nil { - return nil, err - } - // Collect the index columns. If the index is a non-unique secondary - // index, it might have some extra key columns. - indexColIDs := make([]descpb.ColumnID, index.NumKeyColumns(), index.NumKeyColumns()+index.NumKeySuffixColumns()) - for i := 0; i < index.NumKeyColumns(); i++ { - indexColIDs[i] = index.GetKeyColumnID(i) - } - if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() { - for i := 0; i < index.NumKeySuffixColumns(); i++ { - indexColIDs = append(indexColIDs, index.GetKeySuffixColumnID(i)) - } - } - - // Ensure that the input tuple length equals the number of index cols. - if len(rowDatums.D) != len(indexColIDs) { - err := errors.Newf( - "number of values must equal number of columns in index %q", - index.GetName(), - ) - // If the index has some extra key columns, then output an error - // message with some extra information to explain the subtlety. - if index.GetID() != tableDesc.GetPrimaryIndexID() && !index.IsUnique() && index.NumKeySuffixColumns() > 0 { - var extraColNames []string - for i := 0; i < index.NumKeySuffixColumns(); i++ { - id := index.GetKeySuffixColumnID(i) - col, colErr := tableDesc.FindColumnWithID(id) - if colErr != nil { - return nil, errors.CombineErrors(err, colErr) - } - extraColNames = append(extraColNames, col.GetName()) - } - var allColNames []string - for _, id := range indexColIDs { - col, colErr := tableDesc.FindColumnWithID(id) - if colErr != nil { - return nil, errors.CombineErrors(err, colErr) - } - allColNames = append(allColNames, col.GetName()) - } - return nil, errors.WithHintf( - err, - "columns %v are implicitly part of index %q's key, include columns %v in this order", - extraColNames, - index.GetName(), - allColNames, - ) - } - return nil, err - } - - // Check that the input datums are typed as the index columns types. - var datums tree.Datums - for i, d := range rowDatums.D { - // We perform a cast here rather than a type check because datums - // already have a fixed type, and not enough information is known at - // typechecking time to ensure that the datums are typed with the - // types of the index columns. So, try to cast the input datums to - // the types of the index columns here. - var newDatum tree.Datum - col, err := tableDesc.FindColumnWithID(indexColIDs[i]) - if err != nil { - return nil, err - } - if d.ResolvedType() == types.Unknown { - if !col.IsNullable() { - return nil, pgerror.Newf(pgcode.NotNullViolation, "NULL provided as a value for a non-nullable column") - } - newDatum = tree.DNull - } else { - expectedTyp := col.GetType() - newDatum, err = eval.PerformCast(ctx, d, expectedTyp) - if err != nil { - return nil, errors.WithHint(err, "try to explicitly cast each value to the corresponding column type") - } - } - datums = append(datums, newDatum) - } - - // Create a column id to row index map. In this case, each column ID - // just maps to the i'th ordinal. - var colMap catalog.TableColMap - for i, id := range indexColIDs { - colMap.Set(id, i) - } - // Finally, encode the index key using the provided datums. - keyPrefix := rowenc.MakeIndexKeyPrefix(ctx.Codec, tableDesc.GetID(), index.GetID()) - keyAndSuffixCols := tableDesc.IndexFetchSpecKeyAndSuffixColumns(index) - if len(datums) > len(keyAndSuffixCols) { - return nil, errors.Errorf("encoding too many columns (%d)", len(datums)) - } - res, _, err := rowenc.EncodePartialIndexKey(keyAndSuffixCols[:len(datums)], colMap, datums, keyPrefix) + res, err := ctx.CatalogBuiltins.EncodeTableIndexKey( + ctx.Ctx(), tableID, indexID, rowDatums, + func( + _ context.Context, d tree.Datum, t *types.T, + ) (tree.Datum, error) { + return eval.PerformCast(ctx, d, t) + }, + ) if err != nil { return nil, err } @@ -5551,29 +5458,14 @@ value if you rely on the HLC for accuracy.`, if args[0] == tree.DNull || args[1] == tree.DNull || args[2] == tree.DNull { return tree.DZero, nil } - tableID := int(tree.MustBeDInt(args[0])) - indexID := int(tree.MustBeDInt(args[1])) + tableID := catid.DescID(tree.MustBeDInt(args[0])) + indexID := catid.IndexID(tree.MustBeDInt(args[1])) g := tree.MustBeDGeography(args[2]) - // TODO(postamar): give the eval.Context a useful interface - // instead of cobbling a descs.Collection in this way. - cf := descs.NewBareBonesCollectionFactory(ctx.Settings, ctx.Codec) - descsCol := cf.MakeCollection(ctx.Context, descs.NewTemporarySchemaProvider(ctx.SessionDataStack), nil /* monitor */) - tableDesc, err := descsCol.Direct().MustGetTableDescByID(ctx.Ctx(), ctx.Txn, descpb.ID(tableID)) - if err != nil { - return nil, err - } - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) - if err != nil { - return nil, err - } - if index.GetGeoConfig().S2Geography == nil { - return nil, errors.Errorf("index_id %d is not a geography inverted index", indexID) - } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.GetGeoConfig()) + n, err := ctx.CatalogBuiltins.NumGeographyInvertedIndexEntries(ctx.Ctx(), tableID, indexID, g) if err != nil { return nil, err } - return tree.NewDInt(tree.DInt(len(keys))), nil + return tree.NewDInt(tree.DInt(n)), nil }, Info: "This function is used only by CockroachDB's developers for testing purposes.", Volatility: volatility.Stable, @@ -5589,29 +5481,14 @@ value if you rely on the HLC for accuracy.`, if args[0] == tree.DNull || args[1] == tree.DNull || args[2] == tree.DNull { return tree.DZero, nil } - tableID := int(tree.MustBeDInt(args[0])) - indexID := int(tree.MustBeDInt(args[1])) + tableID := catid.DescID(tree.MustBeDInt(args[0])) + indexID := catid.IndexID(tree.MustBeDInt(args[1])) g := tree.MustBeDGeometry(args[2]) - // TODO(postamar): give the eval.Context a useful interface - // instead of cobbling a descs.Collection in this way. - cf := descs.NewBareBonesCollectionFactory(ctx.Settings, ctx.Codec) - descsCol := cf.MakeCollection(ctx.Context, descs.NewTemporarySchemaProvider(ctx.SessionDataStack), nil /* monitor */) - tableDesc, err := descsCol.Direct().MustGetTableDescByID(ctx.Ctx(), ctx.Txn, descpb.ID(tableID)) - if err != nil { - return nil, err - } - index, err := tableDesc.FindIndexWithID(descpb.IndexID(indexID)) - if err != nil { - return nil, err - } - if index.GetGeoConfig().S2Geometry == nil { - return nil, errors.Errorf("index_id %d is not a geometry inverted index", indexID) - } - keys, err := rowenc.EncodeGeoInvertedIndexTableKeys(g, nil, index.GetGeoConfig()) + n, err := ctx.CatalogBuiltins.NumGeometryInvertedIndexEntries(ctx.Ctx(), tableID, indexID, g) if err != nil { return nil, err } - return tree.NewDInt(tree.DInt(len(keys))), nil + return tree.NewDInt(tree.DInt(n)), nil }, Info: "This function is used only by CockroachDB's developers for testing purposes.", Volatility: volatility.Stable, diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 75309df42346..9e09db8f9acb 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -30,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/errors" @@ -86,21 +84,6 @@ var typeBuiltinsHaveUnderscore = map[oid.Oid]struct{}{ types.AnyTuple.Oid(): {}, } -// UpdatableCommand matches update operations in postgres. -type UpdatableCommand tree.DInt - -// The following constants are the values for UpdatableCommand enumeration. -const ( - UpdateCommand UpdatableCommand = 2 + iota - InsertCommand - DeleteCommand -) - -var ( - nonUpdatableEvents = tree.NewDInt(0) - allUpdatableEvents = tree.NewDInt((1 << UpdateCommand) | (1 << InsertCommand) | (1 << DeleteCommand)) -) - // PGIOBuiltinPrefix returns the string prefix to a type's IO functions. This // is either the type's postgres display name or the type's postgres display // name plus an underscore, depending on the type. @@ -1220,28 +1203,11 @@ SELECT description Types: tree.ArgTypes{{"reloid", types.Oid}, {"include_triggers", types.Bool}}, ReturnType: tree.FixedReturnType(types.Int4), Fn: func(ctx *eval.Context, args tree.Datums) (tree.Datum, error) { - oidArg := tree.MustBeDOid(args[0]) - oid := int(oidArg.DInt) - tableDescI, err := ctx.Planner.GetImmutableTableInterfaceByID(ctx.Ctx(), oid) + ret, err := ctx.CatalogBuiltins.PGRelationIsUpdatable(ctx.Ctx(), tree.MustBeDOid(args[0])) if err != nil { - // For postgres compatibility, it is expected that rather returning - // an error this return nonUpdatableEvents (Zero) because there could - // be oid references on deleted tables. - if sqlerrors.IsUndefinedRelationError(err) { - return nonUpdatableEvents, nil - } - return nonUpdatableEvents, err - } - tableDesc := tableDescI.(catalog.TableDescriptor) - if !tableDesc.IsTable() || tableDesc.IsVirtualTable() { - return nonUpdatableEvents, nil + return nil, err } - - // pg_relation_is_updatable was created for compatibility. This - // should return the update events the relation supports, but as crdb - // does not support updatable views or foreign tables, right now this - // basically return allEvents or none. - return allUpdatableEvents, nil + return ret, nil }, Info: `Returns the update events the relation supports.`, Volatility: volatility.Stable, @@ -1258,42 +1224,11 @@ SELECT description }, ReturnType: tree.FixedReturnType(types.Bool), Fn: func(ctx *eval.Context, args tree.Datums) (tree.Datum, error) { - oidArg := tree.MustBeDOid(args[0]) - attNumArg := tree.MustBeDInt(args[1]) - oid := int(oidArg.DInt) - attNum := uint32(attNumArg) - if attNumArg < 0 { - // System columns are not updatable. - return tree.DBoolFalse, nil - } - tableDescI, err := ctx.Planner.GetImmutableTableInterfaceByID(ctx.Ctx(), oid) + ret, err := ctx.CatalogBuiltins.PGColumnIsUpdatable(ctx.Ctx(), tree.MustBeDOid(args[0]), tree.MustBeDInt(args[1])) if err != nil { - if sqlerrors.IsUndefinedRelationError(err) { - // For postgres compatibility, it is expected that rather returning - // an error this return nonUpdatableEvents (Zero) because there could - // be oid references on deleted tables. - return tree.DBoolFalse, nil - } - return tree.DBoolFalse, err - } - tableDesc := tableDescI.(catalog.TableDescriptor) - if !tableDesc.IsTable() || tableDesc.IsVirtualTable() { - return tree.DBoolFalse, nil - } - - column, err := tableDesc.FindColumnWithID(descpb.ColumnID(attNum)) - if err != nil { - if sqlerrors.IsUndefinedColumnError(err) { - // When column does not exist postgres returns true. - return tree.DBoolTrue, nil - } - return tree.DBoolFalse, err + return nil, err } - - // pg_column_is_updatable was created for compatibility. This - // will return true if is a table (not virtual) and column is not - // a computed column. - return tree.MakeDBool(tree.DBool(!column.IsComputed())), nil + return ret, nil }, Info: `Returns whether the given column can be updated.`, Volatility: volatility.Stable, diff --git a/pkg/sql/sem/eval/BUILD.bazel b/pkg/sql/sem/eval/BUILD.bazel index 68b59b8d767e..7b20225264f0 100644 --- a/pkg/sql/sem/eval/BUILD.bazel +++ b/pkg/sql/sem/eval/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") filegroup( name = "testdata", @@ -52,6 +53,7 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/roleoption", "//pkg/sql/sem/cast", + "//pkg/sql/sem/catid", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", "//pkg/sql/sem/tree/treecmp", @@ -130,3 +132,10 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + src = "eval", + disallowed_list = [ + "//pkg/sql/catalog", + ], +) diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index 4c553ac04422..32df55b4314b 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -208,6 +208,10 @@ type Context struct { // crdb_internal.request_statement_bundle builtin to insert a statement bundle // request. StmtDiagnosticsRequestInserter StmtDiagnosticsRequestInsertFunc + + // CatalogBuiltins is used by various builtins which depend on looking up + // catalog information. Unlike the Planner, it is available in DistSQL. + CatalogBuiltins CatalogBuiltins } var _ tree.ParseTimeContext = &Context{} diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 5b95fb3ef08d..7cd660c65062 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -82,6 +83,52 @@ type DatabaseCatalog interface { HasAnyPrivilege(ctx context.Context, specifier HasPrivilegeSpecifier, user username.SQLUsername, privs []privilege.Privilege) (HasAnyPrivilegeResult, error) } +// CastFunc is a function which cases a datum to a given type. +type CastFunc = func(context.Context, tree.Datum, *types.T) (tree.Datum, error) + +// CatalogBuiltins is a set of methods which can be implemented using the +// lower-level descs.Collection for use in builtins. Its functionality is +// available also during DistSQL making it possible to implement these +// functions without disallowing DistSQL. +// +// TODO(ajwerner): Ideally we'd peel more and more catalog functionality off +// of the Planner interface as we subsume its privilege checking into an +// intermediate layer. +type CatalogBuiltins interface { + // EncodeTableIndexKey constructs a deterministic and immutable encoding of + // a table index key from a tuple of datums. It is leveraged as the + // input to a hash function for hash-sharded indexes. + EncodeTableIndexKey( + ctx context.Context, + tableID catid.DescID, + indexID catid.IndexID, + rowDatums *tree.DTuple, + performCast CastFunc, + ) ([]byte, error) + + // NumGeometryInvertedIndexEntries computes the number of inverted index + // entries we'd expect to generate from a given geometry value given the + // index's configuration. + NumGeometryInvertedIndexEntries( + ctx context.Context, tableID catid.DescID, indexID catid.IndexID, g *tree.DGeometry, + ) (int, error) + + // NumGeographyInvertedIndexEntries computes the number of inverted index + // entries we'd expect to generate from a given geography value given the + // index's configuration. + NumGeographyInvertedIndexEntries( + ctx context.Context, tableID catid.DescID, indexID catid.IndexID, g *tree.DGeography, + ) (int, error) + + // PGColumnIsUpdatable returns whether the given column can be updated. + PGColumnIsUpdatable( + ctx context.Context, oidArg *tree.DOid, attNumArg tree.DInt, + ) (*tree.DBool, error) + + // PGRelationIsUpdatable returns the update events the relation supports. + PGRelationIsUpdatable(ctx context.Context, oid *tree.DOid) (*tree.DInt, error) +} + // HasPrivilegeSpecifier specifies an object to lookup privilege for. // Only one of { DatabaseName, DatabaseOID, SchemaName, TableName, TableOID } is filled. type HasPrivilegeSpecifier struct { @@ -145,10 +192,6 @@ type Planner interface { // ExecutorConfig returns *ExecutorConfig ExecutorConfig() interface{} - // GetImmutableTableInterfaceByID returns an interface{} with - // catalog.TableDescriptor to avoid a circular dependency. - GetImmutableTableInterfaceByID(ctx context.Context, id int) (interface{}, error) - // GetTypeFromValidSQLSyntax parses a column type when the input // string uses the parseable SQL representation of a type name, e.g. // `INT(13)`, `mytype`, `"mytype"`, `pg_catalog.int4` or `"public".mytype`.