diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index ba0bd1ef8111..c680b94f648a 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -39,6 +39,18 @@ var slowQueryCols = []columnInfo{ {variable.SlowLogUserStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogConnIDStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogQueryTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogParseTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCompileTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.PreWriteTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.CommitTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.GetCommitTSTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.CommitBackoffTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.ResolveLockTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.LocalLatchWaitTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {execdetails.WriteKeysStr, mysql.TypeLonglong, 22, 0, nil, nil}, + {execdetails.WriteSizeStr, mysql.TypeLonglong, 22, 0, nil, nil}, + {execdetails.PrewriteRegionStr, mysql.TypeLonglong, 22, 0, nil, nil}, + {execdetails.TxnRetryStr, mysql.TypeLonglong, 22, 0, nil, nil}, {execdetails.ProcessTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, {execdetails.WaitTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, {execdetails.BackoffTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, @@ -173,35 +185,47 @@ func getOneLine(reader *bufio.Reader) ([]byte, error) { } type slowQueryTuple struct { - time time.Time - txnStartTs uint64 - user string - host string - connID uint64 - queryTime float64 - processTime float64 - waitTime float64 - backOffTime float64 - requestCount uint64 - totalKeys uint64 - processKeys uint64 - db string - indexIDs string - digest string - statsInfo string - avgProcessTime float64 - p90ProcessTime float64 - maxProcessTime float64 - maxProcessAddress string - avgWaitTime float64 - p90WaitTime float64 - maxWaitTime float64 - maxWaitAddress string - memMax int64 - prevStmt string - sql string - isInternal bool - succ bool + time time.Time + txnStartTs uint64 + user string + host string + connID uint64 + queryTime float64 + parseTime float64 + compileTime float64 + preWriteTime float64 + commitTime float64 + getCommitTSTime float64 + commitBackoffTime float64 + resolveLockTime float64 + localLatchWaitTime float64 + writeKeys uint64 + writeSize uint64 + prewriteRegion uint64 + txnRetry uint64 + processTime float64 + waitTime float64 + backOffTime float64 + requestCount uint64 + totalKeys uint64 + processKeys uint64 + db string + indexIDs string + digest string + statsInfo string + avgProcessTime float64 + p90ProcessTime float64 + maxProcessTime float64 + maxProcessAddress string + avgWaitTime float64 + p90WaitTime float64 + maxWaitTime float64 + maxWaitAddress string + memMax int64 + prevStmt string + sql string + isInternal bool + succ bool } func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) error { @@ -223,6 +247,30 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.connID, err = strconv.ParseUint(value, 10, 64) case variable.SlowLogQueryTimeStr: st.queryTime, err = strconv.ParseFloat(value, 64) + case variable.SlowLogParseTimeStr: + st.parseTime, err = strconv.ParseFloat(value, 64) + case variable.SlowLogCompileTimeStr: + st.compileTime, err = strconv.ParseFloat(value, 64) + case execdetails.PreWriteTimeStr: + st.preWriteTime, err = strconv.ParseFloat(value, 64) + case execdetails.CommitTimeStr: + st.commitTime, err = strconv.ParseFloat(value, 64) + case execdetails.GetCommitTSTimeStr: + st.getCommitTSTime, err = strconv.ParseFloat(value, 64) + case execdetails.CommitBackoffTimeStr: + st.commitBackoffTime, err = strconv.ParseFloat(value, 64) + case execdetails.ResolveLockTimeStr: + st.resolveLockTime, err = strconv.ParseFloat(value, 64) + case execdetails.LocalLatchWaitTimeStr: + st.localLatchWaitTime, err = strconv.ParseFloat(value, 64) + case execdetails.WriteKeysStr: + st.writeKeys, err = strconv.ParseUint(value, 10, 64) + case execdetails.WriteSizeStr: + st.writeSize, err = strconv.ParseUint(value, 10, 64) + case execdetails.PrewriteRegionStr: + st.prewriteRegion, err = strconv.ParseUint(value, 10, 64) + case execdetails.TxnRetryStr: + st.txnRetry, err = strconv.ParseUint(value, 10, 64) case execdetails.ProcessTimeStr: st.processTime, err = strconv.ParseFloat(value, 64) case execdetails.WaitTimeStr: @@ -281,6 +329,18 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record = append(record, types.NewStringDatum(st.user)) record = append(record, types.NewUintDatum(st.connID)) record = append(record, types.NewFloat64Datum(st.queryTime)) + record = append(record, types.NewFloat64Datum(st.parseTime)) + record = append(record, types.NewFloat64Datum(st.compileTime)) + record = append(record, types.NewFloat64Datum(st.preWriteTime)) + record = append(record, types.NewFloat64Datum(st.commitTime)) + record = append(record, types.NewFloat64Datum(st.getCommitTSTime)) + record = append(record, types.NewFloat64Datum(st.commitBackoffTime)) + record = append(record, types.NewFloat64Datum(st.resolveLockTime)) + record = append(record, types.NewFloat64Datum(st.localLatchWaitTime)) + record = append(record, types.NewUintDatum(st.writeKeys)) + record = append(record, types.NewUintDatum(st.writeSize)) + record = append(record, types.NewUintDatum(st.prewriteRegion)) + record = append(record, types.NewUintDatum(st.txnRetry)) record = append(record, types.NewFloat64Datum(st.processTime)) record = append(record, types.NewFloat64Datum(st.waitTime)) record = append(record, types.NewFloat64Datum(st.backOffTime)) diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 18d57b3fe748..c996fa62317b 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -55,7 +55,7 @@ select * from t;`) } recordString += str } - expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,0.05,0.6,0.8,70724,0,update t set i = 1;,select * from t;" + expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,0,0.216905,0,0,0,0,0,0,0,0,0,0,0,0,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,0.05,0.6,0.8,70724,0,update t set i = 1;,select * from t;" c.Assert(expectRecordString, Equals, recordString) // fix sql contain '# ' bug diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 07c27f42267c..95658f2ad9ca 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -494,6 +494,9 @@ func (s *testSuite) TestSlowQuery(c *C) { # User: root@127.0.0.1 # Conn_ID: 6 # Query_time: 4.895492 +# Parse_time: 0.4 +# Compile_time: 0.2 +# Request_count: 1 Prewrite_time: 0.19 Commit_time: 0.01 Commit_backoff_time: 0.18 Resolve_lock_time: 0.03 Write_keys: 15 Write_size: 480 Prewrite_region: 1 Txn_retry: 8 # Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 # Wait_time: 0.101 # Backoff_time: 0.092 @@ -514,10 +517,10 @@ select * from t_slim;`)) 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@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|0.05|0.6|0.8|70724|1|update t set i = 2;|select * from t_slim;")) + "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.4|0.2|0.19|0.01|0|0.18|0.03|0|15|480|1|8|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|0.05|0.6|0.8|70724|1|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@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|0.05|0.6|0.8|70724|1|update t set i = 2;|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.4|0.2|0.19|0.01|0|0.18|0.03|0|15|480|1|8|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|0.05|0.6|0.8|70724|1|update t set i = 2;|select * from t_slim;")) // Test for long query. _, err = f.Write([]byte(` diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 5a468d240de1..0124766a67a7 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -63,6 +63,26 @@ const ( TotalKeysStr = "Total_keys" // ProcessKeysStr means the total processed keys. ProcessKeysStr = "Process_keys" + // PreWriteTimeStr means the time of pre-write. + PreWriteTimeStr = "Prewrite_time" + // CommitTimeStr means the time of commit. + CommitTimeStr = "Commit_time" + // GetCommitTSTimeStr means the time of getting commit ts. + GetCommitTSTimeStr = "Get_commit_ts_time" + // CommitBackoffTimeStr means the time of commit backoff. + CommitBackoffTimeStr = "Commit_backoff_time" + // ResolveLockTimeStr means the time of resolving lock. + ResolveLockTimeStr = "Resolve_lock_time" + // LocalLatchWaitTimeStr means the time of waiting in local latch. + LocalLatchWaitTimeStr = "Local_latch_wait_time" + // WriteKeysStr means the count of keys in the transaction. + WriteKeysStr = "Write_keys" + // WriteSizeStr means the key/value size in the transaction. + WriteSizeStr = "Write_size" + // PrewriteRegionStr means the count of region when pre-write. + PrewriteRegionStr = "Prewrite_region" + // TxnRetryStr means the count of transaction retry. + TxnRetryStr = "Txn_retry" ) // String implements the fmt.Stringer interface. @@ -89,36 +109,36 @@ func (d ExecDetails) String() string { commitDetails := d.CommitDetail if commitDetails != nil { if commitDetails.PrewriteTime > 0 { - parts = append(parts, fmt.Sprintf("Prewrite_time: %v", commitDetails.PrewriteTime.Seconds())) + parts = append(parts, PreWriteTimeStr+": "+strconv.FormatFloat(commitDetails.PrewriteTime.Seconds(), 'f', -1, 64)) } if commitDetails.CommitTime > 0 { - parts = append(parts, fmt.Sprintf("Commit_time: %v", commitDetails.CommitTime.Seconds())) + parts = append(parts, CommitTimeStr+": "+strconv.FormatFloat(commitDetails.CommitTime.Seconds(), 'f', -1, 64)) } if commitDetails.GetCommitTsTime > 0 { - parts = append(parts, fmt.Sprintf("Get_commit_ts_time: %v", commitDetails.GetCommitTsTime.Seconds())) + parts = append(parts, GetCommitTSTimeStr+": "+strconv.FormatFloat(commitDetails.GetCommitTsTime.Seconds(), 'f', -1, 64)) } if commitDetails.TotalBackoffTime > 0 { - parts = append(parts, fmt.Sprintf("Total_backoff_time: %v", commitDetails.TotalBackoffTime.Seconds())) + parts = append(parts, CommitBackoffTimeStr+": "+strconv.FormatFloat(commitDetails.TotalBackoffTime.Seconds(), 'f', -1, 64)) } resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime) if resolveLockTime > 0 { - parts = append(parts, fmt.Sprintf("Resolve_lock_time: %v", time.Duration(resolveLockTime).Seconds())) + parts = append(parts, ResolveLockTimeStr+": "+strconv.FormatFloat(time.Duration(resolveLockTime).Seconds(), 'f', -1, 64)) } if commitDetails.LocalLatchTime > 0 { - parts = append(parts, fmt.Sprintf("Local_latch_wait_time: %v", commitDetails.LocalLatchTime.Seconds())) + parts = append(parts, LocalLatchWaitTimeStr+": "+strconv.FormatFloat(commitDetails.LocalLatchTime.Seconds(), 'f', -1, 64)) } if commitDetails.WriteKeys > 0 { - parts = append(parts, fmt.Sprintf("Write_keys: %d", commitDetails.WriteKeys)) + parts = append(parts, WriteKeysStr+": "+strconv.FormatInt(int64(commitDetails.WriteKeys), 10)) } if commitDetails.WriteSize > 0 { - parts = append(parts, fmt.Sprintf("Write_size: %d", commitDetails.WriteSize)) + parts = append(parts, WriteSizeStr+": "+strconv.FormatInt(int64(commitDetails.WriteSize), 10)) } prewriteRegionNum := atomic.LoadInt32(&commitDetails.PrewriteRegionNum) if prewriteRegionNum > 0 { - parts = append(parts, fmt.Sprintf("Prewrite_region: %d", prewriteRegionNum)) + parts = append(parts, PrewriteRegionStr+": "+strconv.FormatInt(int64(prewriteRegionNum), 10)) } if commitDetails.TxnRetry > 0 { - parts = append(parts, fmt.Sprintf("Txn_retry: %d", commitDetails.TxnRetry)) + parts = append(parts, TxnRetryStr+": "+strconv.FormatInt(int64(commitDetails.TxnRetry), 10)) } } return strings.Join(parts, " ")