Skip to content

Commit

Permalink
*: support auto analyze partition table (#7789) (#8649)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and zz-jason committed Dec 12, 2018
1 parent 86b5ce5 commit 68a19bb
Show file tree
Hide file tree
Showing 5 changed files with 145 additions and 31 deletions.
7 changes: 4 additions & 3 deletions ast/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
23 changes: 23 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,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) {
Expand Down
57 changes: 44 additions & 13 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/opcode"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/planner/property"
Expand Down Expand Up @@ -639,22 +640,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
}
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("can not found the specified partition name %s in the table definition", name.O))
}
}
return []int64{tblInfo.ID}
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})
Expand All @@ -666,13 +691,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 {
Expand All @@ -685,18 +713,21 @@ 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 {
p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalTableID: id, IndexInfo: idx})
}
}
}
return p
return p, nil
}

const (
Expand All @@ -712,11 +743,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 buildShowNextRowID() *expression.Schema {
Expand Down
51 changes: 36 additions & 15 deletions statistics/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -703,31 +703,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("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 {
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)
Expand Down
38 changes: 38 additions & 0 deletions statistics/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down

0 comments on commit 68a19bb

Please sign in to comment.