From be5e5a4b86edf34dfbc5d35ee98a9b2e3a158caa Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 24 Jan 2019 19:47:55 +0800 Subject: [PATCH 01/33] slow log init --- executor/adapter.go | 23 ++++++-------------- sessionctx/variable/session.go | 38 +++++++++++++++++++++++++++++++++ util/execdetails/execdetails.go | 32 +++++++++++++-------------- util/logutil/log.go | 25 ++++++++++++++-------- 4 files changed, 76 insertions(+), 42 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index f37ad05c5a1ab..87db7d4b0eebc 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -379,8 +379,6 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { sessVars := a.Ctx.GetSessionVars() sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() - connID := sessVars.ConnectionID - currentDB := sessVars.CurrentDB var tableIDs, indexIDs string if len(sessVars.StmtCtx.TableIDs) > 0 { tableIDs = strings.Replace(fmt.Sprintf("table_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.TableIDs), " ", ",", -1) @@ -388,26 +386,17 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { if len(sessVars.StmtCtx.IndexIDs) > 0 { indexIDs = strings.Replace(fmt.Sprintf("index_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) } - user := sessVars.User - var internal string - if sessVars.InRestrictedSQL { - internal = "[INTERNAL] " - } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { - logutil.SlowQueryLogger.Debugf( - "[QUERY] %vcost_time:%vs %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", - internal, costTime.Seconds(), execDetail, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) + logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) } else { - logutil.SlowQueryLogger.Warnf( - "[SLOW_QUERY] %vcost_time:%vs %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", - internal, costTime.Seconds(), execDetail, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) + logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) var userString string - if user != nil { - userString = user.String() + if sessVars.User != nil { + userString = sessVars.User.String() } if len(tableIDs) > 10 { tableIDs = tableIDs[10 : len(tableIDs)-1] // Remove "table_ids:" and the last "," @@ -421,10 +410,10 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { Duration: costTime, Detail: sessVars.StmtCtx.GetExecDetails(), Succ: succ, - ConnID: connID, + ConnID: sessVars.ConnectionID, TxnTS: txnTS, User: userString, - DB: currentDB, + DB: sessVars.CurrentDB, TableIDs: tableIDs, IndexIDs: indexIDs, Internal: sessVars.InRestrictedSQL, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 66ba563f0d422..b6a5893a7a8d0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -14,6 +14,7 @@ package variable import ( + "bytes" "crypto/tls" "fmt" "strconv" @@ -36,6 +37,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/timeutil" ) @@ -799,3 +801,39 @@ type BatchSize struct { // MaxChunkSize defines max row count of a Chunk during query execution. MaxChunkSize int } + +const ( + SlowLogTxnStartTSStr = "Txn_start_ts" + SlowLogUserStr = "User" + SlowLogConnIDStr = "Conn_ID" + SlowLogQueryTimeStr = "Query_time" + SlowLogDBStr = "DB" + SlowLogIsInternalStr = "Is_internal" + SlowLogIndexNamesStr = "Index_names" +) + +// SlowLogFormat uses for formatting slow log. +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, index_IDs string, sql string) string { + var buf bytes.Buffer + execDetailStr := execDetail.String() + buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogTxnStartTSStr, txnTS)) + if s.User != nil { + buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogUserStr, s.User)) + } + if s.ConnectionID != 0 { + buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogConnIDStr, s.ConnectionID)) + } + buf.WriteString(fmt.Sprintf("# %s: %f\n", SlowLogQueryTimeStr, costTime.Seconds())) + if len(execDetailStr) > 0 { + buf.WriteString(fmt.Sprintf("# %s\n", execDetailStr)) + } + if len(s.CurrentDB) > 0 { + buf.WriteString(fmt.Sprintf("# %s: %s\n", SlowLogDBStr, s.CurrentDB)) + } + if len(index_IDs) > 0 { + buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogIndexNamesStr, index_IDs)) + } + buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogIsInternalStr, s.InRestrictedSQL)) + buf.WriteString(sql) + return buf.String() +} diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index aaebbda1a3cc8..00e8a2ee5533a 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -53,56 +53,56 @@ type CommitDetails struct { func (d ExecDetails) String() string { parts := make([]string, 0, 6) if d.ProcessTime > 0 { - parts = append(parts, fmt.Sprintf("process_time:%vs", d.ProcessTime.Seconds())) + parts = append(parts, fmt.Sprintf("Process_time: %v", d.ProcessTime.Seconds())) } if d.WaitTime > 0 { - parts = append(parts, fmt.Sprintf("wait_time:%vs", d.WaitTime.Seconds())) + parts = append(parts, fmt.Sprintf("Wait_time: %v", d.WaitTime.Seconds())) } if d.BackoffTime > 0 { - parts = append(parts, fmt.Sprintf("backoff_time:%vs", d.BackoffTime.Seconds())) + parts = append(parts, fmt.Sprintf("Backoff_time: %v", d.BackoffTime.Seconds())) } if d.RequestCount > 0 { - parts = append(parts, fmt.Sprintf("request_count:%d", d.RequestCount)) + parts = append(parts, fmt.Sprintf("Request_count: %d", d.RequestCount)) } if d.TotalKeys > 0 { - parts = append(parts, fmt.Sprintf("total_keys:%d", d.TotalKeys)) + parts = append(parts, fmt.Sprintf("Total_keys: %d", d.TotalKeys)) } if d.ProcessedKeys > 0 { - parts = append(parts, fmt.Sprintf("processed_keys:%d", d.ProcessedKeys)) + parts = append(parts, fmt.Sprintf("Processed_keys: %d", d.ProcessedKeys)) } commitDetails := d.CommitDetail if commitDetails != nil { if commitDetails.PrewriteTime > 0 { - parts = append(parts, fmt.Sprintf("prewrite_time:%vs", commitDetails.PrewriteTime.Seconds())) + parts = append(parts, fmt.Sprintf("Prewrite_time: %v", commitDetails.PrewriteTime.Seconds())) } if commitDetails.CommitTime > 0 { - parts = append(parts, fmt.Sprintf("commit_time:%vs", commitDetails.CommitTime.Seconds())) + parts = append(parts, fmt.Sprintf("Commit_time: %v", commitDetails.CommitTime.Seconds())) } if commitDetails.GetCommitTsTime > 0 { - parts = append(parts, fmt.Sprintf("get_commit_ts_time:%vs", commitDetails.GetCommitTsTime.Seconds())) + parts = append(parts, fmt.Sprintf("Get_commit_ts_time: %v", commitDetails.GetCommitTsTime.Seconds())) } if commitDetails.TotalBackoffTime > 0 { - parts = append(parts, fmt.Sprintf("total_backoff_time:%vs", commitDetails.TotalBackoffTime.Seconds())) + parts = append(parts, fmt.Sprintf("Total_backoff_time: %v", commitDetails.TotalBackoffTime.Seconds())) } resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime) if resolveLockTime > 0 { - parts = append(parts, fmt.Sprintf("resolve_lock_time:%vs", time.Duration(resolveLockTime).Seconds())) + parts = append(parts, fmt.Sprintf("Resolve_lock_time: %v", time.Duration(resolveLockTime).Seconds())) } if commitDetails.LocalLatchTime > 0 { - parts = append(parts, fmt.Sprintf("local_latch_wait_time:%vs", commitDetails.LocalLatchTime.Seconds())) + parts = append(parts, fmt.Sprintf("Local_latch_wait_time: %v", commitDetails.LocalLatchTime.Seconds())) } if commitDetails.WriteKeys > 0 { - parts = append(parts, fmt.Sprintf("write_keys:%d", commitDetails.WriteKeys)) + parts = append(parts, fmt.Sprintf("Write_keys: %d", commitDetails.WriteKeys)) } if commitDetails.WriteSize > 0 { - parts = append(parts, fmt.Sprintf("write_size:%d", commitDetails.WriteSize)) + parts = append(parts, fmt.Sprintf("Write_size: %d", commitDetails.WriteSize)) } prewriteRegionNum := atomic.LoadInt32(&commitDetails.PrewriteRegionNum) if prewriteRegionNum > 0 { - parts = append(parts, fmt.Sprintf("prewrite_region:%d", prewriteRegionNum)) + parts = append(parts, fmt.Sprintf("Prewrite_region: %d", prewriteRegionNum)) } if commitDetails.TxnRetry > 0 { - parts = append(parts, fmt.Sprintf("txn_retry:%d", commitDetails.TxnRetry)) + parts = append(parts, fmt.Sprintf("Txn_retry: %d", commitDetails.TxnRetry)) } } return strings.Join(parts, " ") diff --git a/util/logutil/log.go b/util/logutil/log.go index 24b306e25887f..aea4ece44c57f 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -192,6 +192,21 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { return b.Bytes(), nil } +type slowLogFormatter struct{} + +func (f *slowLogFormatter) Format(entry *log.Entry) ([]byte, error) { + var b *bytes.Buffer + if entry.Buffer != nil { + b = entry.Buffer + } else { + b = &bytes.Buffer{} + } + + fmt.Fprintf(b, "# Time: %s\n", entry.Time.Format("2006-01-02-15:04:05.999999999 -0700")) + fmt.Fprintf(b, "%s\n", entry.Message) + return b.Bytes(), nil +} + func stringToLogFormatter(format string, disableTimestamp bool) log.Formatter { switch strings.ToLower(format) { case "text": @@ -274,15 +289,7 @@ func InitLogger(cfg *LogConfig) error { if err := initFileLog(&tmp, SlowQueryLogger); err != nil { return errors.Trace(err) } - hooks := make(log.LevelHooks) - hooks.Add(&contextHook{}) - SlowQueryLogger.Hooks = hooks - slowQueryFormatter := stringToLogFormatter(cfg.Format, cfg.DisableTimestamp) - ft, ok := slowQueryFormatter.(*textFormatter) - if ok { - ft.EnableEntryOrder = true - } - SlowQueryLogger.Formatter = slowQueryFormatter + SlowQueryLogger.Formatter = &slowLogFormatter{} } return nil From 31314c46eeb22e2b1b1619b3a7ef579a570b7253 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 24 Jan 2019 21:10:20 +0800 Subject: [PATCH 02/33] use table name, index name instead of id in slow log --- executor/adapter.go | 24 +++++++++--------------- executor/builder.go | 8 ++++---- sessionctx/stmtctx/stmtctx.go | 8 ++++---- 3 files changed, 17 insertions(+), 23 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 87db7d4b0eebc..63b4742859941 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -379,18 +379,18 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { sessVars := a.Ctx.GetSessionVars() sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() - var tableIDs, indexIDs string - if len(sessVars.StmtCtx.TableIDs) > 0 { - tableIDs = strings.Replace(fmt.Sprintf("table_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.TableIDs), " ", ",", -1) + var tableNames, indexNames string + if len(sessVars.StmtCtx.TableNames) > 0 { + tableNames = strings.Join(sessVars.StmtCtx.TableNames, ",") } - if len(sessVars.StmtCtx.IndexIDs) > 0 { - indexIDs = strings.Replace(fmt.Sprintf("index_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) + if len(sessVars.StmtCtx.IndexNames) > 0 { + indexNames = strings.Join(sessVars.StmtCtx.IndexNames, ",") } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { - logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) + logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, sql)) } else { - logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) + logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -398,12 +398,6 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { if sessVars.User != nil { userString = sessVars.User.String() } - if len(tableIDs) > 10 { - tableIDs = tableIDs[10 : len(tableIDs)-1] // Remove "table_ids:" and the last "," - } - if len(indexIDs) > 10 { - indexIDs = indexIDs[10 : len(indexIDs)-1] // Remove "index_ids:" and the last "," - } domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{ SQL: sql, Start: a.StartTime, @@ -414,8 +408,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { TxnTS: txnTS, User: userString, DB: sessVars.CurrentDB, - TableIDs: tableIDs, - IndexIDs: indexIDs, + TableIDs: tableNames, + IndexIDs: indexNames, Internal: sessVars.InRestrictedSQL, }) } diff --git a/executor/builder.go b/executor/builder.go index e84c26bae68da..a6f74597750ef 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1616,7 +1616,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx - sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) + sctx.TableNames = append(sctx.TableNames, ts.Table.Name.O) return ret } @@ -1674,7 +1674,7 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) * is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) ret.ranges = is.Ranges sctx := b.ctx.GetSessionVars().StmtCtx - sctx.IndexIDs = append(sctx.IndexIDs, is.Index.ID) + sctx.IndexNames = append(sctx.IndexNames, is.Index.Name.O) return ret } @@ -1754,8 +1754,8 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo ret.ranges = is.Ranges metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor").Inc() sctx := b.ctx.GetSessionVars().StmtCtx - sctx.IndexIDs = append(sctx.IndexIDs, is.Index.ID) - sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) + sctx.IndexNames = append(sctx.IndexNames, is.Index.Name.O) + sctx.TableNames = append(sctx.TableNames, ts.Table.Name.O) return ret } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index ab308f7a8d1dd..35a9a81e92000 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -110,8 +110,8 @@ type StatementContext struct { NotFillCache bool MemTracker *memory.Tracker RuntimeStatsColl *execdetails.RuntimeStatsColl - TableIDs []int64 - IndexIDs []int64 + TableNames []string + IndexNames []string NowTs time.Time SysTs time.Time StmtType string @@ -354,8 +354,8 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.message = "" sc.mu.warnings = nil sc.mu.Unlock() - sc.TableIDs = sc.TableIDs[:0] - sc.IndexIDs = sc.IndexIDs[:0] + sc.TableNames = sc.TableNames[:0] + sc.IndexNames = sc.IndexNames[:0] } // MergeExecDetails merges a single region execution details into self, used to print From e94b4a8f211bf44cf796bcc073c7b4d8f6c64b45 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 25 Jan 2019 00:02:53 +0800 Subject: [PATCH 03/33] fix bug, sql should end with ';', otherwise, pt-query-digest parse will wrong --- sessionctx/variable/session.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b6a5893a7a8d0..f28b11e9f0666 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -834,6 +834,11 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogIndexNamesStr, index_IDs)) } buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogIsInternalStr, s.InRestrictedSQL)) - buf.WriteString(sql) + if len(sql) > 0 { + buf.WriteString(sql) + if sql[len(sql)-1] != ';' { + buf.WriteString(";") + } + } return buf.String() } From 5d6086576ec45ffce8a0ce7e0c05a48b7f737bf3 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 28 Jan 2019 13:55:52 +0800 Subject: [PATCH 04/33] add slow log table, init --- infoschema/slow_log.go | 134 ++++++++++++++++++++++++++++++++ infoschema/slow_log_test.go | 28 +++++++ sessionctx/variable/session.go | 5 ++ util/execdetails/execdetails.go | 9 +++ util/logutil/log.go | 6 +- 5 files changed, 181 insertions(+), 1 deletion(-) create mode 100644 infoschema/slow_log.go create mode 100644 infoschema/slow_log_test.go diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go new file mode 100644 index 0000000000000..93c5049180208 --- /dev/null +++ b/infoschema/slow_log.go @@ -0,0 +1,134 @@ +package infoschema + +import ( + "bufio" + "fmt" + "github.com/pingcap/errors" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil" + log "github.com/sirupsen/logrus" + "os" + "strconv" + "strings" + "time" +) + +var slowLogCols = []columnInfo{ + {variable.SlowLogTimeStr, mysql.TypeDatetime, -1, 0, nil, nil}, + {variable.SlowLogTxnStartTSStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, + {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}, + {execdetails.ProcessTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, +} + +func parseSlowLogFile(filePath string) (map[string]types.Datum, error) { + file, err := os.Open(filePath) + if err != nil { + return nil, errors.Trace(err) + } + defer file.Close() + + rowMap := make(map[string]types.Datum, len(slowLogCols)) + startFlag := false + startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + scanner := bufio.NewScanner(file) + for scanner.Scan() { + line := scanner.Text() + fmt.Println(line) + // check start + if !startFlag && strings.Contains(line, startPrefix) { + t, err := parseTime(line[len(startPrefix):]) + if err != nil { + log.Errorf("parse slow log error: %v", err) + // temporary ignore now. + continue + } + rowMap[variable.SlowLogTimeStr] = types.NewTimeDatum(types.Time{ + Time: types.FromGoTime(t), + Type: mysql.TypeDatetime, + Fsp: types.MaxFsp, + }) + startFlag = true + continue + } + + if startFlag { + // parse field. + if strings.Contains(line, variable.SlowLogPrefixStr) { + line = line[len(variable.SlowLogPrefixStr):] + fieldValues := strings.Split(line, " ") + for i := 0; i < len(fieldValues)-1; i += 2 { + field := fieldValues[i] + if strings.HasSuffix(field, ":") { + field = field[:len(field)-1] + } + col := findColumnByName(slowLogCols, field) + if col == nil { + continue + } + var value types.Datum + switch col.tp { + case mysql.TypeLonglong: + num, err := strconv.ParseUint(fieldValues[i+1], 10, 64) + if err != nil { + log.Errorf("parse slow log error: %v", err) + break + } + value = types.NewUintDatum(num) + case mysql.TypeVarchar: + value = types.NewStringDatum(fieldValues[i+1]) + case mysql.TypeDouble: + num, err := strconv.ParseFloat(fieldValues[i+1], 64) + if err != nil { + log.Errorf("parse slow log error: %v", err) + break + } + value = types.NewDatum(num) + } + rowMap[field] = value + + } + } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { + // get the sql string, and mark the start flag to false. + rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(copyStringHack(line)) + } + } + } + + if err := scanner.Err(); err != nil { + return nil, errors.Trace(err) + } + return rowMap, nil +} + +func parseTime(s string) (time.Time, error) { + t, err := time.Parse(logutil.SlowLogTimeFormat, s) + + if err != nil { + err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\"", s, logutil.SlowLogTimeFormat) + } + return t, err +} + +func findColumnByName(cols []columnInfo, colName string) *columnInfo { + for _, col := range cols { + if col.name == colName { + return &col + } + } + return nil +} + +func copyString(s string) string { + return string([]byte(s)) +} + +func copyStringHack(s string) string { + return string(hack.Slice(s)) +} diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go new file mode 100644 index 0000000000000..9a22afa24bd67 --- /dev/null +++ b/infoschema/slow_log_test.go @@ -0,0 +1,28 @@ +package infoschema + +import ( + "fmt" + "testing" +) + +func TestParseSlowLogFile(t *testing.T) { + rowMap, err := parseSlowLogFile("/Users/cs/code/goread/src/github.com/pingcap/tidb/slow2.log") + if err != nil { + fmt.Println(err) + } + fmt.Println(rowMap) +} + +func BenchmarkCopyString(b *testing.B) { + str := "abcdefghigklmnopqrst" + for i := 0; i < b.N; i++ { + _ = copyString(str) + } +} + +func BenchmarkCopyStringHack(b *testing.B) { + str := "abcdefghigklmnopqrst" + for i := 0; i < b.N; i++ { + _ = copyStringHack(str) + } +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index f28b11e9f0666..d08eeb7a5fd3f 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -803,6 +803,10 @@ type BatchSize struct { } const ( + SlowLogPrefixStr = "# " + SlowLogSpaceMarkStr = ": " + SlowLogSQLSuffixStr = ";" + SlowLogTimeStr = "Time" SlowLogTxnStartTSStr = "Txn_start_ts" SlowLogUserStr = "User" SlowLogConnIDStr = "Conn_ID" @@ -810,6 +814,7 @@ const ( SlowLogDBStr = "DB" SlowLogIsInternalStr = "Is_internal" SlowLogIndexNamesStr = "Index_names" + SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. ) // SlowLogFormat uses for formatting slow log. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 00e8a2ee5533a..1b9360d04bb7d 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -49,6 +49,15 @@ type CommitDetails struct { TxnRetry int } +const ( + ProcessTimeStr = "Process_time" + WaitTimeStr = "Wait_time" + BackoffTimeStr = "Backoff_time" + RequestCountStr = "Request_count" + TotalKeysStr = "Total_keys" + ProcessedKeysStr = "Processed_keys" +) + // String implements the fmt.Stringer interface. func (d ExecDetails) String() string { parts := make([]string, 0, 6) diff --git a/util/logutil/log.go b/util/logutil/log.go index aea4ece44c57f..36759c0949b1d 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -192,6 +192,10 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { return b.Bytes(), nil } +const ( + SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" +) + type slowLogFormatter struct{} func (f *slowLogFormatter) Format(entry *log.Entry) ([]byte, error) { @@ -202,7 +206,7 @@ func (f *slowLogFormatter) Format(entry *log.Entry) ([]byte, error) { b = &bytes.Buffer{} } - fmt.Fprintf(b, "# Time: %s\n", entry.Time.Format("2006-01-02-15:04:05.999999999 -0700")) + fmt.Fprintf(b, "# Time: %s\n", entry.Time.Format(SlowLogTimeFormat)) fmt.Fprintf(b, "%s\n", entry.Message) return b.Bytes(), nil } From 2a6800b96563ea59397740c7598cd9247e1ce14f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 30 Jan 2019 17:21:18 +0800 Subject: [PATCH 05/33] parse slow log update --- infoschema/slow_log.go | 44 ++++++++++++++++++++++++++++++++----- infoschema/slow_log_test.go | 4 +++- infoschema/tables.go | 4 ++++ util/logutil/log.go | 2 +- 4 files changed, 47 insertions(+), 7 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 93c5049180208..204b98da574a0 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -24,23 +25,50 @@ var slowLogCols = []columnInfo{ {variable.SlowLogConnIDStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogQueryTimeStr, mysql.TypeDouble, 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}, + {execdetails.RequestCountStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, + {execdetails.TotalKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, + {execdetails.ProcessedKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, + {variable.SlowLogDBStr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } -func parseSlowLogFile(filePath string) (map[string]types.Datum, error) { +func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { + rowsMap, err := parseSlowLogFile("/Users/cs/code/goread/src/github.com/pingcap/tidb/slow2.log") + if err != nil { + fmt.Println(err) + } + var rows [][]types.Datum + for _, row := range rowsMap { + record := make([]types.Datum, 0, len(slowLogCols)) + for _, col := range slowLogCols { + if v, ok := row[col.name]; ok { + record = append(record, v) + } else { + record = append(record, types.NewDatum(nil)) + } + } + rows = append(rows, record) + } + return rows, nil +} + +func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { file, err := os.Open(filePath) if err != nil { return nil, errors.Trace(err) } defer file.Close() + rows := make([]map[string]types.Datum, 0) rowMap := make(map[string]types.Datum, len(slowLogCols)) startFlag := false - startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + variable.SlowLogSpaceMarkStr scanner := bufio.NewScanner(file) for scanner.Scan() { line := scanner.Text() - fmt.Println(line) // check start if !startFlag && strings.Contains(line, startPrefix) { t, err := parseTime(line[len(startPrefix):]) @@ -90,6 +118,9 @@ func parseSlowLogFile(filePath string) (map[string]types.Datum, error) { break } value = types.NewDatum(num) + case mysql.TypeTiny: + // parse bool + value = types.NewDatum(fieldValues[i+1] == "true") } rowMap[field] = value @@ -97,6 +128,9 @@ func parseSlowLogFile(filePath string) (map[string]types.Datum, error) { } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { // get the sql string, and mark the start flag to false. rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(copyStringHack(line)) + rows = append(rows, rowMap) + rowMap = make(map[string]types.Datum, len(slowLogCols)) + startFlag = false } } } @@ -104,14 +138,14 @@ func parseSlowLogFile(filePath string) (map[string]types.Datum, error) { if err := scanner.Err(); err != nil { return nil, errors.Trace(err) } - return rowMap, nil + return rows, nil } func parseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) if err != nil { - err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\"", s, logutil.SlowLogTimeFormat) + err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) } return t, err } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 9a22afa24bd67..3e0219de4946b 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -10,7 +10,9 @@ func TestParseSlowLogFile(t *testing.T) { if err != nil { fmt.Println(err) } - fmt.Println(rowMap) + for k, v := range rowMap { + fmt.Println(k, v) + } } func BenchmarkCopyString(b *testing.B) { diff --git a/infoschema/tables.go b/infoschema/tables.go index 6e34a589fd0ec..94cc9598888d7 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -66,6 +66,7 @@ const ( tableTableSpaces = "TABLESPACES" tableCollationCharacterSetApplicability = "COLLATION_CHARACTER_SET_APPLICABILITY" tableProcesslist = "PROCESSLIST" + tableSlowLog = "SLOW_LOG" ) type columnInfo struct { @@ -1390,6 +1391,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableTableSpaces: tableTableSpacesCols, tableCollationCharacterSetApplicability: tableCollationCharacterSetApplicabilityCols, tableProcesslist: tableProcesslistCols, + tableSlowLog: slowLogCols, } func createInfoSchemaTable(handle *Handle, meta *model.TableInfo) *infoschemaTable { @@ -1479,6 +1481,8 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows = dataForCollationCharacterSetApplicability() case tableProcesslist: fullRows = dataForProcesslist(ctx) + case tableSlowLog: + fullRows, err = dataForSlowLog(ctx) } if err != nil { return nil, errors.Trace(err) diff --git a/util/logutil/log.go b/util/logutil/log.go index 36759c0949b1d..1e0b7b3bb3732 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -193,7 +193,7 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { } const ( - SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" + SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 +0800" ) type slowLogFormatter struct{} From ee25e4c3c6863638d2cdc261c0d61d6a9d55725f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 30 Jan 2019 20:24:19 +0800 Subject: [PATCH 06/33] add tidb_slow_query_file session variable --- infoschema/slow_log.go | 17 ++++++++--------- infoschema/tables.go | 4 ++-- sessionctx/variable/session.go | 6 ++++++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 3 +++ tidb-server/main.go | 1 + 6 files changed, 21 insertions(+), 11 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 204b98da574a0..3c4bccfb73035 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -2,7 +2,6 @@ package infoschema import ( "bufio" - "fmt" "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" @@ -18,7 +17,7 @@ import ( "time" ) -var slowLogCols = []columnInfo{ +var slowQueryCols = []columnInfo{ {variable.SlowLogTimeStr, mysql.TypeDatetime, -1, 0, nil, nil}, {variable.SlowLogTxnStartTSStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogUserStr, mysql.TypeVarchar, 64, 0, nil, nil}, @@ -36,14 +35,14 @@ var slowLogCols = []columnInfo{ } func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { - rowsMap, err := parseSlowLogFile("/Users/cs/code/goread/src/github.com/pingcap/tidb/slow2.log") + rowsMap, err := parseSlowLogFile(ctx.GetSessionVars().SlowQueryFile) if err != nil { - fmt.Println(err) + return nil, errors.Trace(err) } var rows [][]types.Datum for _, row := range rowsMap { - record := make([]types.Datum, 0, len(slowLogCols)) - for _, col := range slowLogCols { + record := make([]types.Datum, 0, len(slowQueryCols)) + for _, col := range slowQueryCols { if v, ok := row[col.name]; ok { record = append(record, v) } else { @@ -63,7 +62,7 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { defer file.Close() rows := make([]map[string]types.Datum, 0) - rowMap := make(map[string]types.Datum, len(slowLogCols)) + rowMap := make(map[string]types.Datum, len(slowQueryCols)) startFlag := false startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + variable.SlowLogSpaceMarkStr scanner := bufio.NewScanner(file) @@ -96,7 +95,7 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { if strings.HasSuffix(field, ":") { field = field[:len(field)-1] } - col := findColumnByName(slowLogCols, field) + col := findColumnByName(slowQueryCols, field) if col == nil { continue } @@ -129,7 +128,7 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { // get the sql string, and mark the start flag to false. rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(copyStringHack(line)) rows = append(rows, rowMap) - rowMap = make(map[string]types.Datum, len(slowLogCols)) + rowMap = make(map[string]types.Datum, len(slowQueryCols)) startFlag = false } } diff --git a/infoschema/tables.go b/infoschema/tables.go index 94cc9598888d7..2553a7275d087 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -66,7 +66,7 @@ const ( tableTableSpaces = "TABLESPACES" tableCollationCharacterSetApplicability = "COLLATION_CHARACTER_SET_APPLICABILITY" tableProcesslist = "PROCESSLIST" - tableSlowLog = "SLOW_LOG" + tableSlowLog = "SLOW_QUERY" ) type columnInfo struct { @@ -1391,7 +1391,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableTableSpaces: tableTableSpacesCols, tableCollationCharacterSetApplicability: tableCollationCharacterSetApplicabilityCols, tableProcesslist: tableProcesslistCols, - tableSlowLog: slowLogCols, + tableSlowLog: slowQueryCols, } func createInfoSchemaTable(handle *Handle, meta *model.TableInfo) *infoschemaTable { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d08eeb7a5fd3f..c4214b8f223ea 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -334,6 +334,9 @@ type SessionVars struct { // CommandValue indicates which command current session is doing. CommandValue uint32 + + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. + SlowQueryFile string } // NewSessionVars creates a session vars object. @@ -359,6 +362,7 @@ func NewSessionVars() *SessionVars { EnableRadixJoin: false, L2CacheSize: cpuid.CPU.Cache.L2, CommandValue: uint32(mysql.ComSleep), + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -691,6 +695,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) + case TiDBSlowQueryFile: + s.SlowQueryFile = val } s.systems[name] = val return nil diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9befa39267a1c..276ffc33e8862 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -681,6 +681,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, boolToIntStr(DefTiDBUseRadixJoin)}, + {ScopeSession, TiDBSlowQueryFile, ""}, } // SynonymsSysVariables is synonyms of system variables. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 684e2883741f4..7ed4a85b95186 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -228,6 +228,9 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. + TiDBSlowQueryFile = "tidb_slow_query_file" ) // Default TiDB system variable values. diff --git a/tidb-server/main.go b/tidb-server/main.go index 95b8bf01cfc57..e02a62b5f5ba3 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -436,6 +436,7 @@ func setGlobalVars() { variable.SysVars[variable.TIDBMemQuotaQuery].Value = strconv.FormatInt(cfg.MemQuotaQuery, 10) variable.SysVars["lower_case_table_names"].Value = strconv.Itoa(cfg.LowerCaseTableNames) + variable.SysVars[variable.TiDBSlowQueryFile].Value = cfg.Log.SlowQueryFile // For CI environment we default enable prepare-plan-cache. plannercore.SetPreparedPlanCache(config.CheckTableBeforeDrop || cfg.PreparedPlanCache.Enabled) From 08a7fb4111f278259257baf58233341356b1aae0 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 12 Feb 2019 17:28:18 +0800 Subject: [PATCH 07/33] update test --- infoschema/slow_log.go | 29 ++++++++++------- infoschema/slow_log_test.go | 65 +++++++++++++++++++++++++++---------- 2 files changed, 65 insertions(+), 29 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 3c4bccfb73035..350061a6701a7 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -2,6 +2,11 @@ package infoschema import ( "bufio" + "os" + "strconv" + "strings" + "time" + "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" @@ -11,10 +16,6 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" log "github.com/sirupsen/logrus" - "os" - "strconv" - "strings" - "time" ) var slowQueryCols = []columnInfo{ @@ -54,18 +55,12 @@ func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { return rows, nil } -func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { - file, err := os.Open(filePath) - if err != nil { - return nil, errors.Trace(err) - } - defer file.Close() - +func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { rows := make([]map[string]types.Datum, 0) rowMap := make(map[string]types.Datum, len(slowQueryCols)) startFlag := false startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + variable.SlowLogSpaceMarkStr - scanner := bufio.NewScanner(file) + for scanner.Scan() { line := scanner.Text() // check start @@ -140,6 +135,16 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { return rows, nil } +func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { + file, err := os.Open(filePath) + if err != nil { + return nil, errors.Trace(err) + } + defer file.Close() + + return parseSlowLog(bufio.NewScanner(file)) +} + func parseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 3e0219de4946b..da5144510d54e 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -1,30 +1,61 @@ package infoschema import ( - "fmt" + "bufio" + "bytes" "testing" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" ) func TestParseSlowLogFile(t *testing.T) { - rowMap, err := parseSlowLogFile("/Users/cs/code/goread/src/github.com/pingcap/tidb/slow2.log") + slowLog := bytes.NewBufferString( + `# Time: 2019-01-24-22:32:29.313255 +0800 +# Txn_start_ts: 405888132465033227 +# Query_time: 0.216905 +# Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 +# Is_internal: true +select * from t;`) + scanner := bufio.NewScanner(slowLog) + rows, err := parseSlowLog(scanner) if err != nil { - fmt.Println(err) + t.Fatalf("parse slow log failed") } - for k, v := range rowMap { - fmt.Println(k, v) + if len(rows) != 1 { + t.Fatalf("parse slow log failed") } -} - -func BenchmarkCopyString(b *testing.B) { - str := "abcdefghigklmnopqrst" - for i := 0; i < b.N; i++ { - _ = copyString(str) + row := rows[0] + t1 := types.Time{ + Time: types.FromDate(2019, 01, 24, 22, 32, 29, 313255), + Type: mysql.TypeDatetime, + Fsp: types.MaxFsp, } -} - -func BenchmarkCopyStringHack(b *testing.B) { - str := "abcdefghigklmnopqrst" - for i := 0; i < b.N; i++ { - _ = copyStringHack(str) + if logTime, ok := row["Time"]; !ok || logTime.GetMysqlTime() != t1 { + t.Fatalf("parse slow log failed") + } + if ts, ok := row["Txn_start_ts"]; !ok || ts.GetUint64() != 405888132465033227 { + t.Fatalf("parse slow log failed") + } + if queryTime, ok := row["Query_time"]; !ok || queryTime.GetFloat64() != 0.216905 { + t.Fatalf("parse slow log failed") + } + if ProcessTime, ok := row["Process_time"]; !ok || ProcessTime.GetFloat64() != 0.021 { + t.Fatalf("parse slow log failed") + } + if requestCount, ok := row["Request_count"]; !ok || requestCount.GetUint64() != 1 { + t.Fatalf("parse slow log failed") + } + if totalKeys, ok := row["Total_keys"]; !ok || totalKeys.GetUint64() != 637 { + t.Fatalf("parse slow log failed") + } + if processedKeys, ok := row["Processed_keys"]; !ok || processedKeys.GetUint64() != 436 { + t.Fatalf("parse slow log failed") + } + if isInternal, ok := row["Is_internal"]; !ok || isInternal.GetInt64() != 1 { + t.Fatalf("parse slow log failed") + } + if sql, ok := row["Query"]; !ok || sql.GetString() != "select * from t;" { + t.Fatalf("parse slow log failed") } } From 103a354d7a17d027e10f15eccdefa8000f9bbb5f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 12 Feb 2019 18:50:11 +0800 Subject: [PATCH 08/33] fix test --- util/execdetails/execdetails_test.go | 3 ++- util/logutil/log_test.go | 21 ++++++++++++--------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index cd69856070bf8..7affc023abae4 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -39,7 +39,8 @@ func TestString(t *testing.T) { TxnRetry: 1, }, } - expected := "process_time:2.005s wait_time:1s backoff_time:1s request_count:1 total_keys:100 processed_keys:10 prewrite_time:1s commit_time:1s get_commit_ts_time:1s total_backoff_time:1s resolve_lock_time:1s local_latch_wait_time:1s write_keys:1 write_size:1 prewrite_region:1 txn_retry:1" + expected := "Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Total_keys: 100 Processed_keys: 10 Prewrite_time: 1 Commit_time: 1 " + + "Get_commit_ts_time: 1 Total_backoff_time: 1 Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1" if str := detail.String(); str != expected { t.Errorf("got:\n%s\nexpected:\n%s", str, expected) } diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index 5327bc5b8643b..b583b8fe2ccf1 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -106,21 +106,24 @@ func (s *testLogSuite) TestSlowQueryLogger(c *C) { if err != nil { break } - c.Assert(str, Matches, logPattern) + if strings.HasPrefix(str, "# ") { + c.Assert(str, Matches, `# Time: .*?\n`) + } else { + c.Assert(str, Matches, `.*? message\n`) + } } c.Assert(err, Equals, io.EOF) } -func (s *testLogSuite) TestSlowQueryLoggerKeepOrder(c *C) { - fileName := "slow_query" - conf := &LogConfig{Level: "warn", File: FileLogConfig{}, Format: "text", DisableTimestamp: true, SlowQueryFile: fileName} +func (s *testLogSuite) TestLoggerKeepOrder(c *C) { + conf := &LogConfig{Level: "warn", File: FileLogConfig{}, Format: "text", DisableTimestamp: true} c.Assert(InitLogger(conf), IsNil) - defer os.Remove(fileName) - ft, ok := SlowQueryLogger.Formatter.(*textFormatter) + logger := log.StandardLogger() + ft, ok := logger.Formatter.(*textFormatter) c.Assert(ok, IsTrue) - c.Assert(ft.EnableEntryOrder, IsTrue) - SlowQueryLogger.Out = s.buf - logEntry := log.NewEntry(SlowQueryLogger) + ft.EnableEntryOrder = true + logger.Out = s.buf + logEntry := log.NewEntry(logger) logEntry.Data = log.Fields{ "connectionId": 1, "costTime": "1", From a2e68e6e03326cb5e6c7249e8e99c8f2692100da Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 12 Feb 2019 19:14:45 +0800 Subject: [PATCH 09/33] refine code --- infoschema/slow_log.go | 90 ++++++++++++++++++++++++------------------ 1 file changed, 52 insertions(+), 38 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 350061a6701a7..d83ae3f4fe194 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -55,6 +55,16 @@ func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { return rows, nil } +func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { + file, err := os.Open(filePath) + if err != nil { + return nil, errors.Trace(err) + } + defer file.Close() + + return parseSlowLog(bufio.NewScanner(file)) +} + func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { rows := make([]map[string]types.Datum, 0) rowMap := make(map[string]types.Datum, len(slowQueryCols)) @@ -65,17 +75,13 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { line := scanner.Text() // check start if !startFlag && strings.Contains(line, startPrefix) { - t, err := parseTime(line[len(startPrefix):]) + value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(startPrefix):]) if err != nil { log.Errorf("parse slow log error: %v", err) // temporary ignore now. continue } - rowMap[variable.SlowLogTimeStr] = types.NewTimeDatum(types.Time{ - Time: types.FromGoTime(t), - Type: mysql.TypeDatetime, - Fsp: types.MaxFsp, - }) + rowMap[variable.SlowLogTimeStr] = *value startFlag = true continue } @@ -90,33 +96,13 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { if strings.HasSuffix(field, ":") { field = field[:len(field)-1] } - col := findColumnByName(slowQueryCols, field) - if col == nil { + value, err := parseSlowLogField(field, fieldValues[i+1]) + if err != nil { + log.Errorf("parse slow log error: %v", err) + // temporary ignore now. continue } - var value types.Datum - switch col.tp { - case mysql.TypeLonglong: - num, err := strconv.ParseUint(fieldValues[i+1], 10, 64) - if err != nil { - log.Errorf("parse slow log error: %v", err) - break - } - value = types.NewUintDatum(num) - case mysql.TypeVarchar: - value = types.NewStringDatum(fieldValues[i+1]) - case mysql.TypeDouble: - num, err := strconv.ParseFloat(fieldValues[i+1], 64) - if err != nil { - log.Errorf("parse slow log error: %v", err) - break - } - value = types.NewDatum(num) - case mysql.TypeTiny: - // parse bool - value = types.NewDatum(fieldValues[i+1] == "true") - } - rowMap[field] = value + rowMap[field] = *value } } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { @@ -135,14 +121,42 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { return rows, nil } -func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { - file, err := os.Open(filePath) - if err != nil { - return nil, errors.Trace(err) +func parseSlowLogField(field, value string) (*types.Datum, error) { + col := findColumnByName(slowQueryCols, field) + if col == nil { + return nil, errors.Errorf("can't found column %v", field) } - defer file.Close() - - return parseSlowLog(bufio.NewScanner(file)) + var val types.Datum + switch col.tp { + case mysql.TypeLonglong: + num, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + val = types.NewUintDatum(num) + case mysql.TypeVarchar: + val = types.NewStringDatum(value) + case mysql.TypeDouble: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return nil, errors.Trace(err) + } + val = types.NewDatum(num) + case mysql.TypeTiny: + // parse bool + val = types.NewDatum(value == "true") + case mysql.TypeDatetime: + t, err := parseTime(value) + if err != nil { + return nil, errors.Trace(err) + } + val = types.NewTimeDatum(types.Time{ + Time: types.FromGoTime(t), + Type: mysql.TypeDatetime, + Fsp: types.MaxFsp, + }) + } + return &val, nil } func parseTime(s string) (time.Time, error) { From 20342c4dc7c2bdec8bb40298f456ff739277c8d2 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 12 Feb 2019 19:56:11 +0800 Subject: [PATCH 10/33] add test for select slow_query table --- infoschema/slow_log.go | 18 ++++-------------- infoschema/tables_test.go | 36 ++++++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+), 14 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index d83ae3f4fe194..32a8d658e9fe4 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -73,12 +73,11 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { for scanner.Scan() { line := scanner.Text() - // check start + // Check slow log entry start flag. if !startFlag && strings.Contains(line, startPrefix) { value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(startPrefix):]) if err != nil { log.Errorf("parse slow log error: %v", err) - // temporary ignore now. continue } rowMap[variable.SlowLogTimeStr] = *value @@ -87,7 +86,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { } if startFlag { - // parse field. + // Parse slow log field. if strings.Contains(line, variable.SlowLogPrefixStr) { line = line[len(variable.SlowLogPrefixStr):] fieldValues := strings.Split(line, " ") @@ -99,15 +98,14 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { value, err := parseSlowLogField(field, fieldValues[i+1]) if err != nil { log.Errorf("parse slow log error: %v", err) - // temporary ignore now. continue } rowMap[field] = *value } } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { - // get the sql string, and mark the start flag to false. - rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(copyStringHack(line)) + // Get the sql string, and mark the start flag to false. + rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(string(hack.Slice(line))) rows = append(rows, rowMap) rowMap = make(map[string]types.Datum, len(slowQueryCols)) startFlag = false @@ -176,11 +174,3 @@ func findColumnByName(cols []columnInfo, colName string) *columnInfo { } return nil } - -func copyString(s string) string { - return string([]byte(s)) -} - -func copyStringHack(s string) string { - return string(hack.Slice(s)) -} diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index fd450cf134a1f..99f2d24978673 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -14,6 +14,8 @@ package infoschema_test import ( + "fmt" + "os" "strconv" . "github.com/pingcap/check" @@ -24,6 +26,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/util/testutil" ) func (s *testSuite) TestInfoschemaFieldValue(c *C) { @@ -228,3 +231,36 @@ func (s *testSuite) TestTableIDAndIndexID(c *C) { c.Assert(tblID, Greater, 0) tk.MustQuery("select * from information_schema.tidb_indexes where table_schema = 'test' and table_name = 't'").Check(testkit.Rows("test t 0 PRIMARY 1 a 0", "test t 1 k1 1 b 1")) } + +func (s *testSuite) TestSlowQuery(c *C) { + testleak.BeforeTest() + defer testleak.AfterTest(c)() + store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() + session.SetStatsLease(0) + do, err := session.BootstrapSession(store) + c.Assert(err, IsNil) + defer do.Close() + tk := testkit.NewTestKit(c, store) + // Prepare slow log file. + slowLogFileName := "tidb_slow.log" + f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) + c.Assert(err, IsNil) + defer os.Remove(slowLogFileName) + _, err = f.Write([]byte(`# Time: 2019-02-12-19:33:56.571953 +0800 +# Txn_start_ts: 406315658548871171 +# User: root@127.0.0.1 +# Conn_ID: 6 +# Query_time: 4.895492 +# Process_time: 0.161 Request_count: 1 Total_keys: 100001 Processed_keys: 100000 +# DB: test +# Is_internal: false +select * from t_slim;`)) + c.Assert(f.Close(), IsNil) + + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) + re := tk.MustQuery("select * from information_schema.slow_query") + c.Assert(err, IsNil) + re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|||1|100001|100000|test|0|select * from t_slim;")) +} From b1deabd94316fda3aef4714ce387e015bae5f789 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 12 Feb 2019 20:41:13 +0800 Subject: [PATCH 11/33] fix ci errcheck --- infoschema/slow_log.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 32a8d658e9fe4..3b1de43f16bf8 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -60,7 +60,11 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { if err != nil { return nil, errors.Trace(err) } - defer file.Close() + defer func() { + if err = file.Close(); err != nil { + log.Error(err) + } + }() return parseSlowLog(bufio.NewScanner(file)) } From 9ed2b3b6c0d82fdbbc4c2ef6b97c7818076693e0 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 27 Feb 2019 11:08:10 +0800 Subject: [PATCH 12/33] refine code --- sessionctx/variable/session.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 126a95419d947..27a36c7017852 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -826,7 +826,7 @@ const ( ) // SlowLogFormat uses for formatting slow log. -func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, index_IDs string, sql string) string { +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexNames string, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogTxnStartTSStr, txnTS)) @@ -843,8 +843,8 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe if len(s.CurrentDB) > 0 { buf.WriteString(fmt.Sprintf("# %s: %s\n", SlowLogDBStr, s.CurrentDB)) } - if len(index_IDs) > 0 { - buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogIndexNamesStr, index_IDs)) + if len(indexNames) > 0 { + buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogIndexNamesStr, indexNames)) } buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogIsInternalStr, s.InRestrictedSQL)) if len(sql) > 0 { From bfbb5a13f4fe5fb1af905c409924b76b1a7493da Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 27 Feb 2019 13:42:56 +0800 Subject: [PATCH 13/33] address comment and add TODO --- infoschema/slow_log.go | 2 ++ sessionctx/variable/session.go | 16 ++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 3b1de43f16bf8..8d4d27325ca5d 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -55,6 +55,7 @@ func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { return rows, nil } +// TODO: Support parse multiple log-files. func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { file, err := os.Open(filePath) if err != nil { @@ -69,6 +70,7 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { return parseSlowLog(bufio.NewScanner(file)) } +// TODO: optimize for parse huge log-file. func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { rows := make([]map[string]types.Datum, 0) rowMap := make(map[string]types.Datum, len(slowQueryCols)) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 27a36c7017852..5bfecfb6dcf66 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -829,24 +829,24 @@ const ( func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexNames string, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() - buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogTxnStartTSStr, txnTS)) + buf.WriteString("# " + SlowLogTxnStartTSStr + ": " + strconv.FormatUint(txnTS, 10) + "\n") if s.User != nil { - buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogUserStr, s.User)) + buf.WriteString("# " + SlowLogUserStr + ": " + s.User.String() + "\n") } if s.ConnectionID != 0 { - buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogConnIDStr, s.ConnectionID)) + buf.WriteString("# " + SlowLogConnIDStr + ": " + strconv.FormatUint(s.ConnectionID, 10) + "\n") } - buf.WriteString(fmt.Sprintf("# %s: %f\n", SlowLogQueryTimeStr, costTime.Seconds())) + buf.WriteString("# " + SlowLogQueryTimeStr + ": " + strconv.FormatFloat(costTime.Seconds(), 'f', -1, 64) + "\n") if len(execDetailStr) > 0 { - buf.WriteString(fmt.Sprintf("# %s\n", execDetailStr)) + buf.WriteString("# " + execDetailStr + "\n") } if len(s.CurrentDB) > 0 { - buf.WriteString(fmt.Sprintf("# %s: %s\n", SlowLogDBStr, s.CurrentDB)) + buf.WriteString("# " + SlowLogDBStr + ": " + s.CurrentDB + "\n") } if len(indexNames) > 0 { - buf.WriteString(fmt.Sprintf("# %s: %v\n", SlowLogIndexNamesStr, indexNames)) + buf.WriteString("# " + SlowLogIndexNamesStr + ": " + indexNames + "\n") } - buf.WriteString(fmt.Sprintf("# %v: %v\n", SlowLogIsInternalStr, s.InRestrictedSQL)) + buf.WriteString("# " + SlowLogIsInternalStr + ": " + strconv.FormatBool(s.InRestrictedSQL) + "\n") if len(sql) > 0 { buf.WriteString(sql) if sql[len(sql)-1] != ';' { From c17171a38463a9737c630ca6a3f65b1e593957f9 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 27 Feb 2019 19:36:02 +0800 Subject: [PATCH 14/33] address comment and add test --- sessionctx/variable/session.go | 16 ++++++------ sessionctx/variable/session_test.go | 38 +++++++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 8 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 5bfecfb6dcf66..718b8438961d1 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -829,24 +829,24 @@ const ( func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexNames string, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() - buf.WriteString("# " + SlowLogTxnStartTSStr + ": " + strconv.FormatUint(txnTS, 10) + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") if s.User != nil { - buf.WriteString("# " + SlowLogUserStr + ": " + s.User.String() + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogUserStr + SlowLogSpaceMarkStr + s.User.String() + "\n") } if s.ConnectionID != 0 { - buf.WriteString("# " + SlowLogConnIDStr + ": " + strconv.FormatUint(s.ConnectionID, 10) + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogConnIDStr + SlowLogSpaceMarkStr + strconv.FormatUint(s.ConnectionID, 10) + "\n") } - buf.WriteString("# " + SlowLogQueryTimeStr + ": " + strconv.FormatFloat(costTime.Seconds(), 'f', -1, 64) + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogQueryTimeStr + SlowLogSpaceMarkStr + strconv.FormatFloat(costTime.Seconds(), 'f', -1, 64) + "\n") if len(execDetailStr) > 0 { - buf.WriteString("# " + execDetailStr + "\n") + buf.WriteString(SlowLogPrefixStr + execDetailStr + "\n") } if len(s.CurrentDB) > 0 { - buf.WriteString("# " + SlowLogDBStr + ": " + s.CurrentDB + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogDBStr + SlowLogSpaceMarkStr + s.CurrentDB + "\n") } if len(indexNames) > 0 { - buf.WriteString("# " + SlowLogIndexNamesStr + ": " + indexNames + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogIndexNamesStr + SlowLogSpaceMarkStr + indexNames + "\n") } - buf.WriteString("# " + SlowLogIsInternalStr + ": " + strconv.FormatBool(s.InRestrictedSQL) + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogIsInternalStr + SlowLogSpaceMarkStr + strconv.FormatBool(s.InRestrictedSQL) + "\n") if len(sql) > 0 { buf.WriteString(sql) if sql[len(sql)-1] != ';' { diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 72a045e7667ed..2f09c1438b8cc 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -14,7 +14,11 @@ package variable_test import ( + "time" + . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" ) @@ -78,3 +82,37 @@ func (*testSessionSuite) TestSession(c *C) { c.Assert(ss.CopiedRows(), Equals, uint64(0)) c.Assert(ss.WarningCount(), Equals, uint16(0)) } + +func (*testSessionSuite) TestSlowLogFormat(c *C) { + ctx := mock.NewContext() + + seVar := ctx.GetSessionVars() + c.Assert(seVar, NotNil) + + seVar.User = &auth.UserIdentity{Username: "root", Hostname: "192.168.0.1"} + seVar.ConnectionID = 1 + seVar.CurrentDB = "test" + seVar.InRestrictedSQL = true + txnTS := uint64(406649736972468225) + costTime := time.Second + execDetail := execdetails.ExecDetails{ + ProcessTime: time.Second * time.Duration(2), + WaitTime: time.Minute, + BackoffTime: time.Millisecond, + RequestCount: 2, + TotalKeys: 10000, + ProcessedKeys: 20001, + } + resultString := `# Txn_start_ts: 406649736972468225 +# User: root@192.168.0.1 +# Conn_ID: 1 +# Query_time: 1 +# Process_time: 2 Wait_time: 60 Backoff_time: 0.001 Request_count: 2 Total_keys: 10000 Processed_keys: 20001 +# DB: test +# Index_names: idx1,idx2 +# Is_internal: true +select * from t;` + logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "idx1,idx2", "select * from t") + c.Assert(logString, Equals, resultString) + +} From 2b19b61ebe05e052fab710bb46e33eb33dc8b797 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 27 Feb 2019 19:46:43 +0800 Subject: [PATCH 15/33] add comment --- sessionctx/variable/session.go | 12 ++++++++++++ util/execdetails/execdetails.go | 6 ++++++ 2 files changed, 18 insertions(+) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 718b8438961d1..1ccd25e6e553f 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -811,17 +811,29 @@ type BatchSize struct { } const ( + // SlowLogPrefixStr is slow log prefix. SlowLogPrefixStr = "# " + // SlowLogSpaceMarkStr is slow log space mark. SlowLogSpaceMarkStr = ": " + // SlowLogSQLSuffixStr is slow log suffix. SlowLogSQLSuffixStr = ";" + // SlowLogTimeStr is slow log field name. SlowLogTimeStr = "Time" + // SlowLogTxnStartTSStr is slow log field name. SlowLogTxnStartTSStr = "Txn_start_ts" + // SlowLogUserStr is slow log field name. SlowLogUserStr = "User" + // SlowLogConnIDStr is slow log field name. SlowLogConnIDStr = "Conn_ID" + // SlowLogQueryTimeStr is slow log field name. SlowLogQueryTimeStr = "Query_time" + // SlowLogDBStr is slow log field name. SlowLogDBStr = "DB" + // SlowLogIsInternalStr is slow log field name. SlowLogIsInternalStr = "Is_internal" + // SlowLogIndexNamesStr is slow log field name. SlowLogIndexNamesStr = "Index_names" + // SlowLogQuerySQLStr is slow log field name. SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. ) diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index c05c615e23269..69c1ad43a27ed 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -54,11 +54,17 @@ type CommitDetails struct { } const ( + // ProcessTimeStr means the time of all coprocessor process. ProcessTimeStr = "Process_time" + // WaitTimeStr means the time of all coprocessor wait. WaitTimeStr = "Wait_time" + // BackoffTimeStr means the time of all back-off. BackoffTimeStr = "Backoff_time" + // RequestCountStr means the request count. RequestCountStr = "Request_count" + // TotalKeysStr means the total scan keys. TotalKeysStr = "Total_keys" + // ProcessedKeysStr means the total processed keys. ProcessedKeysStr = "Processed_keys" ) From ab6751e99fb502186c7b9502291f8d062fefd587 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 28 Feb 2019 15:45:10 +0800 Subject: [PATCH 16/33] fmt code --- sessionctx/variable/session.go | 18 +++++++++--------- util/execdetails/execdetails.go | 10 +++++----- util/logutil/log.go | 1 + 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1ccd25e6e553f..752fa12ae6248 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -812,29 +812,29 @@ type BatchSize struct { const ( // SlowLogPrefixStr is slow log prefix. - SlowLogPrefixStr = "# " + SlowLogPrefixStr = "# " // SlowLogSpaceMarkStr is slow log space mark. - SlowLogSpaceMarkStr = ": " + SlowLogSpaceMarkStr = ": " // SlowLogSQLSuffixStr is slow log suffix. - SlowLogSQLSuffixStr = ";" + SlowLogSQLSuffixStr = ";" // SlowLogTimeStr is slow log field name. - SlowLogTimeStr = "Time" + SlowLogTimeStr = "Time" // SlowLogTxnStartTSStr is slow log field name. SlowLogTxnStartTSStr = "Txn_start_ts" // SlowLogUserStr is slow log field name. - SlowLogUserStr = "User" + SlowLogUserStr = "User" // SlowLogConnIDStr is slow log field name. - SlowLogConnIDStr = "Conn_ID" + SlowLogConnIDStr = "Conn_ID" // SlowLogQueryTimeStr is slow log field name. - SlowLogQueryTimeStr = "Query_time" + SlowLogQueryTimeStr = "Query_time" // SlowLogDBStr is slow log field name. - SlowLogDBStr = "DB" + SlowLogDBStr = "DB" // SlowLogIsInternalStr is slow log field name. SlowLogIsInternalStr = "Is_internal" // SlowLogIndexNamesStr is slow log field name. SlowLogIndexNamesStr = "Index_names" // SlowLogQuerySQLStr is slow log field name. - SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. + SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. ) // SlowLogFormat uses for formatting slow log. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 69c1ad43a27ed..f705ed089af5c 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -55,15 +55,15 @@ type CommitDetails struct { const ( // ProcessTimeStr means the time of all coprocessor process. - ProcessTimeStr = "Process_time" + ProcessTimeStr = "Process_time" // WaitTimeStr means the time of all coprocessor wait. - WaitTimeStr = "Wait_time" + WaitTimeStr = "Wait_time" // BackoffTimeStr means the time of all back-off. - BackoffTimeStr = "Backoff_time" + BackoffTimeStr = "Backoff_time" // RequestCountStr means the request count. - RequestCountStr = "Request_count" + RequestCountStr = "Request_count" // TotalKeysStr means the total scan keys. - TotalKeysStr = "Total_keys" + TotalKeysStr = "Total_keys" // ProcessedKeysStr means the total processed keys. ProcessedKeysStr = "Processed_keys" ) diff --git a/util/logutil/log.go b/util/logutil/log.go index be1b32e64b781..66a72c526a41a 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -206,6 +206,7 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { } const ( + // SlowLogTimeFormat is the time format for slow log. SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 +0800" ) From b873963c75a3a806d8ead9a1aabe674c02eb6789 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 1 Mar 2019 14:07:17 +0800 Subject: [PATCH 17/33] address comment --- executor/adapter.go | 18 +++++++++--------- executor/builder.go | 8 ++++---- sessionctx/stmtctx/stmtctx.go | 8 ++++---- sessionctx/variable/session.go | 6 +++--- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 63b4742859941..8a212cc13ecf1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -379,18 +379,18 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { sessVars := a.Ctx.GetSessionVars() sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() - var tableNames, indexNames string - if len(sessVars.StmtCtx.TableNames) > 0 { - tableNames = strings.Join(sessVars.StmtCtx.TableNames, ",") + var tableIDs, indexIDs string + if len(sessVars.StmtCtx.TableIDs) > 0 { + tableIDs = strings.Replace(fmt.Sprintf("table_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.TableIDs), " ", ",", -1) } - if len(sessVars.StmtCtx.IndexNames) > 0 { - indexNames = strings.Join(sessVars.StmtCtx.IndexNames, ",") + if len(sessVars.StmtCtx.IndexIDs) > 0 { + indexIDs = strings.Replace(fmt.Sprintf("index_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { - logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, sql)) + logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) } else { - logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexNames, sql)) + logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -408,8 +408,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { TxnTS: txnTS, User: userString, DB: sessVars.CurrentDB, - TableIDs: tableNames, - IndexIDs: indexNames, + TableIDs: tableIDs, + IndexIDs: indexIDs, Internal: sessVars.InRestrictedSQL, }) } diff --git a/executor/builder.go b/executor/builder.go index 97e9f6a7b15ce..5c4cfe7ff8ca7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1623,7 +1623,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx - sctx.TableNames = append(sctx.TableNames, ts.Table.Name.O) + sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) return ret } @@ -1681,7 +1681,7 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) * is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) ret.ranges = is.Ranges sctx := b.ctx.GetSessionVars().StmtCtx - sctx.IndexNames = append(sctx.IndexNames, is.Index.Name.O) + sctx.IndexIDs = append(sctx.IndexIDs, is.Index.ID) return ret } @@ -1761,8 +1761,8 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo ret.ranges = is.Ranges metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor").Inc() sctx := b.ctx.GetSessionVars().StmtCtx - sctx.IndexNames = append(sctx.IndexNames, is.Index.Name.O) - sctx.TableNames = append(sctx.TableNames, ts.Table.Name.O) + sctx.IndexIDs = append(sctx.IndexIDs, is.Index.ID) + sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) return ret } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 35a9a81e92000..ab308f7a8d1dd 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -110,8 +110,8 @@ type StatementContext struct { NotFillCache bool MemTracker *memory.Tracker RuntimeStatsColl *execdetails.RuntimeStatsColl - TableNames []string - IndexNames []string + TableIDs []int64 + IndexIDs []int64 NowTs time.Time SysTs time.Time StmtType string @@ -354,8 +354,8 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.message = "" sc.mu.warnings = nil sc.mu.Unlock() - sc.TableNames = sc.TableNames[:0] - sc.IndexNames = sc.IndexNames[:0] + sc.TableIDs = sc.TableIDs[:0] + sc.IndexIDs = sc.IndexIDs[:0] } // MergeExecDetails merges a single region execution details into self, used to print diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 752fa12ae6248..be7c9d7867250 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -838,7 +838,7 @@ const ( ) // SlowLogFormat uses for formatting slow log. -func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexNames string, sql string) string { +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") @@ -855,8 +855,8 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe if len(s.CurrentDB) > 0 { buf.WriteString(SlowLogPrefixStr + SlowLogDBStr + SlowLogSpaceMarkStr + s.CurrentDB + "\n") } - if len(indexNames) > 0 { - buf.WriteString(SlowLogPrefixStr + SlowLogIndexNamesStr + SlowLogSpaceMarkStr + indexNames + "\n") + if len(indexIDs) > 0 { + buf.WriteString(SlowLogPrefixStr + SlowLogIndexNamesStr + SlowLogSpaceMarkStr + indexIDs + "\n") } buf.WriteString(SlowLogPrefixStr + SlowLogIsInternalStr + SlowLogSpaceMarkStr + strconv.FormatBool(s.InRestrictedSQL) + "\n") if len(sql) > 0 { From baadd28210a72fe86834fcaf36a487295e5aa880 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 1 Mar 2019 14:11:23 +0800 Subject: [PATCH 18/33] remove blank line --- infoschema/slow_log.go | 1 - 1 file changed, 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 8d4d27325ca5d..8f7612b1e9589 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -165,7 +165,6 @@ func parseSlowLogField(field, value string) (*types.Datum, error) { func parseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) - if err != nil { err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) } From 911f9d1b65bb578e0929a0d1ccb9e50984bca1ba Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 1 Mar 2019 14:20:37 +0800 Subject: [PATCH 19/33] add license --- infoschema/slow_log.go | 13 +++++++++++++ infoschema/slow_log_test.go | 13 +++++++++++++ 2 files changed, 26 insertions(+) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 8f7612b1e9589..34ae2e3534839 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -1,3 +1,16 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package infoschema import ( diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index da5144510d54e..9e28705fe6ec3 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -1,3 +1,16 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package infoschema import ( From ea1226c2bc65fd467a43c900d580034a732cb51e Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 1 Mar 2019 14:45:15 +0800 Subject: [PATCH 20/33] address comment --- infoschema/slow_log.go | 6 +++--- sessionctx/variable/session.go | 4 +++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 34ae2e3534839..228fab2223a67 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -69,6 +69,7 @@ func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { } // TODO: Support parse multiple log-files. +// parseSlowLogFile uses to parse slow log file. func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { file, err := os.Open(filePath) if err != nil { @@ -88,13 +89,12 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { rows := make([]map[string]types.Datum, 0) rowMap := make(map[string]types.Datum, len(slowQueryCols)) startFlag := false - startPrefix := variable.SlowLogPrefixStr + variable.SlowLogTimeStr + variable.SlowLogSpaceMarkStr for scanner.Scan() { line := scanner.Text() // Check slow log entry start flag. - if !startFlag && strings.Contains(line, startPrefix) { - value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(startPrefix):]) + if !startFlag && strings.Contains(line, variable.SlowLogStartPrefixStr) { + value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) if err != nil { log.Errorf("parse slow log error: %v", err) continue diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index be7c9d7867250..4ff86278e58e1 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -811,7 +811,7 @@ type BatchSize struct { } const ( - // SlowLogPrefixStr is slow log prefix. + // SlowLogPrefixStr is slow log row prefix. SlowLogPrefixStr = "# " // SlowLogSpaceMarkStr is slow log space mark. SlowLogSpaceMarkStr = ": " @@ -819,6 +819,8 @@ const ( SlowLogSQLSuffixStr = ";" // SlowLogTimeStr is slow log field name. SlowLogTimeStr = "Time" + // SlowLogStartPrefixStr is slow log start row prefix. + SlowLogStartPrefixStr = SlowLogPrefixStr + SlowLogTimeStr + SlowLogSpaceMarkStr // SlowLogTxnStartTSStr is slow log field name. SlowLogTxnStartTSStr = "Txn_start_ts" // SlowLogUserStr is slow log field name. From c6e6c55d8830871466e4bb8561a5a6f81c9c13f3 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 4 Mar 2019 18:39:55 +0800 Subject: [PATCH 21/33] address comment --- infoschema/slow_log.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 228fab2223a67..d4df7cd163540 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -93,7 +93,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { for scanner.Scan() { line := scanner.Text() // Check slow log entry start flag. - if !startFlag && strings.Contains(line, variable.SlowLogStartPrefixStr) { + if !startFlag && strings.HasPrefix(line, variable.SlowLogStartPrefixStr) { value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) if err != nil { log.Errorf("parse slow log error: %v", err) From c7a35d2b94a3d85edb56a22af59b4b8529f4f2c6 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 4 Mar 2019 18:48:12 +0800 Subject: [PATCH 22/33] change default config for slow log file name --- config/config.toml.example | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.toml.example b/config/config.toml.example index 47895d3efa783..e4b5eed642e7a 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -59,7 +59,7 @@ format = "text" disable-timestamp = false # Stores slow query log into separated files. -slow-query-file = "" +slow-query-file = "tidb-slow.log" # Queries with execution time greater than this value will be logged. (Milliseconds) slow-threshold = 300 From 13a4398987c293c1e62bf97a46bd6dbaa759d60b Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 5 Mar 2019 21:08:30 +0800 Subject: [PATCH 23/33] fix test --- config/config.go | 1 + 1 file changed, 1 insertion(+) diff --git a/config/config.go b/config/config.go index 297b24708cebd..8a12976a6dad3 100644 --- a/config/config.go +++ b/config/config.go @@ -289,6 +289,7 @@ var defaultConf = Config{ Level: "info", Format: "text", File: logutil.NewFileLogConfig(true, logutil.DefaultLogMaxSize), + SlowQueryFile: "tidb-slow.log", SlowThreshold: logutil.DefaultSlowThreshold, ExpensiveThreshold: 10000, QueryLogMaxLen: logutil.DefaultQueryLogMaxLen, From 05b82462ded015ab6dd4eec5f78aa7c2720ec502 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 6 Mar 2019 10:34:13 +0800 Subject: [PATCH 24/33] add index name filed to slow_query table --- infoschema/slow_log.go | 1 + infoschema/tables_test.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index d4df7cd163540..d219187a5eed5 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -45,6 +45,7 @@ var slowQueryCols = []columnInfo{ {execdetails.ProcessedKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogDBStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, + {variable.SlowLogIndexNamesStr, mysql.TypeVarchar, 640, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 99f2d24978673..90cee38ab8dc8 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -262,5 +262,5 @@ select * from t_slim;`)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) re := tk.MustQuery("select * from information_schema.slow_query") c.Assert(err, IsNil) - re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|||1|100001|100000|test|0|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|||1|100001|100000|test|0||select * from t_slim;")) } From 227c5b4c870e5a0056ea1b402403d791e15f3964 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 6 Mar 2019 10:37:23 +0800 Subject: [PATCH 25/33] add comment --- sessionctx/variable/session.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4ff86278e58e1..d7c9fd457b493 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -840,6 +840,16 @@ const ( ) // SlowLogFormat uses for formatting slow log. +// The slow log output is like below: +// # Time: 2019-02-12-19:33:56.571953 +0800 +// # Txn_start_ts: 406315658548871171 +// # User: root@127.0.0.1 +// # Conn_ID: 6 +// # Query_time: 4.895492 +// # Process_time: 0.161 Request_count: 1 Total_keys: 100001 Processed_keys: 100000 +// # DB: test +// # Is_internal: false +// select * from t_slim; func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() From 982da25b36495dc6dc1f090f25dde0ab986df0b8 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Thu, 7 Mar 2019 17:34:32 +0800 Subject: [PATCH 26/33] Update util/execdetails/execdetails.go Co-Authored-By: crazycs520 --- util/execdetails/execdetails.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index f705ed089af5c..89826019d63c5 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -54,7 +54,7 @@ type CommitDetails struct { } const ( - // ProcessTimeStr means the time of all coprocessor process. + // ProcessTimeStr represents the sum of process time of all the coprocessor tasks ProcessTimeStr = "Process_time" // WaitTimeStr means the time of all coprocessor wait. WaitTimeStr = "Wait_time" From 91a7e4617a628e88c5b3ac705567be97da0b989a Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Thu, 7 Mar 2019 17:34:46 +0800 Subject: [PATCH 27/33] Update infoschema/slow_log.go Co-Authored-By: crazycs520 --- infoschema/slow_log.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index d219187a5eed5..f7621a046fc55 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -149,7 +149,7 @@ func parseSlowLogField(field, value string) (*types.Datum, error) { case mysql.TypeLonglong: num, err := strconv.ParseUint(value, 10, 64) if err != nil { - return nil, errors.Trace(err) + return nil, errors.AddStack(err) } val = types.NewUintDatum(num) case mysql.TypeVarchar: From c9067b9a532ab1abceca5a09b0c0acc8871c880f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 7 Mar 2019 18:17:59 +0800 Subject: [PATCH 28/33] address comment --- infoschema/slow_log.go | 12 +++++------- infoschema/tables_test.go | 1 - 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index f7621a046fc55..998a432c9bf15 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -97,8 +97,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { if !startFlag && strings.HasPrefix(line, variable.SlowLogStartPrefixStr) { value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) if err != nil { - log.Errorf("parse slow log error: %v", err) - continue + return rows, err } rowMap[variable.SlowLogTimeStr] = *value startFlag = true @@ -117,8 +116,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { } value, err := parseSlowLogField(field, fieldValues[i+1]) if err != nil { - log.Errorf("parse slow log error: %v", err) - continue + return rows, err } rowMap[field] = *value @@ -134,7 +132,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { } if err := scanner.Err(); err != nil { - return nil, errors.Trace(err) + return nil, errors.AddStack(err) } return rows, nil } @@ -157,7 +155,7 @@ func parseSlowLogField(field, value string) (*types.Datum, error) { case mysql.TypeDouble: num, err := strconv.ParseFloat(value, 64) if err != nil { - return nil, errors.Trace(err) + return nil, errors.AddStack(err) } val = types.NewDatum(num) case mysql.TypeTiny: @@ -166,7 +164,7 @@ func parseSlowLogField(field, value string) (*types.Datum, error) { case mysql.TypeDatetime: t, err := parseTime(value) if err != nil { - return nil, errors.Trace(err) + return nil, err } val = types.NewTimeDatum(types.Time{ Time: types.FromGoTime(t), diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 90cee38ab8dc8..7cad006b0fb01 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -261,6 +261,5 @@ select * from t_slim;`)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) re := tk.MustQuery("select * from information_schema.slow_query") - c.Assert(err, IsNil) re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|||1|100001|100000|test|0||select * from t_slim;")) } From 5e696c7daccbcd2b8ad87de4dba9b0579df78a9f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 8 Mar 2019 14:38:26 +0800 Subject: [PATCH 29/33] address comment --- infoschema/slow_log.go | 192 +++++++++++++++++---------- infoschema/slow_log_test.go | 49 ++----- infoschema/tables_test.go | 5 +- sessionctx/variable/session_test.go | 2 +- util/execdetails/execdetails.go | 17 +-- util/execdetails/execdetails_test.go | 2 +- 6 files changed, 151 insertions(+), 116 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 998a432c9bf15..2d31b080d9679 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -42,36 +42,20 @@ var slowQueryCols = []columnInfo{ {execdetails.BackoffTimeStr, mysql.TypeDouble, 22, 0, nil, nil}, {execdetails.RequestCountStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {execdetails.TotalKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, - {execdetails.ProcessedKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, + {execdetails.ProcessKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogDBStr, mysql.TypeVarchar, 64, 0, nil, nil}, - {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogIndexNamesStr, mysql.TypeVarchar, 640, 0, nil, nil}, + {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { - rowsMap, err := parseSlowLogFile(ctx.GetSessionVars().SlowQueryFile) - if err != nil { - return nil, errors.Trace(err) - } - var rows [][]types.Datum - for _, row := range rowsMap { - record := make([]types.Datum, 0, len(slowQueryCols)) - for _, col := range slowQueryCols { - if v, ok := row[col.name]; ok { - record = append(record, v) - } else { - record = append(record, types.NewDatum(nil)) - } - } - rows = append(rows, record) - } - return rows, nil + return parseSlowLogFile(ctx.GetSessionVars().SlowQueryFile) } // TODO: Support parse multiple log-files. // parseSlowLogFile uses to parse slow log file. -func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { +func parseSlowLogFile(filePath string) ([][]types.Datum, error) { file, err := os.Open(filePath) if err != nil { return nil, errors.Trace(err) @@ -86,20 +70,20 @@ func parseSlowLogFile(filePath string) ([]map[string]types.Datum, error) { } // TODO: optimize for parse huge log-file. -func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { - rows := make([]map[string]types.Datum, 0) - rowMap := make(map[string]types.Datum, len(slowQueryCols)) +func parseSlowLog(scanner *bufio.Scanner) ([][]types.Datum, error) { + var rows [][]types.Datum startFlag := false - + var st *slowQueryTuple + var err error for scanner.Scan() { line := scanner.Text() // Check slow log entry start flag. if !startFlag && strings.HasPrefix(line, variable.SlowLogStartPrefixStr) { - value, err := parseSlowLogField(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) + st = &slowQueryTuple{} + err = st.setFieldValue(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) if err != nil { return rows, err } - rowMap[variable.SlowLogTimeStr] = *value startFlag = true continue } @@ -114,65 +98,144 @@ func parseSlowLog(scanner *bufio.Scanner) ([]map[string]types.Datum, error) { if strings.HasSuffix(field, ":") { field = field[:len(field)-1] } - value, err := parseSlowLogField(field, fieldValues[i+1]) + err = st.setFieldValue(field, fieldValues[i+1]) if err != nil { return rows, err } - rowMap[field] = *value - } } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { // Get the sql string, and mark the start flag to false. - rowMap[variable.SlowLogQuerySQLStr] = types.NewStringDatum(string(hack.Slice(line))) - rows = append(rows, rowMap) - rowMap = make(map[string]types.Datum, len(slowQueryCols)) + err = st.setFieldValue(variable.SlowLogQuerySQLStr, string(hack.Slice(line))) + if err != nil { + return rows, err + } + rows = append(rows, st.convertToDatumRow()) startFlag = false } } } - if err := scanner.Err(); err != nil { return nil, errors.AddStack(err) } return rows, nil } -func parseSlowLogField(field, value string) (*types.Datum, error) { - col := findColumnByName(slowQueryCols, field) - if col == nil { - return nil, errors.Errorf("can't found column %v", field) - } - var val types.Datum - switch col.tp { - case mysql.TypeLonglong: +type slowQueryTuple struct { + time time.Time + txnStartTs uint64 + user string + connID uint64 + queryTime float64 + processTime float64 + waitTime float64 + backOffTime float64 + requestCount uint64 + totalKeys uint64 + processKeys uint64 + db string + indexNames string + isInternal bool + sql string +} + +func (st *slowQueryTuple) setFieldValue(field, value string) error { + switch field { + case variable.SlowLogTimeStr: + t, err := parseTime(value) + if err != nil { + return err + } + st.time = t + case variable.SlowLogTxnStartTSStr: + num, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return errors.AddStack(err) + } + st.txnStartTs = num + case variable.SlowLogUserStr: + st.user = value + case variable.SlowLogConnIDStr: num, err := strconv.ParseUint(value, 10, 64) if err != nil { - return nil, errors.AddStack(err) + return errors.AddStack(err) } - val = types.NewUintDatum(num) - case mysql.TypeVarchar: - val = types.NewStringDatum(value) - case mysql.TypeDouble: + st.connID = num + case variable.SlowLogQueryTimeStr: num, err := strconv.ParseFloat(value, 64) if err != nil { - return nil, errors.AddStack(err) + return errors.AddStack(err) } - val = types.NewDatum(num) - case mysql.TypeTiny: - // parse bool - val = types.NewDatum(value == "true") - case mysql.TypeDatetime: - t, err := parseTime(value) + st.queryTime = num + case execdetails.ProcessTimeStr: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.processTime = num + case execdetails.WaitTimeStr: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.waitTime = num + case execdetails.BackoffTimeStr: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.backOffTime = num + case execdetails.RequestCountStr: + num, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return errors.AddStack(err) + } + st.requestCount = num + case execdetails.TotalKeysStr: + num, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return errors.AddStack(err) + } + st.totalKeys = num + case execdetails.ProcessKeysStr: + num, err := strconv.ParseUint(value, 10, 64) if err != nil { - return nil, err + return errors.AddStack(err) } - val = types.NewTimeDatum(types.Time{ - Time: types.FromGoTime(t), - Type: mysql.TypeDatetime, - Fsp: types.MaxFsp, - }) + st.processKeys = num + case variable.SlowLogDBStr: + st.db = value + case variable.SlowLogIndexNamesStr: + st.indexNames = value + case variable.SlowLogIsInternalStr: + st.isInternal = value == "true" + case variable.SlowLogQuerySQLStr: + st.sql = value } - return &val, nil + return nil +} + +func (st *slowQueryTuple) convertToDatumRow() []types.Datum { + record := make([]types.Datum, 0, len(slowQueryCols)) + record = append(record, types.NewTimeDatum(types.Time{ + Time: types.FromGoTime(st.time), + Type: mysql.TypeDatetime, + Fsp: types.MaxFsp, + })) + record = append(record, types.NewUintDatum(st.txnStartTs)) + 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.processTime)) + record = append(record, types.NewFloat64Datum(st.waitTime)) + record = append(record, types.NewFloat64Datum(st.backOffTime)) + record = append(record, types.NewUintDatum(st.requestCount)) + record = append(record, types.NewUintDatum(st.totalKeys)) + record = append(record, types.NewUintDatum(st.processKeys)) + record = append(record, types.NewStringDatum(st.db)) + record = append(record, types.NewStringDatum(st.indexNames)) + record = append(record, types.NewDatum(st.isInternal)) + record = append(record, types.NewStringDatum(st.sql)) + return record } func parseTime(s string) (time.Time, error) { @@ -182,12 +245,3 @@ func parseTime(s string) (time.Time, error) { } return t, err } - -func findColumnByName(cols []columnInfo, colName string) *columnInfo { - for _, col := range cols { - if col.name == colName { - return &col - } - } - return nil -} diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 9e28705fe6ec3..f2fb1db12ce47 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -17,9 +17,6 @@ import ( "bufio" "bytes" "testing" - - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/types" ) func TestParseSlowLogFile(t *testing.T) { @@ -38,37 +35,19 @@ select * from t;`) if len(rows) != 1 { t.Fatalf("parse slow log failed") } - row := rows[0] - t1 := types.Time{ - Time: types.FromDate(2019, 01, 24, 22, 32, 29, 313255), - Type: mysql.TypeDatetime, - Fsp: types.MaxFsp, - } - if logTime, ok := row["Time"]; !ok || logTime.GetMysqlTime() != t1 { - t.Fatalf("parse slow log failed") - } - if ts, ok := row["Txn_start_ts"]; !ok || ts.GetUint64() != 405888132465033227 { - t.Fatalf("parse slow log failed") - } - if queryTime, ok := row["Query_time"]; !ok || queryTime.GetFloat64() != 0.216905 { - t.Fatalf("parse slow log failed") - } - if ProcessTime, ok := row["Process_time"]; !ok || ProcessTime.GetFloat64() != 0.021 { - t.Fatalf("parse slow log failed") - } - if requestCount, ok := row["Request_count"]; !ok || requestCount.GetUint64() != 1 { - t.Fatalf("parse slow log failed") - } - if totalKeys, ok := row["Total_keys"]; !ok || totalKeys.GetUint64() != 637 { - t.Fatalf("parse slow log failed") - } - if processedKeys, ok := row["Processed_keys"]; !ok || processedKeys.GetUint64() != 436 { - t.Fatalf("parse slow log failed") - } - if isInternal, ok := row["Is_internal"]; !ok || isInternal.GetInt64() != 1 { - t.Fatalf("parse slow log failed") - } - if sql, ok := row["Query"]; !ok || sql.GetString() != "select * from t;" { - t.Fatalf("parse slow log failed") + recordString := "" + for i, value := range rows[0] { + str, err := value.ToString() + if err != nil { + t.Fatalf("parse slow log failed") + } + if i > 0 { + recordString += "," + } + recordString += str + } + expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,select * from t;" + if expectRecordString != recordString { + t.Fatalf("parse slow log failed, expect: %v\nbut got: %v\n", expectRecordString, recordString) } } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 7cad006b0fb01..7652422b8a28c 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -253,13 +253,14 @@ func (s *testSuite) TestSlowQuery(c *C) { # User: root@127.0.0.1 # Conn_ID: 6 # Query_time: 4.895492 -# Process_time: 0.161 Request_count: 1 Total_keys: 100001 Processed_keys: 100000 +# Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 # DB: test # Is_internal: false select * from t_slim;`)) c.Assert(f.Close(), IsNil) + c.Assert(err, IsNil) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) 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|||1|100001|100000|test|0||select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0|0|1|100001|100000|test||0|select * from t_slim;")) } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 2f09c1438b8cc..c6465f78cc9ea 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -107,7 +107,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # User: root@192.168.0.1 # Conn_ID: 1 # Query_time: 1 -# Process_time: 2 Wait_time: 60 Backoff_time: 0.001 Request_count: 2 Total_keys: 10000 Processed_keys: 20001 +# Process_time: 2 Wait_time: 60 Backoff_time: 0.001 Request_count: 2 Total_keys: 10000 Process_keys: 20001 # DB: test # Index_names: idx1,idx2 # Is_internal: true diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 89826019d63c5..e0cdb66b04f05 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -16,6 +16,7 @@ package execdetails import ( "fmt" "sort" + "strconv" "strings" "sync" "sync/atomic" @@ -64,30 +65,30 @@ const ( RequestCountStr = "Request_count" // TotalKeysStr means the total scan keys. TotalKeysStr = "Total_keys" - // ProcessedKeysStr means the total processed keys. - ProcessedKeysStr = "Processed_keys" + // ProcessKeysStr means the total processed keys. + ProcessKeysStr = "Process_keys" ) // String implements the fmt.Stringer interface. func (d ExecDetails) String() string { parts := make([]string, 0, 6) if d.ProcessTime > 0 { - parts = append(parts, fmt.Sprintf("Process_time: %v", d.ProcessTime.Seconds())) + parts = append(parts, ProcessTimeStr+": "+strconv.FormatFloat(d.ProcessTime.Seconds(), 'f', -1, 64)) } if d.WaitTime > 0 { - parts = append(parts, fmt.Sprintf("Wait_time: %v", d.WaitTime.Seconds())) + parts = append(parts, WaitTimeStr+": "+strconv.FormatFloat(d.WaitTime.Seconds(), 'f', -1, 64)) } if d.BackoffTime > 0 { - parts = append(parts, fmt.Sprintf("Backoff_time: %v", d.BackoffTime.Seconds())) + parts = append(parts, BackoffTimeStr+": "+strconv.FormatFloat(d.BackoffTime.Seconds(), 'f', -1, 64)) } if d.RequestCount > 0 { - parts = append(parts, fmt.Sprintf("Request_count: %d", d.RequestCount)) + parts = append(parts, RequestCountStr+": "+strconv.FormatInt(int64(d.RequestCount), 10)) } if d.TotalKeys > 0 { - parts = append(parts, fmt.Sprintf("Total_keys: %d", d.TotalKeys)) + parts = append(parts, TotalKeysStr+": "+strconv.FormatInt(d.TotalKeys, 10)) } if d.ProcessedKeys > 0 { - parts = append(parts, fmt.Sprintf("Processed_keys: %d", d.ProcessedKeys)) + parts = append(parts, ProcessKeysStr+": "+strconv.FormatInt(d.ProcessedKeys, 10)) } commitDetails := d.CommitDetail if commitDetails != nil { diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 79502601c9b65..4ab867983163c 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -41,7 +41,7 @@ func TestString(t *testing.T) { TxnRetry: 1, }, } - expected := "Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Total_keys: 100 Processed_keys: 10 Prewrite_time: 1 Commit_time: 1 " + + expected := "Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Total_keys: 100 Process_keys: 10 Prewrite_time: 1 Commit_time: 1 " + "Get_commit_ts_time: 1 Total_backoff_time: 1 Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1" if str := detail.String(); str != expected { t.Errorf("got:\n%s\nexpected:\n%s", str, expected) From d8aa9482814abbc8be3566e71523b3c6280c554d Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 11 Mar 2019 11:13:50 +0800 Subject: [PATCH 30/33] add test for parse time --- infoschema/slow_log_test.go | 21 +++++++++++++++++++++ util/logutil/log.go | 2 +- 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index f2fb1db12ce47..13ebeb237ccda 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -17,6 +17,9 @@ import ( "bufio" "bytes" "testing" + "time" + + "github.com/pingcap/tidb/util/logutil" ) func TestParseSlowLogFile(t *testing.T) { @@ -51,3 +54,21 @@ select * from t;`) t.Fatalf("parse slow log failed, expect: %v\nbut got: %v\n", expectRecordString, recordString) } } + +func TestParseTime(t *testing.T) { + t1Str := "2019-01-24-22:32:29.313255 +0800" + t2Str := "2019-01-24-22:32:29.313255" + t1, err := parseTime(t1Str) + loc, err := time.LoadLocation("Asia/Shanghai") + if err != nil { + t.Fatalf("parse time failed") + } + t2, err := time.ParseInLocation("2006-01-02-15:04:05.999999999", t2Str, loc) + if t1.Unix() != t2.Unix() { + t.Fatalf("parse time failed, %v != %v", t1.Unix(), t2.Unix()) + } + t1Format := t1.In(loc).Format(logutil.SlowLogTimeFormat) + if t1Format != t1Str { + t.Fatalf("parse time failed, %v != %v", t1Format, t1Str) + } +} diff --git a/util/logutil/log.go b/util/logutil/log.go index 66a72c526a41a..eaaa2f6d1ae95 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -207,7 +207,7 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { const ( // SlowLogTimeFormat is the time format for slow log. - SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 +0800" + SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" ) type slowLogFormatter struct{} From 0b8d096c6fe4da9af49db7dd17ba6a78001e4e2b Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 11 Mar 2019 11:56:15 +0800 Subject: [PATCH 31/33] add test for slow log convert timezone --- infoschema/slow_log.go | 27 +++++++++++++--------- infoschema/slow_log_test.go | 45 +++++++++++++++---------------------- infoschema/tables_test.go | 4 ++++ 3 files changed, 38 insertions(+), 38 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 2d31b080d9679..c2473996485fd 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -32,7 +32,7 @@ import ( ) var slowQueryCols = []columnInfo{ - {variable.SlowLogTimeStr, mysql.TypeDatetime, -1, 0, nil, nil}, + {variable.SlowLogTimeStr, mysql.TypeTimestamp, -1, 0, nil, nil}, {variable.SlowLogTxnStartTSStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogUserStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogConnIDStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, @@ -50,12 +50,12 @@ var slowQueryCols = []columnInfo{ } func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { - return parseSlowLogFile(ctx.GetSessionVars().SlowQueryFile) + return parseSlowLogFile(ctx.GetSessionVars().Location(), ctx.GetSessionVars().SlowQueryFile) } // TODO: Support parse multiple log-files. // parseSlowLogFile uses to parse slow log file. -func parseSlowLogFile(filePath string) ([][]types.Datum, error) { +func parseSlowLogFile(tz *time.Location, filePath string) ([][]types.Datum, error) { file, err := os.Open(filePath) if err != nil { return nil, errors.Trace(err) @@ -66,11 +66,12 @@ func parseSlowLogFile(filePath string) ([][]types.Datum, error) { } }() - return parseSlowLog(bufio.NewScanner(file)) + return ParseSlowLog(tz, bufio.NewScanner(file)) } +// ParseSlowLog exports for testing. // TODO: optimize for parse huge log-file. -func parseSlowLog(scanner *bufio.Scanner) ([][]types.Datum, error) { +func ParseSlowLog(tz *time.Location, scanner *bufio.Scanner) ([][]types.Datum, error) { var rows [][]types.Datum startFlag := false var st *slowQueryTuple @@ -80,7 +81,7 @@ func parseSlowLog(scanner *bufio.Scanner) ([][]types.Datum, error) { // Check slow log entry start flag. if !startFlag && strings.HasPrefix(line, variable.SlowLogStartPrefixStr) { st = &slowQueryTuple{} - err = st.setFieldValue(variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) + err = st.setFieldValue(tz, variable.SlowLogTimeStr, line[len(variable.SlowLogStartPrefixStr):]) if err != nil { return rows, err } @@ -98,14 +99,14 @@ func parseSlowLog(scanner *bufio.Scanner) ([][]types.Datum, error) { if strings.HasSuffix(field, ":") { field = field[:len(field)-1] } - err = st.setFieldValue(field, fieldValues[i+1]) + err = st.setFieldValue(tz, field, fieldValues[i+1]) if err != nil { return rows, err } } } else if strings.HasSuffix(line, variable.SlowLogSQLSuffixStr) { // Get the sql string, and mark the start flag to false. - err = st.setFieldValue(variable.SlowLogQuerySQLStr, string(hack.Slice(line))) + err = st.setFieldValue(tz, variable.SlowLogQuerySQLStr, string(hack.Slice(line))) if err != nil { return rows, err } @@ -138,13 +139,16 @@ type slowQueryTuple struct { sql string } -func (st *slowQueryTuple) setFieldValue(field, value string) error { +func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) error { switch field { case variable.SlowLogTimeStr: - t, err := parseTime(value) + t, err := ParseTime(value) if err != nil { return err } + if t.Location() != tz { + t = t.In(tz) + } st.time = t case variable.SlowLogTxnStartTSStr: num, err := strconv.ParseUint(value, 10, 64) @@ -238,7 +242,8 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { return record } -func parseTime(s string) (time.Time, error) { +// ParseTime exports for testing. +func ParseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) if err != nil { err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 13ebeb237ccda..462b684692f85 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -11,18 +11,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -package infoschema +package infoschema_test import ( "bufio" "bytes" - "testing" "time" + . "github.com/pingcap/check" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/util/logutil" ) -func TestParseSlowLogFile(t *testing.T) { +func (s *testSuite) TestParseSlowLogFile(c *C) { slowLog := bytes.NewBufferString( `# Time: 2019-01-24-22:32:29.313255 +0800 # Txn_start_ts: 405888132465033227 @@ -31,44 +32,34 @@ func TestParseSlowLogFile(t *testing.T) { # Is_internal: true select * from t;`) scanner := bufio.NewScanner(slowLog) - rows, err := parseSlowLog(scanner) - if err != nil { - t.Fatalf("parse slow log failed") - } - if len(rows) != 1 { - t.Fatalf("parse slow log failed") - } + loc, err := time.LoadLocation("Asia/Shanghai") + c.Assert(err, IsNil) + rows, err := infoschema.ParseSlowLog(loc, scanner) + c.Assert(err, IsNil) + c.Assert(len(rows), Equals, 1) recordString := "" for i, value := range rows[0] { str, err := value.ToString() - if err != nil { - t.Fatalf("parse slow log failed") - } + c.Assert(err, IsNil) if i > 0 { recordString += "," } recordString += str } expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,select * from t;" - if expectRecordString != recordString { - t.Fatalf("parse slow log failed, expect: %v\nbut got: %v\n", expectRecordString, recordString) - } + c.Assert(expectRecordString, Equals, recordString) } -func TestParseTime(t *testing.T) { +func (s *testSuite) TestSlowLogParseTime(c *C) { t1Str := "2019-01-24-22:32:29.313255 +0800" t2Str := "2019-01-24-22:32:29.313255" - t1, err := parseTime(t1Str) + t1, err := infoschema.ParseTime(t1Str) + c.Assert(err, IsNil) loc, err := time.LoadLocation("Asia/Shanghai") - if err != nil { - t.Fatalf("parse time failed") - } + c.Assert(err, IsNil) t2, err := time.ParseInLocation("2006-01-02-15:04:05.999999999", t2Str, loc) - if t1.Unix() != t2.Unix() { - t.Fatalf("parse time failed, %v != %v", t1.Unix(), t2.Unix()) - } + c.Assert(err, IsNil) + c.Assert(t1.Unix(), Equals, t2.Unix()) t1Format := t1.In(loc).Format(logutil.SlowLogTimeFormat) - if t1Format != t1Str { - t.Fatalf("parse time failed, %v != %v", t1Format, t1Str) - } + c.Assert(t1Format, Equals, t1Str) } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 7652422b8a28c..7a32c9789142d 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -261,6 +261,10 @@ select * from t_slim;`)) c.Assert(err, IsNil) 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@127.0.0.1|6|4.895492|0.161|0|0|1|100001|100000|test||0|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|0|1|100001|100000|test||0|select * from t_slim;")) } From c0993c0c2ee5a8aa1136270080aa26cdcd082da7 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 11 Mar 2019 12:42:40 +0800 Subject: [PATCH 32/33] change slow log time type to timestamp --- infoschema/slow_log.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index c2473996485fd..766932759318e 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -32,7 +32,7 @@ import ( ) var slowQueryCols = []columnInfo{ - {variable.SlowLogTimeStr, mysql.TypeTimestamp, -1, 0, nil, nil}, + {variable.SlowLogTimeStr, mysql.TypeTimestamp, 26, 0, nil, nil}, {variable.SlowLogTxnStartTSStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogUserStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogConnIDStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, From 9161ac7bb4d63dbdd62884dda7e1d952bb5ea0ea Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 11 Mar 2019 13:36:01 +0800 Subject: [PATCH 33/33] address comment --- executor/adapter.go | 4 ++-- infoschema/slow_log.go | 6 +++--- sessionctx/variable/session.go | 18 ++++++++++-------- sessionctx/variable/session_test.go | 5 ++--- util/execdetails/execdetails.go | 2 +- 5 files changed, 18 insertions(+), 17 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 05cdf76082714..0c19f0410a3d7 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -381,10 +381,10 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { var tableIDs, indexIDs string if len(sessVars.StmtCtx.TableIDs) > 0 { - tableIDs = strings.Replace(fmt.Sprintf("table_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.TableIDs), " ", ",", -1) + tableIDs = strings.Replace(fmt.Sprintf("%v", a.Ctx.GetSessionVars().StmtCtx.TableIDs), " ", ",", -1) } if len(sessVars.StmtCtx.IndexIDs) > 0 { - indexIDs = strings.Replace(fmt.Sprintf("index_ids:%v ", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) + indexIDs = strings.Replace(fmt.Sprintf("%v", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 766932759318e..292a7f02314a4 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -44,7 +44,7 @@ var slowQueryCols = []columnInfo{ {execdetails.TotalKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {execdetails.ProcessKeysStr, mysql.TypeLonglong, 20, mysql.UnsignedFlag, nil, nil}, {variable.SlowLogDBStr, mysql.TypeVarchar, 64, 0, nil, nil}, - {variable.SlowLogIndexNamesStr, mysql.TypeVarchar, 640, 0, nil, nil}, + {variable.SlowLogIndexIDsStr, mysql.TypeVarchar, 100, 0, nil, nil}, {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -53,8 +53,8 @@ func dataForSlowLog(ctx sessionctx.Context) ([][]types.Datum, error) { return parseSlowLogFile(ctx.GetSessionVars().Location(), ctx.GetSessionVars().SlowQueryFile) } -// TODO: Support parse multiple log-files. // parseSlowLogFile uses to parse slow log file. +// TODO: Support parse multiple log-files. func parseSlowLogFile(tz *time.Location, filePath string) ([][]types.Datum, error) { file, err := os.Open(filePath) if err != nil { @@ -208,7 +208,7 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.processKeys = num case variable.SlowLogDBStr: st.db = value - case variable.SlowLogIndexNamesStr: + case variable.SlowLogIndexIDsStr: st.indexNames = value case variable.SlowLogIsInternalStr: st.isInternal = value == "true" diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a62c2a935b855..aa322eaf04656 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -837,8 +837,8 @@ const ( SlowLogDBStr = "DB" // SlowLogIsInternalStr is slow log field name. SlowLogIsInternalStr = "Is_internal" - // SlowLogIndexNamesStr is slow log field name. - SlowLogIndexNamesStr = "Index_names" + // SlowLogIndexIDsStr is slow log field name. + SlowLogIndexIDsStr = "Index_ids" // SlowLogQuerySQLStr is slow log field name. SlowLogQuerySQLStr = "Query" // use for slow log table, slow log will not print this field name but print sql directly. ) @@ -852,6 +852,7 @@ const ( // # Query_time: 4.895492 // # Process_time: 0.161 Request_count: 1 Total_keys: 100001 Processed_keys: 100000 // # DB: test +// # Index_ids: [1,2] // # Is_internal: false // select * from t_slim; func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, sql string) string { @@ -872,14 +873,15 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe buf.WriteString(SlowLogPrefixStr + SlowLogDBStr + SlowLogSpaceMarkStr + s.CurrentDB + "\n") } if len(indexIDs) > 0 { - buf.WriteString(SlowLogPrefixStr + SlowLogIndexNamesStr + SlowLogSpaceMarkStr + indexIDs + "\n") + buf.WriteString(SlowLogPrefixStr + SlowLogIndexIDsStr + SlowLogSpaceMarkStr + indexIDs + "\n") } buf.WriteString(SlowLogPrefixStr + SlowLogIsInternalStr + SlowLogSpaceMarkStr + strconv.FormatBool(s.InRestrictedSQL) + "\n") - if len(sql) > 0 { - buf.WriteString(sql) - if sql[len(sql)-1] != ';' { - buf.WriteString(";") - } + if len(sql) == 0 { + sql = ";" + } + buf.WriteString(sql) + if sql[len(sql)-1] != ';' { + buf.WriteString(";") } return buf.String() } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index c6465f78cc9ea..35520ff6fd7c3 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -109,10 +109,9 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # Query_time: 1 # Process_time: 2 Wait_time: 60 Backoff_time: 0.001 Request_count: 2 Total_keys: 10000 Process_keys: 20001 # DB: test -# Index_names: idx1,idx2 +# Index_ids: [1,2] # Is_internal: true select * from t;` - logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "idx1,idx2", "select * from t") + logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", "select * from t") c.Assert(logString, Equals, resultString) - } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index e0cdb66b04f05..07a6e8a9090f5 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -55,7 +55,7 @@ type CommitDetails struct { } const ( - // ProcessTimeStr represents the sum of process time of all the coprocessor tasks + // ProcessTimeStr represents the sum of process time of all the coprocessor tasks. ProcessTimeStr = "Process_time" // WaitTimeStr means the time of all coprocessor wait. WaitTimeStr = "Wait_time"