Skip to content

Commit

Permalink
*: record result_rows in statements_summary and slow log (pingcap#28479)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Oct 12, 2021
1 parent 5a4ee31 commit d5bd9f0
Show file tree
Hide file tree
Showing 12 changed files with 135 additions and 10 deletions.
16 changes: 16 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1000,6 +1000,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
PDTotal: time.Duration(atomic.LoadInt64(&tikvExecDetail.WaitPDRespDuration)),
BackoffTotal: time.Duration(atomic.LoadInt64(&tikvExecDetail.BackoffDuration)),
WriteSQLRespTotal: stmtDetail.WriteSQLRespDuration,
ResultRows: GetResultRowsCount(a.Ctx, a.Plan),
ExecRetryCount: a.retryCount,
}
if a.retryCount > 0 {
Expand Down Expand Up @@ -1050,6 +1051,20 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
}
}

// GetResultRowsCount gets the count of the statement result rows.
func GetResultRowsCount(sctx sessionctx.Context, p plannercore.Plan) int64 {
runtimeStatsColl := sctx.GetSessionVars().StmtCtx.RuntimeStatsColl
if runtimeStatsColl == nil {
return 0
}
rootPlanID := p.ID()
if !runtimeStatsColl.ExistsRootStats(rootPlanID) {
return 0
}
rootStats := runtimeStatsColl.GetRootStats(rootPlanID)
return rootStats.GetActRows()
}

// 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(sctx sessionctx.Context, p plannercore.Plan) string {
cfg := config.GetGlobalConfig()
Expand Down Expand Up @@ -1194,6 +1209,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
PlanInBinding: sessVars.FoundInBinding,
ExecRetryCount: a.retryCount,
StmtExecDetails: stmtDetail,
ResultRows: GetResultRowsCount(a.Ctx, a.Plan),
TiKVExecDetails: tikvExecDetail,
Prepared: a.isPreparedStmt,
}
Expand Down
35 changes: 35 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9076,3 +9076,38 @@ func (s *testSuite) TestCTEWithIndexLookupJoinDeadLock(c *C) {
tk.MustExec("with cte as (with cte1 as (select * from t2 use index(idx_ab) where a > 1 and b > 1) select * from cte1) select /*+use_index(t1 idx_ab)*/ * from cte join t1 on t1.a=cte.a;")
}
}

func (s *testSuite) TestGetResultRowsCount(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int)")
for i := 1; i <= 10; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%v)", i))
}
cases := []struct {
sql string
row int64
}{
{"select * from t", 10},
{"select * from t where a < 0", 0},
{"select * from t where a <= 3", 3},
{"insert into t values (11)", 0},
{"replace into t values (12)", 0},
{"update t set a=13 where a=12", 0},
}

for _, ca := range cases {
if strings.HasPrefix(ca.sql, "select") {
tk.MustQuery(ca.sql)
} else {
tk.MustExec(ca.sql)
}
info := tk.Se.ShowProcess()
c.Assert(info, NotNil)
p, ok := info.Plan.(plannercore.Plan)
c.Assert(ok, IsTrue)
cnt := executor.GetResultRowsCount(tk.Se, p)
c.Assert(ca.row, Equals, cnt, Commentf("sql: %v", ca.sql))
}
}
2 changes: 1 addition & 1 deletion executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -728,7 +728,7 @@ func getColumnValueFactoryByName(sctx sessionctx.Context, colName string, column
row[columnIdx] = types.NewStringDatum(value)
return true, nil
}, nil
case variable.SlowLogMemMax, variable.SlowLogDiskMax:
case variable.SlowLogMemMax, variable.SlowLogDiskMax, variable.SlowLogResultRows:
return func(row []types.Datum, value string, tz *time.Location, checker *slowLogChecker) (valid bool, err error) {
v, err := strconv.ParseInt(value, 10, 64)
if err != nil {
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 @@ -158,7 +158,7 @@ select * from t;`
expectRecordString := `2019-04-28 15:24:04.309074,` +
`405888132465033227,root,localhost,0,57,0.12,0.216905,` +
`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,` +
`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,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`update t set i = 1;,select * from t;`
Expand All @@ -181,7 +181,7 @@ select * from t;`
expectRecordString = `2019-04-28 15:24:04.309074,` +
`405888132465033227,root,localhost,0,57,0.12,0.216905,` +
`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,` +
`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,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`update t set i = 1;,select * from t;`
Expand Down
32 changes: 32 additions & 0 deletions infoschema/cluster_tables_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ func TestClusterTables(t *testing.T) {
t.Run("StmtSummaryEvictedCountTable", SubTestStmtSummaryEvictedCountTable(s))
t.Run("StmtSummaryHistoryTable", SubTestStmtSummaryHistoryTable(s))
t.Run("Issue26379", SubTestIssue26379(s))
t.Run("SubTestStmtSummaryResultRows", SubTestStmtSummaryResultRows(s))
}

func SubTestForClusterServerInfo(s *clusterTablesSuite) func(*testing.T) {
Expand Down Expand Up @@ -457,6 +458,37 @@ func SubTestIssue26379(s *clusterTablesSuite) func(*testing.T) {
}
}

func SubTestStmtSummaryResultRows(s *clusterTablesSuite) func(t *testing.T) {
return func(t *testing.T) {
tk := s.newTestKitWithRoot(t)
tk.MustExec("set global tidb_stmt_summary_refresh_interval=999999999")
tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 3000")
tk.MustExec("set global tidb_stmt_summary_history_size=24")
tk.MustExec("set global tidb_stmt_summary_max_sql_length=4096")
tk.MustExec("set global tidb_enable_stmt_summary=0")
tk.MustExec("set global tidb_enable_stmt_summary=1")
if !config.GetGlobalConfig().EnableCollectExecutionInfo {
tk.MustExec("set @@tidb_enable_collect_execution_info=1")
defer tk.MustExec("set @@tidb_enable_collect_execution_info=0")
}

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int)")
for i := 1; i <= 30; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%v)", i))
}

tk.MustQuery("select * from test.t limit 10;")
tk.MustQuery("select * from test.t limit 20;")
tk.MustQuery("select * from test.t limit 30;")
tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10").
Check(testkit.Rows("10 30 20"))
tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.cluster_statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10").
Check(testkit.Rows("10 30 20"))
}
}

func (s *clusterTablesSuite) setUpRPCService(t *testing.T, addr string) (*grpc.Server, string) {
lis, err := net.Listen("tcp", addr)
require.NoError(t, err)
Expand Down
4 changes: 4 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -857,6 +857,7 @@ var slowQueryCols = []columnInfo{
{name: variable.SlowLogPDTotal, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogBackoffTotal, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogWriteSQLRespTotal, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogResultRows, tp: mysql.TypeLonglong, size: 22},
{name: variable.SlowLogBackoffDetail, tp: mysql.TypeVarchar, size: 4096},
{name: variable.SlowLogPrepared, tp: mysql.TypeTiny, size: 1},
{name: variable.SlowLogSucc, tp: mysql.TypeTiny, size: 1},
Expand Down Expand Up @@ -1251,6 +1252,9 @@ var tableStatementsSummaryCols = []columnInfo{
{name: stmtsummary.AvgPdTimeStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of PD used"},
{name: stmtsummary.AvgBackoffTotalTimeStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of Backoff used"},
{name: stmtsummary.AvgWriteSQLRespTimeStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of write sql resp used"},
{name: stmtsummary.MaxResultRowsStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag, comment: "Max count of sql result rows"},
{name: stmtsummary.MinResultRowsStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag, comment: "Min count of sql result rows"},
{name: stmtsummary.AvgResultRowsStr, tp: mysql.TypeLonglong, size: 22, flag: mysql.NotNullFlag, comment: "Average count of sql result rows"},
{name: stmtsummary.PreparedStr, tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether prepared"},
{name: stmtsummary.AvgAffectedRowsStr, tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of rows affected"},
{name: stmtsummary.FirstSeenStr, tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the first time"},
Expand Down
10 changes: 5 additions & 5 deletions infoschema/tables_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -456,6 +456,7 @@ func prepareSlowLogfile(t *testing.T, slowLogFileName string) {
# Mem_max: 70724
# Disk_max: 65536
# Plan_from_cache: true
# Result_rows: 10
# Succ: true
# Plan: abcd
# Plan_digest: 60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4
Expand Down Expand Up @@ -556,14 +557,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(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|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|1|0|0||||INSERT INTO ...;",
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|10||0|1|1|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||||INSERT INTO ...;",
))
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|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|1|0|0||||INSERT INTO ...;",
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|10||0|1|1|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||||INSERT INTO ...;",
))

// Test for long query.
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1976,6 +1976,8 @@ const (
SlowLogExecRetryTime = "Exec_retry_time"
// SlowLogBackoffDetail is the detail of backoff.
SlowLogBackoffDetail = "Backoff_Detail"
// SlowLogResultRows is the row count of the SQL result.
SlowLogResultRows = "Result_rows"
)

// SlowQueryLogItems is a collection of items that should be included in the
Expand Down Expand Up @@ -2010,6 +2012,7 @@ type SlowQueryLogItems struct {
WriteSQLRespTotal time.Duration
ExecRetryCount uint
ExecRetryTime time.Duration
ResultRows int64
}

// SlowLogFormat uses for formatting slow log.
Expand Down Expand Up @@ -2172,6 +2175,7 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string {
writeSlowLogItem(&buf, SlowLogPDTotal, strconv.FormatFloat(logItems.PDTotal.Seconds(), 'f', -1, 64))
writeSlowLogItem(&buf, SlowLogBackoffTotal, strconv.FormatFloat(logItems.BackoffTotal.Seconds(), 'f', -1, 64))
writeSlowLogItem(&buf, SlowLogWriteSQLRespTotal, strconv.FormatFloat(logItems.WriteSQLRespTotal.Seconds(), 'f', -1, 64))
writeSlowLogItem(&buf, SlowLogResultRows, strconv.FormatInt(logItems.ResultRows, 10))
writeSlowLogItem(&buf, SlowLogSucc, strconv.FormatBool(logItems.Succ))
if len(logItems.Plan) != 0 {
writeSlowLogItem(&buf, SlowLogPlan, logItems.Plan)
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 @@ -233,6 +233,7 @@ func TestSlowLogFormat(t *testing.T) {
# PD_total: 11
# Backoff_total: 12
# Write_sql_response_total: 1
# Result_rows: 12345
# Succ: true`
sql := "select * from t;"
_, digest := parser.NormalizeDigest(sql)
Expand All @@ -259,6 +260,7 @@ func TestSlowLogFormat(t *testing.T) {
PDTotal: 11 * time.Second,
BackoffTotal: 12 * time.Second,
WriteSQLRespTotal: 1 * time.Second,
ResultRows: 12345,
Succ: true,
RewriteInfo: variable.RewritePhaseInfo{
DurationRewrite: 3,
Expand Down
12 changes: 12 additions & 0 deletions util/stmtsummary/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,9 @@ const (
AvgPdTimeStr = "AVG_PD_TIME"
AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME"
AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME"
MaxResultRowsStr = "MAX_RESULT_ROWS"
MinResultRowsStr = "MIN_RESULT_ROWS"
AvgResultRowsStr = "AVG_RESULT_ROWS"
PreparedStr = "PREPARED"
AvgAffectedRowsStr = "AVG_AFFECTED_ROWS"
FirstSeenStr = "FIRST_SEEN"
Expand Down Expand Up @@ -551,6 +554,15 @@ var columnValueFactoryMap = map[string]columnValueFactory{
AvgWriteSQLRespTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} {
return avgInt(int64(ssElement.sumWriteSQLRespTotal), ssElement.commitCount)
},
MaxResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} {
return ssElement.maxResultRows
},
MinResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} {
return ssElement.minResultRows
},
AvgResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} {
return avgInt(ssElement.sumResultRows, ssElement.execCount)
},
PreparedStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} {
return ssElement.prepared
},
Expand Down
17 changes: 17 additions & 0 deletions util/stmtsummary/statement_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"bytes"
"container/list"
"fmt"
"math"
"sort"
"strconv"
"strings"
Expand Down Expand Up @@ -185,6 +186,9 @@ type stmtSummaryByDigestElement struct {
sumPDTotal time.Duration
sumBackoffTotal time.Duration
sumWriteSQLRespTotal time.Duration
sumResultRows int64
maxResultRows int64
minResultRows int64
prepared bool
// The first time this type of SQL executes.
firstSeen time.Time
Expand Down Expand Up @@ -229,6 +233,7 @@ type StmtExecInfo struct {
ExecRetryCount uint
ExecRetryTime time.Duration
execdetails.StmtExecDetails
ResultRows int64
TiKVExecDetails util.ExecDetails
Prepared bool
}
Expand Down Expand Up @@ -598,6 +603,7 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS
planCacheHits: 0,
planInBinding: false,
prepared: sei.Prepared,
minResultRows: math.MaxInt64,
}
ssElement.add(sei, intervalSeconds)
return ssElement
Expand Down Expand Up @@ -802,6 +808,17 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco
ssElement.execRetryCount += sei.ExecRetryCount
ssElement.execRetryTime += sei.ExecRetryTime
}
if sei.ResultRows > 0 {
ssElement.sumResultRows += sei.ResultRows
if ssElement.maxResultRows < sei.ResultRows {
ssElement.maxResultRows = sei.ResultRows
}
if ssElement.minResultRows > sei.ResultRows {
ssElement.minResultRows = sei.ResultRows
}
} else {
ssElement.minResultRows = 0
}
ssElement.sumKVTotal += time.Duration(atomic.LoadInt64(&sei.TiKVExecDetails.WaitKVRespDuration))
ssElement.sumPDTotal += time.Duration(atomic.LoadInt64(&sei.TiKVExecDetails.WaitPDRespDuration))
ssElement.sumBackoffTotal += time.Duration(atomic.LoadInt64(&sei.TiKVExecDetails.BackoffDuration))
Expand Down
Loading

0 comments on commit d5bd9f0

Please sign in to comment.