From fcf2fbf80da9af6195a0bc3157ee7b1c2440d0c7 Mon Sep 17 00:00:00 2001 From: Andrew Kimball Date: Wed, 29 Jan 2025 16:15:40 -0800 Subject: [PATCH] sql: add schema support for vector indexing in CREATE TABLE Support usage of VECTOR INDEX in a CREATE TABLE statement, e.g.: CREATE TABLE simple ( a INT PRIMARY KEY, vec1 VECTOR(3), VECTOR INDEX (vec1) ) Create the corresponding table and index schema objects for the vector index. Check various error conditions, e.g. that only a column of type VECTOR can be the last column in the index. Add unit and logic tests. CREATE VECTOR INDEX support will come in a future PR. Epic: CRDB-42943 Release note: None Co-authored-by: Drew Kimball --- .../logictestccl/testdata/logic_test/vector | 5 +- .../tests/3node-tenant/generated_test.go | 7 + .../local-read-committed/generated_test.go | 7 + .../local-repeatable-read/generated_test.go | 7 + pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/catformat/index.go | 27 ++- pkg/sql/catalog/catformat/index_test.go | 22 +++ pkg/sql/catalog/colinfo/col_type_info.go | 6 + pkg/sql/catalog/descpb/index.go | 22 ++- pkg/sql/catalog/table_elements.go | 14 ++ pkg/sql/catalog/tabledesc/BUILD.bazel | 3 +- pkg/sql/catalog/tabledesc/index.go | 20 ++ pkg/sql/catalog/tabledesc/index_test.go | 16 +- pkg/sql/catalog/tabledesc/structured.go | 172 ++++++++---------- pkg/sql/catalog/tabledesc/structured_test.go | 1 + .../catalog/tabledesc/table_desc_builder.go | 6 +- pkg/sql/create_index.go | 3 +- pkg/sql/create_stats.go | 4 + pkg/sql/create_table.go | 21 ++- .../testdata/logic_test/create_table | 33 +++- .../testdata/logic_test/vector_index | 140 ++++++++++++++ .../tests/fakedist-disk/generated_test.go | 7 + .../tests/fakedist-vec-off/generated_test.go | 7 + .../tests/fakedist/generated_test.go | 7 + .../generated_test.go | 7 + .../tests/local-mixed-24.3/generated_test.go | 7 + .../tests/local-vec-off/generated_test.go | 7 + .../logictest/tests/local/generated_test.go | 7 + pkg/sql/parser/lexer.go | 5 +- pkg/sql/parser/testdata/create_table | 17 ++ .../scbuildstmt/alter_table_add_column.go | 6 +- .../alter_table_alter_primary_key.go | 9 +- .../internal/scbuildstmt/create_index.go | 15 +- pkg/sql/sem/idxtype/BUILD.bazel | 2 + pkg/sql/sem/idxtype/idxtype.go | 46 +++-- pkg/sql/sqlerrors/BUILD.bazel | 1 + pkg/sql/sqlerrors/errors.go | 68 +++++++ pkg/sql/sqltelemetry/schema.go | 9 + pkg/sql/vecindex/vector_index.go | 2 +- 39 files changed, 604 insertions(+), 162 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/vector_index diff --git a/pkg/ccl/logictestccl/testdata/logic_test/vector b/pkg/ccl/logictestccl/testdata/logic_test/vector index 96c50cf6e829..81593589af9c 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/vector +++ b/pkg/ccl/logictestccl/testdata/logic_test/vector @@ -12,7 +12,7 @@ CREATE TABLE v (v vector(0)) statement error pgcode 42601 dimensions for type vector cannot exceed 16000 CREATE TABLE v (v vector(16001)) -statement error column v is of type vector and thus is not indexable +statement error column v of type vector is only allowed as the last column in a vector index CREATE TABLE v (v vector(2) PRIMARY KEY) statement ok @@ -189,7 +189,4 @@ CREATE INDEX ON t_vec USING CSPANN (v); statement ok DROP TABLE t_vec; -statement error pgcode 0A000 pq: unimplemented: VECTOR indexes are not yet supported -CREATE TABLE t_vec (k INT PRIMARY KEY, v VECTOR(128), VECTOR INDEX (v)); - subtest end diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index eb927c8fd045..aeced7b8e280 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -2528,6 +2528,13 @@ func TestTenantLogic_values( runLogicTest(t, "values") } +func TestTenantLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestTenantLogic_vectorize( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go b/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go index a593f3c23355..462d41f51f25 100644 --- a/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-read-committed/generated_test.go @@ -2505,6 +2505,13 @@ func TestReadCommittedLogic_values( runLogicTest(t, "values") } +func TestReadCommittedLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestReadCommittedLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go b/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go index 489b45c7be34..a49f341381bd 100644 --- a/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local-repeatable-read/generated_test.go @@ -2498,6 +2498,13 @@ func TestRepeatableReadLogic_values( runLogicTest(t, "values") } +func TestRepeatableReadLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestRepeatableReadLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 023e720a54c5..7579bfcb0633 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/sql/sessiondatapb", "//pkg/sql/sqlclustersettings", "//pkg/sql/types", + "//pkg/sql/vecindex/vecpb", "//pkg/util", "//pkg/util/hlc", "//pkg/util/intsets", diff --git a/pkg/sql/catalog/catformat/index.go b/pkg/sql/catalog/catformat/index.go index b47c2d9c5d61..abe43b527f9d 100644 --- a/pkg/sql/catalog/catformat/index.go +++ b/pkg/sql/catalog/catformat/index.go @@ -102,8 +102,13 @@ func indexForDisplay( if index.Unique { f.WriteString("UNIQUE ") } - if !f.HasFlags(tree.FmtPGCatalog) && index.Type == idxtype.INVERTED { - f.WriteString("INVERTED ") + if !f.HasFlags(tree.FmtPGCatalog) { + switch index.Type { + case idxtype.INVERTED: + f.WriteString("INVERTED ") + case idxtype.VECTOR: + f.WriteString("VECTOR ") + } } f.WriteString("INDEX ") f.FormatNameP(&index.Name) @@ -114,9 +119,12 @@ func indexForDisplay( if f.HasFlags(tree.FmtPGCatalog) { f.WriteString(" USING") - if index.Type == idxtype.INVERTED { + switch index.Type { + case idxtype.INVERTED: f.WriteString(" gin") - } else { + case idxtype.VECTOR: + f.WriteString(" cspann") + default: f.WriteString(" btree") } } @@ -240,6 +248,8 @@ func FormatIndexElements( } else { f.FormatNameP(&index.KeyColumnNames[i]) } + // TODO(drewk): we might need to print something like "vector_l2_ops" for + // vector indexes. if index.Type == idxtype.INVERTED && col.GetID() == index.InvertedColumnID() && len(index.InvertedColumnKinds) > 0 { switch index.InvertedColumnKinds[0] { @@ -247,10 +257,11 @@ func FormatIndexElements( f.WriteString(" gin_trgm_ops") } } - // The last column of an inverted index cannot have a DESC direction. - // Since the default direction is ASC, we omit the direction entirely - // for inverted index columns. - if i < n-1 || index.Type != idxtype.INVERTED { + // The last column of an inverted or vector index cannot have a DESC + // direction because it does not have a linear ordering. Since the default + // direction is ASC, we omit the direction entirely for inverted/vector + // index columns. + if i < n-1 || index.Type.HasLinearOrdering() { f.WriteByte(' ') f.WriteString(index.KeyColumnDirections[i].String()) } diff --git a/pkg/sql/catalog/catformat/index_test.go b/pkg/sql/catalog/catformat/index_test.go index 22b67c65b5b4..2e0d02c426d0 100644 --- a/pkg/sql/catalog/catformat/index_test.go +++ b/pkg/sql/catalog/catformat/index_test.go @@ -112,6 +112,12 @@ func TestIndexForDisplay(t *testing.T) { ColumnNames: []string{"a"}, } + // VECTOR INDEX baz (a) + vectorIndex := baseIndex + vectorIndex.Type = idxtype.VECTOR + vectorIndex.KeyColumnNames = []string{"a"} + vectorIndex.KeyColumnIDs = descpb.ColumnIDs{1} + testData := []struct { index descpb.IndexDescriptor tableName tree.TableName @@ -266,6 +272,22 @@ func TestIndexForDisplay(t *testing.T) { expected: "CREATE INDEX baz ON foo.public.bar (a DESC) USING HASH WITH (bucket_count=8)", pgExpected: "CREATE INDEX baz ON foo.public.bar USING btree (a DESC) USING HASH WITH (bucket_count=8)", }, + { + index: vectorIndex, + tableName: descpb.AnonymousTable, + partition: "", + displayMode: IndexDisplayDefOnly, + expected: "VECTOR INDEX baz (a)", + pgExpected: "INDEX baz USING cspann (a)", + }, + { + index: vectorIndex, + tableName: tableName, + partition: "", + displayMode: IndexDisplayShowCreate, + expected: "CREATE VECTOR INDEX baz ON foo.public.bar (a)", + pgExpected: "CREATE INDEX baz ON foo.public.bar USING cspann (a)", + }, } sd := &sessiondata.SessionData{} diff --git a/pkg/sql/catalog/colinfo/col_type_info.go b/pkg/sql/catalog/colinfo/col_type_info.go index 09ae9b82b4be..e3a3b111e03f 100644 --- a/pkg/sql/catalog/colinfo/col_type_info.go +++ b/pkg/sql/catalog/colinfo/col_type_info.go @@ -172,6 +172,12 @@ func ColumnTypeIsOnlyInvertedIndexable(t *types.T) bool { return true } +// ColumnTypeIsVectorIndexable returns true if the type t can be indexed using a +// vector index. +func ColumnTypeIsVectorIndexable(t *types.T) bool { + return t.Family() == types.PGVectorFamily +} + // MustBeValueEncoded returns true if columns of the given kind can only be value // encoded. func MustBeValueEncoded(semanticType *types.T) bool { diff --git a/pkg/sql/catalog/descpb/index.go b/pkg/sql/catalog/descpb/index.go index 49c01f8f21ed..bdaaf6d07966 100644 --- a/pkg/sql/catalog/descpb/index.go +++ b/pkg/sql/catalog/descpb/index.go @@ -97,7 +97,7 @@ func (desc *IndexDescriptor) GetName() string { } // InvertedColumnID returns the ColumnID of the inverted column of the inverted -// index. This is always the last column in ColumnIDs. Panics if the index is +// index. This is always the last column in KeyColumnIDs. Panics if the index is // not inverted. func (desc *IndexDescriptor) InvertedColumnID() ColumnID { if desc.Type != idxtype.INVERTED { @@ -126,3 +126,23 @@ func (desc *IndexDescriptor) InvertedColumnKeyType() *types.T { } return types.EncodedKey } + +// VectorColumnID returns the ColumnID of the vector column of the vector index. +// This is always the last column in KeyColumnIDs. Panics if the index is not a +// vector index. +func (desc *IndexDescriptor) VectorColumnID() ColumnID { + if desc.Type != idxtype.VECTOR { + panic(errors.AssertionFailedf("index is not a vector index")) + } + return desc.KeyColumnIDs[len(desc.KeyColumnIDs)-1] +} + +// VectorColumnName returns the name of the vector column of the vector index. +// This is always the last column in KeyColumnNames. Panics if the index is +// not a vector index. +func (desc *IndexDescriptor) VectorColumnName() string { + if desc.Type != idxtype.VECTOR { + panic(errors.AssertionFailedf("index is not a vector index")) + } + return desc.KeyColumnNames[len(desc.KeyColumnNames)-1] +} diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index b0fcd25f73e9..4625e132195b 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -174,6 +175,7 @@ type Index interface { GetPredicate() string GetType() idxtype.T GetGeoConfig() geopb.Config + GetVecConfig() vecpb.Config GetVersion() descpb.IndexDescriptorVersion GetEncodingType() catenumpb.IndexDescriptorEncodingType @@ -223,6 +225,18 @@ type Index interface { // index. InvertedColumnKind() catpb.InvertedIndexColumnKind + // VectorColumnName returns the name of the vector column of the vector + // index. + // + // Panics if the index is not a vector index. + VectorColumnName() string + + // VectorColumnID returns the ColumnID of the vector column of the vector + // index. + // + // Panics if the index is not a vector index. + VectorColumnID() descpb.ColumnID + NumPrimaryStoredColumns() int NumSecondaryStoredColumns() int GetStoredColumnID(storedColumnOrdinal int) descpb.ColumnID diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 3e09efc8f559..5f1e15b597ca 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -21,7 +21,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", - "//pkg/docs", "//pkg/geo/geopb", "//pkg/jobs/jobspb", "//pkg/keys", @@ -59,6 +58,7 @@ go_library( "//pkg/sql/sem/volatility", "//pkg/sql/sqlerrors", "//pkg/sql/types", + "//pkg/sql/vecindex/vecpb", "//pkg/util", "//pkg/util/errorutil/unimplemented", "//pkg/util/hlc", @@ -122,6 +122,7 @@ go_test( "//pkg/sql/sem/idxtype", "//pkg/sql/sem/semenumpb", "//pkg/sql/types", + "//pkg/sql/vecindex/vecpb", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/sql/catalog/tabledesc/index.go b/pkg/sql/catalog/tabledesc/index.go index f7b7c0f82944..bcdfbe72cc41 100644 --- a/pkg/sql/catalog/tabledesc/index.go +++ b/pkg/sql/catalog/tabledesc/index.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecpb" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -223,6 +224,20 @@ func (w index) InvertedColumnKind() catpb.InvertedIndexColumnKind { return w.desc.InvertedColumnKinds[0] } +// VectorColumnID returns the ColumnID of the vector column of the vector index. +// This is always the last column in KeyColumnIDs. Panics if the index is not a +// vector index. +func (w index) VectorColumnID() descpb.ColumnID { + return w.desc.VectorColumnID() +} + +// VectorColumnName returns the name of the vector column of the vector index. +// This is always the last column in KeyColumnIDs. Panics if the index is not a +// vector index. +func (w index) VectorColumnName() string { + return w.desc.VectorColumnName() +} + // CollectKeyColumnIDs creates a new set containing the column IDs in the key // of this index. func (w index) CollectKeyColumnIDs() catalog.TableColSet { @@ -265,6 +280,11 @@ func (w index) GetGeoConfig() geopb.Config { return w.desc.GeoConfig } +// GetVecConfig returns the vec config in the index descriptor. +func (w index) GetVecConfig() vecpb.Config { + return w.desc.VecConfig +} + // GetSharded returns the ShardedDescriptor in the index descriptor func (w index) GetSharded() catpb.ShardedDescriptor { return w.desc.Sharded diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go index 1dd2059142c2..a6b3d08bc1ee 100644 --- a/pkg/sql/catalog/tabledesc/index_test.go +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecpb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -57,17 +58,19 @@ func TestIndexInterface(t *testing.T) { c5 VARCHAR, c6 JSONB, c7 GEOGRAPHY(GEOMETRY,4326) NULL, + c8 VECTOR(3), CONSTRAINT pk PRIMARY KEY (c1 ASC, c2 ASC, c3 ASC), INDEX s1 (c4 DESC, c5 DESC), INVERTED INDEX s2 (c6), INDEX s3 (c2, c3) STORING (c5, c6), INDEX s4 (c5) USING HASH WITH (bucket_count=8), UNIQUE INDEX s5 (c1, c4) WHERE c4 = 'x', - INVERTED INDEX s6 (c7) WITH (s2_level_mod=2) + INVERTED INDEX s6 (c7) WITH (s2_level_mod=2), + VECTOR INDEX s7 (c8) ); `) - indexNames := []string{"pk", "s1", "s2", "s3", "s4", "s5", "s6"} + indexNames := []string{"pk", "s1", "s2", "s3", "s4", "s5", "s6", "s7"} indexColumns := [][]string{ {"c1", "c2", "c3"}, {"c4", "c5"}, @@ -76,6 +79,7 @@ func TestIndexInterface(t *testing.T) { {"crdb_internal_c5_shard_8", "c5"}, {"c1", "c4"}, {"c7"}, + {"c8"}, } extraColumnsAsPkColOrdinals := [][]int{ {}, @@ -85,6 +89,7 @@ func TestIndexInterface(t *testing.T) { {0, 1, 2}, {1, 2}, {0, 1, 2}, + {0, 1, 2}, } immutable := desctestutils.TestingGetPublicTableDescriptor(db, s.Codec(), "d", "t") @@ -110,6 +115,7 @@ func TestIndexInterface(t *testing.T) { s4 := indexes[4] s5 := indexes[5] s6 := indexes[6] + s7 := indexes[7] // Check that GetPrimaryIndex returns the primary index. require.Equal(t, pk, tableI.GetPrimaryIndex()) @@ -265,6 +271,7 @@ func TestIndexInterface(t *testing.T) { require.Equal(t, "c4 = 'x':::STRING", s5.GetPredicate()) require.Equal(t, "crdb_internal_c5_shard_8", s4.GetShardColumnName()) require.Equal(t, int32(2), s6.GetGeoConfig().S2Geography.S2Config.LevelMod) + require.Equal(t, int64(3), s7.GetVecConfig().Dims) for _, idx := range indexes { require.Equalf(t, idx == s5, idx.IsPartial(), errMsgFmt, "IsPartial", idx.GetName()) @@ -284,6 +291,9 @@ func TestIndexInterface(t *testing.T) { errMsgFmt, "GetSharded", idx.GetName()) require.Equalf(t, idx != s3, idx.NumSecondaryStoredColumns() == 0, errMsgFmt, "NumSecondaryStoredColumns", idx.GetName()) + vecConfig := idx.GetVecConfig() + require.Equal(t, idx == s7, !(&vecpb.Config{}).Equal(&vecConfig), + errMsgFmt, "GetVecConfig", idx.GetName()) } // Check index columns. @@ -326,6 +336,8 @@ func TestIndexInterface(t *testing.T) { require.Equal(t, 2, s3.NumSecondaryStoredColumns()) require.Equal(t, "c5", s3.GetStoredColumnName(0)) require.Equal(t, "c6", s3.GetStoredColumnName(1)) + require.Equal(t, s7.GetKeyColumnID(0), s7.VectorColumnID()) + require.Equal(t, "c8", s7.VectorColumnName()) } // TestIndexStrictColumnIDs tests that the index format version value diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 5b2d46c7f2d8..39dc4c6908c7 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -12,7 +12,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -1088,92 +1087,89 @@ func notIndexableError(cols []descpb.ColumnDescriptor) error { if len(cols) == 0 { return nil } - var msg string - var typInfo string if len(cols) == 1 { col := &cols[0] - msg = "column %s is of type %s and thus is not indexable" - typInfo = col.Type.DebugString() - msg = fmt.Sprintf(msg, col.Name, col.Type.Name()) - } else { - msg = "the following columns are not indexable due to their type: " - for i := range cols { - col := &cols[i] - msg += fmt.Sprintf("%s (type %s)", col.Name, col.Type.Name()) - typInfo += col.Type.DebugString() - if i != len(cols)-1 { - msg += ", " - typInfo += "," - } - } + return sqlerrors.NewColumnNotIndexableError(col.Name, col.Type.Name(), col.Type.DebugString()) } - return unimplemented.NewWithIssueDetailf(35730, typInfo, "%s", msg) -} -func checkColumnsValidForIndex(tableDesc *Mutable, indexColNames []string) error { - invalidColumns := make([]descpb.ColumnDescriptor, 0, len(indexColNames)) - for _, indexCol := range indexColNames { - for _, col := range tableDesc.NonDropColumns() { - if col.GetName() == indexCol { - if !colinfo.ColumnTypeIsIndexable(col.GetType()) { - invalidColumns = append(invalidColumns, *col.ColumnDesc()) - } - } + // Use more specific error message when there are multiple columns that + // cannot be indexed. + var typInfo string + msg := "the following columns are not indexable due to their type: " + for i := range cols { + col := &cols[i] + msg += fmt.Sprintf("%s (type %s)", col.Name, col.Type.Name()) + typInfo += col.Type.DebugString() + if i != len(cols)-1 { + msg += ", " + typInfo += "," } } - if len(invalidColumns) > 0 { - return notIndexableError(invalidColumns) - } - return nil + return unimplemented.NewWithIssueDetailf(35730, typInfo, "%s", msg) } -func checkColumnsValidForInvertedIndex( - tableDesc *Mutable, indexColNames []string, colDirs []catenumpb.IndexColumn_Direction, -) error { +func checkColumnsValidForIndex(tableDesc *Mutable, indexDesc *descpb.IndexDescriptor) error { + indexColNames := indexDesc.KeyColumnNames + indexColDirs := indexDesc.KeyColumnDirections + invalidColumns := make([]descpb.ColumnDescriptor, 0, len(indexColNames)) lastCol := len(indexColNames) - 1 for i, indexCol := range indexColNames { - for _, col := range tableDesc.NonDropColumns() { - if col.GetName() == indexCol { - // The last column indexed by an inverted index must be - // inverted indexable. - if i == lastCol && !colinfo.ColumnTypeIsInvertedIndexable(col.GetType()) { - return NewInvalidInvertedColumnError(col.GetName(), col.GetType().String()) + for _, tableCol := range tableDesc.NonDropColumns() { + // Skip until we find the matching column in the table, by name. + if tableCol.GetName() != indexCol { + continue + } + + // Report error if the index type does not allow DESC to be used in + // the last column, because it has no linear ordering. + if !indexDesc.Type.HasLinearOrdering() { + if i == lastCol && indexColDirs[i] == catenumpb.IndexColumn_DESC { + return pgerror.Newf(pgcode.FeatureNotSupported, + "the last column in %s cannot have the DESC option", + idxtype.ErrorText(indexDesc.Type)) } - if i == lastCol && colDirs[i] == catenumpb.IndexColumn_DESC { - return pgerror.New(pgcode.FeatureNotSupported, - "the last column in an inverted index cannot have the DESC option") + } + + // Prefix columns are treated as regular FORWARD columns, but the last + // column has special rules depending on the index type. + if i == lastCol && indexDesc.Type.AllowsPrefixColumns() { + switch indexDesc.Type { + case idxtype.INVERTED: + if !colinfo.ColumnTypeIsInvertedIndexable(tableCol.GetType()) { + return sqlerrors.NewInvalidInvertedColumnError(tableCol.GetName(), tableCol.GetType().String()) + } + case idxtype.VECTOR: + if !colinfo.ColumnTypeIsVectorIndexable(tableCol.GetType()) { + return sqlerrors.NewInvalidVectorColumnError(tableCol.GetName(), tableCol.GetType().String()) + } else if tableCol.GetType().Width() <= 0 { + return sqlerrors.NewInvalidVectorColumnWidthError(tableCol.GetName(), tableCol.GetType().String()) + } } - // Any preceding columns must not be inverted indexable. - if i < lastCol && !colinfo.ColumnTypeIsIndexable(col.GetType()) { - return errors.WithHint( - pgerror.Newf( - pgcode.FeatureNotSupported, - "column %s of type %s is only allowed as the last column in an inverted index", - col.GetName(), - col.GetType().Name(), - ), - "see the documentation for more information about inverted indexes: "+docs.URL("inverted-indexes.html"), - ) + } else { + if !colinfo.ColumnTypeIsIndexable(tableCol.GetType()) { + // Use a more specific error message for column types that can + // be indexed by the right kind of index as the last column. + if colinfo.ColumnTypeIsInvertedIndexable(tableCol.GetType()) { + return sqlerrors.NewColumnOnlyInvertedIndexableError( + tableCol.GetName(), tableCol.GetType().Name()) + } else if colinfo.ColumnTypeIsVectorIndexable(tableCol.GetType()) { + return sqlerrors.NewColumnOnlyVectorIndexableError( + tableCol.GetName(), tableCol.GetType().Name()) + } + + invalidColumns = append(invalidColumns, *tableCol.ColumnDesc()) } } + + break } } + if len(invalidColumns) > 0 { + return notIndexableError(invalidColumns) + } return nil } -// NewInvalidInvertedColumnError returns an error for a column that's not -// inverted indexable. -func NewInvalidInvertedColumnError(colName, colType string) error { - return errors.WithHint( - pgerror.Newf( - pgcode.FeatureNotSupported, - "column %s of type %s is not allowed as the last column in an inverted index", - colName, colType, - ), - "see the documentation for more information about inverted indexes: "+docs.URL("inverted-indexes.html"), - ) -} - // AddColumn adds a column to the table. func (desc *Mutable) AddColumn(col *descpb.ColumnDescriptor) { desc.Columns = append(desc.Columns, *col) @@ -1187,10 +1183,10 @@ func (desc *Mutable) AddFamily(fam descpb.ColumnFamilyDescriptor) { // AddPrimaryIndex adds a primary index to a mutable table descriptor, assuming // that none has yet been set, and performs some sanity checks. func (desc *Mutable) AddPrimaryIndex(idx descpb.IndexDescriptor) error { - if idx.Type == idxtype.INVERTED { - return fmt.Errorf("primary index cannot be inverted") + if !idx.Type.CanBePrimary() { + return fmt.Errorf("primary index cannot be %s", idxtype.ErrorText(idx.Type)) } - if err := checkColumnsValidForIndex(desc, idx.KeyColumnNames); err != nil { + if err := checkColumnsValidForIndex(desc, &idx); err != nil { return err } if desc.PrimaryIndex.Name != "" { @@ -1227,16 +1223,8 @@ func (desc *Mutable) AddPrimaryIndex(idx descpb.IndexDescriptor) error { // AddSecondaryIndex adds a secondary index to a mutable table descriptor. func (desc *Mutable) AddSecondaryIndex(idx descpb.IndexDescriptor) error { - if idx.Type == idxtype.FORWARD { - if err := checkColumnsValidForIndex(desc, idx.KeyColumnNames); err != nil { - return err - } - } else { - if err := checkColumnsValidForInvertedIndex( - desc, idx.KeyColumnNames, idx.KeyColumnDirections, - ); err != nil { - return err - } + if err := checkColumnsValidForIndex(desc, &idx); err != nil { + return err } desc.AddPublicNonPrimaryIndex(idx) return nil @@ -2022,7 +2010,7 @@ func (desc *Mutable) AddColumnMutation( // AddDropIndexMutation adds a a dropping index mutation for the given // index descriptor. func (desc *Mutable) AddDropIndexMutation(idx *descpb.IndexDescriptor) error { - if err := desc.checkValidIndex(idx); err != nil { + if err := checkColumnsValidForIndex(desc, idx); err != nil { return err } m := descpb.DescriptorMutation{ @@ -2038,7 +2026,7 @@ func (desc *Mutable) AddDropIndexMutation(idx *descpb.IndexDescriptor) error { func (desc *Mutable) AddIndexMutationMaybeWithTempIndex( idx *descpb.IndexDescriptor, direction descpb.DescriptorMutation_Direction, ) error { - if err := desc.checkValidIndex(idx); err != nil { + if err := checkColumnsValidForIndex(desc, idx); err != nil { return err } m := descpb.DescriptorMutation{ @@ -2056,7 +2044,7 @@ func (desc *Mutable) AddIndexMutation( direction descpb.DescriptorMutation_Direction, state descpb.DescriptorMutation_State, ) error { - if err := desc.checkValidIndex(idx); err != nil { + if err := checkColumnsValidForIndex(desc, idx); err != nil { return err } stateIsValid := func() bool { @@ -2083,22 +2071,6 @@ func (desc *Mutable) AddIndexMutation( return nil } -func (desc *Mutable) checkValidIndex(idx *descpb.IndexDescriptor) error { - switch idx.Type { - case idxtype.FORWARD: - if err := checkColumnsValidForIndex(desc, idx.KeyColumnNames); err != nil { - return err - } - case idxtype.INVERTED: - if err := checkColumnsValidForInvertedIndex( - desc, idx.KeyColumnNames, idx.KeyColumnDirections, - ); err != nil { - return err - } - } - return nil -} - // AddPrimaryKeySwapMutation adds a PrimaryKeySwap mutation to the table descriptor. func (desc *Mutable) AddPrimaryKeySwapMutation(swap *descpb.PrimaryKeySwap) { desc.addMutation(descpb.DescriptorMutation{ diff --git a/pkg/sql/catalog/tabledesc/structured_test.go b/pkg/sql/catalog/tabledesc/structured_test.go index 9a0dff8250e9..a45164c94c33 100644 --- a/pkg/sql/catalog/tabledesc/structured_test.go +++ b/pkg/sql/catalog/tabledesc/structured_test.go @@ -196,6 +196,7 @@ func TestColumnTypeSQLString(t *testing.T) { {types.String, "STRING"}, {types.MakeString(10), "STRING(10)"}, {types.Bytes, "BYTES"}, + {types.MakePGVector(3), "VECTOR(3)"}, } for i, d := range testData { t.Run(d.colType.DebugString(), func(t *testing.T) { diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index e55c3b894141..0fb635a9e274 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" - "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/intsets" @@ -843,7 +842,10 @@ func maybeUpgradePrimaryIndexFormatVersion(builder *tableDescriptorBuilder) (has func maybeUpgradeSecondaryIndexFormatVersion(idx *descpb.IndexDescriptor) (hasChanged bool) { switch idx.Version { case descpb.SecondaryIndexFamilyFormatVersion: - if idx.Type == idxtype.INVERTED { + // If the index type does not support STORING columns, then it's not + // encoded differently based on whether column families are in use, so + // nothing to do. + if !idx.Type.SupportsStoring() { return false } case descpb.EmptyArraysInInvertedIndexesVersion: diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 7b3a18f734f0..86212802e1c4 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -28,6 +28,7 @@ import ( "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/storageparam" "github.com/cockroachdb/cockroach/pkg/sql/storageparam/indexstorageparam" @@ -424,7 +425,7 @@ func populateInvertedIndexDescriptor( return newUndefinedOpclassError(invCol.OpClass) } default: - return tabledesc.NewInvalidInvertedColumnError(column.GetName(), column.GetType().Name()) + return sqlerrors.NewInvalidInvertedColumnError(column.GetName(), column.GetType().Name()) } return nil } diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index 786a7df72f02..b4099fabef6f 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -578,6 +578,10 @@ func createStatsDefaultColumns( // Add column stats for each secondary index. for _, idx := range desc.PublicNonPrimaryIndexes() { + if idx.GetType() == idxtype.VECTOR { + // Skip vector indexes for now. + continue + } for j, n := 0, idx.NumKeyColumns(); j < n; j++ { colID := idx.GetKeyColumnID(j) isInverted := idx.GetType() == idxtype.INVERTED && colID == idx.InvertedColumnID() diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index ce2da5cf872b..551d40ad927b 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1887,9 +1887,6 @@ func NewTableDesc( // pass, handled above. case *tree.IndexTableDef: - 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 // indexes will be given a unique auto-generated name later on when // AllocateIDs is called. @@ -1950,6 +1947,14 @@ func NewTableDesc( return nil, err } } + if d.Type == idxtype.VECTOR { + column, err := catalog.MustFindColumnByName(&desc, idx.VectorColumnName()) + if err != nil { + return nil, err + } + idx.VecConfig.Dims = column.GetType().Width() + idx.VecConfig.Seed = evalCtx.GetRNG().Int63() + } var idxPartitionBy *tree.PartitionBy if desc.PartitionAllBy && d.PartitionByIndex.ContainsPartitions() { @@ -2393,6 +2398,12 @@ func NewTableDesc( telemetry.Inc(sqltelemetry.PartitionedInvertedIndexCounter) } } + if idx.GetType() == idxtype.VECTOR { + telemetry.Inc(sqltelemetry.VectorIndexCounter) + if idx.NumKeyColumns() > 1 { + telemetry.Inc(sqltelemetry.MultiColumnVectorIndexCounter) + } + } if idx.IsPartial() { telemetry.Inc(sqltelemetry.PartialIndexCounter) } @@ -2809,8 +2820,8 @@ func replaceLikeTableOpts(n *tree.CreateTable, params runParams) (tree.TableDefs indexDef.Columns = append(indexDef.Columns, elem) } // The last column of an inverted or vector index cannot have an - // explicit direction. - if !indexDef.Type.AllowExplicitDirection() { + // explicit direction, because it does not have a linear ordering. + if !indexDef.Type.HasLinearOrdering() { indexDef.Columns[len(indexDef.Columns)-1].Direction = tree.DefaultDirection } for j := 0; j < idx.NumSecondaryStoredColumns(); j++ { diff --git a/pkg/sql/logictest/testdata/logic_test/create_table b/pkg/sql/logictest/testdata/logic_test/create_table index c4cc17f87cf8..831fe6530389 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_table +++ b/pkg/sql/logictest/testdata/logic_test/create_table @@ -108,19 +108,23 @@ CREATE TABLE telemetry ( x INT PRIMARY KEY, y INT, z JSONB, + v VECTOR(3), INVERTED INDEX (z), - INDEX (y) USING HASH WITH (bucket_count=4) + INDEX (y) USING HASH WITH (bucket_count=4), + VECTOR INDEX (v) ) query T rowsort SELECT feature_name FROM crdb_internal.feature_usage WHERE feature_name IN ( 'sql.schema.inverted_index', - 'sql.schema.hash_sharded_index' + 'sql.schema.hash_sharded_index', + 'sql.schema.vector_index' ) ---- sql.schema.inverted_index sql.schema.hash_sharded_index +sql.schema.vector_index subtest like_table @@ -138,12 +142,14 @@ CREATE TABLE like_table ( j JSON, k INT UNIQUE WITHOUT INDEX, t TIMESTAMPTZ DEFAULT current_timestamp() - '5s'::interval ON UPDATE current_timestamp(), + v VECTOR(3), PRIMARY KEY (a, b), UNIQUE INDEX foo (b DESC, c), INDEX (c) STORING (j), INVERTED INDEX (j), UNIQUE WITHOUT INDEX (h), - UNIQUE WITHOUT INDEX (h) WHERE h > 0 + UNIQUE WITHOUT INDEX (h) WHERE h > 0, + VECTOR INDEX (v) ) statement ok @@ -160,6 +166,7 @@ like_none CREATE TABLE public.like_none ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL, + v VECTOR(3) NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT like_none_pkey PRIMARY KEY (rowid ASC) ) @@ -178,6 +185,7 @@ like_constraints CREATE TABLE public.like_constraints ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL, + v VECTOR(3) NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT like_constraints_pkey PRIMARY KEY (rowid ASC), CONSTRAINT check_a CHECK (a > 3:::INT8), @@ -200,10 +208,12 @@ like_indexes CREATE TABLE public.like_indexes ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL, + v VECTOR(3) NULL, CONSTRAINT like_table_pkey PRIMARY KEY (a ASC, b ASC), UNIQUE INDEX foo (b DESC, c ASC), INDEX like_table_c_idx (c ASC) STORING (j), - INVERTED INDEX like_table_j_idx (j) + INVERTED INDEX like_table_j_idx (j), + VECTOR INDEX like_table_v_idx (v) ) # INCLUDING GENERATED adds "generated columns", aka stored columns. @@ -221,6 +231,7 @@ like_generated CREATE TABLE public.like_generated ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL, + v VECTOR(3) NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT like_generated_pkey PRIMARY KEY (rowid ASC) ) @@ -239,6 +250,7 @@ like_defaults CREATE TABLE public.like_defaults ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL DEFAULT current_timestamp():::TIMESTAMPTZ - '00:00:05':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ, + v VECTOR(3) NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT like_defaults_pkey PRIMARY KEY (rowid ASC) ) @@ -257,10 +269,12 @@ like_all CREATE TABLE public.like_all ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL DEFAULT current_timestamp():::TIMESTAMPTZ - '00:00:05':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ, + v VECTOR(3) NULL, CONSTRAINT like_table_pkey PRIMARY KEY (a ASC, b ASC), UNIQUE INDEX foo (b DESC, c ASC), INDEX like_table_c_idx (c ASC) STORING (j), INVERTED INDEX like_table_j_idx (j), + VECTOR INDEX like_table_v_idx (v), CONSTRAINT check_a CHECK (a > 3:::INT8), CONSTRAINT unique_k UNIQUE WITHOUT INDEX (k), CONSTRAINT unique_h UNIQUE WITHOUT INDEX (h), @@ -283,10 +297,12 @@ like_mixed CREATE TABLE public.like_mixed ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL DEFAULT current_timestamp():::TIMESTAMPTZ - '00:00:05':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ, + v VECTOR(3) NULL, CONSTRAINT like_table_pkey PRIMARY KEY (a ASC, b ASC), UNIQUE INDEX foo (b DESC, c ASC), INDEX like_table_c_idx (c ASC) STORING (j), - INVERTED INDEX like_table_j_idx (j) + INVERTED INDEX like_table_j_idx (j), + VECTOR INDEX like_table_v_idx (v) ) statement ok @@ -329,6 +345,7 @@ like_more_specifiers CREATE TABLE public.like_more_specifiers ( j JSONB NULL, k INT8 NULL, t TIMESTAMPTZ NULL, + v VECTOR(3) NULL, z DECIMAL NULL, blah INT8 NULL, rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), @@ -1204,3 +1221,9 @@ CREATE TABLE create_table_index_elem_duplicate_storage_params_e ( ); subtest end + +statement error pq: unimplemented: column col2 is of type refcursor and thus is not indexable\nHINT: You have attempted to use a feature that is not yet implemented.\nSee: https://go.crdb.dev/issue-v/35730/dev +CREATE TABLE not_indexable (COL1 INT PRIMARY KEY, COL2 REFCURSOR, INDEX (COL2)) + +statement error pq: unimplemented: the following columns are not indexable due to their type: col2 \(type refcursor\), col3 \(type refcursor\)\nHINT: You have attempted to use a feature that is not yet implemented.\nSee: https://go.crdb.dev/issue-v/35730/dev +CREATE TABLE not_indexable (COL1 INT PRIMARY KEY, COL2 REFCURSOR, COL3 REFCURSOR, INDEX (COL2, COL3)) diff --git a/pkg/sql/logictest/testdata/logic_test/vector_index b/pkg/sql/logictest/testdata/logic_test/vector_index new file mode 100644 index 000000000000..91a68a590cf3 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/vector_index @@ -0,0 +1,140 @@ +# ------------------------------------------------------------------------------ +# CREATE TABLE tests. +# ------------------------------------------------------------------------------ + +# Simple vector index. +statement ok +CREATE TABLE simple ( + a INT PRIMARY KEY, + vec1 VECTOR(3), + VECTOR INDEX (vec1), + FAMILY (a, vec1) +) + +query TT +SHOW CREATE TABLE simple +---- +simple CREATE TABLE public.simple ( + a INT8 NOT NULL, + vec1 VECTOR(3) NULL, + CONSTRAINT simple_pkey PRIMARY KEY (a ASC), + VECTOR INDEX simple_vec1_idx (vec1), + FAMILY fam_0_a_vec1 (a, vec1) + ) + +statement ok +SHOW INDEX FROM simple + +# Specify name for index. +statement ok +CREATE TABLE alt_syntax ( + a INT PRIMARY KEY, + vec1 VECTOR(3), + VECTOR INDEX vec_idx (vec1), + FAMILY (a, vec1) +) + +query TT +SHOW CREATE TABLE alt_syntax +---- +alt_syntax CREATE TABLE public.alt_syntax ( + a INT8 NOT NULL, + vec1 VECTOR(3) NULL, + CONSTRAINT alt_syntax_pkey PRIMARY KEY (a ASC), + VECTOR INDEX vec_idx (vec1), + FAMILY fam_0_a_vec1 (a, vec1) + ) + +# Multiple vector indexes on same table. +statement ok +CREATE TABLE multiple_indexes ( + a INT PRIMARY KEY, + vec1 VECTOR(3), + vec2 VECTOR(1000), + VECTOR INDEX (vec1), + VECTOR INDEX (vec2), + FAMILY (a, vec1, vec2) +) + +query TT +SHOW CREATE TABLE multiple_indexes +---- +multiple_indexes CREATE TABLE public.multiple_indexes ( + a INT8 NOT NULL, + vec1 VECTOR(3) NULL, + vec2 VECTOR(1000) NULL, + CONSTRAINT multiple_indexes_pkey PRIMARY KEY (a ASC), + VECTOR INDEX multiple_indexes_vec1_idx (vec1), + VECTOR INDEX multiple_indexes_vec2_idx (vec2), + FAMILY fam_0_a_vec1_vec2 (a, vec1, vec2) + ) + +# Use prefix columns in the vector index. +statement ok +CREATE TABLE prefix_cols ( + a INT PRIMARY KEY, + b INT, + c INT, + vec1 VECTOR(3), + VECTOR INDEX (c DESC, b, vec1), + FAMILY (a, b, c, vec1) +) + +query TT +SHOW CREATE TABLE prefix_cols +---- +prefix_cols CREATE TABLE public.prefix_cols ( + a INT8 NOT NULL, + b INT8 NULL, + c INT8 NULL, + vec1 VECTOR(3) NULL, + CONSTRAINT prefix_cols_pkey PRIMARY KEY (a ASC), + VECTOR INDEX prefix_cols_c_b_vec1_idx (c DESC, b ASC, vec1), + FAMILY fam_0_a_b_c_vec1 (a, b, c, vec1) + ) + +# Use mixed-case column for vector index. +statement ok +CREATE TABLE mixed_case ( + a INT PRIMARY KEY, + qUuX VECTOR(3), + VECTOR INDEX (qUuX) +) + +# Try to use vector in primary key. +statement error pq: column a of type vector is only allowed as the last column in a vector index +CREATE TABLE t (a VECTOR(3), PRIMARY KEY (a)) + +statement error pq: column b of type int is not allowed as the last column in a vector index +CREATE TABLE t (a INT PRIMARY KEY, b INT, VECTOR INDEX (b)) + +statement error pq: column c of type vector is only allowed as the last column in a vector index +CREATE TABLE t (a INT PRIMARY KEY, b INT, c VECTOR(3), VECTOR INDEX (c, b)) + +# Try to use inverted indexable type in vector index. +statement error pq: column b of type tsvector is only allowed as the last column in an inverted index\nHINT: see the documentation for more information about inverted indexes: https://www.cockroachlabs.com/docs/dev/inverted-indexes.html +CREATE TABLE t (a INT PRIMARY KEY, b TSVECTOR, c VECTOR(3), VECTOR INDEX (b, c)) + +statement error the last column in a vector index cannot have the DESC option +CREATE TABLE t (a INT PRIMARY KEY, b INT, c VECTOR(3), VECTOR INDEX (b, c DESC)) + +statement error pq: column b of type vector is not allowed as the last column in a vector index\nDETAIL: the vector index column must have a fixed positive number of dimensions +CREATE TABLE t (a INT PRIMARY KEY, b VECTOR, VECTOR INDEX (b)) + +# Try to use vector type in forward index. +statement error pq: column c of type vector is only allowed as the last column in a vector index +CREATE TABLE t (a INT PRIMARY KEY, b INT, c VECTOR(3), INDEX (b, c)) + +# ------------------------------------------------------------------------------ +# Execution tests. +# ------------------------------------------------------------------------------ + +statement ok +CREATE TABLE exec_test ( + a INT PRIMARY KEY, + vec1 VECTOR(3), + VECTOR INDEX (vec1) +) + +statement error unimplemented: execution planning for vector index search is not yet implemented +INSERT INTO exec_test (a, vec1) values (1, '[1, 2, 3]'); diff --git a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go index f23919c47e31..050735385c9a 100644 --- a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go @@ -2481,6 +2481,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go index a1fd572a2db5..9323b8e59168 100644 --- a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go @@ -2488,6 +2488,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist/generated_test.go b/pkg/sql/logictest/tests/fakedist/generated_test.go index 792964409009..683b2d679a1e 100644 --- a/pkg/sql/logictest/tests/fakedist/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist/generated_test.go @@ -2502,6 +2502,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index d574449c2415..87e52dd13fdc 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -2474,6 +2474,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go b/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go index abc1d25ad439..2a2173052c85 100644 --- a/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go +++ b/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go @@ -2488,6 +2488,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-vec-off/generated_test.go b/pkg/sql/logictest/tests/local-vec-off/generated_test.go index 9da9861ec225..5006bcbb94e5 100644 --- a/pkg/sql/logictest/tests/local-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/local-vec-off/generated_test.go @@ -2516,6 +2516,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize_agg( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index f859c26e7db0..a8e4e425645d 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -2740,6 +2740,13 @@ func TestLogic_values( runLogicTest(t, "values") } +func TestLogic_vector_index( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "vector_index") +} + func TestLogic_vectorize( t *testing.T, ) { diff --git a/pkg/sql/parser/lexer.go b/pkg/sql/parser/lexer.go index 9e377869a88e..a8411925dfb8 100644 --- a/pkg/sql/parser/lexer.go +++ b/pkg/sql/parser/lexer.go @@ -164,7 +164,10 @@ func (l *lexer) Lex(lval *sqlSymType) int { (afterCommaOrParen && followedByNonPunctThenParen) || // CREATE ... (INVERTED INDEX abc ( // CREATE ... (x INT, y INT, INVERTED INDEX abc ( - (afterCommaOrParenThenINVERTED && followedByNonPunctThenParen) { + (afterCommaOrParenThenINVERTED && followedByNonPunctThenParen) || + // CREATE ... (VECTOR INDEX abc ( + // CREATE ... (x INT, y INT, VECTOR INDEX abc ( + (afterCommaOrParenThenVECTOR && followedByNonPunctThenParen) { lval.id = INDEX_BEFORE_NAME_THEN_PAREN break } diff --git a/pkg/sql/parser/testdata/create_table b/pkg/sql/parser/testdata/create_table index 7b02d4f24108..2e02f910fc8a 100644 --- a/pkg/sql/parser/testdata/create_table +++ b/pkg/sql/parser/testdata/create_table @@ -1377,6 +1377,23 @@ CREATE TABLE a (b INT8, VECTOR INDEX (b)) -- fully parenthesized CREATE TABLE a (b INT8, VECTOR INDEX (b)) -- literals removed CREATE TABLE _ (_ INT8, VECTOR INDEX (_)) -- identifiers removed +# Explicitly name the index. +parse +CREATE TABLE a (b INT8, INVERTED INDEX i (b)) +---- +CREATE TABLE a (b INT8, INVERTED INDEX i (b)) +CREATE TABLE a (b INT8, INVERTED INDEX i (b)) -- fully parenthesized +CREATE TABLE a (b INT8, INVERTED INDEX i (b)) -- literals removed +CREATE TABLE _ (_ INT8, INVERTED INDEX _ (_)) -- identifiers removed + +parse +CREATE TABLE a (b INT8, VECTOR INDEX i (b)) +---- +CREATE TABLE a (b INT8, VECTOR INDEX i (b)) +CREATE TABLE a (b INT8, VECTOR INDEX i (b)) -- fully parenthesized +CREATE TABLE a (b INT8, VECTOR INDEX i (b)) -- literals removed +CREATE TABLE _ (_ INT8, VECTOR INDEX _ (_)) -- identifiers removed + parse CREATE TABLE a (b INT8, c INT8 REFERENCES foo) ---- diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go index 34182e6b5489..b02ed7c5d572 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go @@ -36,7 +36,6 @@ 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/protoutil" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" @@ -164,10 +163,7 @@ func alterTableAddColumn( !d.Unique.WithoutIndex && !colinfo.ColumnTypeIsIndexable(spec.colType.Type) { typInfo := spec.colType.Type.DebugString() - panic(unimplemented.NewWithIssueDetailf(35730, typInfo, - "column %s is of type %s and thus is not indexable", - d.Name, - spec.colType.Type.Name())) + panic(sqlerrors.NewColumnNotIndexableError(d.Name.String(), spec.colType.Type.Name(), typInfo)) } // Block unsupported types. switch spec.colType.Type.Oid() { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go index cf990f37781a..a55da8874ff7 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go @@ -25,6 +25,7 @@ import ( "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/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -348,12 +349,8 @@ func checkForEarlyExit(b BuildCtx, tbl *scpb.Table, t alterPrimaryKeySpec) { columnType := mustRetrieveColumnTypeElem(b, tbl.TableID, colElem.ColumnID) // Check if the column type is indexable. if !colinfo.ColumnTypeIsIndexable(columnType.Type) { - panic(unimplemented.NewWithIssueDetailf(35730, - columnType.Type.DebugString(), - "column %s is of type %s and thus is not indexable", - col.Column, - columnType.Type), - ) + panic(sqlerrors.NewColumnNotIndexableError( + col.Column.String(), columnType.Type.Name(), columnType.Type.DebugString())) } } } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_index.go index 23bf38478429..8cae743805e7 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_index.go @@ -311,8 +311,9 @@ func processColNodeType( panic(pgerror.New(pgcode.DatatypeMismatch, "operator classes are only allowed for the last column of an inverted index")) } - // Disallow descending last columns in inverted indexes. - if !n.Type.AllowExplicitDirection() && columnNode.Direction == tree.Descending && lastColIdx { + // Disallow descending last column in inverted indexes because they have no + // linear ordering. + if !n.Type.HasLinearOrdering() && columnNode.Direction == tree.Descending && lastColIdx { panic(pgerror.New(pgcode.FeatureNotSupported, "the last column in an inverted index cannot have the DESC option")) } @@ -387,15 +388,11 @@ func processColNodeType( if columnNode.Expr != nil { colNameForErr = columnNode.Expr.String() } - panic(tabledesc.NewInvalidInvertedColumnError(colNameForErr, - columnType.Type.String())) + panic(sqlerrors.NewInvalidInvertedColumnError(colNameForErr, columnType.Type.String())) } else if (n.Type != idxtype.INVERTED || !lastColIdx) && !colinfo.ColumnTypeIsIndexable(columnType.Type) { // Otherwise, check if the column type is indexable. - panic(unimplemented.NewWithIssueDetailf(35730, - columnType.Type.DebugString(), - "column %s is of type %s and thus is not indexable", - colName, - columnType.Type)) + panic(sqlerrors.NewColumnNotIndexableError( + colName, columnType.Type.Name(), columnType.Type.DebugString())) } return invertedKind } diff --git a/pkg/sql/sem/idxtype/BUILD.bazel b/pkg/sql/sem/idxtype/BUILD.bazel index 79bfd0e3fecf..8b68e633cfb4 100644 --- a/pkg/sql/sem/idxtype/BUILD.bazel +++ b/pkg/sql/sem/idxtype/BUILD.bazel @@ -17,6 +17,7 @@ go_library( embed = [":idxtype_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype", visibility = ["//visibility:public"], + deps = ["@com_github_cockroachdb_errors//:errors"], ) go_proto_library( @@ -41,6 +42,7 @@ disallowed_imports_test( "idxtype", allowlist = [ "//pkg/sql/sem/idxtype:idxtype_go_proto", + "@com_github_cockroachdb_errors//errorspb", "@com_github_gogo_protobuf//gogoproto", "@com_github_gogo_protobuf//proto", "@com_github_gogo_protobuf//protoc-gen-gogo/descriptor", diff --git a/pkg/sql/sem/idxtype/idxtype.go b/pkg/sql/sem/idxtype/idxtype.go index 2e49b2729884..ae5e13cd8469 100644 --- a/pkg/sql/sem/idxtype/idxtype.go +++ b/pkg/sql/sem/idxtype/idxtype.go @@ -5,6 +5,11 @@ package idxtype +import ( + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + // CanBePrimary is true if this index type can be the primary index that always // contains unique keys sorted according to the primary ordering of the table. // Secondary indexes refer to rows in the primary index by unique key value. @@ -18,11 +23,12 @@ func (t T) CanBeUnique() bool { return t == FORWARD } -// AllowExplicitDirection is true if this index type allows all of its columns -// to specify an explicit ascending or descending direction. For example, -// inverted and vector indexes do not allow the last column in the index to -// specify an explicit direction. -func (t T) AllowExplicitDirection() bool { +// HasLinearOrdering is true if this index type does not define a linear +// ordering on the last key column in the index. For example, a vector index +// groups nearby vectors, but does not define a linear ordering among them. As +// another example, an inverted index only defines a linear ordering for tokens, +// not for the original JSONB or ARRAY data type. +func (t T) HasLinearOrdering() bool { return t == FORWARD } @@ -33,24 +39,42 @@ func (t T) AllowsPrefixColumns() bool { return t == INVERTED || t == VECTOR } -// SupportsSharding is true if this index can be hash sharded, meaning that its -// rows are grouped according to a hash value and spread across the keyspace. +// SupportsSharding is true if this index type can be hash sharded, meaning that +// its rows are grouped according to a hash value and spread across the +// keyspace. func (t T) SupportsSharding() bool { return t == FORWARD } -// SupportsStoring is true if this index allows STORING values, which are +// SupportsStoring is true if this index type allows STORING values, which are // un-indexed columns from the table that are stored directly in the index for // faster retrieval. func (t T) SupportsStoring() bool { return t == FORWARD } -// SupportsOpClass is true if this index allows columns to specify an operator -// class, which defines an alternate set of operators used when sorting and -// querying those columns. +// SupportsOpClass is true if this index type allows columns to specify an +// operator class, which defines an alternate set of operators used when sorting +// and querying those columns. // NOTE: Currently, only inverted indexes support operator classes, and only on // the last column of the index. func (t T) SupportsOpClass() bool { return t == INVERTED } + +// ErrorText describes the type of the index using the phrase "an inverted +// index" or "a vector index". This is intended to be included in errors that +// apply to multiple index types. +// +// Panics if this is called for a forward index, as "forward" is not a term we +// should be including in error messages, as most users will not understand it. +func ErrorText(t T) redact.SafeString { + switch t { + case INVERTED: + return "an inverted index" + case VECTOR: + return "a vector index" + default: + panic(errors.AssertionFailedf("ErrorText is not defined for index type %v", t)) + } +} diff --git a/pkg/sql/sqlerrors/BUILD.bazel b/pkg/sql/sqlerrors/BUILD.bazel index 2e14735bd4cc..d805f599f2be 100644 --- a/pkg/sql/sqlerrors/BUILD.bazel +++ b/pkg/sql/sqlerrors/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/build", + "//pkg/docs", "//pkg/roachpb", "//pkg/security/username", "//pkg/sql/catalog/catpb", diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index fbaa3cef7d26..7683a0c7cf84 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -12,6 +12,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -512,6 +513,73 @@ func NewInsufficientPrivilegeOnDescriptorError( user, privsStr, descType, descName) } +// NewColumnNotIndexableError returns an error for a column type that cannot be +// indexed. +func NewColumnNotIndexableError(colName string, colType string, detail string) error { + return unimplemented.NewWithIssueDetailf(35730, + detail, + "column %s is of type %s and thus is not indexable", + colName, + colType) +} + +// NewInvalidInvertedColumnError returns an error for a column that's not +// inverted indexable. +func NewInvalidInvertedColumnError(colName, colType string) error { + return errors.WithHint( + pgerror.Newf( + pgcode.FeatureNotSupported, + "column %s of type %s is not allowed as the last column in an inverted index", + colName, colType, + ), + "see the documentation for more information about inverted indexes: "+docs.URL("inverted-indexes.html"), + ) +} + +// NewColumnOnlyInvertedIndexableError returns an error for a column with a type +// that can only be indexed as the last column in an inverted index. +func NewColumnOnlyInvertedIndexableError(colName string, colType string) error { + return errors.WithHint(pgerror.Newf( + pgcode.FeatureNotSupported, + "column %s of type %s is only allowed as the last column in an inverted index", + colName, + colType, + ), "see the documentation for more information about inverted indexes: "+docs.URL("inverted-indexes.html")) +} + +// NewInvalidVectorColumnError returns an error for a column that cannot be +// indexed by a vector index. +func NewInvalidVectorColumnError(colName, colType string) error { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "column %s of type %s is not allowed as the last column in a vector index", + colName, colType, + ) +} + +// NewInvalidVectorColumnWidthError returns an error for a vector column that +// cannot be indexed by a vector index due to invalid type width. +func NewInvalidVectorColumnWidthError(colName string, colType string) error { + return errors.WithDetail( + pgerror.Newf( + pgcode.InvalidTableDefinition, + "column %s of type %s is not allowed as the last column in a vector index", + colName, colType, + ), + "the vector index column must have a fixed positive number of dimensions", + ) +} + +// NewColumnOnlyVectorIndexableError returns an error for a column with a type +// that can only be indexed as the last column in a vector index. +func NewColumnOnlyVectorIndexableError(colName string, colType string) error { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "column %s of type %s is only allowed as the last column in a vector index", + colName, + colType) +} + // QueryTimeoutError is an error representing a query timeout. var QueryTimeoutError = pgerror.New( pgcode.QueryCanceled, "query execution canceled due to statement timeout") diff --git a/pkg/sql/sqltelemetry/schema.go b/pkg/sql/sqltelemetry/schema.go index 4f75640757b5..36722a51db00 100644 --- a/pkg/sql/sqltelemetry/schema.go +++ b/pkg/sql/sqltelemetry/schema.go @@ -85,6 +85,15 @@ var ( // index is created. This includes both regular and inverted expression // indexes. ExpressionIndexCounter = telemetry.GetCounterOnce("sql.schema.expression_index") + + // VectorIndexCounter is to be incremented every time a vector index is + // created. This includes single-column vector indexes, multi-column vector + // indexes, and partial vector indexes. + VectorIndexCounter = telemetry.GetCounterOnce("sql.schema.vector_index") + + // MultiColumnVectorIndexCounter is to be incremented every time a + // multi-column vector index is created. + MultiColumnVectorIndexCounter = telemetry.GetCounterOnce("sql.schema.multi_column_vector_index") ) // SchemaChangeIndexCounter is to be incremented for certain CREATE diff --git a/pkg/sql/vecindex/vector_index.go b/pkg/sql/vecindex/vector_index.go index 6b760d52d1c6..8c7d9f3d8304 100644 --- a/pkg/sql/vecindex/vector_index.go +++ b/pkg/sql/vecindex/vector_index.go @@ -60,7 +60,7 @@ type VectorIndexOptions struct { DisableErrorBounds bool // Seed is used to initialize a deterministic random number generator for // testing purposes. If this is zero, then the global random number generator - // is used intead. + // is used instead. Seed int64 // MaxWorkers specifies the maximum number of background workers that can be // created to process fixups for this vector index instance.