Skip to content

Commit

Permalink
sql: permit opclasses for inverted indexes
Browse files Browse the repository at this point in the history
Previously, it was not permitted to use opclass syntax for index
creation. For example, the `blah_ops` in `CREATE INDEX ON t USING GIN (col blah_ops))`.

Now, this syntax is legal when the opclass is supported for a given
type.

Release note (sql change): permit usage of jsonb_ops, array_ops, and
gin_trgm_ops as an opclass in inverted index creation.
  • Loading branch information
jordanlewis committed May 20, 2022
1 parent c4e48d1 commit e395dcb
Show file tree
Hide file tree
Showing 17 changed files with 192 additions and 62 deletions.
27 changes: 26 additions & 1 deletion pkg/sql/catalog/descpb/structured.proto
Original file line number Diff line number Diff line change
Expand Up @@ -336,6 +336,26 @@ message IndexDescriptor {
INVERTED = 1;
}

// InvertedIndexColumnKind is the kind of the inverted index that we're
// building on a particular column. The reason this needs to be stored is that
// we need to be able to check that the "opclass" passed into an inverted
// index declaration (for example, gin_trgm_ops) is compatible with the
// datatype of a particular column (gin_tgrm_ops is only valid on text). A
// future reason is that it's possible to desire having more than one type of
// inverted index on a particular datatype - for example, you might want to
// create a "stemming" inverted index on text. And without this extra kind, it
// wouldn't be possible to distinguish a text inverted index that uses
// trigrams, vs a text inverted index that uses stemming.
enum InvertedIndexColumnKind {
// DEFAULT is the default kind of inverted index column. JSON, Array, and
// geo inverted indexes all are DEFAULT, though prior to 22.2 they had no
// kind at all.
DEFAULT = 0;
// TRIGRAM is the trigram kind of inverted index column. It's only valid on
// text columns.
TRIGRAM = 1;
}

optional string name = 1 [(gogoproto.nullable) = false];
optional uint32 id = 2 [(gogoproto.nullable) = false,
(gogoproto.customname) = "ID", (gogoproto.casttype) = "IndexID"];
Expand All @@ -362,6 +382,11 @@ message IndexDescriptor {
// columns which are explicitly part of the index (STORING clause).
repeated string store_column_names = 5;

// An ordered list of opclasses that parallels each of the inverted columns
// in the index. n.b.: currently, there can only be a single inverted column
// in an index, so this list will always be of size 0 or 1.
repeated InvertedIndexColumnKind inverted_column_kinds = 27;

// An ordered list of column IDs of which the index key is comprised. This
// list parallels the key_column_names list and does not include any
// additional stored columns. If the index is an inverted index, the last
Expand Down Expand Up @@ -490,7 +515,7 @@ message IndexDescriptor {
optional uint32 constraint_id = 26 [(gogoproto.customname) = "ConstraintID",
(gogoproto.casttype) = "ConstraintID", (gogoproto.nullable) = false];

// Next ID: 27
// Next ID: 28
}

// ConstraintToUpdate represents a constraint to be added to the table and
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,10 @@ type Index interface {
// Panics if the index is not inverted.
InvertedColumnKeyType() *types.T

// InvertedColumnKind returns the kind of the inverted column of the inverted
// index.
InvertedColumnKind() descpb.IndexDescriptor_InvertedIndexColumnKind

NumPrimaryStoredColumns() int
NumSecondaryStoredColumns() int
GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/catalog/tabledesc/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

var _ catalog.Index = (*index)(nil)
Expand Down Expand Up @@ -176,6 +177,22 @@ func (w index) InvertedColumnKeyType() *types.T {
return w.desc.InvertedColumnKeyType()
}

// InvertedColumnKind returns the kind of the inverted column of the inverted
// index.
//
// Panics if the index is not inverted.
func (w index) InvertedColumnKind() descpb.IndexDescriptor_InvertedIndexColumnKind {
if w.desc.Type != descpb.IndexDescriptor_INVERTED {
panic(errors.AssertionFailedf("index is not inverted"))
}
if len(w.desc.InvertedColumnKinds) == 0 {
// Not every inverted index has kinds inside, since no kinds were set prior
// to version 22.2.
return descpb.IndexDescriptor_DEFAULT
}
return w.desc.InvertedColumnKinds[0]
}

// CollectKeyColumnIDs creates a new set containing the column IDs in the key
// of this index.
func (w index) CollectKeyColumnIDs() catalog.TableColSet {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/catalog/tabledesc/validate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,8 @@ var validationMap = []struct {
"StoreColumnNames": {
status: todoIAmKnowinglyAddingTechDebt,
reason: "initial import: TODO(features): add validation"},
"KeyColumnIDs": {status: iSolemnlySwearThisFieldIsValidated},
"InvertedColumnKinds": {status: thisFieldReferencesNoObjects},
"KeyColumnIDs": {status: iSolemnlySwearThisFieldIsValidated},
"KeySuffixColumnIDs": {
status: todoIAmKnowinglyAddingTechDebt,
reason: "initial import: TODO(features): add validation"},
Expand Down
69 changes: 59 additions & 10 deletions pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -211,19 +212,13 @@ func makeIndexDescriptor(
}

indexDesc.Type = descpb.IndexDescriptor_INVERTED
column, err := tableDesc.FindColumnWithName(columns[len(columns)-1].Column)
invCol := columns[len(columns)-1]
column, err := tableDesc.FindColumnWithName(invCol.Column)
if err != nil {
return nil, err
}
switch column.GetType().Family() {
case types.GeometryFamily:
config, err := geoindex.GeometryIndexConfigForSRID(column.GetType().GeoSRIDOrZero())
if err != nil {
return nil, err
}
indexDesc.GeoConfig = *config
case types.GeographyFamily:
indexDesc.GeoConfig = *geoindex.DefaultGeographyIndexConfig()
if err := populateInvertedIndexDescriptor(column, &indexDesc, invCol); err != nil {
return nil, err
}
}

Expand Down Expand Up @@ -303,6 +298,60 @@ func makeIndexDescriptor(
return &indexDesc, nil
}

// populateInvertedIndexDescriptor adds information to the input index descriptor
// for the inverted index given by the input column and invCol, which should
// match (column is the catalog column, and invCol is the grammar node of
// the column in the index creation statement).
func populateInvertedIndexDescriptor(
column catalog.Column, indexDesc *descpb.IndexDescriptor, invCol tree.IndexElem,
) error {
indexDesc.InvertedColumnKinds = []descpb.IndexDescriptor_InvertedIndexColumnKind{descpb.IndexDescriptor_DEFAULT}
switch column.GetType().Family() {
case types.ArrayFamily:
switch invCol.OpClass {
case "array_ops", "":
default:
return pgerror.Newf(pgcode.UndefinedObject, "operator class %q does not exist", invCol.OpClass)
}
case types.JsonFamily:
switch invCol.OpClass {
case "jsonb_ops", "":
case "jsonb_path_ops":
return unimplemented.NewWithIssue(81115, "operator class \"jsonb_path_ops\" is not supported")
default:
return pgerror.Newf(pgcode.UndefinedObject, "operator class %q does not exist", invCol.OpClass)
}
case types.GeometryFamily:
if invCol.OpClass != "" {
return pgerror.Newf(pgcode.UndefinedObject, "operator class %q does not exist", invCol.OpClass)
}
config, err := geoindex.GeometryIndexConfigForSRID(column.GetType().GeoSRIDOrZero())
if err != nil {
return err
}
indexDesc.GeoConfig = *config
case types.GeographyFamily:
if invCol.OpClass != "" {
return pgerror.Newf(pgcode.UndefinedObject, "operator class %q does not exist", invCol.OpClass)
}
indexDesc.GeoConfig = *geoindex.DefaultGeographyIndexConfig()
case types.StringFamily:
// Check the opclass of the last column in the list, which is the column
// we're going to inverted index.
switch invCol.OpClass {
case "gin_trgm_ops":
case "":
return errors.WithHint(
pgerror.New(pgcode.UndefinedObject, "data type text has no default operator class for access method \"gin\""),
"You must specify an operator class for the index (did you mean gin_trgm_ops?)")
default:
return pgerror.Newf(pgcode.UndefinedObject, "operator class %q does not exist", invCol.OpClass)
}
indexDesc.InvertedColumnKinds[0] = descpb.IndexDescriptor_TRIGRAM
}
return nil
}

// validateColumnsAreAccessible validates that the columns for an index are
// accessible. This check must be performed before creating inaccessible columns
// for expression indexes with replaceExpressionElemsWithVirtualCols.
Expand Down
12 changes: 2 additions & 10 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
Expand Down Expand Up @@ -1803,15 +1802,8 @@ func NewTableDesc(
if err != nil {
return nil, err
}
switch column.GetType().Family() {
case types.GeometryFamily:
config, err := geoindex.GeometryIndexConfigForSRID(column.GetType().GeoSRIDOrZero())
if err != nil {
return nil, err
}
idx.GeoConfig = *config
case types.GeographyFamily:
idx.GeoConfig = *geoindex.DefaultGeographyIndexConfig()
if err := populateInvertedIndexDescriptor(column, &idx, columns[len(columns)-1]); err != nil {
return nil, err
}
}

Expand Down
26 changes: 26 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/inverted_index
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,22 @@ c CREATE TABLE public.c (
INVERTED INDEX "c_bAr_idx" ("bAr" ASC)
)

# Test that only the permitted opclasses are usable to make an inverted index.
statement error operator class \"blah_ops\" does not exist
CREATE INVERTED INDEX ON c(foo blah_ops)

statement error operator class \"blah_ops\" does not exist
CREATE INDEX ON c USING GIN(foo blah_ops)

statement error unimplemented: operator class "jsonb_path_ops" is not supported
CREATE INDEX ON c USING GIN(foo jsonb_path_ops)

statement ok
CREATE INVERTED INDEX ON c(foo jsonb_ops)

statement ok
CREATE INDEX ON c USING GIN(foo jsonb_ops)

# Regression test for #42944: make sure that mixed-case columns can be
# inverted indexed.
statement ok
Expand Down Expand Up @@ -1160,6 +1176,16 @@ CREATE TABLE c (
FAMILY "primary" (id, foo, bar)
)

# Test that only the permitted opclasses are usable to make an inverted index.
statement error operator class \"blah_ops\" does not exist
CREATE INVERTED INDEX ON c(foo blah_ops)

statement ok
CREATE INVERTED INDEX blorp ON c(foo array_ops)

statement ok
DROP INDEX blorp

statement ok
INSERT INTO c VALUES(0, NULL, NULL)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,14 @@ CREATE TABLE geo_table(
INVERTED INDEX geom_index(geom)
)

# Test that only the permitted opclasses are usable to make an inverted index.
statement error operator class \"blah_ops\" does not exist
CREATE INVERTED INDEX ON geo_table(geom blah_ops)

# Test that only the permitted opclasses are usable to make an inverted index.
statement error operator class \"blah_ops\" does not exist
CREATE INDEX ON geo_table USING GIST(geom blah_ops)

# Shapes with SRID 26918. We've taken small X, Y values and added 400,000 to the X coordinate
# and 4,000,000 to the Y coordinate to place them inside the bounds of SRID 26918.
statement ok
Expand Down
14 changes: 13 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/trigram_indexes
Original file line number Diff line number Diff line change
@@ -1,9 +1,21 @@
statement ok
CREATE TABLE a (a INT PRIMARY KEY, t TEXT)

statement ok
statement error data type text has no default operator class for access method \"gin\"
CREATE INVERTED INDEX ON a(t)

statement error data type text has no default operator class for access method \"gin\"
CREATE INDEX ON a USING GIN(t)

statement error operator class \"blah_ops\" does not exist
CREATE INVERTED INDEX ON a(t blah_ops)

statement ok
CREATE INVERTED INDEX ON a(t gin_trgm_ops)

statement ok
CREATE INDEX ON a USING GIN(t gin_trgm_ops)

statement ok
INSERT INTO a VALUES (1, 'foozoopa'),
(2, 'Foo'),
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/testdata/trigram_index
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ CREATE TABLE a (
a INT PRIMARY KEY,
b TEXT,
FAMILY (a,b),
INVERTED INDEX(b)
INVERTED INDEX(b gin_trgm_ops)
)

query T
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/opt/memo/statistics_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4583,7 +4583,7 @@ func (sb *statisticsBuilder) buildStatsFromCheckConstraints(
var hasNullValue, ok bool
var values tree.Datums
var distinctVals uint64
invertedIndexableColumnType := colinfo.ColumnTypeIsOnlyInvertedIndexable(colType)
onlyInvertedIndexableColumnType := colinfo.ColumnTypeIsOnlyInvertedIndexable(colType)
if distinctVals, ok = filterConstraint.CalculateMaxResults(sb.evalCtx, cols, cols); ok {
// If the number of values is excessive, don't spend too much time building the histogram,
// as it may slow down the query.
Expand All @@ -4606,7 +4606,7 @@ func (sb *statisticsBuilder) buildStatsFromCheckConstraints(
// types of such columns (JSON, ARRAY, and spatial types) aren't likely to
// occur in CHECK constraints. So, let's play it safe and don't create a
// histogram directly on columns that have a data type which is
// InvertedIndexable since the possible user benefit of adding this
// OnlyInvertedIndexable since the possible user benefit of adding this
// support seems low.
//
// Also, histogram building errors out when the number of samples is
Expand All @@ -4615,7 +4615,7 @@ func (sb *statisticsBuilder) buildStatsFromCheckConstraints(
// not expect to see null values. If we do see a null, something may have
// gone wrong, so do not build a histogram in this case either.
useHistogram := sb.evalCtx.SessionData().OptimizerUseHistograms && numValues > 0 &&
!hasNullValue && !invertedIndexableColumnType && int64(numValues) <= numRows
!hasNullValue && !onlyInvertedIndexableColumnType && int64(numValues) <= numRows
if !useHistogram {
if distinctVals > math.MaxInt32 {
continue
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -525,9 +525,6 @@ func TestUnimplementedSyntax(t *testing.T) {
{`CREATE INDEX a ON b USING SPGIST (c)`, 0, `index using spgist`, ``},
{`CREATE INDEX a ON b USING BRIN (c)`, 0, `index using brin`, ``},

{`CREATE INDEX a ON b(c gin_trgm_ops)`, 41285, `index using gin_trgm_ops`, ``},
{`CREATE INDEX a ON b(c gist_trgm_ops)`, 41285, `index using gist_trgm_ops`, ``},
{`CREATE INDEX a ON b(c bobby)`, 47420, ``, ``},
{`CREATE INDEX a ON b(a NULLS LAST)`, 6224, ``, ``},
{`CREATE INDEX a ON b(a ASC NULLS LAST)`, 6224, ``, ``},
{`CREATE INDEX a ON b(a DESC NULLS FIRST)`, 6224, ``, ``},
Expand Down
8 changes: 1 addition & 7 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -8706,12 +8706,6 @@ index_elem_options:
opClass := $1
dir := $2.dir()
nullsOrder := $3.nullsOrder()
if opClass != "" {
if opClass == "gin_trgm_ops" || opClass == "gist_trgm_ops" {
return unimplementedWithIssueDetail(sqllex, 41285, "index using " + opClass)
}
return unimplementedWithIssue(sqllex, 47420)
}
// We currently only support the opposite of Postgres defaults.
if nullsOrder != tree.DefaultNullsOrder {
if dir == tree.Descending && nullsOrder == tree.NullsFirst {
Expand All @@ -8721,7 +8715,7 @@ index_elem_options:
return unimplementedWithIssue(sqllex, 6224)
}
}
$$.val = tree.IndexElem{Direction: dir, NullsOrder: nullsOrder}
$$.val = tree.IndexElem{Direction: dir, NullsOrder: nullsOrder, OpClass: tree.Name(opClass)}
}

opt_class:
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/rowenc/index_encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -581,9 +581,9 @@ func EncodeInvertedIndexTableKeys(
case types.ArrayFamily:
return encodeArrayInvertedIndexTableKeys(val.(*tree.DArray), inKey, version, false /* excludeNulls */)
case types.StringFamily:
// TODO(jordan): Right now, this is just trigram inverted indexes. What if
// we want to support different types of inverted indexes on strings? We'll
// need to pass in the index type to this function.
// TODO(jordan): Right now, this is just trigram inverted indexes. If we
// want to support different types of inverted indexes on strings, we'll
// need to pass in the inverted index column kind to this function.
// We pad the keys when writing them to the index.
// TODO(jordan): why are we doing this padding at all? Postgres does it.
return encodeTrigramInvertedIndexTableKeys(string(*val.(*tree.DString)), inKey, version, true /* pad */)
Expand Down
Loading

0 comments on commit e395dcb

Please sign in to comment.