diff --git a/executor/adapter.go b/executor/adapter.go index c86a613af96f5..dc1c2f7109ccf 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -179,6 +179,7 @@ type ExecStmt struct { isPreparedStmt bool isSelectForUpdate bool retryCount uint + retryStartTime time.Time // OutputNames will be set if using cached plan OutputNames []*types.FieldName @@ -641,6 +642,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, err error) (E return nil, errors.New("pessimistic lock retry limit reached") } a.retryCount++ + a.retryStartTime = time.Now() err = UpdateForUpdateTS(a.Ctx, newForUpdateTS) if err != nil { return nil, err @@ -895,6 +897,9 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { RewriteInfo: sessVars.RewritePhaseInfo, ExecRetryCount: a.retryCount, } + if a.retryCount > 0 { + slowItems.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime) + } if _, ok := a.StmtNode.(*ast.CommitStmt); ok { slowItems.PrevStmt = sessVars.PrevStmt.String() } @@ -973,7 +978,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { } stmtCtx := sessVars.StmtCtx normalizedSQL, digest := stmtCtx.SQLDigest() - costTime := time.Since(sessVars.StartTime) + costTime := time.Since(sessVars.StartTime) + sessVars.DurationParse var prevSQL, prevSQLDigest string if _, ok := a.StmtNode.(*ast.CommitStmt); ok { @@ -1009,7 +1014,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { memMax := stmtCtx.MemTracker.MaxConsumed() diskMax := stmtCtx.DiskTracker.MaxConsumed() sql := a.GetTextToLog() - stmtsummary.StmtSummaryByDigestMap.AddStatement(&stmtsummary.StmtExecInfo{ + stmtExecInfo := &stmtsummary.StmtExecInfo{ SchemaName: strings.ToLower(sessVars.CurrentDB), OriginalSQL: sql, NormalizedSQL: normalizedSQL, @@ -1032,7 +1037,12 @@ func (a *ExecStmt) SummaryStmt(succ bool) { IsInternal: sessVars.InRestrictedSQL, Succeed: succ, PlanInCache: sessVars.FoundInPlanCache, - }) + ExecRetryCount: a.retryCount, + } + if a.retryCount > 0 { + stmtExecInfo.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime) + } + stmtsummary.StmtSummaryByDigestMap.AddStatement(stmtExecInfo) } // GetTextToLog return the query text to log. diff --git a/executor/slow_query.go b/executor/slow_query.go index 21ae7b4cd3f46..d70bbd77c225e 100644 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -394,6 +394,7 @@ type slowQueryTuple struct { host string connID uint64 execRetryCount uint64 + execRetryTime float64 queryTime float64 parseTime float64 compileTime float64 @@ -492,6 +493,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string, st.connID, err = strconv.ParseUint(value, 10, 64) case variable.SlowLogExecRetryCount: st.execRetryCount, err = strconv.ParseUint(value, 10, 64) + case variable.SlowLogExecRetryTime: + st.execRetryTime, err = strconv.ParseFloat(value, 64) case variable.SlowLogQueryTimeStr: st.queryTime, err = strconv.ParseFloat(value, 64) case variable.SlowLogParseTimeStr: @@ -600,6 +603,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record = append(record, types.NewStringDatum(st.host)) record = append(record, types.NewUintDatum(st.connID)) record = append(record, types.NewUintDatum(st.execRetryCount)) + record = append(record, types.NewFloat64Datum(st.execRetryTime)) record = append(record, types.NewFloat64Datum(st.queryTime)) record = append(record, types.NewFloat64Datum(st.parseTime)) record = append(record, types.NewFloat64Datum(st.compileTime)) diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index 6aacb9a62c613..d8791562a7cb0 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -92,7 +92,7 @@ func (s *testExecSuite) TestParseSlowLogFile(c *C) { `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 # User@Host: root[root] @ localhost [127.0.0.1] -# Exec_retry_count: 57 +# Exec_retry_time: 0.12 Exec_retry_count: 57 # Query_time: 0.216905 # Cop_time: 0.38 Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true @@ -125,7 +125,7 @@ select * from t;` } recordString += str } - expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,root,localhost,0,57,0.216905,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,update t set i = 1;,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.38,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,update t set i = 1;,select * from t;" c.Assert(expectRecordString, Equals, recordString) // fix sql contain '# ' bug diff --git a/infoschema/tables.go b/infoschema/tables.go index ca232cfedc6fb..98e1f7a7b336e 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -719,6 +719,7 @@ var slowQueryCols = []columnInfo{ {name: variable.SlowLogHostStr, tp: mysql.TypeVarchar, size: 64}, {name: variable.SlowLogConnIDStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.UnsignedFlag}, {name: variable.SlowLogExecRetryCount, tp: mysql.TypeLonglong, size: 20, flag: mysql.UnsignedFlag}, + {name: variable.SlowLogExecRetryTime, tp: mysql.TypeDouble, size: 22}, {name: variable.SlowLogQueryTimeStr, tp: mysql.TypeDouble, size: 22}, {name: variable.SlowLogParseTimeStr, tp: mysql.TypeDouble, size: 22}, {name: variable.SlowLogCompileTimeStr, tp: mysql.TypeDouble, size: 22}, @@ -1205,6 +1206,8 @@ var tableStatementsSummaryCols = []columnInfo{ {name: "MAX_PREWRITE_REGIONS", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of involved regions in prewrite phase"}, {name: "AVG_TXN_RETRY", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of transaction retries"}, {name: "MAX_TXN_RETRY", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of transaction retries"}, + {name: "SUM_EXEC_RETRY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum number of execution retries in pessimistic transactions"}, + {name: "SUM_EXEC_RETRY_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum time of execution retries in pessimistic transactions"}, {name: "SUM_BACKOFF_TIMES", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum of retries"}, {name: "BACKOFF_TYPES", tp: mysql.TypeVarchar, size: 1024, comment: "Types of errors and the number of retries for each type"}, {name: "AVG_MEM", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average memory(byte) used"}, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 9675d3fe28ed1..409b8ef6863f6 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -547,7 +547,7 @@ func prepareSlowLogfile(c *C, slowLogFileName string) { # Txn_start_ts: 406315658548871171 # User@Host: root[root] @ localhost [127.0.0.1] # Conn_ID: 6 -# Exec_retry_count: 57 +# Exec_retry_time: 0.12 Exec_retry_count: 57 # Query_time: 4.895492 # Parse_time: 0.4 # Compile_time: 0.2 @@ -636,10 +636,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|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|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.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|1|1|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|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|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.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) // Test for long query. f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index bf277cb461697..e7b923d9253bc 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1612,6 +1612,8 @@ const ( SlowLogWriteSQLRespTotal = "Write_sql_response_total" // SlowLogExecRetryCount is the execution retry count. SlowLogExecRetryCount = "Exec_retry_count" + // SlowLogExecRetryTime is the execution retry time. + SlowLogExecRetryTime = "Exec_retry_time" ) // SlowQueryLogItems is a collection of items that should be included in the @@ -1642,6 +1644,7 @@ type SlowQueryLogItems struct { WriteSQLRespTotal time.Duration RewriteInfo RewritePhaseInfo ExecRetryCount uint + ExecRetryTime time.Duration } // SlowLogFormat uses for formatting slow log. @@ -1680,7 +1683,15 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { writeSlowLogItem(&buf, SlowLogConnIDStr, strconv.FormatUint(s.ConnectionID, 10)) } if logItems.ExecRetryCount > 0 { - writeSlowLogItem(&buf, SlowLogExecRetryCount, strconv.Itoa(int(logItems.ExecRetryCount))) + buf.WriteString(SlowLogRowPrefixStr) + buf.WriteString(SlowLogExecRetryTime) + buf.WriteString(SlowLogSpaceMarkStr) + buf.WriteString(strconv.FormatFloat(logItems.ExecRetryTime.Seconds(), 'f', -1, 64)) + buf.WriteString(" ") + buf.WriteString(SlowLogExecRetryCount) + buf.WriteString(SlowLogSpaceMarkStr) + buf.WriteString(strconv.Itoa(int(logItems.ExecRetryCount))) + buf.WriteString("\n") } writeSlowLogItem(&buf, SlowLogQueryTimeStr, strconv.FormatFloat(logItems.TimeTotal.Seconds(), 'f', -1, 64)) writeSlowLogItem(&buf, SlowLogParseTimeStr, strconv.FormatFloat(logItems.TimeParse.Seconds(), 'f', -1, 64)) diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 98ac2742c7039..b58ab81a6b4d9 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -178,7 +178,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { resultFields := `# Txn_start_ts: 406649736972468225 # User@Host: root[root] @ 192.168.0.1 [192.168.0.1] # Conn_ID: 1 -# Exec_retry_count: 3 +# Exec_retry_time: 5.1 Exec_retry_count: 3 # Query_time: 1 # Parse_time: 0.00000001 # Compile_time: 0.00000001 @@ -234,6 +234,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { PreprocessSubQueries: 2, }, ExecRetryCount: 3, + ExecRetryTime: 5*time.Second + time.Millisecond*100, } logString := seVar.SlowLogFormat(logItems) c.Assert(logString, Equals, resultFields+"\n"+sql) diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index f9e3784bfa4e2..35109f9f892a2 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -157,6 +157,8 @@ type stmtSummaryByDigestElement struct { maxPrewriteRegionNum int32 sumTxnRetry int64 maxTxnRetry int + sumExecRetryCount int64 + sumExecRetryTime time.Duration sumBackoffTimes int64 backoffTypes map[fmt.Stringer]int authUsers map[string]struct{} @@ -173,6 +175,9 @@ type stmtSummaryByDigestElement struct { // plan cache planInCache bool planCacheHits int64 + // pessimistic execution retry information. + execRetryCount uint + execRetryTime time.Duration } // StmtExecInfo records execution information of each statement. @@ -199,6 +204,8 @@ type StmtExecInfo struct { IsInternal bool Succeed bool PlanInCache bool + ExecRetryCount uint + ExecRetryTime time.Duration } // newStmtSummaryByDigestMap creates an empty stmtSummaryByDigestMap. @@ -754,6 +761,10 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco if ssElement.lastSeen.Before(sei.StartTime) { ssElement.lastSeen = sei.StartTime } + if sei.ExecRetryCount > 0 { + ssElement.execRetryCount += sei.ExecRetryCount + ssElement.execRetryTime += sei.ExecRetryTime + } } func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) []types.Datum { @@ -829,6 +840,8 @@ func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) int(ssElement.maxPrewriteRegionNum), avgFloat(ssElement.sumTxnRetry, ssElement.commitCount), ssElement.maxTxnRetry, + int(ssElement.execRetryCount), + int64(ssElement.execRetryTime), ssElement.sumBackoffTimes, formatBackoffTypes(ssElement.backoffTypes), avgInt(ssElement.sumMem, ssElement.execCount), diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index fe547212bd120..7517a923b9816 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -612,7 +612,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, stmtExecInfo1.ExecDetail.CommitDetail.WriteSize, stmtExecInfo1.ExecDetail.CommitDetail.WriteSize, stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, - stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, 1, + stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, 0, 0, 1, "txnLock:1", stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, stmtExecInfo1.StmtCtx.AffectedRows(), t, t, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} match(c, datums[0], expectedDatum...)