From 12f0ce5602f85b4b7d1d7d22b4efd516d049fbd4 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 14:43:46 +0800 Subject: [PATCH 01/12] fixup --- expression/builtin_compare.go | 9 +++++---- expression/builtin_other.go | 3 +-- planner/core/expression_rewriter.go | 3 +-- planner/core/find_best_task.go | 3 +-- planner/core/optimizer.go | 3 +-- planner/core/plan_cache.go | 2 +- sessionctx/stmtctx/stmtctx.go | 12 ++++++++++-- util/ranger/detacher.go | 3 +-- 8 files changed, 21 insertions(+), 17 deletions(-) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 4411f4b6b90ff..c7e65145562c0 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -15,6 +15,7 @@ package expression import ( + "github.com/pkg/errors" "math" "strings" @@ -28,7 +29,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" - "github.com/pkg/errors" ) var ( @@ -1575,12 +1575,13 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express // To keep the result be compatible with MySQL, refine `int non-constant str constant` // here and skip this refine operation in all other cases for safety. if (arg0IsInt && !arg0IsCon && arg1IsString && arg1IsCon) || (arg1IsInt && !arg1IsCon && arg0IsString && arg0IsCon) { - ctx.GetSessionVars().StmtCtx.SkipPlanCache = true + var reason error if arg1IsString { - ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg1.String())) + reason = errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg1.String()) } else { // arg0IsString - ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg0.String())) + reason = errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg0.String()) } + ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(reason) RemoveMutableConst(ctx, args) } else { return args diff --git a/expression/builtin_other.go b/expression/builtin_other.go index c62278c2bd101..c5bd6738aa9df 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -165,8 +165,7 @@ func (c *inFunctionClass) verifyArgs(ctx sessionctx.Context, args []Expression) case columnType.GetType() == mysql.TypeBit && constant.Value.Kind() == types.KindInt64: if constant.Value.GetInt64() < 0 { if MaybeOverOptimized4PlanCache(ctx, args) { - ctx.GetSessionVars().StmtCtx.SkipPlanCache = true - ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: Bit Column in (%v)", constant.Value.GetInt64())) + ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: Bit Column in (%v)", constant.Value.GetInt64())) } continue } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index ddb905dc5c06b..2694d81ef0a65 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1562,8 +1562,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field if c.GetType().EvalType() == types.ETString { // To keep the result be compatible with MySQL, refine `int non-constant str constant` // here and skip this refine operation in all other cases for safety. - er.sctx.GetSessionVars().StmtCtx.SkipPlanCache = true - er.sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: '%v' may be converted to INT", c.String())) + er.sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to INT", c.String())) expression.RemoveMutableConst(er.sctx, []expression.Expression{c}) } else { continue diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 639bc15dbdc98..ca99704aa5a66 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -928,8 +928,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if len(path.Ranges) == 0 { // We should uncache the tableDual plan. if expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) { - ds.ctx.GetSessionVars().StmtCtx.SkipPlanCache = true - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: get a TableDual plan")) + ds.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: get a TableDual plan")) } dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 1064ea529b2d0..1a3e1ea5ab821 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -782,8 +782,7 @@ func setupFineGrainedShuffleInternal(plan PhysicalPlan, helper *fineGrainedShuff // Todo: make more careful check here. func checkPlanCacheable(sctx sessionctx.Context, plan PhysicalPlan) { if sctx.GetSessionVars().StmtCtx.UseCache && useTiFlash(plan) { - sctx.GetSessionVars().StmtCtx.SkipPlanCache = true - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: TiFlash plan is un-cacheable")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: TiFlash plan is un-cacheable")) } } diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index ab4eb4e4912ab..b04dc322574b7 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -278,7 +278,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared // We only cache the tableDual plan when the number of parameters are zero. if containTableDual(p) && paramNum > 0 { - stmtCtx.SkipPlanCache = true + stmtCtx.SetSkipPlanCache(errors.New("cannot cache TableDual")) } if stmtAst.UseCache && !stmtCtx.SkipPlanCache && !ignorePlanCache { // rebuild key to exclude kv.TiFlash when stmt is not read only diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 799f2f8bdeea8..b0730324cd662 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -609,6 +609,15 @@ func (sc *StatementContext) SetPlanHint(hint string) { sc.planHint = hint } +// SetSkipPlanCache sets to skip the plan cache and records the reason. +func (sc *StatementContext) SetSkipPlanCache(reason error) { + if sc.SkipPlanCache { + return + } + sc.SkipPlanCache = true + sc.AppendWarning(reason) +} + // TableEntry presents table in db. type TableEntry struct { DB string @@ -1154,9 +1163,8 @@ func (sc *StatementContext) GetLockWaitStartTime() time.Time { func (sc *StatementContext) RecordRangeFallback(rangeMaxSize int64) { // If range fallback happens, it means ether the query is unreasonable(for example, several long IN lists) or tidb_opt_range_max_size is too small // and the generated plan is probably suboptimal. In that case we don't put it into plan cache. - sc.SkipPlanCache = true if sc.UseCache { - sc.AppendWarning(errors.Errorf("skip plan-cache: in-list is too long")) + sc.SetSkipPlanCache(errors.Errorf("skip plan-cache: in-list is too long")) } if !sc.RangeFallback { sc.AppendWarning(errors.Errorf("Memory capacity of %v bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", rangeMaxSize)) diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 596d27c27d9d8..e4d75711cae2a 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -622,8 +622,7 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex } if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a=@x and a=@y` --> `a=@x if @x==@y` - sctx.GetSessionVars().StmtCtx.SkipPlanCache = true - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("skip plan-cache: some parameters may be overwritten")) + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: some parameters may be overwritten")) } } } From cb91d651fca28a5e48090afac12276f5ca658620 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 14:53:40 +0800 Subject: [PATCH 02/12] fixup --- expression/builtin_compare.go | 2 +- planner/core/plan_cache.go | 2 +- sessionctx/stmtctx/stmtctx.go | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index c7e65145562c0..4ffd0a3dbe6be 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -15,7 +15,6 @@ package expression import ( - "github.com/pkg/errors" "math" "strings" @@ -29,6 +28,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" + "github.com/pkg/errors" ) var ( diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index b04dc322574b7..9739a63f2b8ff 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -278,7 +278,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared // We only cache the tableDual plan when the number of parameters are zero. if containTableDual(p) && paramNum > 0 { - stmtCtx.SetSkipPlanCache(errors.New("cannot cache TableDual")) + stmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: get a TableDual plan")) } if stmtAst.UseCache && !stmtCtx.SkipPlanCache && !ignorePlanCache { // rebuild key to exclude kv.TiFlash when stmt is not read only diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index b0730324cd662..cfe017c7c8805 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -611,8 +611,8 @@ func (sc *StatementContext) SetPlanHint(hint string) { // SetSkipPlanCache sets to skip the plan cache and records the reason. func (sc *StatementContext) SetSkipPlanCache(reason error) { - if sc.SkipPlanCache { - return + if sc.UseCache && sc.SkipPlanCache { + return // avoid unnecessary warnings } sc.SkipPlanCache = true sc.AppendWarning(reason) From a9d6605b1d984951dd08542b3025d7587376913e Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 15:48:29 +0800 Subject: [PATCH 03/12] fixup --- planner/core/integration_test.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 743c6b87dc6d0..31cd5aa9c346b 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7792,8 +7792,6 @@ func TestPlanCacheForTableRangeFallback(t *testing.T) { tk.MustExec("set @a=10, @b=20, @c=30, @d=40, @e=50") tk.MustExec("execute stmt using @a, @b, @c, @d, @e") tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 10 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", - "Warning 1105 skip plan-cache: in-list is too long", - "Warning 1105 skip plan-cache: in-list is too long", "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt using @a, @b, @c, @d, @e") // The plan with range fallback is not cached. @@ -7842,7 +7840,6 @@ func TestPlanCacheForIndexRangeFallback(t *testing.T) { tk.MustExec("set @a='aa', @b='bb', @c='cc', @d='dd', @e='ee', @f='ff', @g='gg', @h='hh', @i='ii', @j='jj'") tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1330 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", - "Warning 1105 skip plan-cache: in-list is too long", "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) @@ -8000,10 +7997,6 @@ func TestPlanCacheForIndexJoinRangeFallback(t *testing.T) { tk.MustExec("set @a='a', @b='b', @c='c', @d='d', @e='e'") tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1275 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", - "Warning 1105 skip plan-cache: in-list is too long", - "Warning 1105 skip plan-cache: in-list is too long", - "Warning 1105 skip plan-cache: in-list is too long", - "Warning 1105 skip plan-cache: in-list is too long", "Warning 1105 skip plan-cache: in-list is too long")) tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) From fa298685f933d474b177b2845852aa01e33b17b2 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 15:54:59 +0800 Subject: [PATCH 04/12] fixup --- expression/expression.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/expression/expression.go b/expression/expression.go index ee4ba67f1f854..c7ce0b22c1c0e 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -816,7 +816,7 @@ func SplitDNFItems(onExpr Expression) []Expression { // If the Expression is a non-constant value, it means the result is unknown. func EvaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expression) Expression { if MaybeOverOptimized4PlanCache(ctx, []Expression{expr}) { - return expr + ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("skip plan-cache: %v affects null check")) } if ctx.GetSessionVars().StmtCtx.InNullRejectCheck { expr, _ = evaluateExprWithNullInNullRejectCheck(ctx, schema, expr) From 2cef7cb43df84032b27a8320052caf088f016038 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 16:01:47 +0800 Subject: [PATCH 05/12] fixup --- planner/core/plan_cache_test.go | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index a480c583d0434..4319167487f29 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -221,6 +221,36 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } +func TestIssue40093(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a int, b int)") + tk.MustExec("create table t2 (a int, b int, key(b, a))") + tk.MustExec("prepare st from 'select * from t1 left join t2 on t1.a=t2.a where t2.b in (?)'") + tk.MustExec("set @b=1") + tk.MustExec("execute st using @b") + + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, + [][]interface{}{ + {"Projection_9"}, + {"└─HashJoin_21"}, + {" ├─IndexReader_26(Build)"}, + {" │ └─IndexRangeScan_25"}, // RangeScan instead of FullScan + {" └─TableReader_24(Probe)"}, + {" └─Selection_23"}, + {" └─TableFullScan_22"}, + }) + + tk.MustExec("execute st using @b") + tk.MustExec("execute st using @b") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + func TestIgnoreInsertStmt(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) From fbfacd1a8feee7789029bdb791dd56f234edacdf Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 16:06:15 +0800 Subject: [PATCH 06/12] fixup --- planner/core/plan_cache_test.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 4319167487f29..9ec09a05ef924 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -251,6 +251,37 @@ func TestIssue40093(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } +func TestIssue38205(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `item` (`id` int, `vid` varbinary(16), `sid` int)") + tk.MustExec("CREATE TABLE `lv` (`item_id` int, `sid` int, KEY (`sid`,`item_id`))") + + tk.MustExec("prepare stmt from 'SELECT /*+ TIDB_INLJ(lv, item) */ * FROM lv LEFT JOIN item ON lv.sid = item.sid AND lv.item_id = item.id WHERE item.sid = ? AND item.vid IN (?, ?)'") + tk.MustExec("set @a=1, @b='1', @c='3'") + tk.MustExec("execute stmt using @a, @b, @c") + + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, + [][]interface{}{ + {"IndexJoin_10"}, + {"├─TableReader_19(Build)"}, + {"│ └─Selection_18"}, + {"│ └─TableFullScan_17"}, // RangeScan instead of FullScan + {"└─IndexReader_9(Probe)"}, + {" └─Selection_8"}, + {" └─IndexRangeScan_7"}, + }) + + tk.MustExec("execute stmt using @a, @b, @c") + tk.MustExec("execute stmt using @a, @b, @c") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + func TestIgnoreInsertStmt(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) From f9c3b130f093407d91754927bc5ba751b6142f36 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 17:03:09 +0800 Subject: [PATCH 07/12] fixup --- expression/expression_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/expression/expression_test.go b/expression/expression_test.go index 79ee1970ba800..e80cf5c41d575 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -75,8 +75,9 @@ func TestEvaluateExprWithNullAndParameters(t *testing.T) { ltWithParam, err := newFunctionForTest(ctx, ast.LT, col0, param) require.NoError(t, err) res = EvaluateExprWithNull(ctx, schema, ltWithParam) - _, isScalarFunc := res.(*ScalarFunction) - require.True(t, isScalarFunc) // the expression with parameters is not evaluated + _, isConst := res.(*Constant) + require.True(t, isConst) // this expression is evaluated and skip-plan cache flag is set. + require.True(t, ctx.GetSessionVars().StmtCtx.SkipPlanCache) } func TestEvaluateExprWithNullNoChangeRetType(t *testing.T) { From c2b9a466341795c41a6a9c66ffa7521e8e2d6612 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 17:59:12 +0800 Subject: [PATCH 08/12] fixup --- util/ranger/detacher.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index e4d75711cae2a..606f53c40265f 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -581,9 +581,8 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex points[offset] = rb.intersection(points[offset], rb.build(cond, collator), collator) if len(points[offset]) == 0 { // Early termination if false expression found if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { - // cannot return an empty-range for plan-cache since the range may become non-empty as parameters change - // for safety, return the whole conditions in this case - return nil, conditions, nil, nil, false + // `a>@x and a<@y` --> `invalid-range if @x>=@y` + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: some parameters may be overwritten")) } return nil, nil, nil, nil, true } @@ -606,9 +605,8 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex accesses[i] = nil } else if len(points[i]) == 0 { // Early termination if false expression found if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { - // cannot return an empty-range for plan-cache since the range may become non-empty as parameters change - // for safety, return the whole conditions in this case - return nil, conditions, nil, nil, false + // `a>@x and a<@y` --> `invalid-range if @x>=@y` + sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: some parameters may be overwritten")) } return nil, nil, nil, nil, true } else { From a1462b21368fbe29339b99a522e303e3a6095232 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 28 Dec 2022 18:09:25 +0800 Subject: [PATCH 09/12] fixup --- planner/core/plan_cache_test.go | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 9ec09a05ef924..e25565a110e08 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -221,6 +221,27 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } +func TestInvalidRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key(a))") + tk.MustExec("prepare st from 'select * from t where a>? and a Date: Thu, 29 Dec 2022 14:22:34 +0800 Subject: [PATCH 10/12] fixup --- executor/explainfor_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 89245fa6b237d..4b0fff6fefef4 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -860,7 +860,7 @@ func TestIndexMerge4PlanCache(t *testing.T) { tk.MustExec("prepare stmt from 'select /*+ use_index_merge(t1) */ * from t1 where c=? or (b=? and (a >= ? and a <= ?));';") tk.MustQuery("execute stmt using @a, @a, @b, @a").Check(testkit.Rows("10 10 10")) tk.MustQuery("execute stmt using @b, @b, @b, @b").Check(testkit.Rows("11 11 11")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) tk.MustExec("prepare stmt from 'select /*+ use_index_merge(t1) */ * from t1 where c=10 or (a >=? and a <= ?);';") tk.MustExec("set @a=9, @b=10, @c=11;") From cc388fc98e1e3539ca3892dd0a5a420070175e3f Mon Sep 17 00:00:00 2001 From: qw4990 Date: Thu, 29 Dec 2022 14:30:21 +0800 Subject: [PATCH 11/12] fixup --- executor/explainfor_test.go | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 4b0fff6fefef4..ddb0578338c6f 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -551,9 +551,9 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 4) - require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) - require.Regexp(t, ".*IndexFullScan.*", res.Rows()[3][0]) + require.Len(t, res.Rows(), 3) + require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexFullScan.*", res.Rows()[2][0]) res = tk.MustQuery("explain format = 'brief' select col1 from UK_GCOL_VIRTUAL_18588 use index(UK_COL1) " + "where col1 between -1696020282760139948 and -2619168038882941276 or col1 < -4004648990067362699;") @@ -589,11 +589,9 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 5) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Equal(t, "lt(test.t.b, 1), or(and(ge(test.t.a, 2), le(test.t.a, 1)), lt(test.t.a, 1))", res.Rows()[1][4]) - require.Regexp(t, ".*IndexReader.*", res.Rows()[2][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[4][0]) + require.Len(t, res.Rows(), 4) + require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) res = tk.MustQuery("explain format = 'brief' select a from t use index(idx) " + "where (a between 0 and 2 or a < 2) and b < 1;") @@ -636,12 +634,11 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 6) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Regexp(t, ".*IndexLookUp.*", res.Rows()[2][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[4][0]) - require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[5][0]) + require.Len(t, res.Rows(), 5) + require.Regexp(t, ".*IndexLookUp.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[2][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[3][0]) + require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[4][0]) res = tk.MustQuery("explain format = 'brief' select /*+ USE_INDEX(t, idx) */ a from t use index(idx) " + "where (a between 0 and 2 or a < 2) and b < 1;") From 5cc158e4469f88c590b9512fee8a28c96d38f95a Mon Sep 17 00:00:00 2001 From: qw4990 Date: Thu, 29 Dec 2022 14:31:55 +0800 Subject: [PATCH 12/12] fixup --- planner/core/prepare_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 4f20721bb9e0a..71eb4c997211d 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -1939,7 +1939,7 @@ func TestPlanCachePointGetAndTableDual(t *testing.T) { tk.MustQuery("execute s0 using @a0, @b0, @a0").Check(testkit.Rows()) tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) tk.MustQuery("execute s0 using @a0, @a0, @b0").Check(testkit.Rows("0000 7777 1")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) tk.MustExec("create table t1(c1 varchar(20), c2 varchar(20), c3 bigint(20), primary key(c1, c2))") tk.MustExec("insert into t1 values('0000','7777',1)")