From 4f175f93d63bf612c88332c592804b68c9a544b5 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 27 Sep 2018 14:33:04 +0800 Subject: [PATCH 1/3] *: support auto analyze partition table --- ast/stats.go | 7 +++-- executor/analyze_test.go | 23 +++++++++++++++ parser/parser.y | 25 ++++++++++++++++ parser/parser_test.go | 5 ++++ planner/core/planbuilder.go | 57 ++++++++++++++++++++++++++++--------- statistics/update.go | 51 +++++++++++++++++++++++---------- statistics/update_test.go | 38 +++++++++++++++++++++++++ 7 files changed, 175 insertions(+), 31 deletions(-) diff --git a/ast/stats.go b/ast/stats.go index d31cb70a8d46a..ea13f24d7a2f0 100644 --- a/ast/stats.go +++ b/ast/stats.go @@ -25,9 +25,10 @@ var ( type AnalyzeTableStmt struct { stmtNode - TableNames []*TableName - IndexNames []model.CIStr - MaxNumBuckets uint64 + TableNames []*TableName + PartitionNames []model.CIStr + IndexNames []model.CIStr + MaxNumBuckets uint64 // IndexFlag is true when we only analyze indices for a table. IndexFlag bool diff --git a/executor/analyze_test.go b/executor/analyze_test.go index ee8ede97710c8..52813d4fa9106 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -61,6 +61,29 @@ PARTITION BY RANGE ( a ) ( c.Assert(idx.Len(), Greater, 0) } } + + tk.MustExec("drop table t") + tk.MustExec(createTable) + for i := 1; i < 21; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) + } + tk.MustExec("alter table t analyze partition p0") + is = executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + pi = table.Meta().GetPartitionInfo() + c.Assert(pi, NotNil) + + for i, def := range pi.Definitions { + statsTbl := handle.GetPartitionStats(table.Meta(), def.ID) + if i == 0 { + c.Assert(statsTbl.Pseudo, IsFalse) + c.Assert(len(statsTbl.Columns), Equals, 2) + c.Assert(len(statsTbl.Indices), Equals, 1) + } else { + c.Assert(statsTbl.Pseudo, IsTrue) + } + } } func (s *testSuite) TestAnalyzeParameters(c *C) { diff --git a/parser/parser.y b/parser/parser.y index 49ba524035805..73a3d3ea3befc 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -685,6 +685,7 @@ import ( PartitionDefinitionList "Partition definition list" PartitionDefinitionListOpt "Partition definition list option" PartitionOpt "Partition option" + PartitionNameList "Partition name list" PartitionNumOpt "PARTITION NUM option" PartDefValuesOpt "VALUES {LESS THAN {(expr | value_list) | MAXVALUE} | IN {value_list}" PartDefOptionsOpt "PartDefOptionList option" @@ -923,6 +924,20 @@ AlterTableStmt: Specs: $5.([]*ast.AlterTableSpec), } } +| "ALTER" IgnoreOptional "TABLE" TableName "ANALYZE" "PARTITION" PartitionNameList MaxNumBuckets + { + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, PartitionNames: $7.([]model.CIStr), MaxNumBuckets: $8.(uint64),} + } +| "ALTER" IgnoreOptional "TABLE" TableName "ANALYZE" "PARTITION" PartitionNameList "INDEX" IndexNameList MaxNumBuckets + { + $$ = &ast.AnalyzeTableStmt{ + TableNames: []*ast.TableName{$4.(*ast.TableName)}, + PartitionNames: $7.([]model.CIStr), + IndexNames: $9.([]model.CIStr), + IndexFlag: true, + MaxNumBuckets: $10.(uint64), + } + } AlterTableSpec: AlterTableOptionListOpt @@ -1170,6 +1185,16 @@ AlterTableSpecList: $$ = append($1.([]*ast.AlterTableSpec), $3.(*ast.AlterTableSpec)) } +PartitionNameList: + Identifier + { + $$ = []model.CIStr{model.NewCIStr($1)} + } +| PartitionNameList ',' Identifier + { + $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + } + ConstraintKeywordOpt: { $$ = nil diff --git a/parser/parser_test.go b/parser/parser_test.go index c649c6615deb7..1aaf64b676ab6 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1701,6 +1701,11 @@ func (s *testParserSuite) TestDDL(c *C) { {"ALTER TABLE t RENAME KEY a TO b;", true}, {"ALTER TABLE t RENAME INDEX a TO b;", true}, + {"alter table t analyze partition a", true}, + {"alter table t analyze partition a with 4 buckets", true}, + {"alter table t analyze partition a index b", true}, + {"alter table t analyze partition a index b with 4 buckets", true}, + // For create index statement {"CREATE INDEX idx ON t (a)", true}, {"CREATE INDEX idx ON t (a) USING HASH", true}, diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 748ec72aaaa5f..10bfdebba7a71 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -19,6 +19,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/model" @@ -622,22 +623,46 @@ func getColsInfo(tn *ast.TableName) (indicesInfo []*model.IndexInfo, colsInfo [] return } -func getPhysicalIDs(tblInfo *model.TableInfo) []int64 { - if pi := tblInfo.GetPartitionInfo(); pi != nil { +func getPhysicalIDs(tblInfo *model.TableInfo, partitionNames []model.CIStr) ([]int64, error) { + pi := tblInfo.GetPartitionInfo() + if pi == nil { + if len(partitionNames) != 0 { + return nil, errors.Trace(ddl.ErrPartitionMgmtOnNonpartitioned) + } + return []int64{tblInfo.ID}, nil + } + if len(partitionNames) == 0 { ids := make([]int64, 0, len(pi.Definitions)) for _, def := range pi.Definitions { ids = append(ids, def.ID) } - return ids + return ids, nil } - return []int64{tblInfo.ID} + ids := make([]int64, 0, len(partitionNames)) + for _, name := range partitionNames { + found := false + for _, def := range pi.Definitions { + if def.Name.L == name.L { + found = true + ids = append(ids, def.ID) + break + } + } + if !found { + return nil, errors.New(fmt.Sprintf("Error in list of partitions to %s", tblInfo.Name.O)) + } + } + return ids, nil } -func (b *planBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) Plan { +func (b *planBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) { p := &Analyze{MaxNumBuckets: as.MaxNumBuckets} for _, tbl := range as.TableNames { idxInfo, colInfo, pkInfo := getColsInfo(tbl) - physicalIDs := getPhysicalIDs(tbl.TableInfo) + physicalIDs, err := getPhysicalIDs(tbl.TableInfo, as.PartitionNames) + if err != nil { + return nil, err + } for _, idx := range idxInfo { for _, id := range physicalIDs { p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalTableID: id, IndexInfo: idx}) @@ -649,13 +674,16 @@ func (b *planBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) Plan { } } } - return p + return p, nil } func (b *planBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) { p := &Analyze{MaxNumBuckets: as.MaxNumBuckets} tblInfo := as.TableNames[0].TableInfo - physicalIDs := getPhysicalIDs(tblInfo) + physicalIDs, err := getPhysicalIDs(tblInfo, as.PartitionNames) + if err != nil { + return nil, err + } for _, idxName := range as.IndexNames { idx := findIndexByName(tblInfo.Indices, idxName) if idx == nil || idx.State != model.StatePublic { @@ -668,10 +696,13 @@ func (b *planBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) return p, nil } -func (b *planBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) Plan { +func (b *planBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) (Plan, error) { p := &Analyze{MaxNumBuckets: as.MaxNumBuckets} tblInfo := as.TableNames[0].TableInfo - physicalIDs := getPhysicalIDs(tblInfo) + physicalIDs, err := getPhysicalIDs(tblInfo, as.PartitionNames) + if err != nil { + return nil, err + } for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for _, id := range physicalIDs { @@ -679,7 +710,7 @@ func (b *planBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) Plan { } } } - return p + return p, nil } const ( @@ -695,11 +726,11 @@ func (b *planBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { } if as.IndexFlag { if len(as.IndexNames) == 0 { - return b.buildAnalyzeAllIndex(as), nil + return b.buildAnalyzeAllIndex(as) } return b.buildAnalyzeIndex(as) } - return b.buildAnalyzeTable(as), nil + return b.buildAnalyzeTable(as) } func buildShowDDLFields() *expression.Schema { diff --git a/statistics/update.go b/statistics/update.go index 5b710579bb1b2..9c93f01de65c3 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -707,31 +707,52 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) error { tbls := is.SchemaTables(model.NewCIStr(db)) for _, tbl := range tbls { tblInfo := tbl.Meta() - statsTbl := h.GetTableStats(tblInfo) - if statsTbl.Pseudo || statsTbl.Count < AutoAnalyzeMinCnt { - continue - } + pi := tblInfo.GetPartitionInfo() tblName := "`" + db + "`.`" + tblInfo.Name.O + "`" - if NeedAnalyzeTable(statsTbl, 20*h.Lease, autoAnalyzeRatio, start, end, time.Now()) { + if pi == nil { + statsTbl := h.GetTableStats(tblInfo) sql := fmt.Sprintf("analyze table %s", tblName) - log.Infof("[stats] auto analyze table %s now", tblName) - return errors.Trace(h.execAutoAnalyze(sql)) - } - for _, idx := range tblInfo.Indices { - if idx.State != model.StatePublic { - continue + analyzed, err := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql) + if analyzed { + return err } - if _, ok := statsTbl.Indices[idx.ID]; !ok { - sql := fmt.Sprintf("analyze table %s index `%s`", tblName, idx.Name.O) - log.Infof("[stats] auto analyze index `%s` for table %s now", idx.Name.O, tblName) - return errors.Trace(h.execAutoAnalyze(sql)) + continue + } + for _, def := range pi.Definitions { + sql := fmt.Sprintf("alter table %s analyze partition `%s`", tblName, def.Name.O) + statsTbl := h.GetPartitionStats(tblInfo, def.ID) + analyzed, err := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql) + if analyzed { + return err } + continue } } } return nil } +func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *Table, start, end time.Time, ratio float64, sql string) (bool, error) { + if statsTbl.Pseudo || statsTbl.Count < AutoAnalyzeMinCnt { + return false, nil + } + if NeedAnalyzeTable(statsTbl, 20*h.Lease, ratio, start, end, time.Now()) { + log.Infof("[stats] auto %s now", sql) + return true, h.execAutoAnalyze(sql) + } + for _, idx := range tblInfo.Indices { + if idx.State != model.StatePublic { + continue + } + if _, ok := statsTbl.Indices[idx.ID]; !ok { + sql = fmt.Sprintf("%s index `%s`", sql, idx.Name.O) + log.Infof("[stats] auto %s now", sql) + return true, h.execAutoAnalyze(sql) + } + } + return false, nil +} + func (h *Handle) execAutoAnalyze(sql string) error { startTime := time.Now() _, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql) diff --git a/statistics/update_test.go b/statistics/update_test.go index 0affb367df228..fd8af3c44966f 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -406,6 +406,44 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { c.Assert(hg.Len(), Equals, 3) } +func (s *testStatsUpdateSuite) TestAutoUpdatePartition(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("set @@session.tidb_enable_table_partition=1") + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t (a int) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") + testKit.MustExec("analyze table t") + + statistics.AutoAnalyzeMinCnt = 0 + testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") + defer func() { + statistics.AutoAnalyzeMinCnt = 1000 + testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") + }() + + do := s.do + is := do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + pi := tableInfo.GetPartitionInfo() + h := do.StatsHandle() + + h.Update(is) + stats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) + c.Assert(stats.Count, Equals, int64(0)) + + testKit.MustExec("insert into t values (1)") + h.DumpStatsDeltaToKV(statistics.DumpAll) + h.Update(is) + err = h.HandleAutoAnalyze(is) + c.Assert(err, IsNil) + stats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) + c.Assert(stats.Count, Equals, int64(1)) + c.Assert(stats.ModifyCount, Equals, int64(0)) +} + func (s *testStatsUpdateSuite) TestTableAnalyzed(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) From d93f8b972f767c1b232221a3a9f57c9e01344db9 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 10 Oct 2018 19:05:47 +0800 Subject: [PATCH 2/3] refine error message --- planner/core/planbuilder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 10bfdebba7a71..6b601bc9cea47 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -649,7 +649,7 @@ func getPhysicalIDs(tblInfo *model.TableInfo, partitionNames []model.CIStr) ([]i } } if !found { - return nil, errors.New(fmt.Sprintf("Error in list of partitions to %s", tblInfo.Name.O)) + return nil, errors.New(fmt.Sprintf("can not found the specified partition name %s in the table definition", name.O)) } } return ids, nil From 788c803dd94b052e2d8755db767a17d2832426b9 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 10 Oct 2018 19:23:57 +0800 Subject: [PATCH 3/3] support syntax `analyze table t partition p` --- parser/parser.y | 14 ++++++++++++++ parser/parser_test.go | 4 ++++ statistics/update.go | 2 +- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/parser/parser.y b/parser/parser.y index 73a3d3ea3befc..03aeb65a81fd2 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -1261,6 +1261,20 @@ AnalyzeTableStmt: { $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$3.(*ast.TableName)}, IndexNames: $5.([]model.CIStr), IndexFlag: true, MaxNumBuckets: $6.(uint64)} } +| "ANALYZE" "TABLE" TableName "PARTITION" PartitionNameList MaxNumBuckets + { + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$3.(*ast.TableName)}, PartitionNames: $5.([]model.CIStr), MaxNumBuckets: $6.(uint64),} + } +| "ANALYZE" "TABLE" TableName "PARTITION" PartitionNameList "INDEX" IndexNameList MaxNumBuckets + { + $$ = &ast.AnalyzeTableStmt{ + TableNames: []*ast.TableName{$3.(*ast.TableName)}, + PartitionNames: $5.([]model.CIStr), + IndexNames: $7.([]model.CIStr), + IndexFlag: true, + MaxNumBuckets: $8.(uint64), + } + } MaxNumBuckets: { diff --git a/parser/parser_test.go b/parser/parser_test.go index 1aaf64b676ab6..54c786c399ec1 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2303,6 +2303,10 @@ func (s *testParserSuite) TestAnalyze(c *C) { {"analyze table t1 index a,b", true}, {"analyze table t with 4 buckets", true}, {"analyze table t index a with 4 buckets", true}, + {"analyze table t partition a", true}, + {"analyze table t partition a with 4 buckets", true}, + {"analyze table t partition a index b", true}, + {"analyze table t partition a index b with 4 buckets", true}, } s.RunTest(c, table) } diff --git a/statistics/update.go b/statistics/update.go index 9c93f01de65c3..8c80baec91585 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -719,7 +719,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) error { continue } for _, def := range pi.Definitions { - sql := fmt.Sprintf("alter table %s analyze partition `%s`", tblName, def.Name.O) + sql := fmt.Sprintf("analyze table %s partition `%s`", tblName, def.Name.O) statsTbl := h.GetPartitionStats(tblInfo, def.ID) analyzed, err := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql) if analyzed {