Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
82433: sql/sem/builtins: remove dependency of builtins on sql/catalog/descs r=ajwerner a=ajwerner

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

Co-authored-by: Andrew Werner <[email protected]>
  • Loading branch information
craig[bot] and ajwerner committed Jun 6, 2022
2 parents f0372fd + 0a3137c commit 2fc45fc
Show file tree
Hide file tree
Showing 19 changed files with 516 additions and 227 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/distsql/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
35 changes: 35 additions & 0 deletions pkg/sql/evalcatalog/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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"],
)
138 changes: 138 additions & 0 deletions pkg/sql/evalcatalog/encode_table_index_key.go
Original file line number Diff line number Diff line change
@@ -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
}
43 changes: 43 additions & 0 deletions pkg/sql/evalcatalog/eval_catalog.go
Original file line number Diff line number Diff line change
@@ -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
}
88 changes: 88 additions & 0 deletions pkg/sql/evalcatalog/geo_inverted_index_entries.go
Original file line number Diff line number Diff line change
@@ -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
}
Loading

0 comments on commit 2fc45fc

Please sign in to comment.