diff --git a/errors.toml b/errors.toml index 9c02f10d6ac04..993285d92cb43 100644 --- a/errors.toml +++ b/errors.toml @@ -1561,6 +1561,11 @@ error = ''' Global Index is needed for index '%-.192s', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption ''' +["ddl:9014"] +error = ''' +TiFlash backfill index failed: TiFlash backfill index failed: %s +''' + ["domain:8027"] error = ''' Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index ee2e1304ebae5..bbe647a808db8 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -1094,10 +1094,14 @@ 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) + errMsg := "with composite index covered or Primary Key covered now" + if indexInfo.VectorInfo != nil { + errMsg = "with Vector Key covered now" + } + return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s "+errMsg, colName) } } } diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index ee9ec8715ee9a..2207dd6d1c87f 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -509,6 +509,38 @@ 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 { + return nil + } + + 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), + } + } + + return errors.Trace(checkTableTypeForVectorIndex(tblInfo)) +} + // 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 +569,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 +1197,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 +1268,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 +1280,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 +1357,7 @@ func BuildTableInfo( pmodel.NewCIStr(indexName), primary, unique, - false, + vector, constr.Keys, constr.Option, model.StatePublic, diff --git a/pkg/ddl/db_change_test.go b/pkg/ddl/db_change_test.go index 1e9f595210a03..06285d5001ff0 100644 --- a/pkg/ddl/db_change_test.go +++ b/pkg/ddl/db_change_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" @@ -1157,6 +1158,35 @@ func TestParallelAlterAddIndex(t *testing.T) { testControlParallelExecSQL(t, tk, store, dom, "", sql1, sql2, f) } +func TestParallelAlterAddVectorIndex(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database test_db_state default charset utf8 default collate utf8_bin") + tk.MustExec("use test_db_state") + tk.MustExec("create table tt (a int, b vector, c vector(3), d vector(4));") + tk.MustExec("alter table tt set tiflash replica 2 location labels 'a','b';") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/"+ + "ddl/MockCheckVectorIndexProcess")) + }() + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + sql1 := "alter table tt add vector index vecIdx((vec_cosine_distance(c))) USING HNSW;" + sql2 := "alter table tt add vector index vecIdx1((vec_cosine_distance(c))) USING HNSW;" + f := func(err1, err2 error) { + require.NoError(t, err1) + require.EqualError(t, err2, + "[ddl:1061]DDL job rollback, error msg: vector index vecIdx function vec_cosine_distance already exist on column c") + } + testControlParallelExecSQL(t, tk, store, dom, "", sql1, sql2, f) +} + func TestParallelAlterAddExpressionIndex(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index ae397fc44ea9d..b823e662cae34 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 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 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..dd3374d44f0dd 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 @@ -422,7 +422,9 @@ func buildVectorInfoWithCheck(indexPartSpecifications []*ast.IndexPartSpecificat continue } if idx.VectorInfo.DistanceMetric == distanceMetric { - return nil, "", dbterror.ErrDupKeyName.FastGen(fmt.Sprintf("Duplicate vector index function name 'vector index: %s, column name: %s, duplicate function name: %s'", idx.Name, colInfo.Name, f.FnName)) + return nil, "", dbterror.ErrDupKeyName.GenWithStack( + fmt.Sprintf("vector index %s function %s already exist on column %s", + idx.Name, f.FnName, colInfo.Name)) } } if colInfo.FieldType.GetFlen() <= 0 { @@ -543,6 +545,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 } @@ -879,14 +884,17 @@ func (w *worker) checkVectorIndexProcessOnTiFlash(jobCtx *jobContext, t *meta.Me func (w *worker) checkVectorIndexProcess(jobCtx *jobContext, tbl table.Table, job *model.Job, index *model.IndexInfo) error { waitTimeout := ReorgWaitTimeout + ticker := time.NewTicker(waitTimeout) + defer ticker.Stop() + notAddedRowCnt := int64(-1) for { select { case <-w.ddlCtx.ctx.Done(): return dbterror.ErrInvalidWorker.GenWithStack("worker is closed") - case <-time.After(waitTimeout): - logutil.DDLLogger().Info("[ddl] index backfill state running, check vector index process", - zap.Stringer("job", job), zap.Stringer("index name", index.Name), zap.Int64("index ID", index.ID), - zap.Duration("wait time", waitTimeout), zap.Int64("total added row count", job.RowCount)) + case <-ticker.C: + logutil.DDLLogger().Info("[ddl] index backfill state running, check vector index process", zap.Stringer("job", job), + zap.Stringer("index name", index.Name), zap.Int64("index ID", index.ID), zap.Duration("wait time", waitTimeout), + zap.Int64("total added row count", job.RowCount), zap.Int64("not added row count", notAddedRowCnt)) return dbterror.ErrWaitReorgTimeout default: } @@ -897,10 +905,11 @@ func (w *worker) checkVectorIndexProcess(jobCtx *jobContext, tbl table.Table, jo return errors.Trace(dbterror.ErrNotOwner) } - isDone, addedIndexCnt, err := w.checkVectorIndexProcessOnce(jobCtx, tbl, index.ID) + isDone, notAddedIndexCnt, addedIndexCnt, err := w.checkVectorIndexProcessOnce(jobCtx, tbl, index.ID) if err != nil { return errors.Trace(err) } + notAddedRowCnt = notAddedIndexCnt job.RowCount = addedIndexCnt if isDone { @@ -913,36 +922,46 @@ func (w *worker) checkVectorIndexProcess(jobCtx *jobContext, tbl table.Table, jo } // checkVectorIndexProcessOnce checks the backfill process of a vector index from TiFlash once. -func (w *worker) checkVectorIndexProcessOnce(jobCtx *jobContext, tbl table.Table, indexID int64) (bool, int64, error) { +func (w *worker) checkVectorIndexProcessOnce(jobCtx *jobContext, tbl table.Table, indexID int64) ( + isDone bool, notAddedIndexCnt, addedIndexCnt int64, err error) { failpoint.Inject("MockCheckVectorIndexProcess", func(val failpoint.Value) { if valInt, ok := val.(int); ok { - if valInt == -1 { - failpoint.Return(false, 0, nil) + if valInt < 0 { + failpoint.Return(false, 0, 0, dbterror.ErrTiFlashBackfillIndex.FastGenByArgs("mock a check error")) + } else if valInt == 0 { + failpoint.Return(false, 0, 0, nil) } else { - failpoint.Return(true, int64(valInt), nil) + failpoint.Return(true, 0, int64(valInt), nil) } } }) - // TODO: We need to add error_msg for to show error information. - sql := fmt.Sprintf("select rows_stable_not_indexed, rows_stable_indexed from information_schema.tiflash_indexes where table_id = %d and index_id = %d;", + sql := fmt.Sprintf("select rows_stable_not_indexed, rows_stable_indexed, error_message from information_schema.tiflash_indexes where table_id = %d and index_id = %d;", tbl.Meta().ID, indexID) rows, err := w.sess.Execute(jobCtx.ctx, sql, "add_vector_index_check_result") if err != nil || len(rows) == 0 { - return false, 0, errors.Trace(err) + return false, 0, 0, errors.Trace(err) } - // handle info from multiple TiFlash nodes - notAddedIndexCnt, addedIndexCnt := int64(0), int64(0) + // Get and process info from multiple TiFlash nodes. + errMsg := "" for _, row := range rows { notAddedIndexCnt += row.GetInt64(0) addedIndexCnt += row.GetInt64(1) + errMsg = row.GetString(2) + if len(errMsg) != 0 { + err = dbterror.ErrTiFlashBackfillIndex.FastGenByArgs(errMsg) + break + } + } + if err != nil { + return false, 0, 0, errors.Trace(err) } if notAddedIndexCnt != 0 { - return false, 0, nil + return false, 0, 0, nil } - return true, rows[0].GetInt64(1), nil + return true, notAddedIndexCnt, addedIndexCnt, nil } func (w *worker) onCreateIndex(jobCtx *jobContext, t *meta.Meta, job *model.Job, isPK bool) (ver int64, err error) { diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 40cd21fb998bf..63b2861efa8e3 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;", @@ -1181,7 +1227,7 @@ func TestAddVectorIndexSimple(t *testing.T) { require.Truef(t, terror.ErrorEqual(dbterror.ErrDupKeyName, lastWarn.Err), "err %v", lastWarn.Err) require.Equal(t, contextutil.WarnLevelNote, lastWarn.Level) tk.MustContainErrMsg("alter table t add vector index if not exists idx((vec_cosine_distance(c))) USING HNSW;", - "Duplicate vector index function name 'vector index: vecIdx, column name: c, duplicate function name: vec_cosine_distance'") + "[ddl:1061]vector index vecIdx function vec_cosine_distance already exist on column c") // normal test cases tk.MustExec("drop table if exists t;") @@ -1211,13 +1257,40 @@ 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 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 modify/change column with a vector index + tk.MustContainErrMsg("alter table t modify column b vector(2)", "[ddl:8200]Unsupported modify column: vector indexes on the column") + tk.MustExec("alter table t modify column b vector(3) not null") + + // 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 @@ -1227,7 +1300,7 @@ func TestAddVectorIndexSimple(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 [1,2.1,3.3]")) 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" + + " `b` vector(3) NOT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // test create a vector index with same name @@ -1241,12 +1314,21 @@ func TestAddVectorIndexSimple(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 [1,2.1,3.3]")) 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" + + " `b` vector(3) NOT NULL,\n" + " 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) NOT 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) @@ -1318,7 +1400,7 @@ func TestAddVectorIndexRollback(t *testing.T) { var checkErr error tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") - testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(0)`) onJobUpdatedExportedFunc := func(job *model.Job) { if checkErr != nil { return @@ -1341,10 +1423,16 @@ func TestAddVectorIndexRollback(t *testing.T) { tk.MustQuery("select count(1) from t1;").Check(testkit.Rows("4")) checkRollbackInfo(model.JobStateRollbackDone) - // Case3: add a vector index normally. + // Case3: test get error message from tiflash testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/onJobUpdated") + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(-1)`) + tk.MustContainErrMsg(addIdxSQL, "[ddl:9014]TiFlash backfill index failed: TiFlash backfill index failed: mock a check error") + checkRollbackInfo(model.JobStateRollbackDone) + + // Case4: add a vector index normally. testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(4)`) tk.MustExec(addIdxSQL) + checkRollbackInfo(model.JobStateSynced) // TODO: add mock TiFlash to make sure the vector index count is equal to row count. // tk.MustQuery("select count(1) from t1 use index(v_idx);").Check(testkit.Rows("4")) diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go index 83a69171115cb..2c9d2f363e197 100644 --- a/pkg/ddl/modify_column.go +++ b/pkg/ddl/modify_column.go @@ -62,6 +62,16 @@ type modifyingColInfo struct { removedIdxs []int64 } +func hasVectorIndexColumn(tblInfo *model.TableInfo, col *model.ColumnInfo) bool { + indexesToChange := FindRelatedIndexesToChange(tblInfo, col.Name) + for _, idx := range indexesToChange { + if idx.IndexInfo.VectorInfo != nil { + return true + } + } + return false +} + func (w *worker) onModifyColumn(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) { dbInfo, tblInfo, oldCol, modifyInfo, err := getModifyColumnInfo(t, job) if err != nil { @@ -113,6 +123,9 @@ func (w *worker) onModifyColumn(jobCtx *jobContext, t *meta.Meta, job *model.Job job.State = model.JobStateCancelled return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table")) } + if hasVectorIndexColumn(tblInfo, oldCol) { + return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("vector indexes on the column")) + } changingCol := modifyInfo.changingCol if changingCol == nil { @@ -762,6 +775,9 @@ func GetModifiableColumnJob( if t.Meta().Partition != nil { return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") } + if hasVectorIndexColumn(t.Meta(), col.ColumnInfo) { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("vector indexes on the column") + } } // Check that the column change does not affect the partitioning column diff --git a/pkg/errno/errcode.go b/pkg/errno/errcode.go index 5d56f339273ff..94dd6f3239640 100644 --- a/pkg/errno/errcode.go +++ b/pkg/errno/errcode.go @@ -1167,4 +1167,5 @@ const ( ErrTiKVMaxTimestampNotSynced = 9011 ErrTiFlashServerTimeout = 9012 ErrTiFlashServerBusy = 9013 + ErrTiFlashBackfillIndex = 9014 ) diff --git a/pkg/errno/errname.go b/pkg/errno/errname.go index aadcbb3dfcbca..ac1eb04c54be6 100644 --- a/pkg/errno/errname.go +++ b/pkg/errno/errname.go @@ -1158,6 +1158,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrTiKVServerBusy: mysql.Message("TiKV server is busy", nil), ErrTiFlashServerTimeout: mysql.Message("TiFlash server timeout", nil), ErrTiFlashServerBusy: mysql.Message("TiFlash server is busy", nil), + ErrTiFlashBackfillIndex: mysql.Message("TiFlash backfill index failed: %s", nil), ErrResolveLockTimeout: mysql.Message("Resolve lock timeout", nil), ErrRegionUnavailable: mysql.Message("Region is unavailable", nil), ErrGCTooEarly: mysql.Message("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", nil), 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/executor/infoschema_reader_test.go b/pkg/executor/infoschema_reader_test.go index 5a83d311a2960..d08bca5b35ec3 100644 --- a/pkg/executor/infoschema_reader_test.go +++ b/pkg/executor/infoschema_reader_test.go @@ -605,7 +605,7 @@ func TestColumnTable(t *testing.T) { testkit.RowsWithSep("|", "test|tbl1|col_2")) tk.MustQuery(`select count(*) from information_schema.columns;`).Check( - testkit.RowsWithSep("|", "4949")) + testkit.RowsWithSep("|", "4950")) } func TestIndexUsageTable(t *testing.T) { diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index 98388aeb16ec5..7fd63574c09b4 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -1525,6 +1525,7 @@ var tableTiFlashIndexesCols = []columnInfo{ {name: "ROWS_STABLE_NOT_INDEXED", tp: mysql.TypeLonglong, size: 64}, {name: "ROWS_DELTA_INDEXED", tp: mysql.TypeLonglong, size: 64}, {name: "ROWS_DELTA_NOT_INDEXED", tp: mysql.TypeLonglong, size: 64}, + {name: "ERROR_MESSAGE", tp: mysql.TypeVarchar, size: 1024}, {name: "TIFLASH_INSTANCE", tp: mysql.TypeVarchar, size: 64}, } diff --git a/pkg/meta/model/bdr.go b/pkg/meta/model/bdr.go index ada92039b9168..48f31d1f45e97 100644 --- a/pkg/meta/model/bdr.go +++ b/pkg/meta/model/bdr.go @@ -101,6 +101,7 @@ var BDRActionMap = map[DDLBDRType][]ActionType{ ActionReorganizePartition, ActionAlterTablePartitioning, ActionRemovePartitioning, + ActionAddVectorIndex, }, UnmanagementDDL: { ActionCreatePlacementPolicy, diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index bf096d9df0497..8605d6aa5115e 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. @@ -748,7 +749,7 @@ func (job *Job) IsPausing() bool { // IsPausable checks whether we can pause the job. func (job *Job) IsPausable() bool { - // TODO: We can remove it after TiFlash support pause operation. + // TODO: We can remove it after TiFlash supports the pause operation. if job.Type == ActionAddVectorIndex && job.SchemaState == StateWriteReorganization { return false } diff --git a/pkg/planner/core/indexmerge_path_test.go b/pkg/planner/core/indexmerge_path_test.go index fcc75e5944326..971456dcfd28f 100644 --- a/pkg/planner/core/indexmerge_path_test.go +++ b/pkg/planner/core/indexmerge_path_test.go @@ -461,7 +461,7 @@ func TestAnalyzeVectorIndex(t *testing.T) { err = domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tblInfo.Meta().ID, true) require.NoError(t, err) - testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(0)`) + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) tk.MustExec("alter table t add vector index idx((VEC_COSINE_DISTANCE(b))) USING HNSW") tk.MustExec("alter table t add vector index idx2((VEC_COSINE_DISTANCE(c))) USING HNSW") diff --git a/pkg/util/dbterror/ddl_terror.go b/pkg/util/dbterror/ddl_terror.go index 9c2ecf1940a1d..a959807391633 100644 --- a/pkg/util/dbterror/ddl_terror.go +++ b/pkg/util/dbterror/ddl_terror.go @@ -420,12 +420,13 @@ var ( // ErrAlterTiFlashModeForTableWithoutTiFlashReplica returns when set tiflash mode on table whose tiflash_replica is null or tiflash_replica_count = 0 ErrAlterTiFlashModeForTableWithoutTiFlashReplica = ClassDDL.NewStdErr(0, parser_mysql.Message("TiFlash mode will take effect after at least one TiFlash replica is set for the table", nil)) - // ErrUnsupportedTiFlashOperationForSysOrMemTable means we don't support the alter tiflash related action(e.g. set tiflash mode, set tiflash replica) for system table. ErrUnsupportedTiFlashOperationForSysOrMemTable = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "ALTER TiFlash settings for system table and memory table"), nil)) - // ErrUnsupportedTiFlashOperationForUnsupportedCharsetTable is used when alter alter tiflash related action(e.g. set tiflash mode, set tiflash replica) with unsupported charset. ErrUnsupportedTiFlashOperationForUnsupportedCharsetTable = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "ALTER TiFlash settings for tables not supported by TiFlash: table contains %s charset"), nil)) + // ErrTiFlashBackfillIndex is the error that tiflash backfill the index failed. + ErrTiFlashBackfillIndex = ClassDDL.NewStdErr(mysql.ErrTiFlashBackfillIndex, + parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrTiFlashBackfillIndex].Raw, "TiFlash backfill index failed: %s"), nil)) // ErrDropIndexNeededInForeignKey returns when drop index which is needed in foreign key. ErrDropIndexNeededInForeignKey = ClassDDL.NewStd(mysql.ErrDropIndexNeededInForeignKey)