Skip to content

Commit

Permalink
*: support for adding/dropping vector index (#55839)
Browse files Browse the repository at this point in the history
close #55840
  • Loading branch information
zimulala authored Sep 14, 2024
1 parent ee3a596 commit 384d4ed
Show file tree
Hide file tree
Showing 32 changed files with 1,211 additions and 218 deletions.
3 changes: 3 additions & 0 deletions pkg/ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,7 @@ go_library(
"//pkg/util/execdetails",
"//pkg/util/filter",
"//pkg/util/gcutil",
"//pkg/util/generatedexpr",
"//pkg/util/generic",
"//pkg/util/hack",
"//pkg/util/intest",
Expand Down Expand Up @@ -319,6 +320,7 @@ go_test(
"//pkg/store/gcworker",
"//pkg/store/helper",
"//pkg/store/mockstore",
"//pkg/store/mockstore/unistore",
"//pkg/table",
"//pkg/table/tables",
"//pkg/tablecodec",
Expand Down Expand Up @@ -347,6 +349,7 @@ go_test(
"@com_github_ngaut_pools//:pools",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//oracle",
Expand Down
26 changes: 23 additions & 3 deletions pkg/ddl/cancel_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/ddl"
"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/meta/model"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -68,6 +69,15 @@ var allTestCase = []testCancelJob{
{"alter table t add index idx_c2(c2)", true, model.StateDeleteOnly, true, true, nil},
{"alter table t add index idx_c2(c2)", true, model.StateWriteOnly, true, true, nil},
{"alter table t add index idx_cx2(c2)", false, model.StatePublic, false, true, nil},
// Drop vector index
{"alter table t drop index v_idx_1", true, model.StatePublic, true, false, []string{"alter table t add vector index v_idx_1((VEC_L2_DISTANCE(v2))) USING HNSW"}},
{"alter table t drop index v_idx_2", false, model.StateWriteOnly, true, false, []string{"alter table t add vector index v_idx_2((VEC_COSINE_DISTANCE(v2))) USING HNSW"}},
{"alter table t drop index v_idx_3", false, model.StateDeleteOnly, false, true, []string{"alter table t add vector index v_idx_3((VEC_COSINE_DISTANCE(v2))) USING HNSW"}},
{"alter table t drop index v_idx_4", false, model.StateDeleteReorganization, false, true, []string{"alter table t add vector index v_idx_4((VEC_COSINE_DISTANCE(v2))) USING HNSW"}},
// Add vector key
{"alter table t add vector index v_idx((VEC_COSINE_DISTANCE(v2))) USING HNSW", true, model.StateNone, true, false, nil},
{"alter table t add vector index v_idx((VEC_COSINE_DISTANCE(v2))) USING HNSW", true, model.StateDeleteOnly, true, true, nil},
{"alter table t add vector index v_idx((VEC_COSINE_DISTANCE(v2))) USING HNSW", true, model.StateWriteOnly, true, true, nil},
// Add column.
{"alter table t add column c4 bigint", true, model.StateNone, true, false, nil},
{"alter table t add column c4 bigint", true, model.StateDeleteOnly, true, true, nil},
Expand Down Expand Up @@ -204,7 +214,7 @@ func cancelSuccess(rs *testkit.Result) bool {
return strings.Contains(rs.Rows()[0][1].(string), "success")
}

func TestCancel(t *testing.T) {
func TestCancelVariousJobs(t *testing.T) {
var enterCnt, exitCnt atomic.Int32
testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/beforeDeliveryJob", func(job *model.Job) { enterCnt.Add(1) })
testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/afterDeliveryJob", func(job *model.Job) { exitCnt.Add(1) })
Expand All @@ -213,10 +223,18 @@ func TestCancel(t *testing.T) {
return enterCnt.Load() == exitCnt.Load()
}, 10*time.Second, 10*time.Millisecond)
}
store := testkit.CreateMockStoreWithSchemaLease(t, 100*time.Millisecond)
store := testkit.CreateMockStoreWithSchemaLease(t, 100*time.Millisecond, withMockTiFlash(2))
tk := testkit.NewTestKit(t, store)
tkCancel := testkit.NewTestKit(t, store)

tiflash := infosync.NewMockTiFlash()
infosync.SetMockTiFlash(tiflash)
defer func() {
tiflash.Lock()
tiflash.StatusServer.Close()
tiflash.Unlock()
}()

// Prepare schema.
tk.MustExec("use test")
tk.MustExec("drop table if exists t_partition;")
Expand All @@ -231,14 +249,16 @@ func TestCancel(t *testing.T) {
partition p4 values less than (7096)
);`)
tk.MustExec(`create table t (
c1 int, c2 int, c3 int, c11 tinyint, index fk_c1(c1)
c1 int, c2 int, c3 int, c11 tinyint, v2 vector(3), index fk_c1(c1)
);`)
tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';")

// Prepare data.
for i := 0; i <= 2048; i++ {
tk.MustExec(fmt.Sprintf("insert into t_partition values(%d, %d, %d)", i*3, i*2, i))
tk.MustExec(fmt.Sprintf("insert into t(c1, c2, c3) values(%d, %d, %d)", i*3, i*2, i))
}
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(2048)`)

// Change some configurations.
ddl.ReorgWaitTimeout = 10 * time.Millisecond
Expand Down
5 changes: 3 additions & 2 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1313,10 +1313,11 @@ func BuildTableInfo(
// build index info.
idxInfo, err := BuildIndexInfo(
ctx,
tbInfo.Columns,
tbInfo,
pmodel.NewCIStr(indexName),
primary,
unique,
false,
constr.Keys,
constr.Option,
model.StatePublic,
Expand Down Expand Up @@ -1480,7 +1481,7 @@ func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) erro
Length: types.UnspecifiedLength,
})
}
idxInfo, err := BuildIndexInfo(ctx, tbInfo.Columns, idxName, false, false, keys, nil, model.StatePublic)
idxInfo, err := BuildIndexInfo(ctx, tbInfo, idxName, false, false, false, keys, nil, model.StatePublic)
if err != nil {
return errors.Trace(err)
}
Expand Down
11 changes: 3 additions & 8 deletions pkg/ddl/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -376,14 +376,9 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, wrapper DelRangeExecWrap
}
case model.ActionDropIndex, model.ActionDropPrimaryKey:
tableID := job.TableID
var indexName any
var partitionIDs []int64
ifExists := make([]bool, 1)
allIndexIDs := make([]int64, 1)
if err := job.DecodeArgs(&indexName, &ifExists[0], &allIndexIDs[0], &partitionIDs); err != nil {
if err = job.DecodeArgs(&indexName, &ifExists, &allIndexIDs, &partitionIDs); err != nil {
return errors.Trace(err)
}
_, _, allIndexIDs, partitionIDs, _, err := job.DecodeDropIndexFinishedArgs()
if err != nil {
return errors.Trace(err)
}
// partitionIDs len is 0 if the dropped index is a global index, even if it is a partitioned table.
if len(partitionIDs) == 0 {
Expand Down
Loading

0 comments on commit 384d4ed

Please sign in to comment.