Skip to content

Commit

Permalink
planner, sessionctx : Add 'last_plan_from_binding' to help know wheth…
Browse files Browse the repository at this point in the history
…er sql's plan is matched with the hints in the binding (#18017)
  • Loading branch information
Reminiscent authored Nov 20, 2020
1 parent 1d668fa commit 2c66371
Show file tree
Hide file tree
Showing 18 changed files with 98 additions and 4 deletions.
22 changes: 22 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1605,3 +1605,25 @@ func (s *testSuite) TestBindingSource(c *C) {
bind = bindData.Bindings[0]
c.Assert(bind.Source, Equals, bindinfo.Capture)
}

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")
}
2 changes: 2 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -918,6 +918,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
Prepared: a.isPreparedStmt,
HasMoreResults: hasMoreResults,
PlanFromCache: sessVars.FoundInPlanCache,
PlanFromBinding: sessVars.FoundInBinding,
RewriteInfo: sessVars.RewritePhaseInfo,
KVTotal: time.Duration(atomic.LoadInt64(&stmtDetail.WaitKVRespDuration)),
PDTotal: time.Duration(atomic.LoadInt64(&stmtDetail.WaitPDRespDuration)),
Expand Down Expand Up @@ -1070,6 +1071,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,
Expand Down
2 changes: 2 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1711,6 +1711,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
vars.PrevFoundInPlanCache = vars.FoundInPlanCache
vars.FoundInPlanCache = false
vars.ClearStmtVars()
vars.PrevFoundInBinding = vars.FoundInBinding
vars.FoundInBinding = false
return
}

Expand Down
4 changes: 4 additions & 0 deletions executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
8 changes: 8 additions & 0 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,6 +484,7 @@ type slowQueryTuple struct {
isInternal bool
succ bool
planFromCache bool
planFromBinding bool
prepared bool
kvTotal float64
pdTotal float64
Expand Down Expand Up @@ -642,6 +643,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:
Expand Down Expand Up @@ -756,6 +759,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))
Expand Down
4 changes: 3 additions & 1 deletion executor/slow_query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ func (s *testExecSerialSuite) 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;
Expand Down Expand Up @@ -107,6 +108,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;
Expand Down Expand Up @@ -134,7 +136,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,10,10,10,10,100,,,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(expectRecordString, Equals, recordString)

Expand Down
1 change: 1 addition & 0 deletions infoschema/perfschema/const.go
Original file line number Diff line number Diff line change
Expand Up @@ -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," +
Expand Down
2 changes: 2 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -782,6 +782,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},
Expand Down Expand Up @@ -1156,6 +1157,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"},
Expand Down
4 changes: 2 additions & 2 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -645,10 +645,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|100|10|10|10|100|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|100|10|10|10|100|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|100|10|10|10|100|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|100|10|10|10|100|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)
Expand Down
10 changes: 10 additions & 0 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,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
Expand Down Expand Up @@ -541,6 +545,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.BoolToOnOff(opt))
return err
}

func init() {
plannercore.OptimizeAstNode = Optimize
}
13 changes: 13 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -722,6 +722,11 @@ 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

// OptimizerUseInvisibleIndexes indicates whether optimizer can use invisible index
OptimizerUseInvisibleIndexes bool

Expand Down Expand Up @@ -881,6 +886,8 @@ func NewSessionVars() *SessionVars {
WindowingUseHighPrecision: true,
PrevFoundInPlanCache: DefTiDBFoundInPlanCache,
FoundInPlanCache: DefTiDBFoundInPlanCache,
PrevFoundInBinding: DefTiDBFoundInBinding,
FoundInBinding: DefTiDBFoundInBinding,
SelectLimit: math.MaxUint64,
AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert,
EnableClusteredIndex: DefTiDBEnableClusteredIndex,
Expand Down Expand Up @@ -1555,6 +1562,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 TiDBEnableCollectExecutionInfo:
config.GetGlobalConfig().EnableCollectExecutionInfo = TiDBOptOn(val)
case SQLSelectLimit:
Expand Down Expand Up @@ -1983,6 +1992,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.
Expand Down Expand Up @@ -2035,6 +2046,7 @@ type SlowQueryLogItems struct {
Succ bool
Prepared bool
PlanFromCache bool
PlanFromBinding bool
HasMoreResults bool
PrevStmt string
Plan string
Expand Down Expand Up @@ -2202,6 +2214,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))
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,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
Expand All @@ -228,6 +229,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) {
DiskMax: diskMax,
Prepared: true,
PlanFromCache: true,
PlanFromBinding: true,
HasMoreResults: true,
KVTotal: 10 * time.Second,
PDTotal: 11 * time.Second,
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1098,6 +1098,7 @@ var defaultSysVars = []*SysVar{
{Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64},
{Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), Type: TypeBool},
{Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool},
{Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool},
{Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: BoolToOnOff(DefTiDBAllowAutoRandExplicitInsert), Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: BoolToOnOff(DefTiDBEnableClusteredIndex), Type: TypeBool},
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,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"

Expand Down Expand Up @@ -568,6 +571,7 @@ const (
DefTiDBMetricSchemaStep = 60 // 60s
DefTiDBMetricSchemaRangeDuration = 60 // 60s
DefTiDBFoundInPlanCache = false
DefTiDBFoundInBinding = false
DefTiDBEnableCollectExecutionInfo = true
DefTiDBAllowAutoRandExplicitInsert = false
DefTiDBEnableClusteredIndex = false
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) {
return CapturePlanBaseline.GetVal(), true, nil
case TiDBFoundInPlanCache:
return BoolToOnOff(s.PrevFoundInPlanCache), true, nil
case TiDBFoundInBinding:
return BoolToOnOff(s.PrevFoundInBinding), true, nil
case TiDBEnableCollectExecutionInfo:
return BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo), true, nil
}
Expand Down
8 changes: 8 additions & 0 deletions sessionctx/variable/varsutil_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,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)
c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep))
c.Assert(vars.EnableChangeColumnType, Equals, DefTiDBChangeColumnType)
Expand Down Expand Up @@ -469,6 +470,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) {
c.Assert(val, Equals, "OFF")
c.Assert(v.systems[TiDBFoundInPlanCache], Equals, "ON")

err = SetSessionSystemVar(v, TiDBFoundInBinding, types.NewStringDatum("1"))
c.Assert(err, IsNil)
val, err = GetSessionSystemVar(v, TiDBFoundInBinding)
c.Assert(err, IsNil)
c.Assert(val, Equals, "OFF")
c.Assert(v.systems[TiDBFoundInBinding], Equals, "ON")

err = SetSessionSystemVar(v, TiDBEnableChangeColumnType, types.NewStringDatum("ON"))
c.Assert(err, IsNil)
val, err = GetSessionSystemVar(v, TiDBEnableChangeColumnType)
Expand Down
11 changes: 11 additions & 0 deletions util/stmtsummary/statement_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,7 @@ type stmtSummaryByDigestElement struct {
// plan cache
planInCache bool
planCacheHits int64
planInBinding bool
// pessimistic execution retry information.
execRetryCount uint
execRetryTime time.Duration
Expand Down Expand Up @@ -219,6 +220,7 @@ type StmtExecInfo struct {
IsInternal bool
Succeed bool
PlanInCache bool
PlanInBinding bool
ExecRetryCount uint
ExecRetryTime time.Duration
execdetails.StmtExecDetails
Expand Down Expand Up @@ -608,6 +610,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)
Expand Down Expand Up @@ -786,6 +789,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
Expand Down Expand Up @@ -913,6 +923,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,
Expand Down
2 changes: 1 addition & 1 deletion util/stmtsummary/statement_summary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -641,7 +641,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)
Expand Down

0 comments on commit 2c66371

Please sign in to comment.