From b8aacf003e87ab9fc6a2aee66ba0c48fc1404d2d Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 13 Dec 2021 18:20:39 +0800 Subject: [PATCH 01/27] collect hist-needed columns --- planner/core/collect_column_stats_usage.go | 293 +++++++++++------- .../core/collect_column_stats_usage_test.go | 121 ++++++-- planner/core/logical_plans.go | 2 +- 3 files changed, 267 insertions(+), 149 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 6396b1ddad34f..499e389a19c35 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -19,27 +19,46 @@ import ( "github.com/pingcap/tidb/parser/model" ) -// predicateColumnCollector collects predicate columns from logical plan. Predicate columns are the columns whose statistics -// are utilized when making query plans, which usually occur in where conditions, join conditions and so on. -type predicateColumnCollector struct { - // colMap maps expression.Column.UniqueID to the table columns whose statistics are utilized to calculate statistics of the column. - colMap map[int64]map[model.TableColumnID]struct{} +const ( + collectPredicateColumns uint64 = 1 << iota + collectHistNeededColumns +) + +// columnStatsUsageCollector collects predicate columns and/or histogram-needed columns from logical plan. +// Predicate columns are the columns whose statistics are utilized when making query plans, which usually occur in where conditions, join conditions and so on. +// Histogram-needed columns are the columns whose histograms are utilized when making query plans, which usually occur in the conditions pushed down to DataSource. +// The set of histogram-needed columns is the subset of that of predicate columns. +type columnStatsUsageCollector struct { + // collectMode indicates whether to collect predicate columns and/or histogram-needed columns + collectMode uint64 // predicateCols records predicate columns. predicateCols map[model.TableColumnID]struct{} + // colMap maps expression.Column.UniqueID to the table columns whose statistics are utilized to calculate statistics of the column. + // It is used for collecting predicate columns. + colMap map[int64]map[model.TableColumnID]struct{} + // histNeededCols records histogram-needed columns + histNeededCols map[model.TableColumnID]struct{} // cols is used to store columns collected from expressions and saves some allocation. cols []*expression.Column } -func newPredicateColumnCollector() *predicateColumnCollector { - return &predicateColumnCollector{ - colMap: make(map[int64]map[model.TableColumnID]struct{}), - predicateCols: make(map[model.TableColumnID]struct{}), +func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector { + collector := &columnStatsUsageCollector{ + collectMode: collectMode, // Pre-allocate a slice to reduce allocation, 8 doesn't have special meaning. cols: make([]*expression.Column, 0, 8), } + if collectMode&collectPredicateColumns != 0 { + collector.predicateCols = make(map[model.TableColumnID]struct{}) + collector.colMap = make(map[int64]map[model.TableColumnID]struct{}) + } + if collectMode&collectHistNeededColumns != 0 { + collector.histNeededCols = make(map[model.TableColumnID]struct{}) + } + return collector } -func (c *predicateColumnCollector) addPredicateColumn(col *expression.Column) { +func (c *columnStatsUsageCollector) addPredicateColumn(col *expression.Column) { tblColIDs, ok := c.colMap[col.UniqueID] if !ok { // It may happen if some leaf of logical plan is LogicalMemTable/LogicalShow/LogicalShowDDLJobs. @@ -50,21 +69,21 @@ func (c *predicateColumnCollector) addPredicateColumn(col *expression.Column) { } } -func (c *predicateColumnCollector) addPredicateColumnsFromExpression(expr expression.Expression) { +func (c *columnStatsUsageCollector) addPredicateColumnsFromExpression(expr expression.Expression) { cols := expression.ExtractColumnsAndCorColumns(c.cols[:0], expr) for _, col := range cols { c.addPredicateColumn(col) } } -func (c *predicateColumnCollector) addPredicateColumnsFromExpressions(list []expression.Expression) { +func (c *columnStatsUsageCollector) addPredicateColumnsFromExpressions(list []expression.Expression) { cols := expression.ExtractColumnsAndCorColumnsFromExpressions(c.cols[:0], list) for _, col := range cols { c.addPredicateColumn(col) } } -func (c *predicateColumnCollector) updateColMap(col *expression.Column, relatedCols []*expression.Column) { +func (c *columnStatsUsageCollector) updateColMap(col *expression.Column, relatedCols []*expression.Column) { if _, ok := c.colMap[col.UniqueID]; !ok { c.colMap[col.UniqueID] = map[model.TableColumnID]struct{}{} } @@ -80,15 +99,15 @@ func (c *predicateColumnCollector) updateColMap(col *expression.Column, relatedC } } -func (c *predicateColumnCollector) updateColMapFromExpression(col *expression.Column, expr expression.Expression) { +func (c *columnStatsUsageCollector) updateColMapFromExpression(col *expression.Column, expr expression.Expression) { c.updateColMap(col, expression.ExtractColumnsAndCorColumns(c.cols[:0], expr)) } -func (c *predicateColumnCollector) updateColMapFromExpressions(col *expression.Column, list []expression.Expression) { +func (c *columnStatsUsageCollector) updateColMapFromExpressions(col *expression.Column, list []expression.Expression) { c.updateColMap(col, expression.ExtractColumnsAndCorColumnsFromExpressions(c.cols[:0], list)) } -func (ds *DataSource) updateColMapAndAddPredicateColumns(c *predicateColumnCollector) { +func (ds *DataSource) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { tblID := ds.TableInfo().ID for _, col := range ds.Schema().Columns { tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} @@ -98,7 +117,7 @@ func (ds *DataSource) updateColMapAndAddPredicateColumns(c *predicateColumnColle c.addPredicateColumnsFromExpressions(ds.pushedDownConds) } -func (p *LogicalJoin) updateColMapAndAddPredicateColumns(c *predicateColumnCollector) { +func (p *LogicalJoin) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { // The only schema change is merging two schemas so there is no new column. // Assume statistics of all the columns in EqualConditions/LeftConditions/RightConditions/OtherConditions are needed. exprs := make([]expression.Expression, 0, len(p.EqualConditions)+len(p.LeftConditions)+len(p.RightConditions)+len(p.OtherConditions)) @@ -117,7 +136,7 @@ func (p *LogicalJoin) updateColMapAndAddPredicateColumns(c *predicateColumnColle c.addPredicateColumnsFromExpressions(exprs) } -func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *predicateColumnCollector) { +func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { // statistics of the ith column of UnionAll come from statistics of the ith column of each child. schemas := make([]*expression.Schema, 0, len(p.Children())) relatedCols := make([]*expression.Column, 0, len(p.Children())) @@ -133,119 +152,153 @@ func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *predicateColumnC } } -func (c *predicateColumnCollector) collectFromPlan(lp LogicalPlan) { +func (ds *DataSource) addHistNeededColumns(c *columnStatsUsageCollector) { + tblID := ds.TableInfo().ID + columns := expression.ExtractColumnsFromExpressions(c.cols[:0], ds.pushedDownConds, nil) + for _, col := range columns { + tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} + c.histNeededCols[tblColID] = struct{}{} + } +} + +func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { for _, child := range lp.Children() { c.collectFromPlan(child) } - switch x := lp.(type) { - case *DataSource: - x.updateColMapAndAddPredicateColumns(c) - case *LogicalIndexScan: - x.Source.updateColMapAndAddPredicateColumns(c) - // TODO: Is it redundant to add predicate columns from LogicalIndexScan.AccessConds? Is LogicalIndexScan.AccessConds a subset of LogicalIndexScan.Source.pushedDownConds. - c.addPredicateColumnsFromExpressions(x.AccessConds) - case *LogicalTableScan: - x.Source.updateColMapAndAddPredicateColumns(c) - // TODO: Is it redundant to add predicate columns from LogicalTableScan.AccessConds? Is LogicalTableScan.AccessConds a subset of LogicalTableScan.Source.pushedDownConds. - c.addPredicateColumnsFromExpressions(x.AccessConds) - case *TiKVSingleGather: - // TODO: Is it redundant? - x.Source.updateColMapAndAddPredicateColumns(c) - case *LogicalProjection: - // Schema change from children to self. - schema := x.Schema() - for i, expr := range x.Exprs { - c.updateColMapFromExpression(schema.Columns[i], expr) - } - case *LogicalSelection: - // Though the conditions in LogicalSelection are complex conditions which cannot be pushed down to DataSource, we still - // regard statistics of the columns in the conditions as needed. - c.addPredicateColumnsFromExpressions(x.Conditions) - case *LogicalAggregation: - // Just assume statistics of all the columns in GroupByItems are needed. - c.addPredicateColumnsFromExpressions(x.GroupByItems) - // Schema change from children to self. - schema := x.Schema() - for i, aggFunc := range x.AggFuncs { - c.updateColMapFromExpressions(schema.Columns[i], aggFunc.Args) - } - case *LogicalWindow: - // Statistics of the columns in LogicalWindow.PartitionBy are used in optimizeByShuffle4Window. - // It seems that we don't use statistics of the columns in LogicalWindow.OrderBy currently? - for _, item := range x.PartitionBy { - c.addPredicateColumn(item.Col) - } - // Schema change from children to self. - windowColumns := x.GetWindowResultColumns() - for i, col := range windowColumns { - c.updateColMapFromExpressions(col, x.WindowFuncDescs[i].Args) - } - case *LogicalJoin: - x.updateColMapAndAddPredicateColumns(c) - case *LogicalApply: - x.updateColMapAndAddPredicateColumns(c) - // Assume statistics of correlated columns are needed. - // Correlated columns can be found in LogicalApply.Children()[0].Schema(). Since we already visit LogicalApply.Children()[0], - // correlated columns must have existed in predicateColumnCollector.colMap. - for _, corCols := range x.CorCols { - c.addPredicateColumn(&corCols.Column) - } - case *LogicalSort: - // Assume statistics of all the columns in ByItems are needed. - for _, item := range x.ByItems { - c.addPredicateColumnsFromExpression(item.Expr) - } - case *LogicalTopN: - // Assume statistics of all the columns in ByItems are needed. - for _, item := range x.ByItems { - c.addPredicateColumnsFromExpression(item.Expr) - } - case *LogicalUnionAll: - x.updateColMapAndAddPredicateColumns(c) - case *LogicalPartitionUnionAll: - x.updateColMapAndAddPredicateColumns(c) - case *LogicalCTE: - // Visit seedPartLogicalPlan and recursivePartLogicalPlan first. - c.collectFromPlan(x.cte.seedPartLogicalPlan) - if x.cte.recursivePartLogicalPlan != nil { - c.collectFromPlan(x.cte.recursivePartLogicalPlan) - } - // Schema change from seedPlan/recursivePlan to self. - columns := x.Schema().Columns - seedColumns := x.cte.seedPartLogicalPlan.Schema().Columns - var recursiveColumns []*expression.Column - if x.cte.recursivePartLogicalPlan != nil { - recursiveColumns = x.cte.recursivePartLogicalPlan.Schema().Columns - } - relatedCols := make([]*expression.Column, 0, 2) - for i, col := range columns { - relatedCols = append(relatedCols[:0], seedColumns[i]) - if recursiveColumns != nil { - relatedCols = append(relatedCols, recursiveColumns[i]) + if c.collectMode&collectPredicateColumns != 0 { + switch x := lp.(type) { + case *DataSource: + x.updateColMapAndAddPredicateColumns(c) + case *LogicalIndexScan: + x.Source.updateColMapAndAddPredicateColumns(c) + c.addPredicateColumnsFromExpressions(x.AccessConds) + case *LogicalTableScan: + x.Source.updateColMapAndAddPredicateColumns(c) + c.addPredicateColumnsFromExpressions(x.AccessConds) + case *LogicalProjection: + // Schema change from children to self. + schema := x.Schema() + for i, expr := range x.Exprs { + c.updateColMapFromExpression(schema.Columns[i], expr) } - c.updateColMap(col, relatedCols) - } - // If IsDistinct is true, then we use getColsNDV to calculate row count(see (*LogicalCTE).DeriveStat). In this case - // statistics of all the columns are needed. - if x.cte.IsDistinct { - for _, col := range columns { - c.addPredicateColumn(col) + case *LogicalSelection: + // Though the conditions in LogicalSelection are complex conditions which cannot be pushed down to DataSource, we still + // regard statistics of the columns in the conditions as needed. + c.addPredicateColumnsFromExpressions(x.Conditions) + case *LogicalAggregation: + // Just assume statistics of all the columns in GroupByItems are needed. + c.addPredicateColumnsFromExpressions(x.GroupByItems) + // Schema change from children to self. + schema := x.Schema() + for i, aggFunc := range x.AggFuncs { + c.updateColMapFromExpressions(schema.Columns[i], aggFunc.Args) + } + case *LogicalWindow: + // Statistics of the columns in LogicalWindow.PartitionBy are used in optimizeByShuffle4Window. + // It seems that we don't use statistics of the columns in LogicalWindow.OrderBy currently? + for _, item := range x.PartitionBy { + c.addPredicateColumn(item.Col) + } + // Schema change from children to self. + windowColumns := x.GetWindowResultColumns() + for i, col := range windowColumns { + c.updateColMapFromExpressions(col, x.WindowFuncDescs[i].Args) + } + case *LogicalJoin: + x.updateColMapAndAddPredicateColumns(c) + case *LogicalApply: + x.updateColMapAndAddPredicateColumns(c) + // Assume statistics of correlated columns are needed. + // Correlated columns can be found in LogicalApply.Children()[0].Schema(). Since we already visit LogicalApply.Children()[0], + // correlated columns must have existed in columnStatsUsageCollector.colMap. + for _, corCols := range x.CorCols { + c.addPredicateColumn(&corCols.Column) + } + case *LogicalSort: + // Assume statistics of all the columns in ByItems are needed. + for _, item := range x.ByItems { + c.addPredicateColumnsFromExpression(item.Expr) + } + case *LogicalTopN: + // Assume statistics of all the columns in ByItems are needed. + for _, item := range x.ByItems { + c.addPredicateColumnsFromExpression(item.Expr) + } + case *LogicalUnionAll: + x.updateColMapAndAddPredicateColumns(c) + case *LogicalPartitionUnionAll: + x.updateColMapAndAddPredicateColumns(c) + case *LogicalCTE: + // Visit seedPartLogicalPlan and recursivePartLogicalPlan first. + c.collectFromPlan(x.cte.seedPartLogicalPlan) + if x.cte.recursivePartLogicalPlan != nil { + c.collectFromPlan(x.cte.recursivePartLogicalPlan) + } + // Schema change from seedPlan/recursivePlan to self. + columns := x.Schema().Columns + seedColumns := x.cte.seedPartLogicalPlan.Schema().Columns + var recursiveColumns []*expression.Column + if x.cte.recursivePartLogicalPlan != nil { + recursiveColumns = x.cte.recursivePartLogicalPlan.Schema().Columns + } + relatedCols := make([]*expression.Column, 0, 2) + for i, col := range columns { + relatedCols = append(relatedCols[:0], seedColumns[i]) + if recursiveColumns != nil { + relatedCols = append(relatedCols, recursiveColumns[i]) + } + c.updateColMap(col, relatedCols) + } + // If IsDistinct is true, then we use getColsNDV to calculate row count(see (*LogicalCTE).DeriveStat). In this case + // statistics of all the columns are needed. + if x.cte.IsDistinct { + for _, col := range columns { + c.addPredicateColumn(col) + } + } + case *LogicalCTETable: + // Schema change from seedPlan to self. + for i, col := range x.Schema().Columns { + c.updateColMap(col, []*expression.Column{x.seedSchema.Columns[i]}) } } - case *LogicalCTETable: - // Schema change from seedPlan to self. - for i, col := range x.Schema().Columns { - c.updateColMap(col, []*expression.Column{x.seedSchema.Columns[i]}) + } + if c.collectMode&collectHistNeededColumns != 0 { + // Histogram-needed columns are the columns which occur in the conditions pushed down to DataSource. + // We don't consider LogicalCTE because seedLogicalPlan and recursiveLogicalPlan haven't got logical optimization + // yet(seedLogicalPlan and recursiveLogicalPlan are optimized in DeriveStats phase). Without logical optimization, + // there is no condition pushed down to DataSource so no histogram-needed column can be collected. + switch x := lp.(type) { + case *DataSource: + x.addHistNeededColumns(c) + case *LogicalIndexScan: + x.Source.addHistNeededColumns(c) + case *LogicalTableScan: + x.Source.addHistNeededColumns(c) } } } -// CollectPredicateColumnsForTest collects predicate columns from logical plan. It is only for test. -func CollectPredicateColumnsForTest(lp LogicalPlan) []model.TableColumnID { - collector := newPredicateColumnCollector() +// CollectColumnStatsUsageForTest is used for test. +// If onlyHistNeeded is true, it collects histogram-needed columns from logical plan. +// Otherwise, it collects predicate columns from logical plan. +func CollectColumnStatsUsageForTest(lp LogicalPlan, onlyHistNeeded bool) []model.TableColumnID { + var collectMode uint64 + if onlyHistNeeded { + collectMode = collectHistNeededColumns + } else { + collectMode = collectPredicateColumns + } + collector := newColumnStatsUsageCollector(collectMode) collector.collectFromPlan(lp) - tblColIDs := make([]model.TableColumnID, 0, len(collector.predicateCols)) - for tblColID := range collector.predicateCols { + var colSet map[model.TableColumnID]struct{} + if onlyHistNeeded { + colSet = collector.histNeededCols + } else { + colSet = collector.predicateCols + } + tblColIDs := make([]model.TableColumnID, 0, len(colSet)) + for tblColID := range colSet { tblColIDs = append(tblColIDs, tblColID) } return tblColIDs diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 912765549fa7b..2e47e31d2c3f2 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -17,8 +17,10 @@ package core_test import ( "context" "fmt" + "go.uber.org/zap" "testing" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" @@ -27,6 +29,31 @@ import ( "github.com/stretchr/testify/require" ) +func getColumnName(is infoschema.InfoSchema, tblColID model.TableColumnID) (string, bool) { + tbl, ok := is.TableByID(tblColID.TableID) + if !ok { + return "", false + } + tblInfo := tbl.Meta() + for _, col := range tblInfo.Columns { + if tblColID.ColumnID == col.ID { + return tblInfo.Name.L + "." + col.Name.L, true + } + } + return "", false +} + +func checkColumnStatsUsage(t *testing.T, is infoschema.InfoSchema, lp plannercore.LogicalPlan, onlyHistNeeded bool, expected []string, comment string) { + tblColIDs := plannercore.CollectColumnStatsUsageForTest(lp, onlyHistNeeded) + cols := make([]string, 0, len(tblColIDs)) + for _, tblColID := range tblColIDs { + col, ok := getColumnName(is, tblColID) + require.True(t, ok, comment) + cols = append(cols, col) + } + require.ElementsMatch(t, expected, cols, comment) +} + func TestCollectPredicateColumns(t *testing.T) { t.Parallel() store, dom, clean := testkit.CreateMockStoreAndDomain(t) @@ -173,30 +200,6 @@ func TestCollectPredicateColumns(t *testing.T) { ctx := context.Background() sctx := tk.Session() is := dom.InfoSchema() - getColName := func(tblColID model.TableColumnID) (string, bool) { - tbl, ok := is.TableByID(tblColID.TableID) - if !ok { - return "", false - } - tblInfo := tbl.Meta() - for _, col := range tblInfo.Columns { - if tblColID.ColumnID == col.ID { - return tblInfo.Name.L + "." + col.Name.L, true - } - } - return "", false - } - checkPredicateColumns := func(lp plannercore.LogicalPlan, expected []string, comment string) { - tblColIDs := plannercore.CollectPredicateColumnsForTest(lp) - cols := make([]string, 0, len(tblColIDs)) - for _, tblColID := range tblColIDs { - col, ok := getColName(tblColID) - require.True(t, ok, comment) - cols = append(cols, col) - } - require.ElementsMatch(t, expected, cols, comment) - } - for _, tt := range tests { comment := fmt.Sprintf("for %s", tt.sql) logutil.BgLogger().Info(comment) @@ -212,11 +215,73 @@ func TestCollectPredicateColumns(t *testing.T) { require.True(t, ok, comment) // We check predicate columns twice, before and after logical optimization. Some logical plan patterns may occur before // logical optimization while others may occur after logical optimization. - // logutil.BgLogger().Info("before logical opt", zap.String("lp", plannercore.ToString(lp))) - checkPredicateColumns(lp, tt.res, comment) + checkColumnStatsUsage(t, is, lp, false, tt.res, comment) lp, err = plannercore.LogicalOptimize(ctx, builder.GetOptFlag(), lp) require.NoError(t, err, comment) - // logutil.BgLogger().Info("after logical opt", zap.String("lp", plannercore.ToString(lp))) - checkPredicateColumns(lp, tt.res, comment) + checkColumnStatsUsage(t, is, lp, false, tt.res, comment) + } +} + +func TestCollectHistNeededColumns(t *testing.T) { + t.Parallel() + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int primary key, b int, c int, index idx_b(b))") + tk.MustExec("create table t2(a int, b int, c int)") + + tests := []struct { + sql string + res []string + }{ + { + sql: "select * from t1 where a > 2", + res: []string{"t1.a"}, + }, + { + sql: "select * from t1 where b in (2, 5) or c = 5", + res: []string{"t1.b", "t1.c"}, + }, + { + sql: "select * from t1 where a + b > 1", + res: []string{"t1.a", "t1.b"}, + }, + { + sql: "select b, count(a) from t1 where b > 1 group by b having count(a) > 2", + res: []string{"t1.b"}, + }, + { + sql: "select * from t1 as x join t2 as y on x.b + y.b > 2 and x.a > 1 and y.c < 1", + res: []string{"t1.a", "t2.c"}, + }, + } + + ctx := context.Background() + sctx := tk.Session() + is := dom.InfoSchema() + for _, tt := range tests { + comment := fmt.Sprintf("for %s", tt.sql) + logutil.BgLogger().Info(comment) + stmts, err := tk.Session().Parse(ctx, tt.sql) + require.NoError(t, err, comment) + stmt := stmts[0] + err = plannercore.Preprocess(sctx, stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err, comment) + builder, _ := plannercore.NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{}) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err, comment) + lp, ok := p.(plannercore.LogicalPlan) + require.True(t, ok, comment) + flags := builder.GetOptFlag() + // JoinReOrder may need columns stats so collecting hist-needed columns must happen before JoinReOrder. + // Hence we disable JoinReOrder and PruneColumnsAgain here. + flags &= ^(uint64(1<<13) | uint64(1<<14)) + lp, err = plannercore.LogicalOptimize(ctx, flags, lp) + require.NoError(t, err, comment) + logutil.BgLogger().Info("sql is ", zap.String("sql", tt.sql)) + logutil.BgLogger().Info("after opt", zap.String("lp", plannercore.ToString(lp))) + checkColumnStatsUsage(t, is, lp, true, tt.res, comment) } } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 212f10d65346a..f36b1f5f6782c 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1310,7 +1310,7 @@ type LogicalCTETable struct { name string idForStorage int - // seedSchema is only used in predicateColumnCollector to get column mapping + // seedSchema is only used in columnStatsUsageCollector to get column mapping seedSchema *expression.Schema } From 58a46fee7338aff1b1c0e84af932c60178329525 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 13 Dec 2021 18:26:20 +0800 Subject: [PATCH 02/27] remove log --- planner/core/collect_column_stats_usage_test.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 2e47e31d2c3f2..2ed73fa5dd470 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -17,7 +17,6 @@ package core_test import ( "context" "fmt" - "go.uber.org/zap" "testing" "github.com/pingcap/tidb/infoschema" @@ -280,8 +279,6 @@ func TestCollectHistNeededColumns(t *testing.T) { flags &= ^(uint64(1<<13) | uint64(1<<14)) lp, err = plannercore.LogicalOptimize(ctx, flags, lp) require.NoError(t, err, comment) - logutil.BgLogger().Info("sql is ", zap.String("sql", tt.sql)) - logutil.BgLogger().Info("after opt", zap.String("lp", plannercore.ToString(lp))) checkColumnStatsUsage(t, is, lp, true, tt.res, comment) } } From cf91ae5628d50e43547faa56521d330a28a28702 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 16 Dec 2021 12:33:55 +0800 Subject: [PATCH 03/27] use ds.physicalTableID rather than ds.TableInfo().ID since we need to consider partition table --- planner/core/collect_column_stats_usage.go | 3 +- .../core/collect_column_stats_usage_test.go | 82 ++++++++++++++++--- 2 files changed, 70 insertions(+), 15 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 499e389a19c35..1c08b16c30691 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -153,10 +153,9 @@ func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *columnStatsUsage } func (ds *DataSource) addHistNeededColumns(c *columnStatsUsageCollector) { - tblID := ds.TableInfo().ID columns := expression.ExtractColumnsFromExpressions(c.cols[:0], ds.pushedDownConds, nil) for _, col := range columns { - tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} + tblColID := model.TableColumnID{TableID: ds.physicalTableID, ColumnID: col.ID} c.histNeededCols[tblColID] = struct{}{} } } diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index caa73a785431f..ab1f7d9903a2e 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -28,15 +28,38 @@ import ( "github.com/stretchr/testify/require" ) -func getColumnName(is infoschema.InfoSchema, tblColID model.TableColumnID) (string, bool) { - tbl, ok := is.TableByID(tblColID.TableID) - if !ok { - return "", false +func getColumnName(t *testing.T, is infoschema.InfoSchema, tblColID model.TableColumnID) (string, bool) { + var tblInfo *model.TableInfo + var prefix string + if tbl, ok := is.TableByID(tblColID.TableID); ok { + tblInfo = tbl.Meta() + prefix = tblInfo.Name.L + "." + } else { + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tbl := range db.Tables { + pi := tbl.GetPartitionInfo() + if pi == nil { + continue + } + for _, def := range pi.Definitions { + if def.ID == tblColID.TableID { + tblInfo = tbl + prefix = tbl.Name.L + "." + def.Name.L + "." + break + } + } + if tblInfo != nil { + break + } + } + if tblInfo == nil { + return "", false + } } - tblInfo := tbl.Meta() for _, col := range tblInfo.Columns { if tblColID.ColumnID == col.ID { - return tblInfo.Name.L + "." + col.Name.L, true + return prefix + col.Name.L, true } } return "", false @@ -46,7 +69,7 @@ func checkColumnStatsUsage(t *testing.T, is infoschema.InfoSchema, lp plannercor tblColIDs := plannercore.CollectColumnStatsUsageForTest(lp, onlyHistNeeded) cols := make([]string, 0, len(tblColIDs)) for _, tblColID := range tblColIDs { - col, ok := getColumnName(is, tblColID) + col, ok := getColumnName(t, is, tblColID) require.True(t, ok, comment) cols = append(cols, col) } @@ -59,7 +82,6 @@ func TestCollectPredicateColumns(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") - tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") tk.MustExec("create table t1(a int, b int, c int)") tk.MustExec("create table t2(a int, b int, c int)") tk.MustExec("create table t3(a int, b int, c int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") @@ -168,11 +190,6 @@ func TestCollectPredicateColumns(t *testing.T) { sql: "select * from ((select a, b from t1) union all (select a, c from t2)) as tmp where tmp.b > 2", res: []string{"t1.b", "t2.c"}, }, - { - // LogicalPartitionUnionAll - sql: "select * from t3 where a < 15 and b > 1", - res: []string{"t3.a", "t3.b"}, - }, { // LogicalCTE sql: "with cte(x, y) as (select a + 1, b from t1 where b > 1) select * from cte where x > 3", @@ -193,12 +210,32 @@ func TestCollectPredicateColumns(t *testing.T) { sql: "with recursive cte(x, y) as (select a, b from t1 union select x + 1, y from cte where x < 5) select * from cte", res: []string{"t1.a", "t1.b"}, }, + { + sql: "set @@session.tidb_partition_prune_mode= 'static'", + }, + { + // LogicalPartitionUnionAll, static partition prune mode, use table ID rather than partition ID + sql: "select * from t3 where a < 15 and b > 1", + res: []string{"t3.a", "t3.b"}, + }, + { + sql: "set @@tidb_partition_prune_mode = 'dynamic'", + }, + { + // dynamic partition prune mode, use table ID rather than partition ID + sql: "select * from t3 where a < 15 and b > 1", + res: []string{"t3.a", "t3.b"}, + }, } ctx := context.Background() sctx := tk.Session() is := dom.InfoSchema() for _, tt := range tests { + if tt.res == nil { + tk.MustExec(tt.sql) + continue + } comment := fmt.Sprintf("for %s", tt.sql) logutil.BgLogger().Info(comment) stmts, err := tk.Session().Parse(ctx, tt.sql) @@ -229,6 +266,7 @@ func TestCollectHistNeededColumns(t *testing.T) { tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int primary key, b int, c int, index idx_b(b))") tk.MustExec("create table t2(a int, b int, c int)") + tk.MustExec("create table t3(a int, b int, c int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") tests := []struct { sql string @@ -254,12 +292,30 @@ func TestCollectHistNeededColumns(t *testing.T) { sql: "select * from t1 as x join t2 as y on x.b + y.b > 2 and x.a > 1 and y.c < 1", res: []string{"t1.a", "t2.c"}, }, + { + sql: "set @@tidb_partition_prune_mode = 'static'", + }, + { + sql: "select * from t3 where a < 15 and b > 1", + res: []string{"t3.p0.a", "t3.p1.a", "t3.p0.b", "t3.p1.b"}, + }, + { + sql: "set @@tidb_partition_prune_mode = 'dynamic'", + }, + { + sql: "select * from t3 where a < 15 and b > 1", + res: []string{"t3.a", "t3.b"}, + }, } ctx := context.Background() sctx := tk.Session() is := dom.InfoSchema() for _, tt := range tests { + if tt.res == nil { + tk.MustExec(tt.sql) + continue + } comment := fmt.Sprintf("for %s", tt.sql) logutil.BgLogger().Info(comment) stmts, err := tk.Session().Parse(ctx, tt.sql) From 4938715304ccd7273475c43c9e871c23a0cefa41 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 16 Dec 2021 20:03:33 +0800 Subject: [PATCH 04/27] implement dump predicate columns and anlyze predicate columns --- domain/domain.go | 6 +++ planner/core/collect_column_stats_usage.go | 13 ++++++ planner/core/optimizer.go | 2 + planner/core/planbuilder.go | 29 +++++++++--- session/session.go | 7 +++ sessionctx/context.go | 4 ++ statistics/handle/handle.go | 32 +++++++++++-- statistics/handle/update.go | 52 +++++++++++++++++++++- util/mock/context.go | 3 ++ 9 files changed, 136 insertions(+), 12 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index d3a9664b97fe7..8fdd0f86324cf 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1371,6 +1371,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) gcStatsTicker := time.NewTicker(100 * lease) dumpFeedbackTicker := time.NewTicker(200 * lease) loadFeedbackTicker := time.NewTicker(5 * lease) + dumpColStatsUsageTicker := time.NewTicker(50 * lease) statsHandle := do.StatsHandle() defer func() { loadFeedbackTicker.Stop() @@ -1421,6 +1422,11 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) if err != nil { logutil.BgLogger().Debug("GC stats failed", zap.Error(err)) } + case <-dumpColStatsUsageTicker.C: + err := statsHandle.DumpColStatsUsageToKV() + if err != nil { + logutil.BgLogger().Debug("dump column stats usage failed", zap.Error(err)) + } } } } diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 1c08b16c30691..bc7c36364d648 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -15,6 +15,8 @@ package core import ( + "time" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" ) @@ -302,3 +304,14 @@ func CollectColumnStatsUsageForTest(lp LogicalPlan, onlyHistNeeded bool) []model } return tblColIDs } + +func collectPredicateColumnsFromPlan(lp LogicalPlan) { + collector := newColumnStatsUsageCollector(collectPredicateColumns) + collector.collectFromPlan(lp) + colStatsUsage := make(map[model.TableColumnID]time.Time, len(collector.predicateCols)) + t := time.Now() + for tblColID := range collector.predicateCols { + colStatsUsage[tblColID] = t + } + lp.SCtx().UpdateColStatsUsage(colStatsUsage) +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 89b156e632cea..a271bdb43f632 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -260,6 +260,8 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { + // TODO: move it to ColumnStatsSyncLoad logic. + collectPredicateColumnsFromPlan(logic) // if there is something after flagPrunColumns, do flagPrunColumnsAgain if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { flag |= flagPrunColumnsAgain diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b830d26da025d..f357ea3d180e5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1818,16 +1818,33 @@ func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] // 3. Otherwise it returns all the columns. func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.TableName) ([]*model.ColumnInfo, error) { tblInfo := tbl.TableInfo - if len(as.ColumnNames) == 0 { + if as.ColumnChoice == model.AllColumns || as.ColumnChoice == model.DefaultChoice { + // TODO: use analyze column config for DefaultChoice return tblInfo.Columns, nil } columnIDs := make(map[int64]struct{}, len(tblInfo.Columns)) - for _, colName := range as.ColumnNames { - colInfo := model.FindColumnInfo(tblInfo.Columns, colName.L) - if colInfo == nil { - return nil, ErrAnalyzeMissColumn.GenWithStackByArgs(colName.O, tblInfo.Name.O) + if as.ColumnChoice == model.ColumnList { + for _, colName := range as.ColumnNames { + colInfo := model.FindColumnInfo(tblInfo.Columns, colName.L) + if colInfo == nil { + return nil, ErrAnalyzeMissColumn.GenWithStackByArgs(colName.O, tblInfo.Name.O) + } + columnIDs[colInfo.ID] = struct{}{} + } + } else if as.ColumnChoice == model.PredicateColumns { + do := domain.GetDomain(b.ctx) + h := do.StatsHandle() + cols, err := h.GetPredicateColumns(tblInfo.ID) + if err != nil { + return nil, err + } + if len(cols) == 0 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("There is no predicate column so all the columns would be analyzed")) + return tblInfo.Columns, nil + } + for _, id := range cols { + columnIDs[id] = struct{}{} } - columnIDs[colInfo.ID] = struct{}{} } missingCols := make(map[int64]struct{}, len(tblInfo.Columns)-len(columnIDs)) if len(tblInfo.Indices) > 0 { diff --git a/session/session.go b/session/session.go index 465de576b37c7..0be8821b5d1f2 100644 --- a/session/session.go +++ b/session/session.go @@ -410,6 +410,13 @@ func (s *session) StoreQueryFeedback(feedback interface{}) { } } +func (s *session) UpdateColStatsUsage(colStatsUsage map[model.TableColumnID]time.Time) { + if s.statsCollector == nil { + return + } + s.statsCollector.UpdateColStatsUsage(colStatsUsage) +} + // StoreIndexUsage stores index usage information in idxUsageCollector. func (s *session) StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64) { if s.idxUsageCollector == nil { diff --git a/sessionctx/context.go b/sessionctx/context.go index 2f9a50aa211f6..03fe41d7d7a00 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -101,6 +101,10 @@ type Context interface { // StoreQueryFeedback stores the query feedback. StoreQueryFeedback(feedback interface{}) + // UpdateColStatsUsage updates the column stats usage. + // TODO: maybe we can use a method called GetSessionStatsCollector to replace both StoreQueryFeedback and UpdateColStatsUsage but we need to deal with import circle if we do so. + UpdateColStatsUsage(colStatsUsage map[model.TableColumnID]time.Time) + // HasDirtyContent checks whether there's dirty update on the given table. HasDirtyContent(tid int64) bool diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 0889d00e431e5..9f63a4d875ac2 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -108,6 +108,11 @@ type Handle struct { sync.Mutex data *statistics.QueryFeedbackMap } + // colMap contains all the column stats usage information from collectors when we dump them to KV. + colMap struct { + sync.Mutex + data colStatsUsageMap + } lease atomic2.Duration @@ -185,6 +190,9 @@ func (h *Handle) Clear() { h.globalMap.Lock() h.globalMap.data = make(tableDeltaMap) h.globalMap.Unlock() + h.colMap.Lock() + h.colMap.data = make(colStatsUsageMap) + h.colMap.Unlock() h.mu.rateMap = make(errorRateDeltaMap) h.mu.Unlock() } @@ -209,6 +217,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* handle.statsCache.Store(statsCache{tables: make(map[int64]*statistics.Table)}) handle.globalMap.data = make(tableDeltaMap) handle.feedback.data = statistics.NewQueryFeedbackMap() + handle.colMap.data = make(colStatsUsageMap) err := handle.RefreshVars() if err != nil { return nil, err @@ -1790,24 +1799,24 @@ func (h *Handle) CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalIDs []int return statistics.CheckAnalyzeVerOnTable(tbl, version) } -type colStatsUsage struct { +type colStatsTimeInfo struct { LastUsedAt *types.Time LastAnalyzedAt *types.Time } // LoadColumnStatsUsage loads column stats usage information from disk. -func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsUsage, error) { +func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInfo, error) { rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT table_id, column_id, last_used_at, last_analyzed_at FROM mysql.column_stats_usage") if err != nil { return nil, errors.Trace(err) } - colStatsMap := make(map[model.TableColumnID]colStatsUsage, len(rows)) + colStatsMap := make(map[model.TableColumnID]colStatsTimeInfo, len(rows)) for _, row := range rows { if row.IsNull(0) || row.IsNull(1) { continue } tblColID := model.TableColumnID{TableID: row.GetInt64(0), ColumnID: row.GetInt64(1)} - var statsUsage colStatsUsage + var statsUsage colStatsTimeInfo if !row.IsNull(2) { t := row.GetTime(2) statsUsage.LastUsedAt = &t @@ -1845,3 +1854,18 @@ func (h *Handle) CollectColumnsInExtendedStats(tableID int64) ([]int64, error) { } return columnIDs, nil } + +// GetPredicateColumns returns IDs of predicate columns, which are the columns whose stats are used(needed) when generating query plans. +func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { + rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT column_id FROM mysql.column_stats_usage WHERE table_id = %? AND last_used_at IS NOT NULL", tableID) + if err != nil { + return nil, errors.Trace(err) + } + columnIDs := make([]int64, 0, len(rows)) + for _, row := range rows { + if !row.IsNull(0) { + columnIDs = append(columnIDs, row.GetInt64(0)) + } + } + return columnIDs, nil +} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index a36f12bbdd7d2..50054cd5d079a 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -132,13 +132,26 @@ func (m errorRateDeltaMap) clear(tableID int64, histID int64, isIndex bool) { m[tableID] = item } -func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, feedback *statistics.QueryFeedbackMap) { +// colStatsUsageMap maps (tableID, columnID) to the last time when the column stats are used(needed). +type colStatsUsageMap map[model.TableColumnID]time.Time + +func (m colStatsUsageMap) merge(other colStatsUsageMap) { + for id, t := range other { + if mt, ok := m[id]; !ok || mt.Before(t) { + m[id] = t + } + } +} + +func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, feedback *statistics.QueryFeedbackMap, colMap colStatsUsageMap) { deltaMap.merge(s.mapper) s.mapper = make(tableDeltaMap) rateMap.merge(s.rateMap) s.rateMap = make(errorRateDeltaMap) feedback.Merge(s.feedback) s.feedback = statistics.NewQueryFeedbackMap() + colMap.merge(s.colMap) + s.colMap = make(colStatsUsageMap) } // SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it. @@ -148,6 +161,7 @@ type SessionStatsCollector struct { mapper tableDeltaMap feedback *statistics.QueryFeedbackMap rateMap errorRateDeltaMap + colMap colStatsUsageMap next *SessionStatsCollector // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. deleted bool @@ -203,6 +217,13 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand return nil } +// UpdateColStatsUsage updates the last time when the column stats are used(needed). +func (s *SessionStatsCollector) UpdateColStatsUsage(usageMap colStatsUsageMap) { + s.Lock() + defer s.Unlock() + s.colMap.merge(usageMap) +} + // NewSessionStatsCollector allocates a stats collector for a session. func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { h.listHead.Lock() @@ -212,6 +233,7 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { rateMap: make(errorRateDeltaMap), next: h.listHead.next, feedback: statistics.NewQueryFeedbackMap(), + colMap: make(colStatsUsageMap), } h.listHead.next = newCollector return newCollector @@ -383,12 +405,13 @@ func (h *Handle) sweepList() { deltaMap := make(tableDeltaMap) errorRateMap := make(errorRateDeltaMap) feedback := statistics.NewQueryFeedbackMap() + colMap := make(colStatsUsageMap) prev := h.listHead prev.Lock() for curr := prev.next; curr != nil; curr = curr.next { curr.Lock() // Merge the session stats into deltaMap, errorRateMap and feedback respectively. - merge(curr, deltaMap, errorRateMap, feedback) + merge(curr, deltaMap, errorRateMap, feedback, colMap) if curr.deleted { prev.next = curr.next // Since the session is already closed, we can safely unlock it here. @@ -410,6 +433,9 @@ func (h *Handle) sweepList() { h.feedback.data.Merge(feedback) h.feedback.data.SiftFeedbacks() h.feedback.Unlock() + h.colMap.Lock() + h.colMap.data.merge(colMap) + h.colMap.Unlock() } // DumpStatsDeltaToKV sweeps the whole list and updates the global map, then we dumps every table that held in map to KV. @@ -844,6 +870,28 @@ func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.Query return errors.Trace(err) } +func (h *Handle) DumpColStatsUsageToKV() error { + h.sweepList() + h.colMap.Lock() + colMap := h.colMap.data + h.colMap.data = make(colStatsUsageMap) + h.colMap.Unlock() + defer func() { + h.colMap.Lock() + h.colMap.data.merge(colMap) + h.colMap.Unlock() + }() + for col, lastUsedAt := range colMap { + const sql = "INSERT INTO mysql.column_stats_usage (table_id, column_id, last_used_at) VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE last_used_at = CASE WHEN last_used_at IS NULL THEN %? ELSE GREATEST(last_used_at, %?) END" + _, _, err := h.execRestrictedSQL(context.Background(), sql, col.TableID, col.ColumnID, lastUsedAt, lastUsedAt, lastUsedAt) + if err != nil { + return err + } + delete(colMap, col) + } + return nil +} + const ( // StatsOwnerKey is the stats owner path that is saved to etcd. StatsOwnerKey = "/tidb/stats/owner" diff --git a/util/mock/context.go b/util/mock/context.go index 7bece0df638d9..2e369191d78bc 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -284,6 +284,9 @@ func (c *Context) GoCtx() context.Context { // StoreQueryFeedback stores the query feedback. func (c *Context) StoreQueryFeedback(_ interface{}) {} +// UpdateColStatsUsage updates the column stats usage. +func (c *Context) UpdateColStatsUsage(_ map[model.TableColumnID]time.Time) {} + // StoreIndexUsage strores the index usage information. func (c *Context) StoreIndexUsage(_ int64, _ int64, _ int64) {} From 7e2cb1618cbda43709d5e666bfea2dfab37d46d1 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 20 Dec 2021 10:46:36 +0800 Subject: [PATCH 05/27] rename functions --- planner/core/collect_column_stats_usage.go | 28 +++++++++++----------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 1c08b16c30691..a8aed02db2262 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -107,7 +107,7 @@ func (c *columnStatsUsageCollector) updateColMapFromExpressions(col *expression. c.updateColMap(col, expression.ExtractColumnsAndCorColumnsFromExpressions(c.cols[:0], list)) } -func (ds *DataSource) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { +func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *DataSource) { tblID := ds.TableInfo().ID for _, col := range ds.Schema().Columns { tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} @@ -117,7 +117,7 @@ func (ds *DataSource) updateColMapAndAddPredicateColumns(c *columnStatsUsageColl c.addPredicateColumnsFromExpressions(ds.pushedDownConds) } -func (p *LogicalJoin) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { +func (c *columnStatsUsageCollector) collectPredicateColumnsForJoin(p *LogicalJoin) { // The only schema change is merging two schemas so there is no new column. // Assume statistics of all the columns in EqualConditions/LeftConditions/RightConditions/OtherConditions are needed. exprs := make([]expression.Expression, 0, len(p.EqualConditions)+len(p.LeftConditions)+len(p.RightConditions)+len(p.OtherConditions)) @@ -136,7 +136,7 @@ func (p *LogicalJoin) updateColMapAndAddPredicateColumns(c *columnStatsUsageColl c.addPredicateColumnsFromExpressions(exprs) } -func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *columnStatsUsageCollector) { +func (c *columnStatsUsageCollector) collectPredicateColumnsForUnionAll(p *LogicalUnionAll) { // statistics of the ith column of UnionAll come from statistics of the ith column of each child. schemas := make([]*expression.Schema, 0, len(p.Children())) relatedCols := make([]*expression.Column, 0, len(p.Children())) @@ -152,7 +152,7 @@ func (p *LogicalUnionAll) updateColMapAndAddPredicateColumns(c *columnStatsUsage } } -func (ds *DataSource) addHistNeededColumns(c *columnStatsUsageCollector) { +func (c *columnStatsUsageCollector) addHistNeededColumns(ds *DataSource) { columns := expression.ExtractColumnsFromExpressions(c.cols[:0], ds.pushedDownConds, nil) for _, col := range columns { tblColID := model.TableColumnID{TableID: ds.physicalTableID, ColumnID: col.ID} @@ -167,12 +167,12 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { if c.collectMode&collectPredicateColumns != 0 { switch x := lp.(type) { case *DataSource: - x.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForDataSource(x) case *LogicalIndexScan: - x.Source.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForDataSource(x.Source) c.addPredicateColumnsFromExpressions(x.AccessConds) case *LogicalTableScan: - x.Source.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForDataSource(x.Source) c.addPredicateColumnsFromExpressions(x.AccessConds) case *LogicalProjection: // Schema change from children to self. @@ -204,9 +204,9 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { c.updateColMapFromExpressions(col, x.WindowFuncDescs[i].Args) } case *LogicalJoin: - x.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForJoin(x) case *LogicalApply: - x.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForJoin(&x.LogicalJoin) // Assume statistics of correlated columns are needed. // Correlated columns can be found in LogicalApply.Children()[0].Schema(). Since we already visit LogicalApply.Children()[0], // correlated columns must have existed in columnStatsUsageCollector.colMap. @@ -224,9 +224,9 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { c.addPredicateColumnsFromExpression(item.Expr) } case *LogicalUnionAll: - x.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForUnionAll(x) case *LogicalPartitionUnionAll: - x.updateColMapAndAddPredicateColumns(c) + c.collectPredicateColumnsForUnionAll(&x.LogicalUnionAll) case *LogicalCTE: // Visit seedPartLogicalPlan and recursivePartLogicalPlan first. c.collectFromPlan(x.cte.seedPartLogicalPlan) @@ -269,11 +269,11 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { // there is no condition pushed down to DataSource so no histogram-needed column can be collected. switch x := lp.(type) { case *DataSource: - x.addHistNeededColumns(c) + c.addHistNeededColumns(x) case *LogicalIndexScan: - x.Source.addHistNeededColumns(c) + c.addHistNeededColumns(x.Source) case *LogicalTableScan: - x.Source.addHistNeededColumns(c) + c.addHistNeededColumns(x.Source) } } } From 3e0da0eacd270df9acd57af9e96d9c78bf99ac15 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 20 Dec 2021 19:45:26 +0800 Subject: [PATCH 06/27] update exported function --- planner/core/collect_column_stats_usage.go | 34 +++++++------------ .../core/collect_column_stats_usage_test.go | 7 +++- 2 files changed, 18 insertions(+), 23 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index a8aed02db2262..d7ccbda8def6c 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -194,7 +194,7 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { } case *LogicalWindow: // Statistics of the columns in LogicalWindow.PartitionBy are used in optimizeByShuffle4Window. - // It seems that we don't use statistics of the columns in LogicalWindow.OrderBy currently? + // We don't use statistics of the columns in LogicalWindow.OrderBy currently. for _, item := range x.PartitionBy { c.addPredicateColumn(item.Col) } @@ -278,27 +278,17 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { } } -// CollectColumnStatsUsageForTest is used for test. -// If onlyHistNeeded is true, it collects histogram-needed columns from logical plan. -// Otherwise, it collects predicate columns from logical plan. -func CollectColumnStatsUsageForTest(lp LogicalPlan, onlyHistNeeded bool) []model.TableColumnID { - var collectMode uint64 - if onlyHistNeeded { - collectMode = collectHistNeededColumns - } else { - collectMode = collectPredicateColumns - } - collector := newColumnStatsUsageCollector(collectMode) +// CollectColumnStatsUsage collects column stats usage from logical plan. +// The first return value is predicate columns and the second return value is histogram-needed columns. +func CollectColumnStatsUsage(lp LogicalPlan) ([]model.TableColumnID, []model.TableColumnID) { + collector := newColumnStatsUsageCollector(collectPredicateColumns | collectHistNeededColumns) collector.collectFromPlan(lp) - var colSet map[model.TableColumnID]struct{} - if onlyHistNeeded { - colSet = collector.histNeededCols - } else { - colSet = collector.predicateCols - } - tblColIDs := make([]model.TableColumnID, 0, len(colSet)) - for tblColID := range colSet { - tblColIDs = append(tblColIDs, tblColID) + set2slice := func(set map[model.TableColumnID]struct{}) []model.TableColumnID { + ret := make([]model.TableColumnID, 0, len(set)) + for tblColID := range set { + ret = append(ret, tblColID) + } + return ret } - return tblColIDs + return set2slice(collector.predicateCols), set2slice(collector.histNeededCols) } diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index ab1f7d9903a2e..6a81e5c9e9f60 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -66,7 +66,12 @@ func getColumnName(t *testing.T, is infoschema.InfoSchema, tblColID model.TableC } func checkColumnStatsUsage(t *testing.T, is infoschema.InfoSchema, lp plannercore.LogicalPlan, onlyHistNeeded bool, expected []string, comment string) { - tblColIDs := plannercore.CollectColumnStatsUsageForTest(lp, onlyHistNeeded) + var tblColIDs []model.TableColumnID + if onlyHistNeeded { + _, tblColIDs = plannercore.CollectColumnStatsUsage(lp) + } else { + tblColIDs, _ = plannercore.CollectColumnStatsUsage(lp) + } cols := make([]string, 0, len(tblColIDs)) for _, tblColID := range tblColIDs { col, ok := getColumnName(t, is, tblColID) From dffab35c2c731f6a03c04be420aa999339305f7a Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 20 Dec 2021 20:23:05 +0800 Subject: [PATCH 07/27] update some function interface --- planner/core/collect_column_stats_usage.go | 13 ------------- planner/core/optimizer.go | 5 +++-- session/session.go | 9 +++++++-- sessionctx/context.go | 2 +- statistics/handle/update.go | 4 ++-- util/mock/context.go | 2 +- 6 files changed, 14 insertions(+), 21 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 2b2fc8e62a93a..d7ccbda8def6c 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -15,8 +15,6 @@ package core import ( - "time" - "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" ) @@ -294,14 +292,3 @@ func CollectColumnStatsUsage(lp LogicalPlan) ([]model.TableColumnID, []model.Tab } return set2slice(collector.predicateCols), set2slice(collector.histNeededCols) } - -func collectPredicateColumnsFromPlan(lp LogicalPlan) { - collector := newColumnStatsUsageCollector(collectPredicateColumns) - collector.collectFromPlan(lp) - colStatsUsage := make(map[model.TableColumnID]time.Time, len(collector.predicateCols)) - t := time.Now() - for tblColID := range collector.predicateCols { - colStatsUsage[tblColID] = t - } - lp.SCtx().UpdateColStatsUsage(colStatsUsage) -} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index a271bdb43f632..092c3674ec2a4 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -260,8 +260,9 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { - // TODO: move it to ColumnStatsSyncLoad logic. - collectPredicateColumnsFromPlan(logic) + // TODO: move it to the logic of sync load hist-needed columns. + predicateColumns, _ := CollectColumnStatsUsage(logic) + sctx.UpdateColStatsUsage(predicateColumns) // if there is something after flagPrunColumns, do flagPrunColumnsAgain if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { flag |= flagPrunColumnsAgain diff --git a/session/session.go b/session/session.go index 3805a6510ca25..895879ed11691 100644 --- a/session/session.go +++ b/session/session.go @@ -412,11 +412,16 @@ func (s *session) StoreQueryFeedback(feedback interface{}) { } } -func (s *session) UpdateColStatsUsage(colStatsUsage map[model.TableColumnID]time.Time) { +func (s *session) UpdateColStatsUsage(predicateColumns []model.TableColumnID) { if s.statsCollector == nil { return } - s.statsCollector.UpdateColStatsUsage(colStatsUsage) + t := time.Now() + colMap := make(map[model.TableColumnID]time.Time, len(predicateColumns)) + for _, col := range predicateColumns { + colMap[col] = t + } + s.statsCollector.UpdateColStatsUsage(colMap) } // StoreIndexUsage stores index usage information in idxUsageCollector. diff --git a/sessionctx/context.go b/sessionctx/context.go index 03fe41d7d7a00..396bf5ab6ff7d 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -103,7 +103,7 @@ type Context interface { // UpdateColStatsUsage updates the column stats usage. // TODO: maybe we can use a method called GetSessionStatsCollector to replace both StoreQueryFeedback and UpdateColStatsUsage but we need to deal with import circle if we do so. - UpdateColStatsUsage(colStatsUsage map[model.TableColumnID]time.Time) + UpdateColStatsUsage(predicateColumns []model.TableColumnID) // HasDirtyContent checks whether there's dirty update on the given table. HasDirtyContent(tid int64) bool diff --git a/statistics/handle/update.go b/statistics/handle/update.go index b87e478b4ed23..3d979ae40bbf4 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -219,10 +219,10 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand } // UpdateColStatsUsage updates the last time when the column stats are used(needed). -func (s *SessionStatsCollector) UpdateColStatsUsage(usageMap colStatsUsageMap) { +func (s *SessionStatsCollector) UpdateColStatsUsage(colMap colStatsUsageMap) { s.Lock() defer s.Unlock() - s.colMap.merge(usageMap) + s.colMap.merge(colMap) } // NewSessionStatsCollector allocates a stats collector for a session. diff --git a/util/mock/context.go b/util/mock/context.go index 2e369191d78bc..0e27bbb4a4b32 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -285,7 +285,7 @@ func (c *Context) GoCtx() context.Context { func (c *Context) StoreQueryFeedback(_ interface{}) {} // UpdateColStatsUsage updates the column stats usage. -func (c *Context) UpdateColStatsUsage(_ map[model.TableColumnID]time.Time) {} +func (c *Context) UpdateColStatsUsage(_ []model.TableColumnID) {} // StoreIndexUsage strores the index usage information. func (c *Context) StoreIndexUsage(_ int64, _ int64, _ int64) {} From fc50879892f263502a92f6b1d61f42125248c1df Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 20 Dec 2021 23:15:02 +0800 Subject: [PATCH 08/27] handle error --- planner/core/planbuilder.go | 41 +++++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f357ea3d180e5..0f7968ee193e2 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1818,20 +1818,14 @@ func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] // 3. Otherwise it returns all the columns. func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.TableName) ([]*model.ColumnInfo, error) { tblInfo := tbl.TableInfo - if as.ColumnChoice == model.AllColumns || as.ColumnChoice == model.DefaultChoice { + columnIDs := make(map[int64]struct{}, len(tblInfo.Columns)) + switch as.ColumnChoice { + case model.DefaultChoice: // TODO: use analyze column config for DefaultChoice return tblInfo.Columns, nil - } - columnIDs := make(map[int64]struct{}, len(tblInfo.Columns)) - if as.ColumnChoice == model.ColumnList { - for _, colName := range as.ColumnNames { - colInfo := model.FindColumnInfo(tblInfo.Columns, colName.L) - if colInfo == nil { - return nil, ErrAnalyzeMissColumn.GenWithStackByArgs(colName.O, tblInfo.Name.O) - } - columnIDs[colInfo.ID] = struct{}{} - } - } else if as.ColumnChoice == model.PredicateColumns { + case model.AllColumns: + return tblInfo.Columns, nil + case model.PredicateColumns: do := domain.GetDomain(b.ctx) h := do.StatsHandle() cols, err := h.GetPredicateColumns(tblInfo.ID) @@ -1839,16 +1833,23 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T return nil, err } if len(cols) == 0 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("There is no predicate column so all the columns would be analyzed")) - return tblInfo.Columns, nil + return nil, errors.Errorf("No predicate column has been collected yet for table %s", tblInfo.Name.L) } for _, id := range cols { columnIDs[id] = struct{}{} } + case model.ColumnList: + for _, colName := range as.ColumnNames { + colInfo := model.FindColumnInfo(tblInfo.Columns, colName.L) + if colInfo == nil { + return nil, ErrAnalyzeMissColumn.GenWithStackByArgs(colName.O, tblInfo.Name.O) + } + columnIDs[colInfo.ID] = struct{}{} + } } missingCols := make(map[int64]struct{}, len(tblInfo.Columns)-len(columnIDs)) if len(tblInfo.Indices) > 0 { - // add indexed columns + // Add indexed columns. // Some indexed columns are generated columns so we also need to add the columns that make up those generated columns. columns, _, err := expression.ColumnInfos2ColumnsAndNames(b.ctx, tbl.Schema, tbl.Name, tblInfo.Columns, tblInfo) if err != nil { @@ -1894,7 +1895,7 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T } } if b.ctx.GetSessionVars().EnableExtendedStats { - // add the columns related to extended stats + // Add the columns related to extended stats. // TODO: column_ids read from mysql.stats_extended in optimization phase may be different from that in execution phase((*Handle).BuildExtendedStats) // if someone inserts data into mysql.stats_extended between the two time points, the new added extended stats may not be computed. statsHandle := domain.GetDomain(b.ctx).StatsHandle() @@ -1909,7 +1910,7 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T } } } - if len(missingCols) > 0 { + if as.ColumnChoice == model.ColumnList && len(missingCols) > 0 { missingNames := make([]string, 0, len(missingCols)) for _, col := range tblInfo.Columns { if _, ok := missingCols[col.ID]; ok { @@ -2045,7 +2046,11 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A } continue } - if len(as.ColumnNames) > 0 { + // TODO: deal with as.ColumnChoice == model.DefaultChoice + if as.ColumnChoice == model.PredicateColumns { + return nil, errors.Errorf("Only the analyze version 2 supports analyzing predicate columns") + } + if as.ColumnChoice == model.ColumnList { return nil, errors.Errorf("Only the analyze version 2 supports analyzing the specified columns") } for _, idx := range idxInfo { From 9de2fe1768d56baa6f1c368b66be5168f30f99a2 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Tue, 21 Dec 2021 16:29:41 +0800 Subject: [PATCH 09/27] move test from core_test to core --- .../core/collect_column_stats_usage_test.go | 281 ++++++++---------- planner/core/logical_plan_test.go | 18 +- 2 files changed, 141 insertions(+), 158 deletions(-) diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 6a81e5c9e9f60..e4fce1955d24a 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -12,23 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package core import ( "context" "fmt" - "testing" + "sort" + . "github.com/pingcap/check" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/hint" - "github.com/pingcap/tidb/util/logutil" - "github.com/stretchr/testify/require" + "github.com/pingcap/tidb/util/testleak" ) -func getColumnName(t *testing.T, is infoschema.InfoSchema, tblColID model.TableColumnID) (string, bool) { +func getColumnName(c *C, is infoschema.InfoSchema, tblColID model.TableColumnID, comment CommentInterface) (string, bool) { var tblInfo *model.TableInfo var prefix string if tbl, ok := is.TableByID(tblColID.TableID); ok { @@ -36,7 +34,7 @@ func getColumnName(t *testing.T, is infoschema.InfoSchema, tblColID model.TableC prefix = tblInfo.Name.L + "." } else { db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) + c.Assert(exists, IsTrue, comment) for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() if pi == nil { @@ -65,280 +63,251 @@ func getColumnName(t *testing.T, is infoschema.InfoSchema, tblColID model.TableC return "", false } -func checkColumnStatsUsage(t *testing.T, is infoschema.InfoSchema, lp plannercore.LogicalPlan, onlyHistNeeded bool, expected []string, comment string) { +func checkColumnStatsUsage(c *C, is infoschema.InfoSchema, lp LogicalPlan, onlyHistNeeded bool, expected []string, comment CommentInterface) { var tblColIDs []model.TableColumnID if onlyHistNeeded { - _, tblColIDs = plannercore.CollectColumnStatsUsage(lp) + _, tblColIDs = CollectColumnStatsUsage(lp) } else { - tblColIDs, _ = plannercore.CollectColumnStatsUsage(lp) + tblColIDs, _ = CollectColumnStatsUsage(lp) } cols := make([]string, 0, len(tblColIDs)) for _, tblColID := range tblColIDs { - col, ok := getColumnName(t, is, tblColID) - require.True(t, ok, comment) + col, ok := getColumnName(c, is, tblColID, comment) + c.Assert(ok, IsTrue, comment) cols = append(cols, col) } - require.ElementsMatch(t, expected, cols, comment) + sort.Strings(cols) + c.Assert(cols, DeepEquals, expected, comment) } -func TestCollectPredicateColumns(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int)") - tk.MustExec("create table t2(a int, b int, c int)") - tk.MustExec("create table t3(a int, b int, c int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") - +func (s *testPlanSuite) TestCollectPredicateColumns(c *C) { + defer testleak.AfterTest(c)() tests := []struct { - sql string - res []string + pruneMode string + sql string + res []string }{ { // DataSource - sql: "select * from t1 where a > 2", - res: []string{"t1.a"}, + sql: "select * from t where a > 2", + res: []string{"t.a"}, }, { // DataSource - sql: "select * from t1 where b in (2, 5) or c = 5", - res: []string{"t1.b", "t1.c"}, + sql: "select * from t where b in (2, 5) or c = 5", + res: []string{"t.b", "t.c"}, }, { // LogicalProjection - sql: "select * from (select a + b as ab, c from t1) as tmp where ab > 4", - res: []string{"t1.a", "t1.b"}, + sql: "select * from (select a + b as ab, c from t) as tmp where ab > 4", + res: []string{"t.a", "t.b"}, }, { // LogicalAggregation - sql: "select b, count(*) from t1 group by b", - res: []string{"t1.b"}, + sql: "select b, count(*) from t group by b", + res: []string{"t.b"}, }, { // LogicalAggregation - sql: "select b, sum(a) from t1 group by b having sum(a) > 3", - res: []string{"t1.a", "t1.b"}, + sql: "select b, sum(a) from t group by b having sum(a) > 3", + res: []string{"t.a", "t.b"}, }, { // LogicalAggregation - sql: "select count(*), sum(a), sum(c) from t1", + sql: "select count(*), sum(a), sum(c) from t", res: []string{}, }, { // LogicalAggregation - sql: "(select a, b from t1) union (select a, c from t2)", - res: []string{"t1.a", "t1.b", "t2.a", "t2.c"}, + sql: "(select a, c from t) union (select a, b from t2)", + res: []string{"t.a", "t.c", "t2.a", "t2.b"}, }, { // LogicalWindow - sql: "select avg(b) over(partition by a) from t1", - res: []string{"t1.a"}, + sql: "select avg(b) over(partition by a) from t", + res: []string{"t.a"}, }, { // LogicalWindow - sql: "select * from (select avg(b) over(partition by a) as w from t1) as tmp where w > 4", - res: []string{"t1.a", "t1.b"}, + sql: "select * from (select avg(b) over(partition by a) as w from t) as tmp where w > 4", + res: []string{"t.a", "t.b"}, }, { // LogicalWindow - sql: "select row_number() over(partition by a order by c) from t1", - res: []string{"t1.a"}, + sql: "select row_number() over(partition by a order by c) from t", + res: []string{"t.a"}, }, { // LogicalJoin - sql: "select * from t1, t2 where t1.a = t2.a", - res: []string{"t1.a", "t2.a"}, + sql: "select * from t, t2 where t.a = t2.a", + res: []string{"t.a", "t2.a"}, }, { // LogicalJoin - sql: "select * from t1 as x join t2 as y on x.b + y.c > 2", - res: []string{"t1.b", "t2.c"}, + sql: "select * from t as x join t2 as y on x.c + y.b > 2", + res: []string{"t.c", "t2.b"}, }, { // LogicalJoin - sql: "select * from t1 as x join t2 as y on x.a = y.a and x.b < 3 and y.c > 2", - res: []string{"t1.a", "t1.b", "t2.a", "t2.c"}, + sql: "select * from t as x join t2 as y on x.a = y.a and x.c < 3 and y.b > 2", + res: []string{"t.a", "t.c", "t2.a", "t2.b"}, }, { // LogicalJoin - sql: "select x.b, y.c, sum(x.c), sum(y.b) from t1 as x join t2 as y on x.a = y.a group by x.b, y.c order by x.b", - res: []string{"t1.a", "t1.b", "t2.a", "t2.c"}, + sql: "select x.c, y.b, sum(x.b), sum(y.a) from t as x join t2 as y on x.a < y.a group by x.c, y.b order by x.c", + res: []string{"t.a", "t.c", "t2.a", "t2.b"}, }, { // LogicalApply - sql: "select * from t1 where t1.b > all(select b from t2 where t2.c > 2)", - res: []string{"t1.b", "t2.b", "t2.c"}, + sql: "select * from t2 where t2.b > all(select b from t where t.c > 2)", + res: []string{"t.b", "t.c", "t2.b"}, }, { // LogicalApply - sql: "select * from t1 where t1.b > (select count(b) from t2 where t2.c > t1.a)", - res: []string{"t1.a", "t1.b", "t2.b", "t2.c"}, + sql: "select * from t2 where t2.b > (select count(b) from t where t.c > t2.a)", + res: []string{"t.b", "t.c", "t2.a", "t2.b"}, }, { // LogicalApply - sql: "select * from t1 where t1.b > (select count(*) from t2 where t2.c > t1.a)", - res: []string{"t1.a", "t1.b", "t2.c"}, + sql: "select * from t where t.b > (select count(*) from t2 where t2.a > t.a)", + res: []string{"t.a", "t.b", "t2.a"}, }, { // LogicalSort - sql: "select * from t1 order by c", - res: []string{"t1.c"}, + sql: "select * from t order by c", + res: []string{"t.c"}, }, { // LogicalTopN - sql: "select * from t1 order by a + b limit 10", - res: []string{"t1.a", "t1.b"}, + sql: "select * from t order by a + b limit 10", + res: []string{"t.a", "t.b"}, }, { // LogicalUnionAll - sql: "select * from ((select a, b from t1) union all (select a, c from t2)) as tmp where tmp.b > 2", - res: []string{"t1.b", "t2.c"}, + sql: "select * from ((select a, c from t) union all (select a, b from t2)) as tmp where tmp.c > 2", + res: []string{"t.c", "t2.b"}, }, { // LogicalCTE - sql: "with cte(x, y) as (select a + 1, b from t1 where b > 1) select * from cte where x > 3", - res: []string{"t1.a", "t1.b"}, + sql: "with cte(x, y) as (select a + 1, b from t where b > 1) select * from cte where x > 3", + res: []string{"t.a", "t.b"}, }, { // LogicalCTE, LogicalCTETable - sql: "with recursive cte(x, y) as (select c, 1 from t1 union all select x + 1, y from cte where x < 5) select * from cte", - res: []string{"t1.c"}, + sql: "with recursive cte(x, y) as (select c, 1 from t union all select x + 1, y from cte where x < 5) select * from cte", + res: []string{"t.c"}, }, { // LogicalCTE, LogicalCTETable - sql: "with recursive cte(x, y) as (select 1, c from t1 union all select x + 1, y from cte where x < 5) select * from cte where y > 1", - res: []string{"t1.c"}, + sql: "with recursive cte(x, y) as (select 1, c from t union all select x + 1, y from cte where x < 5) select * from cte where y > 1", + res: []string{"t.c"}, }, { // LogicalCTE, LogicalCTETable - sql: "with recursive cte(x, y) as (select a, b from t1 union select x + 1, y from cte where x < 5) select * from cte", - res: []string{"t1.a", "t1.b"}, - }, - { - sql: "set @@session.tidb_partition_prune_mode= 'static'", + sql: "with recursive cte(x, y) as (select a, b from t union select x + 1, y from cte where x < 5) select * from cte", + res: []string{"t.a", "t.b"}, }, { // LogicalPartitionUnionAll, static partition prune mode, use table ID rather than partition ID - sql: "select * from t3 where a < 15 and b > 1", - res: []string{"t3.a", "t3.b"}, - }, - { - sql: "set @@tidb_partition_prune_mode = 'dynamic'", + pruneMode: "static", + sql: "select * from pt1 where ptn < 20 and b > 1", + res: []string{"pt1.b", "pt1.ptn"}, }, { // dynamic partition prune mode, use table ID rather than partition ID - sql: "select * from t3 where a < 15 and b > 1", - res: []string{"t3.a", "t3.b"}, + pruneMode: "dynamic", + sql: "select * from pt1 where ptn < 20 and b > 1", + res: []string{"pt1.b", "pt1.ptn"}, }, } ctx := context.Background() - sctx := tk.Session() - is := dom.InfoSchema() for _, tt := range tests { - if tt.res == nil { - tk.MustExec(tt.sql) - continue + if tt.sql == "select x.c, y.b from t as x join t2 as y on x.a = y.a group by x.c, y.b order by x.c" { + fmt.Println("hi") + } + comment := Commentf("for %s", tt.sql) + if len(tt.pruneMode) > 0 { + s.ctx.GetSessionVars().PartitionPruneMode.Store(tt.pruneMode) } - comment := fmt.Sprintf("for %s", tt.sql) - logutil.BgLogger().Info(comment) - stmts, err := tk.Session().Parse(ctx, tt.sql) - require.NoError(t, err, comment) - stmt := stmts[0] - err = plannercore.Preprocess(sctx, stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is})) - require.NoError(t, err, comment) - builder, _ := plannercore.NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{}) + stmt, err := s.ParseOneStmt(tt.sql, "", "") + c.Assert(err, IsNil, comment) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + c.Assert(err, IsNil, comment) + builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) - require.NoError(t, err, comment) - lp, ok := p.(plannercore.LogicalPlan) - require.True(t, ok, comment) + c.Assert(err, IsNil, comment) + lp, ok := p.(LogicalPlan) + c.Assert(ok, IsTrue, comment) // We check predicate columns twice, before and after logical optimization. Some logical plan patterns may occur before // logical optimization while others may occur after logical optimization. - checkColumnStatsUsage(t, is, lp, false, tt.res, comment) - lp, err = plannercore.LogicalOptimize(ctx, builder.GetOptFlag(), lp) - require.NoError(t, err, comment) - checkColumnStatsUsage(t, is, lp, false, tt.res, comment) + checkColumnStatsUsage(c, s.is, lp, false, tt.res, comment) + lp, err = LogicalOptimize(ctx, builder.GetOptFlag(), lp) + c.Assert(err, IsNil, comment) + checkColumnStatsUsage(c, s.is, lp, false, tt.res, comment) } } -func TestCollectHistNeededColumns(t *testing.T) { - t.Parallel() - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int primary key, b int, c int, index idx_b(b))") - tk.MustExec("create table t2(a int, b int, c int)") - tk.MustExec("create table t3(a int, b int, c int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") - +func (s *testPlanSuite) TestCollectHistNeededColumns(c *C) { + defer testleak.AfterTest(c)() tests := []struct { - sql string - res []string + pruneMode string + sql string + res []string }{ { - sql: "select * from t1 where a > 2", - res: []string{"t1.a"}, - }, - { - sql: "select * from t1 where b in (2, 5) or c = 5", - res: []string{"t1.b", "t1.c"}, - }, - { - sql: "select * from t1 where a + b > 1", - res: []string{"t1.a", "t1.b"}, + sql: "select * from t where a > 2", + res: []string{"t.a"}, }, { - sql: "select b, count(a) from t1 where b > 1 group by b having count(a) > 2", - res: []string{"t1.b"}, + sql: "select * from t where b in (2, 5) or c = 5", + res: []string{"t.b", "t.c"}, }, { - sql: "select * from t1 as x join t2 as y on x.b + y.b > 2 and x.a > 1 and y.c < 1", - res: []string{"t1.a", "t2.c"}, + sql: "select * from t where a + b > 1", + res: []string{"t.a", "t.b"}, }, { - sql: "set @@tidb_partition_prune_mode = 'static'", + sql: "select b, count(a) from t where b > 1 group by b having count(a) > 2", + res: []string{"t.b"}, }, { - sql: "select * from t3 where a < 15 and b > 1", - res: []string{"t3.p0.a", "t3.p1.a", "t3.p0.b", "t3.p1.b"}, + sql: "select * from t as x join t2 as y on x.b + y.b > 2 and x.c > 1 and y.a < 1", + res: []string{"t.c", "t2.a"}, }, { - sql: "set @@tidb_partition_prune_mode = 'dynamic'", + pruneMode: "static", + sql: "select * from pt1 where ptn < 20 and b > 1", + res: []string{"pt1.p1.b", "pt1.p1.ptn", "pt1.p2.b", "pt1.p2.ptn"}, }, { - sql: "select * from t3 where a < 15 and b > 1", - res: []string{"t3.a", "t3.b"}, + pruneMode: "dynamic", + sql: "select * from pt1 where ptn < 20 and b > 1", + res: []string{"pt1.b", "pt1.ptn"}, }, } ctx := context.Background() - sctx := tk.Session() - is := dom.InfoSchema() for _, tt := range tests { - if tt.res == nil { - tk.MustExec(tt.sql) - continue + comment := Commentf("for %s", tt.sql) + if len(tt.pruneMode) > 0 { + s.ctx.GetSessionVars().PartitionPruneMode.Store(tt.pruneMode) } - comment := fmt.Sprintf("for %s", tt.sql) - logutil.BgLogger().Info(comment) - stmts, err := tk.Session().Parse(ctx, tt.sql) - require.NoError(t, err, comment) - stmt := stmts[0] - err = plannercore.Preprocess(sctx, stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is})) - require.NoError(t, err, comment) - builder, _ := plannercore.NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{}) + stmt, err := s.ParseOneStmt(tt.sql, "", "") + c.Assert(err, IsNil, comment) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + c.Assert(err, IsNil, comment) + builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) - require.NoError(t, err, comment) - lp, ok := p.(plannercore.LogicalPlan) - require.True(t, ok, comment) + c.Assert(err, IsNil, comment) + lp, ok := p.(LogicalPlan) + c.Assert(ok, IsTrue, comment) flags := builder.GetOptFlag() // JoinReOrder may need columns stats so collecting hist-needed columns must happen before JoinReOrder. // Hence we disable JoinReOrder and PruneColumnsAgain here. - flags &= ^(uint64(1<<13) | uint64(1<<14)) - lp, err = plannercore.LogicalOptimize(ctx, flags, lp) - require.NoError(t, err, comment) - checkColumnStatsUsage(t, is, lp, true, tt.res, comment) + flags &= ^(flagJoinReOrder | flagPrunColumnsAgain) + lp, err = LogicalOptimize(ctx, flags, lp) + c.Assert(err, IsNil, comment) + checkColumnStatsUsage(c, s.is, lp, true, tt.res, comment) } } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index e381da64fcdb6..0136545eff430 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -59,8 +59,22 @@ type testPlanSuite struct { } func (s *testPlanSuite) SetUpSuite(c *C) { - s.is = infoschema.MockInfoSchema([]*model.TableInfo{MockSignedTable(), MockUnsignedTable(), MockView(), MockNoPKTable(), - MockRangePartitionTable(), MockHashPartitionTable(), MockListPartitionTable()}) + tblInfos := []*model.TableInfo{MockSignedTable(), MockUnsignedTable(), MockView(), MockNoPKTable(), + MockRangePartitionTable(), MockHashPartitionTable(), MockListPartitionTable()} + id := int64(0) + for _, tblInfo := range tblInfos { + tblInfo.ID = id + id += 1 + pi := tblInfo.GetPartitionInfo() + if pi == nil { + continue + } + for _, def := range pi.Definitions { + def.ID = id + id += 1 + } + } + s.is = infoschema.MockInfoSchema(tblInfos) s.ctx = MockContext() domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is) s.ctx.GetSessionVars().EnableWindowFunction = true From 608046c8a22fa7dd073eb217e1b761b53f325003 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Tue, 21 Dec 2021 16:30:55 +0800 Subject: [PATCH 10/27] upd --- planner/core/collect_column_stats_usage_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index e4fce1955d24a..aaae1b3ed650c 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -242,7 +242,7 @@ func (s *testPlanSuite) TestCollectPredicateColumns(c *C) { // We check predicate columns twice, before and after logical optimization. Some logical plan patterns may occur before // logical optimization while others may occur after logical optimization. checkColumnStatsUsage(c, s.is, lp, false, tt.res, comment) - lp, err = LogicalOptimize(ctx, builder.GetOptFlag(), lp) + lp, err = logicalOptimize(ctx, builder.GetOptFlag(), lp) c.Assert(err, IsNil, comment) checkColumnStatsUsage(c, s.is, lp, false, tt.res, comment) } @@ -306,7 +306,7 @@ func (s *testPlanSuite) TestCollectHistNeededColumns(c *C) { // JoinReOrder may need columns stats so collecting hist-needed columns must happen before JoinReOrder. // Hence we disable JoinReOrder and PruneColumnsAgain here. flags &= ^(flagJoinReOrder | flagPrunColumnsAgain) - lp, err = LogicalOptimize(ctx, flags, lp) + lp, err = logicalOptimize(ctx, flags, lp) c.Assert(err, IsNil, comment) checkColumnStatsUsage(c, s.is, lp, true, tt.res, comment) } From 767734a8d2c6e87f4a694bc4ba9ca2508d5d6fee Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 22 Dec 2021 00:03:57 +0800 Subject: [PATCH 11/27] add tidb_wide_table_column_count --- domain/sysvar_cache.go | 7 ++++ executor/set_test.go | 22 +++++++++++++ planner/core/collect_column_stats_usage.go | 33 +++++++++++-------- .../core/collect_column_stats_usage_test.go | 9 +++-- sessionctx/variable/sysvar.go | 6 ++++ sessionctx/variable/tidb_vars.go | 4 +++ 6 files changed, 66 insertions(+), 15 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index d89ba88a76ee0..c80e61bbc2d87 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -240,6 +240,13 @@ func (do *Domain) checkEnableServerGlobalVar(name, sVal string) { break } storekv.StoreLimit.Store(val) + case variable.TiDBWideTableColumnCount: + var val uint64 + val, err = strconv.ParseUint(sVal, 10, 64) + if err != nil { + break + } + variable.WideTableColumnCount.Store(val) } if err != nil { logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err)) diff --git a/executor/set_test.go b/executor/set_test.go index da121e77b2422..8a33ab88377a1 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -588,6 +588,28 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("1")) tk.MustExec("set global tidb_enable_historical_stats = 0") tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("0")) + + // test for tidb_wide_table_column_count + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("100")) + tk.MustQuery("set global tidb_wide_table_column_count = 1") + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) + tk.MustQuery("set global tidb_wide_table_column_count = 4097") + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) + tk.MustQuery("set global tidb_wide_table_column_count = 50") + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("50")) + tk.MustQuery("set global tidb_wide_table_column_count = 0") + tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '0'")) + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) + tk.MustQuery("set global tidb_wide_table_column_count = -1") + tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '-1'")) + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) + tk.MustQuery("set global tidb_wide_table_column_count = 10000") + tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '10000'")) + tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) + err = tk.ExecToErr("set tidb_wide_table_column_count = 50") + c.Assert(err.Error(), Equals, "[variable:1229]Variable 'tidb_wide_table_column_count' is a GLOBAL variable and should be set with SET GLOBAL") + err = tk.ExecToErr("select @@session.tidb_wide_table_column_count") + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'tidb_wide_table_column_count' is a GLOBAL variable") } func (s *testSuite5) TestTruncateIncorrectIntSessionVar(c *C) { diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index d7ccbda8def6c..fcf79bd99ebad 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" ) const ( @@ -36,6 +37,8 @@ type columnStatsUsageCollector struct { // colMap maps expression.Column.UniqueID to the table columns whose statistics are utilized to calculate statistics of the column. // It is used for collecting predicate columns. colMap map[int64]map[model.TableColumnID]struct{} + // we only collect predicate columns for the tables which column count is no less than wideTableColumnCount. + wideTableColumnCount int // histNeededCols records histogram-needed columns histNeededCols map[model.TableColumnID]struct{} // cols is used to store columns collected from expressions and saves some allocation. @@ -51,6 +54,7 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector if collectMode&collectPredicateColumns != 0 { collector.predicateCols = make(map[model.TableColumnID]struct{}) collector.colMap = make(map[int64]map[model.TableColumnID]struct{}) + collector.wideTableColumnCount = int(variable.WideTableColumnCount.Load()) } if collectMode&collectHistNeededColumns != 0 { collector.histNeededCols = make(map[model.TableColumnID]struct{}) @@ -61,7 +65,6 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector func (c *columnStatsUsageCollector) addPredicateColumn(col *expression.Column) { tblColIDs, ok := c.colMap[col.UniqueID] if !ok { - // It may happen if some leaf of logical plan is LogicalMemTable/LogicalShow/LogicalShowDDLJobs. return } for tblColID := range tblColIDs { @@ -90,7 +93,6 @@ func (c *columnStatsUsageCollector) updateColMap(col *expression.Column, related for _, relatedCol := range relatedCols { tblColIDs, ok := c.colMap[relatedCol.UniqueID] if !ok { - // It may happen if some leaf of logical plan is LogicalMemTable/LogicalShow/LogicalShowDDLJobs. continue } for tblColID := range tblColIDs { @@ -108,12 +110,16 @@ func (c *columnStatsUsageCollector) updateColMapFromExpressions(col *expression. } func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *DataSource) { - tblID := ds.TableInfo().ID - for _, col := range ds.Schema().Columns { - tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} - c.colMap[col.UniqueID] = map[model.TableColumnID]struct{}{tblColID: {}} + if len(ds.TableInfo().Columns) >= c.wideTableColumnCount { + tblID := ds.TableInfo().ID + for _, col := range ds.Schema().Columns { + tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} + c.colMap[col.UniqueID] = map[model.TableColumnID]struct{}{tblColID: {}} + } } // We should use `pushedDownConds` here. `allConds` is used for partition pruning, which doesn't need stats. + // We still need to add predicate columns from `pushedDownConds` even if the number of columns is less than `wideTableColumnCount` + // because there may be some correlated columns in `pushedDownConds`. c.addPredicateColumnsFromExpressions(ds.pushedDownConds) } @@ -278,17 +284,18 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { } } +func set2slice(set map[model.TableColumnID]struct{}) []model.TableColumnID { + ret := make([]model.TableColumnID, 0, len(set)) + for tblColID := range set { + ret = append(ret, tblColID) + } + return ret +} + // CollectColumnStatsUsage collects column stats usage from logical plan. // The first return value is predicate columns and the second return value is histogram-needed columns. func CollectColumnStatsUsage(lp LogicalPlan) ([]model.TableColumnID, []model.TableColumnID) { collector := newColumnStatsUsageCollector(collectPredicateColumns | collectHistNeededColumns) collector.collectFromPlan(lp) - set2slice := func(set map[model.TableColumnID]struct{}) []model.TableColumnID { - ret := make([]model.TableColumnID, 0, len(set)) - for tblColID := range set { - ret = append(ret, tblColID) - } - return ret - } return set2slice(collector.predicateCols), set2slice(collector.histNeededCols) } diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index aaae1b3ed650c..311f8daa78f66 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -66,9 +66,14 @@ func getColumnName(c *C, is infoschema.InfoSchema, tblColID model.TableColumnID, func checkColumnStatsUsage(c *C, is infoschema.InfoSchema, lp LogicalPlan, onlyHistNeeded bool, expected []string, comment CommentInterface) { var tblColIDs []model.TableColumnID if onlyHistNeeded { - _, tblColIDs = CollectColumnStatsUsage(lp) + collector := newColumnStatsUsageCollector(collectHistNeededColumns) + collector.collectFromPlan(lp) + tblColIDs = set2slice(collector.histNeededCols) } else { - tblColIDs, _ = CollectColumnStatsUsage(lp) + collector := newColumnStatsUsageCollector(collectPredicateColumns) + collector.wideTableColumnCount = 1 + collector.collectFromPlan(lp) + tblColIDs = set2slice(collector.predicateCols) } cols := make([]string, 0, len(tblColIDs)) for _, tblColID := range tblColIDs { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 3491f28bc73dc..62d1ab3dd5aee 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1325,6 +1325,12 @@ var defaultSysVars = []*SysVar{ s.EnablePaging = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal, Name: TiDBWideTableColumnCount, Value: strconv.Itoa(DefTiDBWideTableColumnCount), Type: TypeUnsigned, MinValue: 1, MaxValue: config.DefMaxOfTableColumnCountLimit + 1, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatUint(WideTableColumnCount.Load(), 10), nil + }, SetGlobal: func(s *SessionVars, val string) error { + WideTableColumnCount.Store(uint64(tidbOptPositiveInt32(val, DefTiDBWideTableColumnCount))) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ee01348a76441..8b3ec1d79c000 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -622,6 +622,8 @@ const ( TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" // TiDBEnableHistoricalStats enables the historical statistics feature (default off) TiDBEnableHistoricalStats = "tidb_enable_historical_stats" + // TiDBWideTableColumnCount indicates the threshold of the number of columns in the wide table. + TiDBWideTableColumnCount = "tidb_wide_table_column_count" ) // TiDB intentional limits @@ -779,6 +781,7 @@ const ( DefTiDBRegardNULLAsPoint = true DefEnablePlacementCheck = true DefTimestamp = "0" + DefTiDBWideTableColumnCount = 100 ) // Process global variables. @@ -814,6 +817,7 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + WideTableColumnCount = atomic.NewUint64(DefTiDBWideTableColumnCount) ) // TopSQL is the variable for control top sql feature. From 3a366ef5de5c557f277f59dbb503ba8c0c975648 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 22 Dec 2021 00:12:55 +0800 Subject: [PATCH 12/27] upd --- planner/core/planbuilder.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 0f7968ee193e2..b875cb43eff58 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1826,6 +1826,9 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T case model.AllColumns: return tblInfo.Columns, nil case model.PredicateColumns: + if len(tblInfo.Columns) < int(variable.WideTableColumnCount.Load()) { + return nil, errors.Errorf("The number of columns in table %s is less than `tidb_wide_table_column_count` so analyzing predicate columns cannot be applied", tblInfo.Name.L) + } do := domain.GetDomain(b.ctx) h := do.StatsHandle() cols, err := h.GetPredicateColumns(tblInfo.ID) From da0c84adeec5ee1afbbec56421bb6eced6be4490 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 22 Dec 2021 16:26:15 +0800 Subject: [PATCH 13/27] add tests --- domain/domain.go | 3 +- executor/set_test.go | 12 ++--- executor/show_stats.go | 7 +-- planner/core/collect_column_stats_usage.go | 3 ++ statistics/handle/update.go | 1 + statistics/handle/update_test.go | 57 ++++++++++++++++++++++ 6 files changed, 73 insertions(+), 10 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 8fdd0f86324cf..7f6a0c6ec17ad 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1371,7 +1371,8 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) gcStatsTicker := time.NewTicker(100 * lease) dumpFeedbackTicker := time.NewTicker(200 * lease) loadFeedbackTicker := time.NewTicker(5 * lease) - dumpColStatsUsageTicker := time.NewTicker(50 * lease) + //dumpColStatsUsageTicker := time.NewTicker(20 * lease) + dumpColStatsUsageTicker := time.NewTicker(lease) statsHandle := do.StatsHandle() defer func() { loadFeedbackTicker.Stop() diff --git a/executor/set_test.go b/executor/set_test.go index 8a33ab88377a1..4a4090c7da4ce 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -591,19 +591,19 @@ func (s *testSerialSuite1) TestSetVar(c *C) { // test for tidb_wide_table_column_count tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("100")) - tk.MustQuery("set global tidb_wide_table_column_count = 1") + tk.MustExec("set global tidb_wide_table_column_count = 1") tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustQuery("set global tidb_wide_table_column_count = 4097") + tk.MustExec("set global tidb_wide_table_column_count = 4097") tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) - tk.MustQuery("set global tidb_wide_table_column_count = 50") + tk.MustExec("set global tidb_wide_table_column_count = 50") tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("50")) - tk.MustQuery("set global tidb_wide_table_column_count = 0") + tk.MustExec("set global tidb_wide_table_column_count = 0") tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '0'")) tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustQuery("set global tidb_wide_table_column_count = -1") + tk.MustExec("set global tidb_wide_table_column_count = -1") tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '-1'")) tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustQuery("set global tidb_wide_table_column_count = 10000") + tk.MustExec("set global tidb_wide_table_column_count = 10000") tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '10000'")) tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) err = tk.ExecToErr("set tidb_wide_table_column_count = 50") diff --git a/executor/show_stats.go b/executor/show_stats.go index 6cfdbc70daa8c..7226e807c66eb 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -500,9 +500,10 @@ func (e *ShowExec) fetchShowColumnStatsUsage() error { for _, db := range dbs { for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() - if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { - appendTableForColumnStatsUsage(db.Name.O, tbl, pi != nil, nil) - } + // Though partition tables in static pruning mode don't have global stats, we dump predicate columns of partitions with table ID + // rather than partition ID. Hence appendTableForColumnStatsUsage needs to be called for both partition and global in both dynamic + // and static pruning mode. + appendTableForColumnStatsUsage(db.Name.O, tbl, pi != nil, nil) if pi != nil { for i := range pi.Definitions { appendTableForColumnStatsUsage(db.Name.O, tbl, false, &pi.Definitions[i]) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index fcf79bd99ebad..73110f7bb2d25 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -110,7 +110,10 @@ func (c *columnStatsUsageCollector) updateColMapFromExpressions(col *expression. } func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *DataSource) { + // We don't collect predicate columns for the tables whose column count is less than wideTableColumnCount. if len(ds.TableInfo().Columns) >= c.wideTableColumnCount { + // For partition tables, no matter whether it is static or dynamic pruning mode, we use table ID rather than partition ID to + // set TableColumnID.TableID. In this way, we keep the set of predicate columns consistent between different partitions and global table. tblID := ds.TableInfo().ID for _, col := range ds.Schema().Columns { tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 3d979ae40bbf4..c69ef18308db8 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -871,6 +871,7 @@ func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.Query return errors.Trace(err) } +// DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (h *Handle) DumpColStatsUsageToKV() error { h.sweepList() h.colMap.Lock() diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 23a061cd96f6a..4e1f51471c5c1 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -2336,3 +2336,60 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeRatio(c *C) { c.Assert(h.Update(is), IsNil) c.Assert(h.HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) } + +func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { + defer cleanEnv(c, s.store, s.do) + tk := testkit.NewTestKit(c, s.store) + + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + tk.MustExec("set global tidb_wide_table_column_count = 2") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + + h := s.do.StatsHandle() + tk.MustExec("use test") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("create table t2(a int)") + tk.MustExec("create table t3(a int, b int) partition by range(a) (partition p0 values less than (10), partition p1 values less than maxvalue)") + tk.MustExec("insert into t1 values (1, 2), (3, 4), (5, 6)") + tk.MustExec("insert into t2 values (1), (2), (3)") + tk.MustExec("insert into t3 values (1, 2), (3, 4), (11, 12), (13, 14)") + tk.MustExec("select * from t1 where a > 1") + tk.MustExec("select * from t2 where a > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + // t1.a is collected as predicate column + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) == "", IsTrue) + // the number of column in t2 is less than wideTableColumnCount, so no predicate column is collected. + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() + c.Assert(len(rows), Equals, 0) + + tk.MustExec("analyze table t1") + tk.MustExec("select * from t1 where b > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + // t1.a updates last_used_at first and then updates last_analyzed_at while t1.b updates last_analyzed_at first and then updates last_used_at. + // Check both of them behave as expected. + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() + c.Assert(len(rows), Equals, 2) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) != "", IsTrue) + c.Assert(rows[1][:4], DeepEquals, []interface{}{"test", "t1", "", "b"}) + c.Assert(rows[1][4].(string) != "", IsTrue) + c.Assert(rows[1][5].(string) != "", IsTrue) + + // No matter whether it is static or dynamic pruning mode, we record predicate columns using table ID rather than partition ID. + for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", val)) + tk.MustExec("delete from mysql.column_stats_usage") + tk.MustExec("select * from t3 where a > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't3'").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "global", "a"}) + } +} From 36934e402cdc1caa846e4c36ef4890b4fdd1a15e Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Fri, 24 Dec 2021 13:32:53 +0800 Subject: [PATCH 14/27] upd tests --- executor/analyze_test.go | 486 +++++++++++++++++++++++ planner/core/rule_predicate_push_down.go | 2 +- statistics/handle/handle_test.go | 280 ------------- statistics/handle/update_test.go | 17 +- 4 files changed, 503 insertions(+), 282 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 3ee10fe9a3626..0a7307532096d 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -17,6 +17,9 @@ package executor_test import ( "context" "fmt" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/parser/terror" "strconv" "strings" "sync" @@ -1233,3 +1236,486 @@ func TestAnalyzeSamplingWorkPanic(t *testing.T) { require.NotNil(t, err) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic")) } + +func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice){ + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t (a int, b int, c int primary key)") + tk.MustExec("insert into t values (1,1,1), (1,1,2), (2,2,3), (2,2,4), (3,3,5), (4,3,6), (5,4,7), (6,4,8), (null,null,9)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + tk.MustExec("set global tidb_wide_table_column_count = 1") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + tk.MustExec("select * from t where a > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "a", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, 2, len(rows)) + require.Equal(t, "a", rows[0][3]) + require.Equal(t, "c", rows[1][3]) + + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 9")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t a 0 1 2", + "test t a 0 2 2", + "test t c 0 1 1", + "test t c 0 2 1")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 6 1 8 2 1", + "0 2 0 0 8 0 0", // column b is not analyzed + "0 3 9 0 9 2 1", + )) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t a 0 0 3 1 3 5 0", + "test t a 0 1 4 1 6 6 0", + "test t c 0 0 4 1 3 6 0", + "test t c 0 1 7 1 7 9 0")) + }(val) + } +} + +func TestAnalyzeColumnsWithIndex(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t (a int, b int, c int, d int, index idx_b_d(b, d))") + tk.MustExec("insert into t values (1,1,null,1), (2,1,9,1), (1,1,8,1), (2,2,7,2), (1,3,7,3), (2,4,6,4), (1,4,6,5), (2,4,6,5), (1,5,6,5)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + tk.MustExec("set global tidb_wide_table_column_count = 1") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + tk.MustExec("select * from t where c > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "c", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, 3, len(rows)) + require.Equal(t, "b", rows[0][3]) + require.Equal(t, "c", rows[1][3]) + require.Equal(t, "d", rows[2][3]) + + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 9")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t b 0 1 3", + "test t b 0 4 3", + "test t c 0 6 4", + "test t c 0 7 2", + "test t d 0 1 3", + "test t d 0 5 3", + "test t idx_b_d 1 (1, 1) 3", + "test t idx_b_d 1 (4, 5) 2")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 0 0 9 0 0", // column a is not analyzed + "0 2 5 0 9 2 1", + "0 3 4 1 8 2 -0.07", + "0 4 5 0 9 2 1", + "1 1 6 0 18 2 0")) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t b 0 0 2 1 2 3 0", + "test t b 0 1 3 1 5 5 0", + "test t c 0 0 2 1 8 9 0", + "test t d 0 0 2 1 2 3 0", + "test t d 0 1 3 1 4 4 0", + "test t idx_b_d 1 0 3 1 (2, 2) (4, 4) 0", + "test t idx_b_d 1 1 4 1 (5, 5) (5, 5) 0")) + }(val) + } +} + +func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t (a int, b int, c int, d int, primary key(b, d) clustered)") + tk.MustExec("insert into t values (1,1,null,1), (2,2,9,2), (1,3,8,3), (2,4,7,4), (1,5,7,5), (2,6,6,6), (1,7,6,7), (2,8,6,8), (1,9,6,9)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + tk.MustExec("set global tidb_wide_table_column_count = 1") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + tk.MustExec("select * from t where c > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "c", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, 3, len(rows)) + require.Equal(t, "b", rows[0][3]) + require.Equal(t, "c", rows[1][3]) + require.Equal(t, "d", rows[2][3]) + + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 9")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t PRIMARY 1 (1, 1) 1", + "test t PRIMARY 1 (2, 2) 1", + "test t b 0 1 1", + "test t b 0 2 1", + "test t c 0 6 4", + "test t c 0 7 2", + "test t d 0 1 1", + "test t d 0 2 1")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 0 0 9 0 0", // column a is not analyzed + "0 2 9 0 9 2 1", + "0 3 4 1 8 2 -0.07", + "0 4 9 0 9 2 1", + "1 1 9 0 18 2 0")) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t PRIMARY 1 0 4 1 (3, 3) (6, 6) 0", + "test t PRIMARY 1 1 7 1 (7, 7) (9, 9) 0", + "test t b 0 0 4 1 3 6 0", + "test t b 0 1 7 1 7 9 0", + "test t c 0 0 2 1 8 9 0", + "test t d 0 0 4 1 3 6 0", + "test t d 0 1 7 1 7 9 0")) + }(val) + } +} + +func TestAnalyzeColumnsError(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int)") + + // analyze version 1 doesn't support `ANALYZE COLUMNS c1, ..., cn`/`ANALYZE PREDICATE COLUMNS` currently + tk.MustExec("set @@tidb_analyze_version = 1") + err := tk.ExecToErr("analyze table t columns a") + require.Equal(t, "Only the analyze version 2 supports analyzing the specified columns", err.Error()) + err = tk.ExecToErr("analyze table t predicate columns") + require.Equal(t, "Only the analyze version 2 supports analyzing predicate columns", err.Error()) + + tk.MustExec("set @@tidb_analyze_version = 2") + // invalid column + err = tk.ExecToErr("analyze table t columns c") + terr := errors.Cause(err).(*terror.Error) + require.Equal(t, errors.ErrCode(errno.ErrAnalyzeMissColumn), terr.Code()) + tk.MustExec("analyze table t columns a") + + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + tk.MustExec("set global tidb_wide_table_column_count = 3") + err = tk.ExecToErr("analyze table t predicate columns") + require.Equal(t, "The number of columns in table t is less than `tidb_wide_table_column_count` so analyzing predicate columns cannot be applied", err.Error()) + tk.MustExec("set global tidb_wide_table_column_count = 2") + err = tk.ExecToErr("analyze table t predicate columns") + require.Equal(t, "No predicate column has been collected yet for table t", err.Error()) + tk.MustExec("select * from t where a > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + tk.MustExec("analyze table t predicate columns") +} + +func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("create table t (a int, b int, c int, index idx(c)) partition by range (a) (partition p0 values less than (10), partition p1 values less than maxvalue)") + tk.MustExec("insert into t values (1,2,1), (2,4,1), (3,6,1), (4,8,2), (4,8,2), (5,10,3), (5,10,4), (5,10,5), (null,null,6), (11,22,7), (12,24,8), (13,26,9), (14,28,10), (15,30,11), (16,32,12), (16,32,13), (16,32,13), (16,32,14), (17,34,14), (17,34,14)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + defs := tbl.Meta().Partition.Definitions + p0ID := defs[0].ID + p1ID := defs[1].ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + tk.MustExec("set global tidb_wide_table_column_count = 1") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + }() + tk.MustExec("select * from t where a < 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "a", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, 6, len(rows)) + require.Equal(t, []interface{}{"test", "t", "global", "a"}, rows[0][:4]) + require.Equal(t, []interface{}{"test", "t", "global", "c"}, rows[1][:4]) + require.Equal(t, []interface{}{"test", "t", "p0", "a"}, rows[2][:4]) + require.Equal(t, []interface{}{"test", "t", "p0", "c"}, rows[3][:4]) + require.Equal(t, []interface{}{"test", "t", "p1", "a"}, rows[4][:4]) + require.Equal(t, []interface{}{"test", "t", "p1", "c"}, rows[5][:4]) + + rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows() + require.Equal(t, 3, len(rows)) + require.Equal(t, []interface{}{"test", "t", "global", "0", "20"}, append(rows[0][:3], rows[0][4:]...)) + require.Equal(t, []interface{}{"test", "t", "p0", "0", "9"}, append(rows[1][:3], rows[1][4:]...)) + require.Equal(t, []interface{}{"test", "t", "p1", "0", "11"}, append(rows[2][:3], rows[2][4:]...)) + + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t global a 0 16 4", + "test t global a 0 5 3", + "test t global c 0 1 3", + "test t global c 0 14 3", + "test t p0 a 0 4 2", + "test t p0 a 0 5 3", + "test t p0 c 0 1 3", + "test t p0 c 0 2 2", + "test t p1 a 0 16 4", + "test t p1 a 0 17 2", + "test t p1 c 0 13 2", + "test t p1 c 0 14 3")) + + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t global idx 1 1 3", + "test t global idx 1 14 3", + "test t p0 idx 1 1 3", + "test t p0 idx 1 2 2", + "test t p1 idx 1 13 2", + "test t p1 idx 1 14 3")) + + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t global a 0 0 5 2 1 4 0", + "test t global a 0 1 12 2 17 17 0", + "test t global c 0 0 6 1 2 6 0", + "test t global c 0 1 14 2 13 13 0", + "test t p0 a 0 0 2 1 1 2 0", + "test t p0 a 0 1 3 1 3 3 0", + "test t p0 c 0 0 3 1 3 5 0", + "test t p0 c 0 1 4 1 6 6 0", + "test t p1 a 0 0 3 1 11 13 0", + "test t p1 a 0 1 5 1 14 15 0", + "test t p1 c 0 0 4 1 7 10 0", + "test t p1 c 0 1 6 1 11 12 0")) + + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t global idx 1 0 6 1 2 6 0", + "test t global idx 1 1 14 2 13 13 0", + "test t p0 idx 1 0 3 1 3 5 0", + "test t p0 idx 1 1 4 1 6 6 0", + "test t p1 idx 1 0 4 1 7 10 0", + "test t p1 idx 1 1 6 1 11 12 0")) + + tk.MustQuery("select table_id, is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms order by table_id, is_index, hist_id asc").Check( + testkit.Rows(fmt.Sprintf("%d 0 1 12 1 19 2 0", tblID), // global, a + fmt.Sprintf("%d 0 3 14 0 20 2 0", tblID), // global, c + fmt.Sprintf("%d 1 1 14 0 0 2 0", tblID), // global, idx + fmt.Sprintf("%d 0 1 5 1 8 2 1", p0ID), // p0, a + fmt.Sprintf("%d 0 2 0 0 8 0 0", p0ID), // p0, b, not analyzed + fmt.Sprintf("%d 0 3 6 0 9 2 1", p0ID), // p0, c + fmt.Sprintf("%d 1 1 6 0 9 2 0", p0ID), // p0, idx + fmt.Sprintf("%d 0 1 7 0 11 2 1", p1ID), // p1, a + fmt.Sprintf("%d 0 2 0 0 11 0 0", p1ID), // p1, b, not analyzed + fmt.Sprintf("%d 0 3 8 0 11 2 1", p1ID), // p1, c + fmt.Sprintf("%d 1 1 8 0 11 2 0", p1ID), // p1, idx + )) + }(val) + } +} + + +func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec("create table t (a int, b int, c int, index idx(c)) partition by range (a) (partition p0 values less than (10), partition p1 values less than maxvalue)") + tk.MustExec("insert into t values (1,2,1), (2,4,1), (3,6,1), (4,8,2), (4,8,2), (5,10,3), (5,10,4), (5,10,5), (null,null,6), (11,22,7), (12,24,8), (13,26,9), (14,28,10), (15,30,11), (16,32,12), (16,32,13), (16,32,13), (16,32,14), (17,34,14), (17,34,14)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + defs := tbl.Meta().Partition.Definitions + p0ID := defs[0].ID + p1ID := defs[1].ID + + tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, t, len(rows)) + require.Equal(t, []interface{}{"test", "t", "p0", "a"}, rows[0][:4]) + require.Equal(t, []interface{}{"test", "t", "p0", "c"}, rows[1][:4]) + require.Equal(t, []interface{}{"test", "t", "p1", "a"}, rows[2][:4]) + require.Equal(t, []interface{}{"test", "t", "p1", "c"}, rows[3][:4]) + + rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows() + require.Equal(t, 2, len(rows)) + require.Equal(t, []interface{}{"test", "t", "p0", "0", "9"}, append(rows[0][:3], rows[0][4:]...)) + require.Equal(t, []interface{}{"test", "t", "p1", "0", "11"}, append(rows[1][:3], rows[1][4:]...)) + + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t p0 a 0 4 2", + "test t p0 a 0 5 3", + "test t p0 c 0 1 3", + "test t p0 c 0 2 2", + "test t p1 a 0 16 4", + "test t p1 a 0 17 2", + "test t p1 c 0 13 2", + "test t p1 c 0 14 3")) + + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t p0 idx 1 1 3", + "test t p0 idx 1 2 2", + "test t p1 idx 1 13 2", + "test t p1 idx 1 14 3")) + + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t p0 a 0 0 2 1 1 2 0", + "test t p0 a 0 1 3 1 3 3 0", + "test t p0 c 0 0 3 1 3 5 0", + "test t p0 c 0 1 4 1 6 6 0", + "test t p1 a 0 0 3 1 11 13 0", + "test t p1 a 0 1 5 1 14 15 0", + "test t p1 c 0 0 4 1 7 10 0", + "test t p1 c 0 1 6 1 11 12 0")) + + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t p0 idx 1 0 3 1 3 5 0", + "test t p0 idx 1 1 4 1 6 6 0", + "test t p1 idx 1 0 4 1 7 10 0", + "test t p1 idx 1 1 6 1 11 12 0")) + + tk.MustQuery("select table_id, is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms order by table_id, is_index, hist_id asc").Check( + testkit.Rows(fmt.Sprintf("%d 0 1 5 1 8 2 1", p0ID), // p0, a + fmt.Sprintf("%d 0 2 0 0 8 0 0", p0ID), // p0, b, not analyzed + fmt.Sprintf("%d 0 3 6 0 9 2 1", p0ID), // p0, c + fmt.Sprintf("%d 1 1 6 0 9 2 0", p0ID), // p0, idx + fmt.Sprintf("%d 0 1 7 0 11 2 1", p1ID), // p1, a + fmt.Sprintf("%d 0 2 0 0 11 0 0", p1ID), // p1, b, not analyzed + fmt.Sprintf("%d 0 3 8 0 11 2 1", p1ID), // p1, c + fmt.Sprintf("%d 1 1 8 0 11 2 0", p1ID), // p1, idx + )) + }(val) + } +} + diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index f59fedc25ba28..375b0a32c04a3 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -391,7 +391,7 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression } for _, cond := range predicates { newFilter := expression.ColumnSubstitute(cond, p.Schema(), p.Exprs) - if !expression.HasGetSetVarFunc(newFilter) { + if !expression.HasGetSetVarFunc(newFilter) && expression.ExprFromSchema(newFilter, p.Children()[0].Schema()) { canBePushed = append(canBePushed, newFilter) } else { canNotBePushed = append(canNotBePushed, cond) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 924ff178aee73..2598f250ccf1b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -27,10 +27,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" @@ -3105,284 +3103,6 @@ func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount"), IsNil) } -func (s *testStatsSuite) TestAnalyzeColumnsWithPrimaryKey(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t (a int, b int, c int primary key)") - tk.MustExec("insert into t values (1,1,1), (1,1,2), (2,2,3), (2,2,4), (3,3,5), (4,3,6), (5,4,7), (6,4,8), (null,null,9)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0][3], Equals, "a") - c.Assert(rows[1][3], Equals, "c") - - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 9")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t a 0 1 2", - "test t a 0 2 2", - "test t c 0 1 1", - "test t c 0 2 1")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 6 1 8 2 1", - "0 2 0 0 8 0 0", // column b is not analyzed - "0 3 9 0 9 2 1", - )) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t a 0 0 3 1 3 5 0", - "test t a 0 1 4 1 6 6 0", - "test t c 0 0 4 1 3 6 0", - "test t c 0 1 7 1 7 9 0")) -} - -func (s *testStatsSuite) TestAnalyzeColumnsWithIndex(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t (a int, b int, c int, d int, index idx_b_d(b, d))") - tk.MustExec("insert into t values (1,1,null,1), (2,1,9,1), (1,1,8,1), (2,2,7,2), (1,3,7,3), (2,4,6,4), (1,4,6,5), (2,4,6,5), (1,5,6,5)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 3) - c.Assert(rows[0][3], Equals, "b") - c.Assert(rows[1][3], Equals, "c") - c.Assert(rows[2][3], Equals, "d") - - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 9")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t b 0 1 3", - "test t b 0 4 3", - "test t c 0 6 4", - "test t c 0 7 2", - "test t d 0 1 3", - "test t d 0 5 3", - "test t idx_b_d 1 (1, 1) 3", - "test t idx_b_d 1 (4, 5) 2")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 0 0 9 0 0", // column a is not analyzed - "0 2 5 0 9 2 1", - "0 3 4 1 8 2 -0.07", - "0 4 5 0 9 2 1", - "1 1 6 0 18 2 0")) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t b 0 0 2 1 2 3 0", - "test t b 0 1 3 1 5 5 0", - "test t c 0 0 2 1 8 9 0", - "test t d 0 0 2 1 2 3 0", - "test t d 0 1 3 1 4 4 0", - "test t idx_b_d 1 0 3 1 (2, 2) (4, 4) 0", - "test t idx_b_d 1 1 4 1 (5, 5) (5, 5) 0")) -} - -func (s *testStatsSuite) TestAnalyzeColumnsWithClusteredIndex(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t (a int, b int, c int, d int, primary key(b, d) clustered)") - tk.MustExec("insert into t values (1,1,null,1), (2,2,9,2), (1,3,8,3), (2,4,7,4), (1,5,7,5), (2,6,6,6), (1,7,6,7), (2,8,6,8), (1,9,6,9)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 3) - c.Assert(rows[0][3], Equals, "b") - c.Assert(rows[1][3], Equals, "c") - c.Assert(rows[2][3], Equals, "d") - - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 9")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t PRIMARY 1 (1, 1) 1", - "test t PRIMARY 1 (2, 2) 1", - "test t b 0 1 1", - "test t b 0 2 1", - "test t c 0 6 4", - "test t c 0 7 2", - "test t d 0 1 1", - "test t d 0 2 1")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 0 0 9 0 0", // column a is not analyzed - "0 2 9 0 9 2 1", - "0 3 4 1 8 2 -0.07", - "0 4 9 0 9 2 1", - "1 1 9 0 18 2 0")) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t PRIMARY 1 0 4 1 (3, 3) (6, 6) 0", - "test t PRIMARY 1 1 7 1 (7, 7) (9, 9) 0", - "test t b 0 0 4 1 3 6 0", - "test t b 0 1 7 1 7 9 0", - "test t c 0 0 2 1 8 9 0", - "test t d 0 0 4 1 3 6 0", - "test t d 0 1 7 1 7 9 0")) -} - -func (s *testStatsSuite) TestAnalyzeColumnsError(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int)") - - // analyze version 1 doesn't support `ANALYZE COLUMNS c1, ..., cn` currently - tk.MustExec("set @@tidb_analyze_version = 1") - err := tk.ExecToErr("analyze table t columns a") - c.Assert(err.Error(), Equals, "Only the analyze version 2 supports analyzing the specified columns") - - // invalid column - tk.MustExec("set @@tidb_analyze_version = 2") - err = tk.ExecToErr("analyze table t columns c") - terr := errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.ErrAnalyzeMissColumn)) -} - -func (s *testStatsSuite) TestAnalyzeColumnsWithDynamicPartitionTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - tk.MustExec("create table t (a int, b int, c int, index idx(c)) partition by range (a) (partition p0 values less than (10), partition p1 values less than maxvalue)") - tk.MustExec("insert into t values (1,2,1), (2,4,1), (3,6,1), (4,8,2), (4,8,2), (5,10,3), (5,10,4), (5,10,5), (null,null,6), (11,22,7), (12,24,8), (13,26,9), (14,28,10), (15,30,11), (16,32,12), (16,32,13), (16,32,13), (16,32,14), (17,34,14), (17,34,14)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - defs := tbl.Meta().Partition.Definitions - p0ID := defs[0].ID - p1ID := defs[1].ID - - tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 6) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t", "global", "a"}) - c.Assert(rows[1][:4], DeepEquals, []interface{}{"test", "t", "global", "c"}) - c.Assert(rows[2][:4], DeepEquals, []interface{}{"test", "t", "p0", "a"}) - c.Assert(rows[3][:4], DeepEquals, []interface{}{"test", "t", "p0", "c"}) - c.Assert(rows[4][:4], DeepEquals, []interface{}{"test", "t", "p1", "a"}) - c.Assert(rows[5][:4], DeepEquals, []interface{}{"test", "t", "p1", "c"}) - - rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows() - c.Assert(len(rows), Equals, 3) - c.Assert(append(rows[0][:3], rows[0][4:]...), DeepEquals, []interface{}{"test", "t", "global", "0", "20"}) - c.Assert(append(rows[1][:3], rows[1][4:]...), DeepEquals, []interface{}{"test", "t", "p0", "0", "9"}) - c.Assert(append(rows[2][:3], rows[2][4:]...), DeepEquals, []interface{}{"test", "t", "p1", "0", "11"}) - - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t global a 0 16 4", - "test t global a 0 5 3", - "test t global c 0 1 3", - "test t global c 0 14 3", - "test t p0 a 0 4 2", - "test t p0 a 0 5 3", - "test t p0 c 0 1 3", - "test t p0 c 0 2 2", - "test t p1 a 0 16 4", - "test t p1 a 0 17 2", - "test t p1 c 0 13 2", - "test t p1 c 0 14 3")) - - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t global idx 1 1 3", - "test t global idx 1 14 3", - "test t p0 idx 1 1 3", - "test t p0 idx 1 2 2", - "test t p1 idx 1 13 2", - "test t p1 idx 1 14 3")) - - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t global a 0 0 5 2 1 4 0", - "test t global a 0 1 12 2 17 17 0", - "test t global c 0 0 6 1 2 6 0", - "test t global c 0 1 14 2 13 13 0", - "test t p0 a 0 0 2 1 1 2 0", - "test t p0 a 0 1 3 1 3 3 0", - "test t p0 c 0 0 3 1 3 5 0", - "test t p0 c 0 1 4 1 6 6 0", - "test t p1 a 0 0 3 1 11 13 0", - "test t p1 a 0 1 5 1 14 15 0", - "test t p1 c 0 0 4 1 7 10 0", - "test t p1 c 0 1 6 1 11 12 0")) - - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t global idx 1 0 6 1 2 6 0", - "test t global idx 1 1 14 2 13 13 0", - "test t p0 idx 1 0 3 1 3 5 0", - "test t p0 idx 1 1 4 1 6 6 0", - "test t p1 idx 1 0 4 1 7 10 0", - "test t p1 idx 1 1 6 1 11 12 0")) - - tk.MustQuery("select table_id, is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms order by table_id, is_index, hist_id asc").Check( - testkit.Rows(fmt.Sprintf("%d 0 1 12 1 19 2 0", tblID), // global, a - fmt.Sprintf("%d 0 3 14 0 20 2 0", tblID), // global, c - fmt.Sprintf("%d 1 1 14 0 0 2 0", tblID), // global, idx - fmt.Sprintf("%d 0 1 5 1 8 2 1", p0ID), // p0, a - fmt.Sprintf("%d 0 2 0 0 8 0 0", p0ID), // p0, b, not analyzed - fmt.Sprintf("%d 0 3 6 0 9 2 1", p0ID), // p0, c - fmt.Sprintf("%d 1 1 6 0 9 2 0", p0ID), // p0, idx - fmt.Sprintf("%d 0 1 7 0 11 2 1", p1ID), // p1, a - fmt.Sprintf("%d 0 2 0 0 11 0 0", p1ID), // p1, b, not analyzed - fmt.Sprintf("%d 0 3 8 0 11 2 1", p1ID), // p1, c - fmt.Sprintf("%d 1 1 8 0 11 2 0", p1ID), // p1, idx - )) -} - func (s *testStatsSuite) TestAnalyzeColumnsWithStaticPartitionTable(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 4e1f51471c5c1..d983f98f97e39 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -2382,14 +2382,29 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(rows[1][4].(string) != "", IsTrue) c.Assert(rows[1][5].(string) != "", IsTrue) + // Test partition table. // No matter whether it is static or dynamic pruning mode, we record predicate columns using table ID rather than partition ID. for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", val)) tk.MustExec("delete from mysql.column_stats_usage") - tk.MustExec("select * from t3 where a > 1") + tk.MustExec("select * from t3 where a < 5") c.Assert(h.DumpColStatsUsageToKV(), IsNil) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't3'").Rows() c.Assert(len(rows), Equals, 1) c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "global", "a"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) == "", IsTrue) } + + // Test non-correlated subquery. + // Non-correlated subquery will be executed during the plan building phase, which cannot be done by mock in (*testPlanSuite).TestCollectPredicateColumns. + // Hence we put the test of collecting predicate columns for non-correlated subquery here. + tk.MustExec("delete from mysql.column_stats_usage") + tk.MustExec("select * from t2 where t2.a > (select count(*) from t1 where t1.b > 1)") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "b"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) == "", IsTrue) } From 5e6212e3a33154201e723253015c72c489a26094 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 27 Dec 2021 00:21:25 +0800 Subject: [PATCH 15/27] use switch tidb_enable_column_tracking and update tests --- domain/domain.go | 2 +- domain/sysvar_cache.go | 9 +- executor/analyze_test.go | 220 +++++++++++++++--- executor/set_test.go | 29 +-- planner/core/collect_column_stats_usage.go | 23 +- .../core/collect_column_stats_usage_test.go | 4 +- planner/core/optimizer.go | 6 +- planner/core/planbuilder.go | 6 +- sessionctx/variable/sysvar.go | 14 +- sessionctx/variable/tidb_vars.go | 15 +- statistics/handle/handle.go | 77 +++++- statistics/handle/handle_test.go | 181 -------------- statistics/handle/update.go | 30 ++- statistics/handle/update_test.go | 17 +- 14 files changed, 349 insertions(+), 284 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 7f6a0c6ec17ad..cde82be0c1cd7 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1372,7 +1372,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) dumpFeedbackTicker := time.NewTicker(200 * lease) loadFeedbackTicker := time.NewTicker(5 * lease) //dumpColStatsUsageTicker := time.NewTicker(20 * lease) - dumpColStatsUsageTicker := time.NewTicker(lease) + dumpColStatsUsageTicker := time.NewTicker(lease) // TODO: Set to 1 * lease for easily debugging. Remember to restore the default value. statsHandle := do.StatsHandle() defer func() { loadFeedbackTicker.Stop() diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 52596ea3a12a2..4bcf56278274e 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -240,13 +240,8 @@ func (do *Domain) checkEnableServerGlobalVar(name, sVal string) { break } storekv.StoreLimit.Store(val) - case variable.TiDBWideTableColumnCount: - var val uint64 - val, err = strconv.ParseUint(sVal, 10, 64) - if err != nil { - break - } - variable.WideTableColumnCount.Store(val) + case variable.TiDBEnableColumnTracking: + variable.EnableColumnTracking.Store(variable.TiDBOptOn(sVal)) } if err != nil { logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err)) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 0a7307532096d..a7f18020b3fd4 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1239,7 +1239,7 @@ func TestAnalyzeSamplingWorkPanic(t *testing.T) { func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice){ + func(choice model.ColumnChoice) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -1265,11 +1265,11 @@ func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", )) case model.PredicateColumns: - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) - tk.MustExec("set global tidb_wide_table_column_count = 1") + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() + tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where a > 1") require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() @@ -1334,11 +1334,11 @@ func TestAnalyzeColumnsWithIndex(t *testing.T) { "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", )) case model.PredicateColumns: - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) - tk.MustExec("set global tidb_wide_table_column_count = 1") + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() + tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where c > 1") require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() @@ -1412,11 +1412,11 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", )) case model.PredicateColumns: - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) - tk.MustExec("set global tidb_wide_table_column_count = 1") + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() + tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where c > 1") require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() @@ -1463,11 +1463,10 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { } func TestAnalyzeColumnsError(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) + store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) - h := dom.StatsHandle() tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int)") @@ -1484,21 +1483,17 @@ func TestAnalyzeColumnsError(t *testing.T) { err = tk.ExecToErr("analyze table t columns c") terr := errors.Cause(err).(*terror.Error) require.Equal(t, errors.ErrCode(errno.ErrAnalyzeMissColumn), terr.Code()) - tk.MustExec("analyze table t columns a") - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() - tk.MustExec("set global tidb_wide_table_column_count = 3") - err = tk.ExecToErr("analyze table t predicate columns") - require.Equal(t, "The number of columns in table t is less than `tidb_wide_table_column_count` so analyzing predicate columns cannot be applied", err.Error()) - tk.MustExec("set global tidb_wide_table_column_count = 2") - err = tk.ExecToErr("analyze table t predicate columns") - require.Equal(t, "No predicate column has been collected yet for table t", err.Error()) - tk.MustExec("select * from t where a > 1") - require.NoError(t, h.DumpColStatsUsageToKV()) + tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("analyze table t predicate columns") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", + )) } func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { @@ -1534,16 +1529,16 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", )) case model.PredicateColumns: - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) - tk.MustExec("set global tidb_wide_table_column_count = 1") + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() + tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where a < 1") require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() require.Equal(t, 1, len(rows)) - require.Equal(t, "a", rows[0][3]) + require.Equal(t, []interface{}{"test", "t", "global", "a"}, rows[0][:4]) tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") } @@ -1627,7 +1622,6 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { } } - func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { func(choice model.ColumnChoice) { @@ -1651,14 +1645,30 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { p0ID := defs[0].ID p1ID := defs[1].ID - tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where a < 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, []interface{}{"test", "t", "global", "a"}, rows[0][:4]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - require.Equal(t, t, len(rows)) + require.Equal(t, 4, len(rows)) require.Equal(t, []interface{}{"test", "t", "p0", "a"}, rows[0][:4]) require.Equal(t, []interface{}{"test", "t", "p0", "c"}, rows[1][:4]) require.Equal(t, []interface{}{"test", "t", "p1", "a"}, rows[2][:4]) @@ -1719,3 +1729,143 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { } } +func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set @@tidb_enable_extended_stats = on") + tk.MustExec("create table t (a int, b int, c int)") + tk.MustExec("alter table t add stats_extended s1 correlation(b,c)") + tk.MustExec("insert into t values (5,1,1), (4,2,2), (3,3,3), (2,4,4), (1,5,5)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() + require.Equal(t, 2, len(rows)) + require.Equal(t, "b", rows[0][3]) + require.Equal(t, "c", rows[1][3]) + + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 5")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t b 0 1 1", + "test t b 0 2 1", + "test t c 0 1 1", + "test t c 0 2 1")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 0 0 5 0 0", // column a is not analyzed + "0 2 5 0 5 2 1", + "0 3 5 0 5 2 1", + )) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t b 0 0 2 1 3 4 0", + "test t b 0 1 3 1 5 5 0", + "test t c 0 0 2 1 3 4 0", + "test t c 0 1 3 1 5 5 0")) + rows = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, []interface{}{"test", "t", "s1", "[b,c]", "correlation", "1.000000"}, rows[0][:len(rows[0])-1]) + }(val) + } +} + +func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t (a int, b int, c int as (b+1), index idx(c))") + tk.MustExec("insert into t (a,b) values (1,1), (2,2), (3,3), (4,4), (5,4), (6,5), (7,5), (8,5), (null,null)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + switch val { + case model.ColumnList: + tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", + )) + case model.PredicateColumns: + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + // virtual column c is skipped when dumping stats into disk, so only the stats of column b are updated + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 9")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t b 0 4 2", + "test t b 0 5 3", + "test t idx 1 5 2", + "test t idx 1 6 3")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 0 0 0 0", // column a is not analyzed + "0 2 5 1 2 1", + "0 3 0 0 0 0", // column c is not analyzed + "1 1 5 1 2 0")) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t b 0 0 2 1 1 2 0", + "test t b 0 1 3 1 3 3 0", + "test t idx 1 0 2 1 2 3 0", + "test t idx 1 1 3 1 4 4 0")) + }(val) + } +} diff --git a/executor/set_test.go b/executor/set_test.go index 4a4090c7da4ce..d8e26ab3253d5 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -589,27 +589,14 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustExec("set global tidb_enable_historical_stats = 0") tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("0")) - // test for tidb_wide_table_column_count - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("100")) - tk.MustExec("set global tidb_wide_table_column_count = 1") - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustExec("set global tidb_wide_table_column_count = 4097") - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) - tk.MustExec("set global tidb_wide_table_column_count = 50") - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("50")) - tk.MustExec("set global tidb_wide_table_column_count = 0") - tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '0'")) - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustExec("set global tidb_wide_table_column_count = -1") - tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '-1'")) - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("1")) - tk.MustExec("set global tidb_wide_table_column_count = 10000") - tk.MustQuery("show warnings").Check(testkit.Rows(" Truncated incorrect tidb_wide_table_column_count value: '10000'")) - tk.MustQuery("select @@tidb_wide_table_column_count").Check(testkit.Rows("4097")) - err = tk.ExecToErr("set tidb_wide_table_column_count = 50") - c.Assert(err.Error(), Equals, "[variable:1229]Variable 'tidb_wide_table_column_count' is a GLOBAL variable and should be set with SET GLOBAL") - err = tk.ExecToErr("select @@session.tidb_wide_table_column_count") - c.Assert(err.Error(), Equals, "[variable:1238]Variable 'tidb_wide_table_column_count' is a GLOBAL variable") + // test for tidb_enable_column_tracking + tk.MustQuery("select @@tidb_enable_column_tracking").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_enable_column_tracking = 0") + tk.MustQuery("select @@tidb_enable_column_tracking").Check(testkit.Rows("0")) + // When set tidb_enable_column_tracking off, we record the time of the setting operation. + tk.MustQuery("select count(1) from mysql.tidb where variable_name = 'tidb_disable_column_tracking_time' and variable_value is not null").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustQuery("select @@tidb_enable_column_tracking").Check(testkit.Rows("1")) } func (s *testSuite5) TestTruncateIncorrectIntSessionVar(c *C) { diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 3f28ae56ac7b5..af75f449f2f45 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -281,9 +281,24 @@ func set2slice(set map[model.TableColumnID]struct{}) []model.TableColumnID { } // CollectColumnStatsUsage collects column stats usage from logical plan. -// The first return value is predicate columns and the second return value is histogram-needed columns. -func CollectColumnStatsUsage(lp LogicalPlan) ([]model.TableColumnID, []model.TableColumnID) { - collector := newColumnStatsUsageCollector(collectPredicateColumns | collectHistNeededColumns) +// predicate indicates whether to collect predicate columns and histNeeded indicates whether to collect histogram-needed columns. +// The first return value is predicate columns(nil if predicate is false) and the second return value is histogram-needed columns(nil if histNeeded is false). +func CollectColumnStatsUsage(lp LogicalPlan, predicate, histNeeded bool) ([]model.TableColumnID, []model.TableColumnID) { + var mode uint64 + if predicate { + mode |= collectPredicateColumns + } + if histNeeded { + mode |= collectHistNeededColumns + } + collector := newColumnStatsUsageCollector(mode) collector.collectFromPlan(lp) - return set2slice(collector.predicateCols), set2slice(collector.histNeededCols) + var predicateCols, histNeededCols []model.TableColumnID + if predicate { + predicateCols = set2slice(collector.predicateCols) + } + if histNeeded { + histNeededCols = set2slice(collector.histNeededCols) + } + return predicateCols, histNeededCols } diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 5cc64a80e831e..43131e056c818 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -65,9 +65,9 @@ func getColumnName(c *C, is infoschema.InfoSchema, tblColID model.TableColumnID, func checkColumnStatsUsage(c *C, is infoschema.InfoSchema, lp LogicalPlan, onlyHistNeeded bool, expected []string, comment CommentInterface) { var tblColIDs []model.TableColumnID if onlyHistNeeded { - _, tblColIDs = CollectColumnStatsUsage(lp) + _, tblColIDs = CollectColumnStatsUsage(lp, false, true) } else { - tblColIDs, _ = CollectColumnStatsUsage(lp) + tblColIDs, _ = CollectColumnStatsUsage(lp, true, false) } cols := make([]string, 0, len(tblColIDs)) for _, tblColID := range tblColIDs { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 23e28639edd38..280d3b5cdd591 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -261,8 +261,10 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { // TODO: move it to the logic of sync load hist-needed columns. - predicateColumns, _ := CollectColumnStatsUsage(logic) - sctx.UpdateColStatsUsage(predicateColumns) + if variable.EnableColumnTracking.Load() { + predicateColumns, _ := CollectColumnStatsUsage(logic, true, false) + sctx.UpdateColStatsUsage(predicateColumns) + } // if there is something after flagPrunColumns, do flagPrunColumnsAgain if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { flag |= flagPrunColumnsAgain diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b875cb43eff58..bcb14c27175ec 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1826,9 +1826,6 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T case model.AllColumns: return tblInfo.Columns, nil case model.PredicateColumns: - if len(tblInfo.Columns) < int(variable.WideTableColumnCount.Load()) { - return nil, errors.Errorf("The number of columns in table %s is less than `tidb_wide_table_column_count` so analyzing predicate columns cannot be applied", tblInfo.Name.L) - } do := domain.GetDomain(b.ctx) h := do.StatsHandle() cols, err := h.GetPredicateColumns(tblInfo.ID) @@ -1836,7 +1833,8 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T return nil, err } if len(cols) == 0 { - return nil, errors.Errorf("No predicate column has been collected yet for table %s", tblInfo.Name.L) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed", tbl.Schema.L, tbl.Name.L)) + return tblInfo.Columns, nil } for _, id := range cols { columnIDs[id] = struct{}{} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 62d1ab3dd5aee..0a542b458af3e 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1325,10 +1325,18 @@ var defaultSysVars = []*SysVar{ s.EnablePaging = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBWideTableColumnCount, Value: strconv.Itoa(DefTiDBWideTableColumnCount), Type: TypeUnsigned, MinValue: 1, MaxValue: config.DefMaxOfTableColumnCountLimit + 1, GetGlobal: func(s *SessionVars) (string, error) { - return strconv.FormatUint(WideTableColumnCount.Load(), 10), nil + {Scope: ScopeGlobal, Name: TiDBEnableColumnTracking, Value: BoolToOnOff(DefTiDBEnableColumnTracking), skipInit: true, Type: TypeBool, GetGlobal: func(s *SessionVars) (string, error) { + return BoolToOnOff(EnableColumnTracking.Load()), nil }, SetGlobal: func(s *SessionVars, val string) error { - WideTableColumnCount.Store(uint64(tidbOptPositiveInt32(val, DefTiDBWideTableColumnCount))) + v := TiDBOptOn(val) + if !v { + // Set the location to UTC to avoid time zone interference. + disableTime := time.Now().UTC().Format(types.TimeFormat) + if err := setTiDBTableValue(s, TiDBDisableColumnTrackingTime, disableTime, "Record the last time tidb_enable_column_tracking is set off"); err != nil { + return err + } + } + EnableColumnTracking.Store(v) return nil }}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ec5406e8e3640..f643e07830db7 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -15,11 +15,10 @@ package variable import ( - "math" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" + "math" ) /* @@ -622,8 +621,12 @@ const ( TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" // TiDBEnableHistoricalStats enables the historical statistics feature (default off) TiDBEnableHistoricalStats = "tidb_enable_historical_stats" - // TiDBWideTableColumnCount indicates the threshold of the number of columns in the wide table. - TiDBWideTableColumnCount = "tidb_wide_table_column_count" + // TiDBEnableColumnTracking enables collecting predicate columns. + TiDBEnableColumnTracking = "tidb_enable_column_tracking" + // TiDBDisableColumnTrackingTime records the last time TiDBEnableColumnTracking is set off. + // It is used to invalidate the collected predicate columns after turning off TiDBEnableColumnTracking, which avoids physical deletion. + // It doesn't have cache in memory and we directly get/set the variable value from/to mysql.tidb. + TiDBDisableColumnTrackingTime = "tidb_disable_column_tracking_time" ) // TiDB intentional limits @@ -781,7 +784,7 @@ const ( DefTiDBRegardNULLAsPoint = true DefEnablePlacementCheck = true DefTimestamp = "0" - DefTiDBWideTableColumnCount = 100 + DefTiDBEnableColumnTracking = true ) // Process global variables. @@ -817,7 +820,7 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) - WideTableColumnCount = atomic.NewUint64(DefTiDBWideTableColumnCount) + EnableColumnTracking = atomic.NewBool(DefTiDBEnableColumnTracking) ) // TopSQL is the variable for control top sql feature. diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 487770ec9744e..4fa1ed14cac8a 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -168,6 +168,14 @@ func (h *Handle) execRestrictedSQLWithSnapshot(ctx context.Context, sql string, }) } +func execRestrictedSQL(ctx context.Context, exec sqlexec.RestrictedSQLExecutor, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { + stmt, err := exec.ParseWithParamsInternal(ctx, sql, params...) + if err != nil { + return nil, nil, errors.Trace(err) + } + return exec.ExecRestrictedStmt(ctx, stmt) +} + // Clear the statsCache, only for test. func (h *Handle) Clear() { // TODO: Here h.mu seems to protect all the fields of Handle. Is is reasonable? @@ -1804,9 +1812,41 @@ type colStatsTimeInfo struct { LastAnalyzedAt *types.Time } +func getDisableColumnTrackingTime(exec sqlexec.RestrictedSQLExecutor) (*time.Time, error) { + rows, fields, err := execRestrictedSQL(context.Background(), exec, "SELECT variable_value FROM %n.%n WHERE variable_name = %?", mysql.SystemDB, mysql.TiDBTable, variable.TiDBDisableColumnTrackingTime) + if err != nil { + return nil, err + } + if len(rows) == 0 { + return nil, nil + } + d := rows[0].GetDatum(0, &fields[0].Column.FieldType) + value, err := d.ToString() + if err != nil { + return nil, err + } + t, err := time.Parse(types.TimeFormat, value) + if err != nil { + return nil, err + } + return &t, nil +} + // LoadColumnStatsUsage loads column stats usage information from disk. func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInfo, error) { - rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT table_id, column_id, last_used_at, last_analyzed_at FROM mysql.column_stats_usage") + se, err := h.pool.Get() + if err != nil { + return nil, errors.Trace(err) + } + defer h.pool.Put(se) + sctx := se.(sessionctx.Context) + loc := sctx.GetSessionVars().Location() + exec := se.(sqlexec.RestrictedSQLExecutor) + disableTime, err := getDisableColumnTrackingTime(exec) + if err != nil { + return nil, errors.Trace(err) + } + rows, _, err := execRestrictedSQL(context.Background(), exec, "SELECT table_id, column_id, last_used_at, last_analyzed_at FROM mysql.column_stats_usage") if err != nil { return nil, errors.Trace(err) } @@ -1823,7 +1863,13 @@ func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInf } if !row.IsNull(3) { t := row.GetTime(3) - statsUsage.LastAnalyzedAt = &t + gt, err := t.GoTime(loc) + if err != nil { + return nil, errors.Trace(err) + } + if disableTime == nil || gt.After(*disableTime) { + statsUsage.LastAnalyzedAt = &t + } } colStatsMap[tblColID] = statsUsage } @@ -1857,14 +1903,35 @@ func (h *Handle) CollectColumnsInExtendedStats(tableID int64) ([]int64, error) { // GetPredicateColumns returns IDs of predicate columns, which are the columns whose stats are used(needed) when generating query plans. func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { - rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT column_id FROM mysql.column_stats_usage WHERE table_id = %? AND last_used_at IS NOT NULL", tableID) + se, err := h.pool.Get() + if err != nil { + return nil, errors.Trace(err) + } + defer h.pool.Put(se) + sctx := se.(sessionctx.Context) + loc := sctx.GetSessionVars().Location() + exec := se.(sqlexec.RestrictedSQLExecutor) + disableTime, err := getDisableColumnTrackingTime(exec) + if err != nil { + return nil, errors.Trace(err) + } + rows, _, err := execRestrictedSQL(context.Background(), exec, "SELECT column_id, last_used_at FROM mysql.column_stats_usage WHERE table_id = %? AND last_used_at IS NOT NULL", tableID) if err != nil { return nil, errors.Trace(err) } columnIDs := make([]int64, 0, len(rows)) for _, row := range rows { - if !row.IsNull(0) { - columnIDs = append(columnIDs, row.GetInt64(0)) + if row.IsNull(0) || row.IsNull(1) { + continue + } + colID := row.GetInt64(0) + t := row.GetTime(1) + gt, err := t.GoTime(loc) + if err != nil { + return nil, errors.Trace(err) + } + if disableTime == nil || gt.After(*disableTime) { + columnIDs = append(columnIDs, colID) } } return columnIDs, nil diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 2598f250ccf1b..3b20a470bc96b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -3103,187 +3103,6 @@ func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount"), IsNil) } -func (s *testStatsSuite) TestAnalyzeColumnsWithStaticPartitionTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("set @@tidb_partition_prune_mode = 'static'") - tk.MustExec("create table t (a int, b int, c int, index idx(c)) partition by range (a) (partition p0 values less than (10), partition p1 values less than maxvalue)") - tk.MustExec("insert into t values (1,2,1), (2,4,1), (3,6,1), (4,8,2), (4,8,2), (5,10,3), (5,10,4), (5,10,5), (null,null,6), (11,22,7), (12,24,8), (13,26,9), (14,28,10), (15,30,11), (16,32,12), (16,32,13), (16,32,13), (16,32,14), (17,34,14), (17,34,14)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - defs := tbl.Meta().Partition.Definitions - p0ID := defs[0].ID - p1ID := defs[1].ID - - tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0.", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 4) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t", "p0", "a"}) - c.Assert(rows[1][:4], DeepEquals, []interface{}{"test", "t", "p0", "c"}) - c.Assert(rows[2][:4], DeepEquals, []interface{}{"test", "t", "p1", "a"}) - c.Assert(rows[3][:4], DeepEquals, []interface{}{"test", "t", "p1", "c"}) - - rows = tk.MustQuery("show stats_meta where db_name = 'test' and table_name = 't'").Sort().Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(append(rows[0][:3], rows[0][4:]...), DeepEquals, []interface{}{"test", "t", "p0", "0", "9"}) - c.Assert(append(rows[1][:3], rows[1][4:]...), DeepEquals, []interface{}{"test", "t", "p1", "0", "11"}) - - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t p0 a 0 4 2", - "test t p0 a 0 5 3", - "test t p0 c 0 1 3", - "test t p0 c 0 2 2", - "test t p1 a 0 16 4", - "test t p1 a 0 17 2", - "test t p1 c 0 13 2", - "test t p1 c 0 14 3")) - - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t p0 idx 1 1 3", - "test t p0 idx 1 2 2", - "test t p1 idx 1 13 2", - "test t p1 idx 1 14 3")) - - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 0").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t p0 a 0 0 2 1 1 2 0", - "test t p0 a 0 1 3 1 3 3 0", - "test t p0 c 0 0 3 1 3 5 0", - "test t p0 c 0 1 4 1 6 6 0", - "test t p1 a 0 0 3 1 11 13 0", - "test t p1 a 0 1 5 1 14 15 0", - "test t p1 c 0 0 4 1 7 10 0", - "test t p1 c 0 1 6 1 11 12 0")) - - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't' and is_index = 1").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t p0 idx 1 0 3 1 3 5 0", - "test t p0 idx 1 1 4 1 6 6 0", - "test t p1 idx 1 0 4 1 7 10 0", - "test t p1 idx 1 1 6 1 11 12 0")) - - tk.MustQuery("select table_id, is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms order by table_id, is_index, hist_id asc").Check( - testkit.Rows(fmt.Sprintf("%d 0 1 5 1 8 2 1", p0ID), // p0, a - fmt.Sprintf("%d 0 2 0 0 8 0 0", p0ID), // p0, b, not analyzed - fmt.Sprintf("%d 0 3 6 0 9 2 1", p0ID), // p0, c - fmt.Sprintf("%d 1 1 6 0 9 2 0", p0ID), // p0, idx - fmt.Sprintf("%d 0 1 7 0 11 2 1", p1ID), // p1, a - fmt.Sprintf("%d 0 2 0 0 11 0 0", p1ID), // p1, b, not analyzed - fmt.Sprintf("%d 0 3 8 0 11 2 1", p1ID), // p1, c - fmt.Sprintf("%d 1 1 8 0 11 2 0", p1ID), // p1, idx - )) -} - -func (s *testStatsSuite) TestAnalyzeColumnsWithExtendedStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("set @@tidb_enable_extended_stats = on") - tk.MustExec("create table t (a int, b int, c int)") - tk.MustExec("alter table t add stats_extended s1 correlation(b,c)") - tk.MustExec("insert into t values (5,1,1), (4,2,2), (3,3,3), (2,4,4), (1,5,5)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0][3], Equals, "b") - c.Assert(rows[1][3], Equals, "c") - - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 5")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t b 0 1 1", - "test t b 0 2 1", - "test t c 0 1 1", - "test t c 0 2 1")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 0 0 5 0 0", // column a is not analyzed - "0 2 5 0 5 2 1", - "0 3 5 0 5 2 1", - )) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t b 0 0 2 1 3 4 0", - "test t b 0 1 3 1 5 5 0", - "test t c 0 0 2 1 3 4 0", - "test t c 0 1 3 1 5 5 0")) - rows = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:len(rows[0])-1], DeepEquals, []interface{}{"test", "t", "s1", "[b,c]", "correlation", "1.000000"}) -} - -func (s *testStatsSuite) TestAnalyzeColumnsWithVirtualColumnIndex(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t (a int, b int, c int as (b+1), index idx(c))") - tk.MustExec("insert into t (a,b) values (1,1), (2,2), (3,3), (4,4), (5,4), (6,5), (7,5), (8,5), (null,null)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats.", - )) - // virtual column c is skipped when dumping stats into disk, so only the stats of column b are updated - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "b") - - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 9")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t b 0 4 2", - "test t b 0 5 3", - "test t idx 1 5 2", - "test t idx 1 6 3")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 0 0 0 0", // column a is not analyzed - "0 2 5 1 2 1", - "0 3 0 0 0 0", // column c is not analyzed - "1 1 5 1 2 0")) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t b 0 0 2 1 1 2 0", - "test t b 0 1 3 1 3 3 0", - "test t idx 1 0 2 1 2 3 0", - "test t idx 1 1 3 1 4 4 0")) -} - func (s *testStatsSuite) TestAnalyzeColumnsAfterAnalyzeAll(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) diff --git a/statistics/handle/update.go b/statistics/handle/update.go index c69ef18308db8..57593ae9838b2 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -871,8 +871,30 @@ func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.Query return errors.Trace(err) } +func (h *Handle) dumpColStatsUsage(tblColID model.TableColumnID, lastUsedAt time.Time) error { + se, err := h.pool.Get() + if err != nil { + return errors.Trace(err) + } + defer h.pool.Put(se) + sctx := se.(sessionctx.Context) + loc := sctx.GetSessionVars().Location() + // We need to use the time under the session's time zone because in the following sql, TiDB will convert the time from + // the session's time zone to UTC. + t := lastUsedAt.In(loc).Format(types.TimeFormat) + exec := se.(sqlexec.RestrictedSQLExecutor) + const sql = "INSERT INTO mysql.column_stats_usage (table_id, column_id, last_used_at) VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE last_used_at = CASE WHEN last_used_at IS NULL THEN %? ELSE GREATEST(last_used_at, %?) END" + if _, _, err := execRestrictedSQL(context.Background(), exec, sql, tblColID.TableID, tblColID.ColumnID, t, t, t); err != nil { + return errors.Trace(err) + } + return nil +} + // DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (h *Handle) DumpColStatsUsageToKV() error { + if !variable.EnableColumnTracking.Load() { + return nil + } h.sweepList() h.colMap.Lock() colMap := h.colMap.data @@ -883,13 +905,11 @@ func (h *Handle) DumpColStatsUsageToKV() error { h.colMap.data.merge(colMap) h.colMap.Unlock() }() - for col, lastUsedAt := range colMap { - const sql = "INSERT INTO mysql.column_stats_usage (table_id, column_id, last_used_at) VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE last_used_at = CASE WHEN last_used_at IS NULL THEN %? ELSE GREATEST(last_used_at, %?) END" - _, _, err := h.execRestrictedSQL(context.Background(), sql, col.TableID, col.ColumnID, lastUsedAt, lastUsedAt, lastUsedAt) - if err != nil { + for tblColID, lastUsedAt := range colMap { + if err := h.dumpColStatsUsage(tblColID, lastUsedAt); err != nil { return err } - delete(colMap, col) + delete(colMap, tblColID) } return nil } diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index d983f98f97e39..037093eee39d5 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -2341,11 +2341,11 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) - wideTableColumnCount := tk.MustQuery("select @@tidb_wide_table_column_count").Rows()[0][0].(string) - tk.MustExec("set global tidb_wide_table_column_count = 2") + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_wide_table_column_count = '%v'", wideTableColumnCount)) + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) }() + tk.MustExec("set global tidb_enable_column_tracking = 1") h := s.do.StatsHandle() tk.MustExec("use test") @@ -2356,7 +2356,6 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { tk.MustExec("insert into t2 values (1), (2), (3)") tk.MustExec("insert into t3 values (1, 2), (3, 4), (11, 12), (13, 14)") tk.MustExec("select * from t1 where a > 1") - tk.MustExec("select * from t2 where a > 1") c.Assert(h.DumpColStatsUsageToKV(), IsNil) // t1.a is collected as predicate column rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() @@ -2364,9 +2363,6 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) c.Assert(rows[0][4].(string) != "", IsTrue) c.Assert(rows[0][5].(string) == "", IsTrue) - // the number of column in t2 is less than wideTableColumnCount, so no predicate column is collected. - rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 0) tk.MustExec("analyze table t1") tk.MustExec("select * from t1 where b > 1") @@ -2391,7 +2387,7 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(h.DumpColStatsUsageToKV(), IsNil) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't3'").Rows() c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "global", "a"}) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t3", "global", "a"}) c.Assert(rows[0][4].(string) != "", IsTrue) c.Assert(rows[0][5].(string) == "", IsTrue) } @@ -2407,4 +2403,9 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "b"}) c.Assert(rows[0][4].(string) != "", IsTrue) c.Assert(rows[0][5].(string) == "", IsTrue) + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't2'").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t2", "", "a"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) == "", IsTrue) } From 8622eb0614fb8a9d69009b18f98906397c171f1e Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 27 Dec 2021 12:24:54 +0800 Subject: [PATCH 16/27] update some commets --- planner/core/collect_column_stats_usage.go | 4 ++-- planner/core/rule_predicate_push_down.go | 2 +- sessionctx/variable/tidb_vars.go | 3 ++- statistics/handle/handle.go | 4 ++++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index af75f449f2f45..6cd357a205257 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -64,6 +64,7 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector func (c *columnStatsUsageCollector) addPredicateColumn(col *expression.Column) { tblColIDs, ok := c.colMap[col.UniqueID] if !ok { + // It may happen if some leaf of logical plan is LogicalMemTable/LogicalShow/LogicalShowDDLJobs. return } for tblColID := range tblColIDs { @@ -85,6 +86,7 @@ func (c *columnStatsUsageCollector) updateColMap(col *expression.Column, related for _, relatedCol := range relatedCols { tblColIDs, ok := c.colMap[relatedCol.UniqueID] if !ok { + // It may happen if some leaf of logical plan is LogicalMemTable/LogicalShow/LogicalShowDDLJobs. continue } for tblColID := range tblColIDs { @@ -106,8 +108,6 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *Dat c.colMap[col.UniqueID] = map[model.TableColumnID]struct{}{tblColID: {}} } // We should use `pushedDownConds` here. `allConds` is used for partition pruning, which doesn't need stats. - // We still need to add predicate columns from `pushedDownConds` even if the number of columns is less than `wideTableColumnCount` - // because there may be some correlated columns in `pushedDownConds`. c.addPredicateColumnsFromExpressions(ds.pushedDownConds) } diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 5c6fc56a327ad..feed34d7ee567 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -404,7 +404,7 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression } for _, cond := range predicates { newFilter := expression.ColumnSubstitute(cond, p.Schema(), p.Exprs) - if !expression.HasGetSetVarFunc(newFilter) && expression.ExprFromSchema(newFilter, p.Children()[0].Schema()) { + if !expression.HasGetSetVarFunc(newFilter) { canBePushed = append(canBePushed, newFilter) } else { canNotBePushed = append(canNotBePushed, cond) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index f643e07830db7..ef3bf977b7c50 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -15,10 +15,11 @@ package variable import ( + "math" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" - "math" ) /* diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 4fa1ed14cac8a..d455c33e33c5a 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1867,6 +1867,8 @@ func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInf if err != nil { return nil, errors.Trace(err) } + // If `last_used_at` is before the time when `set global enable_column_tracking = 0`, we should ignore it because + // `set global enable_column_tracking = 0` indicates all the predicate columns collected before. if disableTime == nil || gt.After(*disableTime) { statsUsage.LastAnalyzedAt = &t } @@ -1930,6 +1932,8 @@ func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { if err != nil { return nil, errors.Trace(err) } + // If `last_used_at` is before the time when `set global enable_column_tracking = 0`, we don't regard the column as predicate column because + // `set global enable_column_tracking = 0` indicates all the predicate columns collected before. if disableTime == nil || gt.After(*disableTime) { columnIDs = append(columnIDs, colID) } From 3710bd78b1d3cfc61ae66ed5d7e768aca0f434ca Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Mon, 27 Dec 2021 21:51:54 +0800 Subject: [PATCH 17/27] refine time loc conversion --- executor/analyze_test.go | 100 ++++++++++++++++++++----------- executor/show_stats.go | 7 +-- sessionctx/variable/sysvar.go | 2 +- statistics/handle/handle.go | 62 +++++++------------ statistics/handle/update.go | 26 ++------ statistics/handle/update_test.go | 15 +++++ 6 files changed, 113 insertions(+), 99 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index a7f18020b3fd4..3f8dc6889ad51 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1462,40 +1462,6 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { } } -func TestAnalyzeColumnsError(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int)") - - // analyze version 1 doesn't support `ANALYZE COLUMNS c1, ..., cn`/`ANALYZE PREDICATE COLUMNS` currently - tk.MustExec("set @@tidb_analyze_version = 1") - err := tk.ExecToErr("analyze table t columns a") - require.Equal(t, "Only the analyze version 2 supports analyzing the specified columns", err.Error()) - err = tk.ExecToErr("analyze table t predicate columns") - require.Equal(t, "Only the analyze version 2 supports analyzing predicate columns", err.Error()) - - tk.MustExec("set @@tidb_analyze_version = 2") - // invalid column - err = tk.ExecToErr("analyze table t columns c") - terr := errors.Cause(err).(*terror.Error) - require.Equal(t, errors.ErrCode(errno.ErrAnalyzeMissColumn), terr.Code()) - - originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) - defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) - }() - tk.MustExec("set global tidb_enable_column_tracking = 1") - tk.MustExec("analyze table t predicate columns") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", - )) -} - func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { func(choice model.ColumnChoice) { @@ -1869,3 +1835,69 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { }(val) } } + +func TestAnalyzeColumnsError(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int)") + + // analyze version 1 doesn't support `ANALYZE COLUMNS c1, ..., cn`/`ANALYZE PREDICATE COLUMNS` currently + tk.MustExec("set @@tidb_analyze_version = 1") + err := tk.ExecToErr("analyze table t columns a") + require.Equal(t, "Only the analyze version 2 supports analyzing the specified columns", err.Error()) + err = tk.ExecToErr("analyze table t predicate columns") + require.Equal(t, "Only the analyze version 2 supports analyzing predicate columns", err.Error()) + + tk.MustExec("set @@tidb_analyze_version = 2") + // invalid column + err = tk.ExecToErr("analyze table t columns c") + terr := errors.Cause(err).(*terror.Error) + require.Equal(t, errors.ErrCode(errno.ErrAnalyzeMissColumn), terr.Code()) + + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("analyze table t predicate columns") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", + )) +} + +func TestEnableColumnTracking(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t1 (a int, b int)") + + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + tk.MustExec("analyze table t predicate columns") + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + + tk.MustExec("set global tidb_enable_column_tracking = 0") + tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) + +} diff --git a/executor/show_stats.go b/executor/show_stats.go index 7226e807c66eb..2068679f613c4 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -16,9 +16,6 @@ package executor import ( "fmt" - "sort" - "strings" - "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/ast" @@ -27,6 +24,8 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/tikv/client-go/v2/oracle" + "sort" + "strings" ) func (e *ShowExec) fetchShowStatsExtended() error { @@ -459,7 +458,7 @@ func (e *ShowExec) fetchShowAnalyzeStatus() { func (e *ShowExec) fetchShowColumnStatsUsage() error { do := domain.GetDomain(e.ctx) h := do.StatsHandle() - colStatsMap, err := h.LoadColumnStatsUsage() + colStatsMap, err := h.LoadColumnStatsUsage(e.ctx.GetSessionVars().Location()) if err != nil { return err } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0a542b458af3e..b5072d45cd607 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1331,7 +1331,7 @@ var defaultSysVars = []*SysVar{ v := TiDBOptOn(val) if !v { // Set the location to UTC to avoid time zone interference. - disableTime := time.Now().UTC().Format(types.TimeFormat) + disableTime := time.Now().UTC().Format(types.TimeFormat + " UTC") if err := setTiDBTableValue(s, TiDBDisableColumnTrackingTime, disableTime, "Record the last time tidb_enable_column_tracking is set off"); err != nil { return err } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index d455c33e33c5a..39ab30db16217 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -168,14 +168,6 @@ func (h *Handle) execRestrictedSQLWithSnapshot(ctx context.Context, sql string, }) } -func execRestrictedSQL(ctx context.Context, exec sqlexec.RestrictedSQLExecutor, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { - stmt, err := exec.ParseWithParamsInternal(ctx, sql, params...) - if err != nil { - return nil, nil, errors.Trace(err) - } - return exec.ExecRestrictedStmt(ctx, stmt) -} - // Clear the statsCache, only for test. func (h *Handle) Clear() { // TODO: Here h.mu seems to protect all the fields of Handle. Is is reasonable? @@ -1812,8 +1804,9 @@ type colStatsTimeInfo struct { LastAnalyzedAt *types.Time } -func getDisableColumnTrackingTime(exec sqlexec.RestrictedSQLExecutor) (*time.Time, error) { - rows, fields, err := execRestrictedSQL(context.Background(), exec, "SELECT variable_value FROM %n.%n WHERE variable_name = %?", mysql.SystemDB, mysql.TiDBTable, variable.TiDBDisableColumnTrackingTime) +// getDisableColumnTrackingTime reads the value of tidb_disable_column_tracking_time from mysql.tidb if it exists. +func (h *Handle) getDisableColumnTrackingTime() (*time.Time, error) { + rows, fields, err := h.execRestrictedSQL(context.Background(), "SELECT variable_value FROM %n.%n WHERE variable_name = %?", mysql.SystemDB, mysql.TiDBTable, variable.TiDBDisableColumnTrackingTime) if err != nil { return nil, err } @@ -1821,11 +1814,12 @@ func getDisableColumnTrackingTime(exec sqlexec.RestrictedSQLExecutor) (*time.Tim return nil, nil } d := rows[0].GetDatum(0, &fields[0].Column.FieldType) + // The string represents the UTC time when tidb_enable_column_tracking is set to 0. value, err := d.ToString() if err != nil { return nil, err } - t, err := time.Parse(types.TimeFormat, value) + t, err := time.Parse(types.TimeFormat+" UTC", value) if err != nil { return nil, err } @@ -1833,20 +1827,12 @@ func getDisableColumnTrackingTime(exec sqlexec.RestrictedSQLExecutor) (*time.Tim } // LoadColumnStatsUsage loads column stats usage information from disk. -func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInfo, error) { - se, err := h.pool.Get() +func (h *Handle) LoadColumnStatsUsage(loc *time.Location) (map[model.TableColumnID]colStatsTimeInfo, error) { + disableTime, err := h.getDisableColumnTrackingTime() if err != nil { return nil, errors.Trace(err) } - defer h.pool.Put(se) - sctx := se.(sessionctx.Context) - loc := sctx.GetSessionVars().Location() - exec := se.(sqlexec.RestrictedSQLExecutor) - disableTime, err := getDisableColumnTrackingTime(exec) - if err != nil { - return nil, errors.Trace(err) - } - rows, _, err := execRestrictedSQL(context.Background(), exec, "SELECT table_id, column_id, last_used_at, last_analyzed_at FROM mysql.column_stats_usage") + rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT table_id, column_id, CONVERT_TZ(last_used_at, @@TIME_ZONE, '+00:00'), CONVERT_TZ(last_analyzed_at, @@TIME_ZONE, '+00:00') FROM mysql.column_stats_usage") if err != nil { return nil, errors.Trace(err) } @@ -1859,19 +1845,25 @@ func (h *Handle) LoadColumnStatsUsage() (map[model.TableColumnID]colStatsTimeInf var statsUsage colStatsTimeInfo if !row.IsNull(2) { t := row.GetTime(2) - statsUsage.LastUsedAt = &t - } - if !row.IsNull(3) { - t := row.GetTime(3) - gt, err := t.GoTime(loc) + gt, err := t.GoTime(time.UTC) if err != nil { return nil, errors.Trace(err) } // If `last_used_at` is before the time when `set global enable_column_tracking = 0`, we should ignore it because // `set global enable_column_tracking = 0` indicates all the predicate columns collected before. if disableTime == nil || gt.After(*disableTime) { - statsUsage.LastAnalyzedAt = &t + if err := t.ConvertTimeZone(time.UTC, loc); err != nil { + return nil, errors.Trace(err) + } + statsUsage.LastUsedAt = &t + } + } + if !row.IsNull(3) { + t := row.GetTime(3) + if err := t.ConvertTimeZone(time.UTC, loc); err != nil { + return nil, errors.Trace(err) } + statsUsage.LastAnalyzedAt = &t } colStatsMap[tblColID] = statsUsage } @@ -1905,19 +1897,11 @@ func (h *Handle) CollectColumnsInExtendedStats(tableID int64) ([]int64, error) { // GetPredicateColumns returns IDs of predicate columns, which are the columns whose stats are used(needed) when generating query plans. func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { - se, err := h.pool.Get() - if err != nil { - return nil, errors.Trace(err) - } - defer h.pool.Put(se) - sctx := se.(sessionctx.Context) - loc := sctx.GetSessionVars().Location() - exec := se.(sqlexec.RestrictedSQLExecutor) - disableTime, err := getDisableColumnTrackingTime(exec) + disableTime, err := h.getDisableColumnTrackingTime() if err != nil { return nil, errors.Trace(err) } - rows, _, err := execRestrictedSQL(context.Background(), exec, "SELECT column_id, last_used_at FROM mysql.column_stats_usage WHERE table_id = %? AND last_used_at IS NOT NULL", tableID) + rows, _, err := h.execRestrictedSQL(context.Background(), "SELECT column_id, CONVERT_TZ(last_used_at, @@TIME_ZONE, '+00:00') FROM mysql.column_stats_usage WHERE table_id = %? AND last_used_at IS NOT NULL", tableID) if err != nil { return nil, errors.Trace(err) } @@ -1928,7 +1912,7 @@ func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { } colID := row.GetInt64(0) t := row.GetTime(1) - gt, err := t.GoTime(loc) + gt, err := t.GoTime(time.UTC) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 57593ae9838b2..59ff1fc716694 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -871,25 +871,6 @@ func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.Query return errors.Trace(err) } -func (h *Handle) dumpColStatsUsage(tblColID model.TableColumnID, lastUsedAt time.Time) error { - se, err := h.pool.Get() - if err != nil { - return errors.Trace(err) - } - defer h.pool.Put(se) - sctx := se.(sessionctx.Context) - loc := sctx.GetSessionVars().Location() - // We need to use the time under the session's time zone because in the following sql, TiDB will convert the time from - // the session's time zone to UTC. - t := lastUsedAt.In(loc).Format(types.TimeFormat) - exec := se.(sqlexec.RestrictedSQLExecutor) - const sql = "INSERT INTO mysql.column_stats_usage (table_id, column_id, last_used_at) VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE last_used_at = CASE WHEN last_used_at IS NULL THEN %? ELSE GREATEST(last_used_at, %?) END" - if _, _, err := execRestrictedSQL(context.Background(), exec, sql, tblColID.TableID, tblColID.ColumnID, t, t, t); err != nil { - return errors.Trace(err) - } - return nil -} - // DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (h *Handle) DumpColStatsUsageToKV() error { if !variable.EnableColumnTracking.Load() { @@ -906,8 +887,11 @@ func (h *Handle) DumpColStatsUsageToKV() error { h.colMap.Unlock() }() for tblColID, lastUsedAt := range colMap { - if err := h.dumpColStatsUsage(tblColID, lastUsedAt); err != nil { - return err + // TODO: maybe there is a better way to insert UTC timestamp. + t := lastUsedAt.UTC().Format(types.TimeFormat) + const sql = "INSERT INTO mysql.column_stats_usage (table_id, column_id, last_used_at) VALUES (%?, %?, CONVERT_TZ(%?, '+00:00', @@TIME_ZONE)) ON DUPLICATE KEY UPDATE last_used_at = CASE WHEN last_used_at IS NULL THEN CONVERT_TZ(%?, '+00:00', @@TIME_ZONE) ELSE GREATEST(last_used_at, CONVERT_TZ(%?, '+00:00', @@TIME_ZONE)) END" + if _, _, err := h.execRestrictedSQL(context.Background(), sql, tblColID.TableID, tblColID.ColumnID, t, t, t); err != nil { + return errors.Trace(err) } delete(colMap, tblColID) } diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 037093eee39d5..26b912f0666ab 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -2408,4 +2408,19 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t2", "", "a"}) c.Assert(rows[0][4].(string) != "", IsTrue) c.Assert(rows[0][5].(string) == "", IsTrue) + + // Test prepare and execute. + tk.MustExec("delete from mysql.column_stats_usage") + tk.MustExec("prepare stmt from 'select * from t1 where a > ?'") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + // Prepare only converts sql string to ast and doesn't do optimization, so no predicate column is collected. + tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Check(testkit.Rows()) + tk.MustExec("set @p = 1") + tk.MustExec("execute stmt using @p") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) + c.Assert(rows[0][4].(string) != "", IsTrue) + c.Assert(rows[0][5].(string) == "", IsTrue) } From e23387961faa1143c3565f44a1cb2669854ebcf3 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Tue, 28 Dec 2021 00:41:29 +0800 Subject: [PATCH 18/27] upd tests --- executor/analyze_test.go | 42 +++--------------------------- sessionctx/variable/tidb_vars.go | 2 +- statistics/handle/update_test.go | 44 ++++++++++++++++++++++++++++++++ 3 files changed, 49 insertions(+), 39 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 3f8dc6889ad51..112efede345d1 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1836,7 +1836,7 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { } } -func TestAnalyzeColumnsError(t *testing.T) { +func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -1858,46 +1858,12 @@ func TestAnalyzeColumnsError(t *testing.T) { terr := errors.Cause(err).(*terror.Error) require.Equal(t, errors.ErrCode(errno.ErrAnalyzeMissColumn), terr.Code()) - originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) - defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) - }() - tk.MustExec("set global tidb_enable_column_tracking = 1") + // If no predicate column is collected, analyze predicate columns gives a warning and falls back to analyze all columns. tk.MustExec("analyze table t predicate columns") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", )) -} - -func TestEnableColumnTracking(t *testing.T) { - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t1 (a int, b int)") - - originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) - defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) - }() - - tk.MustExec("set global tidb_enable_column_tracking = 1") - tk.MustExec("select * from t where b > 1") - require.NoError(t, h.DumpColStatsUsageToKV()) - rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() - require.Equal(t, 1, len(rows)) - require.Equal(t, "b", rows[0][3]) - tk.MustExec("analyze table t predicate columns") - rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() - require.Equal(t, 1, len(rows)) - require.Equal(t, "b", rows[0][3]) - - tk.MustExec("set global tidb_enable_column_tracking = 0") - tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) - + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() + require.Equal(t, 2, len(rows)) } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 16ca63ff236ec..5550cc9b52915 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -786,7 +786,7 @@ const ( DefEnablePlacementCheck = true DefTimestamp = "0" DefTiDBEnableIndexMerge = true - DefTiDBEnableColumnTracking = true + DefTiDBEnableColumnTracking = true ) // Process global variables. diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 26b912f0666ab..2d8c27e7ef249 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -2424,3 +2424,47 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { c.Assert(rows[0][4].(string) != "", IsTrue) c.Assert(rows[0][5].(string) == "", IsTrue) } + +func (s *testSerialStatsSuite) TestEnableAndDisableColumnTracking(c *C) { + defer cleanEnv(c, s.store, s.do) + tk := testkit.NewTestKit(c, s.store) + h := s.do.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, c int)") + + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][3], Equals, "b") + + tk.MustExec("set global tidb_enable_column_tracking = 0") + // After tidb_enable_column_tracking is set to 0, the predicate columns collected before are invalidated. + tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) + + // Sleep for 1.5s to let `last_used_at` be larger than `tidb_disable_tracking_time`. + time.Sleep(1500 * time.Millisecond) + tk.MustExec("select * from t where a > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + // We don't collect predicate columns when tidb_enable_column_tracking = 0 + tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) + + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b < 1 and c > 1") + c.Assert(h.DumpColStatsUsageToKV(), IsNil) + rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Sort().Rows() + c.Assert(len(rows), Equals, 2) + c.Assert(rows[0][3], Equals, "b") + c.Assert(rows[1][3], Equals, "c") + + // Test invalidating predicate columns again in order to check that tidb_disable_tracking_time can be updated. + tk.MustExec("set global tidb_enable_column_tracking = 0") + tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) +} From b86cb437e6705ddfd4ee76aeb23e4b0998340772 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Tue, 28 Dec 2021 11:26:05 +0800 Subject: [PATCH 19/27] fix ut and address comment --- domain/domain.go | 3 +-- executor/show_stats.go | 5 +++-- planner/core/collect_column_stats_usage.go | 15 +++++++-------- planner/core/collect_column_stats_usage_test.go | 4 ++-- statistics/handle/handle.go | 12 +++++------- 5 files changed, 18 insertions(+), 21 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 45f982ecfb81d..e6931f4d8e4ba 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1390,8 +1390,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) gcStatsTicker := time.NewTicker(100 * lease) dumpFeedbackTicker := time.NewTicker(200 * lease) loadFeedbackTicker := time.NewTicker(5 * lease) - //dumpColStatsUsageTicker := time.NewTicker(20 * lease) - dumpColStatsUsageTicker := time.NewTicker(lease) // TODO: Set to 1 * lease for easily debugging. Remember to restore the default value. + dumpColStatsUsageTicker := time.NewTicker(20 * lease) statsHandle := do.StatsHandle() defer func() { loadFeedbackTicker.Stop() diff --git a/executor/show_stats.go b/executor/show_stats.go index 2068679f613c4..cea7fdc13b8fe 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -16,6 +16,9 @@ package executor import ( "fmt" + "sort" + "strings" + "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/ast" @@ -24,8 +27,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/tikv/client-go/v2/oracle" - "sort" - "strings" ) func (e *ShowExec) fetchShowStatsExtended() error { diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 6cd357a205257..e6f9e7ebed2b6 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -272,14 +272,6 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp LogicalPlan) { } } -func set2slice(set map[model.TableColumnID]struct{}) []model.TableColumnID { - ret := make([]model.TableColumnID, 0, len(set)) - for tblColID := range set { - ret = append(ret, tblColID) - } - return ret -} - // CollectColumnStatsUsage collects column stats usage from logical plan. // predicate indicates whether to collect predicate columns and histNeeded indicates whether to collect histogram-needed columns. // The first return value is predicate columns(nil if predicate is false) and the second return value is histogram-needed columns(nil if histNeeded is false). @@ -293,6 +285,13 @@ func CollectColumnStatsUsage(lp LogicalPlan, predicate, histNeeded bool) ([]mode } collector := newColumnStatsUsageCollector(mode) collector.collectFromPlan(lp) + set2slice := func(set map[model.TableColumnID]struct{}) []model.TableColumnID { + ret := make([]model.TableColumnID, 0, len(set)) + for tblColID := range set { + ret = append(ret, tblColID) + } + return ret + } var predicateCols, histNeededCols []model.TableColumnID if predicate { predicateCols = set2slice(collector.predicateCols) diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 43131e056c818..56ce5b9934388 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -62,9 +62,9 @@ func getColumnName(c *C, is infoschema.InfoSchema, tblColID model.TableColumnID, return "", false } -func checkColumnStatsUsage(c *C, is infoschema.InfoSchema, lp LogicalPlan, onlyHistNeeded bool, expected []string, comment CommentInterface) { +func checkColumnStatsUsage(c *C, is infoschema.InfoSchema, lp LogicalPlan, histNeededOnly bool, expected []string, comment CommentInterface) { var tblColIDs []model.TableColumnID - if onlyHistNeeded { + if histNeededOnly { _, tblColIDs = CollectColumnStatsUsage(lp, false, true) } else { tblColIDs, _ = CollectColumnStatsUsage(lp, true, false) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 39ab30db16217..19602fff18172 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1852,17 +1852,16 @@ func (h *Handle) LoadColumnStatsUsage(loc *time.Location) (map[model.TableColumn // If `last_used_at` is before the time when `set global enable_column_tracking = 0`, we should ignore it because // `set global enable_column_tracking = 0` indicates all the predicate columns collected before. if disableTime == nil || gt.After(*disableTime) { - if err := t.ConvertTimeZone(time.UTC, loc); err != nil { - return nil, errors.Trace(err) - } + t = types.NewTime(types.FromGoTime(gt.In(loc)), mysql.TypeTimestamp, types.DefaultFsp) statsUsage.LastUsedAt = &t } } if !row.IsNull(3) { - t := row.GetTime(3) - if err := t.ConvertTimeZone(time.UTC, loc); err != nil { + gt, err := row.GetTime(3).GoTime(time.UTC) + if err != nil { return nil, errors.Trace(err) } + t := types.NewTime(types.FromGoTime(gt.In(loc)), mysql.TypeTimestamp, types.DefaultFsp) statsUsage.LastAnalyzedAt = &t } colStatsMap[tblColID] = statsUsage @@ -1911,8 +1910,7 @@ func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { continue } colID := row.GetInt64(0) - t := row.GetTime(1) - gt, err := t.GoTime(time.UTC) + gt, err := row.GetTime(1).GoTime(time.UTC) if err != nil { return nil, errors.Trace(err) } From 84451c0f80e1c1cc6df6bb8496e2812883809fb5 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Tue, 28 Dec 2021 11:34:24 +0800 Subject: [PATCH 20/27] fmt import --- executor/analyze_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 112efede345d1..37ab7ebd61c00 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -17,9 +17,6 @@ package executor_test import ( "context" "fmt" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/parser/terror" "strconv" "strings" "sync" @@ -27,14 +24,17 @@ import ( "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" From fe56889e86cba19cacf94b3d5ce0e8e2e9758201 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 29 Dec 2021 00:55:45 +0800 Subject: [PATCH 21/27] deal with the case when old stats is analyze version 1 --- executor/analyze_test.go | 43 +++++++++++++++++++++++++++++-------- planner/core/planbuilder.go | 19 ++++++++++++---- 2 files changed, 49 insertions(+), 13 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 5454c86c76597..bb6f2259d4994 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1287,7 +1287,7 @@ func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { require.NoError(t, err) tblID := tbl.Meta().ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1356,7 +1356,7 @@ func TestAnalyzeColumnsWithIndex(t *testing.T) { require.NoError(t, err) tblID := tbl.Meta().ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1434,7 +1434,7 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { require.NoError(t, err) tblID := tbl.Meta().ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1516,7 +1516,7 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { p0ID := defs[0].ID p1ID := defs[1].ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1641,7 +1641,7 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { p0ID := defs[0].ID p1ID := defs[1].ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1747,7 +1747,7 @@ func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { require.NoError(t, err) tblID := tbl.Meta().ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1818,7 +1818,7 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { require.NoError(t, err) tblID := tbl.Meta().ID - switch val { + switch choice { case model.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( @@ -1867,7 +1867,7 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { } func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -1892,8 +1892,33 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { tk.MustExec("analyze table t predicate columns") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", - "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", + "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed.", )) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() require.Equal(t, 2, len(rows)) + + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + tk.MustExec("set @@tidb_analyze_version = 1") + tk.MustExec("analyze table t") + tk.MustExec("set @@tidb_analyze_version = 2") + switch choice { + case model.ColumnList: + tk.MustExec("analyze table t columns b") + case model.PredicateColumns: + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + require.NoError(t, dom.StatsHandle().DumpColStatsUsageToKV()) + tk.MustExec("analyze table t predicate columns") + } + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.", + "Warning 1105 Table test.t has version 1 statistics so all the columns must be analyzed to overwrite the current statistics.", + )) + }(val) + } } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 67b1a274cc3cb..15ff88288596a 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1818,10 +1818,21 @@ func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] // 1. For `ANALYZE TABLE t PREDICATE COLUMNS`, it returns union of the predicate columns and the columns in index/primary key/extended stats. // 2. For `ANALYZE TABLE t COLUMNS c1, c2, ..., cn`, it returns union of the specified columns(c1, c2, ..., cn) and the columns in index/primary key/extended stats. // 3. Otherwise it returns all the columns. -func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.TableName) ([]*model.ColumnInfo, error) { +func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.TableName, physicalIDs []int64, version int) ([]*model.ColumnInfo, error) { + choice := as.ColumnChoice + if choice == model.PredicateColumns || choice == model.ColumnList { + statsHandle := domain.GetDomain(b.ctx).StatsHandle() + versionIsSame := statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &version) + if !versionIsSame { + // If @@tidb_analyze_version is 2 while the current statistics of the table is version 1, we cannot just use PredicateColumns or ColumnList, + // which may cause different analyze versions among the columns. Hence we fall back to AllColumns. + choice = model.AllColumns + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics.", tbl.Schema.L, tbl.Name.L)) + } + } tblInfo := tbl.TableInfo columnIDs := make(map[int64]struct{}, len(tblInfo.Columns)) - switch as.ColumnChoice { + switch choice { case model.DefaultChoice: // TODO: use analyze column config for DefaultChoice return tblInfo.Columns, nil @@ -1835,7 +1846,7 @@ func (b *PlanBuilder) getAnalyzeColumnsInfo(as *ast.AnalyzeTableStmt, tbl *ast.T return nil, err } if len(cols) == 0 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed", tbl.Schema.L, tbl.Name.L)) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed.", tbl.Schema.L, tbl.Name.L)) return tblInfo.Columns, nil } for _, id := range cols { @@ -1979,7 +1990,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( if as.Incremental { b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling")) } - colsInfo, err := b.getAnalyzeColumnsInfo(as, tbl) + colsInfo, err := b.getAnalyzeColumnsInfo(as, tbl, physicalIDs, version) if err != nil { return nil, err } From f298badb640bff90d4ddafce3b271853cda04565 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 29 Dec 2021 16:02:56 +0800 Subject: [PATCH 22/27] upd tests --- executor/set_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/executor/set_test.go b/executor/set_test.go index d8e26ab3253d5..c0d9bc1f9dfbf 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -597,6 +597,9 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustQuery("select count(1) from mysql.tidb where variable_name = 'tidb_disable_column_tracking_time' and variable_value is not null").Check(testkit.Rows("1")) tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustQuery("select @@tidb_enable_column_tracking").Check(testkit.Rows("1")) + c.Assert(tk.ExecToErr("select @@session.tidb_enable_column_tracking"), NotNil) + c.Assert(tk.ExecToErr("set tidb_enable_column_tracking = 0"), NotNil) + c.Assert(tk.ExecToErr("set global tidb_enable_column_tracking = -1"), NotNil) } func (s *testSuite5) TestTruncateIncorrectIntSessionVar(c *C) { From d1faef962ad04e4da98560e41dd4bbe1f7b08d96 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Wed, 29 Dec 2021 17:38:23 +0800 Subject: [PATCH 23/27] use UTCTimeFormat --- sessionctx/variable/sysvar.go | 2 +- statistics/handle/handle.go | 2 +- types/time.go | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 62bb555f8a19c..936e7f8c1af30 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1331,7 +1331,7 @@ var defaultSysVars = []*SysVar{ v := TiDBOptOn(val) if !v { // Set the location to UTC to avoid time zone interference. - disableTime := time.Now().UTC().Format(types.TimeFormat + " UTC") + disableTime := time.Now().UTC().Format(types.UTCTimeFormat) if err := setTiDBTableValue(s, TiDBDisableColumnTrackingTime, disableTime, "Record the last time tidb_enable_column_tracking is set off"); err != nil { return err } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index ede40b92bcf13..aa9fa33edeeb9 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1819,7 +1819,7 @@ func (h *Handle) getDisableColumnTrackingTime() (*time.Time, error) { if err != nil { return nil, err } - t, err := time.Parse(types.TimeFormat+" UTC", value) + t, err := time.Parse(types.UTCTimeFormat, value) if err != nil { return nil, err } diff --git a/types/time.go b/types/time.go index ada6b21f2135f..3218195a0c625 100644 --- a/types/time.go +++ b/types/time.go @@ -39,6 +39,8 @@ const ( TimeFormat = "2006-01-02 15:04:05" // TimeFSPFormat is time format with fractional seconds precision. TimeFSPFormat = "2006-01-02 15:04:05.000000" + // UTCTimeFormat is used to parse and format gotime. + UTCTimeFormat = "2006-01-02 15:04:05 UTC" ) const ( From 46c91a67b360970d41dc1450d54c063258daf143 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 30 Dec 2021 06:23:44 +0800 Subject: [PATCH 24/27] remove used --- planner/core/planbuilder.go | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 0e3366e6508a0..1271cc61f045b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2246,25 +2246,6 @@ func mergeColumnList(choice1 model.ColumnChoice, list1 []*model.ColumnInfo, choi return choice2, list2 } -func (b *PlanBuilder) getFinalAnalyzeColList(choice model.ColumnChoice, list []*model.ColumnInfo, predicateCols, mustAnalyzedCols *calcOnceMap, tbl *ast.TableName) ([]*model.ColumnInfo, []*model.ColumnInfo, error) { - fullColumns := tbl.TableInfo.Cols() - emptyColumns := make([]*model.ColumnInfo, 0) - switch choice { - case model.AllColumns: - return fullColumns, emptyColumns, nil - case model.ColumnList: - list, err := b.getFullAnalyzeColumnsInfo(list, tbl, false) - if err != nil { - return nil, nil, err - } - return list, list, nil - case model.PredicateColumns: // TODO - return fullColumns, emptyColumns, nil - default: - return fullColumns, emptyColumns, nil - } -} - func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { p := &Analyze{Opts: opts} p.OptionsMap = make(map[int64]V2AnalyzeOptions) From 35d24b4fdce981afa6ae3fd967a487d5523d7b17 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 30 Dec 2021 06:30:25 +0800 Subject: [PATCH 25/27] fix build --- planner/core/planbuilder.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1271cc61f045b..f6972d4a2ded6 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1882,19 +1882,19 @@ func (b *PlanBuilder) getMustAnalyzedColumns(tbl *ast.TableName, cols *calcOnceM return cols.data, nil } -func (b *PlanBuilder) getPredicateColumns(tblInfo *model.TableInfo, cols *calcOnceMap) (map[int64]struct{}, error) { +func (b *PlanBuilder) getPredicateColumns(tbl *ast.TableName, cols *calcOnceMap) (map[int64]struct{}, error) { if cols.calculated { return cols.data, nil } do := domain.GetDomain(b.ctx) h := do.StatsHandle() - colList, err := h.GetPredicateColumns(tblInfo.ID) + colList, err := h.GetPredicateColumns(tbl.TableInfo.ID) if err != nil { return nil, err } if len(colList) == 0 { b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed.", tbl.Schema.L, tbl.Name.L)) - for _, colInfo := range tblInfo.Columns { + for _, colInfo := range tbl.TableInfo.Columns { cols.data[colInfo.ID] = struct{}{} } } else { @@ -1941,7 +1941,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( case model.DefaultChoice, model.AllColumns: return tbl.TableInfo.Columns, nil, nil case model.PredicateColumns: - predicate, err := b.getPredicateColumns(tbl.TableInfo, predicateCols) + predicate, err := b.getPredicateColumns(tbl, predicateCols) if err != nil { return nil, nil, err } @@ -1989,6 +1989,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( colList := colSet2colList(colSet) return colList, colList, nil } + return nil, nil, nil } func getColOffsetForAnalyze(colsInfo []*model.ColumnInfo, colID int64) int { From ae9185f49debcbd7cc176e6184e68cff6158b1a1 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 30 Dec 2021 07:08:25 +0800 Subject: [PATCH 26/27] fix --- planner/core/planbuilder.go | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f6972d4a2ded6..32306c6b5c4bf 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1886,15 +1886,17 @@ func (b *PlanBuilder) getPredicateColumns(tbl *ast.TableName, cols *calcOnceMap) if cols.calculated { return cols.data, nil } + tblInfo := tbl.TableInfo + cols.data = make(map[int64]struct{}, len(tblInfo.Columns)) do := domain.GetDomain(b.ctx) h := do.StatsHandle() - colList, err := h.GetPredicateColumns(tbl.TableInfo.ID) + colList, err := h.GetPredicateColumns(tblInfo.ID) if err != nil { return nil, err } if len(colList) == 0 { b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("No predicate column has been collected yet for table %s.%s so all columns are analyzed.", tbl.Schema.L, tbl.Name.L)) - for _, colInfo := range tbl.TableInfo.Columns { + for _, colInfo := range tblInfo.Columns { cols.data[colInfo.ID] = struct{}{} } } else { @@ -1926,8 +1928,12 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( columnChoice model.ColumnChoice, specifiedCols []*model.ColumnInfo, predicateCols, mustAnalyzedCols *calcOnceMap, + mustAllColumns bool, warning bool, ) ([]*model.ColumnInfo, []*model.ColumnInfo, error) { + if mustAllColumns && warning && (columnChoice == model.PredicateColumns || columnChoice == model.ColumnList) { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics.", tbl.Schema.L, tbl.Name.L)) + } colSet2colList := func(colSet map[int64]struct{}) []*model.ColumnInfo { colList := make([]*model.ColumnInfo, 0, len(colSet)) for _, colInfo := range tbl.TableInfo.Columns { @@ -1941,6 +1947,9 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( case model.DefaultChoice, model.AllColumns: return tbl.TableInfo.Columns, nil, nil case model.PredicateColumns: + if mustAllColumns { + return tbl.TableInfo.Columns, nil, nil + } predicate, err := b.getPredicateColumns(tbl, predicateCols) if err != nil { return nil, nil, err @@ -1987,6 +1996,9 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( colSet[colID] = struct{}{} } colList := colSet2colList(colSet) + if mustAllColumns { + return tbl.TableInfo.Columns, colList, nil + } return colList, colList, nil } return nil, nil, nil @@ -2051,12 +2063,16 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( return nil, err } var predicateCols, mustAnalyzedCols calcOnceMap - astColsInfo, _, err := b.getFullAnalyzeColumnsInfo(tbl, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols, true) + ver := version + statsHandle := domain.GetDomain(b.ctx).StatsHandle() + // If the statistics of the table is version 1, we must analyze all columns to overwrites all of old statistics. + mustAllColumns := !statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &ver) + astColsInfo, _, err := b.getFullAnalyzeColumnsInfo(tbl, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols, mustAllColumns, true) if err != nil { return nil, err } isAnalyzeTable := len(as.PartitionNames) == 0 - optionsMap, colsInfoMap, err := b.genV2AnalyzeOptions(persistOpts, tbl, isAnalyzeTable, physicalIDs, astOpts, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols) + optionsMap, colsInfoMap, err := b.genV2AnalyzeOptions(persistOpts, tbl, isAnalyzeTable, physicalIDs, astOpts, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols, mustAllColumns) if err != nil { return nil, err } @@ -2113,6 +2129,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( astColChoice model.ColumnChoice, astColList []*model.ColumnInfo, predicateCols, mustAnalyzedCols *calcOnceMap, + mustAllColumns bool, ) (map[int64]V2AnalyzeOptions, map[int64][]*model.ColumnInfo, error) { optionsMap := make(map[int64]V2AnalyzeOptions, len(physicalIDs)) colsInfoMap := make(map[int64][]*model.ColumnInfo, len(physicalIDs)) @@ -2131,7 +2148,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( tblColChoice, tblColList = mergeColumnList(astColChoice, astColList, tblSavedColChoice, tblSavedColList) } tblFilledOpts := fillAnalyzeOptionsV2(tblOpts) - tblColsInfo, tblColList, err := b.getFullAnalyzeColumnsInfo(tbl, tblColChoice, tblColList, predicateCols, mustAnalyzedCols, false) + tblColsInfo, tblColList, err := b.getFullAnalyzeColumnsInfo(tbl, tblColChoice, tblColList, predicateCols, mustAnalyzedCols, mustAllColumns, false) if err != nil { return nil, nil, err } @@ -2157,7 +2174,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( mergedOpts := mergeAnalyzeOptions(astOpts, savedOpts) filledMergedOpts := fillAnalyzeOptionsV2(mergedOpts) finalColChoice, mergedColList := mergeColumnList(astColChoice, astColList, savedColChoice, savedColList) - finalColsInfo, finalColList, err := b.getFullAnalyzeColumnsInfo(tbl, finalColChoice, mergedColList, predicateCols, mustAnalyzedCols, false) + finalColsInfo, finalColList, err := b.getFullAnalyzeColumnsInfo(tbl, finalColChoice, mergedColList, predicateCols, mustAnalyzedCols, mustAllColumns, false) if err != nil { return nil, nil, err } @@ -2282,7 +2299,6 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A } continue } - // TODO: deal with as.ColumnChoice == model.DefaultChoice if as.ColumnChoice == model.PredicateColumns { return nil, errors.Errorf("Only the analyze version 2 supports analyzing predicate columns") } From f00e4a372dd485f24066293ca73a538faff40372 Mon Sep 17 00:00:00 2001 From: xuyifan <675434007@qq.com> Date: Thu, 30 Dec 2021 14:09:48 +0800 Subject: [PATCH 27/27] add more tests --- executor/analyze_test.go | 178 +++++++++++++++++++++++++++++++ statistics/handle/handle_test.go | 56 ---------- 2 files changed, 178 insertions(+), 56 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 06b0d6b9e4ee5..de35933562e5a 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1751,6 +1751,105 @@ func TestSavedAnalyzeOptionsForMultipleTables(t *testing.T) { require.Equal(t, "2", rs.Rows()[0][4]) } +func TestSavedAnalyzeColumnOptions(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + originalVal1 := tk.MustQuery("select @@tidb_persist_analyze_options").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_persist_analyze_options = %v", originalVal1)) + }() + tk.MustExec("set global tidb_persist_analyze_options = true") + originalVal2 := tk.MustQuery("select @@tidb_auto_analyze_ratio").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2)) + }() + tk.MustExec("set global tidb_auto_analyze_ratio = 0.01") + originalVal3 := handle.AutoAnalyzeMinCnt + defer func() { + handle.AutoAnalyzeMinCnt = originalVal3 + }() + handle.AutoAnalyzeMinCnt = 0 + originalVal4 := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal4)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4)") + + h := dom.StatsHandle() + oriLease := h.Lease() + h.SetLease(1) + defer func() { + h.SetLease(oriLease) + }() + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.Nil(t, err) + tblInfo := tbl.Meta() + tk.MustExec("select * from t where b > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + tk.MustExec("analyze table t predicate columns") + require.NoError(t, h.LoadNeededHistograms()) + tblStats := h.GetTableStats(tblInfo) + lastVersion := tblStats.Version + // column b is analyzed + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[0].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[1].ID].LastUpdateVersion) + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[2].ID].LastUpdateVersion) + tk.MustQuery(fmt.Sprintf("select column_choice, column_ids from mysql.analyze_options where table_id = %v", tblInfo.ID)).Check(testkit.Rows("PREDICATE ")) + + tk.MustExec("select * from t where c > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + // manually analyze uses the saved option(predicate columns). + tk.MustExec("analyze table t") + require.NoError(t, h.LoadNeededHistograms()) + tblStats = h.GetTableStats(tblInfo) + require.Less(t, lastVersion, tblStats.Version) + lastVersion = tblStats.Version + // column b, c are analyzed + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[0].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[1].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[2].ID].LastUpdateVersion) + + tk.MustExec("insert into t values (5,5,5),(6,6,6)") + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, h.Update(is)) + // auto analyze uses the saved option(predicate columns). + h.HandleAutoAnalyze(is) + tblStats = h.GetTableStats(tblInfo) + require.Less(t, lastVersion, tblStats.Version) + lastVersion = tblStats.Version + // column b, c are analyzed + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[0].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[1].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[2].ID].LastUpdateVersion) + + tk.MustExec("analyze table t columns a") + tblStats = h.GetTableStats(tblInfo) + require.Less(t, lastVersion, tblStats.Version) + lastVersion = tblStats.Version + // column a is analyzed + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[0].ID].LastUpdateVersion) + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[1].ID].LastUpdateVersion) + require.Greater(t, lastVersion, tblStats.Columns[tblInfo.Columns[2].ID].LastUpdateVersion) + tk.MustQuery(fmt.Sprintf("select column_choice, column_ids from mysql.analyze_options where table_id = %v", tblInfo.ID)).Check(testkit.Rows(fmt.Sprintf("LIST %v", tblInfo.Columns[0].ID))) + + tk.MustExec("analyze table t all columns") + tblStats = h.GetTableStats(tblInfo) + require.Less(t, lastVersion, tblStats.Version) + lastVersion = tblStats.Version + // column a, b, c are analyzed + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[0].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[1].ID].LastUpdateVersion) + require.Equal(t, lastVersion, tblStats.Columns[tblInfo.Columns[2].ID].LastUpdateVersion) + tk.MustQuery(fmt.Sprintf("select column_choice, column_ids from mysql.analyze_options where table_id = %v", tblInfo.ID)).Check(testkit.Rows("ALL ")) +} + func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { func(choice model.ColumnChoice) { @@ -2350,6 +2449,85 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { } } +func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { + for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { + func(choice model.ColumnChoice) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("create table t (a int, b int)") + tk.MustExec("insert into t (a,b) values (1,1), (1,1), (2,2), (2,2), (3,3), (4,4)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblID := tbl.Meta().ID + + tk.MustExec("analyze table t with 2 topn, 2 buckets") + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 6")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t a 0 1 2", + "test t a 0 2 2", + "test t b 0 1 2", + "test t b 0 2 2")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 4 0 6 2 1", + "0 2 4 0 6 2 1")) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t a 0 0 2 1 3 4 0", + "test t b 0 0 2 1 3 4 0")) + + tk.MustExec("insert into t (a,b) values (1,1), (6,6)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + + switch choice { + case model.ColumnList: + tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") + case model.PredicateColumns: + originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) + }() + tk.MustExec("set global tidb_enable_column_tracking = 1") + tk.MustExec("select * from t where b > 1") + require.NoError(t, h.DumpColStatsUsageToKV()) + rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, "b", rows[0][3]) + tk.MustExec("analyze table t predicate columns with 2 topn, 2 buckets") + } + + // Column a is not analyzed in second ANALYZE. We keep the outdated stats of column a rather than delete them. + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 8")) + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_idx, value, count + testkit.Rows("test t a 0 1 2", + "test t a 0 2 2", + "test t b 0 1 3", + "test t b 0 2 2")) + tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( + testkit.Rows("0 1 4 0 8 2 1", // tot_col_size of column a is updated to 8 by DumpStatsDeltaToKV + "0 2 5 0 8 2 0.76")) + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( + // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv + testkit.Rows("test t a 0 0 2 1 3 4 0", + "test t b 0 0 2 1 3 4 0", + "test t b 0 1 3 1 6 6 0")) + tk.MustQuery(fmt.Sprintf("select hist_id from mysql.stats_histograms where version = (select version from mysql.stats_meta where table_id = %d)", tblID)).Check(testkit.Rows("2")) + }(val) + } +} + func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 90a47d159d782..4cb91b4928501 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -3107,59 +3107,3 @@ func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseCount"), IsNil) c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount"), IsNil) } - -func (s *testStatsSuite) TestAnalyzeColumnsAfterAnalyzeAll(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("create table t (a int, b int)") - tk.MustExec("insert into t (a,b) values (1,1), (1,1), (2,2), (2,2), (3,3), (4,4)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - tk.MustExec("analyze table t with 2 topn, 2 buckets") - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 6")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t a 0 1 2", - "test t a 0 2 2", - "test t b 0 1 2", - "test t b 0 2 2")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 4 0 6 2 1", - "0 2 4 0 6 2 1")) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t a 0 0 2 1 3 4 0", - "test t b 0 0 2 1 3 4 0")) - - tk.MustExec("insert into t (a,b) values (1,1), (6,6)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - - tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") - // Column a is not analyzed in second ANALYZE. We keep the outdated stats of column a rather than delete them. - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 8")) - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_idx, value, count - testkit.Rows("test t a 0 1 2", - "test t a 0 2 2", - "test t b 0 1 3", - "test t b 0 2 2")) - tk.MustQuery(fmt.Sprintf("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, truncate(correlation,2) from mysql.stats_histograms where table_id = %d", tblID)).Sort().Check( - testkit.Rows("0 1 4 0 8 2 1", // tot_col_size of column a is updated to 8 by DumpStatsDeltaToKV - "0 2 5 0 8 2 0.76")) - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check( - // db, tbl, part, col, is_index, bucket_id, count, repeats, lower, upper, ndv - testkit.Rows("test t a 0 0 2 1 3 4 0", - "test t b 0 0 2 1 3 4 0", - "test t b 0 1 3 1 6 6 0")) - tk.MustQuery(fmt.Sprintf("select hist_id from mysql.stats_histograms where version = (select version from mysql.stats_meta where table_id = %d)", tblID)).Check(testkit.Rows("2")) -}