Skip to content
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion pkg/planner/core/casetest/vectorsearch/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,21 +9,25 @@ go_test(
],
data = glob(["testdata/**"]),
flaky = True,
shard_count = 4,
shard_count = 5,
deps = [
"//pkg/config",
"//pkg/domain",
"//pkg/domain/infosync",
"//pkg/meta/model",
"//pkg/parser/model",
"//pkg/planner",
"//pkg/planner/core",
"//pkg/planner/core/base",
"//pkg/planner/core/resolve",
"//pkg/session",
"//pkg/store/mockstore",
"//pkg/testkit",
"//pkg/testkit/testdata",
"//pkg/testkit/testfailpoint",
"//pkg/testkit/testmain",
"//pkg/testkit/testsetup",
"//pkg/types",
"//pkg/util/plancodec",
"@com_github_pingcap_tipb//go-tipb",
"@com_github_stretchr_testify//require",
Expand Down
68 changes: 68 additions & 0 deletions pkg/planner/core/casetest/vectorsearch/vector_index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,16 @@ import (
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/meta/model"
pmodel "github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/planner"
"github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/resolve"
"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/testdata"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/plancodec"
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -244,3 +248,67 @@ func TestANNInexWithSimpleCBO(t *testing.T) {
testkit.SetTiFlashReplica(t, dom, "test", "t1")
tk.MustUseIndex("select * from t1 order by vec_cosine_distance(vec, '[1,1,1]') limit 1", "vector_index")
}

func TestANNIndexWithNonIntClusteredPk(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,
primary key (a, b)
)
`)
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]')")
dom := domain.GetDomain(tk.Session())
testkit.SetTiFlashReplica(t, dom, "test", "t1")
sctx := tk.Session()
stmts, err := session.Parse(sctx, "select * from t1 use index(vector_index) order by vec_cosine_distance(vec, '[1,1,1]') limit 1")
require.NoError(t, err)
require.Len(t, stmts, 1)
stmt := stmts[0]
ret := &core.PreprocessorReturn{}
nodeW := resolve.NewNodeW(stmt)
err = core.Preprocess(context.Background(), sctx, nodeW, core.WithPreprocessorReturn(ret))
require.NoError(t, err)
var finalPlanTree base.Plan
finalPlanTree, _, err = planner.Optimize(context.Background(), sctx, nodeW, ret.InfoSchema)
require.NoError(t, err)
physicalTree, ok := finalPlanTree.(base.PhysicalPlan)
require.True(t, ok)
// Find the PhysicalTableReader node.
tableReader := physicalTree
for ; len(tableReader.Children()) > 0; tableReader = tableReader.Children()[0] {
}
castedTableReader, ok := tableReader.(*core.PhysicalTableReader)
require.True(t, ok)
tableScan, err := castedTableReader.GetTableScan()
require.NoError(t, err)
// Check that it has the extra vector index information.
require.NotNil(t, tableScan.AnnIndexExtra)
require.Len(t, tableScan.Ranges, 1)
// Check that it's full scan.
require.Equal(t, "[-inf,+inf]", tableScan.Ranges[0].String())
// Check that the -inf and +inf are the correct types.
require.Equal(t, types.KindMinNotNull, tableScan.Ranges[0].LowVal[0].Kind())
require.Equal(t, types.KindMaxValue, tableScan.Ranges[0].HighVal[0].Kind())
}
26 changes: 13 additions & 13 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -761,6 +761,19 @@ func compareCandidates(sctx base.PlanContext, prop *property.PhysicalProperty, l
}

func isMatchProp(ds *logicalop.DataSource, path *util.AccessPath, prop *property.PhysicalProperty) bool {
if prop.VectorProp.VectorHelper != nil && path.Index != 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
}
Comment on lines +764 to +776
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I changed it to the highest priority.
Because when for the vector index path, the IsIntHandlePath can also be true. We need to use a new way to check the real pk path.

var isMatchProp bool
if path.IsIntHandlePath {
pkCol := ds.GetPKIsHandleCol()
Expand Down Expand Up @@ -808,19 +821,6 @@ 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
}

Expand Down
11 changes: 7 additions & 4 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5502,7 +5502,10 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (
if dbName == "" {
dbName = b.ctx.GetSessionVars().CurrentDB
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil)
// Avoid adding CTE table to the SELECT privilege list, maybe we have better way to do this?
if _, ok := b.nameMapCTE[t.Name.L]; !ok {
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why should we need this for? any direct effect from the issue?

}

oldSchemaLen := p.Schema().Len()
Expand Down Expand Up @@ -7406,12 +7409,12 @@ func (b *PlanBuilder) genCTETableNameForError() string {

func (b *PlanBuilder) buildWith(ctx context.Context, w *ast.WithClause) ([]*cteInfo, error) {
// Check CTE name must be unique.
nameMap := make(map[string]struct{})
b.nameMapCTE = make(map[string]struct{})
for _, cte := range w.CTEs {
if _, ok := nameMap[cte.Name.L]; ok {
if _, ok := b.nameMapCTE[cte.Name.L]; ok {
return nil, plannererrors.ErrNonUniqTable
}
nameMap[cte.Name.L] = struct{}{}
b.nameMapCTE[cte.Name.L] = struct{}{}
}
ctes := make([]*cteInfo, 0, len(w.CTEs))
for _, cte := range w.CTEs {
Expand Down
10 changes: 8 additions & 2 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,8 +269,10 @@ type PlanBuilder struct {
allocIDForCTEStorage int
buildingRecursivePartForCTE bool
buildingCTE bool
//Check whether the current building query is a CTE
// Check whether the current building query is a CTE
isCTE bool
// CTE table name in lower case, it can be nil
nameMapCTE map[string]struct{}

// subQueryCtx and subQueryHintFlags are for handling subquery related hints.
// Note: "subquery" here only contains subqueries that are handled by the expression rewriter, i.e., [NOT] IN,
Expand Down Expand Up @@ -1191,15 +1193,19 @@ func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, in
continue
}
}
path := &util.AccessPath{Index: index}
if index.VectorInfo != nil {
// Because the value of `TiFlashReplica.Available` changes as the user modify replica, it is not ideal if the state of index changes accordingly.
// So the current way to use the vector indexes is to require the TiFlash Replica to be available.
if !tblInfo.TiFlashReplica.Available {
continue
}
path := genTiFlashPath(tblInfo)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I feel that the original logic is different here set up "tablePath. IsCommonHandlePath = true". What does this setting do? Why is the original logic not needed?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A normal index will create the index's ranges.
But the vector index still uses the table range. So we need to correctly set the pk type. So it can generate the correct table range.
For int pk, it will use [MinInt64, MaxInt64] as full range. But for other cases, [MinNotNull, MaxValue] will be full range.

Copy link
Contributor

@AilinKid AilinKid Nov 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

image

i think if we don't set it here, we will step into here right? which will derive a int range for common handle tiflash path

path.StoreType = kv.TiFlash
path.Index = index
publicPaths = append(publicPaths, path)
continue
}
path := &util.AccessPath{Index: index}
publicPaths = append(publicPaths, path)
}
}
Expand Down
3 changes: 0 additions & 3 deletions pkg/planner/core/runtime_filter_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,9 +83,6 @@ func (generator *RuntimeFilterGenerator) GenerateRuntimeFilter(plan base.Physica
func (generator *RuntimeFilterGenerator) generateRuntimeFilterInterval(hashJoinPlan *PhysicalHashJoin) {
// precondition: the storage type of hash join must be TiFlash
if hashJoinPlan.storeTp != kv.TiFlash {
logutil.BgLogger().Warn("RF only support TiFlash compute engine while storage type of hash join node is not TiFlash",
zap.Int("PhysicalHashJoinId", hashJoinPlan.ID()),
zap.String("StoreTP", hashJoinPlan.storeTp.Name()))
return
}
// check hash join pattern
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ func (path *AccessPath) IsTiKVTablePath() bool {

// 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
return path.StoreType == kv.TiFlash && path.Index == nil
}

// SplitCorColAccessCondFromFilters move the necessary filter in the form of index_col = corrlated_col to access conditions.
Expand Down
16 changes: 16 additions & 0 deletions tests/integrationtest/r/privilege/privileges.result
Original file line number Diff line number Diff line change
Expand Up @@ -647,3 +647,19 @@ ADMIN SHOW SLOW TOP ALL 3;
Error 8121 (HY000): privilege check for 'Super' fail
ADMIN ALTER DDL JOBS 10 THREAD = 3, BATCH_SIZE = 100, MAX_WRITE_SPEED = '10MiB';
Error 8121 (HY000): privilege check for 'Super' fail
create table privilege__privileges.tt1 (id bigint,pid bigint,name varchar(20),fullname varchar(20));
insert into privilege__privileges.tt1 values (1,null,'a',''),(2,1,'b',''),(3,2,'c','');
CREATE USER u53490;
GRANT USAGE ON *.* TO 'u53490';
GRANT SELECT,INSERT,UPDATE,DELETE,CREATE,DROP,CREATE ROUTINE,ALTER ROUTINE,ALTER,EXECUTE,INDEX,CREATE VIEW,SHOW VIEW ON privilege__privileges.* TO 'u53490';
with t_f as (
select id,pid,name,'AAA' fullname from privilege__privileges.tt1 )
update privilege__privileges.tt1 inner join t_f
set tt1.fullname=t_f.fullname
where tt1.id=t_f.id;
with t_f as (
select id,pid,name,'AAA' fullname from privilege__privileges.tt1 )
update privilege__privileges.tt1 inner join t_f
set t_f.fullname=t_f.fullname
where tt1.id=t_f.id;
Error 1288 (HY000): The target table t_f of the UPDATE is not updatable
27 changes: 27 additions & 0 deletions tests/integrationtest/t/privilege/privileges.test
Original file line number Diff line number Diff line change
Expand Up @@ -871,3 +871,30 @@ ADMIN ALTER DDL JOBS 10 THREAD = 3, BATCH_SIZE = 100, MAX_WRITE_SPEED = '10MiB';

disconnect without_super;
connection default;

# TestIssue53490
create table privilege__privileges.tt1 (id bigint,pid bigint,name varchar(20),fullname varchar(20));
insert into privilege__privileges.tt1 values (1,null,'a',''),(2,1,'b',''),(3,2,'c','');

CREATE USER u53490;
GRANT USAGE ON *.* TO 'u53490';
GRANT SELECT,INSERT,UPDATE,DELETE,CREATE,DROP,CREATE ROUTINE,ALTER ROUTINE,ALTER,EXECUTE,INDEX,CREATE VIEW,SHOW VIEW ON privilege__privileges.* TO 'u53490';

connect (u53490,localhost,u53490,,);
connection u53490;

with t_f as (
select id,pid,name,'AAA' fullname from privilege__privileges.tt1 )
update privilege__privileges.tt1 inner join t_f
set tt1.fullname=t_f.fullname
where tt1.id=t_f.id;

-- error 1288
with t_f as (
select id,pid,name,'AAA' fullname from privilege__privileges.tt1 )
update privilege__privileges.tt1 inner join t_f
set t_f.fullname=t_f.fullname
where tt1.id=t_f.id;

disconnect u53490;
connection default;