Skip to content

Commit

Permalink
infoschema/slow_query: add more field in slow_query table (#13007) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored and zz-jason committed Nov 5, 2019
1 parent 4c7bfb8 commit 2fc33da
Show file tree
Hide file tree
Showing 4 changed files with 125 additions and 42 deletions.
118 changes: 89 additions & 29 deletions infoschema/slow_log.go
Expand Up @@ -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},
Expand Down Expand Up @@ -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 {
Expand All @@ -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:
Expand Down Expand Up @@ -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))
Expand Down
2 changes: 1 addition & 1 deletion infoschema/slow_log_test.go
Expand Up @@ -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
Expand Down
7 changes: 5 additions & 2 deletions infoschema/tables_test.go
Expand Up @@ -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
Expand All @@ -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(`
Expand Down
40 changes: 30 additions & 10 deletions util/execdetails/execdetails.go
Expand Up @@ -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.
Expand All @@ -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, " ")
Expand Down

0 comments on commit 2fc33da

Please sign in to comment.