From a52656fde4c83f483c9b3564b1cc4199d613f5b1 Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 20 Jul 2022 18:21:09 +0800 Subject: [PATCH] *: add binary plan in stmt summary and slow log/query (#35698) close pingcap/tidb#35889 --- cmd/explaintest/r/index_merge.result | 6 +- executor/adapter.go | 87 ++-- executor/slow_query.go | 8 + executor/slow_query_test.go | 4 +- infoschema/tables.go | 2 + infoschema/tables_test.go | 8 +- planner/core/binary_plan_test.go | 371 ++++++++++++++++++ planner/core/common_plans.go | 6 +- planner/core/main_test.go | 1 + .../core/testdata/binary_plan_suite_out.json | 7 +- sessionctx/stmtctx/stmtctx.go | 11 + sessionctx/variable/session.go | 12 + sessionctx/variable/sysvar.go | 4 + sessionctx/variable/tidb_vars.go | 3 + util/plancodec/codec.go | 10 + util/stmtsummary/reader.go | 4 + util/stmtsummary/statement_summary.go | 115 +++--- util/stmtsummary/statement_summary_test.go | 2 +- 18 files changed, 565 insertions(+), 96 deletions(-) diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 5c7fece7d5161..32b2369d0c549 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -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 diff --git a/executor/adapter.go b/executor/adapter.go index 88bd635b6e670..6083cd05f9f1c 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -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{ @@ -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, @@ -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() @@ -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. @@ -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) diff --git a/executor/slow_query.go b/executor/slow_query.go index 1708ebe7cc6f4..395d8f4eba8ac 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -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, diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index dea0ab2cb16cd..32412d2f1ac70 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -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 @@ -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 diff --git a/infoschema/tables.go b/infoschema/tables.go index e6c843e45f8bf..f8584724b2704 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -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}, } @@ -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{ diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 5e7b5b3bcc209..36aff22cdfaaf 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -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. diff --git a/planner/core/binary_plan_test.go b/planner/core/binary_plan_test.go index 4e07bf0e51ffe..be8aadd6e7f4c 100644 --- a/planner/core/binary_plan_test.go +++ b/planner/core/binary_plan_test.go @@ -17,13 +17,20 @@ package core_test import ( "encoding/base64" "fmt" + "os" "regexp" + "strings" "testing" "github.com/golang/snappy" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" ) @@ -76,6 +83,25 @@ func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { // If we don't set this, it will be false sometimes and the cost in the result will be different. tk.MustExec("set @@tidb_enable_chunk_rpc=true") + // Prepare the slow log + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + tk.MustExec("set tidb_slow_log_threshold=0;") + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + }() + var input []string var output []struct { SQL string @@ -98,6 +124,13 @@ func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { require.Equal(t, len(result), 1, comment) s := result[0] + // assert that the binary plan in the slow log is the same as the result in the EXPLAIN statement + slowLogResult := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query = "` + test + `;" ` + + "order by time desc limit 1").Rows()) + require.Lenf(t, slowLogResult, 1, comment) + require.Equal(t, s, slowLogResult[0], comment) + b, err := base64.StdEncoding.DecodeString(s) require.NoError(t, err) b, err = snappy.Decode(nil, b) @@ -114,6 +147,344 @@ func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { } } +func TestBinaryPlanSwitch(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + + tk.MustExec("use test") + // 1. assert binary plan is generated if the variable is turned on + tk.MustExec("set global tidb_generate_binary_plan = 1") + tk.MustQuery("select sleep(1)") + + result := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query like "%select sleep(1)%" and query not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s := result[0] + b, err := base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.statements_summary " + + `where QUERY_SAMPLE_TEXT like "%select sleep(1)%" and QUERY_SAMPLE_TEXT not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s = result[0] + b, err = base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary = &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + + // 2. assert binary plan is not generated if the variable is turned off + tk.MustExec("set global tidb_generate_binary_plan = 0") + tk.MustQuery("select 1 > sleep(1)") + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query like "%select 1 > sleep(1)%" and query not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s = result[0] + require.Empty(t, s) + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.statements_summary " + + `where QUERY_SAMPLE_TEXT like "%select 1 > sleep(1)%" and QUERY_SAMPLE_TEXT not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s = result[0] + require.Empty(t, s) +} + +// TestTooLongBinaryPlan asserts that if the binary plan is larger than 1024*1024 bytes, it should be output to slow query but not to stmt summary. +func TestTooLongBinaryPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + + tk.MustExec("use test") + tk.MustExec("drop table if exists th") + tk.MustExec("set @@session.tidb_enable_table_partition = 1") + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;") + tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i") + + result := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query like "%th t1 join th t2 join th t3%" and query not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s := result[0] + require.Greater(t, len(s), stmtsummary.MaxEncodedPlanSizeInBytes) + b, err := base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + require.False(t, binary.DiscardedDueToTooLong) + require.True(t, binary.WithRuntimeStats) + require.NotNil(t, binary.Main) + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.statements_summary " + + `where QUERY_SAMPLE_TEXT like "%th t1 join th t2 join th t3%" and QUERY_SAMPLE_TEXT not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s = result[0] + b, err = base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary = &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + require.True(t, binary.DiscardedDueToTooLong) + require.Nil(t, binary.Main) + require.Nil(t, binary.Ctes) +} + +// TestLongBinaryPlan asserts that if the binary plan is smaller than 1024*1024 bytes, it should be output to both slow query and stmt summary. +// The size of the binary plan in this test case is designed to be larger than 1024*1024*0.85 bytes but smaller than 1024*1024 bytes. +func TestLongBinaryPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + + tk.MustExec("use test") + + tk.MustExec("drop table if exists th") + tk.MustExec("set @@session.tidb_enable_table_partition = 1") + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 1700;") + tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i") + + result := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query like "%th t1 join th t2 join th t3%" and query not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s1 := result[0] + // The binary plan in this test case is expected to be slightly smaller than MaxEncodedPlanSizeInBytes. + // If the size of the binary plan changed and this case failed in the future, you can adjust the partition numbers in the CREATE TABLE statement above. + require.Less(t, len(s1), stmtsummary.MaxEncodedPlanSizeInBytes) + require.Greater(t, len(s1), int(float64(stmtsummary.MaxEncodedPlanSizeInBytes)*0.85)) + b, err := base64.StdEncoding.DecodeString(s1) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + require.False(t, binary.DiscardedDueToTooLong) + require.True(t, binary.WithRuntimeStats) + require.NotNil(t, binary.Main) + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.statements_summary " + + `where QUERY_SAMPLE_TEXT like "%th t1 join th t2 join th t3%" and QUERY_SAMPLE_TEXT not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s2 := result[0] + require.Equal(t, s1, s2) +} + +func TestBinaryPlanOfPreparedStmt(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + + tk.MustExec("use test") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int);") + tk.MustExec("insert into t value(30,30);") + tk.MustExec(`prepare stmt from "select sleep(1), b from t where a > ?"`) + tk.MustExec("set @a = 20") + tk.MustQuery("execute stmt using @a") + + result := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query like "%select sleep%" and query not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s1 := result[0] + b, err := base64.StdEncoding.DecodeString(s1) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + require.False(t, binary.DiscardedDueToTooLong) + require.True(t, binary.WithRuntimeStats) + require.NotNil(t, binary.Main) + + result = testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.statements_summary " + + `where QUERY_SAMPLE_TEXT like "%select sleep%" and QUERY_SAMPLE_TEXT not like "%like%" ` + + "limit 1;").Rows()) + require.Len(t, result, 1) + s2 := result[0] + require.Equal(t, s1, s2) +} + +// TestDecodeBinaryPlan asserts that the result of EXPLAIN ANALYZE FORMAT = 'verbose' is the same as tidb_decode_binary_plan(). +func TestDecodeBinaryPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // Prepare the slow log + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + tk.MustExec("set tidb_slow_log_threshold=0;") + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + }() + tk.MustExec(`create table tp (a int, b int) partition by range(a) ( + partition p0 values less than (100), + partition p1 values less than (200), + partition p2 values less than (300), + partition p3 values less than maxvalue + )`) + tk.MustExec("insert into tp value(1,1), (10,10), (123,234), (213, 234);") + tk.MustExec("create table t(a int, b int, c int, index ia(a));") + tk.MustExec("insert into t value(1,1,1), (10,10,10), (123,234,345), (-213, -234, -234);") + cases := []string{ + "explain analyze format = 'verbose' select * from t", + "explain analyze format = 'verbose' select * from t where a > 10", + "explain analyze format = 'verbose' select /*+ inl_join(t1) */ * from t t1 join t t2 where t1.a = t2.a", + "explain analyze format = 'verbose' WITH RECURSIVE cte(n) AS (SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT * FROM cte", + "set @@tidb_partition_prune_mode='static'", + "explain analyze format = 'verbose' select * from tp", + "explain analyze format = 'verbose' select * from tp t1 join tp t2 on t1.b > t2.b", + "explain analyze format = 'verbose' select * from tp where a > 400", + "explain analyze format = 'verbose' select * from tp where a < 30", + "explain analyze format = 'verbose' select * from tp where a > 0", + "set @@tidb_partition_prune_mode='dynamic'", + "explain analyze format = 'verbose' select * from tp", + "explain analyze format = 'verbose' select * from tp t1 join tp t2 on t1.b > t2.b", + "explain analyze format = 'verbose' select * from tp where a > 400", + "explain analyze format = 'verbose' select * from tp where a < 30", + "explain analyze format = 'verbose' select * from tp where a > 0", + } + + for _, c := range cases { + if len(c) < 7 || c[:7] != "explain" { + tk.MustExec(c) + continue + } + comment := fmt.Sprintf("sql:%s", c) + + var res1, res2 []string + + explainResult := tk.MustQuery(c).Rows() + for _, row := range explainResult { + for _, val := range row { + str := val.(string) + str = strings.TrimSpace(str) + if len(str) > 0 { + res1 = append(res1, str) + } + } + } + + slowLogResult := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query = "` + c + `;" ` + + "order by time desc limit 1").Rows()) + require.Lenf(t, slowLogResult, 1, comment) + decoded := testdata.ConvertRowsToStrings(tk.MustQuery(`select tidb_decode_binary_plan('` + slowLogResult[0] + `')`).Rows())[0] + decodedRows := strings.Split(decoded, "\n") + // remove the first newline and the title row + decodedRows = decodedRows[2:] + for _, decodedRow := range decodedRows { + vals := strings.Split(decodedRow, "|") + for _, val := range vals { + val = strings.TrimSpace(val) + if len(val) > 0 { + res2 = append(res2, val) + } + } + } + + require.Equalf(t, res1, res2, comment) + } +} + func TestInvalidDecodeBinaryPlan(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 77d3a8adc34cb..9568096deb589 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1510,7 +1510,11 @@ func binaryOpFromFlatOp(explainCtx sessionctx.Context, op *FlatOperator, out *ti } if op.IsPhysicalPlan { p := op.Origin.(PhysicalPlan) - out.Cost = p.Cost() + if p.SCtx().GetSessionVars().EnableNewCostInterface { + out.Cost, _ = p.GetPlanCost(property.RootTaskType, 0) + } else { + out.Cost = p.Cost() + } } if rootStats != nil { basic, groups := rootStats.MergeStats() diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 6b8bf974b6b6c..97247f7766e0d 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -58,6 +58,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), } callback := func(i int) int { diff --git a/planner/core/testdata/binary_plan_suite_out.json b/planner/core/testdata/binary_plan_suite_out.json index bea5a39bb2b73..35738707d9258 100644 --- a/planner/core/testdata/binary_plan_suite_out.json +++ b/planner/core/testdata/binary_plan_suite_out.json @@ -131,6 +131,7 @@ "operator_info": "CARTESIAN inner join" } ], + "cost": 120107882, "est_rows": 100000000, "act_rows": 8, "task_type": 1, @@ -228,7 +229,7 @@ "labels": [ 2 ], - "cost": 108852.66666666666, + "cost": 24000, "est_rows": 6400, "act_rows": 2, "task_type": 1, @@ -250,7 +251,7 @@ "labels": [ 1 ], - "cost": 108852.66666666666, + "cost": 24000, "est_rows": 6400, "act_rows": 4, "task_type": 1, @@ -258,7 +259,7 @@ "operator_info": "not(isnull(test.t2.a))" } ], - "cost": 241729.7333333333, + "cost": 72024.4, "est_rows": 8000, "task_type": 1, "store_type": 1, diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index f6be6534be7fd..f03e67ceedebe 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -220,6 +220,7 @@ type StatementContext struct { encodedPlan string planHint string planHintSet bool + binaryPlan string // To avoid cycle import, we use interface{} for the following two fields. // flatPlan should be a *plannercore.FlatPhysicalPlan if it's not nil flatPlan interface{} @@ -411,6 +412,16 @@ func (sc *StatementContext) SetFlatPlan(flat interface{}) { sc.flatPlan = flat } +// GetBinaryPlan gets the binaryPlan field of stmtctx +func (sc *StatementContext) GetBinaryPlan() string { + return sc.binaryPlan +} + +// SetBinaryPlan sets the binaryPlan field of stmtctx +func (sc *StatementContext) SetBinaryPlan(binaryPlan string) { + sc.binaryPlan = binaryPlan +} + // GetResourceGroupTagger returns the implementation of tikvrpc.ResourceGroupTagger related to self. func (sc *StatementContext) GetResourceGroupTagger() tikvrpc.ResourceGroupTagger { normalized, digest := sc.SQLDigest() diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e18df38da4a97..7abe495bf16e3 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2298,8 +2298,12 @@ const ( SlowLogPlan = "Plan" // SlowLogPlanDigest is used to record the query plan digest. SlowLogPlanDigest = "Plan_digest" + // SlowLogBinaryPlan is used to record the binary plan. + SlowLogBinaryPlan = "Binary_plan" // SlowLogPlanPrefix is the prefix of the plan value. SlowLogPlanPrefix = ast.TiDBDecodePlan + "('" + // SlowLogBinaryPlanPrefix is the prefix of the binary plan value. + SlowLogBinaryPlanPrefix = ast.TiDBDecodeBinaryPlan + "('" // SlowLogPlanSuffix is the suffix of the plan value. SlowLogPlanSuffix = "')" // SlowLogPrevStmtPrefix is the prefix of Prev_stmt in slow log file. @@ -2326,6 +2330,10 @@ const ( SlowLogIsWriteCacheTable = "IsWriteCacheTable" ) +// GenerateBinaryPlan decides whether we should record binary plan in slow log and stmt summary. +// It's controlled by the global variable `tidb_generate_binary_plan`. +var GenerateBinaryPlan atomic2.Bool + // SlowQueryLogItems is a collection of items that should be included in the // slow query log. type SlowQueryLogItems struct { @@ -2351,6 +2359,7 @@ type SlowQueryLogItems struct { PrevStmt string Plan string PlanDigest string + BinaryPlan string RewriteInfo RewritePhaseInfo KVTotal time.Duration PDTotal time.Duration @@ -2534,6 +2543,9 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { if len(logItems.PlanDigest) != 0 { writeSlowLogItem(&buf, SlowLogPlanDigest, logItems.PlanDigest) } + if len(logItems.BinaryPlan) != 0 { + writeSlowLogItem(&buf, SlowLogBinaryPlan, logItems.BinaryPlan) + } if logItems.PrevStmt != "" { writeSlowLogItem(&buf, SlowLogPrevStmt, logItems.PrevStmt) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 366b449abf322..5cd4a8430b6da 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1697,6 +1697,10 @@ var defaultSysVars = []*SysVar{ s.EnableAnalyzeSnapshot = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal, Name: TiDBGenerateBinaryPlan, Value: BoolToOnOff(DefTiDBGenerateBinaryPlan), Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { + GenerateBinaryPlan.Store(TiDBOptOn(val)) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index cd7bdaee7743a..be5d0735bb1b6 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -772,6 +772,8 @@ const ( TiDBAuthSigningCert = "tidb_auth_signing_cert" // TiDBAuthSigningKey indicates the path of the signing key to do token-based authentication. TiDBAuthSigningKey = "tidb_auth_signing_key" + // TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary. + TiDBGenerateBinaryPlan = "tidb_generate_binary_plan" ) // TiDB intentional limits @@ -981,6 +983,7 @@ const ( DefTiFlashFineGrainedShuffleBatchSize = 8192 DefAdaptiveClosestReadThreshold = 4096 DefTiDBEnableAnalyzeSnapshot = false + DefTiDBGenerateBinaryPlan = true ) // Process global variables. diff --git a/util/plancodec/codec.go b/util/plancodec/codec.go index 8f84c4d40db69..179ef945d23ff 100644 --- a/util/plancodec/codec.go +++ b/util/plancodec/codec.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/texttree" + "github.com/pingcap/tipb/go-tipb" ) const ( @@ -46,6 +47,15 @@ var ( // PlanDiscardedEncoded indicates the discard plan because it is too long PlanDiscardedEncoded = "[discard]" planDiscardedDecoded = "(plan discarded because too long)" + // BinaryPlanDiscardedEncoded is a special binary plan that represents it's discarded because of too long. + BinaryPlanDiscardedEncoded = func() string { + binary := &tipb.ExplainData{DiscardedDueToTooLong: true} + proto, err := binary.Marshal() + if err != nil { + return "" + } + return Compress(proto) + }() ) var decoderPool = sync.Pool{ diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index d9c3e338fdd6f..eca48ae4e82eb 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -309,6 +309,7 @@ const ( PrevSampleTextStr = "PREV_SAMPLE_TEXT" PlanDigestStr = "PLAN_DIGEST" PlanStr = "PLAN" + BinaryPlan = "BINARY_PLAN" ) type columnValueFactory func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} @@ -616,4 +617,7 @@ var columnValueFactoryMap = map[string]columnValueFactory{ } return plan }, + BinaryPlan: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + return ssElement.sampleBinaryPlan + }, } diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index 329b0d16ce2de..3d90c107b7cbf 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -112,16 +112,17 @@ type stmtSummaryByDigestElement struct { beginTime int64 endTime int64 // basic - sampleSQL string - charset string - collation string - prevSQL string - samplePlan string - planHint string - indexNames []string - execCount int64 - sumErrors int - sumWarnings int + sampleSQL string + charset string + collation string + prevSQL string + samplePlan string + sampleBinaryPlan string + planHint string + indexNames []string + execCount int64 + sumErrors int + sumWarnings int // latency sumLatency time.Duration maxLatency time.Duration @@ -211,33 +212,34 @@ type stmtSummaryByDigestElement struct { // StmtExecInfo records execution information of each statement. type StmtExecInfo struct { - SchemaName string - OriginalSQL string - Charset string - Collation string - NormalizedSQL string - Digest string - PrevSQL string - PrevSQLDigest string - PlanGenerator func() (string, string) - PlanDigest string - PlanDigestGen func() string - User string - TotalLatency time.Duration - ParseLatency time.Duration - CompileLatency time.Duration - StmtCtx *stmtctx.StatementContext - CopTasks *stmtctx.CopTasksDetails - ExecDetail *execdetails.ExecDetails - MemMax int64 - DiskMax int64 - StartTime time.Time - IsInternal bool - Succeed bool - PlanInCache bool - PlanInBinding bool - ExecRetryCount uint - ExecRetryTime time.Duration + SchemaName string + OriginalSQL string + Charset string + Collation string + NormalizedSQL string + Digest string + PrevSQL string + PrevSQLDigest string + PlanGenerator func() (string, string) + BinaryPlanGenerator func() string + PlanDigest string + PlanDigestGen func() string + User string + TotalLatency time.Duration + ParseLatency time.Duration + CompileLatency time.Duration + StmtCtx *stmtctx.StatementContext + CopTasks *stmtctx.CopTasksDetails + ExecDetail *execdetails.ExecDetails + MemMax int64 + DiskMax int64 + StartTime time.Time + IsInternal bool + Succeed bool + PlanInCache bool + PlanInBinding bool + ExecRetryCount uint + ExecRetryTime time.Duration execdetails.StmtExecDetails ResultRows int64 TiKVExecDetails util.ExecDetails @@ -592,15 +594,23 @@ func (ssbd *stmtSummaryByDigest) collectHistorySummaries(checker *stmtSummaryChe return ssElements } -var maxEncodedPlanSizeInBytes = 1024 * 1024 +// MaxEncodedPlanSizeInBytes is the upper limit of the size of the plan and the binary plan in the stmt summary. +var MaxEncodedPlanSizeInBytes = 1024 * 1024 func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalSeconds int64) *stmtSummaryByDigestElement { // sampleSQL / authUsers(sampleUser) / samplePlan / prevSQL / indexNames store the values shown at the first time, // because it compacts performance to update every time. samplePlan, planHint := sei.PlanGenerator() - if len(samplePlan) > maxEncodedPlanSizeInBytes { + if len(samplePlan) > MaxEncodedPlanSizeInBytes { samplePlan = plancodec.PlanDiscardedEncoded } + binPlan := "" + if sei.BinaryPlanGenerator != nil { + binPlan = sei.BinaryPlanGenerator() + if len(binPlan) > MaxEncodedPlanSizeInBytes { + binPlan = plancodec.BinaryPlanDiscardedEncoded + } + } ssElement := &stmtSummaryByDigestElement{ beginTime: beginTime, sampleSQL: formatSQL(sei.OriginalSQL), @@ -609,19 +619,20 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS // PrevSQL is already truncated to cfg.Log.QueryLogMaxLen. prevSQL: sei.PrevSQL, // samplePlan needs to be decoded so it can't be truncated. - samplePlan: samplePlan, - planHint: planHint, - indexNames: sei.StmtCtx.IndexNames, - minLatency: sei.TotalLatency, - firstSeen: sei.StartTime, - lastSeen: sei.StartTime, - backoffTypes: make(map[string]int), - authUsers: make(map[string]struct{}), - planInCache: false, - planCacheHits: 0, - planInBinding: false, - prepared: sei.Prepared, - minResultRows: math.MaxInt64, + samplePlan: samplePlan, + sampleBinaryPlan: binPlan, + planHint: planHint, + indexNames: sei.StmtCtx.IndexNames, + minLatency: sei.TotalLatency, + firstSeen: sei.StartTime, + lastSeen: sei.StartTime, + backoffTypes: make(map[string]int), + authUsers: make(map[string]struct{}), + planInCache: false, + planCacheHits: 0, + planInBinding: false, + prepared: sei.Prepared, + minResultRows: math.MaxInt64, } ssElement.add(sei, intervalSeconds) return ssElement diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 264ea838bde75..a883c4aa51c28 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -443,7 +443,7 @@ func TestAddStatement(t *testing.T) { stmtExecInfo7 := stmtExecInfo1 stmtExecInfo7.PlanDigest = "plan_digest7" stmtExecInfo7.PlanGenerator = func() (string, string) { - buf := make([]byte, maxEncodedPlanSizeInBytes+1) + buf := make([]byte, MaxEncodedPlanSizeInBytes+1) for i := range buf { buf[i] = 'a' }