forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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
Showing
19 changed files
with
516 additions
and
227 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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"], | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} |
Oops, something went wrong.