Skip to content

Commit

Permalink
tree: use idxtype package
Browse files Browse the repository at this point in the history
Update the tree package to use idxtype.T directly rather than using an
alias for it. Add a check to the idxtype BUILD.bazel package that ensures
it stays a leaf package, so that the tree package doesn't take on
additional dependencies beyond what idxtype needs.

Epic: CRDB-42943

Release note: None
  • Loading branch information
andy-kimball committed Jan 28, 2025
1 parent c8040db commit 794a768
Show file tree
Hide file tree
Showing 15 changed files with 70 additions and 67 deletions.
1 change: 1 addition & 0 deletions pkg/internal/sqlsmith/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_library(
"//pkg/sql/sem/cast",
"//pkg/sql/sem/catid",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/idxtype",
"//pkg/sql/sem/plpgsqltree",
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treebin",
Expand Down
5 changes: 3 additions & 2 deletions pkg/internal/sqlsmith/alter.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/cast"
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -340,7 +341,7 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) {
}
var cols tree.IndexElemList
seen := map[tree.Name]bool{}
indexType := tree.IndexTypeForward
indexType := idxtype.FORWARD
unique := s.coin()
for len(cols) < 1 || s.coin() {
col := tableRef.Columns[s.rnd.Intn(len(tableRef.Columns))]
Expand All @@ -351,7 +352,7 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) {
// If this is the first column and it's invertible (i.e., JSONB), make an inverted index.
if len(cols) == 0 &&
colinfo.ColumnTypeIsOnlyInvertedIndexable(tree.MustBeStaticallyKnownType(col.Type)) {
indexType = tree.IndexTypeInverted
indexType = idxtype.INVERTED
unique = false
cols = append(cols, tree.IndexElem{
Column: col.Name,
Expand Down
7 changes: 4 additions & 3 deletions pkg/internal/sqlsmith/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -184,7 +185,7 @@ func (s *Smither) getRandTable() (*aliasedTableRef, bool) {
var indexFlags tree.IndexFlags
indexNames := make([]tree.Name, 0, len(indexes))
for _, index := range indexes {
if index.Type == tree.IndexTypeForward {
if index.Type == idxtype.FORWARD {
indexNames = append(indexNames, index.Name)
}
}
Expand Down Expand Up @@ -499,9 +500,9 @@ func (s *Smither) extractIndexes(
return nil, err
}
if _, ok := indexes[idx]; !ok {
indexType := tree.IndexTypeForward
indexType := idxtype.FORWARD
if inverted {
indexType = tree.IndexTypeInverted
indexType = idxtype.INVERTED
}
indexes[idx] = &tree.CreateIndex{
Name: idx,
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
"github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
Expand Down Expand Up @@ -274,7 +275,7 @@ func (n *alterTableNode) startExec(params runParams) error {
n.tableDesc,
tableName,
columns,
tree.IndexTypeForward,
idxtype.FORWARD,
false, /* isNewTable */
params.p.SemaCtx(),
params.ExecCfg().Settings.Version.ActiveVersion(params.ctx),
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ func makeIndexDescriptor(
`"bucket_count" storage param should only be set with "USING HASH" for hash sharded index`,
)
}
if n.Type == tree.IndexTypeVector {
if n.Type == idxtype.VECTOR {
return nil, unimplemented.NewWithIssuef(137370, "VECTOR indexes are not yet supported")
}

Expand Down Expand Up @@ -227,7 +227,7 @@ func makeIndexDescriptor(
"%s indexes can't be unique", strings.ToLower(n.Type.String()))
}

if n.Type == tree.IndexTypeInverted {
if n.Type == idxtype.INVERTED {
invCol := columns[len(columns)-1]
column, err := catalog.MustFindColumnByTreeName(tableDesc, invCol.Column)
if err != nil {
Expand Down Expand Up @@ -323,7 +323,7 @@ func checkIndexColumns(
desc catalog.TableDescriptor,
columns tree.IndexElemList,
storing tree.NameList,
indexType tree.IndexType,
indexType idxtype.T,
version clusterversion.ClusterVersion,
) error {
for i, colDef := range columns {
Expand Down Expand Up @@ -484,7 +484,7 @@ func replaceExpressionElemsWithVirtualCols(
desc *tabledesc.Mutable,
tn *tree.TableName,
elems tree.IndexElemList,
indexType tree.IndexType,
indexType idxtype.T,
isNewTable bool,
semaCtx *tree.SemaContext,
version clusterversion.ClusterVersion,
Expand Down Expand Up @@ -549,7 +549,7 @@ func replaceExpressionElemsWithVirtualCols(
)
}

if indexType != tree.IndexTypeInverted && !colinfo.ColumnTypeIsIndexable(typ) {
if indexType != idxtype.INVERTED && !colinfo.ColumnTypeIsIndexable(typ) {
if colinfo.ColumnTypeIsInvertedIndexable(typ) {
return errors.WithHint(
pgerror.Newf(
Expand All @@ -569,7 +569,7 @@ func replaceExpressionElemsWithVirtualCols(
)
}

if indexType == tree.IndexTypeInverted {
if indexType == idxtype.INVERTED {
if i < lastColumnIdx && !colinfo.ColumnTypeIsIndexable(typ) {
return errors.WithHint(
pgerror.Newf(
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1887,7 +1887,7 @@ func NewTableDesc(
// pass, handled above.

case *tree.IndexTableDef:
if d.Type == tree.IndexTypeVector {
if d.Type == idxtype.VECTOR {
return nil, unimplemented.NewWithIssuef(137370, "VECTOR indexes are not yet supported")
}
// If the index is named, ensure that the name is unique. Unnamed
Expand Down Expand Up @@ -1940,7 +1940,7 @@ func NewTableDesc(
if err := idx.FillColumns(columns); err != nil {
return nil, err
}
if d.Type == tree.IndexTypeInverted {
if d.Type == idxtype.INVERTED {
column, err := catalog.MustFindColumnByName(&desc, idx.InvertedColumnName())
if err != nil {
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1653,7 +1653,7 @@ func partialIndexCount(tab cat.Table) int {
func vectorIndexCount(tab cat.Table) int {
count := 0
for i, n := 0, tab.DeletableIndexCount(); i < n; i++ {
if tab.Index(i).Type() == tree.IndexTypeVector {
if tab.Index(i).Type() == idxtype.VECTOR {
count++
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/parser/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ go_library(
"//pkg/sql/privilege", # keep
"//pkg/sql/scanner",
"//pkg/sql/sem/builtins/builtinsregistry",
"//pkg/sql/sem/idxtype", # keep
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treebin", # keep
"//pkg/sql/sem/tree/treecmp", # keep
Expand Down
33 changes: 17 additions & 16 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow"
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/vector"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -940,8 +941,8 @@ func (u *sqlSymUnion) triggerTransitions() []*tree.TriggerTransition {
func (u *sqlSymUnion) triggerForEach() tree.TriggerForEach {
return u.val.(tree.TriggerForEach)
}
func (u *sqlSymUnion) indexType() tree.IndexType {
return u.val.(tree.IndexType)
func (u *sqlSymUnion) indexType() idxtype.T {
return u.val.(idxtype.T)
}
func (u *sqlSymUnion) doBlockOptions() tree.DoBlockOptions {
return u.val.(tree.DoBlockOptions)
Expand Down Expand Up @@ -1546,7 +1547,7 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption {
%type <[]*tree.Order> sortby_list sortby_no_index_list
%type <tree.IndexElemList> index_params create_as_params
%type <tree.IndexInvisibility> opt_index_visible alter_index_visible
%type <tree.IndexType> opt_index_access_method
%type <idxtype.T> opt_index_access_method
%type <tree.NameList> name_list privilege_list
%type <[]int32> opt_array_bounds
%type <*tree.Batch> opt_batch_clause
Expand Down Expand Up @@ -11231,7 +11232,7 @@ index_def:
$$.val = &tree.IndexTableDef{
Name: "",
Columns: $4.idxElems(),
Type: tree.IndexTypeInverted,
Type: idxtype.INVERTED,
PartitionByIndex: $6.partitionByIndex(),
StorageParams: $7.storageParams(),
Predicate: $8.expr(),
Expand All @@ -11243,7 +11244,7 @@ index_def:
$$.val = &tree.IndexTableDef{
Name: tree.Name($3),
Columns: $5.idxElems(),
Type: tree.IndexTypeInverted,
Type: idxtype.INVERTED,
PartitionByIndex: $7.partitionByIndex(),
StorageParams: $8.storageParams(),
Predicate: $9.expr(),
Expand All @@ -11255,7 +11256,7 @@ index_def:
$$.val = &tree.IndexTableDef{
Name: "",
Columns: $4.idxElems(),
Type: tree.IndexTypeVector,
Type: idxtype.VECTOR,
PartitionByIndex: $6.partitionByIndex(),
StorageParams: $7.storageParams(),
Predicate: $8.expr(),
Expand All @@ -11267,7 +11268,7 @@ index_def:
$$.val = &tree.IndexTableDef{
Name: tree.Name($3),
Columns: $5.idxElems(),
Type: tree.IndexTypeVector,
Type: idxtype.VECTOR,
PartitionByIndex: $7.partitionByIndex(),
StorageParams: $8.storageParams(),
Predicate: $9.expr(),
Expand Down Expand Up @@ -12236,7 +12237,7 @@ create_index_stmt:
Name: tree.Name($6),
Table: table,
Unique: $2.bool(),
Type: tree.IndexTypeInverted,
Type: idxtype.INVERTED,
Columns: $10.idxElems(),
Storing: $12.nameList(),
PartitionByIndex: $13.partitionByIndex(),
Expand All @@ -12253,7 +12254,7 @@ create_index_stmt:
Name: tree.Name($9),
Table: table,
Unique: $2.bool(),
Type: tree.IndexTypeInverted,
Type: idxtype.INVERTED,
IfNotExists: true,
Columns: $13.idxElems(),
Storing: $15.nameList(),
Expand All @@ -12271,7 +12272,7 @@ create_index_stmt:
Name: tree.Name($6),
Table: table,
Unique: $2.bool(),
Type: tree.IndexTypeVector,
Type: idxtype.VECTOR,
Columns: $10.idxElems(),
Storing: $12.nameList(),
PartitionByIndex: $13.partitionByIndex(),
Expand All @@ -12288,7 +12289,7 @@ create_index_stmt:
Name: tree.Name($9),
Table: table,
Unique: $2.bool(),
Type: tree.IndexTypeVector,
Type: idxtype.VECTOR,
IfNotExists: true,
Columns: $13.idxElems(),
Storing: $15.nameList(),
Expand All @@ -12305,14 +12306,14 @@ opt_index_access_method:
USING name
{
/* FORCE DOC */
var val tree.IndexType
var val idxtype.T
switch $2 {
case "gin", "gist":
val = tree.IndexTypeInverted
val = idxtype.INVERTED
case "btree":
val = tree.IndexTypeForward
val = idxtype.FORWARD
case "cspann":
val = tree.IndexTypeVector
val = idxtype.VECTOR
case "hash", "spgist", "brin":
return unimplemented(sqllex, "index using " + $2)
default:
Expand All @@ -12323,7 +12324,7 @@ opt_index_access_method:
}
| /* EMPTY */
{
$$.val = tree.IndexTypeForward
$$.val = idxtype.FORWARD
}

opt_concurrently:
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/randgen/mutator.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -806,7 +807,7 @@ func postgresCreateTableMutator(
// TODO(rafi): Postgres supports inverted indexes with a different
// syntax than Cockroach. Maybe we could add it later.
// The syntax is `CREATE INDEX name ON table USING gin(column)`.
if def.Type == tree.IndexTypeForward {
if def.Type == idxtype.FORWARD {
mutated = append(mutated, &tree.CreateIndex{
Name: def.Name,
Table: mutatedStmt.Table,
Expand Down
Loading

0 comments on commit 794a768

Please sign in to comment.