From add4469a7bd8c81d63ea5c10f84aa6cf7c34edac Mon Sep 17 00:00:00 2001 From: Yiding Date: Tue, 8 Oct 2024 15:58:56 +0800 Subject: [PATCH 01/20] planner: support vector index by CBO --- pkg/ddl/BUILD.bazel | 2 - pkg/ddl/cancel_test.go | 3 +- pkg/ddl/db_change_test.go | 3 +- pkg/ddl/index.go | 4 +- pkg/ddl/index_modify_test.go | 29 +- pkg/executor/show.go | 2 +- pkg/expression/BUILD.bazel | 1 + pkg/expression/vs_helper.go | 98 ++++ pkg/meta/model/index.go | 22 +- pkg/planner/core/BUILD.bazel | 2 +- pkg/planner/core/access_object.go | 13 + .../core/casetest/enforcempp/BUILD.bazel | 1 + .../casetest/enforcempp/enforce_mpp_test.go | 9 +- pkg/planner/core/casetest/hint/BUILD.bazel | 1 + pkg/planner/core/casetest/hint/hint_test.go | 5 +- pkg/planner/core/casetest/index/BUILD.bazel | 8 +- pkg/planner/core/casetest/index/index_test.go | 93 ++++ .../casetest/physicalplantest/BUILD.bazel | 1 + .../physicalplantest/physical_plan_test.go | 17 +- .../core/casetest/vectorsearch/BUILD.bazel | 26 + .../core/casetest/vectorsearch/main_test.go | 59 +++ .../testdata/ann_index_suite_in.json | 48 ++ .../testdata/ann_index_suite_out.json | 482 ++++++++++++++++++ .../vectorsearch/vector_index_test.go | 99 ++++ pkg/planner/core/explain.go | 18 + pkg/planner/core/find_best_task.go | 6 + pkg/planner/core/indexmerge_path_test.go | 3 - pkg/planner/core/physical_plan_test.go | 8 +- pkg/planner/core/physical_plans.go | 13 + pkg/planner/core/plan_cost_ver2.go | 81 +-- pkg/planner/core/plan_to_pb.go | 4 + pkg/planner/core/planbuilder.go | 13 +- pkg/planner/core/task.go | 63 +++ pkg/planner/util/coretestsdk/BUILD.bazel | 4 - pkg/planner/util/coretestsdk/testkit.go | 25 - pkg/planner/util/path.go | 9 +- pkg/sessionctx/variable/varsutil.go | 13 - pkg/store/mockstore/BUILD.bazel | 1 + pkg/store/mockstore/mockstore.go | 22 + pkg/testkit/testkit.go | 11 + 40 files changed, 1186 insertions(+), 136 deletions(-) create mode 100644 pkg/expression/vs_helper.go create mode 100644 pkg/planner/core/casetest/vectorsearch/BUILD.bazel create mode 100644 pkg/planner/core/casetest/vectorsearch/main_test.go create mode 100644 pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json create mode 100644 pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json create mode 100644 pkg/planner/core/casetest/vectorsearch/vector_index_test.go diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 04dc65cff812d..6a9118fe09e7e 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -325,7 +325,6 @@ go_test( "//pkg/store/gcworker", "//pkg/store/helper", "//pkg/store/mockstore", - "//pkg/store/mockstore/unistore", "//pkg/table", "//pkg/table/tables", "//pkg/tablecodec", @@ -353,7 +352,6 @@ 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", diff --git a/pkg/ddl/cancel_test.go b/pkg/ddl/cancel_test.go index e4d1557630435..fc50778c2f56c 100644 --- a/pkg/ddl/cancel_test.go +++ b/pkg/ddl/cancel_test.go @@ -27,6 +27,7 @@ import ( "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/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -223,7 +224,7 @@ func TestCancelVariousJobs(t *testing.T) { return enterCnt.Load() == exitCnt.Load() }, 10*time.Second, 10*time.Millisecond) } - store := testkit.CreateMockStoreWithSchemaLease(t, 100*time.Millisecond, withMockTiFlash(2)) + store := testkit.CreateMockStoreWithSchemaLease(t, 100*time.Millisecond, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tkCancel := testkit.NewTestKit(t, store) diff --git a/pkg/ddl/db_change_test.go b/pkg/ddl/db_change_test.go index 0d4661ced5689..2b640c029a4fb 100644 --- a/pkg/ddl/db_change_test.go +++ b/pkg/ddl/db_change_test.go @@ -37,6 +37,7 @@ import ( sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -1159,7 +1160,7 @@ func TestParallelAlterAddIndex(t *testing.T) { } func TestParallelAlterAddVectorIndex(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(2)) + store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, mockstore.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") diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index f62906d334d4f..54eddf2e28d31 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -385,9 +385,9 @@ func buildVectorInfoWithCheck(indexPartSpecifications []*ast.IndexPartSpecificat if !ok { return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs(fmt.Sprintf("unsupported function: %v", idxPart.Expr)) } - distanceMetric, ok := variable.DistanceMetric4VectorIndex[f.FnName.L] + distanceMetric, ok := model.FnNameToDistanceMetric[f.FnName.L] if !ok { - return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported function") + return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("currently only L2 and Cosine distance is indexable") } colExpr, ok := f.Args[0].(*ast.ColumnNameExpr) if !ok { diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 727998f90cc52..ea7e6fd767400 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -26,7 +26,6 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/pkg/ddl" testddlutil "github.com/pingcap/tidb/pkg/ddl/testutil" "github.com/pingcap/tidb/pkg/domain/infosync" @@ -42,7 +41,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" - "github.com/pingcap/tidb/pkg/store/mockstore/unistore" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -56,7 +54,6 @@ import ( "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/sqlexec" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/testutils" ) const indexModifyLease = 600 * time.Millisecond @@ -1066,26 +1063,6 @@ func TestAddIndexUniqueFailOnDuplicate(t *testing.T) { ddl.ResultCounterForTest = nil } -// withMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). -func withMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { - return mockstore.WithMultipleOptions( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockCluster := c.(*unistore.Cluster) - _, _, region1 := mockstore.BootstrapWithSingleStore(c) - tiflashIdx := 0 - for tiflashIdx < nodes { - store2 := c.AllocID() - peer2 := c.AllocID() - addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) - mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) - mockCluster.AddPeer(region1, store2, peer2) - tiflashIdx++ - } - }), - mockstore.WithStoreType(mockstore.EmbedUnistore), - ) -} - func getJobsBySQL(se sessiontypes.Session, tbl, condition string) ([]*model.Job, error) { rs, err := se.Execute(context.Background(), fmt.Sprintf("select job_meta from mysql.%s %s", tbl, condition)) if err != nil { @@ -1138,7 +1115,7 @@ func TestCreateTableWithVectorIndex(t *testing.T) { // test TiFlash store count is 2 mockTiflashStoreCnt := uint64(2) - store, dom = testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(int(mockTiflashStoreCnt)), mockstore.WithDDLChecker()) + store, dom = testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, mockstore.WithMockTiFlash(int(mockTiflashStoreCnt)), mockstore.WithDDLChecker()) tk = testkit.NewTestKit(t, store) tk.MustExec("use test") checkCreateTableWithVectorIdx(1) @@ -1158,7 +1135,7 @@ func TestCreateTableWithVectorIndex(t *testing.T) { } func TestAddVectorIndexSimple(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(2)) + store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, pt;") @@ -1343,7 +1320,7 @@ func TestAddVectorIndexSimple(t *testing.T) { } func TestAddVectorIndexRollback(t *testing.T) { - store, _ := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, withMockTiFlash(2)) + store, _ := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tiflashReplicaLease, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") diff --git a/pkg/executor/show.go b/pkg/executor/show.go index 2717338a8887c..ed5f9f1f29a0c 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -1230,7 +1230,7 @@ func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *pmodel.CIS cols = append(cols, colInfo) } if idxInfo.VectorInfo != nil { - funcName := variable.Function4VectorIndex[idxInfo.VectorInfo.DistanceMetric] + funcName := model.DistanceMetricToFnName[idxInfo.VectorInfo.DistanceMetric] fmt.Fprintf(buf, "((%s(%s)))", strings.ToUpper(funcName), strings.Join(cols, ",")) } else { fmt.Fprintf(buf, "(%s)", strings.Join(cols, ",")) diff --git a/pkg/expression/BUILD.bazel b/pkg/expression/BUILD.bazel index f837ef9240d94..55298bb6d405c 100644 --- a/pkg/expression/BUILD.bazel +++ b/pkg/expression/BUILD.bazel @@ -68,6 +68,7 @@ go_library( "simple_rewriter.go", "util.go", "vectorized.go", + "vs_helper.go", ], importpath = "github.com/pingcap/tidb/pkg/expression", visibility = ["//visibility:public"], diff --git a/pkg/expression/vs_helper.go b/pkg/expression/vs_helper.go new file mode 100644 index 0000000000000..475a0f30e16a1 --- /dev/null +++ b/pkg/expression/vs_helper.go @@ -0,0 +1,98 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package expression + +import ( + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" +) + +var ( + vsDistanceFnNamesLower = map[string]struct{}{ + strings.ToLower(ast.VecL1Distance): {}, + strings.ToLower(ast.VecL2Distance): {}, + strings.ToLower(ast.VecCosineDistance): {}, + strings.ToLower(ast.VecNegativeInnerProduct): {}, + } +) + +// VectorSearchExpr defines a minimal Vector Search expression, which is +// a vector distance function, a column to search with, and a reference vector. +type VectorSearchExpr struct { + DistanceFnName model.CIStr + Vec types.VectorFloat32 + Column *Column +} + +// ExtractVectorSearch extracts a VectorSearchExpr from an expression. +// NOTE: not all VectorSearch functions are supported by the index. The caller +// needs to check the distance function name. +func ExtractVectorSearch(expr Expression) (*VectorSearchExpr, error) { + x, ok := expr.(*ScalarFunction) + if !ok { + return nil, nil + } + + if _, isVecFn := vsDistanceFnNamesLower[x.FuncName.L]; !isVecFn { + return nil, nil + } + + args := x.GetArgs() + if len(args) != 2 { + return nil, errors.Errorf("internal: expect 2 args for function %s, but got %d", x.FuncName.L, len(args)) + } + + // One arg must be a vector column ref, and one arg must be a vector constant. + // Note: this must be run after constant folding. + + var vectorConstant *Constant = nil + var vectorColumn *Column = nil + nVectorColumns := 0 + nVectorConstants := 0 + for _, arg := range args { + if v, ok := arg.(*Column); ok { + if v.RetType.GetType() != mysql.TypeTiDBVectorFloat32 { + break + } + vectorColumn = v + nVectorColumns++ + } else if v, ok := arg.(*Constant); ok { + if v.RetType.GetType() != mysql.TypeTiDBVectorFloat32 { + break + } + vectorConstant = v + nVectorConstants++ + } + } + if nVectorColumns != 1 || nVectorConstants != 1 { + return nil, nil + } + + // All check passed. + if vectorConstant.Value.Kind() != types.KindVectorFloat32 { + return nil, errors.Errorf("internal: expect vectorFloat32 constant, but got %s", vectorConstant.Value.String()) + } + + return &VectorSearchExpr{ + DistanceFnName: x.FuncName, + Vec: vectorConstant.Value.GetVectorFloat32(), + Column: vectorColumn, + }, nil +} diff --git a/pkg/meta/model/index.go b/pkg/meta/model/index.go index 6515e75325ed5..ea812580dd494 100644 --- a/pkg/meta/model/index.go +++ b/pkg/meta/model/index.go @@ -15,12 +15,14 @@ package model import ( + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/types" ) // DistanceMetric is the distance metric used by the vector index. -// `DistanceMetric` is actually vector functions in ast package. Use `DistanceMetric` to avoid cycle dependency +// Note that not all distance functions are indexable. +// See FnNameToDistanceMetric for a list of indexable distance functions. type DistanceMetric string // Note: tipb.VectorDistanceMetric's enum names must be aligned with these constant values. @@ -29,13 +31,29 @@ const ( // DistanceMetricCosine is cosine distance. DistanceMetricCosine DistanceMetric = "COSINE" // DistanceMetricInnerProduct is inner product. + // Currently this distance metric is not supported. It is placed here only for + // reminding what's the desired naming convension (UPPER_UNDER_SCORE) if this + // is going to be implemented. DistanceMetricInnerProduct DistanceMetric = "INNER_PRODUCT" ) +// FnNameToDistanceMetric maps a distance function name to the distance metric. +// Only indexable distance functions should be listed here! +var FnNameToDistanceMetric = map[string]DistanceMetric{ + ast.VecCosineDistance: DistanceMetricCosine, + ast.VecL2Distance: DistanceMetricL2, +} + +// DistanceMetricToFnName maps a distance metric to the distance function name. +var DistanceMetricToFnName = map[DistanceMetric]string{ + DistanceMetricCosine: ast.VecCosineDistance, + DistanceMetricL2: ast.VecL2Distance, +} + // VectorIndexInfo is the information of vector index of a column. type VectorIndexInfo struct { // Dimension is the dimension of the vector. - Dimension uint64 `json:"dimension"` // Set to 0 when initially parsed from comment. Will be assigned to flen later. + Dimension uint64 `json:"dimension"` // DistanceMetric is the distance metric used by the index. DistanceMetric DistanceMetric `json:"distance_metric"` } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 3b631d6741f36..fd080c5d754f1 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -267,7 +267,6 @@ go_test( deps = [ "//pkg/config", "//pkg/domain", - "//pkg/domain/infosync", "//pkg/expression", "//pkg/expression/aggregation", "//pkg/expression/exprctx", @@ -301,6 +300,7 @@ go_test( "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/statistics", + "//pkg/store/mockstore", "//pkg/table", "//pkg/testkit", "//pkg/testkit/ddlhelper", diff --git a/pkg/planner/core/access_object.go b/pkg/planner/core/access_object.go index fb83b52b5ae92..323a8ac1d187c 100644 --- a/pkg/planner/core/access_object.go +++ b/pkg/planner/core/access_object.go @@ -283,6 +283,19 @@ func (p *PhysicalTableScan) AccessObject() base.AccessObject { res.Partitions = []string{partitionName} } } + if p.AnnIndexExtra != nil { + index := IndexAccess{ + Name: p.AnnIndexExtra.IndexInfo.Name.O, + } + for _, idxCol := range p.AnnIndexExtra.IndexInfo.Columns { + if tblCol := p.Table.Columns[idxCol.Offset]; tblCol.Hidden { + index.Cols = append(index.Cols, tblCol.GeneratedExprString) + } else { + index.Cols = append(index.Cols, idxCol.Name.O) + } + } + res.Indexes = []IndexAccess{index} + } return res } diff --git a/pkg/planner/core/casetest/enforcempp/BUILD.bazel b/pkg/planner/core/casetest/enforcempp/BUILD.bazel index 65369c052c5e4..88dd77649cc07 100644 --- a/pkg/planner/core/casetest/enforcempp/BUILD.bazel +++ b/pkg/planner/core/casetest/enforcempp/BUILD.bazel @@ -15,6 +15,7 @@ go_test( "//pkg/meta/model", "//pkg/parser/model", "//pkg/planner/util/coretestsdk", + "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/external", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go index d1ef3a47e7c76..80e9575be181c 100644 --- a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go +++ b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -461,7 +462,7 @@ func TestMPPSingleDistinct3Stage(t *testing.T) { // // since it doesn't change the schema out (index ref is still the right), so by now it's fine. SEE case: EXPLAIN select count(distinct a), count(distinct b), sum(c) from t. func TestMPPMultiDistinct3Stage(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) // test table @@ -510,7 +511,7 @@ func TestMPPMultiDistinct3Stage(t *testing.T) { // Test null-aware semi join push down for MPP mode func TestMPPNullAwareSemiJoinPushDown(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) // test table @@ -558,7 +559,7 @@ func TestMPPNullAwareSemiJoinPushDown(t *testing.T) { } func TestMPPSharedCTEScan(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) // test table @@ -606,7 +607,7 @@ func TestMPPSharedCTEScan(t *testing.T) { } func TestRollupMPP(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/pkg/planner/core/casetest/hint/BUILD.bazel b/pkg/planner/core/casetest/hint/BUILD.bazel index 388a30b1ad4db..34126a6b4a9f7 100644 --- a/pkg/planner/core/casetest/hint/BUILD.bazel +++ b/pkg/planner/core/casetest/hint/BUILD.bazel @@ -17,6 +17,7 @@ go_test( "//pkg/parser/model", "//pkg/planner/util/coretestsdk", "//pkg/sessionctx/variable", + "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/hint/hint_test.go b/pkg/planner/core/casetest/hint/hint_test.go index 3dbc953e8b364..3df832df12bd7 100644 --- a/pkg/planner/core/casetest/hint/hint_test.go +++ b/pkg/planner/core/casetest/hint/hint_test.go @@ -23,6 +23,7 @@ import ( pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -69,7 +70,7 @@ func TestReadFromStorageHint(t *testing.T) { } func TestAllViewHintType(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -130,7 +131,7 @@ func TestAllViewHintType(t *testing.T) { } func TestJoinHintCompatibility(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/pkg/planner/core/casetest/index/BUILD.bazel b/pkg/planner/core/casetest/index/BUILD.bazel index 84411f57244fe..e4bb65791f767 100644 --- a/pkg/planner/core/casetest/index/BUILD.bazel +++ b/pkg/planner/core/casetest/index/BUILD.bazel @@ -9,13 +9,19 @@ go_test( ], data = glob(["testdata/**"]), flaky = True, - shard_count = 6, + shard_count = 8, deps = [ + "//pkg/domain", + "//pkg/domain/infosync", + "//pkg/parser/model", + "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", + "//pkg/testkit/testfailpoint", "//pkg/testkit/testmain", "//pkg/testkit/testsetup", "//pkg/util", + "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], ) diff --git a/pkg/planner/core/casetest/index/index_test.go b/pkg/planner/core/casetest/index/index_test.go index ac4c77b31b930..a5348daae9854 100644 --- a/pkg/planner/core/casetest/index/index_test.go +++ b/pkg/planner/core/casetest/index/index_test.go @@ -15,12 +15,20 @@ package index import ( + "context" "fmt" "testing" + "time" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/domain/infosync" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/util" + "github.com/stretchr/testify/require" ) func TestNullConditionForPrefixIndex(t *testing.T) { @@ -230,3 +238,88 @@ func TestOrderedIndexWithIsNull(t *testing.T) { " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#5", " └─IndexRangeScan_16 3.00 cop[tikv] table:t2, index:index_on_id(id) range:[NULL,NULL], keep order:false")) } + +const tiflashReplicaLease = 600 * time.Millisecond + +func TestVectorIndex(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, tiflashReplicaLease, mockstore.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + + tk.MustExec("create table t (a int, b vector, c vector(3), d vector(4));") + tk.MustExec("alter table t set tiflash replica 1;") + tk.MustExec("alter table t add vector index vecIdx1((vec_cosine_distance(d))) USING HNSW;") + tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance(d, '[1,1,1,1]') limit 1", "vecIdx1") + tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance('[1,1,1,1]', d) limit 1", "vecIdx1") + tk.MustNoIndexUsed("select * from t use index(vecIdx1) order by vec_l2_distance(d, '[1,1,1,1]') limit 1") + tk.MustNoIndexUsed("select * from t use index(vecIdx1) where a = 5 order by vec_cosine_distance(d, '[1,1,1,1]') limit 1") +} + +func TestAnalyzeVectorIndex(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 200*time.Millisecond, mockstore.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + tk.MustExec(`create table t(a int, b vector(2), c vector(3), j json, index(a))`) + tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';") + tblInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + 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)`) + 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") + + tk.MustUseIndex("select * from t use index(idx) order by vec_cosine_distance(b, '[1, 0]') limit 1", "idx") + tk.MustUseIndex("select * from t order by vec_cosine_distance(b, '[1, 0]') limit 1", "idx") + tk.MustNoIndexUsed("select * from t ignore index(idx) order by vec_cosine_distance(b, '[1, 0]') limit 1") + + tk.MustExec("set tidb_analyze_version=2") + tk.MustExec("analyze table t") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Warning 1105 No predicate column has been collected yet for table test.t, so only indexes and the columns composing the indexes will be analyzed", + "Warning 1105 analyzing vector index is not supported, skip idx", + "Warning 1105 analyzing vector index is not supported, skip idx2")) + tk.MustExec("analyze table t index idx") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty, use sample-rate=1\"", + "Warning 1105 No predicate column has been collected yet for table test.t, so only indexes and the columns composing the indexes will be analyzed", + "Warning 1105 The version 2 would collect all statistics not only the selected indexes", + "Warning 1105 analyzing vector index is not supported, skip idx", + "Warning 1105 analyzing vector index is not supported, skip idx2")) + + tk.MustExec("set tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Warning 1105 analyzing vector index is not supported, skip idx", + "Warning 1105 analyzing vector index is not supported, skip idx2")) + tk.MustExec("analyze table t index idx") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Warning 1105 analyzing vector index is not supported, skip idx")) + tk.MustExec("analyze table t index a") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows()) + tk.MustExec("analyze table t index a, idx, idx2") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Warning 1105 analyzing vector index is not supported, skip idx", + "Warning 1105 analyzing vector index is not supported, skip idx2")) +} diff --git a/pkg/planner/core/casetest/physicalplantest/BUILD.bazel b/pkg/planner/core/casetest/physicalplantest/BUILD.bazel index 4dc70207d8987..2fa0537a0253a 100644 --- a/pkg/planner/core/casetest/physicalplantest/BUILD.bazel +++ b/pkg/planner/core/casetest/physicalplantest/BUILD.bazel @@ -23,6 +23,7 @@ go_test( "//pkg/planner/core", "//pkg/planner/core/resolve", "//pkg/planner/util/coretestsdk", + "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/external", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go b/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go index 925d2acdca769..ba29b7adef271 100644 --- a/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go +++ b/pkg/planner/core/casetest/physicalplantest/physical_plan_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -217,7 +218,7 @@ func TestUnmatchedTableInHint(t *testing.T) { } func TestIssue37520(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -247,7 +248,7 @@ func TestIssue37520(t *testing.T) { } func TestMPPHints(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -289,7 +290,7 @@ func TestMPPHints(t *testing.T) { } func TestMPPHintsScope(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -346,7 +347,7 @@ func TestMPPBCJModel(t *testing.T) { Probe: sizeof(Data) * 2 / 3 exchange size: Build + Probe = 4/3 * sizeof(Data) */ - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(3)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(3)) { cnt, err := store.GetMPPClient().GetMPPStoreCount() require.Equal(t, cnt, 3) @@ -388,7 +389,7 @@ func TestMPPBCJModel(t *testing.T) { } func TestMPPPreferBCJ(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(3)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(3)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -455,7 +456,7 @@ func TestMPPBCJModelOneTiFlash(t *testing.T) { Probe: sizeof(Data) * 0 / 1 exchange size: Build + Probe = 0 * sizeof(Data) */ - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(1)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(1)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") @@ -506,7 +507,7 @@ func TestMPPBCJModelOneTiFlash(t *testing.T) { } func TestMPPRightSemiJoin(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(3)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(3)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -562,7 +563,7 @@ func TestMPPRightSemiJoin(t *testing.T) { } func TestMPPRightOuterJoin(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(3)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(3)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") diff --git a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel new file mode 100644 index 0000000000000..b15a17c086bb0 --- /dev/null +++ b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel @@ -0,0 +1,26 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "vectorsearch_test", + timeout = "short", + srcs = [ + "main_test.go", + "vector_index_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + deps = [ + "//pkg/config", + "//pkg/domain/infosync", + "//pkg/meta/model", + "//pkg/store/mockstore", + "//pkg/testkit", + "//pkg/testkit/testdata", + "//pkg/testkit/testfailpoint", + "//pkg/testkit/testmain", + "//pkg/testkit/testsetup", + "@com_github_pingcap_tipb//go-tipb", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/planner/core/casetest/vectorsearch/main_test.go b/pkg/planner/core/casetest/vectorsearch/main_test.go new file mode 100644 index 0000000000000..903a8a38c1910 --- /dev/null +++ b/pkg/planner/core/casetest/vectorsearch/main_test.go @@ -0,0 +1,59 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vectorsearch + +import ( + "flag" + "testing" + + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/testkit/testmain" + "github.com/pingcap/tidb/pkg/testkit/testsetup" + "go.uber.org/goleak" +) + +var testDataMap = make(testdata.BookKeeper) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + flag.Parse() + testDataMap.LoadTestSuiteData("testdata", "ann_index_suite") + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Performance.EnableStatsCacheMemQuota = true + }) + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetANNIndexSuiteData() testdata.TestData { + return testDataMap["ann_index_suite"] +} diff --git a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json new file mode 100644 index 0000000000000..48665b3fb68b3 --- /dev/null +++ b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json @@ -0,0 +1,48 @@ +[ + { + "name": "TestTiFlashANNIndex", + "cases": [ + "explain select * from t1;", + "explain select * from t1 where a<1;", + "explain select * from t1 where vec = '[1,1,1]'", + "explain select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", + "explain select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", + "explain select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1" + ] + }, + { + "name": "TestTiFlashANNIndexForPartition", + "cases": [ + "explain select * from t1;", + "explain select * from t1 where a<1;", + "explain select * from t1 where vec = '[1,1,1]'", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 where store_id between 80 and 120 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "explain select * from t1 partition (p0) order by vec_cosine_distance(vec, '[1,1,1]') limit 1" + ] + } +] diff --git a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json new file mode 100644 index 0000000000000..0fe3d710d4065 --- /dev/null +++ b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json @@ -0,0 +1,482 @@ +[ + { + "Name": "TestTiFlashANNIndex", + "Cases": [ + { + "SQL": "explain select * from t1;", + "Plan": [ + "TableReader_10 10000.00 root MppVersion: 2, data:ExchangeSender_9", + "└─ExchangeSender_9 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_8 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where a<1;", + "Plan": [ + "TableReader_13 3323.33 root MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_11 3323.33 mpp[tiflash] lt(test.t1.a, 1)", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where vec = '[1,1,1]'", + "Plan": [ + "TableReader_13 10.00 root MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 10.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:eq(test.t1.vec, [1,1,1]), keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", + "Plan": [ + "TableReader_13 8000.00 root MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_11 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", + "Plan": [ + "TableReader_13 8000.00 root MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_5 8000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─Selection_11 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "Plan": [ + "Projection_13 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─Sort_4 10000.00 root Column#7", + " └─Projection_14 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_12 10000.00 root MppVersion: 2, data:ExchangeSender_11", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_10 1.00 root Column#8, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_10 1.00 root Column#8, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7:desc, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7:desc, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, test.t1.vec, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.c, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", + "Plan": [ + "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_9 1.00 root Column#7, offset:0, count:1", + " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "Plan": [ + "Sort_4 10000.00 root Column#7", + "└─TableReader_13 10000.00 root MppVersion: 2, data:ExchangeSender_12", + " └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 10000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_11 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "Plan": [ + "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + "└─Projection_21 1.00 root test.t1.vec", + " └─TopN_10 1.00 root Column#9, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", + " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec", + " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", + " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + ], + "Warn": null + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "Plan": [ + "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7, test.t1.vec, test.t1.a, test.t1.b", + "└─Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b", + " └─TopN_10 1.00 root Column#9, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", + " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b", + " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", + " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_24 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN_11 1.00 root Column#8, offset:0, count:1", + " └─Projection_25 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader_21 1.00 root MppVersion: 2, data:ExchangeSender_20", + " └─ExchangeSender_20 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_22 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN_19 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection_23 10.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_17 10.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:eq(test.t1.a, 0), keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestTiFlashANNIndexForPartition", + "Cases": [ + { + "SQL": "explain select * from t1;", + "Plan": [ + "TableReader_10 49152.00 root partition:all MppVersion: 2, data:ExchangeSender_9", + "└─ExchangeSender_9 49152.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_8 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where a<1;", + "Plan": [ + "TableReader_13 0.00 root partition:all MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 0.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 0.00 mpp[tiflash] table:t1 pushed down filter:lt(test.t1.a, 1), keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where vec = '[1,1,1]'", + "Plan": [ + "TableReader_13 16384.00 root partition:all MppVersion: 2, data:ExchangeSender_12", + "└─ExchangeSender_12 16384.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 16384.00 mpp[tiflash] table:t1 pushed down filter:eq(test.t1.vec, [1,1,1]), keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "Plan": [ + "Projection_13 49152.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─Sort_4 49152.00 root Column#6", + " └─Projection_14 49152.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableReader_12 49152.00 root partition:all MppVersion: 2, data:ExchangeSender_11", + " └─ExchangeSender_11 49152.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: only Top N queries (like ORDER BY ... LIMIT ...) can use ANN index" + ] + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_10 1.00 root Column#7, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_16 1.00 mpp[tiflash] Column#6, offset:0, count:1", + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1 annIndex:COSINE(test.t1.vec..[1,1,1], limit:1), keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_10 1.00 root Column#7:desc, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_16 1.00 mpp[tiflash] Column#6:desc, offset:0, count:1", + " └─Projection_20 49152.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_15 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: index can be used only when ordering by vec_cosine_distance() in ASC order" + ] + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_10 1.00 root Column#7:desc, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_16 1.00 mpp[tiflash] Column#6:desc, offset:0, count:1", + " └─Projection_20 49152.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_15 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: index can be used only when ordering by vec_cosine_distance() in ASC order" + ] + }, + { + "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_10 1.00 root Column#7, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#7", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_16 1.00 mpp[tiflash] Column#6, offset:0, count:1", + " └─Projection_20 49152.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#6", + " └─TableFullScan_15 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: not ordering by a vector distance function" + ] + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "Plan": [ + "Sort_4 49152.00 root Column#6", + "└─TableReader_13 49152.00 root partition:all MppVersion: 2, data:ExchangeSender_12", + " └─ExchangeSender_12 49152.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 49152.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_11 49152.00 mpp[tiflash] table:t1 keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: only Top N queries (like ORDER BY ... LIMIT ...) can use ANN index" + ] + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "Plan": [ + "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + "└─Projection_21 1.00 root test.t1.vec", + " └─TopN_10 1.00 root Column#8, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec", + " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1 annIndex:COSINE(test.t1.vec..[1,1,1], limit:1), keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "Plan": [ + "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6, test.t1.vec, test.t1.a, test.t1.b", + "└─Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b", + " └─TopN_10 1.00 root Column#8, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader_18 1.00 root partition:all MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b", + " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1 annIndex:COSINE(test.t1.vec..[1,1,1], limit:1), keep order:false, PartitionTableScan:true" + ], + "Warn": null + }, + { + "SQL": "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_24 0.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_11 0.00 root Column#7, offset:0, count:1", + " └─Projection_25 0.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_21 0.00 root partition:all MppVersion: 2, data:ExchangeSender_20", + " └─ExchangeSender_20 0.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_22 0.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_19 0.00 mpp[tiflash] Column#6, offset:0, count:1", + " └─Projection_23 0.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_17 0.00 mpp[tiflash] table:t1 pushed down filter:eq(test.t1.a, 0), keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: cannot utilize ANN index when there is a WHERE or HAVING clause" + ] + }, + { + "SQL": "explain select * from t1 where store_id between 80 and 120 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_24 0.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_11 0.00 root Column#7, offset:0, count:1", + " └─Projection_25 0.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_21 0.00 root partition:p0,p1 MppVersion: 2, data:ExchangeSender_20", + " └─ExchangeSender_20 0.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_22 0.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_19 0.00 mpp[tiflash] Column#6, offset:0, count:1", + " └─Projection_23 0.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_17 0.00 mpp[tiflash] table:t1 pushed down filter:ge(test.t1.store_id, 80), le(test.t1.store_id, 120), keep order:false, PartitionTableScan:true" + ], + "Warn": [ + "ANN index not used: cannot utilize ANN index when there is a WHERE or HAVING clause" + ] + }, + { + "SQL": "explain select * from t1 partition (p0) order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "Plan": [ + "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + "└─TopN_10 1.00 root Column#7, offset:0, count:1", + " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader_18 1.00 root partition:p0 MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id", + " └─TopN_16 1.00 mpp[tiflash] Column#6, offset:0, count:1", + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.store_id, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#6", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1 annIndex:COSINE(test.t1.vec..[1,1,1], limit:1), keep order:false, PartitionTableScan:true" + ], + "Warn": null + } + ] + } +] diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go new file mode 100644 index 0000000000000..f9c28ac59ab34 --- /dev/null +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -0,0 +1,99 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vectorsearch + +import ( + "strings" + "testing" + "time" + + "github.com/pingcap/tidb/pkg/domain/infosync" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/store/mockstore" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/testkit/testfailpoint" + "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" +) + +func getPlanRows(planStr string) []string { + planStr = strings.Replace(planStr, "\t", " ", -1) + return strings.Split(planStr, "\n") +} + +func TestVectorIndexProtobufMatch(t *testing.T) { + require.EqualValues(t, tipb.VectorDistanceMetric_INNER_PRODUCT.String(), model.DistanceMetricInnerProduct) +} + +func TestTiFlashANNIndex(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second, mockstore.WithMockTiFlash(2)) + + tk := testkit.NewTestKit(t, store) + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec(` + create table t1 ( + vec vector(3), + a int, + b int, + c vector(3), + d vector + ) + `) + tk.MustExec("alter table t1 set tiflash replica 1;") + tk.MustExec("alter table t1 add vector index ((vec_cosine_distance(vec))) USING HNSW;") + tk.MustExec(` + insert into t1 values + ('[1,1,1]', 1, 1, '[1,1,1]', '[1,1,1]'), + ('[2,2,2]', 2, 2, '[2,2,2]', '[2,2,2]'), + ('[3,3,3]', 3, 3, '[3,3,3]', '[3,3,3]') + `) + for i := 0; i < 14; i++ { + tk.MustExec("insert into t1(vec, a, b, c, d) select vec, a, b, c, d from t1") + } + tk.MustExec("analyze table t1") + + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := GetANNIndexSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} diff --git a/pkg/planner/core/explain.go b/pkg/planner/core/explain.go index 81c6057aed2be..e4d4aabb5cfd7 100644 --- a/pkg/planner/core/explain.go +++ b/pkg/planner/core/explain.go @@ -270,6 +270,24 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { buffer.WriteString(runtimeFilter.ExplainInfo(false)) } } + if p.AnnIndexExtra != nil && p.AnnIndexExtra.PushDownQueryInfo != nil { + buffer.WriteString(", annIndex:") + buffer.WriteString(p.AnnIndexExtra.PushDownQueryInfo.GetDistanceMetric().String()) + buffer.WriteString("(") + buffer.WriteString(p.AnnIndexExtra.PushDownQueryInfo.GetColumnName()) + buffer.WriteString("..") + { + v, _, err := types.ZeroCopyDeserializeVectorFloat32(p.AnnIndexExtra.PushDownQueryInfo.RefVecF32) + if err != nil { + buffer.WriteString("[?]") + } else { + buffer.WriteString(v.String()) + } + } + buffer.WriteString(", limit:") + buffer.WriteString(fmt.Sprint(p.AnnIndexExtra.PushDownQueryInfo.TopK)) + buffer.WriteString(")") + } return buffer.String() } diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 57b98cb3e17a6..8564d66654465 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2503,6 +2503,12 @@ func convertToTableScan(ds *logicalop.DataSource, prop *property.PhysicalPropert if prop.MPPPartitionTp != property.AnyType { return base.InvalidTask, nil } + if candidate.path.Index != nil && candidate.path.Index.VectorInfo != nil { + ts.AnnIndexExtra = &VectorIndexExtra{ + IndexInfo: candidate.path.Index, + PushDownQueryInfo: nil, + } + } // ********************************** future deprecated start **************************/ var hasVirtualColumn bool for _, col := range ts.schema.Columns { diff --git a/pkg/planner/core/indexmerge_path_test.go b/pkg/planner/core/indexmerge_path_test.go index c06abd1e8bb3c..572698a71f7ea 100644 --- a/pkg/planner/core/indexmerge_path_test.go +++ b/pkg/planner/core/indexmerge_path_test.go @@ -20,10 +20,7 @@ import ( "math/rand" "strings" "testing" - "time" - "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" diff --git a/pkg/planner/core/physical_plan_test.go b/pkg/planner/core/physical_plan_test.go index ac6fa59be8025..3978f035de38a 100644 --- a/pkg/planner/core/physical_plan_test.go +++ b/pkg/planner/core/physical_plan_test.go @@ -35,8 +35,8 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" @@ -229,7 +229,7 @@ func TestIndexLookupCartesianJoin(t *testing.T) { } func TestMPPHintsWithBinding(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -278,7 +278,7 @@ func TestMPPHintsWithBinding(t *testing.T) { } func TestJoinHintCompatibilityWithBinding(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -452,7 +452,7 @@ func TestPhysicalPlanMemoryTrace(t *testing.T) { } func TestPhysicalTableScanExtractCorrelatedCols(t *testing.T) { - store := testkit.CreateMockStore(t, coretestsdk.WithMockTiFlash(2)) + store := testkit.CreateMockStore(t, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t1 (id int, client_type tinyint, client_no char(18), taxpayer_no varchar(50), status tinyint, update_time datetime)") diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 2a304707e24fb..11965265b839b 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -955,6 +955,19 @@ type PhysicalTableScan struct { // for runtime filter runtimeFilterList []*RuntimeFilter `plan-cache-clone:"must-nil"` // plan with runtime filter is not cached maxWaitTimeMs int + + AnnIndexExtra *VectorIndexExtra `plan-cache-clone:"must-nil"` // MPP plan should not be cached. +} + +// VectorIndexExtra is the extra information for vector index. +type VectorIndexExtra struct { + // Note: Even if IndexInfo is not nil, it doesn't mean the VectorSearch push down + // will happen because optimizer will explore all available vector indexes and fill them + // in IndexInfo, and later invalid plans are filtered out according to a topper executor. + IndexInfo *model.IndexInfo + + // Not nil if there is an VectorSearch push down. + PushDownQueryInfo *tipb.ANNQueryInfo } // Clone implements op.PhysicalPlan interface. diff --git a/pkg/planner/core/plan_cost_ver2.go b/pkg/planner/core/plan_cost_ver2.go index 75f010fe66d9c..c5d5221f3e379 100644 --- a/pkg/planner/core/plan_cost_ver2.go +++ b/pkg/planner/core/plan_cost_ver2.go @@ -182,6 +182,21 @@ func (p *PhysicalTableScan) GetPlanCostVer2(taskType property.TaskType, option * } } + if p.AnnIndexExtra != nil { + if p.AnnIndexExtra.PushDownQueryInfo == nil { + p.PlanCostVer2 = costusage.NewCostVer2(option, defaultVer2Factors.ANNIndexNoTopK, defaultVer2Factors.ANNIndexNoTopK.Value, func() string { + return fmt.Sprintf("ann-index-no-topk(%v)", defaultVer2Factors.ANNIndexNoTopK) + }) + } else { + p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, costusage.NewCostVer2(option, defaultVer2Factors.ANNIndexStart, rows*defaultVer2Factors.ANNIndexStart.Value, func() string { + return fmt.Sprintf("ann-index-start(%v*%v)", rows, defaultVer2Factors.ANNIndexStart) + })) + p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, costusage.NewCostVer2(option, defaultVer2Factors.ANNIndexScanRow, float64(p.AnnIndexExtra.PushDownQueryInfo.TopK)*defaultVer2Factors.ANNIndexScanRow.Value, func() string { + return fmt.Sprintf("ann-index-topk(%v*%v)", p.AnnIndexExtra.PushDownQueryInfo.TopK, defaultVer2Factors.ANNIndexScanRow) + })) + } + } + p.PlanCostInit = true return p.PlanCostVer2, nil } @@ -913,21 +928,24 @@ func doubleReadCostVer2(option *optimizetrace.PlanCostOption, numTasks float64, // In Cost Ver2, we hide cost factors from users and deprecate SQL variables like `tidb_opt_scan_factor`. type costVer2Factors struct { - TiDBTemp costusage.CostVer2Factor // operations on TiDB temporary table - TiKVScan costusage.CostVer2Factor // per byte - TiKVDescScan costusage.CostVer2Factor // per byte - TiFlashScan costusage.CostVer2Factor // per byte - TiDBCPU costusage.CostVer2Factor // per column or expression - TiKVCPU costusage.CostVer2Factor // per column or expression - TiFlashCPU costusage.CostVer2Factor // per column or expression - TiDB2KVNet costusage.CostVer2Factor // per byte - TiDB2FlashNet costusage.CostVer2Factor // per byte - TiFlashMPPNet costusage.CostVer2Factor // per byte - TiDBMem costusage.CostVer2Factor // per byte - TiKVMem costusage.CostVer2Factor // per byte - TiFlashMem costusage.CostVer2Factor // per byte - TiDBDisk costusage.CostVer2Factor // per byte - TiDBRequest costusage.CostVer2Factor // per net request + TiDBTemp costusage.CostVer2Factor // operations on TiDB temporary table + TiKVScan costusage.CostVer2Factor // per byte + TiKVDescScan costusage.CostVer2Factor // per byte + TiFlashScan costusage.CostVer2Factor // per byte + TiDBCPU costusage.CostVer2Factor // per column or expression + TiKVCPU costusage.CostVer2Factor // per column or expression + TiFlashCPU costusage.CostVer2Factor // per column or expression + TiDB2KVNet costusage.CostVer2Factor // per byte + TiDB2FlashNet costusage.CostVer2Factor // per byte + TiFlashMPPNet costusage.CostVer2Factor // per byte + TiDBMem costusage.CostVer2Factor // per byte + TiKVMem costusage.CostVer2Factor // per byte + TiFlashMem costusage.CostVer2Factor // per byte + TiDBDisk costusage.CostVer2Factor // per byte + TiDBRequest costusage.CostVer2Factor // per net request + ANNIndexStart costusage.CostVer2Factor // ANN index's warmup cost, related to row num. + ANNIndexScanRow costusage.CostVer2Factor // ANN index's scan cost, by row. + ANNIndexNoTopK costusage.CostVer2Factor // special factor for ANN index without top-k: max uint64 } func (c costVer2Factors) tolist() (l []costusage.CostVer2Factor) { @@ -936,21 +954,24 @@ func (c costVer2Factors) tolist() (l []costusage.CostVer2Factor) { } var defaultVer2Factors = costVer2Factors{ - TiDBTemp: costusage.CostVer2Factor{Name: "tidb_temp_table_factor", Value: 0.00}, - TiKVScan: costusage.CostVer2Factor{Name: "tikv_scan_factor", Value: 40.70}, - TiKVDescScan: costusage.CostVer2Factor{Name: "tikv_desc_scan_factor", Value: 61.05}, - TiFlashScan: costusage.CostVer2Factor{Name: "tiflash_scan_factor", Value: 11.60}, - TiDBCPU: costusage.CostVer2Factor{Name: "tidb_cpu_factor", Value: 49.90}, - TiKVCPU: costusage.CostVer2Factor{Name: "tikv_cpu_factor", Value: 49.90}, - TiFlashCPU: costusage.CostVer2Factor{Name: "tiflash_cpu_factor", Value: 2.40}, - TiDB2KVNet: costusage.CostVer2Factor{Name: "tidb_kv_net_factor", Value: 3.96}, - TiDB2FlashNet: costusage.CostVer2Factor{Name: "tidb_flash_net_factor", Value: 2.20}, - TiFlashMPPNet: costusage.CostVer2Factor{Name: "tiflash_mpp_net_factor", Value: 1.00}, - TiDBMem: costusage.CostVer2Factor{Name: "tidb_mem_factor", Value: 0.20}, - TiKVMem: costusage.CostVer2Factor{Name: "tikv_mem_factor", Value: 0.20}, - TiFlashMem: costusage.CostVer2Factor{Name: "tiflash_mem_factor", Value: 0.05}, - TiDBDisk: costusage.CostVer2Factor{Name: "tidb_disk_factor", Value: 200.00}, - TiDBRequest: costusage.CostVer2Factor{Name: "tidb_request_factor", Value: 6000000.00}, + TiDBTemp: costusage.CostVer2Factor{Name: "tidb_temp_table_factor", Value: 0.00}, + TiKVScan: costusage.CostVer2Factor{Name: "tikv_scan_factor", Value: 40.70}, + TiKVDescScan: costusage.CostVer2Factor{Name: "tikv_desc_scan_factor", Value: 61.05}, + TiFlashScan: costusage.CostVer2Factor{Name: "tiflash_scan_factor", Value: 11.60}, + TiDBCPU: costusage.CostVer2Factor{Name: "tidb_cpu_factor", Value: 49.90}, + TiKVCPU: costusage.CostVer2Factor{Name: "tikv_cpu_factor", Value: 49.90}, + TiFlashCPU: costusage.CostVer2Factor{Name: "tiflash_cpu_factor", Value: 2.40}, + TiDB2KVNet: costusage.CostVer2Factor{Name: "tidb_kv_net_factor", Value: 3.96}, + TiDB2FlashNet: costusage.CostVer2Factor{Name: "tidb_flash_net_factor", Value: 2.20}, + TiFlashMPPNet: costusage.CostVer2Factor{Name: "tiflash_mpp_net_factor", Value: 1.00}, + TiDBMem: costusage.CostVer2Factor{Name: "tidb_mem_factor", Value: 0.20}, + TiKVMem: costusage.CostVer2Factor{Name: "tikv_mem_factor", Value: 0.20}, + TiFlashMem: costusage.CostVer2Factor{Name: "tiflash_mem_factor", Value: 0.05}, + TiDBDisk: costusage.CostVer2Factor{Name: "tidb_disk_factor", Value: 200.00}, + TiDBRequest: costusage.CostVer2Factor{Name: "tidb_request_factor", Value: 6000000.00}, + ANNIndexStart: costusage.CostVer2Factor{Name: "ann_index_start_factor", Value: 0.000144}, + ANNIndexScanRow: costusage.CostVer2Factor{Name: "ann_index_scan_factor", Value: 1.65}, + ANNIndexNoTopK: costusage.CostVer2Factor{Name: "ann_index_no_topk_factor", Value: math.MaxUint64}, } func getTaskCPUFactorVer2(_ base.PhysicalPlan, taskType property.TaskType) costusage.CostVer2Factor { diff --git a/pkg/planner/core/plan_to_pb.go b/pkg/planner/core/plan_to_pb.go index 24401d9af3a7d..f67983ba3aaa6 100644 --- a/pkg/planner/core/plan_to_pb.go +++ b/pkg/planner/core/plan_to_pb.go @@ -273,6 +273,10 @@ func (p *PhysicalTableScan) ToPB(ctx *base.BuildPBContext, storeType kv.StoreTyp tsExec.PushedDownFilterConditions = conditions } + if p.AnnIndexExtra != nil && p.AnnIndexExtra.PushDownQueryInfo != nil { + tsExec.AnnQuery = p.AnnIndexExtra.PushDownQueryInfo + } + var err error tsExec.RuntimeFilterList, err = RuntimeFilterListToPB(ctx, p.runtimeFilterList, ctx.GetClient()) if err != nil { diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index b0be7597959ad..716b1ecc0331d 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -1076,11 +1076,12 @@ func getPathByIndexName(paths []*util.AccessPath, idxName pmodel.CIStr, tblInfo var primaryIdxPath, indexPrefixPath *util.AccessPath prefixMatches := 0 for _, path := range paths { - if path.StoreType == kv.TiFlash { + if path.IsTiKVTablePath() { + primaryIdxPath = path continue } - if path.IsTablePath() { - primaryIdxPath = path + // If it's not a tikv table path and the index is nil, it could not be any index path. + if path.Index == nil { continue } if path.Index.Name.L == idxName.L { @@ -1193,7 +1194,11 @@ func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, in continue } } - publicPaths = append(publicPaths, &util.AccessPath{Index: index}) + path := &util.AccessPath{Index: index} + if index.VectorInfo != nil { + path.StoreType = kv.TiFlash + } + publicPaths = append(publicPaths, path) } } diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 14c49e3ae78b7..c2b526bf9c869 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/paging" "github.com/pingcap/tidb/pkg/util/plancodec" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -990,6 +992,9 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { } else if mppTask, ok := t.(*MppTask); ok && needPushDown && p.canPushDownToTiFlash(mppTask) { pushedDownTopN := p.getPushedDownTopN(mppTask.p) mppTask.p = pushedDownTopN + if !fixTopNForANNIndex(pushedDownTopN) { + return base.InvalidTask + } } rootTask := t.ConvertToRootTask(p.SCtx()) // Skip TopN with partition on the root. This is a derived topN and window function @@ -1000,6 +1005,64 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { return attachPlan2Task(p, rootTask) } +func fixTopNForANNIndex(p *PhysicalTopN) bool { + ts, ok := p.Children()[0].(*PhysicalTableScan) + if !ok || ts.AnnIndexExtra == nil { + return true + } + if ts.AnnIndexExtra.IndexInfo.VectorInfo == nil { + // AnnIndex is attached with a non-vector index. + // This should not happen. + return false + } + if len(p.ByItems) != 1 { + return false + } + if p.ByItems[0].Desc { + // Currently all vector search indexes must be used for ASC order. + // DESC order can be only applied to INNER_PRODUCT, which is not + // supported yet. + return false + } + vs, err := expression.ExtractVectorSearch(p.ByItems[0].Expr) + if err != nil || vs == nil { + return false + } + // Note that even if this is a vector search expression, it may not hit vector index + // because not all vector search functions are indexable. + distanceMetric, ok := model.FnNameToDistanceMetric[vs.DistanceFnName.L] + if !ok { + return false + } + // User may build a vector index with different distance metric. + // In this case the index shall not push down. + if distanceMetric != ts.AnnIndexExtra.IndexInfo.VectorInfo.DistanceMetric { + return false + } + // User may build a vector index with different vector column. + // In this case the index shall not push down. + col := ts.Table.Columns[ts.AnnIndexExtra.IndexInfo.Columns[0].Offset] + if col.ID != vs.Column.ID { + return false + } + + distanceMetricPB, ok := tipb.VectorDistanceMetric_value[string(distanceMetric)] + if !ok { + // This should not happen. + return false + } + ts.AnnIndexExtra.PushDownQueryInfo = &tipb.ANNQueryInfo{ + QueryType: tipb.ANNQueryType_OrderBy, + DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), + TopK: uint32(p.Count), + ColumnName: col.Name.L, + RefVecF32: vs.Vec.SerializeTo(nil), + IndexId: int64(ts.AnnIndexExtra.IndexInfo.ID), + } + ts.PlanCostInit = false + return true +} + // Attach2Task implements the PhysicalPlan interface. func (p *PhysicalExpand) Attach2Task(tasks ...base.Task) base.Task { t := tasks[0].Copy() diff --git a/pkg/planner/util/coretestsdk/BUILD.bazel b/pkg/planner/util/coretestsdk/BUILD.bazel index 34f1f16d362b9..7ad54af813ec7 100644 --- a/pkg/planner/util/coretestsdk/BUILD.bazel +++ b/pkg/planner/util/coretestsdk/BUILD.bazel @@ -9,10 +9,6 @@ go_library( "//pkg/domain", "//pkg/meta/model", "//pkg/parser/model", - "//pkg/store/mockstore", - "//pkg/store/mockstore/unistore", - "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//testutils", ], ) diff --git a/pkg/planner/util/coretestsdk/testkit.go b/pkg/planner/util/coretestsdk/testkit.go index 4677839fcf4b7..bd346203b0a12 100644 --- a/pkg/planner/util/coretestsdk/testkit.go +++ b/pkg/planner/util/coretestsdk/testkit.go @@ -16,18 +16,13 @@ package coretestsdk import ( "context" - "fmt" "strings" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" pmodel "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/store/mockstore" - "github.com/pingcap/tidb/pkg/store/mockstore/unistore" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/testutils" ) // SetTiFlashReplica is to set TiFlash replica @@ -41,26 +36,6 @@ func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName strin } } -// WithMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). -func WithMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { - return mockstore.WithMultipleOptions( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockCluster := c.(*unistore.Cluster) - _, _, region1 := mockstore.BootstrapWithSingleStore(c) - tiflashIdx := 0 - for tiflashIdx < nodes { - store2 := c.AllocID() - peer2 := c.AllocID() - addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) - mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) - mockCluster.AddPeer(region1, store2, peer2) - tiflashIdx++ - } - }), - mockstore.WithStoreType(mockstore.EmbedUnistore), - ) -} - // GetFieldValue is to get field value. func GetFieldValue(prefix, row string) string { if idx := strings.Index(row, prefix); idx > 0 { diff --git a/pkg/planner/util/path.go b/pkg/planner/util/path.go index eebf112779b22..d8f88f27b7c9d 100644 --- a/pkg/planner/util/path.go +++ b/pkg/planner/util/path.go @@ -144,9 +144,14 @@ func (path *AccessPath) Clone() *AccessPath { return ret } -// IsTablePath returns true if it's IntHandlePath or CommonHandlePath. +// IsTablePath returns true if it's IntHandlePath or CommonHandlePath. Including tiflash table scan. func (path *AccessPath) IsTablePath() bool { - return path.IsIntHandlePath || path.IsCommonHandlePath + return path.IsIntHandlePath || path.IsCommonHandlePath || (path.Index != nil && path.StoreType == kv.TiFlash) +} + +// IsTiKVTablePath returns true if it's IntHandlePath or CommonHandlePath. And the store type is TiKV. +func (path *AccessPath) IsTiKVTablePath() bool { + return (path.IsIntHandlePath || path.IsCommonHandlePath) && path.StoreType == kv.TiKV } // SplitCorColAccessCondFromFilters move the necessary filter in the form of index_col = corrlated_col to access conditions. diff --git a/pkg/sessionctx/variable/varsutil.go b/pkg/sessionctx/variable/varsutil.go index dd19c3a08dc64..17146c1a95f8a 100644 --- a/pkg/sessionctx/variable/varsutil.go +++ b/pkg/sessionctx/variable/varsutil.go @@ -27,7 +27,6 @@ import ( "github.com/docker/go-units" "github.com/pingcap/errors" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -643,15 +642,3 @@ func parseSchemaCacheSize(s *SessionVars, normalizedValue string, originalValue return 0, "", ErrTruncatedWrongValue.GenWithStackByArgs(TiDBSchemaCacheSize, originalValue) } - -// DistanceMetric4VectorIndex stores distance metrics for the vector index. -var DistanceMetric4VectorIndex = map[string]model.DistanceMetric{ - ast.VecCosineDistance: model.DistanceMetricCosine, - ast.VecL2Distance: model.DistanceMetricL2, -} - -// Function4VectorIndex stores functions for the vector index. -var Function4VectorIndex = map[model.DistanceMetric]string{ - model.DistanceMetricCosine: ast.VecCosineDistance, - model.DistanceMetricL2: ast.VecL2Distance, -} diff --git a/pkg/store/mockstore/BUILD.bazel b/pkg/store/mockstore/BUILD.bazel index a9419cfc3d4c4..dd683ec168f81 100644 --- a/pkg/store/mockstore/BUILD.bazel +++ b/pkg/store/mockstore/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/testkit/testenv", "@com_github_otiai10_copy//:copy", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_tikv_client_go_v2//testutils", "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_client_go_v2//tikvrpc", diff --git a/pkg/store/mockstore/mockstore.go b/pkg/store/mockstore/mockstore.go index 63d3734136270..acf805c04e371 100644 --- a/pkg/store/mockstore/mockstore.go +++ b/pkg/store/mockstore/mockstore.go @@ -15,6 +15,7 @@ package mockstore import ( + "fmt" "net/url" "os" "path/filepath" @@ -22,6 +23,7 @@ import ( cp "github.com/otiai10/copy" "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/store/mockstore/unistore" @@ -177,6 +179,26 @@ func WithDDLChecker() MockTiKVStoreOption { } } +// WithMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). +func WithMockTiFlash(nodes int) MockTiKVStoreOption { + return WithMultipleOptions( + WithClusterInspector(func(c testutils.Cluster) { + mockCluster := c.(*unistore.Cluster) + _, _, region1 := BootstrapWithSingleStore(c) + tiflashIdx := 0 + for tiflashIdx < nodes { + store2 := c.AllocID() + peer2 := c.AllocID() + addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) + mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) + mockCluster.AddPeer(region1, store2, peer2) + tiflashIdx++ + } + }), + WithStoreType(EmbedUnistore), + ) +} + // DDLCheckerInjector is used to break import cycle. var DDLCheckerInjector func(kv.Storage) kv.Storage diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index 2c7611927175e..94853cc841553 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -561,6 +561,17 @@ func (tk *TestKit) MustUseIndex(sql string, index string, args ...any) bool { return false } +// MustNoIndexUsed checks if the result execution plan contains no index. +func (tk *TestKit) MustNoIndexUsed(sql string, args ...any) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Contains(rs.rows[i][3], "index:") { + return false + } + } + return true +} + // MustUseIndex4ExplainFor checks if the result execution plan contains specific index(es). func (tk *TestKit) MustUseIndex4ExplainFor(result *Result, index string) bool { for i := range result.rows { From e35fd47316bcc9fc3f62b258e03cb10d176c5b43 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 16:46:24 +0800 Subject: [PATCH 02/20] Pass column_id to TiFlash Signed-off-by: Wish --- go.mod | 4 ++-- go.sum | 4 ++-- pkg/planner/core/task.go | 1 + 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index d15640f4fc50a..8e3af9853b983 100644 --- a/go.mod +++ b/go.mod @@ -90,7 +90,7 @@ require ( github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 github.com/pingcap/tidb/pkg/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20240919023442-cf70966bef25 + github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837 github.com/prometheus/client_golang v1.20.4 github.com/prometheus/client_model v0.6.1 github.com/prometheus/common v0.57.0 @@ -307,7 +307,7 @@ require ( google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect google.golang.org/protobuf v1.34.2 gopkg.in/inf.v0 v0.9.1 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/apimachinery v0.28.6 // indirect k8s.io/klog/v2 v2.120.1 // indirect diff --git a/go.sum b/go.sum index 05d98bb1b4d10..e95703c2e19cf 100644 --- a/go.sum +++ b/go.sum @@ -678,8 +678,8 @@ github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfU github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 h1:T4pXRhBflzDeAhmOQHNPRRogMYxP13V7BkYw3ZsoSfE= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5/go.mod h1:rlimy0GcTvjiJqvD5mXTRr8O2eNZPBrcUgiWVYp9530= -github.com/pingcap/tipb v0.0.0-20240919023442-cf70966bef25 h1:qH261jtqDvqWhhsnle+eURWV+PmFdqEGP/aHDqPFJAs= -github.com/pingcap/tipb v0.0.0-20240919023442-cf70966bef25/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837 h1:tyIymn821fB8gUmqafdvLlcFkVOpgyJXImoYJ8n9oJE= +github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index c2b526bf9c869..3b14244438b2c 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -1056,6 +1056,7 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), TopK: uint32(p.Count), ColumnName: col.Name.L, + ColumnId: col.ID, RefVecF32: vs.Vec.SerializeTo(nil), IndexId: int64(ts.AnnIndexExtra.IndexInfo.ID), } From 2d015d85ef6691e5868059097c655841e5c49b3b Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 17:07:28 +0800 Subject: [PATCH 03/20] Cherry-pick https://github.com/tidbcloud/tidb-cse/pull/940 https://github.com/tidbcloud/tidb-cse/pull/920 Signed-off-by: Wish --- .../vectorsearch/vector_index_test.go | 87 +++++++++++++++++++ pkg/planner/core/explain.go | 10 ++- pkg/planner/core/plan_to_pb.go | 9 +- 3 files changed, 103 insertions(+), 3 deletions(-) diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index f9c28ac59ab34..6ee643a52b8fb 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -21,10 +21,13 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" + "github.com/pingcap/tidb/pkg/util/plancodec" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" ) @@ -97,3 +100,87 @@ func TestTiFlashANNIndex(t *testing.T) { require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } + +func TestANNIndexNormalizedPlan(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second, mockstore.WithMockTiFlash(2)) + + tk := testkit.NewTestKit(t, store) + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + + getNormalizedPlan := func() ([]string, string) { + info := tk.Session().ShowProcess() + require.NotNil(t, info) + p, ok := info.Plan.(base.Plan) + require.True(t, ok) + plan, digest := core.NormalizePlan(p) + + // test the new normalization code + flat := core.FlattenPhysicalPlan(p, false) + newNormalized, newDigest := core.NormalizeFlatPlan(flat) + require.Equal(t, plan, newNormalized) + require.Equal(t, digest, newDigest) + + normalizedPlan, err := plancodec.DecodeNormalizedPlan(plan) + normalizedPlanRows := getPlanRows(normalizedPlan) + require.NoError(t, err) + + return normalizedPlanRows, digest.String() + } + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(` + create table t ( + vec vector(3) + ) + `) + tk.MustExec("alter table t set tiflash replica 1;") + tk.MustExec("alter table t add vector index ((vec_cosine_distance(vec))) using hnsw;") + tk.MustExec(` + insert into t values + ('[1,1,1]'), + ('[2,2,2]'), + ('[3,3,3]') + `) + + tk.MustExec("analyze table t") + + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + tk.MustExec("explain select * from t order by vec_cosine_distance(vec, '[0,0,0]') limit 1") + p1, d1 := getNormalizedPlan() + require.Equal(t, []string{ + " Projection root test.t.vec", + " └─TopN root ?", + " └─Projection root test.t.vec, vec_cosine_distance(test.t.vec, ?)", + " └─TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Projection cop[tiflash] test.t.vec", + " └─TopN cop[tiflash] ?", + " └─Projection cop[tiflash] test.t.vec, vec_cosine_distance(test.t.vec, ?)", + " └─TableFullScan cop[tiflash] table:t, index:vector_index(vec), range:[?,?], keep order:false, annIndex:COSINE(vec..[?], limit:?)", + }, p1) + + tk.MustExec("explain select * from t order by vec_cosine_distance(vec, '[1,2,3]') limit 3") + _, d2 := getNormalizedPlan() + + tk.MustExec("explain select * from t order by vec_cosine_distance(vec, '[]') limit 3") + _, d3 := getNormalizedPlan() + + // Projection differs, so that normalized plan should differ. + tk.MustExec("explain select * from t order by vec_cosine_distance('[1,2,3]', vec) limit 3") + _, dx1 := getNormalizedPlan() + + require.Equal(t, d1, d2) + require.Equal(t, d1, d3) + require.NotEqual(t, d1, dx1) +} diff --git a/pkg/planner/core/explain.go b/pkg/planner/core/explain.go index e4d4aabb5cfd7..5e2b118943d47 100644 --- a/pkg/planner/core/explain.go +++ b/pkg/planner/core/explain.go @@ -276,7 +276,9 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { buffer.WriteString("(") buffer.WriteString(p.AnnIndexExtra.PushDownQueryInfo.GetColumnName()) buffer.WriteString("..") - { + if normalized { + buffer.WriteString("[?]") + } else { v, _, err := types.ZeroCopyDeserializeVectorFloat32(p.AnnIndexExtra.PushDownQueryInfo.RefVecF32) if err != nil { buffer.WriteString("[?]") @@ -285,7 +287,11 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { } } buffer.WriteString(", limit:") - buffer.WriteString(fmt.Sprint(p.AnnIndexExtra.PushDownQueryInfo.TopK)) + if normalized { + buffer.WriteString("?") + } else { + buffer.WriteString(fmt.Sprint(p.AnnIndexExtra.PushDownQueryInfo.TopK)) + } buffer.WriteString(")") } return buffer.String() diff --git a/pkg/planner/core/plan_to_pb.go b/pkg/planner/core/plan_to_pb.go index f67983ba3aaa6..1857cfad88deb 100644 --- a/pkg/planner/core/plan_to_pb.go +++ b/pkg/planner/core/plan_to_pb.go @@ -274,7 +274,8 @@ func (p *PhysicalTableScan) ToPB(ctx *base.BuildPBContext, storeType kv.StoreTyp } if p.AnnIndexExtra != nil && p.AnnIndexExtra.PushDownQueryInfo != nil { - tsExec.AnnQuery = p.AnnIndexExtra.PushDownQueryInfo + annQueryCopy := *p.AnnIndexExtra.PushDownQueryInfo + tsExec.AnnQuery = &annQueryCopy } var err error @@ -316,6 +317,12 @@ func (p *PhysicalTableScan) partitionTableScanToPBForFlash(ctx *base.BuildPBCont ptsExec.MaxWaitTimeMs = int32(p.maxWaitTimeMs) ptsExec.Desc = p.Desc + + if p.AnnIndexExtra != nil && p.AnnIndexExtra.PushDownQueryInfo != nil { + annQueryCopy := *p.AnnIndexExtra.PushDownQueryInfo + ptsExec.AnnQuery = &annQueryCopy + } + executorID := p.ExplainID().String() err = tables.SetPBColumnsDefaultValue(ctx.GetExprCtx(), ptsExec.Columns, p.Columns) return &tipb.Executor{Tp: tipb.ExecType_TypePartitionTableScan, PartitionTableScan: ptsExec, ExecutorId: &executorID}, err From bf3a8e75eea5eb466f2bcd6a1fbfce1b9cd63269 Mon Sep 17 00:00:00 2001 From: Yiding Date: Tue, 8 Oct 2024 17:51:05 +0800 Subject: [PATCH 04/20] address comments --- pkg/ddl/index.go | 2 +- pkg/executor/show.go | 2 +- pkg/expression/vs_helper.go | 40 ++++++++++++++----------------------- pkg/meta/model/index.go | 8 ++++---- pkg/planner/core/task.go | 21 +++++++++---------- 5 files changed, 30 insertions(+), 43 deletions(-) diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 54eddf2e28d31..53ecfb002352a 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -385,7 +385,7 @@ func buildVectorInfoWithCheck(indexPartSpecifications []*ast.IndexPartSpecificat if !ok { return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs(fmt.Sprintf("unsupported function: %v", idxPart.Expr)) } - distanceMetric, ok := model.FnNameToDistanceMetric[f.FnName.L] + distanceMetric, ok := model.IndexableFnNameToDistanceMetric[f.FnName.L] if !ok { return nil, "", dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("currently only L2 and Cosine distance is indexable") } diff --git a/pkg/executor/show.go b/pkg/executor/show.go index ed5f9f1f29a0c..7a5630167ad57 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -1230,7 +1230,7 @@ func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *pmodel.CIS cols = append(cols, colInfo) } if idxInfo.VectorInfo != nil { - funcName := model.DistanceMetricToFnName[idxInfo.VectorInfo.DistanceMetric] + funcName := model.IndexableDistanceMetricToFnName[idxInfo.VectorInfo.DistanceMetric] fmt.Fprintf(buf, "((%s(%s)))", strings.ToUpper(funcName), strings.Join(cols, ",")) } else { fmt.Fprintf(buf, "(%s)", strings.Join(cols, ",")) diff --git a/pkg/expression/vs_helper.go b/pkg/expression/vs_helper.go index 475a0f30e16a1..18a825e8891d7 100644 --- a/pkg/expression/vs_helper.go +++ b/pkg/expression/vs_helper.go @@ -17,11 +17,11 @@ package expression import ( "strings" - "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/intest" ) var ( @@ -33,35 +33,28 @@ var ( } ) -// VectorSearchExpr defines a minimal Vector Search expression, which is -// a vector distance function, a column to search with, and a reference vector. -type VectorSearchExpr struct { +// VectorHelper is a helper struct for vector indexes. +type VectorHelper struct { DistanceFnName model.CIStr Vec types.VectorFloat32 - Column *Column + ColumnID int64 } -// ExtractVectorSearch extracts a VectorSearchExpr from an expression. +// ExtractVectorHelper extracts a VectorSearchExpr from an expression. // NOTE: not all VectorSearch functions are supported by the index. The caller // needs to check the distance function name. -func ExtractVectorSearch(expr Expression) (*VectorSearchExpr, error) { +func ExtractVectorHelper(expr Expression) *VectorHelper { x, ok := expr.(*ScalarFunction) if !ok { - return nil, nil + return nil } if _, isVecFn := vsDistanceFnNamesLower[x.FuncName.L]; !isVecFn { - return nil, nil + return nil } args := x.GetArgs() - if len(args) != 2 { - return nil, errors.Errorf("internal: expect 2 args for function %s, but got %d", x.FuncName.L, len(args)) - } - // One arg must be a vector column ref, and one arg must be a vector constant. - // Note: this must be run after constant folding. - var vectorConstant *Constant = nil var vectorColumn *Column = nil nVectorColumns := 0 @@ -69,30 +62,27 @@ func ExtractVectorSearch(expr Expression) (*VectorSearchExpr, error) { for _, arg := range args { if v, ok := arg.(*Column); ok { if v.RetType.GetType() != mysql.TypeTiDBVectorFloat32 { - break + return nil } vectorColumn = v nVectorColumns++ } else if v, ok := arg.(*Constant); ok { if v.RetType.GetType() != mysql.TypeTiDBVectorFloat32 { - break + return nil } vectorConstant = v nVectorConstants++ } } if nVectorColumns != 1 || nVectorConstants != 1 { - return nil, nil + return nil } - // All check passed. - if vectorConstant.Value.Kind() != types.KindVectorFloat32 { - return nil, errors.Errorf("internal: expect vectorFloat32 constant, but got %s", vectorConstant.Value.String()) - } + intest.Assert(vectorConstant.Value.Kind() == types.KindVectorFloat32, "internal: expect vectorFloat32 constant, but got %s", vectorConstant.Value.String()) - return &VectorSearchExpr{ + return &VectorHelper{ DistanceFnName: x.FuncName, Vec: vectorConstant.Value.GetVectorFloat32(), - Column: vectorColumn, - }, nil + ColumnID: vectorColumn.ID, + } } diff --git a/pkg/meta/model/index.go b/pkg/meta/model/index.go index ea812580dd494..1121373fef78e 100644 --- a/pkg/meta/model/index.go +++ b/pkg/meta/model/index.go @@ -37,15 +37,15 @@ const ( DistanceMetricInnerProduct DistanceMetric = "INNER_PRODUCT" ) -// FnNameToDistanceMetric maps a distance function name to the distance metric. +// IndexableFnNameToDistanceMetric maps a distance function name to the distance metric. // Only indexable distance functions should be listed here! -var FnNameToDistanceMetric = map[string]DistanceMetric{ +var IndexableFnNameToDistanceMetric = map[string]DistanceMetric{ ast.VecCosineDistance: DistanceMetricCosine, ast.VecL2Distance: DistanceMetricL2, } -// DistanceMetricToFnName maps a distance metric to the distance function name. -var DistanceMetricToFnName = map[DistanceMetric]string{ +// IndexableDistanceMetricToFnName maps a distance metric to the distance function name. +var IndexableDistanceMetricToFnName = map[DistanceMetric]string{ DistanceMetricCosine: ast.VecCosineDistance, DistanceMetricL2: ast.VecL2Distance, } diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index c2b526bf9c869..7f06ae1646f98 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/paging" "github.com/pingcap/tidb/pkg/util/plancodec" @@ -1024,33 +1025,27 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { // supported yet. return false } - vs, err := expression.ExtractVectorSearch(p.ByItems[0].Expr) - if err != nil || vs == nil { + vs := expression.ExtractVectorHelper(p.ByItems[0].Expr) + if vs == nil { return false } // Note that even if this is a vector search expression, it may not hit vector index // because not all vector search functions are indexable. - distanceMetric, ok := model.FnNameToDistanceMetric[vs.DistanceFnName.L] - if !ok { - return false - } + distanceMetric, ok := model.IndexableFnNameToDistanceMetric[vs.DistanceFnName.L] // User may build a vector index with different distance metric. // In this case the index shall not push down. - if distanceMetric != ts.AnnIndexExtra.IndexInfo.VectorInfo.DistanceMetric { + if !ok || distanceMetric != ts.AnnIndexExtra.IndexInfo.VectorInfo.DistanceMetric { return false } // User may build a vector index with different vector column. // In this case the index shall not push down. col := ts.Table.Columns[ts.AnnIndexExtra.IndexInfo.Columns[0].Offset] - if col.ID != vs.Column.ID { + if col.ID != vs.ColumnID { return false } distanceMetricPB, ok := tipb.VectorDistanceMetric_value[string(distanceMetric)] - if !ok { - // This should not happen. - return false - } + intest.Assert(distanceMetricPB != 0, "invalid distance metric") ts.AnnIndexExtra.PushDownQueryInfo = &tipb.ANNQueryInfo{ QueryType: tipb.ANNQueryType_OrderBy, DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), @@ -1059,6 +1054,8 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { RefVecF32: vs.Vec.SerializeTo(nil), IndexId: int64(ts.AnnIndexExtra.IndexInfo.ID), } + ts.AnnIndexExtra.PushDownQueryInfo.ColumnId = new(int64) + *ts.AnnIndexExtra.PushDownQueryInfo.ColumnId = vs.ColumnID ts.PlanCostInit = false return true } From a0b175583ef855b23352bbc6ed7ca67513256e73 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 20:41:07 +0800 Subject: [PATCH 05/20] Fix lint Signed-off-by: Wish --- pkg/planner/core/task.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index b6e5ac0175469..81f943f734835 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -1045,18 +1045,17 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { } distanceMetricPB, ok := tipb.VectorDistanceMetric_value[string(distanceMetric)] - intest.Assert(distanceMetricPB != 0, "invalid distance metric") + intest.Assert(ok && distanceMetricPB != 0, "invalid distance metric") + ts.AnnIndexExtra.PushDownQueryInfo = &tipb.ANNQueryInfo{ QueryType: tipb.ANNQueryType_OrderBy, DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), TopK: uint32(p.Count), ColumnName: col.Name.L, ColumnId: col.ID, + IndexId: ts.AnnIndexExtra.IndexInfo.ID, RefVecF32: vs.Vec.SerializeTo(nil), - IndexId: int64(ts.AnnIndexExtra.IndexInfo.ID), } - ts.AnnIndexExtra.PushDownQueryInfo.ColumnId = new(int64) - *ts.AnnIndexExtra.PushDownQueryInfo.ColumnId = vs.ColumnID ts.PlanCostInit = false return true } From c638b2638281717a19129a239efa3038b569942d Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 20:47:41 +0800 Subject: [PATCH 06/20] Fix bazel Signed-off-by: Wish --- DEPS.bzl | 12 ++++++------ pkg/planner/core/casetest/vectorsearch/BUILD.bazel | 4 ++++ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index fb4713a61069e..ab92bf380e351 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -5906,13 +5906,13 @@ def go_deps(): name = "com_github_pingcap_tipb", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/tipb", - sha256 = "119f73cede33a7d3712a9fedeb80f806228ecf1ec5258112b2b62af88731a304", - strip_prefix = "github.com/pingcap/tipb@v0.0.0-20240919023442-cf70966bef25", + sha256 = "b39e154272ba36d145c6049947a012a76be740b32a44a46d7253caa145c56cc9", + strip_prefix = "github.com/pingcap/tipb@v0.0.0-20241008083645-0bcddae67837", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20240919023442-cf70966bef25.zip", - "http://ats.apps.svc/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20240919023442-cf70966bef25.zip", - "https://cache.hawkingrei.com/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20240919023442-cf70966bef25.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20240919023442-cf70966bef25.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20241008083645-0bcddae67837.zip", + "http://ats.apps.svc/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20241008083645-0bcddae67837.zip", + "https://cache.hawkingrei.com/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20241008083645-0bcddae67837.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/tipb/com_github_pingcap_tipb-v0.0.0-20241008083645-0bcddae67837.zip", ], ) go_repository( diff --git a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel index b15a17c086bb0..4d50f3160dc3f 100644 --- a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel +++ b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel @@ -9,16 +9,20 @@ go_test( ], data = glob(["testdata/**"]), flaky = True, + shard_count = 3, deps = [ "//pkg/config", "//pkg/domain/infosync", "//pkg/meta/model", + "//pkg/planner/core", + "//pkg/planner/core/base", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testfailpoint", "//pkg/testkit/testmain", "//pkg/testkit/testsetup", + "//pkg/util/plancodec", "@com_github_pingcap_tipb//go-tipb", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", From 0650ca543cd843152dc3ac8663fae8b79ae8b136 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 21:33:25 +0800 Subject: [PATCH 07/20] Fix Signed-off-by: Wish --- pkg/planner/core/indexmerge_path_test.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pkg/planner/core/indexmerge_path_test.go b/pkg/planner/core/indexmerge_path_test.go index 572698a71f7ea..776c449f5228f 100644 --- a/pkg/planner/core/indexmerge_path_test.go +++ b/pkg/planner/core/indexmerge_path_test.go @@ -20,7 +20,10 @@ import ( "math/rand" "strings" "testing" + "time" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" @@ -30,8 +33,8 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/resolve" - "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/util/hint" @@ -440,7 +443,7 @@ func randMVIndexValue(opts randMVIndexValOpts) string { } func TestAnalyzeVectorIndex(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 200*time.Millisecond, coretestsdk.WithMockTiFlash(2)) + store, dom := testkit.CreateMockStoreAndDomainWithSchemaLease(t, 200*time.Millisecond, mockstore.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") From 19748a57fc10123f008739745a64a58634e06639 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 21:42:07 +0800 Subject: [PATCH 08/20] Fix Signed-off-by: Wish --- pkg/planner/core/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index fd080c5d754f1..8967a4df154d9 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -267,6 +267,7 @@ go_test( deps = [ "//pkg/config", "//pkg/domain", + "//pkg/domain/infosync", "//pkg/expression", "//pkg/expression/aggregation", "//pkg/expression/exprctx", From b73f16bde9e97d85aa69ef7025efdf24dd031fc7 Mon Sep 17 00:00:00 2001 From: Wish Date: Tue, 8 Oct 2024 22:06:30 +0800 Subject: [PATCH 09/20] Regenerate Signed-off-by: Wish --- pkg/planner/core/plan_clone_generated.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/planner/core/plan_clone_generated.go b/pkg/planner/core/plan_clone_generated.go index c6ed47e2026ed..ae7ae7591c02e 100644 --- a/pkg/planner/core/plan_clone_generated.go +++ b/pkg/planner/core/plan_clone_generated.go @@ -50,6 +50,9 @@ func (op *PhysicalTableScan) CloneForPlanCache(newCtx base.PlanContext) (base.Pl if op.runtimeFilterList != nil { return nil, false } + if op.AnnIndexExtra != nil { + return nil, false + } return cloned, true } From 75c621fafb239e1356a09dfb7e226c195ff68052 Mon Sep 17 00:00:00 2001 From: Yiding Date: Tue, 8 Oct 2024 22:29:13 +0800 Subject: [PATCH 10/20] fix row count --- pkg/planner/core/task.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 81f943f734835..99b2cc45aa249 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -1056,6 +1056,7 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { IndexId: ts.AnnIndexExtra.IndexInfo.ID, RefVecF32: vs.Vec.SerializeTo(nil), } + util.DeriveLimitStats(ts.StatsInfo(), float64(p.Count)) ts.PlanCostInit = false return true } From 1e3853ce9ffed9bd0deed3e8bc74e74452d4583e Mon Sep 17 00:00:00 2001 From: Wish Date: Wed, 9 Oct 2024 10:55:20 +0800 Subject: [PATCH 11/20] Fix Signed-off-by: Wish --- pkg/planner/core/task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 99b2cc45aa249..787f4e33ec445 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -1056,7 +1056,7 @@ func fixTopNForANNIndex(p *PhysicalTopN) bool { IndexId: ts.AnnIndexExtra.IndexInfo.ID, RefVecF32: vs.Vec.SerializeTo(nil), } - util.DeriveLimitStats(ts.StatsInfo(), float64(p.Count)) + ts.SetStats(util.DeriveLimitStats(ts.StatsInfo(), float64(p.Count))) ts.PlanCostInit = false return true } From 1e59ad203ea703b0e514dea60b8969c11aad6a60 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 15:19:27 +0800 Subject: [PATCH 12/20] correct USE INDEX's behavior --- pkg/planner/core/planbuilder.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 716b1ecc0331d..5c8e7f69cd947 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -1073,12 +1073,12 @@ func (*PlanBuilder) detectSelectWindow(sel *ast.SelectStmt) bool { } func getPathByIndexName(paths []*util.AccessPath, idxName pmodel.CIStr, tblInfo *model.TableInfo) *util.AccessPath { - var primaryIdxPath, indexPrefixPath *util.AccessPath + var indexPrefixPath *util.AccessPath prefixMatches := 0 for _, path := range paths { - if path.IsTiKVTablePath() { - primaryIdxPath = path - continue + // Only accept tikv's primary key table path. + if path.IsTiKVTablePath() && isPrimaryIndex(idxName) && tblInfo.HasClusteredIndex() { + return path } // If it's not a tikv table path and the index is nil, it could not be any index path. if path.Index == nil { @@ -1092,9 +1092,6 @@ func getPathByIndexName(paths []*util.AccessPath, idxName pmodel.CIStr, tblInfo prefixMatches++ } } - if isPrimaryIndex(idxName) && tblInfo.HasClusteredIndex() { - return primaryIdxPath - } // Return only unique prefix matches if prefixMatches == 1 { From fb5dfffb8eee88956139fae5c3c1d055ad58c0a8 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 15:19:43 +0800 Subject: [PATCH 13/20] update tests --- .../testdata/ann_index_suite_out.json | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json index 0fe3d710d4065..cc3466f3a68fd 100644 --- a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json +++ b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json @@ -73,8 +73,8 @@ " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", - " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, @@ -88,8 +88,8 @@ " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", - " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, @@ -235,8 +235,8 @@ " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection_19 1.00 mpp[tiflash] test.t1.vec", " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", - " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, @@ -251,8 +251,8 @@ " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b", " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", - " └─Projection_20 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableFullScan_15 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, From b0d4767b0c805a706181ef3c5ace9cd91b28f258 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 19:54:43 +0800 Subject: [PATCH 14/20] re-implement the logic --- pkg/expression/vs_helper.go | 7 ++- .../core/casetest/vectorsearch/BUILD.bazel | 2 +- .../vectorsearch/vector_index_test.go | 32 ++++++++++ pkg/planner/core/exhaust_physical_plans.go | 37 ++++++++++- pkg/planner/core/find_best_task.go | 41 +++++++++++-- pkg/planner/core/task.go | 61 ------------------- pkg/planner/property/physical_property.go | 11 ++++ 7 files changed, 122 insertions(+), 69 deletions(-) diff --git a/pkg/expression/vs_helper.go b/pkg/expression/vs_helper.go index 18a825e8891d7..32c5f84304503 100644 --- a/pkg/expression/vs_helper.go +++ b/pkg/expression/vs_helper.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/intest" + "github.com/pingcap/tipb/go-tipb" ) var ( @@ -36,8 +37,9 @@ var ( // VectorHelper is a helper struct for vector indexes. type VectorHelper struct { DistanceFnName model.CIStr + FnPbCode tipb.ScalarFuncSig Vec types.VectorFloat32 - ColumnID int64 + Column *Column } // ExtractVectorHelper extracts a VectorSearchExpr from an expression. @@ -82,7 +84,8 @@ func ExtractVectorHelper(expr Expression) *VectorHelper { return &VectorHelper{ DistanceFnName: x.FuncName, + FnPbCode: x.Function.PbCode(), Vec: vectorConstant.Value.GetVectorFloat32(), - ColumnID: vectorColumn.ID, + Column: vectorColumn, } } diff --git a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel index 4d50f3160dc3f..ad20378434946 100644 --- a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel +++ b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel @@ -9,7 +9,7 @@ go_test( ], data = glob(["testdata/**"]), flaky = True, - shard_count = 3, + shard_count = 4, deps = [ "//pkg/config", "//pkg/domain/infosync", diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index 6ee643a52b8fb..0b8219af44ef0 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -184,3 +184,35 @@ func TestANNIndexNormalizedPlan(t *testing.T) { require.Equal(t, d1, d3) require.NotEqual(t, d1, dx1) } + +func TestANNInexWithSimpleCBO(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second, mockstore.WithMockTiFlash(2)) + + tk := testkit.NewTestKit(t, store) + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec(` + create table t1 ( + vec vector(3), + a int, + b int, + c vector(3), + d vector + ) + `) + tk.MustExec("alter table t1 set tiflash replica 1;") + tk.MustExec("alter table t1 add vector index ((vec_cosine_distance(vec))) USING HNSW;") + tk.MustUseIndex("select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", "vector_index") + tk.MustNoIndexUsed("select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 10000") +} diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index 4f98146f46a74..68c5229336d89 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -2169,7 +2169,8 @@ func getPhysTopN(lt *logicalop.LogicalTopN, prop *property.PhysicalProperty) []b if !pushLimitOrTopNForcibly(lt) { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if lt.SCtx().GetSessionVars().IsMPPAllowed() { + mppAllowed := lt.SCtx().GetSessionVars().IsMPPAllowed() + if mppAllowed { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]base.PhysicalPlan, 0, len(allTaskTypes)) @@ -2183,6 +2184,40 @@ func getPhysTopN(lt *logicalop.LogicalTopN, prop *property.PhysicalProperty) []b }.Init(lt.SCtx(), lt.StatsInfo(), lt.QueryBlockOffset(), resultProp) ret = append(ret, topN) } + // If we can generate MPP task and there's vector distance function in the order by column. + // We will try to generate a property for possible vector indexes. + if mppAllowed { + if len(lt.ByItems) > 1 { + return ret + } + vs := expression.ExtractVectorHelper(lt.ByItems[0].Expr) + if vs == nil { + return ret + } + // Currently vector index only accept ascending order. + if lt.ByItems[0].Desc { + return ret + } + // Currently, we only deal with the case the TopN is directly above a DataSource. + _, ok := lt.Children()[0].(*logicalop.DataSource) + if !ok { + return ret + } + resultProp := &property.PhysicalProperty{ + TaskTp: property.MppTaskType, + ExpectedCnt: math.MaxFloat64, + CTEProducerStatus: prop.CTEProducerStatus, + } + resultProp.VectorProp.VectorHelper = vs + resultProp.VectorProp.TopK = uint32(lt.Count + lt.Offset) + topN := PhysicalTopN{ + ByItems: lt.ByItems, + PartitionBy: lt.PartitionBy, + Count: lt.Count, + Offset: lt.Offset, + }.Init(lt.SCtx(), lt.StatsInfo(), lt.QueryBlockOffset(), resultProp) + ret = append(ret, topN) + } return ret } diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 8564d66654465..e08eb14c38b8c 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -43,9 +43,11 @@ import ( "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" h "github.com/pingcap/tidb/pkg/util/hint" + "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/ranger" "github.com/pingcap/tidb/pkg/util/tracing" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -806,6 +808,19 @@ func isMatchProp(ds *logicalop.DataSource, path *util.AccessPath, prop *property } } } + if prop.VectorProp.VectorHelper != nil && path.Index.VectorInfo != nil { + if path.Index == nil || path.Index.VectorInfo == nil { + return false + } + if ds.TableInfo.Columns[path.Index.Columns[0].Offset].ID != prop.VectorProp.Column.ID { + return false + } + + if model.IndexableFnNameToDistanceMetric[prop.VectorProp.DistanceFnName.L] != path.Index.VectorInfo.DistanceMetric { + return false + } + return true + } return isMatchProp } @@ -2503,11 +2518,29 @@ func convertToTableScan(ds *logicalop.DataSource, prop *property.PhysicalPropert if prop.MPPPartitionTp != property.AnyType { return base.InvalidTask, nil } + if candidate.path.Index != nil && !candidate.isMatchProp { + return base.InvalidTask, nil + } if candidate.path.Index != nil && candidate.path.Index.VectorInfo != nil { + // Only the corresponding index can generate a valid task. + intest.Assert(ts.Table.Columns[candidate.path.Index.Columns[0].Offset].ID == prop.VectorProp.Column.ID, "The passed vector column is not matched with the index") + distanceMetric := model.IndexableFnNameToDistanceMetric[prop.VectorProp.DistanceFnName.L] + distanceMetricPB := tipb.VectorDistanceMetric_value[string(distanceMetric)] + intest.Assert(distanceMetricPB != 0, "unexpected distance metric") + ts.AnnIndexExtra = &VectorIndexExtra{ - IndexInfo: candidate.path.Index, - PushDownQueryInfo: nil, - } + IndexInfo: candidate.path.Index, + PushDownQueryInfo: &tipb.ANNQueryInfo{ + QueryType: tipb.ANNQueryType_OrderBy, + DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), + TopK: prop.VectorProp.TopK, + ColumnName: prop.VectorProp.Column.OrigName, + ColumnId: prop.VectorProp.Column.ID, + IndexId: candidate.path.Index.ID, + RefVecF32: prop.VectorProp.Vec.SerializeTo(nil), + }, + } + ts.SetStats(util.DeriveLimitStats(ts.StatsInfo(), float64(prop.VectorProp.TopK))) } // ********************************** future deprecated start **************************/ var hasVirtualColumn bool @@ -2877,7 +2910,7 @@ func getOriginalPhysicalTableScan(ds *logicalop.DataSource, prop *property.Physi if usedStats != nil && usedStats.GetUsedInfo(ts.physicalTableID) != nil { ts.usedStatsInfo = usedStats.GetUsedInfo(ts.physicalTableID) } - if isMatchProp { + if isMatchProp && prop.VectorProp.VectorHelper == nil { ts.Desc = prop.SortItems[0].Desc ts.KeepOrder = true } diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index 787f4e33ec445..14c49e3ae78b7 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -36,11 +35,9 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" - "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/paging" "github.com/pingcap/tidb/pkg/util/plancodec" - "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -993,9 +990,6 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { } else if mppTask, ok := t.(*MppTask); ok && needPushDown && p.canPushDownToTiFlash(mppTask) { pushedDownTopN := p.getPushedDownTopN(mppTask.p) mppTask.p = pushedDownTopN - if !fixTopNForANNIndex(pushedDownTopN) { - return base.InvalidTask - } } rootTask := t.ConvertToRootTask(p.SCtx()) // Skip TopN with partition on the root. This is a derived topN and window function @@ -1006,61 +1000,6 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { return attachPlan2Task(p, rootTask) } -func fixTopNForANNIndex(p *PhysicalTopN) bool { - ts, ok := p.Children()[0].(*PhysicalTableScan) - if !ok || ts.AnnIndexExtra == nil { - return true - } - if ts.AnnIndexExtra.IndexInfo.VectorInfo == nil { - // AnnIndex is attached with a non-vector index. - // This should not happen. - return false - } - if len(p.ByItems) != 1 { - return false - } - if p.ByItems[0].Desc { - // Currently all vector search indexes must be used for ASC order. - // DESC order can be only applied to INNER_PRODUCT, which is not - // supported yet. - return false - } - vs := expression.ExtractVectorHelper(p.ByItems[0].Expr) - if vs == nil { - return false - } - // Note that even if this is a vector search expression, it may not hit vector index - // because not all vector search functions are indexable. - distanceMetric, ok := model.IndexableFnNameToDistanceMetric[vs.DistanceFnName.L] - // User may build a vector index with different distance metric. - // In this case the index shall not push down. - if !ok || distanceMetric != ts.AnnIndexExtra.IndexInfo.VectorInfo.DistanceMetric { - return false - } - // User may build a vector index with different vector column. - // In this case the index shall not push down. - col := ts.Table.Columns[ts.AnnIndexExtra.IndexInfo.Columns[0].Offset] - if col.ID != vs.ColumnID { - return false - } - - distanceMetricPB, ok := tipb.VectorDistanceMetric_value[string(distanceMetric)] - intest.Assert(ok && distanceMetricPB != 0, "invalid distance metric") - - ts.AnnIndexExtra.PushDownQueryInfo = &tipb.ANNQueryInfo{ - QueryType: tipb.ANNQueryType_OrderBy, - DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), - TopK: uint32(p.Count), - ColumnName: col.Name.L, - ColumnId: col.ID, - IndexId: ts.AnnIndexExtra.IndexInfo.ID, - RefVecF32: vs.Vec.SerializeTo(nil), - } - ts.SetStats(util.DeriveLimitStats(ts.StatsInfo(), float64(p.Count))) - ts.PlanCostInit = false - return true -} - // Attach2Task implements the PhysicalPlan interface. func (p *PhysicalExpand) Attach2Task(tasks ...base.Task) base.Task { t := tasks[0].Copy() diff --git a/pkg/planner/property/physical_property.go b/pkg/planner/property/physical_property.go index 6953d7cc424c9..669fda907159c 100644 --- a/pkg/planner/property/physical_property.go +++ b/pkg/planner/property/physical_property.go @@ -221,6 +221,11 @@ type PhysicalProperty struct { RejectSort bool CTEProducerStatus cteProducerStatus + + VectorProp struct { + *expression.VectorHelper + TopK uint32 + } } // NewPhysicalProperty builds property from columns. @@ -349,6 +354,12 @@ func (p *PhysicalProperty) HashCode() []byte { for _, col := range p.MPPPartitionCols { p.hashcode = append(p.hashcode, col.hashCode()...) } + if p.VectorProp.VectorHelper != nil { + // We only accpect the vector information from the TopN which is directly above the DataSource. + // So it's safe to not hash the vector constant. + p.hashcode = append(p.hashcode, p.VectorProp.Column.HashCode()...) + p.hashcode = codec.EncodeInt(p.hashcode, int64(p.VectorProp.FnPbCode)) + } } p.hashcode = append(p.hashcode, codec.EncodeInt(nil, int64(p.CTEProducerStatus))...) return p.hashcode From e8074a24fd54f71c82c501bb31ccb5eb10637c8a Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 20:34:13 +0800 Subject: [PATCH 15/20] fix tests --- pkg/planner/core/exhaust_physical_plans.go | 2 +- pkg/planner/core/find_best_task.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index 68c5229336d89..ccfb17bd17aca 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -2187,7 +2187,7 @@ func getPhysTopN(lt *logicalop.LogicalTopN, prop *property.PhysicalProperty) []b // If we can generate MPP task and there's vector distance function in the order by column. // We will try to generate a property for possible vector indexes. if mppAllowed { - if len(lt.ByItems) > 1 { + if len(lt.ByItems) != 1 { return ret } vs := expression.ExtractVectorHelper(lt.ByItems[0].Expr) diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index e08eb14c38b8c..a609876058724 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2518,7 +2518,8 @@ func convertToTableScan(ds *logicalop.DataSource, prop *property.PhysicalPropert if prop.MPPPartitionTp != property.AnyType { return base.InvalidTask, nil } - if candidate.path.Index != nil && !candidate.isMatchProp { + // If it has vector property, we need to check the candidate.isMatchProp. + if candidate.path.Index != nil && candidate.path.Index.VectorInfo != nil && !candidate.isMatchProp { return base.InvalidTask, nil } if candidate.path.Index != nil && candidate.path.Index.VectorInfo != nil { From bce942f354a522d928b500984dcd4d5696cae070 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 22:42:28 +0800 Subject: [PATCH 16/20] fix tests --- .../integration_test/integration_test.go | 32 +- pkg/planner/core/casetest/index/BUILD.bazel | 1 + pkg/planner/core/casetest/index/index_test.go | 8 +- .../core/casetest/vectorsearch/BUILD.bazel | 2 + .../testdata/ann_index_suite_in.json | 44 +-- .../testdata/ann_index_suite_out.json | 348 ++++++++++-------- .../vectorsearch/vector_index_test.go | 4 + pkg/planner/core/exhaust_physical_plans.go | 6 +- pkg/planner/core/find_best_task.go | 2 +- pkg/planner/core/planbuilder.go | 2 +- 10 files changed, 246 insertions(+), 203 deletions(-) diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 64dd807171fbd..b35d3a74240f5 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -327,24 +327,24 @@ func TestVectorConstantExplain(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE t(c VECTOR);") - tk.MustQuery(`EXPLAIN SELECT VEC_COSINE_DISTANCE(c, '[1,2,3,4,5,6,7,8,9,10,11]') FROM t;`).Check(testkit.Rows( - "Projection_3 10000.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", - "└─TableReader_5 10000.00 root data:TableFullScan_4", - " └─TableFullScan_4 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + tk.MustQuery(`EXPLAIN format='brief' SELECT VEC_COSINE_DISTANCE(c, '[1,2,3,4,5,6,7,8,9,10,11]') FROM t;`).Check(testkit.Rows( + "Projection 10000.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", )) - tk.MustQuery(`EXPLAIN SELECT VEC_COSINE_DISTANCE(c, VEC_FROM_TEXT('[1,2,3,4,5,6,7,8,9,10,11]')) FROM t;`).Check(testkit.Rows( - "Projection_3 10000.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", - "└─TableReader_5 10000.00 root data:TableFullScan_4", - " └─TableFullScan_4 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + tk.MustQuery(`EXPLAIN format='brief' SELECT VEC_COSINE_DISTANCE(c, VEC_FROM_TEXT('[1,2,3,4,5,6,7,8,9,10,11]')) FROM t;`).Check(testkit.Rows( + "Projection 10000.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", )) - tk.MustQuery(`EXPLAIN SELECT VEC_COSINE_DISTANCE(c, '[1,2,3,4,5,6,7,8,9,10,11]') AS d FROM t ORDER BY d LIMIT 10;`).Check(testkit.Rows( - "Projection_6 10.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", - "└─Projection_13 10.00 root test.t.c", - " └─TopN_7 10.00 root Column#4, offset:0, count:10", - " └─Projection_14 10.00 root test.t.c, vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#4", - " └─TableReader_12 10.00 root data:TopN_11", - " └─TopN_11 10.00 cop[tikv] vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...]), offset:0, count:10", - " └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + tk.MustQuery(`EXPLAIN format = 'brief' SELECT VEC_COSINE_DISTANCE(c, '[1,2,3,4,5,6,7,8,9,10,11]') AS d FROM t ORDER BY d LIMIT 10;`).Check(testkit.Rows( + "Projection 10.00 root vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#3", + "└─Projection 10.00 root test.t.c", + " └─TopN 10.00 root Column#4, offset:0, count:10", + " └─Projection 10.00 root test.t.c, vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...])->Column#4", + " └─TableReader 10.00 root data:TopN", + " └─TopN 10.00 cop[tikv] vec_cosine_distance(test.t.c, [1,2,3,4,5,(6 more)...]), offset:0, count:10", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", )) // Prepare a large Vector string diff --git a/pkg/planner/core/casetest/index/BUILD.bazel b/pkg/planner/core/casetest/index/BUILD.bazel index e4bb65791f767..5b69edf638b12 100644 --- a/pkg/planner/core/casetest/index/BUILD.bazel +++ b/pkg/planner/core/casetest/index/BUILD.bazel @@ -14,6 +14,7 @@ go_test( "//pkg/domain", "//pkg/domain/infosync", "//pkg/parser/model", + "//pkg/planner/util/coretestsdk", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/index/index_test.go b/pkg/planner/core/casetest/index/index_test.go index a5348daae9854..f273bd8ddb738 100644 --- a/pkg/planner/core/casetest/index/index_test.go +++ b/pkg/planner/core/casetest/index/index_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -261,8 +262,8 @@ func TestVectorIndex(t *testing.T) { tk.MustExec("alter table t add vector index vecIdx1((vec_cosine_distance(d))) USING HNSW;") tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance(d, '[1,1,1,1]') limit 1", "vecIdx1") tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance('[1,1,1,1]', d) limit 1", "vecIdx1") - tk.MustNoIndexUsed("select * from t use index(vecIdx1) order by vec_l2_distance(d, '[1,1,1,1]') limit 1") - tk.MustNoIndexUsed("select * from t use index(vecIdx1) where a = 5 order by vec_cosine_distance(d, '[1,1,1,1]') limit 1") + tk.MustExecToErr("select * from t use index(vecIdx1) order by vec_l2_distance(d, '[1,1,1,1]') limit 1") + tk.MustExecToErr("select * from t use index(vecIdx1) where a = 5 order by vec_cosine_distance(d, '[1,1,1,1]') limit 1") } func TestAnalyzeVectorIndex(t *testing.T) { @@ -284,8 +285,9 @@ func TestAnalyzeVectorIndex(t *testing.T) { require.NoError(t, err) err = domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tblInfo.Meta().ID, true) require.NoError(t, err) + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") - 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/planner/core/casetest/vectorsearch/BUILD.bazel b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel index ad20378434946..d311e27898f55 100644 --- a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel +++ b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel @@ -12,10 +12,12 @@ go_test( shard_count = 4, deps = [ "//pkg/config", + "//pkg/domain", "//pkg/domain/infosync", "//pkg/meta/model", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/util/coretestsdk", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json index 48665b3fb68b3..43ca98699edd7 100644 --- a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json +++ b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_in.json @@ -2,28 +2,28 @@ { "name": "TestTiFlashANNIndex", "cases": [ - "explain select * from t1;", - "explain select * from t1 where a<1;", - "explain select * from t1 where vec = '[1,1,1]'", - "explain select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", - "explain select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", - "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", - "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", - "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", - "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", - "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", - "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", - "explain select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", - "explain select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", - "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", - "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", - "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", - "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1" + "explain format = 'brief' select * from t1;", + "explain format = 'brief' select * from t1 where a<1;", + "explain format = 'brief' select * from t1 where vec = '[1,1,1]'", + "explain format = 'brief' select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", + "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", + "explain format = 'brief' select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", + "explain format = 'brief' select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", + "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "explain format = 'brief' select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1" ] }, { diff --git a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json index cc3466f3a68fd..7794dc615d9fe 100644 --- a/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json +++ b/pkg/planner/core/casetest/vectorsearch/testdata/ann_index_suite_out.json @@ -3,271 +3,301 @@ "Name": "TestTiFlashANNIndex", "Cases": [ { - "SQL": "explain select * from t1;", + "SQL": "explain format = 'brief' select * from t1;", "Plan": [ - "TableReader_10 10000.00 root MppVersion: 2, data:ExchangeSender_9", - "└─ExchangeSender_9 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_8 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 where a<1;", + "SQL": "explain format = 'brief' select * from t1 where a<1;", "Plan": [ - "TableReader_13 3323.33 root MppVersion: 2, data:ExchangeSender_12", - "└─ExchangeSender_12 3323.33 mpp[tiflash] ExchangeType: PassThrough", - " └─Selection_11 3323.33 mpp[tiflash] lt(test.t1.a, 1)", - " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + "TableReader 3323.33 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 3323.33 mpp[tiflash] lt(test.t1.a, 1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 where vec = '[1,1,1]'", + "SQL": "explain format = 'brief' select * from t1 where vec = '[1,1,1]'", "Plan": [ - "TableReader_13 10.00 root MppVersion: 2, data:ExchangeSender_12", - "└─ExchangeSender_12 10.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_10 10.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:eq(test.t1.vec, [1,1,1]), keep order:false, stats:pseudo" + "TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10.00 mpp[tiflash] table:t1 pushed down filter:eq(test.t1.vec, [1,1,1]), keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", + "SQL": "explain format = 'brief' select * from t1 where vec_cosine_distance(vec, '[1,1,1]') < 0.1", "Plan": [ - "TableReader_13 8000.00 root MppVersion: 2, data:ExchangeSender_12", - "└─ExchangeSender_12 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Selection_11 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", - " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", + "SQL": "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as dis from t1 having dis < 0.1", "Plan": [ - "TableReader_13 8000.00 root MppVersion: 2, data:ExchangeSender_12", - "└─ExchangeSender_12 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_5 8000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─Selection_11 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", - " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:empty, keep order:false, stats:pseudo" + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─Selection 8000.00 mpp[tiflash] lt(vec_cosine_distance(test.t1.vec, [1,1,1]), 0.1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')", "Plan": [ - "Projection_13 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─Sort_4 10000.00 root Column#7", - " └─Projection_14 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_12 10000.00 root MppVersion: 2, data:ExchangeSender_11", - " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_10 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─Sort 10000.00 root Column#7", + " └─Projection 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", "Plan": [ - "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_10 1.00 root Column#8, offset:0, count:1", - " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", - " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", - " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) limit 1", "Plan": [ - "Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_10 1.00 root Column#8, offset:0, count:1", - " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", - " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - " └─TopN_16 1.00 mpp[tiflash] Column#7, offset:0, count:1", - " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') desc limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7:desc, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8:desc, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7:desc, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, vec_from_text('[1,1,1]')) desc limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7:desc, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8:desc, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7:desc, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]')+1 limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, plus(vec_cosine_distance(test.t1.vec, [1,1,1]), 1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '[1,1,1]'),vec limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, test.t1.vec, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, test.t1.vec, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, test.t1.vec, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_l2_distance(vec, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_l1_distance(vec, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_l2_distance(c, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.c, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.c, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.c, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_l2_distance(d, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.d, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.d, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l2_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_cosine_distance(d, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.d, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.d, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 order by vec_l1_distance(d, '[1,1,1]') limit 1", "Plan": [ - "Projection_17 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_9 1.00 root Column#7, offset:0, count:1", - " └─Projection_18 10000.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.d, [1,1,1])->Column#7", - " └─TableReader_14 10000.00 root MppVersion: 2, data:ExchangeSender_13", - " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.d, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10000.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_l1_distance(test.t1.d, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", + "SQL": "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d", "Plan": [ - "Sort_4 10000.00 root Column#7", - "└─TableReader_13 10000.00 root MppVersion: 2, data:ExchangeSender_12", - " └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_7 10000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_11 10000.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo" + "Sort 10000.00 root Column#7", + "└─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": null }, { - "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", + "SQL": "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d from t1 order by d limit 1", "Plan": [ - "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - "└─Projection_21 1.00 root test.t1.vec", - " └─TopN_10 1.00 root Column#9, offset:0, count:1", - " └─Projection_22 1.00 root test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", - " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", - " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_19 1.00 mpp[tiflash] test.t1.vec", - " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", - " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + "Projection 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + "└─Projection 1.00 root test.t1.vec", + " └─TopN 1.00 root Column#9, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec", + " └─TopN 1.00 mpp[tiflash] Column#8, offset:0, count:1", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, { - "SQL": "explain select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", + "SQL": "explain format = 'brief' select vec_cosine_distance(vec, '[1,1,1]') as d, vec, a, b from t1 order by d limit 1", "Plan": [ - "Projection_6 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7, test.t1.vec, test.t1.a, test.t1.b", - "└─Projection_21 1.00 root test.t1.vec, test.t1.a, test.t1.b", - " └─TopN_10 1.00 root Column#9, offset:0, count:1", - " └─Projection_22 1.00 root test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", - " └─TableReader_18 1.00 root MppVersion: 2, data:ExchangeSender_17", - " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_19 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b", - " └─TopN_16 1.00 mpp[tiflash] Column#8, offset:0, count:1", - " └─Projection_20 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableFullScan_15 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" + "Projection 1.00 root vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7, test.t1.vec, test.t1.a, test.t1.b", + "└─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b", + " └─TopN 1.00 root Column#9, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#9", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b", + " └─TopN 1.00 mpp[tiflash] Column#8, offset:0, count:1", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableFullScan 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1,1,1], limit:1)" ], "Warn": null }, { - "SQL": "explain select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", + "SQL": "explain format = 'brief' select * from t1 where a=0 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", "Plan": [ - "Projection_24 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─TopN_11 1.00 root Column#8, offset:0, count:1", - " └─Projection_25 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", - " └─TableReader_21 1.00 root MppVersion: 2, data:ExchangeSender_20", - " └─ExchangeSender_20 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_22 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", - " └─TopN_19 1.00 mpp[tiflash] Column#7, offset:0, count:1", - " └─Projection_23 10.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", - " └─TableFullScan_17 10.00 mpp[tiflash] table:t1, index:vector_index(vec) pushed down filter:eq(test.t1.a, 0), keep order:false, stats:pseudo" + "Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#8", + " └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d", + " └─TopN 1.00 mpp[tiflash] Column#7, offset:0, count:1", + " └─Projection 10.00 mpp[tiflash] test.t1.vec, test.t1.a, test.t1.b, test.t1.c, test.t1.d, vec_cosine_distance(test.t1.vec, [1,1,1])->Column#7", + " └─TableFullScan 10.00 mpp[tiflash] table:t1 pushed down filter:eq(test.t1.a, 0), keep order:false, stats:pseudo" ], "Warn": null } diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index 0b8219af44ef0..c0cc8d0b4e6c1 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -19,10 +19,12 @@ import ( "testing" "time" + "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -78,6 +80,8 @@ func TestTiFlashANNIndex(t *testing.T) { for i := 0; i < 14; i++ { tk.MustExec("insert into t1(vec, a, b, c, d) select vec, a, b, c, d from t1") } + dom := domain.GetDomain(tk.Session()) + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") tk.MustExec("analyze table t1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index ccfb17bd17aca..0edbf45a6cd03 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -2199,10 +2199,14 @@ func getPhysTopN(lt *logicalop.LogicalTopN, prop *property.PhysicalProperty) []b return ret } // Currently, we only deal with the case the TopN is directly above a DataSource. - _, ok := lt.Children()[0].(*logicalop.DataSource) + ds, ok := lt.Children()[0].(*logicalop.DataSource) if !ok { return ret } + // Reject any filters. + if len(ds.PushedDownConds) > 0 { + return ret + } resultProp := &property.PhysicalProperty{ TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index a609876058724..6deb698fdc6e6 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2535,7 +2535,7 @@ func convertToTableScan(ds *logicalop.DataSource, prop *property.PhysicalPropert QueryType: tipb.ANNQueryType_OrderBy, DistanceMetric: tipb.VectorDistanceMetric(distanceMetricPB), TopK: prop.VectorProp.TopK, - ColumnName: prop.VectorProp.Column.OrigName, + ColumnName: ts.Table.Columns[candidate.path.Index.Columns[0].Offset].Name.L, ColumnId: prop.VectorProp.Column.ID, IndexId: candidate.path.Index.ID, RefVecF32: prop.VectorProp.Vec.SerializeTo(nil), diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 5c8e7f69cd947..bb5d862020ff0 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -1192,7 +1192,7 @@ func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, in } } path := &util.AccessPath{Index: index} - if index.VectorInfo != nil { + if index.VectorInfo != nil && tblInfo.TiFlashReplica.Available { path.StoreType = kv.TiFlash } publicPaths = append(publicPaths, path) From e1f54aed3bbdae20b660d5037ff4f0b6a08ee828 Mon Sep 17 00:00:00 2001 From: Yiding Date: Wed, 9 Oct 2024 23:07:20 +0800 Subject: [PATCH 17/20] go mod tidy & fix test --- pkg/planner/core/casetest/index/index_test.go | 2 ++ pkg/planner/core/casetest/vectorsearch/vector_index_test.go | 3 +++ 2 files changed, 5 insertions(+) diff --git a/pkg/planner/core/casetest/index/index_test.go b/pkg/planner/core/casetest/index/index_test.go index f273bd8ddb738..baf63cf059b86 100644 --- a/pkg/planner/core/casetest/index/index_test.go +++ b/pkg/planner/core/casetest/index/index_test.go @@ -260,6 +260,8 @@ func TestVectorIndex(t *testing.T) { tk.MustExec("create table t (a int, b vector, c vector(3), d vector(4));") tk.MustExec("alter table t set tiflash replica 1;") tk.MustExec("alter table t add vector index vecIdx1((vec_cosine_distance(d))) USING HNSW;") + dom := domain.GetDomain(tk.Session()) + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance(d, '[1,1,1,1]') limit 1", "vecIdx1") tk.MustUseIndex("select * from t use index(vecIdx1) order by vec_cosine_distance('[1,1,1,1]', d) limit 1", "vecIdx1") tk.MustExecToErr("select * from t use index(vecIdx1) order by vec_l2_distance(d, '[1,1,1,1]') limit 1") diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index c0cc8d0b4e6c1..0ce008113e4d1 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -156,6 +156,9 @@ func TestANNIndexNormalizedPlan(t *testing.T) { ('[3,3,3]') `) + dom := domain.GetDomain(tk.Session()) + coretestsdk.SetTiFlashReplica(t, dom, "test", "t") + tk.MustExec("analyze table t") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") From 0f761e6c90e0fb87cdeae145146fd5f70571c7f4 Mon Sep 17 00:00:00 2001 From: Yiding Date: Thu, 10 Oct 2024 00:27:51 +0800 Subject: [PATCH 18/20] go mod tidy & don't do sync load --- go.mod | 2 +- pkg/planner/core/collect_column_stats_usage.go | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 470cbf3b2270e..39131b79cb048 100644 --- a/go.mod +++ b/go.mod @@ -307,7 +307,7 @@ require ( google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect google.golang.org/protobuf v1.34.2 gopkg.in/inf.v0 v0.9.1 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.2.1 + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/apimachinery v0.28.6 // indirect k8s.io/klog/v2 v2.120.1 // indirect diff --git a/pkg/planner/core/collect_column_stats_usage.go b/pkg/planner/core/collect_column_stats_usage.go index e38fd39d39e79..29682be13fe78 100644 --- a/pkg/planner/core/collect_column_stats_usage.go +++ b/pkg/planner/core/collect_column_stats_usage.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -207,6 +208,10 @@ func (c *columnStatsUsageCollector) addHistNeededColumns(ds *logicalop.DataSourc if col.ID < 0 { continue } + // Don't need to load stats for vector type currently. + if col.RetType.GetType() == mysql.TypeTiDBVectorFloat32 { + continue + } tblColID := model.TableItemID{TableID: ds.PhysicalTableID, ID: col.ID, IsIndex: false} colIDSet.Insert(int(col.ID)) c.histNeededCols[tblColID] = true @@ -217,6 +222,10 @@ func (c *columnStatsUsageCollector) addHistNeededColumns(ds *logicalop.DataSourc if column.ID < 0 { continue } + // Don't need to load stats for vector type currently. + if column.FieldType.GetType() == mysql.TypeTiDBVectorFloat32 { + continue + } if !column.Hidden { tblColID := model.TableItemID{TableID: ds.PhysicalTableID, ID: column.ID, IsIndex: false} if _, ok := c.histNeededCols[tblColID]; !ok { From f60bfc7409c25558aecba24139e7f919236bcb0c Mon Sep 17 00:00:00 2001 From: Yiding Date: Thu, 10 Oct 2024 15:27:04 +0800 Subject: [PATCH 19/20] fix wrong test --- pkg/planner/core/casetest/vectorsearch/vector_index_test.go | 3 ++- pkg/planner/core/planbuilder.go | 2 +- pkg/planner/util/path.go | 4 ++++ pkg/testkit/testkit.go | 5 ++--- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index 0ce008113e4d1..16528e1968208 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -220,6 +220,7 @@ func TestANNInexWithSimpleCBO(t *testing.T) { `) tk.MustExec("alter table t1 set tiflash replica 1;") tk.MustExec("alter table t1 add vector index ((vec_cosine_distance(vec))) USING HNSW;") + dom := domain.GetDomain(tk.Session()) + coretestsdk.SetTiFlashReplica(t, dom, "test", "t1") tk.MustUseIndex("select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", "vector_index") - tk.MustNoIndexUsed("select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 10000") } diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index bb5d862020ff0..b49400fb4dc72 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -1382,7 +1382,7 @@ func removeIgnoredPaths(paths, ignoredPaths []*util.AccessPath, tblInfo *model.T } remainedPaths := make([]*util.AccessPath, 0, len(paths)) for _, path := range paths { - if path.IsTablePath() || getPathByIndexName(ignoredPaths, path.Index.Name, tblInfo) == nil { + if path.IsTiKVTablePath() || path.IsTiFlashSimpleTablePath() || getPathByIndexName(ignoredPaths, path.Index.Name, tblInfo) == nil { remainedPaths = append(remainedPaths, path) } } diff --git a/pkg/planner/util/path.go b/pkg/planner/util/path.go index d8f88f27b7c9d..5382f1a51d40b 100644 --- a/pkg/planner/util/path.go +++ b/pkg/planner/util/path.go @@ -154,6 +154,10 @@ func (path *AccessPath) IsTiKVTablePath() bool { return (path.IsIntHandlePath || path.IsCommonHandlePath) && path.StoreType == kv.TiKV } +func (path *AccessPath) IsTiFlashSimpleTablePath() bool { + return (path.IsIntHandlePath || path.IsCommonHandlePath) && path.StoreType == kv.TiFlash +} + // SplitCorColAccessCondFromFilters move the necessary filter in the form of index_col = corrlated_col to access conditions. // The function consider the `idx_col_1 = const and index_col_2 = cor_col and index_col_3 = const` case. // It enables more index columns to be considered. The range will be rebuilt in 'ResolveCorrelatedColumns'. diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index 94853cc841553..8ed821fa91c56 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -562,14 +562,13 @@ func (tk *TestKit) MustUseIndex(sql string, index string, args ...any) bool { } // MustNoIndexUsed checks if the result execution plan contains no index. -func (tk *TestKit) MustNoIndexUsed(sql string, args ...any) bool { +func (tk *TestKit) MustNoIndexUsed(sql string, args ...any) { rs := tk.MustQuery("explain "+sql, args...) for i := range rs.rows { if strings.Contains(rs.rows[i][3], "index:") { - return false + tk.require.Fail("index is used") } } - return true } // MustUseIndex4ExplainFor checks if the result execution plan contains specific index(es). From 95feba8e5ae04d03899b921a8b844e5007b0ebdb Mon Sep 17 00:00:00 2001 From: Yiding Date: Thu, 10 Oct 2024 21:30:58 +0800 Subject: [PATCH 20/20] fix static check --- pkg/planner/util/path.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/planner/util/path.go b/pkg/planner/util/path.go index 5382f1a51d40b..92da155765238 100644 --- a/pkg/planner/util/path.go +++ b/pkg/planner/util/path.go @@ -154,6 +154,7 @@ func (path *AccessPath) IsTiKVTablePath() bool { return (path.IsIntHandlePath || path.IsCommonHandlePath) && path.StoreType == kv.TiKV } +// IsTiFlashSimpleTablePath returns true if it's a TiFlash path and will not use any special indexes like vector index. func (path *AccessPath) IsTiFlashSimpleTablePath() bool { return (path.IsIntHandlePath || path.IsCommonHandlePath) && path.StoreType == kv.TiFlash }