From 12b13339dbef2f53a59e8ad6d90b5860f60367cc Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 18 Sep 2024 17:02:24 +0800 Subject: [PATCH] *: support create table with vector indexes and others --- pkg/ddl/column.go | 4 +- pkg/ddl/create_table.go | 66 ++++++++++++--- pkg/ddl/db_integration_test.go | 6 +- pkg/ddl/index.go | 5 +- pkg/ddl/index_modify_test.go | 84 ++++++++++++++++++- pkg/executor/builder.go | 4 + .../integration_test/integration_test.go | 2 +- pkg/meta/model/job.go | 1 + 8 files changed, 151 insertions(+), 21 deletions(-) diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index ee2e1304ebae5..180608b9f85e8 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -1094,10 +1094,10 @@ func isColumnWithIndex(colName string, indices []*model.IndexInfo) bool { func isColumnCanDropWithIndex(colName string, indices []*model.IndexInfo) error { for _, indexInfo := range indices { - if indexInfo.Primary || len(indexInfo.Columns) > 1 { + if indexInfo.Primary || len(indexInfo.Columns) > 1 || indexInfo.VectorInfo != nil { for _, col := range indexInfo.Columns { if col.Name.L == colName { - return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s with composite index covered or Primary Key covered now", colName) + return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s with composite index covered or Primary Key or Vector Key covered now", colName) } } } diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index ee9ec8715ee9a..5e15f6ea8f290 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -509,6 +509,40 @@ func checkGeneratedColumn(ctx sessionctx.Context, schemaName pmodel.CIStr, table return nil } +func checkVectorIndexIfNeedTiFlashReplica(ctx sessionctx.Context, tblInfo *model.TableInfo) error { + var hasVectorIndex bool + for _, idx := range tblInfo.Indices { + if idx.VectorInfo != nil { + hasVectorIndex = true + break + } + } + + if hasVectorIndex { + if tblInfo.TiFlashReplica == nil { + replicas, err := infoschema.GetTiFlashStoreCount(ctx) + if err != nil { + return errors.Trace(err) + } + if replicas == 0 { + return errors.Trace(dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported TiFlash store count is 0")) + } + + // Always try to set to 1 as the default replica count. + defaultReplicas := uint64(1) + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: defaultReplicas, + LocationLabels: make([]string, 0), + } + } + + if err := checkTableTypeForVectorIndex(tblInfo); err != nil { + return errors.Trace(err) + } + } + return nil +} + // checkTableInfoValidExtra is like checkTableInfoValid, but also assumes the // table info comes from untrusted source and performs further checks such as // name length and column count. @@ -537,6 +571,10 @@ func checkTableInfoValidExtra(ctx sessionctx.Context, tbInfo *model.TableInfo) e if err := checkGlobalIndexes(ctx, tbInfo); err != nil { return errors.Trace(err) } + // A special rule on Serverless is to add TiFlash replica by default if there is a vector index. + if err := checkVectorIndexIfNeedTiFlashReplica(ctx, tbInfo); err != nil { + return errors.Trace(err) + } // FIXME: perform checkConstraintNames if err := checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate); err != nil { @@ -1161,14 +1199,13 @@ func BuildTableInfo( } foreignKeyID := tbInfo.MaxForeignKeyID for _, constr := range constraints { - if constr.Tp == ast.ConstraintVector { - return nil, dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("not currently supported") - } - - // Build hidden columns if necessary. - hiddenCols, err := buildHiddenColumnInfoWithCheck(ctx, constr.Keys, pmodel.NewCIStr(constr.Name), tbInfo, tblColumns) - if err != nil { - return nil, err + var hiddenCols []*model.ColumnInfo + if constr.Tp != ast.ConstraintVector { + // Build hidden columns if necessary. + hiddenCols, err = buildHiddenColumnInfoWithCheck(ctx, constr.Keys, pmodel.NewCIStr(constr.Name), tbInfo, tblColumns) + if err != nil { + return nil, err + } } for _, hiddenCol := range hiddenCols { hiddenCol.State = model.StatePublic @@ -1233,11 +1270,11 @@ func BuildTableInfo( } var ( - indexName = constr.Name - primary, unique bool + indexName = constr.Name + primary, unique, vector bool ) - // Check if the index is primary or unique. + // Check if the index is primary, unique or vector. switch constr.Tp { case ast.ConstraintPrimaryKey: primary = true @@ -1245,6 +1282,11 @@ func BuildTableInfo( indexName = mysql.PrimaryKeyName case ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex: unique = true + case ast.ConstraintVector: + if constr.Option.Visibility == ast.IndexVisibilityInvisible { + return nil, dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("set vector index invisible") + } + vector = true } // check constraint @@ -1317,7 +1359,7 @@ func BuildTableInfo( pmodel.NewCIStr(indexName), primary, unique, - false, + vector, constr.Keys, constr.Option, model.StatePublic, diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index ae397fc44ea9d..dde678e2da8e9 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -1967,11 +1967,13 @@ func TestDropColumnWithCompositeIndex(t *testing.T) { defer tk.MustExec("drop table if exists t_drop_column_with_comp_idx") tk.MustExec("create index idx_bc on t_drop_column_with_comp_idx(b, c)") tk.MustExec("create index idx_b on t_drop_column_with_comp_idx(b)") - tk.MustGetErrMsg("alter table t_drop_column_with_comp_idx drop column b", "[ddl:8200]can't drop column b with composite index covered or Primary Key covered now") + tk.MustGetErrMsg("alter table t_drop_column_with_comp_idx drop column b", + "[ddl:8200]can't drop column b with composite index covered or Primary Key or Vector Key covered now") tk.MustQuery(query).Check(testkit.Rows("idx_b YES", "idx_bc YES")) tk.MustExec("alter table t_drop_column_with_comp_idx alter index idx_bc invisible") tk.MustExec("alter table t_drop_column_with_comp_idx alter index idx_b invisible") - tk.MustGetErrMsg("alter table t_drop_column_with_comp_idx drop column b", "[ddl:8200]can't drop column b with composite index covered or Primary Key covered now") + tk.MustGetErrMsg("alter table t_drop_column_with_comp_idx drop column b", + "[ddl:8200]can't drop column b with composite index covered or Primary Key or Vector Key covered now") tk.MustQuery(query).Check(testkit.Rows("idx_b NO", "idx_bc NO")) } diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index b9fd09e875c19..604f3742317fe 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -410,7 +410,7 @@ func buildVectorInfoWithCheck(indexPartSpecifications []*ast.IndexPartSpecificat } colInfo := findColumnByName(colExpr.Name.Name.L, tblInfo) if colInfo == nil { - return nil, "", infoschema.ErrColumnNotExists.GenWithStackByArgs(colExpr.Name.Name.String()) + return nil, "", infoschema.ErrColumnNotExists.GenWithStackByArgs(colExpr.Name.Name, tblInfo.Name) } // check duplicated function on the same column @@ -543,6 +543,9 @@ func validateAlterIndexVisibility(ctx sessionctx.Context, indexName pmodel.CIStr return true, nil } } + if idx.VectorInfo != nil { + return false, dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("set vector index invisible") + } return false, nil } diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 40cd21fb998bf..9c746c53a73f5 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -31,6 +31,7 @@ import ( testddlutil "github.com/pingcap/tidb/pkg/ddl/testutil" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" @@ -1117,6 +1118,51 @@ func getJobsBySQL(se sessiontypes.Session, tbl, condition string) ([]*model.Job, return jobs, nil } +func TestCreateTableWithVectorIndex(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + checkCreateTableWithVectorIdx := func(replicaCnt uint64) { + tk.MustExec("create table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW);") + tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, replicaCnt, tbl.Meta().TiFlashReplica.Count) + indexes := tbl.Meta().Indices + require.Equal(t, 1, len(indexes)) + require.Equal(t, pmodel.IndexTypeHNSW, indexes[0].Tp) + require.Equal(t, model.DistanceMetricCosine, indexes[0].VectorInfo.DistanceMetric) + tk.MustExec(`DROP TABLE t`) + } + + // test TiFlash store count is 0 + replicas, err := infoschema.GetTiFlashStoreCount(tk.Session()) + require.NoError(t, err) + require.Equal(t, uint64(0), replicas) + tk.MustContainErrMsg("create table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW);", + "Unsupported add vector index: unsupported TiFlash store count is 0") + + // test TiFlash store count is 2 + mockTiflashStoreCnt := uint64(2) + store, dom = testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(int(mockTiflashStoreCnt)), mockstore.WithDDLChecker()) + tk = testkit.NewTestKit(t, store) + tk.MustExec("use test") + checkCreateTableWithVectorIdx(1) + + // test unsupported table types + tk.MustContainErrMsg("create temporary table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW)", + "`vector index` is unsupported on temporary tables.") + // global and local temporary table using different way to handle, so we have two test cases. + tk.MustContainErrMsg("create global temporary table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW) on commit delete rows;", + "`vector index` is unsupported on temporary tables.") + tk.MustContainErrMsg("create table pt(id bigint, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW) "+ + "partition by range(id) (partition p0 values less than (20), partition p1 values less than (100));", + "Unsupported add vector index: unsupported partition table") + // a vector index with invisible + tk.MustContainErrMsg("create table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW INVISIBLE)", + "Unsupported set vector index invisible") +} + func TestAddVectorIndexSimple(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(2)) tk := testkit.NewTestKit(t, store) @@ -1152,7 +1198,7 @@ func TestAddVectorIndexSimple(t *testing.T) { tk.MustContainErrMsg("alter table t add key idx(a) USING HNSW;", "Only support vector index with HNSW type, but it's non-vector index") // for a wrong column - tk.MustGetErrCode("alter table t add vector index ((vec_cosine_distance(n))) USING HNSW;", errno.ErrBadField) + tk.MustContainErrMsg("alter table t add vector index ((vec_cosine_distance(n))) USING HNSW;", "[schema:1054]Unknown column 'n' in 't'") // for wrong functions tk.MustGetErrCode("alter table t add vector index ((vec_cosine_distance(a))) USING HNSW;", errno.ErrUnsupportedDDLOperation) tk.MustContainErrMsg("alter table t add vector index ((vec_cosine_distance(a,'[1,2.1,3.3]'))) USING HNSW;", @@ -1211,13 +1257,36 @@ func TestAddVectorIndexSimple(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 [1,2.1,3.3]")) tk.MustExec("admin check table t") tk.MustExec("admin check index t idx") + tk.MustContainErrMsg("admin cleanup index t idx", "vector index `idx` is not supported for cleanup index") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + " `a` int(11) DEFAULT NULL,\n" + " `b` vector(3) DEFAULT NULL,\n" + " VECTOR INDEX `idx`((VEC_COSINE_DISTANCE(`b`))) COMMENT 'b comment'\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - tk.MustExec("alter table t drop index idx;") + // test multi-schema change for unsupported operations + tk.MustContainErrMsg("alter table t drop column b;", + "can't drop column b with composite index covered or Primary Key or Vector Key covered now") + tk.MustContainErrMsg("alter table t add index idx2(a), add vector index idx3((vec_l2_distance(b))) USING HNSW COMMENT 'b comment'", + "Unsupported multi schema change for add vector index") + + // test alter index visibility + tk.MustContainErrMsg("alter table t alter index idx invisible", "Unsupported set vector index invisible") + query := "select distinct index_name, is_visible from information_schema.statistics where table_schema = 'test' and table_name = 't' order by index_name" + tk.MustQuery(query).Check(testkit.Rows("idx YES")) + tk.MustExec("alter table t alter index idx visible") + + // test rename index + tk.MustExec("alter table t rename index idx to vecIdx") + tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + indexes1 := tbl.Meta().Indices + require.Equal(t, 1, len(indexes1)) + require.Equal(t, indexes[0].Tp, indexes1[0].Tp) + require.Equal(t, indexes[0].VectorInfo.DistanceMetric, indexes1[0].VectorInfo.DistanceMetric) + + // test drop a vector index + tk.MustExec("alter table t drop index vecIdx;") tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) require.NoError(t, err) indexes = tbl.Meta().Indices @@ -1245,8 +1314,17 @@ func TestAddVectorIndexSimple(t *testing.T) { " VECTOR INDEX `idx`((VEC_COSINE_DISTANCE(`b`))) COMMENT 'b comment'\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // test multi-schema change for dropping indexes + tk.MustExec("alter table t add index idx2(a)") + tk.MustExec("alter table t drop index idx, drop index idx2") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` vector(3) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery("select * from t;").Check(testkit.Rows("1 [1,2.1,3.3]")) + tk.MustExec("admin check table t") + // test anonymous index - tk.MustExec("alter table t drop index idx;") tk.MustExec("alter table t add vector index ((vec_l2_distance(b))) USING HNSW;") tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) require.NoError(t, err) diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 532faa60b5f13..f6d7f2f000098 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -676,6 +676,10 @@ func (b *executorBuilder) buildCleanupIndex(v *plannercore.CleanupIndex) exec.Ex b.err = errors.Errorf("secondary index `%v` is not found in table `%v`", v.IndexName, v.Table.Name.O) return nil } + if index.Meta().VectorInfo != nil { + b.err = errors.Errorf("vector index `%v` is not supported for cleanup index", v.IndexName) + return nil + } e := &CleanupIndexExec{ BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID()), columns: buildIdxColsConcatHandleCols(tblInfo, index.Meta(), false), diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index a1cd13635bb9d..eea8ebba570bb 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -1667,7 +1667,7 @@ func TestPrimaryKeyRequiredSysvar(t *testing.T) { c1 int(11) DEFAULT NULL, PRIMARY KEY(id) NONCLUSTERED)`) tk.MustGetErrMsg(`ALTER TABLE t2 - DROP COLUMN id`, "[ddl:8200]can't drop column id with composite index covered or Primary Key covered now") + DROP COLUMN id`, "[ddl:8200]can't drop column id with composite index covered or Primary Key or Vector Key covered now") tk.MustGetErrCode(`ALTER TABLE t2 DROP PRIMARY KEY`, errno.ErrTableWithoutPrimaryKey) // this sysvar is ignored in internal sessions diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index bf096d9df0497..56faded653ba2 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -182,6 +182,7 @@ var ActionMap = map[ActionType]string{ ActionDropResourceGroup: "drop resource group", ActionAlterTablePartitioning: "alter table partition by", ActionRemovePartitioning: "alter table remove partitioning", + ActionAddVectorIndex: "add vector index", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility.