Skip to content

Commit

Permalink
*: add binary plan in stmt summary and slow log/query (#35698)
Browse files Browse the repository at this point in the history
close #35889
  • Loading branch information
time-and-fate authored Jul 20, 2022
1 parent 6256c34 commit a52656f
Show file tree
Hide file tree
Showing 18 changed files with 565 additions and 96 deletions.
6 changes: 3 additions & 3 deletions cmd/explaintest/r/index_merge.result
Original file line number Diff line number Diff line change
Expand Up @@ -452,9 +452,9 @@ c1 c2 c3
///// MEMORY Table
explain select count(c1) from (select /*+ use_index_merge(t_alias), stream_agg() */ count(1) c1 from information_schema.statements_summary where sum_latency >= 0 or max_latency >= 0 order by 1) dt;
id estRows task access object operator info
StreamAgg_10 1.00 root funcs:count(Column#92)->Column#93
└─Sort_11 1.00 root Column#92
└─StreamAgg_14 1.00 root funcs:count(1)->Column#92
StreamAgg_10 1.00 root funcs:count(Column#93)->Column#94
└─Sort_11 1.00 root Column#93
└─StreamAgg_14 1.00 root funcs:count(1)->Column#93
└─MemTableScan_18 10000.00 root table:STATEMENTS_SUMMARY
show warnings;
Level Code Message
Expand Down
87 changes: 57 additions & 30 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1032,6 +1032,12 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
diskMax := stmtCtx.DiskTracker.MaxConsumed()
_, planDigest := getPlanDigest(stmtCtx)

binaryPlan := ""
if variable.GenerateBinaryPlan.Load() {
binaryPlan = getBinaryPlan(a.Ctx)
binaryPlan = variable.SlowLogBinaryPlanPrefix + binaryPlan + variable.SlowLogPlanSuffix
}

resultRows := GetResultRowsCount(stmtCtx, a.Plan)

slowItems := &variable.SlowQueryLogItems{
Expand All @@ -1052,6 +1058,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
Succ: succ,
Plan: getPlanTree(stmtCtx),
PlanDigest: planDigest.String(),
BinaryPlan: binaryPlan,
Prepared: a.isPreparedStmt,
HasMoreResults: hasMoreResults,
PlanFromCache: sessVars.FoundInPlanCache,
Expand Down Expand Up @@ -1147,6 +1154,18 @@ func getFlatPlan(stmtCtx *stmtctx.StatementContext) *plannercore.FlatPhysicalPla
return nil
}

func getBinaryPlan(sCtx sessionctx.Context) string {
stmtCtx := sCtx.GetSessionVars().StmtCtx
binaryPlan := stmtCtx.GetBinaryPlan()
if len(binaryPlan) > 0 {
return binaryPlan
}
flat := getFlatPlan(stmtCtx)
binaryPlan = plannercore.BinaryPlanStrFromFlatPlan(sCtx, flat)
stmtCtx.SetBinaryPlan(binaryPlan)
return binaryPlan
}

// getPlanTree will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement.
func getPlanTree(stmtCtx *stmtctx.StatementContext) string {
cfg := config.GetGlobalConfig()
Expand Down Expand Up @@ -1245,6 +1264,13 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
planGenerator := func() (string, string) {
return getEncodedPlan(stmtCtx, !sessVars.InRestrictedSQL)
}
var binPlanGen func() string
if variable.GenerateBinaryPlan.Load() {
binPlanGen = func() string {
binPlan := getBinaryPlan(a.Ctx)
return binPlan
}
}
// Generating plan digest is slow, only generate it once if it's 'Point_Get'.
// If it's a point get, different SQLs leads to different plans, so SQL digest
// is enough to distinguish different plans in this case.
Expand Down Expand Up @@ -1288,36 +1314,37 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
resultRows := GetResultRowsCount(stmtCtx, a.Plan)

stmtExecInfo := &stmtsummary.StmtExecInfo{
SchemaName: strings.ToLower(sessVars.CurrentDB),
OriginalSQL: sql,
Charset: charset,
Collation: collation,
NormalizedSQL: normalizedSQL,
Digest: digest.String(),
PrevSQL: prevSQL,
PrevSQLDigest: prevSQLDigest,
PlanGenerator: planGenerator,
PlanDigest: planDigest,
PlanDigestGen: planDigestGen,
User: userString,
TotalLatency: costTime,
ParseLatency: sessVars.DurationParse,
CompileLatency: sessVars.DurationCompile,
StmtCtx: stmtCtx,
CopTasks: copTaskInfo,
ExecDetail: &execDetail,
MemMax: memMax,
DiskMax: diskMax,
StartTime: sessVars.StartTime,
IsInternal: sessVars.InRestrictedSQL,
Succeed: succ,
PlanInCache: sessVars.FoundInPlanCache,
PlanInBinding: sessVars.FoundInBinding,
ExecRetryCount: a.retryCount,
StmtExecDetails: stmtDetail,
ResultRows: resultRows,
TiKVExecDetails: tikvExecDetail,
Prepared: a.isPreparedStmt,
SchemaName: strings.ToLower(sessVars.CurrentDB),
OriginalSQL: sql,
Charset: charset,
Collation: collation,
NormalizedSQL: normalizedSQL,
Digest: digest.String(),
PrevSQL: prevSQL,
PrevSQLDigest: prevSQLDigest,
PlanGenerator: planGenerator,
BinaryPlanGenerator: binPlanGen,
PlanDigest: planDigest,
PlanDigestGen: planDigestGen,
User: userString,
TotalLatency: costTime,
ParseLatency: sessVars.DurationParse,
CompileLatency: sessVars.DurationCompile,
StmtCtx: stmtCtx,
CopTasks: copTaskInfo,
ExecDetail: &execDetail,
MemMax: memMax,
DiskMax: diskMax,
StartTime: sessVars.StartTime,
IsInternal: sessVars.InRestrictedSQL,
Succeed: succ,
PlanInCache: sessVars.FoundInPlanCache,
PlanInBinding: sessVars.FoundInBinding,
ExecRetryCount: a.retryCount,
StmtExecDetails: stmtDetail,
ResultRows: resultRows,
TiKVExecDetails: tikvExecDetail,
Prepared: a.isPreparedStmt,
}
if a.retryCount > 0 {
stmtExecInfo.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime)
Expand Down
8 changes: 8 additions & 0 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -740,6 +740,14 @@ func getColumnValueFactoryByName(sctx sessionctx.Context, colName string, column
row[columnIdx] = types.NewStringDatum(plan)
return true, nil
}, nil
case variable.SlowLogBinaryPlan:
return func(row []types.Datum, value string, tz *time.Location, checker *slowLogChecker) (bool, error) {
if strings.HasPrefix(value, variable.SlowLogBinaryPlanPrefix) {
value = value[len(variable.SlowLogBinaryPlanPrefix) : len(value)-len(variable.SlowLogPlanSuffix)]
}
row[columnIdx] = types.NewStringDatum(value)
return true, nil
}, nil
case variable.SlowLogConnIDStr, variable.SlowLogExecRetryCount, variable.SlowLogPreprocSubQueriesStr,
execdetails.WriteKeysStr, execdetails.WriteSizeStr, execdetails.PrewriteRegionStr, execdetails.TxnRetryStr,
execdetails.RequestCountStr, execdetails.TotalKeysStr, execdetails.ProcessKeysStr,
Expand Down
4 changes: 2 additions & 2 deletions executor/slow_query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ select * from t;`
`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,` +
`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,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`update t set i = 1;,select * from t;`
`,update t set i = 1;,select * from t;`
require.Equal(t, expectRecordString, recordString)

// Issue 20928
Expand All @@ -186,7 +186,7 @@ select * from t;`
`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,` +
`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,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`update t set i = 1;,select * from t;`
`,update t set i = 1;,select * from t;`
require.Equal(t, expectRecordString, recordString)

// fix sql contain '# ' bug
Expand Down
2 changes: 2 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -888,6 +888,7 @@ var slowQueryCols = []columnInfo{
{name: variable.SlowLogHasMoreResults, tp: mysql.TypeTiny, size: 1},
{name: variable.SlowLogPlan, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength},
{name: variable.SlowLogPlanDigest, tp: mysql.TypeVarchar, size: 128},
{name: variable.SlowLogBinaryPlan, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength},
{name: variable.SlowLogPrevStmt, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength},
{name: variable.SlowLogQuerySQLStr, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength},
}
Expand Down Expand Up @@ -1294,6 +1295,7 @@ var tableStatementsSummaryCols = []columnInfo{
{name: stmtsummary.PrevSampleTextStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The previous statement before commit"},
{name: stmtsummary.PlanDigestStr, tp: mysql.TypeVarchar, size: 64, comment: "Digest of its execution plan"},
{name: stmtsummary.PlanStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled execution plan"},
{name: stmtsummary.BinaryPlan, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled binary plan"},
}

var tableStorageStatsCols = []columnInfo{
Expand Down
8 changes: 4 additions & 4 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -567,13 +567,13 @@ func TestSlowQuery(t *testing.T) {
tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName))
tk.MustExec("set time_zone = '+08:00';")
re := tk.MustQuery("select * from information_schema.slow_query")
re.Check(testkit.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|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;",
"2021-09-08|14:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0||||INSERT INTO ...;",
re.Check(testkit.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|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4||update t set i = 2;|select * from t_slim;",
"2021-09-08|14:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0|||||INSERT INTO ...;",
))
tk.MustExec("set time_zone = '+00:00';")
re = tk.MustQuery("select * from information_schema.slow_query")
re.Check(testkit.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|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;",
"2021-09-08|06:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0||||INSERT INTO ...;",
re.Check(testkit.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|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4||update t set i = 2;|select * from t_slim;",
"2021-09-08|06:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0|||||INSERT INTO ...;",
))

// Test for long query.
Expand Down
Loading

0 comments on commit a52656f

Please sign in to comment.