From a4707e23bc3c8708a56808ace2bb4dc40535e534 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 11 Apr 2019 16:07:19 +0800 Subject: [PATCH] log: generate digest and log it in slow log (#9662) (#10093) --- domain/topn_slow_query.go | 1 + executor/adapter.go | 7 +++-- executor/aggregate_test.go | 2 +- executor/executor.go | 5 ++++ executor/prepared.go | 1 + infoschema/slow_log.go | 5 ++++ infoschema/slow_log_test.go | 3 +- infoschema/tables_test.go | 5 ++-- planner/core/planbuilder.go | 1 + sessionctx/stmtctx/stmtctx.go | 16 ++++++++++ sessionctx/variable/session.go | 7 ++++- sessionctx/variable/session_test.go | 6 +++- store/tikv/tikv_test.go | 45 ----------------------------- 13 files changed, 51 insertions(+), 53 deletions(-) diff --git a/domain/topn_slow_query.go b/domain/topn_slow_query.go index 0bf6721454d6c..4b5b4d51c1438 100644 --- a/domain/topn_slow_query.go +++ b/domain/topn_slow_query.go @@ -222,4 +222,5 @@ type SlowQueryInfo struct { TableIDs string IndexIDs string Internal bool + Digest string } diff --git a/executor/adapter.go b/executor/adapter.go index 6f39c915222e7..72c4ba29056b1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -403,9 +403,11 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { - logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) + _, digest := sessVars.StmtCtx.SQLDigest() + logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) } else { - logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, sql)) + _, digest := sessVars.StmtCtx.SQLDigest() + logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -415,6 +417,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{ SQL: sql, + Digest: digest, Start: a.StartTime, Duration: costTime, Detail: sessVars.StmtCtx.GetExecDetails(), diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 88b3f013b9566..c4425e5e43160 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -239,7 +239,7 @@ func (s *testSuite) TestAggregation(c *C) { result = tk.MustQuery("select count(*) from information_schema.columns") // When adding new memory columns in information_schema, please update this variable. - columnCountOfAllInformationSchemaTables := "784" + columnCountOfAllInformationSchemaTables := "785" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") diff --git a/executor/executor.go b/executor/executor.go index bef9955dbb685..fe654e1cd74b5 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -600,6 +600,7 @@ func (e *ShowSlowExec) Next(ctx context.Context, chk *chunk.Chunk) error { } else { chk.AppendInt64(11, 1) } + chk.AppendString(12, slow.Digest) e.cursor++ } return nil @@ -1359,6 +1360,10 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { if err != nil { return errors.Trace(err) } + if s != nil { + // execute missed stmtID uses empty sql + sc.OriginalSQL = s.Text() + } vars.StmtCtx = sc return } diff --git a/executor/prepared.go b/executor/prepared.go index 4cd28f1c7679d..9675ff211b692 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -271,6 +271,7 @@ func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...inter } if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID]; ok { stmt.Text = prepared.Stmt.Text() + ctx.GetSessionVars().StmtCtx.OriginalSQL = stmt.Text } return stmt, nil } diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 292a7f02314a4..ea8ab80fa9183 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -46,6 +46,7 @@ var slowQueryCols = []columnInfo{ {variable.SlowLogDBStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogIndexIDsStr, mysql.TypeVarchar, 100, 0, nil, nil}, {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, + {variable.SlowLogDigestStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -136,6 +137,7 @@ type slowQueryTuple struct { db string indexNames string isInternal bool + digest string sql string } @@ -212,6 +214,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.indexNames = value case variable.SlowLogIsInternalStr: st.isInternal = value == "true" + case variable.SlowLogDigestStr: + st.digest = value case variable.SlowLogQuerySQLStr: st.sql = value } @@ -238,6 +242,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { 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.digest)) record = append(record, types.NewStringDatum(st.sql)) return record } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 462b684692f85..00f5228ac5d70 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -30,6 +30,7 @@ func (s *testSuite) TestParseSlowLogFile(c *C) { # Query_time: 0.216905 # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true +# Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 select * from t;`) scanner := bufio.NewScanner(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") @@ -46,7 +47,7 @@ select * from t;`) } 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;" + expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,select * from t;" c.Assert(expectRecordString, Equals, recordString) } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index de73b11d116d5..4e744d40dea3a 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -304,6 +304,7 @@ func (s *testSuite) TestSlowQuery(c *C) { # Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 # DB: test # Is_internal: false +# Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 select * from t_slim;`)) c.Assert(f.Close(), IsNil) c.Assert(err, IsNil) @@ -311,8 +312,8 @@ select * from t_slim;`)) 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;")) + 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|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|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;")) + 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|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|select * from t_slim;")) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index fb43eb371e802..6ea53a28e9293 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -861,6 +861,7 @@ func buildShowSlowSchema() *expression.Schema { schema.Append(buildColumn("", "TABLE_IDS", mysql.TypeVarchar, 256)) schema.Append(buildColumn("", "INDEX_IDS", mysql.TypeVarchar, 256)) schema.Append(buildColumn("", "INTERNAL", mysql.TypeTiny, tinySize)) + schema.Append(buildColumn("", "DIGEST", mysql.TypeVarchar, 64)) return schema } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index fd014bc0a9fec..493d2c9a72ffc 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -18,6 +18,7 @@ import ( "sync" "time" + "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" @@ -93,6 +94,21 @@ type StatementContext struct { IndexIDs []int64 StmtType string Tables []TableEntry + OriginalSQL string + digestMemo struct { + sync.Once + normalized string + digest string + } +} + +// SQLDigest gets normalized and digest for provided sql. +// it will cache result after first calling. +func (sc *StatementContext) SQLDigest() (normalized, sqlDigest string) { + sc.digestMemo.Do(func() { + sc.digestMemo.normalized, sc.digestMemo.digest = parser.NormalizeDigest(sc.OriginalSQL) + }) + return sc.digestMemo.normalized, sc.digestMemo.digest } // TableEntry presents table in db. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 3c90258dda4cb..7fa00ae28afff 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -775,6 +775,8 @@ const ( SlowLogIsInternalStr = "Is_internal" // SlowLogIndexIDsStr is slow log field name. SlowLogIndexIDsStr = "Index_ids" + // SlowLogDigestStr is slow log field name. + SlowLogDigestStr = "Digest" // 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. ) @@ -791,7 +793,7 @@ const ( // # 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 { +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") @@ -812,6 +814,9 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe buf.WriteString(SlowLogPrefixStr + SlowLogIndexIDsStr + SlowLogSpaceMarkStr + indexIDs + "\n") } buf.WriteString(SlowLogPrefixStr + SlowLogIsInternalStr + SlowLogSpaceMarkStr + strconv.FormatBool(s.InRestrictedSQL) + "\n") + if len(digest) > 0 { + buf.WriteString(SlowLogPrefixStr + SlowLogDigestStr + SlowLogSpaceMarkStr + digest + "\n") + } if len(sql) == 0 { sql = ";" } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index ed3273f3aad37..4a7d9a29834a6 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -17,6 +17,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" @@ -83,7 +84,10 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # DB: test # Index_ids: [1,2] # Is_internal: true +# Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 select * from t;` - logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", "select * from t") + sql := "select * from t" + digest := parser.DigestHash(sql) + logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, sql) c.Assert(logString, Equals, resultString) } diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go index a19a7d941075f..a4db0b7df60ec 100644 --- a/store/tikv/tikv_test.go +++ b/store/tikv/tikv_test.go @@ -14,14 +14,7 @@ package tikv import ( - "go/build" - "os" - "path" - "reflect" - "strings" - . "github.com/pingcap/check" - "github.com/pingcap/parser" ) // OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. @@ -45,41 +38,3 @@ type testTiKVSuite struct { } var _ = Suite(&testTiKVSuite{}) - -func getImportedPackages(c *C, srcDir string, pkgName string, pkgs *map[string][]string) { - if pkgName == "C" { - return - } - if _, exists := (*pkgs)[pkgName]; exists { - return - } - if strings.HasPrefix(pkgName, "golang_org") { - pkgName = path.Join("vendor", pkgName) - } - pkg, err := build.Import(pkgName, srcDir, 0) - c.Assert(err, IsNil) - (*pkgs)[pkgName] = pkg.Imports - for _, name := range (*pkgs)[pkgName] { - getImportedPackages(c, srcDir, name, pkgs) - } -} - -// TestParserNoDep tests whether this package does not depend on tidb/parser. -func (s *testTiKVSuite) TestParserNoDep(c *C) { - srcDir, err := os.Getwd() - c.Assert(err, IsNil) - - pkgs := make(map[string][]string) - currentPkgName := reflect.TypeOf(testTiKVSuite{}).PkgPath() - getImportedPackages(c, srcDir, currentPkgName, &pkgs) - - parse := parser.New() - parserPkgName := reflect.TypeOf(*parse).PkgPath() - - for pkgName, imports := range pkgs { - for _, importName := range imports { - c.Assert(importName == parserPkgName, IsFalse, - Commentf("`%s` is imported from `%s`, which is a child dependency of `%s`", parserPkgName, pkgName, currentPkgName)) - } - } -}