Skip to content

Commit

Permalink
*: address comments
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Sep 10, 2024
1 parent ffc8f35 commit 8ac0891
Show file tree
Hide file tree
Showing 7 changed files with 27 additions and 91 deletions.
6 changes: 3 additions & 3 deletions pkg/ddl/cancel_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func TestCancelVariousJobs(t *testing.T) {
return enterCnt.Load() == exitCnt.Load()
}, 10*time.Second, 10*time.Millisecond)
}
store, _ := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 100*time.Millisecond, withMockTiFlash(2))
store := testkit.CreateMockStoreWithSchemaLease(t, 100*time.Millisecond, withMockTiFlash(2))
tk := testkit.NewTestKit(t, store)
tkCancel := testkit.NewTestKit(t, store)

Expand All @@ -248,9 +248,9 @@ func TestCancelVariousJobs(t *testing.T) {
partition p3 values less than (4096),
partition p4 values less than (7096)
);`)
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
testfailpoint.Enable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
testfailpoint.Disable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
}()
tk.MustExec(`create table t (
c1 int, c2 int, c3 int, c11 tinyint, v2 vector(3), index fk_c1(c1)
Expand Down
49 changes: 7 additions & 42 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -4558,7 +4558,7 @@ func checkIndexNameAndColumns(ctx sessionctx.Context, t table.Table, indexName m
"a background job is trying to add the same index, "+
"please check by `ADMIN SHOW DDL JOBS`", indexName)
} else {
err = dbterror.ErrDupKeyName.GenWithStack("index already exist %s", indexName)
err = dbterror.ErrDupKeyName.GenWithStackByArgs(indexName)
}
if ifNotExists {
ctx.GetSessionVars().StmtCtx.AppendNote(err)
Expand Down Expand Up @@ -4596,6 +4596,9 @@ func checkTableTypeForVectorIndex(tblInfo *model.TableInfo) error {
if tblInfo.GetPartitionInfo() != nil {
return dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported partition table")
}
if tblInfo.TiFlashReplica == nil || tblInfo.TiFlashReplica.Count == 0 {
return dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported empty TiFlash replica, the replica is nil")
}

return nil
}
Expand All @@ -4612,15 +4615,11 @@ func (e *executor) createVectorIndex(ctx sessionctx.Context, ti ast.Ident, index
return errors.Trace(err)
}

if tblInfo.TiFlashReplica == nil || tblInfo.TiFlashReplica.Count == 0 {
return dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported empty TiFlash replica, the replica is nil")
}

indexName, _, err = checkIndexNameAndColumns(ctx, t, indexName, indexPartSpecifications, true, ifNotExists)
if err != nil {
return errors.Trace(err)
}
_, funcExpr, err := buildVectorInfoWithCheck(indexPartSpecifications, indexOption, indexName, tblInfo)
_, funcExpr, err := buildVectorInfoWithCheck(indexPartSpecifications, indexOption, tblInfo)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -4734,46 +4733,12 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast
if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable {
return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Create Index"))
}
// Deal with anonymous index.
if len(indexName.L) == 0 {
colName := model.NewCIStr("expression_index")
if indexPartSpecifications[0].Column != nil {
colName = indexPartSpecifications[0].Column.Name
}
indexName = GetName4AnonymousIndex(t, colName, model.NewCIStr(""))
}

if indexInfo := t.Meta().FindIndexByName(indexName.L); indexInfo != nil {
if indexInfo.State != model.StatePublic {
// NOTE: explicit error message. See issue #18363.
err = dbterror.ErrDupKeyName.GenWithStack("Duplicate key name '%s'; "+
"a background job is trying to add the same index, "+
"please check by `ADMIN SHOW DDL JOBS`", indexName)
} else {
err = dbterror.ErrDupKeyName.GenWithStackByArgs(indexName)
}
if ifNotExists {
ctx.GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
}

if err = checkTooLongIndex(indexName); err != nil {
return errors.Trace(err)
}

tblInfo := t.Meta()

// Build hidden columns if necessary.
hiddenCols, err := buildHiddenColumnInfoWithCheck(ctx, indexPartSpecifications, indexName, t.Meta(), t.Cols())
indexName, hiddenCols, err := checkIndexNameAndColumns(ctx, t, indexName, indexPartSpecifications, false, ifNotExists)
if err != nil {
return err
}
if err = checkAddColumnTooManyColumns(len(t.Cols()) + len(hiddenCols)); err != nil {
return errors.Trace(err)
}

tblInfo := t.Meta()
finalColumns := make([]*model.ColumnInfo, len(tblInfo.Columns), len(tblInfo.Columns)+len(hiddenCols))
copy(finalColumns, tblInfo.Columns)
finalColumns = append(finalColumns, hiddenCols...)
Expand Down
19 changes: 2 additions & 17 deletions pkg/ddl/generated_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,13 +276,11 @@ func checkModifyGeneratedColumn(sctx sessionctx.Context, schemaName model.CIStr,
}

type illegalFunctionChecker struct {
genType int
hasIllegalFunc bool
hasAggFunc bool
hasRowVal bool // hasRowVal checks whether the functional index refers to a row value
hasWindowFunc bool
hasNotGAFunc4ExprIdx bool
hasFunc4VectorIdx bool
hasCastArrayFunc bool
disallowCastArrayFunc bool
otherErr error
Expand All @@ -302,17 +300,6 @@ func (c *illegalFunctionChecker) Enter(inNode ast.Node) (outNode ast.Node, skipC
c.hasIllegalFunc = true
return inNode, true
}
if c.genType == typeVectorIndex {
_, isFunc4Vec := variable.DistanceMetric4VectorIndex[node.FnName.L]
if isFunc4Vec {
c.hasFunc4VectorIdx = true
if len(node.Args) != 1 {
c.otherErr = expression.ErrIncorrectParameterCount.GenWithStackByArgs(node.FnName)
return inNode, true
}
}
break
}
err := expression.VerifyArgsWrapper(node.FnName.L, len(node.Args))
if err != nil {
c.otherErr = err
Expand Down Expand Up @@ -356,21 +343,19 @@ func (*illegalFunctionChecker) Leave(inNode ast.Node) (node ast.Node, ok bool) {
const (
typeColumn = iota
typeIndex
typeVectorIndex
)

func checkIllegalFn4Generated(name string, genType int, expr ast.ExprNode) error {
if expr == nil {
return nil
}
var c illegalFunctionChecker
c.genType = genType
expr.Accept(&c)
if c.hasIllegalFunc {
switch genType {
case typeColumn:
return dbterror.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs(name)
case typeIndex, typeVectorIndex:
case typeIndex:
return dbterror.ErrFunctionalIndexFunctionIsNotAllowed.GenWithStackByArgs(name)
}
}
Expand All @@ -381,7 +366,7 @@ func checkIllegalFn4Generated(name string, genType int, expr ast.ExprNode) error
switch genType {
case typeColumn:
return dbterror.ErrGeneratedColumnRowValueIsNotAllowed.GenWithStackByArgs(name)
case typeIndex, typeVectorIndex:
case typeIndex:
return dbterror.ErrFunctionalIndexRowValueIsNotAllowed.GenWithStackByArgs(name)
}
}
Expand Down
20 changes: 3 additions & 17 deletions pkg/ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -351,7 +351,7 @@ func BuildIndexInfo(
}

if isVector {
vectorInfo, _, err := buildVectorInfoWithCheck(indexPartSpecifications, indexOption, indexName, tblInfo)
vectorInfo, _, err := buildVectorInfoWithCheck(indexPartSpecifications, indexOption, tblInfo)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -388,21 +388,16 @@ func BuildIndexInfo(
}

func buildVectorInfoWithCheck(indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption,
indexName model.CIStr, tblInfo *model.TableInfo) (*model.VectorIndexInfo, string, error) {
tblInfo *model.TableInfo) (*model.VectorIndexInfo, string, error) {
if len(indexPartSpecifications) != 1 {
return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported no function")
}

idxPart := indexPartSpecifications[0]
if err := checkIllegalFn4Generated(indexName.L, typeVectorIndex, idxPart.Expr); err != nil {
return nil, "", errors.Trace(err)
}

f, ok := idxPart.Expr.(*ast.FuncCallExpr)
if !ok {
return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs(fmt.Sprintf("unsupported function: %v", idxPart.Expr))
}

distanceMetric, ok := variable.DistanceMetric4VectorIndex[f.FnName.L]
if !ok {
return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported function")
Expand Down Expand Up @@ -681,7 +676,6 @@ func checkAndBuildIndexInfo(job *model.Job, tblInfo *model.TableInfo, indexName
indexInfo := tblInfo.FindIndexByName(indexName.L)
if indexInfo != nil {
if indexInfo.State == model.StatePublic {
job.State = model.JobStateCancelled
err = dbterror.ErrDupKeyName.GenWithStack("index already exist %s", indexName)
if isPK {
err = infoschema.ErrMultiplePriKey
Expand All @@ -695,7 +689,6 @@ func checkAndBuildIndexInfo(job *model.Job, tblInfo *model.TableInfo, indexName
columnInfo := model.FindColumnInfo(tblInfo.Columns, hiddenCol.Name.L)
if columnInfo != nil && columnInfo.State == model.StatePublic {
// We already have a column with the same column name.
job.State = model.JobStateCancelled
// TODO: refine the error message
return nil, infoschema.ErrColumnExists.GenWithStackByArgs(hiddenCol.Name)
}
Expand All @@ -707,7 +700,6 @@ func checkAndBuildIndexInfo(job *model.Job, tblInfo *model.TableInfo, indexName
}
}
if err = checkAddColumnTooManyColumns(len(tblInfo.Columns)); err != nil {
job.State = model.JobStateCancelled
return nil, errors.Trace(err)
}
indexInfo, err = BuildIndexInfo(
Expand All @@ -722,26 +714,22 @@ func checkAndBuildIndexInfo(job *model.Job, tblInfo *model.TableInfo, indexName
model.StateNone,
)
if err != nil {
job.State = model.JobStateCancelled
return nil, errors.Trace(err)
}
if isPK {
if _, err = CheckPKOnGeneratedColumn(tblInfo, indexPartSpecifications); err != nil {
job.State = model.JobStateCancelled
return nil, err
}
}
indexInfo.ID = AllocateIndexID(tblInfo)
tblInfo.Indices = append(tblInfo.Indices, indexInfo)
if err = checkTooManyIndexes(tblInfo.Indices); err != nil {
job.State = model.JobStateCancelled
return nil, errors.Trace(err)
}
// Here we need do this check before set state to `DeleteOnly`,
// because if hidden columns has been set to `DeleteOnly`,
// the `DeleteOnly` columns are missing when we do this check.
if err := checkInvisibleIndexOnPK(tblInfo); err != nil {
job.State = model.JobStateCancelled
return nil, err
}
logutil.DDLLogger().Info("[ddl] run add index job", zap.String("job", job.String()), zap.Reflect("indexInfo", indexInfo))
Expand All @@ -767,9 +755,6 @@ func (w *worker) onCreateVectorIndex(jobCtx *jobContext, t *meta.Meta, job *mode
if err := checkTableTypeForVectorIndex(tblInfo); err != nil {
return ver, errors.Trace(err)
}
if tblInfo.TiFlashReplica == nil || tblInfo.TiFlashReplica.Count == 0 {
return ver, dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported empty TiFlash replica, the replica is nil")
}

var (
indexName model.CIStr
Expand Down Expand Up @@ -976,6 +961,7 @@ func (w *worker) onCreateIndex(jobCtx *jobContext, t *meta.Meta, job *model.Job,
for i, indexName := range indexNames {
indexInfo, err := checkAndBuildIndexInfo(job, tblInfo, indexName, isPK, uniques[i], false, indexPartSpecifications[i], indexOption[i], hiddenCols[i])
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
allIndexInfos = append(allIndexInfos, indexInfo)
Expand Down
13 changes: 6 additions & 7 deletions pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/pkg/ddl"
testddlutil "github.com/pingcap/tidb/pkg/ddl/testutil"
Expand Down Expand Up @@ -1122,9 +1121,9 @@ func TestAddVectorIndex(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t, pt;")

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
testfailpoint.Enable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
testfailpoint.Disable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
}()
tiflash := infosync.NewMockTiFlash()
infosync.SetMockTiFlash(tiflash)
Expand Down Expand Up @@ -1158,7 +1157,8 @@ func TestAddVectorIndex(t *testing.T) {
tk.MustGetErrCode("alter table t add vector index ((vec_cosine_distance(n))) USING HNSW;", errno.ErrBadField)
// for wrong functions
tk.MustGetErrCode("alter table t add vector index ((vec_cosine_distance(a))) USING HNSW;", errno.ErrUnsupportedDDLOperation)
tk.MustGetErrCode("alter table t add vector index ((vec_cosine_distance(a,'[1,2.1,3.3]'))) USING HNSW;", errno.ErrWrongParamcountToNativeFct)
tk.MustContainErrMsg("alter table t add vector index ((vec_cosine_distance(a,'[1,2.1,3.3]'))) USING HNSW;",
"Unsupported add vector index: only support vector type, but this is type: int(11)")
tk.MustGetErrCode("alter table t add vector index ((vec_l1_distance(b))) USING HNSW;", errno.ErrUnsupportedDDLOperation)
tk.MustGetErrCode("alter table t add vector index ((vec_negative_inner_product(b))) USING HNSW;", errno.ErrUnsupportedDDLOperation)
tk.MustGetErrCode("alter table t add vector index ((lower(b))) USING HNSW;", errno.ErrUnsupportedDDLOperation)
Expand Down Expand Up @@ -1248,9 +1248,9 @@ func TestAddVectorIndexRollback(t *testing.T) {
}()

// mock TiFlash replicas
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
testfailpoint.Enable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
testfailpoint.Disable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
}()
tk.MustExec("create table t1 (c1 int, b vector, c vector(3), unique key(c1));")
tk.MustExec("alter table t1 set tiflash replica 2 location labels 'a','b';")
Expand Down Expand Up @@ -1292,7 +1292,6 @@ 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(1)`)
onJobUpdatedExportedFunc := func(job *model.Job) {
if checkErr != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -303,6 +303,7 @@ go_test(
"//pkg/testkit/ddlhelper",
"//pkg/testkit/external",
"//pkg/testkit/testdata",
"//pkg/testkit/testfailpoint",
"//pkg/testkit/testmain",
"//pkg/testkit/testsetup",
"//pkg/testkit/testutil",
Expand Down
10 changes: 5 additions & 5 deletions pkg/planner/core/indexmerge_path_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/expression"
Expand All @@ -35,6 +34,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/util/coretestsdk"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util/hint"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -444,9 +444,9 @@ func TestAnalyzeVectorIndex(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t;")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
testfailpoint.Enable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount"))
testfailpoint.Disable(t, "github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
}()

tiflash := infosync.NewMockTiFlash()
Expand All @@ -463,9 +463,9 @@ func TestAnalyzeVectorIndex(t *testing.T) {
err = domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tblInfo.Meta().ID, true)
require.NoError(t, err)

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(0)`))
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(0)`)
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess"))
testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess")
}()
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")
Expand Down

0 comments on commit 8ac0891

Please sign in to comment.