Files
tidb/statistics/update.go

772 lines
23 KiB
Go

// Copyright 2017 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package statistics
import (
"context"
"fmt"
"math"
"strconv"
"strings"
"sync"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/sqlexec"
log "github.com/sirupsen/logrus"
)
type tableDeltaMap map[int64]variable.TableDelta
func (m tableDeltaMap) update(id int64, delta int64, count int64, colSize *map[int64]int64) {
item := m[id]
item.Delta += delta
item.Count += count
if item.ColSize == nil {
item.ColSize = make(map[int64]int64)
}
if colSize != nil {
for key, val := range *colSize {
item.ColSize[key] += val
}
}
m[id] = item
}
type errorRateDelta struct {
PkID int64
PkErrorRate *ErrorRate
IdxErrorRate map[int64]*ErrorRate
}
type errorRateDeltaMap map[int64]errorRateDelta
func (m errorRateDeltaMap) update(tableID int64, histID int64, rate float64, isIndex bool) {
item := m[tableID]
if isIndex {
if item.IdxErrorRate == nil {
item.IdxErrorRate = make(map[int64]*ErrorRate)
}
if item.IdxErrorRate[histID] == nil {
item.IdxErrorRate[histID] = &ErrorRate{}
}
item.IdxErrorRate[histID].update(rate)
} else {
if item.PkErrorRate == nil {
item.PkID = histID
item.PkErrorRate = &ErrorRate{}
}
item.PkErrorRate.update(rate)
}
m[tableID] = item
}
func (m errorRateDeltaMap) merge(deltaMap errorRateDeltaMap) {
for tableID, item := range deltaMap {
tbl := m[tableID]
for histID, errorRate := range item.IdxErrorRate {
if tbl.IdxErrorRate == nil {
tbl.IdxErrorRate = make(map[int64]*ErrorRate)
}
if tbl.IdxErrorRate[histID] == nil {
tbl.IdxErrorRate[histID] = &ErrorRate{}
}
tbl.IdxErrorRate[histID].merge(errorRate)
}
if item.PkErrorRate != nil {
if tbl.PkErrorRate == nil {
tbl.PkID = item.PkID
tbl.PkErrorRate = &ErrorRate{}
}
tbl.PkErrorRate.merge(item.PkErrorRate)
}
m[tableID] = tbl
}
}
func (m errorRateDeltaMap) clear(tableID int64, histID int64, isIndex bool) {
item := m[tableID]
if isIndex {
delete(item.IdxErrorRate, histID)
} else {
item.PkErrorRate = nil
}
m[tableID] = item
}
func (h *Handle) merge(s *SessionStatsCollector, rateMap errorRateDeltaMap) {
for id, item := range s.mapper {
h.globalMap.update(id, item.Delta, item.Count, &item.ColSize)
}
s.mapper = make(tableDeltaMap)
rateMap.merge(s.rateMap)
s.rateMap = make(errorRateDeltaMap)
h.feedback = mergeQueryFeedback(h.feedback, s.feedback)
s.feedback = s.feedback[:0]
}
// SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it.
type SessionStatsCollector struct {
sync.Mutex
mapper tableDeltaMap
feedback []*QueryFeedback
rateMap errorRateDeltaMap
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
}
// Delete only sets the deleted flag true, it will be deleted from list when DumpStatsDeltaToKV is called.
func (s *SessionStatsCollector) Delete() {
s.Lock()
defer s.Unlock()
s.deleted = true
}
// Update will updates the delta and count for one table id.
func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSize *map[int64]int64) {
s.Lock()
defer s.Unlock()
s.mapper.update(id, delta, count, colSize)
}
func mergeQueryFeedback(lq []*QueryFeedback, rq []*QueryFeedback) []*QueryFeedback {
for _, q := range rq {
if len(lq) >= MaxQueryFeedbackCount {
break
}
lq = append(lq, q)
}
return lq
}
var (
// MinLogScanCount is the minimum scan count for a feedback to be logged.
MinLogScanCount = int64(1000)
// MinLogErrorRate is the minimum error rate for a feedback to be logged.
MinLogErrorRate = 0.5
)
// StoreQueryFeedback will merges the feedback into stats collector.
func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle) error {
q := feedback.(*QueryFeedback)
// TODO: If the error rate is small or actual scan count is small, we do not need to store the feed back.
if !q.valid || q.hist == nil {
return nil
}
err := q.recalculateExpectCount(h)
if err != nil {
return errors.Trace(err)
}
expected := float64(q.expected)
var rate float64
if q.actual == 0 {
if expected == 0 {
rate = 0
} else {
rate = 1
}
} else {
rate = math.Abs(expected-float64(q.actual)) / float64(q.actual)
}
if rate >= MinLogErrorRate && (q.actual >= MinLogScanCount || q.expected >= MinLogScanCount) {
metrics.SignificantFeedbackCounter.Inc()
if log.GetLevel() == log.DebugLevel {
q.logDetailedInfo(h)
}
}
metrics.StatsInaccuracyRate.Observe(rate)
s.Lock()
defer s.Unlock()
isIndex := q.tp == indexType
s.rateMap.update(q.physicalID, q.hist.ID, rate, isIndex)
if len(s.feedback) < MaxQueryFeedbackCount {
s.feedback = append(s.feedback, q)
}
return nil
}
// NewSessionStatsCollector allocates a stats collector for a session.
func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector {
h.listHead.Lock()
defer h.listHead.Unlock()
newCollector := &SessionStatsCollector{
mapper: make(tableDeltaMap),
rateMap: make(errorRateDeltaMap),
next: h.listHead.next,
}
h.listHead.next = newCollector
return newCollector
}
var (
// DumpStatsDeltaRatio is the lower bound of `Modify Count / Table Count` for stats delta to be dumped.
DumpStatsDeltaRatio = 1 / 10000.0
// dumpStatsMaxDuration is the max duration since last update.
dumpStatsMaxDuration = time.Hour
)
// needDumpStatsDelta returns true when only updates a small portion of the table and the time since last update
// do not exceed one hour.
func needDumpStatsDelta(h *Handle, id int64, item variable.TableDelta, currentTime time.Time) bool {
if item.InitTime.IsZero() {
item.InitTime = currentTime
}
tbl, ok := h.statsCache.Load().(statsCache)[id]
if !ok {
// No need to dump if the stats is invalid.
return false
}
if currentTime.Sub(item.InitTime) > dumpStatsMaxDuration {
// Dump the stats to kv at least once an hour.
return true
}
if tbl.Count == 0 || float64(item.Count)/float64(tbl.Count) > DumpStatsDeltaRatio {
// Dump the stats when there are many modifications.
return true
}
return false
}
const (
// DumpAll indicates dump all the delta info in to kv
DumpAll = true
// DumpDelta indicates dump part of the delta info in to kv.
DumpDelta = false
)
// sweepList will loop over the list, merge each session's local stats into handle
// and remove closed session's collector.
func (h *Handle) sweepList() {
prev := h.listHead
prev.Lock()
errorRateMap := make(errorRateDeltaMap)
for curr := prev.next; curr != nil; curr = curr.next {
curr.Lock()
// Merge the session stats into handle and error rate map.
h.merge(curr, errorRateMap)
if curr.deleted {
prev.next = curr.next
// Since the session is already closed, we can safely unlock it here.
curr.Unlock()
} else {
// Unlock the previous lock, so we only holds at most two session's lock at the same time.
prev.Unlock()
prev = curr
}
}
prev.Unlock()
h.mu.Lock()
h.mu.rateMap.merge(errorRateMap)
h.mu.Unlock()
}
// DumpStatsDeltaToKV sweeps the whole list and updates the global map, then we dumps every table that held in map to KV.
// If the `dumpAll` is false, it will only dump that delta info that `Modify Count / Table Count` greater than a ratio.
func (h *Handle) DumpStatsDeltaToKV(dumpMode bool) error {
h.sweepList()
currentTime := time.Now()
for id, item := range h.globalMap {
if dumpMode == DumpDelta && !needDumpStatsDelta(h, id, item, currentTime) {
continue
}
updated, err := h.dumpTableStatCountToKV(id, item)
if err != nil {
return errors.Trace(err)
}
if updated {
h.globalMap.update(id, -item.Delta, -item.Count, nil)
}
if err = h.dumpTableStatColSizeToKV(id, item); err != nil {
return errors.Trace(err)
}
if updated {
delete(h.globalMap, id)
} else {
m := h.globalMap[id]
m.ColSize = nil
h.globalMap[id] = m
}
}
return nil
}
// dumpTableStatDeltaToKV dumps a single delta with some table to KV and updates the version.
func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (updated bool, err error) {
if delta.Count == 0 {
return true, nil
}
h.mu.Lock()
defer h.mu.Unlock()
ctx := context.TODO()
exec := h.mu.ctx.(sqlexec.SQLExecutor)
_, err = exec.Execute(ctx, "begin")
if err != nil {
return false, errors.Trace(err)
}
defer func() {
err = finishTransaction(context.Background(), exec, err)
}()
txn, err := h.mu.ctx.Txn(true)
if err != nil {
return false, errors.Trace(err)
}
startTS := txn.StartTS()
var sql string
if delta.Delta < 0 {
sql = fmt.Sprintf("update mysql.stats_meta set version = %d, count = count - %d, modify_count = modify_count + %d where table_id = %d and count >= %d", startTS, -delta.Delta, delta.Count, id, -delta.Delta)
} else {
sql = fmt.Sprintf("update mysql.stats_meta set version = %d, count = count + %d, modify_count = modify_count + %d where table_id = %d", startTS, delta.Delta, delta.Count, id)
}
_, err = h.mu.ctx.(sqlexec.SQLExecutor).Execute(ctx, sql)
if err != nil {
return
}
updated = h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0
return
}
func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) error {
if len(delta.ColSize) == 0 {
return nil
}
values := make([]string, 0, len(delta.ColSize))
for histID, deltaColSize := range delta.ColSize {
if deltaColSize == 0 {
continue
}
values = append(values, fmt.Sprintf("(%d, 0, %d, 0, %d)", id, histID, deltaColSize))
}
if len(values) == 0 {
return nil
}
sql := fmt.Sprintf("insert into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, tot_col_size) "+
"values %s on duplicate key update tot_col_size = tot_col_size + values(tot_col_size)", strings.Join(values, ","))
_, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql)
return errors.Trace(err)
}
// DumpStatsFeedbackToKV dumps the stats feedback to KV.
func (h *Handle) DumpStatsFeedbackToKV() error {
var err error
var successCount int
for _, fb := range h.feedback {
if fb.tp == pkType {
err = h.dumpFeedbackToKV(fb)
} else {
t, ok := h.statsCache.Load().(statsCache)[fb.physicalID]
if ok {
err = dumpFeedbackForIndex(h, fb, t)
}
}
if err != nil {
break
}
successCount++
}
h.feedback = h.feedback[successCount:]
return errors.Trace(err)
}
func (h *Handle) dumpFeedbackToKV(fb *QueryFeedback) error {
vals, err := encodeFeedback(fb)
if err != nil {
log.Debugf("error occurred when encoding feedback, err: %s", errors.ErrorStack(err))
return nil
}
var isIndex int64
if fb.tp == indexType {
isIndex = 1
}
sql := fmt.Sprintf("insert into mysql.stats_feedback (table_id, hist_id, is_index, feedback) values "+
"(%d, %d, %d, X'%X')", fb.physicalID, fb.hist.ID, isIndex, vals)
h.mu.Lock()
_, err = h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql)
h.mu.Unlock()
if err != nil {
metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblError).Inc()
} else {
metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblOK).Inc()
}
return errors.Trace(err)
}
// UpdateStatsByLocalFeedback will update statistics by the local feedback.
// Currently, we dump the feedback with the period of 10 minutes, which means
// it takes 10 minutes for a feedback to take effect. However, we can use the
// feedback locally on this tidb-server, so it could be used more timely.
func (h *Handle) UpdateStatsByLocalFeedback(is infoschema.InfoSchema) {
h.sweepList()
for _, fb := range h.feedback {
h.mu.Lock()
table, ok := h.getTableByPhysicalID(is, fb.physicalID)
h.mu.Unlock()
if !ok {
continue
}
tblStats := h.GetPartitionStats(table.Meta(), fb.physicalID)
newTblStats := tblStats.copy()
if fb.tp == indexType {
idx, ok := tblStats.Indices[fb.hist.ID]
if !ok || idx.Histogram.Len() == 0 {
continue
}
newIdx := *idx
eqFB, ranFB := splitFeedbackByQueryType(fb.feedback)
newIdx.CMSketch = UpdateCMSketch(idx.CMSketch, eqFB)
newIdx.Histogram = *UpdateHistogram(&idx.Histogram, &QueryFeedback{feedback: ranFB})
newIdx.Histogram.PreCalculateScalar()
newTblStats.Indices[fb.hist.ID] = &newIdx
} else {
col, ok := tblStats.Columns[fb.hist.ID]
if !ok || col.Histogram.Len() == 0 {
continue
}
newCol := *col
// only use the range query to update primary key
_, ranFB := splitFeedbackByQueryType(fb.feedback)
newFB := &QueryFeedback{feedback: ranFB}
newFB = newFB.decodeIntValues()
newCol.Histogram = *UpdateHistogram(&col.Histogram, newFB)
newTblStats.Columns[fb.hist.ID] = &newCol
}
h.UpdateTableStats([]*Table{newTblStats}, nil)
}
}
// UpdateErrorRate updates the error rate of columns from h.rateMap to cache.
func (h *Handle) UpdateErrorRate(is infoschema.InfoSchema) {
h.mu.Lock()
tbls := make([]*Table, 0, len(h.mu.rateMap))
for id, item := range h.mu.rateMap {
table, ok := h.getTableByPhysicalID(is, id)
if !ok {
continue
}
tbl := h.GetPartitionStats(table.Meta(), id).copy()
if item.PkErrorRate != nil && tbl.Columns[item.PkID] != nil {
col := *tbl.Columns[item.PkID]
col.ErrorRate.merge(item.PkErrorRate)
tbl.Columns[item.PkID] = &col
}
for key, val := range item.IdxErrorRate {
if tbl.Indices[key] == nil {
continue
}
idx := *tbl.Indices[key]
idx.ErrorRate.merge(val)
tbl.Indices[key] = &idx
}
tbls = append(tbls, tbl)
delete(h.mu.rateMap, id)
}
h.mu.Unlock()
h.UpdateTableStats(tbls, nil)
}
// HandleUpdateStats update the stats using feedback.
func (h *Handle) HandleUpdateStats(is infoschema.InfoSchema) error {
sql := "select table_id, hist_id, is_index, feedback from mysql.stats_feedback order by table_id, hist_id, is_index"
rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql)
if len(rows) == 0 || err != nil {
return errors.Trace(err)
}
var groupedRows [][]chunk.Row
preIdx := 0
tableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2)
for i := 1; i < len(rows); i++ {
row := rows[i]
if row.GetInt64(0) != tableID || row.GetInt64(1) != histID || row.GetInt64(2) != isIndex {
groupedRows = append(groupedRows, rows[preIdx:i])
tableID, histID, isIndex = row.GetInt64(0), row.GetInt64(1), row.GetInt64(2)
preIdx = i
}
}
groupedRows = append(groupedRows, rows[preIdx:])
for _, rows := range groupedRows {
if err := h.handleSingleHistogramUpdate(is, rows); err != nil {
return errors.Trace(err)
}
}
return nil
}
// handleSingleHistogramUpdate updates the Histogram and CM Sketch using these feedbacks. All the feedbacks for
// the same index or column are gathered in `rows`.
func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []chunk.Row) (err error) {
physicalTableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2)
defer func() {
if err == nil {
err = errors.Trace(h.deleteOutdatedFeedback(physicalTableID, histID, isIndex))
}
}()
h.mu.Lock()
table, ok := h.getTableByPhysicalID(is, physicalTableID)
h.mu.Unlock()
// The table has been deleted.
if !ok {
return nil
}
var tbl *Table
if table.Meta().GetPartitionInfo() != nil {
tbl = h.GetPartitionStats(table.Meta(), physicalTableID)
} else {
tbl = h.GetTableStats(table.Meta())
}
var cms *CMSketch
var hist *Histogram
if isIndex == 1 {
idx, ok := tbl.Indices[histID]
if ok && idx.Histogram.Len() > 0 {
idxHist := idx.Histogram
hist = &idxHist
cms = idx.CMSketch.copy()
}
} else {
col, ok := tbl.Columns[histID]
if ok && col.Histogram.Len() > 0 {
colHist := col.Histogram
hist = &colHist
}
}
// The column or index has been deleted.
if hist == nil {
return nil
}
q := &QueryFeedback{}
for _, row := range rows {
err1 := decodeFeedback(row.GetBytes(3), q, cms, mysql.HasUnsignedFlag(hist.Tp.Flag))
if err1 != nil {
log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err))
}
}
err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms)
return errors.Trace(err)
}
func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error {
h.mu.Lock()
h.mu.ctx.GetSessionVars().BatchDelete = true
sql := fmt.Sprintf("delete from mysql.stats_feedback where table_id = %d and hist_id = %d and is_index = %d", tableID, histID, isIndex)
_, err := h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql)
h.mu.ctx.GetSessionVars().BatchDelete = false
h.mu.Unlock()
return errors.Trace(err)
}
func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *QueryFeedback, hist *Histogram, cms *CMSketch) error {
hist = UpdateHistogram(hist, q)
err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, 0)
metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc()
return errors.Trace(err)
}
const (
// StatsOwnerKey is the stats owner path that is saved to etcd.
StatsOwnerKey = "/tidb/stats/owner"
// StatsPrompt is the prompt for stats owner manager.
StatsPrompt = "stats"
)
// AutoAnalyzeMinCnt means if the count of table is less than this value, we needn't do auto analyze.
var AutoAnalyzeMinCnt int64 = 1000
// TableAnalyzed checks if the table is analyzed.
func TableAnalyzed(tbl *Table) bool {
for _, col := range tbl.Columns {
if col.Count > 0 {
return true
}
}
for _, idx := range tbl.Indices {
if idx.Histogram.Len() > 0 {
return true
}
}
return false
}
// withinTimePeriod tests whether `now` is between `start` and `end`.
func withinTimePeriod(start, end, now time.Time) bool {
// Converts to UTC and only keeps the hour and minute info.
start, end, now = start.UTC(), end.UTC(), now.UTC()
start = time.Date(0, 0, 0, start.Hour(), start.Minute(), 0, 0, time.UTC)
end = time.Date(0, 0, 0, end.Hour(), end.Minute(), 0, 0, time.UTC)
now = time.Date(0, 0, 0, now.Hour(), now.Minute(), 0, 0, time.UTC)
// for cases like from 00:00 to 06:00
if end.Sub(start) >= 0 {
return now.Sub(start) >= 0 && now.Sub(end) <= 0
}
// for cases like from 22:00 to 06:00
return now.Sub(end) <= 0 || now.Sub(start) >= 0
}
// NeedAnalyzeTable checks if we need to analyze the table:
// 1. If the table has never been analyzed, we need to analyze it when it has
// not been modified for a while.
// 2. If the table had been analyzed before, we need to analyze it when
// "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio" and the current time is
// between `start` and `end`.
func NeedAnalyzeTable(tbl *Table, limit time.Duration, autoAnalyzeRatio float64, start, end, now time.Time) (bool, string) {
analyzed := TableAnalyzed(tbl)
if !analyzed {
t := time.Unix(0, oracle.ExtractPhysical(tbl.Version)*int64(time.Millisecond))
dur := time.Since(t)
return dur >= limit, fmt.Sprintf("table unanalyzed, time since last updated %vs", dur)
}
// Auto analyze is disabled.
if autoAnalyzeRatio == 0 {
return false, ""
}
// No need to analyze it.
if float64(tbl.ModifyCount)/float64(tbl.Count) <= autoAnalyzeRatio {
return false, ""
}
// Tests if current time is within the time period.
return withinTimePeriod(start, end, now), fmt.Sprintf("too many modifications(%v/%v)", tbl.ModifyCount, tbl.Count)
}
const (
minAutoAnalyzeRatio = 0.3
)
func (h *Handle) getAutoAnalyzeParameters() map[string]string {
sql := fmt.Sprintf("select variable_name, variable_value from mysql.global_variables where variable_name in ('%s', '%s', '%s')",
variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime)
rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql)
if err != nil {
return map[string]string{}
}
parameters := make(map[string]string)
for _, row := range rows {
parameters[row.GetString(0)] = row.GetString(1)
}
return parameters
}
func parseAutoAnalyzeRatio(ratio string) float64 {
autoAnalyzeRatio, err := strconv.ParseFloat(ratio, 64)
if err != nil {
return variable.DefAutoAnalyzeRatio
}
if autoAnalyzeRatio > 0 {
autoAnalyzeRatio = math.Max(autoAnalyzeRatio, minAutoAnalyzeRatio)
}
return autoAnalyzeRatio
}
func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) {
if start == "" {
start = variable.DefAutoAnalyzeStartTime
}
if end == "" {
end = variable.DefAutoAnalyzeEndTime
}
s, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, start, time.UTC)
if err != nil {
return s, s, errors.Trace(err)
}
e, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, end, time.UTC)
if err != nil {
return s, e, errors.Trace(err)
}
return s, e, nil
}
// HandleAutoAnalyze analyzes the newly created table or index.
func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) {
dbs := is.AllSchemaNames()
parameters := h.getAutoAnalyzeParameters()
autoAnalyzeRatio := parseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
start, end, err := parseAnalyzePeriod(parameters[variable.TiDBAutoAnalyzeStartTime], parameters[variable.TiDBAutoAnalyzeEndTime])
if err != nil {
log.Errorf("[stats] parse auto analyze period failed: %v", errors.ErrorStack(err))
return
}
for _, db := range dbs {
tbls := is.SchemaTables(model.NewCIStr(db))
for _, tbl := range tbls {
tblInfo := tbl.Meta()
pi := tblInfo.GetPartitionInfo()
tblName := "`" + db + "`.`" + tblInfo.Name.O + "`"
if pi == nil {
statsTbl := h.GetTableStats(tblInfo)
sql := fmt.Sprintf("analyze table %s", tblName)
analyzed := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql)
if analyzed {
return
}
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 := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql)
if analyzed {
return
}
continue
}
}
}
return
}
func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *Table, start, end time.Time, ratio float64, sql string) bool {
if statsTbl.Pseudo || statsTbl.Count < AutoAnalyzeMinCnt {
return false
}
if needAnalyze, reason := NeedAnalyzeTable(statsTbl, 20*h.Lease, ratio, start, end, time.Now()); needAnalyze {
log.Infof("[stats] %s, auto %s now", sql, reason)
h.execAutoAnalyze(sql)
return true
}
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] index unanalyzed, auto %s now", sql)
h.execAutoAnalyze(sql)
return true
}
}
return false
}
func (h *Handle) execAutoAnalyze(sql string) {
startTime := time.Now()
_, _, err := h.restrictedExec.ExecRestrictedSQL(nil, sql)
dur := time.Since(startTime)
metrics.AutoAnalyzeHistogram.Observe(dur.Seconds())
if err != nil {
log.Errorf("[stats] auto %v failed: %v, cost_time:%vs", sql, errors.ErrorStack(err), dur.Seconds())
metrics.AutoAnalyzeCounter.WithLabelValues("failed").Inc()
} else {
metrics.AutoAnalyzeCounter.WithLabelValues("succ").Inc()
}
}