From c017999d023db1357a3ee3faf8d57d6a43456ea5 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 6 Feb 2023 18:53:57 +0800 Subject: [PATCH 1/2] sessionctx: fix hidden and user var problems in encoding session states (#40754) close pingcap/tidb#40614 --- session/session.go | 5 +-- .../sessionstates/session_states_test.go | 3 +- sessionctx/variable/BUILD.bazel | 2 ++ sessionctx/variable/session.go | 6 ++-- sessionctx/variable/session_test.go | 36 +++++++++++++++++++ 5 files changed, 45 insertions(+), 7 deletions(-) diff --git a/session/session.go b/session/session.go index f6403c67b258c..834a1b645d3c6 100644 --- a/session/session.go +++ b/session/session.go @@ -4150,8 +4150,9 @@ func (s *session) EncodeSessionStates(ctx context.Context, sctx sessionctx.Conte sessionStates.SystemVars = make(map[string]string) for _, sv := range variable.GetSysVars() { switch { - case sv.Hidden, sv.HasNoneScope(), sv.HasInstanceScope(), !sv.HasSessionScope(): - // Hidden and none-scoped variables cannot be modified. + case sv.HasNoneScope(), sv.HasInstanceScope(), !sv.HasSessionScope(): + // Hidden attribute is deprecated. + // None-scoped variables cannot be modified. // Instance-scoped variables don't need to be encoded. // Noop variables should also be migrated even if they are noop. continue diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 4d1541cc9443d..a4a92c55a4495 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -107,8 +107,9 @@ func TestSystemVars(t *testing.T) { }, { // hidden variable - inSessionStates: false, + inSessionStates: true, varName: variable.TiDBTxnReadTS, + expectedValue: "", }, { // none-scoped variable diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index c1d6e0b8b9651..60fec443c6a0a 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -96,10 +96,12 @@ go_test( "//parser/mysql", "//parser/terror", "//planner/core", + "//sessionctx/sessionstates", "//sessionctx/stmtctx", "//testkit", "//testkit/testsetup", "//types", + "//util", "//util/chunk", "//util/execdetails", "//util/gctuner", diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 036886457e323..64c79ea32646e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2320,16 +2320,16 @@ func (s *SessionVars) GetTemporaryTable(tblInfo *model.TableInfo) tableutil.Temp // EncodeSessionStates saves session states into SessionStates. func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *sessionstates.SessionStates) (err error) { // Encode user-defined variables. + s.userVars.lock.RLock() sessionStates.UserVars = make(map[string]*types.Datum, len(s.userVars.values)) sessionStates.UserVarTypes = make(map[string]*ptypes.FieldType, len(s.userVars.types)) - s.userVars.lock.RLock() - defer s.userVars.lock.RUnlock() for name, userVar := range s.userVars.values { sessionStates.UserVars[name] = userVar.Clone() } for name, userVarType := range s.userVars.types { sessionStates.UserVarTypes[name] = userVarType.Clone() } + s.userVars.lock.RUnlock() // Encode other session contexts. sessionStates.PreparedStmtID = s.preparedStmtID @@ -2357,11 +2357,9 @@ func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *se // DecodeSessionStates restores session states from SessionStates. func (s *SessionVars) DecodeSessionStates(ctx context.Context, sessionStates *sessionstates.SessionStates) (err error) { // Decode user-defined variables. - s.userVars.values = make(map[string]types.Datum, len(sessionStates.UserVars)) for name, userVar := range sessionStates.UserVars { s.SetUserVarVal(name, *userVar.Clone()) } - s.userVars.types = make(map[string]*ptypes.FieldType, len(sessionStates.UserVarTypes)) for name, userVarType := range sessionStates.UserVarTypes { s.SetUserVarType(name, userVarType.Clone()) } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 5df5e187088d0..62c94e91867e9 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -15,6 +15,8 @@ package variable_test import ( + "context" + "strconv" "sync" "testing" "time" @@ -25,10 +27,12 @@ import ( "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/sessionstates" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" + util2 "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" @@ -533,3 +537,35 @@ func TestPretectedTSList(t *testing.T) { require.Equal(t, uint64(0), lst.GetMinProtectedTS(1)) require.Equal(t, 0, lst.Size()) } + +func TestUserVarConcurrently(t *testing.T) { + sv := variable.NewSessionVars(nil) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + var wg util2.WaitGroupWrapper + wg.Run(func() { + for i := 0; ; i++ { + select { + case <-time.After(time.Millisecond): + name := strconv.Itoa(i) + sv.SetUserVarVal(name, types.Datum{}) + sv.GetUserVarVal(name) + case <-ctx.Done(): + return + } + } + }) + wg.Run(func() { + for { + select { + case <-time.After(time.Millisecond): + var states sessionstates.SessionStates + require.NoError(t, sv.EncodeSessionStates(ctx, &states)) + require.NoError(t, sv.DecodeSessionStates(ctx, &states)) + case <-ctx.Done(): + return + } + } + }) + wg.Wait() + cancel() +} From ba41d921151855e7d3cd415ab24df8fcf01f93a6 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 6 Feb 2023 19:49:57 +0800 Subject: [PATCH 2/2] Revert "planner: skip the plan cache if non-int values are converted into int when optimization" (#41077) ref pingcap/tidb#41032 --- expression/builtin_compare.go | 41 +++++++++++++++-------------- planner/core/expression_rewriter.go | 2 +- planner/core/plan_cache_test.go | 19 ------------- 3 files changed, 22 insertions(+), 40 deletions(-) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index bcb27a1233da7..dec5d06983679 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -1565,17 +1565,33 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express arg0Type, arg1Type := args[0].GetType(), args[1].GetType() arg0IsInt := arg0Type.EvalType() == types.ETInt arg1IsInt := arg1Type.EvalType() == types.ETInt + arg0IsString := arg0Type.EvalType() == types.ETString + arg1IsString := arg1Type.EvalType() == types.ETString arg0, arg0IsCon := args[0].(*Constant) arg1, arg1IsCon := args[1].(*Constant) isExceptional, finalArg0, finalArg1 := false, args[0], args[1] isPositiveInfinite, isNegativeInfinite := false, false - // int non-constant [cmp] non-int constant - if arg0IsInt && !arg0IsCon && !arg1IsInt && arg1IsCon { - if MaybeOverOptimized4PlanCache(ctx, []Expression{arg1}) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg1.String())) + if MaybeOverOptimized4PlanCache(ctx, args) { + // 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) { + var reason error + if arg1IsString { + reason = errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg1.String()) + } else { // arg0IsString + 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 } - + } else if !ctx.GetSessionVars().StmtCtx.UseCache { + // We should remove the mutable constant for correctness, because its value may be changed. + RemoveMutableConst(ctx, args) + } + // int non-constant [cmp] non-int constant + if arg0IsInt && !arg0IsCon && !arg1IsInt && arg1IsCon { arg1, isExceptional = RefineComparedConstant(ctx, *arg0Type, arg1, c.op) // Why check not null flag // eg: int_col > const_val(which is less than min_int32) @@ -1603,11 +1619,6 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express } // non-int constant [cmp] int non-constant if arg1IsInt && !arg1IsCon && !arg0IsInt && arg0IsCon { - if MaybeOverOptimized4PlanCache(ctx, []Expression{arg0}) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to INT", arg0.String())) - RemoveMutableConst(ctx, args) - } - arg0, isExceptional = RefineComparedConstant(ctx, *arg1Type, arg0, symmetricOp[c.op]) if !isExceptional || (isExceptional && mysql.HasNotNullFlag(arg1Type.GetFlag())) { finalArg0 = arg0 @@ -1625,11 +1636,6 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express } // int constant [cmp] year type if arg0IsCon && arg0IsInt && arg1Type.GetType() == mysql.TypeYear && !arg0.Value.IsNull() { - if MaybeOverOptimized4PlanCache(ctx, []Expression{arg0}) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to YEAR", arg0.String())) - RemoveMutableConst(ctx, args) - } - adjusted, failed := types.AdjustYear(arg0.Value.GetInt64(), false) if failed == nil { arg0.Value.SetInt64(adjusted) @@ -1638,11 +1644,6 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express } // year type [cmp] int constant if arg1IsCon && arg1IsInt && arg0Type.GetType() == mysql.TypeYear && !arg1.Value.IsNull() { - if MaybeOverOptimized4PlanCache(ctx, []Expression{arg1}) { - ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to YEAR", arg1.String())) - RemoveMutableConst(ctx, args) - } - adjusted, failed := types.AdjustYear(arg1.Value.GetInt64(), false) if failed == nil { arg1.Value.SetInt64(adjusted) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index c7af8385d556c..a2d1f242a0ff6 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1563,7 +1563,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field continue // no need to refine it } er.sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("skip plan-cache: '%v' may be converted to INT", c.String())) - expression.RemoveMutableConst(er.sctx, args) + expression.RemoveMutableConst(er.sctx, []expression.Expression{c}) } args[i], isExceptional = expression.RefineComparedConstant(er.sctx, *leftFt, c, opcode.EQ) if isExceptional { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index e3a0d276e9815..8278050681553 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -506,22 +506,3 @@ func TestPlanCacheWithLimit(t *testing.T) { tk.MustExec("execute stmt using @a") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: limit count more than 10000")) } - -func TestIssue40679(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 use index(a) where a < ?'") - tk.MustExec("set @a1=1.1") - tk.MustExec("execute st using @a1") - - tkProcess := tk.Session().ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - require.True(t, strings.Contains(rows[1][0].(string), "RangeScan")) // RangeScan not FullScan - - tk.MustExec("execute st using @a1") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: '1.1' may be converted to INT")) -}