From 9443af8fb64bd9519792df8691a90e9907582135 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 15 Apr 2019 18:04:48 +0800 Subject: [PATCH 1/5] *: add stats version in slow_query and slow log. (#10082) --- cmd/explaintest/r/explain_easy.result | 6 ++--- cmd/explaintest/r/explain_easy_stats.result | 10 ++++---- cmd/explaintest/r/topn_push_down.result | 2 +- executor/adapter.go | 27 +++++++++++++++++++-- infoschema/slow_log.go | 5 ++++ infoschema/slow_log_test.go | 3 ++- infoschema/tables_test.go | 5 ++-- planner/core/exhaust_physical_plans.go | 11 +++++++-- planner/core/explain.go | 5 ++-- planner/core/find_best_task.go | 13 ++++++++-- planner/core/physical_plans.go | 24 ++++++++++++++++++ planner/core/stats.go | 13 +++++++--- planner/property/stats_info.go | 16 ++++++------ sessionctx/variable/session.go | 26 +++++++++++++++++++- sessionctx/variable/session_test.go | 5 +++- statistics/table.go | 7 ++++-- 16 files changed, 142 insertions(+), 36 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index c9b307b1c9bf5..abbd582403637 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -78,7 +78,7 @@ IndexLookUp_12 1.11 root ├─Selection_10 3.33 cop gt(test.t1.c1, 1) │ └─IndexScan_8 10.00 cop table:t1, index:c2, range:[1,1], keep order:false, stats:pseudo └─Selection_11 1.11 cop lt(test.t1.c3, 1) - └─TableScan_9 3.33 cop table:t1, keep order:false + └─TableScan_9 3.33 cop table:t1, keep order:false, stats:pseudo explain select * from t1 where c1 = 1 and c2 > 1; id count task operator info TableReader_7 0.33 root data:Selection_6 @@ -118,7 +118,7 @@ Projection_12 10000.00 root eq(test.t1.c2, test.t2.c2) └─IndexLookUp_43 1.00 root ├─Limit_42 1.00 cop offset:0, count:1 │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo - └─TableScan_41 1.00 cop table:t2, keep order:false + └─TableScan_41 1.00 cop table:t2, keep order:false, stats:pseudo explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -133,7 +133,7 @@ Limit_9 1.00 root offset:0, count:1 │ └─IndexScan_12 9.00 cop table:t4, index:a, b, range:(1 +inf,+inf +inf], keep order:false, stats:pseudo └─Limit_16 1.00 cop offset:0, count:1 └─Selection_15 1.00 cop gt(test.t4.c, 1) - └─TableScan_13 3.00 cop table:t4, keep order:false + └─TableScan_13 3.00 cop table:t4, keep order:false, stats:pseudo explain select * from t4 where a > 1 and c > 1 limit 1; id count task operator info Limit_8 1.00 root offset:0, count:1 diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 5d3bb798cda86..6e05d521f9033 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -50,7 +50,7 @@ Projection_6 2481.25 root test.t1.c1, test.t1.c2, test.t1.c3, test.t2.c1, test.t ├─IndexLookUp_17 1998.00 root │ ├─Selection_16 1998.00 cop gt(test.t1.c1, 1) │ │ └─IndexScan_14 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true - │ └─TableScan_15 1998.00 cop table:t1, keep order:false + │ └─TableScan_15 1998.00 cop table:t1, keep order:false, stats:pseudo └─IndexLookUp_21 1985.00 root ├─IndexScan_19 1985.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true └─TableScan_20 1985.00 cop table:t2, keep order:false @@ -84,7 +84,7 @@ IndexLookUp_12 0.00 root ├─Selection_10 0.00 cop gt(test.t1.c1, 1) │ └─IndexScan_8 0.00 cop table:t1, index:c2, range:[1,1], keep order:false └─Selection_11 0.00 cop lt(test.t1.c3, 1) - └─TableScan_9 0.00 cop table:t1, keep order:false + └─TableScan_9 0.00 cop table:t1, keep order:false, stats:pseudo explain select * from t1 where c1 = 1 and c2 > 1; id count task operator info TableReader_7 0.50 root data:Selection_6 @@ -106,7 +106,7 @@ Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2) └─IndexLookUp_43 1.00 root ├─Limit_42 1.00 cop offset:0, count:1 │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true - └─TableScan_41 1.00 cop table:t2, keep order:false + └─TableScan_41 1.00 cop table:t2, keep order:false, stats:pseudo explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -163,14 +163,14 @@ Limit_9 1.00 root offset:1, count:1 └─IndexLookUp_15 1.00 root ├─Limit_14 1.00 cop offset:0, count:2 │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_13 1.00 cop table:index_prune, keep order:false + └─TableScan_13 1.00 cop table:index_prune, keep order:false, stats:pseudo explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1, 0; id count task operator info Limit_9 0.00 root offset:1, count:0 └─IndexLookUp_15 0.00 root ├─Limit_14 0.00 cop offset:0, count:1 │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_13 0.00 cop table:index_prune, keep order:false + └─TableScan_13 0.00 cop table:index_prune, keep order:false, stats:pseudo explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 0, 1; id count task operator info Point_Get_1 1.00 root table:index_prune, index:a b diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 8aebd07e1ff3d..6b5ee3ff229fe 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -176,7 +176,7 @@ Projection_12 0.00 root te.expect_time │ │ ├─Selection_80 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) │ │ │ └─IndexScan_78 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo │ │ └─Selection_81 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) - │ │ └─TableScan_79 0.00 cop table:tr, keep order:false + │ │ └─TableScan_79 0.00 cop table:tr, keep order:false, stats:pseudo │ └─IndexLookUp_33 250.00 root │ ├─IndexScan_30 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo │ └─Selection_32 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) diff --git a/executor/adapter.go b/executor/adapter.go index 72c4ba29056b1..e8a4bfc93ff05 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -402,12 +402,13 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { indexIDs = strings.Replace(fmt.Sprintf("%v", a.Ctx.GetSessionVars().StmtCtx.IndexIDs), " ", ",", -1) } execDetail := sessVars.StmtCtx.GetExecDetails() + statsInfos := a.getStatsInfo() if costTime < threshold { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) + logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, sql)) } else { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) + logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) @@ -433,6 +434,28 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } } +func (a *ExecStmt) getStatsInfo() map[string]uint64 { + var physicalPlan plannercore.PhysicalPlan + switch p := a.Plan.(type) { + case *plannercore.Insert: + physicalPlan = p.SelectPlan + case *plannercore.Update: + physicalPlan = p.SelectPlan + case *plannercore.Delete: + physicalPlan = p.SelectPlan + case plannercore.PhysicalPlan: + physicalPlan = p + } + + if physicalPlan == nil { + return nil + } + + statsInfos := make(map[string]uint64) + statsInfos = plannercore.CollectPlanStatsVersion(physicalPlan, statsInfos) + return statsInfos +} + // IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions: // 1. ctx is auto commit tagged // 2. txn is not valid diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index ea8ab80fa9183..f92eed51d9f1e 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -47,6 +47,7 @@ var slowQueryCols = []columnInfo{ {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.SlowLogStatsInfoStr, mysql.TypeVarchar, 512, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -138,6 +139,7 @@ type slowQueryTuple struct { indexNames string isInternal bool digest string + statsInfo string sql string } @@ -216,6 +218,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.isInternal = value == "true" case variable.SlowLogDigestStr: st.digest = value + case variable.SlowLogStatsInfoStr: + st.statsInfo = value case variable.SlowLogQuerySQLStr: st.sql = value } @@ -243,6 +247,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { 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.statsInfo)) record = append(record, types.NewStringDatum(st.sql)) return record } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 00f5228ac5d70..f05eb28d7077b 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -31,6 +31,7 @@ func (s *testSuite) TestParseSlowLogFile(c *C) { # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:1,t2:2 select * from t;`) scanner := bufio.NewScanner(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") @@ -47,7 +48,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,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,select * from t;" + expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,select * from t;" c.Assert(expectRecordString, Equals, recordString) } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 4e744d40dea3a..1f9fb1d7eda8d 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -305,6 +305,7 @@ func (s *testSuite) TestSlowQuery(c *C) { # DB: test # Is_internal: false # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:1,t2:2 select * from t_slim;`)) c.Assert(f.Close(), IsNil) c.Assert(err, IsNil) @@ -312,8 +313,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|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|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.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0|0|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|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.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index f779619f2d6d8..7039968647b35 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" @@ -491,7 +492,10 @@ func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Col ts.SetSchema(ds.schema) ts.stats = property.NewSimpleStats(1) - ts.stats.UsePseudoStats = ds.statisticTable.Pseudo + ts.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + ts.stats.StatsVersion = statistics.PseudoVersion + } copTask := &copTask{ tablePlan: ts, @@ -526,7 +530,10 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn rowCount = ds.statisticTable.PseudoAvgCountPerValue() } is.stats = property.NewSimpleStats(rowCount) - is.stats.UsePseudoStats = ds.statisticTable.Pseudo + is.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + is.stats.StatsVersion = statistics.PseudoVersion + } cop := &copTask{ indexPlan: is, diff --git a/planner/core/explain.go b/planner/core/explain.go index 8e77f2ab0d33d..bbd6cb9c625b9 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/statistics" ) // ExplainInfo implements PhysicalPlan interface. @@ -67,7 +68,7 @@ func (p *PhysicalIndexScan) ExplainInfo() string { if p.Desc { buffer.WriteString(", desc") } - if p.stats.UsePseudoStats { + if p.stats.StatsVersion == statistics.PseudoVersion { buffer.WriteString(", stats:pseudo") } return buffer.String() @@ -108,7 +109,7 @@ func (p *PhysicalTableScan) ExplainInfo() string { if p.Desc { buffer.WriteString(", desc") } - if p.stats.UsePseudoStats { + if p.stats.StatsVersion == statistics.PseudoVersion { buffer.WriteString(", stats:pseudo") } return buffer.String() diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e4fe91ee11501..5c3c670b35279 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "golang.org/x/tools/container/intsets" @@ -502,7 +503,11 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) } is.stats = property.NewSimpleStats(rowCount) - is.stats.UsePseudoStats = ds.statisticTable.Pseudo + is.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + is.stats.StatsVersion = statistics.PseudoVersion + } + cop.cst = rowCount * scanFactor task = cop if candidate.isMatchProp { @@ -654,7 +659,11 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) } ts.stats = property.NewSimpleStats(rowCount) - ts.stats.UsePseudoStats = ds.statisticTable.Pseudo + ts.stats.StatsVersion = ds.statisticTable.Version + if ds.statisticTable.Pseudo { + ts.stats.StatsVersion = statistics.PseudoVersion + } + copTask.cst = rowCount * scanFactor if candidate.isMatchProp { if prop.Desc { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 68de1e1e2481a..5997f5e750213 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -374,3 +374,27 @@ type PhysicalTableDual struct { RowCount int } + +// CollectPlanStatsVersion uses to collect the statistics version of the plan. +func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) map[string]uint64 { + for _, child := range plan.Children() { + statsInfos = CollectPlanStatsVersion(child, statsInfos) + } + switch copPlan := plan.(type) { + case *PhysicalTableReader: + statsInfos = CollectPlanStatsVersion(copPlan.tablePlan, statsInfos) + case *PhysicalIndexReader: + statsInfos = CollectPlanStatsVersion(copPlan.indexPlan, statsInfos) + case *PhysicalIndexLookUpReader: + // For index loop up, only the indexPlan is necessary, + // because they use the same stats and we do not set the stats info for tablePlan. + statsInfos = CollectPlanStatsVersion(copPlan.indexPlan, statsInfos) + case *PhysicalIndexScan: + statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + case *PhysicalTableScan: + statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + } + + return statsInfos +} + diff --git a/planner/core/stats.go b/planner/core/stats.go index 38ae954a32d2a..6c689c218b771 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -15,6 +15,7 @@ package core import ( "context" + "github.com/pingcap/tidb/statistics" "math" "github.com/pingcap/errors" @@ -77,11 +78,15 @@ func (ds *DataSource) getColumnNDV(colID int64) (ndv float64) { func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *property.StatsInfo { profile := &property.StatsInfo{ - RowCount: float64(ds.statisticTable.Count), - Cardinality: make([]float64, len(ds.Columns)), - HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), - UsePseudoStats: ds.statisticTable.Pseudo, + RowCount: float64(ds.statisticTable.Count), + Cardinality: make([]float64, len(ds.Columns)), + HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), + StatsVersion: ds.statisticTable.Version, } + if ds.statisticTable.Pseudo { + profile.StatsVersion = statistics.PseudoVersion + } + for i, col := range ds.Columns { profile.Cardinality[i] = ds.getColumnNDV(col.ID) } diff --git a/planner/property/stats_info.go b/planner/property/stats_info.go index 67c112c20d256..21894731c2cab 100644 --- a/planner/property/stats_info.go +++ b/planner/property/stats_info.go @@ -24,10 +24,10 @@ type StatsInfo struct { RowCount float64 Cardinality []float64 - HistColl statistics.HistColl - // UsePseudoStats indicates whether the StatsInfo is calculated using the - // pseudo statistics on a table. - UsePseudoStats bool + HistColl *statistics.HistColl + // StatsVersion indicates the statistics version of a table. + // If the StatsInfo is calculated using the pseudo statistics on a table, StatsVersion will be PseudoVersion. + StatsVersion uint64 } // NewSimpleStats creates a simple StatsInfo with rowCount. @@ -48,10 +48,10 @@ func (s *StatsInfo) Count() int64 { // Scale receives a selectivity and multiplies it with RowCount and Cardinality. func (s *StatsInfo) Scale(factor float64) *StatsInfo { profile := &StatsInfo{ - RowCount: s.RowCount * factor, - Cardinality: make([]float64, len(s.Cardinality)), - HistColl: s.HistColl, - UsePseudoStats: s.UsePseudoStats, + RowCount: s.RowCount * factor, + Cardinality: make([]float64, len(s.Cardinality)), + HistColl: s.HistColl, + StatsVersion: s.StatsVersion, } for i := range profile.Cardinality { profile.Cardinality[i] = s.Cardinality[i] * factor diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index ff58d6670aa3b..8415f22534cc2 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -782,6 +782,8 @@ const ( 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. + // SlowLogStatsInfoStr is plan stats info. + SlowLogStatsInfoStr = "Stats" ) // SlowLogFormat uses for formatting slow log. @@ -795,8 +797,10 @@ const ( // # DB: test // # Index_ids: [1,2] // # Is_internal: false +// # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +// # Stats: t1:1,t2:2 // select * from t_slim; -func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest, sql string) string { +func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest string, statsInfos map[string]uint64, sql string) string { var buf bytes.Buffer execDetailStr := execDetail.String() buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n") @@ -820,6 +824,26 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe if len(digest) > 0 { buf.WriteString(SlowLogPrefixStr + SlowLogDigestStr + SlowLogSpaceMarkStr + digest + "\n") } + if len(statsInfos) > 0 { + buf.WriteString(SlowLogPrefixStr + SlowLogStatsInfoStr + SlowLogSpaceMarkStr) + firstComma := false + vStr := "" + for k, v := range statsInfos { + if v == 0 { + vStr = "pseudo" + } else { + vStr = strconv.FormatUint(v, 10) + + } + if firstComma { + buf.WriteString("," + k + ":" + vStr) + } else { + buf.WriteString(k + ":" + vStr) + firstComma = true + } + } + buf.WriteString("\n") + } if len(sql) == 0 { sql = ";" } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 4a7d9a29834a6..e487331dd1587 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -76,6 +76,8 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { TotalKeys: 10000, ProcessedKeys: 20001, } + statsInfos := make(map[string]uint64) + statsInfos["t1"] = 0 resultString := `# Txn_start_ts: 406649736972468225 # User: root@192.168.0.1 # Conn_ID: 1 @@ -85,9 +87,10 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # Index_ids: [1,2] # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:pseudo select * from t;` sql := "select * from t" digest := parser.DigestHash(sql) - logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, sql) + logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, statsInfos, sql) c.Assert(logString, Equals, resultString) } diff --git a/statistics/table.go b/statistics/table.go index 51f980259c129..21e5ad83b9922 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -45,6 +45,9 @@ const ( outOfRangeBetweenRate = 100 ) +// PseudoVersion means the pseudo statistics version is 0. +const PseudoVersion uint64 = 0 + // Table represents statistics for a table. type Table struct { HistColl @@ -472,7 +475,7 @@ func getOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int } // GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxID and IdxID2ColIDs is built from the given parameter. -func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) HistColl { +func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) *HistColl { newColHistMap := make(map[int64]*Column) colInfoID2UniqueID := make(map[int64]int64) colNames2UniqueID := make(map[string]int64) @@ -512,7 +515,7 @@ func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, newIdxHistMap[idxHist.ID] = idxHist idx2Columns[idxHist.ID] = ids } - newColl := HistColl{ + newColl := &HistColl{ PhysicalID: coll.PhysicalID, HavePhysicalID: coll.HavePhysicalID, Pseudo: coll.Pseudo, From d925712b7a9221415a5a20fed1ce866ef78812df Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 22 Apr 2019 16:29:30 +0800 Subject: [PATCH 2/5] fmt code --- planner/core/physical_plans.go | 1 - planner/core/stats.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 5997f5e750213..4f92e7fb84c9e 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -397,4 +397,3 @@ func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) ma return statsInfos } - diff --git a/planner/core/stats.go b/planner/core/stats.go index 6c689c218b771..426f1c6118992 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -15,12 +15,12 @@ package core import ( "context" - "github.com/pingcap/tidb/statistics" "math" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) From 9a35528bba457ca30ab9f0a08f28ad54798d5bfe Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 22 Apr 2019 16:38:53 +0800 Subject: [PATCH 3/5] fix test --- infoschema/tables_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 1f9fb1d7eda8d..fb1dcc72ed63b 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -302,6 +302,8 @@ func (s *testSuite) TestSlowQuery(c *C) { # Conn_ID: 6 # Query_time: 4.895492 # Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 +# Wait_time: 0.101 +# Backoff_time: 0.092 # DB: test # Is_internal: false # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 From 4e6497b7825ab20ee007fbde64adab65b5e9da4f Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 22 Apr 2019 17:06:08 +0800 Subject: [PATCH 4/5] fix test --- executor/aggregate_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index c4425e5e43160..88b3f013b9566 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 := "785" + columnCountOfAllInformationSchemaTables := "784" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") From d7bd31fb264686a2cbfcf1e643fceb49542a50c4 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 22 Apr 2019 17:11:51 +0800 Subject: [PATCH 5/5] fix test --- executor/aggregate_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 88b3f013b9566..e0545dc4b6dd9 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 := "786" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1")