From 5edebf98fd349956459033a611e4c8299be8973a Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 12 Mar 2021 14:50:42 +0800 Subject: [PATCH] planner, sessionctx : Add 'last_plan_from_binding' to help know whether sql's plan is matched with the hints in the binding (#18017) (#21430) --- bindinfo/bind_test.go | 22 ++++++++++++++++++++++ executor/adapter.go | 2 ++ executor/executor.go | 2 ++ executor/set.go | 4 ++++ executor/slow_query.go | 8 ++++++++ executor/slow_query_test.go | 4 +++- infoschema/perfschema/const.go | 1 + infoschema/tables.go | 2 ++ infoschema/tables_test.go | 4 ++-- planner/optimize.go | 10 ++++++++++ sessionctx/variable/session.go | 12 ++++++++++++ sessionctx/variable/session_test.go | 2 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 ++++ sessionctx/variable/varsutil.go | 2 ++ sessionctx/variable/varsutil_test.go | 8 ++++++++ util/stmtsummary/statement_summary.go | 11 +++++++++++ util/stmtsummary/statement_summary_test.go | 2 +- 18 files changed, 97 insertions(+), 4 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index cbc0a08852034..5daecc922a765 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -1541,6 +1541,28 @@ func (s *testSuite) TestCapturePlanBaselineIgnoreTiFlash(c *C) { c.Assert(rows[0][1], Equals, "SELECT /*+ use_index(@`sel_1` `test`.`t` )*/ * FROM `test`.`t`") } +func (s *testSuite) TestSPMHitInfo(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + + c.Assert(tk.HasPlan("SELECT * from t1,t2 where t1.id = t2.id", "HashJoin"), IsTrue) + c.Assert(tk.HasPlan("SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id", "MergeJoin"), IsTrue) + + tk.MustExec("SELECT * from t1,t2 where t1.id = t2.id") + tk.MustQuery(`select @@last_plan_from_binding;`).Check(testkit.Rows("0")) + tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id") + + c.Assert(tk.HasPlan("SELECT * from t1,t2 where t1.id = t2.id", "MergeJoin"), IsTrue) + tk.MustExec("SELECT * from t1,t2 where t1.id = t2.id") + tk.MustQuery(`select @@last_plan_from_binding;`).Check(testkit.Rows("1")) + tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id") +} + func (s *testSuite) TestNotEvolvePlanForReadStorageHint(c *C) { tk := testkit.NewTestKit(c, s.store) s.cleanBindingEnv(tk) diff --git a/executor/adapter.go b/executor/adapter.go index 8a7974e35a081..35476e1d22e75 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -921,6 +921,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { Prepared: a.isPreparedStmt, HasMoreResults: hasMoreResults, PlanFromCache: sessVars.FoundInPlanCache, + PlanFromBinding: sessVars.FoundInBinding, KVTotal: time.Duration(atomic.LoadInt64(&stmtDetail.WaitKVRespDuration)), PDTotal: time.Duration(atomic.LoadInt64(&stmtDetail.WaitPDRespDuration)), BackoffTotal: time.Duration(atomic.LoadInt64(&stmtDetail.BackoffDuration)), @@ -1113,6 +1114,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { IsInternal: sessVars.InRestrictedSQL, Succeed: succ, PlanInCache: sessVars.FoundInPlanCache, + PlanInBinding: sessVars.FoundInBinding, ExecRetryCount: a.retryCount, StmtExecDetails: stmtDetail, Prepared: a.isPreparedStmt, diff --git a/executor/executor.go b/executor/executor.go index 852b05a72d702..7192a1ecc45d9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1772,6 +1772,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.StmtCtx = sc vars.PrevFoundInPlanCache = vars.FoundInPlanCache vars.FoundInPlanCache = false + vars.PrevFoundInBinding = vars.FoundInBinding + vars.FoundInBinding = false return } diff --git a/executor/set.go b/executor/set.go index 75b6b0db95c60..c1753f3ed1313 100644 --- a/executor/set.go +++ b/executor/set.go @@ -184,6 +184,10 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e sessionVars.StmtCtx.AppendWarning(fmt.Errorf("Set operation for '%s' will not take effect", variable.TiDBFoundInPlanCache)) return nil } + if name == variable.TiDBFoundInBinding { + sessionVars.StmtCtx.AppendWarning(fmt.Errorf("Set operation for '%s' will not take effect", variable.TiDBFoundInBinding)) + return nil + } err = variable.SetSessionSystemVar(sessionVars, name, value) if err != nil { return err diff --git a/executor/slow_query.go b/executor/slow_query.go index 68a5cf84f3a42..84bdeedf691f6 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -498,6 +498,7 @@ type slowQueryTuple struct { isInternal bool succ bool planFromCache bool + planFromBinding bool prepared bool kvTotal float64 pdTotal float64 @@ -641,6 +642,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string, st.succ, err = strconv.ParseBool(value) case variable.SlowLogPlanFromCache: st.planFromCache, err = strconv.ParseBool(value) + case variable.SlowLogPlanFromBinding: + st.planFromBinding, err = strconv.ParseBool(value) case variable.SlowLogPlan: st.plan = value case variable.SlowLogPlanDigest: @@ -750,6 +753,11 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { } else { record = append(record, types.NewIntDatum(0)) } + if st.planFromBinding { + record = append(record, types.NewIntDatum(1)) + } else { + record = append(record, types.NewIntDatum(0)) + } record = append(record, types.NewStringDatum(parsePlan(st.plan))) record = append(record, types.NewStringDatum(st.planDigest)) record = append(record, types.NewStringDatum(st.prevStmt)) diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index 09aaf9ad77e6e..9d627d26f1687 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -68,6 +68,7 @@ func (s *testExecSuite) TestParseSlowLogPanic(c *C) { # Mem_max: 70724 # Disk_max: 65536 # Plan_from_cache: true +# Plan_from_binding: true # Succ: false # Plan_digest: 60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4 # Prev_stmt: update t set i = 1; @@ -106,6 +107,7 @@ func (s *testExecSuite) TestParseSlowLogFile(c *C) { # Mem_max: 70724 # Disk_max: 65536 # Plan_from_cache: true +# Plan_from_binding: true # Succ: false # Plan_digest: 60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4 # Prev_stmt: update t set i = 1; @@ -133,7 +135,7 @@ select * from t;` `0,0,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` + `0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,` + `Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` + - `0,0,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + + `0,0,1,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + `update t set i = 1;,select * from t;` c.Assert(recordString, Equals, expectRecordString) diff --git a/infoschema/perfschema/const.go b/infoschema/perfschema/const.go index 86a5f1766739f..157ce4c7ae7c9 100644 --- a/infoschema/perfschema/const.go +++ b/infoschema/perfschema/const.go @@ -415,6 +415,7 @@ const tableEventsStatementsSummaryByDigest = "CREATE TABLE if not exists perform "LAST_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," + "PLAN_IN_CACHE bool NOT NULL," + "PLAN_CACHE_HITS bigint unsigned NOT NULL," + + "PLAN_IN_BINDING bool NOT NULL," + "QUANTILE_95 bigint unsigned NOT NULL," + "QUANTILE_99 bigint unsigned NOT NULL," + "QUANTILE_999 bigint unsigned NOT NULL," + diff --git a/infoschema/tables.go b/infoschema/tables.go index e4dd6fa2f7165..845f78877f271 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -773,6 +773,7 @@ var slowQueryCols = []columnInfo{ {name: variable.SlowLogPrepared, tp: mysql.TypeTiny, size: 1}, {name: variable.SlowLogSucc, tp: mysql.TypeTiny, size: 1}, {name: variable.SlowLogPlanFromCache, tp: mysql.TypeTiny, size: 1}, + {name: variable.SlowLogPlanFromBinding, tp: mysql.TypeTiny, size: 1}, {name: variable.SlowLogPlan, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength}, {name: variable.SlowLogPlanDigest, tp: mysql.TypeVarchar, size: 128}, {name: variable.SlowLogPrevStmt, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength}, @@ -1234,6 +1235,7 @@ var tableStatementsSummaryCols = []columnInfo{ {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the last time"}, {name: "PLAN_IN_CACHE", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the last statement hit plan cache"}, {name: "PLAN_CACHE_HITS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag, comment: "The number of times these statements hit plan cache"}, + {name: "PLAN_IN_BINDING", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the last statement is matched with the hints in the binding"}, {name: "QUERY_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled original statement"}, {name: "PREV_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The previous statement before commit"}, {name: "PLAN_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of its execution plan"}, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6759efa55f8c1..5e425730a9231 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -638,10 +638,10 @@ func (s *testTableSuite) TestSlowQuery(c *C) { tk.MustExec("set time_zone = '+08:00';") re := tk.MustQuery("select * from information_schema.slow_query") re.Check(testutil.RowsWithSep("|", - "2019-02-12 19:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) + "2019-02-12 19:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0||0|1|1|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0||0|1|1|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) // Test for long query. f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) diff --git a/planner/optimize.go b/planner/optimize.go index 09d2de310616a..0eefceffc2262 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -124,6 +124,10 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in sctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("sql_select_limit is set, so plan binding is not activated")) return bestPlan, names, nil } + err = setFoundInBinding(sctx, true) + if err != nil { + return nil, nil, err + } bestPlanHint := plannercore.GenHintsFromPhysicalPlan(bestPlan) if len(bindRecord.Bindings) > 0 { orgBinding := bindRecord.Bindings[0] // the first is the original binding @@ -495,6 +499,12 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin return } +func setFoundInBinding(sctx sessionctx.Context, opt bool) error { + vars := sctx.GetSessionVars() + err := vars.SetSystemVar(variable.TiDBFoundInBinding, variable.BoolToIntStr(opt)) + return err +} + func init() { plannercore.OptimizeAstNode = Optimize } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d9c1c44fcca9d..2ec0d56e965c8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -656,6 +656,10 @@ type SessionVars struct { // PrevFoundInPlanCache indicates whether the last statement was found in plan cache. PrevFoundInPlanCache bool + // FoundInBinding indicates whether the execution plan is matched with the hints in the binding. + FoundInBinding bool + // PrevFoundInBinding indicates whether the last execution plan is matched with the hints in the binding. + PrevFoundInBinding bool // SelectLimit limits the max counts of select statement's output SelectLimit uint64 @@ -760,6 +764,8 @@ func NewSessionVars() *SessionVars { WindowingUseHighPrecision: true, PrevFoundInPlanCache: DefTiDBFoundInPlanCache, FoundInPlanCache: DefTiDBFoundInPlanCache, + PrevFoundInBinding: DefTiDBFoundInBinding, + FoundInBinding: DefTiDBFoundInBinding, SelectLimit: math.MaxUint64, AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, @@ -1370,6 +1376,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) case TiDBFoundInPlanCache: s.FoundInPlanCache = TiDBOptOn(val) + case TiDBFoundInBinding: + s.FoundInBinding = TiDBOptOn(val) case SQLSelectLimit: result, err := strconv.ParseUint(val, 10, 64) if err != nil { @@ -1643,6 +1651,8 @@ const ( SlowLogPrepared = "Prepared" // SlowLogPlanFromCache is used to indicate whether this plan is from plan cache. SlowLogPlanFromCache = "Plan_from_cache" + // SlowLogPlanFromBinding is used to indicate whether this plan is matched with the hints in the binding. + SlowLogPlanFromBinding = "Plan_from_binding" // SlowLogHasMoreResults is used to indicate whether this sql has more following results. SlowLogHasMoreResults = "Has_more_results" // SlowLogSucc is used to indicate whether this sql execute successfully. @@ -1695,6 +1705,7 @@ type SlowQueryLogItems struct { Succ bool Prepared bool PlanFromCache bool + PlanFromBinding bool HasMoreResults bool PrevStmt string Plan string @@ -1862,6 +1873,7 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { writeSlowLogItem(&buf, SlowLogPrepared, strconv.FormatBool(logItems.Prepared)) writeSlowLogItem(&buf, SlowLogPlanFromCache, strconv.FormatBool(logItems.PlanFromCache)) + writeSlowLogItem(&buf, SlowLogPlanFromBinding, strconv.FormatBool(logItems.PlanFromBinding)) writeSlowLogItem(&buf, SlowLogHasMoreResults, strconv.FormatBool(logItems.HasMoreResults)) writeSlowLogItem(&buf, SlowLogKVTotal, strconv.FormatFloat(logItems.KVTotal.Seconds(), 'f', -1, 64)) writeSlowLogItem(&buf, SlowLogPDTotal, strconv.FormatFloat(logItems.PDTotal.Seconds(), 'f', -1, 64)) diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 7ab9927005305..3e13fd0b0ba3e 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -201,6 +201,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # Disk_max: 6666 # Prepared: true # Plan_from_cache: true +# Plan_from_binding: true # Has_more_results: true # KV_total: 10 # PD_total: 11 @@ -226,6 +227,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { DiskMax: diskMax, Prepared: true, PlanFromCache: true, + PlanFromBinding: true, HasMoreResults: true, KVTotal: 10 * time.Second, PDTotal: 11 * time.Second, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cef0093d86eb8..5049c650fb48a 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -734,6 +734,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBQueryLogMaxLen, strconv.Itoa(logutil.DefaultQueryLogMaxLen)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, {ScopeSession, TiDBFoundInPlanCache, BoolToIntStr(DefTiDBFoundInPlanCache)}, + {ScopeSession, TiDBFoundInBinding, BoolToIntStr(DefTiDBFoundInBinding)}, {ScopeSession, TiDBEnableCollectExecutionInfo, BoolToIntStr(DefTiDBEnableCollectExecutionInfo)}, {ScopeGlobal | ScopeSession, TiDBAllowAutoRandExplicitInsert, boolToOnOff(DefTiDBAllowAutoRandExplicitInsert)}, {ScopeGlobal | ScopeSession, TiDBSlowLogMasking, BoolToIntStr(DefTiDBRedactLog)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d81cdf775db46..b48920e185f8b 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -181,6 +181,9 @@ const ( // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache TiDBFoundInPlanCache = "last_plan_from_cache" + // TiDBFoundInBinding indicates whether the last statement was matched with the hints in the binding. + TiDBFoundInBinding = "last_plan_from_binding" + // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. TiDBAllowAutoRandExplicitInsert = "allow_auto_random_explicit_insert" ) @@ -525,6 +528,7 @@ const ( DefTiDBMetricSchemaStep = 60 // 60s DefTiDBMetricSchemaRangeDuration = 60 // 60s DefTiDBFoundInPlanCache = false + DefTiDBFoundInBinding = false DefTiDBEnableCollectExecutionInfo = true DefTiDBAllowAutoRandExplicitInsert = false DefTiDBRedactLog = false diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index f5fcabe795211..676dcc2fb455c 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -163,6 +163,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return CapturePlanBaseline.GetVal(), true, nil case TiDBFoundInPlanCache: return BoolToIntStr(s.PrevFoundInPlanCache), true, nil + case TiDBFoundInBinding: + return BoolToIntStr(s.PrevFoundInBinding), true, nil case TiDBEnableCollectExecutionInfo: return BoolToIntStr(config.GetGlobalConfig().EnableCollectExecutionInfo), true, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index be47632f29636..536d490a57847 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -87,6 +87,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) c.Assert(vars.EnableFastAnalyze, Equals, DefTiDBUseFastAnalyze) c.Assert(vars.FoundInPlanCache, Equals, DefTiDBFoundInPlanCache) + c.Assert(vars.FoundInBinding, Equals, DefTiDBFoundInBinding) c.Assert(vars.AllowAutoRandExplicitInsert, Equals, DefTiDBAllowAutoRandExplicitInsert) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) @@ -456,6 +457,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "0") c.Assert(v.systems[TiDBFoundInPlanCache], Equals, "1") + err = SetSessionSystemVar(v, TiDBFoundInBinding, types.NewStringDatum("1")) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBFoundInBinding) + c.Assert(err, IsNil) + c.Assert(val, Equals, "0") + c.Assert(v.systems[TiDBFoundInBinding], Equals, "1") + err = SetSessionSystemVar(v, "UnknownVariable", types.NewStringDatum("on")) c.Assert(err, ErrorMatches, ".*]Unknown system variable 'UnknownVariable'") } diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index ccd2d8b9b4f70..1250a6a8bf780 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -183,6 +183,7 @@ type stmtSummaryByDigestElement struct { // plan cache planInCache bool planCacheHits int64 + planInBinding bool // pessimistic execution retry information. execRetryCount uint execRetryTime time.Duration @@ -214,6 +215,7 @@ type StmtExecInfo struct { IsInternal bool Succeed bool PlanInCache bool + PlanInBinding bool ExecRetryCount uint ExecRetryTime time.Duration execdetails.StmtExecDetails @@ -627,6 +629,7 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS authUsers: make(map[string]struct{}), planInCache: false, planCacheHits: 0, + planInBinding: false, prepared: sei.Prepared, } ssElement.add(sei, intervalSeconds) @@ -782,6 +785,13 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco ssElement.planInCache = false } + // SPM + if sei.PlanInBinding { + ssElement.planInBinding = true + } else { + ssElement.planInBinding = false + } + // other ssElement.sumAffectedRows += sei.StmtCtx.AffectedRows() ssElement.sumMem += sei.MemMax @@ -899,6 +909,7 @@ func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) types.NewTime(types.FromGoTime(ssElement.lastSeen), mysql.TypeTimestamp, 0), ssElement.planInCache, ssElement.planCacheHits, + ssElement.planInBinding, ssElement.sampleSQL, ssElement.prevSQL, ssbd.planDigest, diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 39df0a45147bb..51adb3d6f22a9 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -616,7 +616,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, 0, 0, 1, "txnLock:1", stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, 0, 0, 0, 0, 0, stmtExecInfo1.StmtCtx.AffectedRows(), - t, t, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} + t, t, 0, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} match(c, datums[0], expectedDatum...) datums = s.ssMap.ToHistoryDatum(nil, true) c.Assert(len(datums), Equals, 1)