Skip to content

Commit

Permalink
*: support create table with vector indexes and others
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Sep 18, 2024
1 parent 384d4ed commit 12b1333
Show file tree
Hide file tree
Showing 8 changed files with 151 additions and 21 deletions.
4 changes: 2 additions & 2 deletions pkg/ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
}
Expand Down
66 changes: 54 additions & 12 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -1233,18 +1270,23 @@ 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
unique = true
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
Expand Down Expand Up @@ -1317,7 +1359,7 @@ func BuildTableInfo(
pmodel.NewCIStr(indexName),
primary,
unique,
false,
vector,
constr.Keys,
constr.Option,
model.StatePublic,
Expand Down
6 changes: 4 additions & 2 deletions pkg/ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"))
}

Expand Down
5 changes: 4 additions & 1 deletion pkg/ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
}

Expand Down
84 changes: 81 additions & 3 deletions pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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;",
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down
4 changes: 4 additions & 0 deletions pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/integration_test/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions pkg/meta/model/job.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down

0 comments on commit 12b1333

Please sign in to comment.