diff --git a/pkg/internal/sqlsmith/BUILD.bazel b/pkg/internal/sqlsmith/BUILD.bazel index a990f0941eef..b73a4cc0199a 100644 --- a/pkg/internal/sqlsmith/BUILD.bazel +++ b/pkg/internal/sqlsmith/BUILD.bazel @@ -29,7 +29,6 @@ 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", diff --git a/pkg/internal/sqlsmith/alter.go b/pkg/internal/sqlsmith/alter.go index 07121c82c70e..0629574dbae4 100644 --- a/pkg/internal/sqlsmith/alter.go +++ b/pkg/internal/sqlsmith/alter.go @@ -13,7 +13,6 @@ 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" @@ -341,7 +340,7 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) { } var cols tree.IndexElemList seen := map[tree.Name]bool{} - indexType := idxtype.FORWARD + indexType := tree.IndexTypeForward unique := s.coin() for len(cols) < 1 || s.coin() { col := tableRef.Columns[s.rnd.Intn(len(tableRef.Columns))] @@ -352,7 +351,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 = idxtype.INVERTED + indexType = tree.IndexTypeInverted unique = false cols = append(cols, tree.IndexElem{ Column: col.Name, @@ -367,7 +366,7 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) { } } var storing tree.NameList - for idxtype.SupportsStoring(indexType) && s.coin() { + for indexType == tree.IndexTypeForward && s.coin() { col := tableRef.Columns[s.rnd.Intn(len(tableRef.Columns))] if seen[col.Name] { continue @@ -391,7 +390,7 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) { Unique: unique, Columns: cols, Storing: storing, - Type: tree.IndexType(indexType), + Type: indexType, Concurrently: s.coin(), Invisibility: invisibility, }, true diff --git a/pkg/internal/sqlsmith/schema.go b/pkg/internal/sqlsmith/schema.go index e15aa9e05d5a..6f5a124d03e3 100644 --- a/pkg/internal/sqlsmith/schema.go +++ b/pkg/internal/sqlsmith/schema.go @@ -18,7 +18,6 @@ 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" @@ -500,14 +499,14 @@ func (s *Smither) extractIndexes( return nil, err } if _, ok := indexes[idx]; !ok { - indexType := idxtype.FORWARD + indexType := tree.IndexTypeForward if inverted { - indexType = idxtype.INVERTED + indexType = tree.IndexTypeInverted } indexes[idx] = &tree.CreateIndex{ Name: idx, Table: *t.TableName, - Type: tree.IndexType(indexType), + Type: indexType, } } create := indexes[idx] diff --git a/pkg/sql/opt/cat/BUILD.bazel b/pkg/sql/opt/cat/BUILD.bazel index 4cd147613bc2..56b9b0c2c84f 100644 --- a/pkg/sql/opt/cat/BUILD.bazel +++ b/pkg/sql/opt/cat/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/roleoption", "//pkg/sql/sem/catid", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/types", diff --git a/pkg/sql/opt/cat/index.go b/pkg/sql/opt/cat/index.go index c11da3cd884c..a8ef4ab467a4 100644 --- a/pkg/sql/opt/cat/index.go +++ b/pkg/sql/opt/cat/index.go @@ -9,6 +9,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo/geopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -45,15 +46,12 @@ type Index interface { // Specifically idx = Table().Index(idx.Ordinal). Ordinal() IndexOrdinal + // Type returns the type of this index: forward, inverted, vector, etc. + Type() idxtype.T + // IsUnique returns true if this index is declared as UNIQUE in the schema. IsUnique() bool - // IsInverted returns true if this is an inverted index. - IsInverted() bool - - // IsVector returns true if this is a vector index. - IsVector() bool - // GetInvisibility returns index invisibility. GetInvisibility() float64 diff --git a/pkg/sql/opt/cat/utils.go b/pkg/sql/opt/cat/utils.go index 52e4fafb5b52..a8d26055628a 100644 --- a/pkg/sql/opt/cat/utils.go +++ b/pkg/sql/opt/cat/utils.go @@ -11,6 +11,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/treeprinter" @@ -117,16 +118,19 @@ func FormatTable( // formatCatalogIndex nicely formats a catalog index using a treeprinter for // debugging and testing. func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node, redactableValues bool) { - idx := tab.Index(ord) - idxType := "" - if idx.Ordinal() == PrimaryIndex { - idxType = "PRIMARY " - } else if idx.IsUnique() { - idxType = "UNIQUE " - } else if idx.IsInverted() { - idxType = "INVERTED " - } else if idx.IsVector() { - idxType = "VECTOR " + index := tab.Index(ord) + indexType := "" + if index.Ordinal() == PrimaryIndex { + indexType = "PRIMARY " + } else if index.IsUnique() { + indexType = "UNIQUE " + } else { + switch index.Type() { + case idxtype.INVERTED: + indexType = "INVERTED " + case idxtype.VECTOR: + indexType = "VECTOR " + } } mutation := "" if IsMutationIndex(tab, ord) { @@ -134,7 +138,7 @@ func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node, redactableValue } idxVisibililty := "" - if invisibility := idx.GetInvisibility(); invisibility != 0.0 { + if invisibility := index.GetInvisibility(); invisibility != 0.0 { if invisibility == 1.0 { idxVisibililty = " NOT VISIBLE" } else { @@ -142,41 +146,41 @@ func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node, redactableValue } } - child := tp.Childf("%sINDEX %s%s%s", idxType, idx.Name(), mutation, idxVisibililty) + child := tp.Childf("%sINDEX %s%s%s", indexType, index.Name(), mutation, idxVisibililty) var buf bytes.Buffer - colCount := idx.ColumnCount() + colCount := index.ColumnCount() if ord == PrimaryIndex { // Omit the "stored" columns from the primary index. - colCount = idx.KeyColumnCount() + colCount = index.KeyColumnCount() } for i := 0; i < colCount; i++ { buf.Reset() - idxCol := idx.Column(i) + idxCol := index.Column(i) formatColumn(idxCol.Column, &buf, redactableValues) if idxCol.Descending { fmt.Fprintf(&buf, " desc") } - if i >= idx.LaxKeyColumnCount() { + if i >= index.LaxKeyColumnCount() { fmt.Fprintf(&buf, " (storing)") } - if i < idx.ImplicitColumnCount() { + if i < index.ImplicitColumnCount() { fmt.Fprintf(&buf, " (implicit)") } child.Child(buf.String()) } - FormatZone(idx.Zone(), child) + FormatZone(index.Zone(), child) - if n := idx.PartitionCount(); n > 0 { + if n := index.PartitionCount(); n > 0 { c := child.Child("partitions") for i := 0; i < n; i++ { - p := idx.Partition(i) + p := index.Partition(i) part := c.Child(p.Name()) prefixes := part.Child("partition by list prefixes") for _, datums := range p.PartitionByListPrefixes() { @@ -185,7 +189,7 @@ func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node, redactableValue FormatZone(p.Zone(), part) } } - if pred, isPartial := idx.Predicate(); isPartial { + if pred, isPartial := index.Predicate(); isPartial { child.Childf("WHERE %s", MaybeMarkRedactable(pred, redactableValues)) } } diff --git a/pkg/sql/opt/exec/execbuilder/BUILD.bazel b/pkg/sql/opt/exec/execbuilder/BUILD.bazel index b13e0c745a99..9b80cbb7245d 100644 --- a/pkg/sql/opt/exec/execbuilder/BUILD.bazel +++ b/pkg/sql/opt/exec/execbuilder/BUILD.bazel @@ -39,6 +39,7 @@ go_library( "//pkg/sql/sem/builtins/builtinsregistry", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", "//pkg/sql/sem/tree/treecmp", diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 93cca29213db..899bdb3c8878 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "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/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" @@ -584,7 +585,7 @@ func (b *Builder) scanParams( // index in the memo. if scan.Flags.ForceIndex && scan.Flags.Index != scan.Index { idx := tab.Index(scan.Flags.Index) - isInverted := idx.IsInverted() + isInverted := idx.Type() == idxtype.INVERTED _, isPartial := idx.Predicate() var err error @@ -756,11 +757,13 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (_ execPlan, outputCols colOrdM } idx := tab.Index(scan.Index) - if idx.IsInverted() && len(scan.InvertedConstraint) == 0 && scan.Constraint == nil { - return execPlan{}, colOrdMap{}, - errors.AssertionFailedf("expected inverted index scan to have a constraint") + if idx.Type() == idxtype.INVERTED { + if len(scan.InvertedConstraint) == 0 && scan.Constraint == nil { + return execPlan{}, colOrdMap{}, + errors.AssertionFailedf("expected inverted index scan to have a constraint") + } } - if idx.IsVector() { + if idx.Type() == tree.IndexTypeVector { return execPlan{}, colOrdMap{}, errors.AssertionFailedf( "only VectorSearch operators can use vector indexes") } diff --git a/pkg/sql/opt/exec/explain/BUILD.bazel b/pkg/sql/opt/exec/explain/BUILD.bazel index a03d9535682f..443681fe3ac9 100644 --- a/pkg/sql/opt/exec/explain/BUILD.bazel +++ b/pkg/sql/opt/exec/explain/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sessiondatapb", "//pkg/sql/types", diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 0bdb6adf6a55..1f628d7d38ad 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "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/types" "github.com/cockroachdb/cockroach/pkg/util" @@ -681,12 +682,8 @@ func (u *unknownIndex) IsUnique() bool { return false } -func (u *unknownIndex) IsInverted() bool { - return false -} - -func (u *unknownIndex) IsVector() bool { - return false +func (u *unknownIndex) Type() idxtype.T { + return idxtype.FORWARD } func (u *unknownIndex) GetInvisibility() float64 { diff --git a/pkg/sql/opt/indexrec/BUILD.bazel b/pkg/sql/opt/indexrec/BUILD.bazel index 4b3c72d4be01..16762a2c6824 100644 --- a/pkg/sql/opt/indexrec/BUILD.bazel +++ b/pkg/sql/opt/indexrec/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/sql/opt", "//pkg/sql/opt/cat", "//pkg/sql/opt/memo", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/intsets", diff --git a/pkg/sql/opt/indexrec/hypothetical_index.go b/pkg/sql/opt/indexrec/hypothetical_index.go index 025e79884dd0..e7920c14a5e5 100644 --- a/pkg/sql/opt/indexrec/hypothetical_index.go +++ b/pkg/sql/opt/indexrec/hypothetical_index.go @@ -11,6 +11,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/intsets" @@ -41,8 +42,8 @@ type hypotheticalIndex struct { // columns (neither index columns nor suffix key columns). storedCols []cat.IndexColumn - // inverted indicates if an index is inverted. - inverted bool + // typ indicates the type of the index - forward, inverted, or vector. + typ idxtype.T } var _ cat.Index = &hypotheticalIndex{} @@ -52,14 +53,14 @@ func (hi *hypotheticalIndex) init( name tree.Name, cols []cat.IndexColumn, indexOrd int, - inverted bool, + typ idxtype.T, zone cat.Zone, ) { hi.tab = tab hi.name = name hi.cols = cols hi.indexOrdinal = indexOrd - hi.inverted = inverted + hi.typ = typ hi.zone = zone // Build an index column ordinal set. @@ -77,8 +78,8 @@ func (hi *hypotheticalIndex) init( } } - // Build the stored cols for non-inverted indexes only. - if !inverted { + // Build the stored cols for forward indexes only. + if typ == idxtype.FORWARD { keyColsOrds := colsOrdSet.Union(pkColOrds) hi.storedCols = make([]cat.IndexColumn, 0, tab.ColumnCount()) for i, n := 0, tab.ColumnCount(); i < n; i++ { @@ -106,14 +107,9 @@ func (hi *hypotheticalIndex) IsUnique() bool { return false } -// IsInverted is part of the cat.Index interface. -func (hi *hypotheticalIndex) IsInverted() bool { - return hi.inverted -} - -// IsVector is part of the cat.Index interface. -func (hi *hypotheticalIndex) IsVector() bool { - return false +// Type is part of the cat.Index interface. +func (hi *hypotheticalIndex) Type() idxtype.T { + return hi.typ } // GetInvisibility is part of the cat.Index interface. @@ -150,7 +146,7 @@ func (hi *hypotheticalIndex) LaxKeyColumnCount() int { // PrefixColumnCount is part of the cat.Index interface. func (hi *hypotheticalIndex) PrefixColumnCount() int { - if !hi.IsInverted() && !hi.IsVector() { + if hi.Type() == idxtype.FORWARD { panic(errors.AssertionFailedf("only inverted and vector indexes have prefix columns")) } return len(hi.cols) - 1 @@ -173,7 +169,7 @@ func (hi *hypotheticalIndex) Column(i int) cat.IndexColumn { // InvertedColumn is part of the cat.Index interface. func (hi *hypotheticalIndex) InvertedColumn() cat.IndexColumn { - if !hi.IsInverted() { + if hi.Type() != idxtype.INVERTED { panic(errors.AssertionFailedf("non-inverted indexes do not have inverted columns")) } return hi.cols[len(hi.cols)-1] @@ -221,7 +217,7 @@ func (hi *hypotheticalIndex) ImplicitPartitioningColumnCount() int { // GeoConfig is part of the cat.Index interface. func (hi *hypotheticalIndex) GeoConfig() geopb.Config { - if hi.IsInverted() { + if hi.Type() == idxtype.INVERTED { srcCol := hi.tab.Column(hi.InvertedColumn().InvertedSourceColumnOrdinal()) switch srcCol.DatumType().Family() { case types.GeometryFamily: @@ -253,20 +249,18 @@ func (hi *hypotheticalIndex) Partition(i int) cat.Partition { // have the exact same list, length, and order. If the index is inverted, it // also checks to make sure that the inverted column has the same source column. // If so, it returns true. -func (hi *hypotheticalIndex) hasSameExplicitCols(existingIndex cat.Index, isInverted bool) bool { +func (hi *hypotheticalIndex) hasSameExplicitCols(existingIndex cat.Index) bool { indexCols := hi.cols if existingIndex.ExplicitColumnCount() != len(indexCols) { return false } - return hi.hasPrefixOfExplicitCols(existingIndex, isInverted) + return hi.hasPrefixOfExplicitCols(existingIndex) } // hasPrefixOfExplicitCols returns true if the explicit columns in the // hypothetical index are a prefix of the explicit columns in the given existing // index. -func (hi *hypotheticalIndex) hasPrefixOfExplicitCols( - existingIndex cat.Index, isInverted bool, -) bool { +func (hi *hypotheticalIndex) hasPrefixOfExplicitCols(existingIndex cat.Index) bool { indexCols := hi.cols if existingIndex.ExplicitColumnCount() < len(indexCols) { return false diff --git a/pkg/sql/opt/indexrec/hypothetical_table.go b/pkg/sql/opt/indexrec/hypothetical_table.go index 0ffa07fde1d4..623c9897ca1b 100644 --- a/pkg/sql/opt/indexrec/hypothetical_table.go +++ b/pkg/sql/opt/indexrec/hypothetical_table.go @@ -11,6 +11,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/intsets" @@ -39,9 +40,11 @@ func BuildOptAndHypTableMaps( lastKeyCol := indexCols[len(indexCols)-1] // TODO (Shivam): Index recommendations should not only allow JSON columns // to be part of inverted indexes since they are also forward indexable. - inverted := !colinfo.ColumnTypeIsIndexable(lastKeyCol.DatumType()) || - lastKeyCol.DatumType().Family() == types.JsonFamily - if inverted { + indexType := idxtype.FORWARD + if !colinfo.ColumnTypeIsIndexable(lastKeyCol.DatumType()) || + lastKeyCol.DatumType().Family() == types.JsonFamily { + indexType = idxtype.INVERTED + invertedCol := hypTable.addInvertedCol(lastKeyCol.Column) indexCols[len(indexCols)-1] = cat.IndexColumn{Column: invertedCol} } @@ -51,7 +54,7 @@ func BuildOptAndHypTableMaps( tree.Name(fmt.Sprintf("_hyp_%d", indexOrd)), indexCols, indexOrd, - inverted, + indexType, t.Zone(), ) @@ -59,7 +62,7 @@ func BuildOptAndHypTableMaps( // index with the same key. Inverted indexes do not have stored columns, // so we should not make a recommendation if the same index already // exists. - if !inverted || hypTable.existingRedundantIndex(&hypIndex) == nil { + if indexType != idxtype.INVERTED || hypTable.existingRedundantIndex(&hypIndex) == nil { hypIndexes = append(hypIndexes, hypIndex) } } @@ -159,7 +162,7 @@ func (ht *HypotheticalTable) FullyQualifiedName(ctx context.Context) (cat.DataSo func (ht *HypotheticalTable) existingRedundantIndex(index *hypotheticalIndex) cat.Index { for i, n := 0, ht.Table.IndexCount(); i < n; i++ { existingIndex := ht.Table.Index(i) - indexExists := index.hasSameExplicitCols(existingIndex, index.IsInverted()) + indexExists := index.hasSameExplicitCols(existingIndex) _, isPartialIndex := existingIndex.Predicate() if indexExists && !isPartialIndex && existingIndex.GetInvisibility() == 0.0 { return existingIndex diff --git a/pkg/sql/opt/indexrec/rec.go b/pkg/sql/opt/indexrec/rec.go index 5fa4451f4a15..d488758ce92c 100644 --- a/pkg/sql/opt/indexrec/rec.go +++ b/pkg/sql/opt/indexrec/rec.go @@ -13,6 +13,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/intsets" ) @@ -163,7 +164,7 @@ func findBestExistingIndexToReplace( continue } if existingIndex.GetInvisibility() != 0.0 { - if hypIndex.hasPrefixOfExplicitCols(existingIndex, hypIndex.IsInverted()) { + if hypIndex.hasPrefixOfExplicitCols(existingIndex) { existingIndexAllCols := getAllCols(existingIndex) if newStoredCols.Difference(existingIndexAllCols).Empty() { // There exists an invisible index containing every explicit column in @@ -186,7 +187,7 @@ func findBestExistingIndexToReplace( // hasSameExplicitCols returns true iff the existing index and hypIndex has // the same explicit columns. If hypIndex is inverted, it also makes sure // that their inverted column comes from the same source column. - hasSameExplicitCols := hypIndex.hasSameExplicitCols(existingIndex, hypIndex.IsInverted()) + hasSameExplicitCols := hypIndex.hasSameExplicitCols(existingIndex) if hasSameExplicitCols { // If hasSameExplicitCols, this existing index is a candidate for // potential index replacement. @@ -256,13 +257,6 @@ func (ir *indexRecommendation) constructIndexRec(ctx context.Context) (Rec, erro return Rec{}, err } - indexType := tree.IndexTypeForward - if ir.index.IsInverted() { - indexType = tree.IndexTypeInverted - } else if ir.index.IsVector() { - indexType = tree.IndexTypeVector - } - // Formats index recommendation to its final output struct Rec. switch recType { case TypeCreateIndex: @@ -271,7 +265,7 @@ func (ir *indexRecommendation) constructIndexRec(ctx context.Context) (Rec, erro Columns: indexCols, Storing: storing, Unique: false, - Type: indexType, + Type: ir.index.Type(), } sb.WriteString(createCmd.String()) sb.WriteByte(';') @@ -289,7 +283,7 @@ func (ir *indexRecommendation) constructIndexRec(ctx context.Context) (Rec, erro Storing: storing, // Maintain uniqueness and inverted if the existing index is unique. Unique: existingIndex.IsUnique(), - Type: indexType, + Type: ir.index.Type(), } sb.WriteString(createCmd.String()) sb.WriteByte(';') @@ -452,7 +446,7 @@ func (ir *indexRecommendation) indexCols() []tree.IndexElem { indexCol := ir.index.Column(i) colName := indexCol.Column.ColName() - if ir.index.IsInverted() && i == len(ir.index.cols)-1 { + if ir.index.Type() == idxtype.INVERTED && i == len(ir.index.cols)-1 { colName = ir.index.tab.Column(indexCol.InvertedSourceColumnOrdinal()).ColName() } diff --git a/pkg/sql/opt/invertedidx/BUILD.bazel b/pkg/sql/opt/invertedidx/BUILD.bazel index bbceef4d1f7b..515897d78641 100644 --- a/pkg/sql/opt/invertedidx/BUILD.bazel +++ b/pkg/sql/opt/invertedidx/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//pkg/sql/opt/props", "//pkg/sql/rowenc", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treecmp", "//pkg/sql/types", diff --git a/pkg/sql/opt/invertedidx/inverted_index_expr.go b/pkg/sql/opt/invertedidx/inverted_index_expr.go index ce719c67f99a..4f370a5bee2a 100644 --- a/pkg/sql/opt/invertedidx/inverted_index_expr.go +++ b/pkg/sql/opt/invertedidx/inverted_index_expr.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/norm" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "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/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -449,7 +450,7 @@ func TryJoinInvertedIndex( index cat.Index, inputCols opt.ColSet, ) opt.ScalarExpr { - if !index.IsInverted() { + if index.Type() != idxtype.INVERTED { return nil } diff --git a/pkg/sql/opt/memo/BUILD.bazel b/pkg/sql/opt/memo/BUILD.bazel index df7e0ba9b5d6..9665b51d5d2f 100644 --- a/pkg/sql/opt/memo/BUILD.bazel +++ b/pkg/sql/opt/memo/BUILD.bazel @@ -43,6 +43,7 @@ go_library( "//pkg/sql/sem/cast", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treewindow", "//pkg/sql/sem/volatility", diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index 6c10ed2bae16..3d6a0896470f 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "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/treewindow" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" @@ -862,7 +863,7 @@ func (s *ScanPrivate) IsFullIndexScan() bool { // IsInvertedScan returns true if the index being scanned is an inverted // index. func (s *ScanPrivate) IsInvertedScan(md *opt.Metadata) bool { - return md.Table(s.Table).Index(s.Index).IsInverted() + return md.Table(s.Table).Index(s.Index).Type() == idxtype.INVERTED } // IsVirtualTable returns true if the table being scanned is a virtual table. diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 2029e911a3a0..f4c008263124 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -1473,10 +1474,10 @@ func (f *ExprFmtCtx) formatIndex(tabID opt.TableID, idxOrd cat.IndexOrdinal, rev if reverse { f.Buffer.WriteString(",rev") } - if index.IsInverted() { + switch index.Type() { + case idxtype.INVERTED: f.Buffer.WriteString(",inverted") - } - if index.IsVector() { + case idxtype.VECTOR: f.Buffer.WriteString(",vector") } if _, isPartial := index.Predicate(); isPartial { diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 018b98bb2b06..ad2761860a9a 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "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/treewindow" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -2019,13 +2020,9 @@ func MakeTableFuncDep(md *opt.Metadata, tabID opt.TableID) *props.FuncDepSet { continue } - if index.IsInverted() { - // Skip inverted indexes for now. - continue - } - - if index.IsVector() { - // Skip vector indexes for now. + switch index.Type() { + case idxtype.INVERTED, idxtype.VECTOR: + // Skip inverted and vector indexes for now. continue } diff --git a/pkg/sql/opt/memo/statistics_builder.go b/pkg/sql/opt/memo/statistics_builder.go index 703f1d840ca5..45048e10bf00 100644 --- a/pkg/sql/opt/memo/statistics_builder.go +++ b/pkg/sql/opt/memo/statistics_builder.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "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/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -635,7 +636,7 @@ func (sb *statisticsBuilder) makeTableStatistics(tabID opt.TableID) *props.Stati invertedIndexCols := make(map[int]invertedIndexColInfo) for indexI, indexN := 0, tab.IndexCount(); indexI < indexN; indexI++ { index := tab.Index(indexI) - if !index.IsInverted() { + if index.Type() != idxtype.INVERTED { continue } col := index.InvertedColumn() @@ -1947,8 +1948,8 @@ func (sb *statisticsBuilder) buildZigzagJoin( // join ends up having a higher row count and therefore higher cost than // a competing index join + constrained scan. tab := sb.md.Table(zigzag.LeftTable) - leftIndexInverted := tab.Index(zigzag.LeftIndex).IsInverted() - rightIndexInverted := tab.Index(zigzag.RightIndex).IsInverted() + leftIndexInverted := tab.Index(zigzag.LeftIndex).Type() == idxtype.INVERTED + rightIndexInverted := tab.Index(zigzag.RightIndex).Type() == idxtype.INVERTED if leftIndexInverted { unapplied.unknown += len(zigzag.LeftFixedCols) * 2 } diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index 14eb16e214cf..b6a08b38d163 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -88,6 +88,7 @@ go_library( "//pkg/sql/sem/catconstants", "//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", diff --git a/pkg/sql/opt/optbuilder/alter_table.go b/pkg/sql/opt/optbuilder/alter_table.go index de5a2fdb5aed..960122bff66f 100644 --- a/pkg/sql/opt/optbuilder/alter_table.go +++ b/pkg/sql/opt/optbuilder/alter_table.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -197,7 +198,7 @@ func getIndexColumnNamesAndTypes(index cat.Index) (colNames []string, colTypes [ colNames[i] = string(c.ColName()) colTypes[i] = c.DatumType() } - if index.IsInverted() && !index.GeoConfig().IsEmpty() { + if index.Type() == idxtype.INVERTED && !index.GeoConfig().IsEmpty() { // TODO(sumeer): special case Array too. JSON is harder since the split // needs to be a Datum and the JSON inverted column is not. // diff --git a/pkg/sql/opt/optbuilder/groupby.go b/pkg/sql/opt/optbuilder/groupby.go index 429e96b65d55..9671248d1c28 100644 --- a/pkg/sql/opt/optbuilder/groupby.go +++ b/pkg/sql/opt/optbuilder/groupby.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -989,7 +990,7 @@ func (b *Builder) allowImplicitGroupingColumn(colID opt.ColumnID, g *groupby) bo // Check UNIQUE INDEX constraints. for i := 1; i < tab.IndexCount(); i++ { index := tab.Index(i) - if !index.IsUnique() || index.IsInverted() || index.IsVector() { + if !index.IsUnique() || index.Type() != idxtype.FORWARD { continue } // If any of the key columns is nullable, uniqueCols is suffixed with the diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 9a03dc008606..49b6c24ab459 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "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/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -1117,7 +1118,7 @@ func (mb *mutationBuilder) projectVectorIndexColsImpl(delete bool) { index := mb.tab.Index(i) // Skip non-vector indexes. - if !index.IsVector() { + if index.Type() != idxtype.VECTOR { continue } vectorColOrd := index.VectorColumn().Ordinal() @@ -1652,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).IsVector() { + if tab.Index(i).Type() == tree.IndexTypeVector { count++ } } diff --git a/pkg/sql/opt/ordering/BUILD.bazel b/pkg/sql/opt/ordering/BUILD.bazel index 8222e6d58114..f229cd710ec5 100644 --- a/pkg/sql/opt/ordering/BUILD.bazel +++ b/pkg/sql/opt/ordering/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/sql/opt/memo", "//pkg/sql/opt/props", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/util/buildutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/opt/ordering/interesting_orderings.go b/pkg/sql/opt/ordering/interesting_orderings.go index 640a21d25b20..f2913990e0ef 100644 --- a/pkg/sql/opt/ordering/interesting_orderings.go +++ b/pkg/sql/opt/ordering/interesting_orderings.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" ) // DeriveRestrictedInterestingOrderings calculates and returns the entry of the @@ -105,7 +106,8 @@ func interestingOrderingsForScan(scan *memo.ScanExpr) props.OrderingSet { addIndexOrdering := func(indexOrd cat.IndexOrdinal, fds *props.FuncDepSet, exactPrefix int) { index := tab.Index(indexOrd) - if index.IsInverted() { + if index.Type() != idxtype.FORWARD { + // Do not consider inverted or vector indexes. return } numIndexCols := index.KeyColumnCount() diff --git a/pkg/sql/opt/testutils/testcat/BUILD.bazel b/pkg/sql/opt/testutils/testcat/BUILD.bazel index 35ed5e5cbfcd..06235c614961 100644 --- a/pkg/sql/opt/testutils/testcat/BUILD.bazel +++ b/pkg/sql/opt/testutils/testcat/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treecmp", "//pkg/sql/sem/volatility", diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index cce53a756dc6..f04057a26a6c 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "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/treecmp" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -883,8 +884,7 @@ func (tt *Table) addIndexWithVersion( idx := &Index{ IdxName: tt.makeIndexName(def.Name, def.Columns, typ), Unique: typ != nonUniqueIndex, - Inverted: def.Type == tree.IndexTypeInverted, - Vector: def.Type == tree.IndexTypeVector, + Typ: def.Type, IdxZone: cat.EmptyZone(), table: tt, version: version, @@ -913,9 +913,9 @@ func (tt *Table) addIndexWithVersion( isLastIndexCol := i == len(def.Columns)-1 if isLastIndexCol { switch def.Type { - case tree.IndexTypeInverted: + case idxtype.INVERTED: idx.invertedOrd = i - case tree.IndexTypeVector: + case idxtype.VECTOR: idx.vectorOrd = i } } @@ -925,7 +925,7 @@ func (tt *Table) addIndexWithVersion( notNullIndex = false } - if isLastIndexCol && def.Type == tree.IndexTypeInverted { + if isLastIndexCol && def.Type == idxtype.INVERTED { switch tt.Columns[col.InvertedSourceColumnOrdinal()].DatumType().Family() { case types.GeometryFamily: // Don't use the default config because it creates a huge number of spans. @@ -1071,9 +1071,9 @@ func (tt *Table) addIndexWithVersion( // Add storing columns. for _, name := range def.Storing { switch def.Type { - case tree.IndexTypeInverted: + case idxtype.INVERTED: panic("inverted indexes don't support stored columns") - case tree.IndexTypeVector: + case idxtype.VECTOR: panic("vector indexes don't support stored columns") } // Only add storing columns that weren't added as part of adding implicit @@ -1235,7 +1235,7 @@ func (ti *Index) addColumn( colName = elem.Column } - if ti.Inverted && isLastIndexCol { + if ti.Typ == idxtype.INVERTED && isLastIndexCol { // The last column of an inverted index is special: the index key does // not contain values from the column itself, but contains inverted // index entries derived from that column. Create a virtual column to be @@ -1319,7 +1319,7 @@ func (ti *Index) addColumnByOrdinal( )) } } else if typ.Family() == types.PGVectorFamily { - if !ti.Vector { + if ti.Typ != idxtype.VECTOR { panic(fmt.Errorf( "column %s of type %s is not allowed in a non-vector index", col.ColName(), typ, )) diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 977a9e2c67f8..828b4ed163e6 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "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/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/stats" @@ -1094,11 +1095,8 @@ type Index struct { // Unique is true if this index is declared as UNIQUE in the schema. Unique bool - // Inverted is true when this index is an inverted index. - Inverted bool - - // Vector is true when this index is a vector index. - Vector bool + // Typ is the type of the index: forward, inverted, vector, etc. + Typ idxtype.T // Invisibility specifies the invisibility of an index and can be any float64 // between [0.0, 1.0]. An index with invisibility 0.0 means that the index is @@ -1170,14 +1168,9 @@ func (ti *Index) IsUnique() bool { return ti.Unique } -// IsInverted is part of the cat.Index interface. -func (ti *Index) IsInverted() bool { - return ti.Inverted -} - -// IsVector is part of the cat.Index interface. -func (ti *Index) IsVector() bool { - return ti.Vector +// Type is part of the cat.Index interface. +func (ti *Index) Type() idxtype.T { + return ti.Typ } // GetInvisibility is part of the cat.Index interface. @@ -1207,13 +1200,14 @@ func (ti *Index) LaxKeyColumnCount() int { // PrefixColumnCount is part of the cat.Index interface. func (ti *Index) PrefixColumnCount() int { - if ti.IsInverted() { + switch ti.Type() { + case idxtype.INVERTED: return ti.invertedOrd - } - if ti.IsVector() { + case idxtype.VECTOR: return ti.vectorOrd + default: + panic("only supported for inverted and vector indexes") } - panic("only supported for inverted and vector indexes") } // Column is part of the cat.Index interface. @@ -1223,7 +1217,7 @@ func (ti *Index) Column(i int) cat.IndexColumn { // InvertedColumn is part of the cat.Index interface. func (ti *Index) InvertedColumn() cat.IndexColumn { - if !ti.IsInverted() { + if ti.Type() != idxtype.INVERTED { panic("non-inverted indexes do not have inverted columns") } return ti.Column(ti.invertedOrd) @@ -1231,7 +1225,7 @@ func (ti *Index) InvertedColumn() cat.IndexColumn { // VectorColumn is part of the cat.Index interface. func (ti *Index) VectorColumn() cat.IndexColumn { - if !ti.IsVector() { + if ti.Type() != idxtype.VECTOR { panic("non-vector indexes do not have indexed vector columns") } return ti.Column(ti.vectorOrd) diff --git a/pkg/sql/opt/workloadindexrec/BUILD.bazel b/pkg/sql/opt/workloadindexrec/BUILD.bazel index f42d06a1a7a4..ec2afaff8f27 100644 --- a/pkg/sql/opt/workloadindexrec/BUILD.bazel +++ b/pkg/sql/opt/workloadindexrec/BUILD.bazel @@ -11,6 +11,7 @@ go_library( deps = [ "//pkg/sql/parser", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/opt/workloadindexrec/workload_indexrecs.go b/pkg/sql/opt/workloadindexrec/workload_indexrecs.go index 69fc583b6819..b535670a46f3 100644 --- a/pkg/sql/opt/workloadindexrec/workload_indexrecs.go +++ b/pkg/sql/opt/workloadindexrec/workload_indexrecs.go @@ -11,6 +11,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "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/sessiondata" "github.com/cockroachdb/errors" @@ -125,7 +126,7 @@ func collectIndexRecs( switch stmt := stmt.AST.(type) { case *tree.CreateIndex: // Ignore all the inverted, vector, partial, sharded, etc. indexes right now. - if stmt.Type == tree.IndexTypeForward && stmt.Predicate == nil && stmt.Sharded == nil { + if stmt.Type == idxtype.FORWARD && stmt.Predicate == nil && stmt.Sharded == nil { cis = append(cis, *stmt) } case *tree.DropIndex: diff --git a/pkg/sql/opt/xform/BUILD.bazel b/pkg/sql/opt/xform/BUILD.bazel index 987994e8ef6a..c9076fc1b99f 100644 --- a/pkg/sql/opt/xform/BUILD.bazel +++ b/pkg/sql/opt/xform/BUILD.bazel @@ -48,6 +48,7 @@ go_library( "//pkg/sql/opt/props/physical", "//pkg/sql/rowinfra", "//pkg/sql/sem/eval", + "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", "//pkg/sql/sem/volatility", "//pkg/sql/sessiondatapb", diff --git a/pkg/sql/opt/xform/general_funcs.go b/pkg/sql/opt/xform/general_funcs.go index 9eafd7fceb8f..063aa0ba227e 100644 --- a/pkg/sql/opt/xform/general_funcs.go +++ b/pkg/sql/opt/xform/general_funcs.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/ordering" "github.com/cockroachdb/cockroach/pkg/sql/opt/partialidx" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -59,7 +60,7 @@ func (c *CustomFuncs) HasInvertedIndexes(scanPrivate *memo.ScanPrivate) bool { // Skip the primary index because it cannot be inverted. for i := 1; i < tab.IndexCount(); i++ { - if tab.Index(i).IsInverted() { + if tab.Index(i).Type() == idxtype.INVERTED { return true } } diff --git a/pkg/sql/opt/xform/limit_funcs.go b/pkg/sql/opt/xform/limit_funcs.go index b38859b6da60..248d82ac013c 100644 --- a/pkg/sql/opt/xform/limit_funcs.go +++ b/pkg/sql/opt/xform/limit_funcs.go @@ -12,6 +12,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/ordering" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -190,7 +191,7 @@ func (c *CustomFuncs) ScanIsLimited(sp *memo.ScanPrivate) bool { func (c *CustomFuncs) ScanIsInverted(sp *memo.ScanPrivate) bool { md := c.e.mem.Metadata() idx := md.Table(sp.Table).Index(sp.Index) - return idx.IsInverted() + return idx.Type() == idxtype.INVERTED } // SplitLimitedScanIntoUnionScans returns a UnionAll tree of Scan operators with diff --git a/pkg/sql/opt/xform/placeholder_fast_path.go b/pkg/sql/opt/xform/placeholder_fast_path.go index 6663938a84fb..d48dafab9686 100644 --- a/pkg/sql/opt/xform/placeholder_fast_path.go +++ b/pkg/sql/opt/xform/placeholder_fast_path.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil" @@ -121,7 +122,8 @@ func (o *Optimizer) TryPlaceholderFastPath() (_ opt.Expr, ok bool, err error) { var foundIndex cat.Index for ord, n := 0, tabMeta.Table.IndexCount(); ord < n; ord++ { index := tabMeta.Table.Index(ord) - if index.IsInverted() || index.IsVector() { + if index.Type() != idxtype.FORWARD { + // Skip inverted and vector indexes. continue } diff --git a/pkg/sql/opt/xform/scan_index_iter.go b/pkg/sql/opt/xform/scan_index_iter.go index b0a6a6ee1d8f..4f28052b7cdf 100644 --- a/pkg/sql/opt/xform/scan_index_iter.go +++ b/pkg/sql/opt/xform/scan_index_iter.go @@ -12,6 +12,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/partialidx" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/errors" ) @@ -234,22 +235,22 @@ func (it *scanIndexIter) ForEachStartingAfter(ord int, f enumerateIndexFunc) { } // Skip over inverted indexes if rejectInvertedIndexes is set. - if it.hasRejectFlags(rejectInvertedIndexes) && index.IsInverted() { + if it.hasRejectFlags(rejectInvertedIndexes) && index.Type() == idxtype.INVERTED { continue } // Skip over non-inverted indexes if rejectNonInvertedIndexes is set. - if it.hasRejectFlags(rejectNonInvertedIndexes) && !index.IsInverted() { + if it.hasRejectFlags(rejectNonInvertedIndexes) && index.Type() != idxtype.INVERTED { continue } // Skip over vector indexes if rejectVectorIndexes is set. - if it.hasRejectFlags(rejectVectorIndexes) && index.IsVector() { + if it.hasRejectFlags(rejectVectorIndexes) && index.Type() == idxtype.VECTOR { continue } // Skip over non-vector indexes if rejectNonVectorIndexes is set. - if it.hasRejectFlags(rejectNonVectorIndexes) && !index.IsVector() { + if it.hasRejectFlags(rejectNonVectorIndexes) && index.Type() != idxtype.VECTOR { continue } diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index e98707ac678b..e72428743e97 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -1638,7 +1638,7 @@ func (oi *optIndex) init( } // Populate columnOrds. - inverted := oi.IsInverted() + inverted := oi.Type() == idxtype.INVERTED numKeyCols := idx.NumKeyColumns() numKeySuffixCols := idx.NumKeySuffixColumns() oi.columnOrds = make([]int, oi.numCols) @@ -1668,22 +1668,16 @@ func (oi *optIndex) Name() tree.Name { return tree.Name(oi.idx.GetName()) } +// Type is part of the cat.Index interface. +func (oi *optIndex) Type() idxtype.T { + return oi.idx.GetType() +} + // IsUnique is part of the cat.Index interface. func (oi *optIndex) IsUnique() bool { return oi.idx.IsUnique() } -// IsInverted is part of the cat.Index interface. -func (oi *optIndex) IsInverted() bool { - return oi.idx.GetType() == idxtype.INVERTED -} - -// IsVector is part of the cat.Index interface. -func (oi *optIndex) IsVector() bool { - // TODO(#137370): check the index type. - return false -} - // GetInvisibility is part of the cat.Index interface. func (oi *optIndex) GetInvisibility() float64 { return oi.idx.GetInvisibility() @@ -1711,7 +1705,7 @@ func (oi *optIndex) LaxKeyColumnCount() int { // PrefixColumnCount is part of the cat.Index interface. func (oi *optIndex) PrefixColumnCount() int { - if !oi.IsInverted() && !oi.IsVector() { + if oi.Type() == idxtype.FORWARD { panic(errors.AssertionFailedf("only inverted and vector indexes have prefix columns")) } return oi.idx.NumKeyColumns() - 1 @@ -1730,7 +1724,7 @@ func (oi *optIndex) Column(i int) cat.IndexColumn { // InvertedColumn is part of the cat.Index interface. func (oi *optIndex) InvertedColumn() cat.IndexColumn { - if !oi.IsInverted() { + if oi.Type() != idxtype.INVERTED { panic(errors.AssertionFailedf("non-inverted indexes do not have inverted columns")) } ord := oi.idx.NumKeyColumns() - 1 @@ -1739,7 +1733,7 @@ func (oi *optIndex) InvertedColumn() cat.IndexColumn { // VectorColumn is part of the cat.Index interface. func (oi *optIndex) VectorColumn() cat.IndexColumn { - if !oi.IsVector() { + if oi.Type() != idxtype.VECTOR { panic(errors.AssertionFailedf("non-vector indexes do not have inverted columns")) } ord := oi.idx.NumKeyColumns() - 1 @@ -2607,6 +2601,11 @@ func (oi *optVirtualIndex) Name() tree.Name { return tree.Name(oi.idx.GetName()) } +// Type is part of the cat.Index interface. +func (oi *optVirtualIndex) Type() idxtype.T { + return idxtype.FORWARD +} + // IsUnique is part of the cat.Index interface. func (oi *optVirtualIndex) IsUnique() bool { if oi.idx == nil { @@ -2616,16 +2615,6 @@ func (oi *optVirtualIndex) IsUnique() bool { return oi.idx.IsUnique() } -// IsInverted is part of the cat.Index interface. -func (oi *optVirtualIndex) IsInverted() bool { - return false -} - -// IsVector is part of the cat.Index interface. -func (oi *optVirtualIndex) IsVector() bool { - return false -} - // GetInvisibility is part of the cat.Index interface. func (oi *optVirtualIndex) GetInvisibility() float64 { return 0.0