Skip to content

Commit

Permalink
*: Add binlog prewrite time to record binlog prewrite time consume (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
cfzjywxk authored and XiaTianliang committed Dec 21, 2019
1 parent 89f8a4d commit 11fbeab
Show file tree
Hide file tree
Showing 5 changed files with 23 additions and 10 deletions.
5 changes: 5 additions & 0 deletions infoschema/slow_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ var slowQueryCols = []columnInfo{
{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.BinlogPrewriteTimeStr, 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},
Expand Down Expand Up @@ -202,6 +203,7 @@ type slowQueryTuple struct {
parseTime float64
compileTime float64
preWriteTime float64
binlogPrewriteTime float64
commitTime float64
getCommitTSTime float64
commitBackoffTime float64
Expand Down Expand Up @@ -270,6 +272,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string,
st.compileTime, err = strconv.ParseFloat(value, 64)
case execdetails.PreWriteTimeStr:
st.preWriteTime, err = strconv.ParseFloat(value, 64)
case execdetails.BinlogPrewriteTimeStr:
st.binlogPrewriteTime, err = strconv.ParseFloat(value, 64)
case execdetails.CommitTimeStr:
st.commitTime, err = strconv.ParseFloat(value, 64)
case execdetails.GetCommitTSTimeStr:
Expand Down Expand Up @@ -360,6 +364,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum {
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.binlogPrewriteTime))
record = append(record, types.NewFloat64Datum(st.commitTime))
record = append(record, types.NewFloat64Datum(st.getCommitTSTime))
record = append(record, types.NewFloat64Datum(st.commitBackoffTime))
Expand Down
2 changes: 1 addition & 1 deletion infoschema/slow_log_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ select * from t;`)
}
recordString += str
}
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,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,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,0.021,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,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,update t set i = 1;,select * from t;"
c.Assert(expectRecordString, Equals, recordString)

// fix sql contain '# ' bug
Expand Down
6 changes: 3 additions & 3 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -665,7 +665,7 @@ func prepareSlowLogfile(c *C, slowLogFileName string) {
# 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 Backoff_types: [txnLock] Resolve_lock_time: 0.03 Write_keys: 15 Write_size: 480 Prewrite_region: 1 Txn_retry: 8
# Request_count: 1 Prewrite_time: 0.19 Binlog_prewrite_time: 0.21 Commit_time: 0.01 Commit_backoff_time: 0.18 Backoff_types: [txnLock] 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 Down Expand Up @@ -738,10 +738,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|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.01|0|0.18|[txnLock]|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|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|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.21|0.01|0|0.18|[txnLock]|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|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|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|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.01|0|0.18|[txnLock]|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|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|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|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.21|0.01|0|0.18|[txnLock]|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|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|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)
Expand Down
2 changes: 2 additions & 0 deletions store/tikv/2pc.go
Original file line number Diff line number Diff line change
Expand Up @@ -1046,6 +1046,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
}
}()

binlogPrewriteStart := time.Now()
binlogChan := c.prewriteBinlog(ctx)
prewriteBo := NewBackoffer(ctx, PrewriteMaxBackoff).WithVars(c.txn.vars)
start := time.Now()
Expand All @@ -1068,6 +1069,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) {
}
}
}
commitDetail.BinlogPrewriteTime = time.Since(binlogPrewriteStart)
if err != nil {
logutil.Logger(ctx).Debug("2PC failed on prewrite",
zap.Error(err),
Expand Down
18 changes: 12 additions & 6 deletions util/execdetails/execdetails.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,13 @@ type ExecDetails struct {

// CommitDetails contains commit detail information.
type CommitDetails struct {
GetCommitTsTime time.Duration
PrewriteTime time.Duration
CommitTime time.Duration
LocalLatchTime time.Duration
CommitBackoffTime int64
Mu struct {
GetCommitTsTime time.Duration
PrewriteTime time.Duration
BinlogPrewriteTime time.Duration
CommitTime time.Duration
LocalLatchTime time.Duration
CommitBackoffTime int64
Mu struct {
sync.Mutex
BackoffTypes []fmt.Stringer
}
Expand All @@ -78,6 +79,8 @@ const (
ProcessKeysStr = "Process_keys"
// PreWriteTimeStr means the time of pre-write.
PreWriteTimeStr = "Prewrite_time"
// BinlogPrewriteTimeStr means the time of binlog prewrite
BinlogPrewriteTimeStr = "Binlog_prewrite_time"
// CommitTimeStr means the time of commit.
CommitTimeStr = "Commit_time"
// GetCommitTSTimeStr means the time of getting commit ts.
Expand Down Expand Up @@ -126,6 +129,9 @@ func (d ExecDetails) String() string {
if commitDetails.PrewriteTime > 0 {
parts = append(parts, PreWriteTimeStr+": "+strconv.FormatFloat(commitDetails.PrewriteTime.Seconds(), 'f', -1, 64))
}
if commitDetails.BinlogPrewriteTime > 0 {
parts = append(parts, BinlogPrewriteTimeStr+": "+strconv.FormatFloat(commitDetails.BinlogPrewriteTime.Seconds(), 'f', -1, 64))
}
if commitDetails.CommitTime > 0 {
parts = append(parts, CommitTimeStr+": "+strconv.FormatFloat(commitDetails.CommitTime.Seconds(), 'f', -1, 64))
}
Expand Down

0 comments on commit 11fbeab

Please sign in to comment.