From 25a2d013330194b424703dcb3b25bc649e99bb6d Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Fri, 12 Jan 2024 17:40:39 +0800 Subject: [PATCH] fix intersection type mv index's filters mutations composition Signed-off-by: AilinKid <314806019@qq.com> --- pkg/planner/core/indexmerge_path.go | 30 ++++++--- pkg/planner/core/indexmerge_path_test.go | 78 ++++++++++++++++++++++++ pkg/planner/core/logical_plans.go | 5 ++ 3 files changed, 103 insertions(+), 10 deletions(-) diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index 5fe5f9f17696b..7483e4bdfcd74 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -41,7 +41,7 @@ import ( func init() { cardinality.CollectFilters4MVIndex = collectFilters4MVIndex cardinality.BuildPartialPaths4MVIndex = buildPartialPaths4MVIndex - statistics.PrepareCols4MVIndex = prepareCols4MVIndex + statistics.PrepareCols4MVIndex = PrepareCols4MVIndex } // generateIndexMergePath generates IndexMerge AccessPaths on this DataSource. @@ -690,7 +690,7 @@ func (ds *DataSource) generateMVIndexPartialPath4Or(normalPathCnt int, indexMerg bestNeedSelection bool ) for _, onePossibleMVIndexPath := range possibleMVIndexPaths { - idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), onePossibleMVIndexPath.Index, ds.TblCols) + idxCols, ok := PrepareCols4MVIndex(ds.table.Meta(), onePossibleMVIndexPath.Index, ds.TblCols) if !ok { continue } @@ -758,11 +758,11 @@ func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, in mvAndPartialPath := make([]*util.AccessPath, 0, len(possibleMVIndexPaths)) usedAccessCondsMap := make(map[string]expression.Expression, len(indexMergeConds)) for idx := 0; idx < len(possibleMVIndexPaths); idx++ { - idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), possibleMVIndexPaths[idx].Index, ds.TblCols) + idxCols, ok := PrepareCols4MVIndex(ds.table.Meta(), possibleMVIndexPaths[idx].Index, ds.TblCols) if !ok { continue } - accessFilters, _, mvColOffset, mvFilterMutations := ds.collectFilters4MVIndexMutations(indexMergeConds, idxCols) + accessFilters, _, mvColOffset, mvFilterMutations := CollectFilters4MVIndexMutations(ds.SCtx(), indexMergeConds, idxCols) if len(accessFilters) == 0 { // cannot use any filter on this MVIndex continue } @@ -865,7 +865,7 @@ func (ds *DataSource) generateIndexMergeOnDNF4MVIndex(normalPathCnt int, filters continue // not a MVIndex path } - idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) + idxCols, ok := PrepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) if !ok { continue } @@ -1126,7 +1126,7 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex continue // not a MVIndex path } - idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) + idxCols, ok := PrepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) if !ok { continue } @@ -1313,7 +1313,8 @@ func buildPartialPath4MVIndex( return partialPath, true, nil } -func prepareCols4MVIndex( +// PrepareCols4MVIndex exported for test. +func PrepareCols4MVIndex( tableInfo *model.TableInfo, mvIndex *model.IndexInfo, tblCols []*expression.Column, @@ -1377,6 +1378,7 @@ func collectFilters4MVIndex(sctx sessionctx.Context, filters []expression.Expres return accessFilters, remainingFilters } +// CollectFilters4MVIndexMutations exported for unit test. // For idx(x, cast(a as array), z), `x=1 and (2 member of a) and (1 member of a) and z=1 and x+z>0` is split to: // accessFilters combination: // 1: `x=1 and (2 member of a) and z=1`, remaining: `x+z>0`. @@ -1413,7 +1415,7 @@ func collectFilters4MVIndex(sctx sessionctx.Context, filters []expression.Expres // accessFilters: [x=1, (2 member of a), z=1], remainingFilters: [x+z>0], mvColOffset: 1, mvFilterMutations[(2 member of a), (1 member of a)] // // the outer usage will be: accessFilter[mvColOffset] = each element of mvFilterMutations to get the mv access filters mutation combination. -func (ds *DataSource) collectFilters4MVIndexMutations(filters []expression.Expression, +func CollectFilters4MVIndexMutations(sctx sessionctx.Context, filters []expression.Expression, idxCols []*expression.Column) (accessFilters, remainingFilters []expression.Expression, mvColOffset int, mvFilterMutations []expression.Expression) { usedAsAccess := make([]bool, len(filters)) // accessFilters [x, a, z] @@ -1427,12 +1429,20 @@ func (ds *DataSource) collectFilters4MVIndexMutations(filters []expression.Expre if usedAsAccess[i] { continue } - if checkFilter4MVIndexColumn(ds.SCtx(), f, col) { + if checkFilter4MVIndexColumn(sctx, f, col) { if col.VirtualExpr != nil && col.VirtualExpr.GetType().IsArray() { // assert jsonColOffset should always be the same. // if the filter is from virtual expression, it means it is about the mv json col. mvFilterMutations = append(mvFilterMutations, f) - mvColOffset = z + if mvColOffset == -1 { + // means first encountering, recording offset pos, and append it as occupation of access filter. + mvColOffset = z + accessFilters = append(accessFilters, f) + } + // additional encountering, just map it as used access. + usedAsAccess[i] = true + found = true + continue } accessFilters = append(accessFilters, f) usedAsAccess[i] = true diff --git a/pkg/planner/core/indexmerge_path_test.go b/pkg/planner/core/indexmerge_path_test.go index 6358389049349..e73156f91883d 100644 --- a/pkg/planner/core/indexmerge_path_test.go +++ b/pkg/planner/core/indexmerge_path_test.go @@ -15,14 +15,92 @@ package core_test import ( + "context" "fmt" "math/rand" "strings" "testing" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util/hint" + "github.com/stretchr/testify/require" ) +func TestCollectFilters4MVIndexMutations(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, domains json null, images json null, KEY `a_domains_b` (a, (cast(`domains` as char(253) array)), b))") + sql := "SELECT * FROM t WHERE 15975127 member of (domains) AND 15975128 member of (domains) AND a = 1 AND b = 2" + + par := parser.New() + par.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) + // Make sure the table schema is the new schema. + err := domain.Reload() + require.NoError(t, err) + is := domain.InfoSchema() + is = &infoschema.SessionExtendedInfoSchema{InfoSchema: is} + require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) + require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), nil)) + stmt, err := par.ParseOneStmt(sql, "", "") + require.NoError(t, err) + tk.Session().GetSessionVars().PlanID.Store(0) + tk.Session().GetSessionVars().PlanColumnID.Store(0) + err = core.Preprocess(context.Background(), tk.Session(), stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).AdviseWarmup()) + builder, _ := core.NewPlanBuilder().Init(tk.Session(), is, hint.NewQBHintHandler(nil)) + p, err := builder.Build(context.TODO(), stmt) + require.NoError(t, err) + logicalP, err := core.LogicalOptimizeTest(context.TODO(), builder.GetOptFlag(), p.(core.LogicalPlan)) + require.NoError(t, err) + + ds, ok := logicalP.(*core.DataSource) + for !ok { + p := logicalP.Children()[0] + ds, ok = p.(*core.DataSource) + } + cnfs := ds.GetAllConds() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + idxCols, ok := core.PrepareCols4MVIndex(tbl.Meta(), tbl.Meta().FindIndexByName("a_domains_b"), ds.TblCols) + require.True(t, ok) + accessFilters, _, mvColOffset, mvFilterMutations := core.CollectFilters4MVIndexMutations(tk.Session(), cnfs, idxCols) + + // assert mv col access filters. + require.Equal(t, len(accessFilters), 3) + sf, ok := accessFilters[0].(*expression.ScalarFunction) + require.True(t, ok) + require.Equal(t, sf.FuncName.L, ast.EQ) + sf, ok = accessFilters[1].(*expression.ScalarFunction) + require.True(t, ok) + require.Equal(t, sf.FuncName.L, ast.JSONMemberOf) + sf, ok = accessFilters[2].(*expression.ScalarFunction) + require.True(t, ok) + require.Equal(t, sf.FuncName.L, ast.EQ) + + // assert mv col offset + require.Equal(t, mvColOffset, 1) + + // assert mv col condition mutations. + require.Equal(t, len(mvFilterMutations), 2) + sf, ok = mvFilterMutations[0].(*expression.ScalarFunction) + require.True(t, ok) + require.Equal(t, sf.FuncName.L, ast.JSONMemberOf) + sf, ok = mvFilterMutations[1].(*expression.ScalarFunction) + require.True(t, ok) + require.Equal(t, sf.FuncName.L, ast.JSONMemberOf) +} + func TestMultiMVIndexRandom(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index c00cc962bc61e..8668630ad9d0e 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -1407,6 +1407,11 @@ type LogicalUnionScan struct { handleCols HandleCols } +// GetAllConds Exported for unit test. +func (ds *DataSource) GetAllConds() []expression.Expression { + return ds.allConds +} + // DataSource represents a tableScan without condition push down. type DataSource struct { logicalSchemaProducer