From 3bd7e45bc525fd4138efc5bed32d411a90ce9b37 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Mon, 22 Nov 2021 19:53:08 +0800 Subject: [PATCH 01/38] init commit for lazy load --- domain/domain.go | 14 +- metrics/metrics.go | 17 +-- planner/core/optimizer.go | 43 +++++- sessionctx/stmtctx/stmtctx.go | 13 +- statistics/handle/handle.go | 51 ++++--- statistics/handle/handle_hist.go | 219 +++++++++++++++++++++++++++++++ util/timeutil/timer.go | 88 +++++++++++++ util/wait_group.go | 34 +++++ 8 files changed, 445 insertions(+), 34 deletions(-) create mode 100644 statistics/handle/handle_hist.go create mode 100644 util/timeutil/timer.go create mode 100644 util/wait_group.go diff --git a/domain/domain.go b/domain/domain.go index 8ed2c18e58cd4..b7ef73bd26712 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -17,6 +17,7 @@ package domain import ( "context" "fmt" + "github.com/pingcap/tidb/session" "math/rand" "strconv" "sync" @@ -1244,7 +1245,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { // Negative stats lease indicates that it is in test, it does not need update. if do.statsLease >= 0 { do.wg.Add(1) - go do.loadStatsWorker() + go do.loadStatsWorker(ctx) } owner := do.newOwnerManager(handle.StatsPrompt, handle.StatsOwnerKey) if do.indexUsageSyncLease > 0 { @@ -1280,7 +1281,7 @@ func (do *Domain) newOwnerManager(prompt, ownerKey string) owner.Manager { return statsOwner } -func (do *Domain) loadStatsWorker() { +func (do *Domain) loadStatsWorker(ctx sessionctx.Context) { defer util.Recover(metrics.LabelDomain, "loadStatsWorker", nil, false) lease := do.statsLease if lease == 0 { @@ -1300,6 +1301,15 @@ func (do *Domain) loadStatsWorker() { } else { logutil.BgLogger().Info("init stats info time", zap.Duration("take time", time.Since(t))) } + // TODO config concurrency + for i := 0; i < 20; i++ { + sess, err := session.CreateSession(ctx.GetStore()) + if err != nil { + logutil.BgLogger().Fatal("Create sub load session failed", zap.Error(err)) + } else { + go statsHandle.SubLoadWorker(sess) + } + } for { select { case <-loadTicker.C: diff --git a/metrics/metrics.go b/metrics/metrics.go index 772663f530575..2002437667a03 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -32,14 +32,15 @@ var ( // metrics labels. const ( - LabelSession = "session" - LabelDomain = "domain" - LabelDDLOwner = "ddl-owner" - LabelDDL = "ddl" - LabelDDLWorker = "ddl-worker" - LabelDDLSyncer = "ddl-syncer" - LabelGCWorker = "gcworker" - LabelAnalyze = "analyze" + LabelSession = "session" + LabelDomain = "domain" + LabelDDLOwner = "ddl-owner" + LabelDDL = "ddl" + LabelDDLWorker = "ddl-worker" + LabelDDLSyncer = "ddl-syncer" + LabelGCWorker = "gcworker" + LabelAnalyze = "analyze" + LabelStatsLoadWorker = "stats_load_worker" LabelBatchRecvLoop = "batch-recv-loop" LabelBatchSendLoop = "batch-send-loop" diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index b964818245966..226b3809471f3 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -16,8 +16,6 @@ package core import ( "context" - "math" - "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -31,12 +29,17 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" utilhint "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" + "math" + "time" ) // OptimizeAstNode optimizes the query to a physical plan directly. @@ -262,7 +265,14 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if checkStableResultMode(sctx) { flag |= flagStabilizeResults } - logic, err := logicalOptimize(ctx, flag, logic) + flag1 := flag - flagJoinReOrder - flagPrunColumnsAgain + flag2 := flag & flagJoinReOrder & flagPrunColumnsAgain + logic, err := logicalOptimize(ctx, flag1, logic) + if err != nil { + return nil, 0, err + } + handleNeededColumns(logic, sctx.GetSessionVars().StmtCtx) + logic, err = logicalOptimize(ctx, flag2, logic) if err != nil { return nil, 0, err } @@ -281,6 +291,33 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic return finalPlan, cost, nil } +func handleNeededColumns(plan LogicalPlan, stmtCtx *stmtctx.StatementContext) { + neededColumns := collectNeededColumns(plan) + stmtCtx.StatsLoad.NeededColumnMap = neededColumns + wg := stmtCtx.StatsLoad.Wg + wg.Add(len(neededColumns)) + for col := range neededColumns { + handle.AppendNeededColumn(col, wg) + } + if util.WaitTimeout(wg, time.Second*10) { // TODO configurable timeout + stmtCtx.StatsLoad.Fallback = true + } + // check if all loaded from statsCache +} + +func collectNeededColumns(plan LogicalPlan) map[model.TableColumnID]struct{} { + var neededColumns map[model.TableColumnID]struct{} + switch x := plan.(type) { + case *LogicalSelection: + exprs := x.Conditions + cols := make([]*expression.Column, 0, len(exprs)) + cols = expression.ExtractColumnsFromExpressions(cols, exprs, nil) + // TODO + default: + } + return neededColumns +} + // mergeContinuousSelections merge continuous selections which may occur after changing plans. func mergeContinuousSelections(p PhysicalPlan) { if sel, ok := p.(*PhysicalSelection); ok { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 03c488c883a0c..1ffabece159a1 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -193,10 +193,19 @@ type StatementContext struct { // InVerboseExplain indicates the statement is "explain format='verbose' ...". InVerboseExplain bool - // EnableOptimizeTrace indicates whether the statement is enable optimize trace + // EnableOptimizeTrace indicates whether the statement is enable optimize trace. EnableOptimizeTrace bool - // LogicalOptimizeTrace indicates the trace for optimize + // LogicalOptimizeTrace indicates the trace for optimize. LogicalOptimizeTrace *tracing.LogicalOptimizeTracer + + StatsLoad struct { + // NeededColumnMap stores the columns whose stats are needed for planner. + NeededColumnMap map[model.TableColumnID]struct{} + // Wg is the wait group waiting for all need columns to be loaded. + Wg *sync.WaitGroup + // Fallback indicates if the planner uses full-loaded stats or fallback all to pseudo/simple. + Fallback bool + } } // StmtHints are SessionVars related sql hints. diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 872cdc3daf16f..a4e86ba24ab5b 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -586,13 +586,13 @@ func (sc statsCache) update(tables []*statistics.Table, deletedIDs []int64, newV // LoadNeededHistograms will load histograms for those needed columns. func (h *Handle) LoadNeededHistograms() (err error) { cols := statistics.HistogramNeededColumns.AllCols() - reader, err := h.getStatsReader(0) + reader, err := h.getGlobalStatsReader(0) if err != nil { return err } defer func() { - err1 := h.releaseStatsReader(reader) + err1 := h.releaseGlobalStatsReader(reader) if err1 != nil && err == nil { err = err1 } @@ -851,12 +851,12 @@ func (h *Handle) columnStatsFromStorage(reader *statsReader, row chunk.Row, tabl // TableStatsFromStorage loads table stats info from storage. func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool, snapshot uint64) (_ *statistics.Table, err error) { - reader, err := h.getStatsReader(snapshot) + reader, err := h.getGlobalStatsReader(snapshot) if err != nil { return nil, err } defer func() { - err1 := h.releaseStatsReader(reader) + err1 := h.releaseGlobalStatsReader(reader) if err == nil && err1 != nil { err = err1 } @@ -956,12 +956,12 @@ func (h *Handle) extendedStatsFromStorage(reader *statsReader, table *statistics // StatsMetaCountAndModifyCount reads count and modify_count for the given table from mysql.stats_meta. func (h *Handle) StatsMetaCountAndModifyCount(tableID int64) (int64, int64, error) { - reader, err := h.getStatsReader(0) + reader, err := h.getGlobalStatsReader(0) if err != nil { return 0, 0, err } defer func() { - err1 := h.releaseStatsReader(reader) + err1 := h.releaseGlobalStatsReader(reader) if err1 != nil && err == nil { err = err1 } @@ -1387,38 +1387,51 @@ func (sr *statsReader) isHistory() bool { return sr.snapshot > 0 } -func (h *Handle) getStatsReader(snapshot uint64) (reader *statsReader, err error) { +func (h *Handle) getGlobalStatsReader(snapshot uint64) (reader *statsReader, err error) { + h.mu.Lock() + defer func() { + if r := recover(); r != nil { + err = fmt.Errorf("getGlobalStatsReader panic %v", r) + } + if err != nil { + h.mu.Unlock() + } + }() + return h.getStatsReader(snapshot, h.mu.ctx.(sqlexec.RestrictedSQLExecutor)) +} + +func (h *Handle) releaseGlobalStatsReader(reader *statsReader) error { + defer h.mu.Unlock() + return h.releaseStatsReader(reader, h.mu.ctx.(sqlexec.RestrictedSQLExecutor)) +} + +func (h *Handle) getStatsReader(snapshot uint64, ctx sqlexec.RestrictedSQLExecutor) (reader *statsReader, err error) { failpoint.Inject("mockGetStatsReaderFail", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("gofail genStatsReader error")) } }) if snapshot > 0 { - return &statsReader{ctx: h.mu.ctx.(sqlexec.RestrictedSQLExecutor), snapshot: snapshot}, nil + return &statsReader{ctx: ctx, snapshot: snapshot}, nil } - h.mu.Lock() defer func() { if r := recover(); r != nil { err = fmt.Errorf("getStatsReader panic %v", r) } - if err != nil { - h.mu.Unlock() - } }() failpoint.Inject("mockGetStatsReaderPanic", nil) - _, err = h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), "begin") + _, err = ctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), "begin") if err != nil { return nil, err } - return &statsReader{ctx: h.mu.ctx.(sqlexec.RestrictedSQLExecutor)}, nil + return &statsReader{ctx: ctx}, nil } -func (h *Handle) releaseStatsReader(reader *statsReader) error { +func (h *Handle) releaseStatsReader(reader *statsReader, ctx sqlexec.RestrictedSQLExecutor) error { if reader.snapshot > 0 { return nil } - _, err := h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), "commit") - h.mu.Unlock() + _, err := ctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), "commit") return err } @@ -1562,12 +1575,12 @@ func (h *Handle) removeExtendedStatsItem(tableID int64, statsName string) { // ReloadExtendedStatistics drops the cache for extended statistics and reload data from mysql.stats_extended. func (h *Handle) ReloadExtendedStatistics() error { - reader, err := h.getStatsReader(0) + reader, err := h.getGlobalStatsReader(0) if err != nil { return err } defer func() { - err1 := h.releaseStatsReader(reader) + err1 := h.releaseGlobalStatsReader(reader) if err1 != nil && err == nil { err = err1 } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go new file mode 100644 index 0000000000000..378127eb96b72 --- /dev/null +++ b/statistics/handle/handle_hist.go @@ -0,0 +1,219 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package handle + +import ( + "runtime" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/timeutil" + "go.uber.org/zap" +) + +// HistogramNeeded buffers the histogram needs from optimizer/statistics and is consumed by stats LoadStatsWorker. +var HistogramNeeded = NeededColumnsCh{NeededColumnsCh: make(chan *NeededColumnTask, 100000), TimeoutColumnsCh: make(chan *NeededColumnTask, 1000)} + +type NeededColumnsCh struct { + NeededColumnsCh chan *NeededColumnTask + TimeoutColumnsCh chan *NeededColumnTask +} + +// NeededColumnTask represents one needed column with expire time. +type NeededColumnTask struct { + TableColumnID model.TableColumnID + ToTimeout time.Time + Wg *sync.WaitGroup +} + +// NeededColumnTimeout is the milliseconds the SQL will wait for stats loading +var NeededColumnTimeout int64 = 100 + +// AppendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. It's not thread-safe. TODO +func AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup) { + toTimout := time.Now().Local().Add(time.Millisecond * time.Duration(NeededColumnTimeout)) + colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, Wg: wg} + HistogramNeeded.NeededColumnsCh <- colTask +} + +// SubLoadWorker +func (h *Handle) SubLoadWorker(ctx sessionctx.Context) error { + reader, err0 := h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) + if err0 != nil { + return err0 + } + defer func() { + err1 := h.releaseStatsReader(reader, ctx.(sqlexec.RestrictedSQLExecutor)) + if err1 != nil && err0 == nil { + logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err1)) + } + }() + batched := 0 + for { + batched += 1 + err := h.handleOneTask(reader) + if err != nil { + // TODO should behave differently for different errors + time.Sleep(500 * time.Millisecond) + } + if batched >= 100 { + // refresh statsReader after a while for latest stats + err = h.releaseStatsReader(reader, ctx.(sqlexec.RestrictedSQLExecutor)) + if err != nil { + logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err)) + } + // TODO will begin/commit fail? + reader, err = h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) + if err != nil { + logutil.BgLogger().Error("Fail to new stats loader: ", zap.Error(err)) + } + batched = 0 + } + } +} + +// handleOneTask +func (h *Handle) handleOneTask(reader *statsReader) error { + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + logutil.BgLogger().Error("stats loading panicked", zap.String("stack", string(buf))) + metrics.PanicCounter.WithLabelValues(metrics.LabelStatsLoadWorker).Inc() + } + }() + task, err0 := h.drainColTask() + if err0 != nil && task == nil { + logutil.BgLogger().Fatal("Fail to drain task for stats loading.") + return err0 + } + col := task.TableColumnID + oldCache := h.statsCache.Load().(statsCache) + tbl, ok := oldCache.tables[col.TableID] + if !ok { + task.Wg.Done() + return nil + } + c, ok := tbl.Columns[col.ColumnID] + if !ok || c.Len() > 0 { + task.Wg.Done() + return nil + } + hist, err := h.readStatsForOne(col, c, reader) + if err != nil { + // TODO Put task back to align with old code + return err + } + if hist != nil && h.updateCachedColumn(col, hist) { + task.Wg.Done() + } + return nil +} + +// TODO load data via kv-get asynchronously +func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, reader *statsReader) (*statistics.Column, error) { + hg, err := h.histogramFromStorage(reader, col.TableID, c.ID, &c.Info.FieldType, c.Histogram.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) + if err != nil { + return nil, errors.Trace(err) + } + cms, topN, err := h.cmSketchAndTopNFromStorage(reader, col.TableID, 0, col.ColumnID) + if err != nil { + return nil, errors.Trace(err) + } + fms, err := h.fmSketchFromStorage(reader, col.TableID, 0, col.ColumnID) + if err != nil { + return nil, errors.Trace(err) + } + rows, _, err := reader.read("select stats_ver from mysql.stats_histograms where is_index = 0 and table_id = %? and hist_id = %?", col.TableID, col.ColumnID) + if err != nil { + return nil, errors.Trace(err) + } + if len(rows) == 0 { + logutil.BgLogger().Error("fail to get stats version for this histogram", zap.Int64("table_id", col.TableID), zap.Int64("hist_id", col.ColumnID)) + } + colHist := &statistics.Column{ + PhysicalID: col.TableID, + Histogram: *hg, + Info: c.Info, + CMSketch: cms, + TopN: topN, + FMSketch: fms, + IsHandle: c.IsHandle, + StatsVer: rows[0].GetInt64(0), + } + // Column.Count is calculated by Column.TotalRowCount(). Hence, we don't set Column.Count when initializing colHist. + colHist.Count = int64(colHist.TotalRowCount()) + return colHist, nil +} + +// drainColTask will hang until a column task can return. +func (h *Handle) drainColTask() (*NeededColumnTask, error) { + timeout := time.Nanosecond * 100 + to := timeutil.NewGoodTimer(timeout) + for { + to.Reset(timeout) + select { // select NeededColumnsCh firstly since the priority + case task, ok := <-HistogramNeeded.NeededColumnsCh: + if !ok { + return nil, errors.New("drainColTask: cannot read from a closed NeededColumnsCh, maybe the chan is closed.") + } + if time.Now().After(task.ToTimeout) { + HistogramNeeded.TimeoutColumnsCh <- task + continue + } + return task, nil + case <-to.C(): + to.SetRead() + select { // select TimeoutColumnsCh if there's no task from NeededColumnsCh currently + case task, ok := <-HistogramNeeded.TimeoutColumnsCh: + if !ok { + return nil, errors.New("drainColTask: cannot read from a closed TimeoutColumnsCh, maybe the chan is closed.") + } + return task, nil + case <-to.C(): + to.SetRead() + continue + } + } + } +} + +// updateCachedColumn updates the column hist to global statsCache. +func (h *Handle) updateCachedColumn(col model.TableColumnID, colHist *statistics.Column) (updated bool) { + h.statsCache.Lock() + defer h.statsCache.Unlock() + // Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions + // like `GetPartitionStats` called in `fmSketchFromStorage` would have modified the stats cache already. + oldCache := h.statsCache.Load().(statsCache) + tbl, ok := oldCache.tables[col.TableID] + if !ok { + return true + } + c, ok := tbl.Columns[col.ColumnID] + if !ok || c.Len() > 0 { + return true + } + tbl = tbl.Copy() + tbl.Columns[c.ID] = colHist + return h.updateStatsCache(oldCache.update([]*statistics.Table{tbl}, nil, oldCache.version)) +} diff --git a/util/timeutil/timer.go b/util/timeutil/timer.go new file mode 100644 index 0000000000000..65d175996bca4 --- /dev/null +++ b/util/timeutil/timer.go @@ -0,0 +1,88 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package timeutil + +import ( + "time" +) + +// GoodTimer wraps the standard time.Timer to provide more user-friendly interfaces. +// As Russ Cox suggested (here and here), the correct way to use time.Timer is: +// All the Timer operations (Timer.Stop, Timer.Reset and receiving from or draining the channel) should be done in the same goroutine. +// The program should manage an extra status showing whether it has received from the Timer's channel or not. +// **NOTE**: All the functions of GoodTimer *should* be used in the same goroutine. +type GoodTimer struct { + t *time.Timer // The actual timer + read bool // Whether t.C has already been read from +} + +// NewGoodTimer creates an instance of GoodTimer. +func NewGoodTimer(d time.Duration) *GoodTimer { + return &GoodTimer{t: time.NewTimer(d)} +} + +// ReadC waits until it can read from the wrapped timer's channel C. +// It returns the time value received from the channel C, a zero time value if the channel C has already been read from. +func (gt *GoodTimer) ReadC() time.Time { + if gt.read { + return time.Time{} + } + tv := <-gt.t.C + gt.read = true + return tv +} + +// TryReadC waits for at most the duration d, in order to read from the wrapped timer's channel C. +// It returns the time value received from the channel C, a zero time value if the channel C has already been read from or if the timeout is reached. +func (gt *GoodTimer) TryReadC(timeout time.Duration) time.Time { + if gt.read { + return time.Time{} + } + select { + case tv := <-gt.t.C: + gt.read = true + return tv + case <-time.After(timeout): + return time.Time{} + } +} + +// C returns the chan of wrapped timer for select. +func (gt *GoodTimer) C() <-chan time.Time { + return gt.t.C +} + +// SetRead set the read flag, you must call it if timer chan is read, otherwise stop/reset will hang. +func (gt *GoodTimer) SetRead() { + gt.read = true +} + +// Reset changes the timer to expire after duration d. +func (gt *GoodTimer) Reset(d time.Duration) { + gt.Stop() + gt.t.Reset(d) + gt.read = false +} + +// Stop prevents the Timer from firing. +// It returns true if the call stops the timer, false if the timer has already expired or been stopped. +func (gt *GoodTimer) Stop() bool { + stopped := gt.t.Stop() + if !stopped && !gt.read { + // Drain the gt.t.C if it has not been read from already + <-gt.t.C + } + return stopped +} diff --git a/util/wait_group.go b/util/wait_group.go new file mode 100644 index 0000000000000..168d655c45b5a --- /dev/null +++ b/util/wait_group.go @@ -0,0 +1,34 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "sync" + "time" +) + +func WaitTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { + c := make(chan struct{}) + go func() { + defer close(c) + wg.Wait() + }() + select { + case <-c: + return false // completed normally + case <-time.After(timeout): + return true // timed out + } +} From f016416417b5554fc6cc9295897f3a380efcbd27 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 25 Nov 2021 17:41:01 +0800 Subject: [PATCH 02/38] configs/vars for concurrency,queue-size,syncWait,fallbackPseudo --- config/config.go | 24 +++++++++--- domain/domain.go | 19 +++++----- planner/core/optimizer.go | 60 ++++++++++++++++++++++-------- sessionctx/stmtctx/stmtctx.go | 2 +- sessionctx/variable/session.go | 27 ++++++++++++++ sessionctx/variable/sysvar.go | 16 ++++++++ sessionctx/variable/tidb_vars.go | 7 ++++ statistics/handle/bootstrap.go | 15 ++++++++ statistics/handle/handle.go | 9 +++++ statistics/handle/handle_hist.go | 63 +++++++++++++++++--------------- 10 files changed, 183 insertions(+), 59 deletions(-) diff --git a/config/config.go b/config/config.go index 72e90225ad2bf..997322bc51708 100644 --- a/config/config.go +++ b/config/config.go @@ -189,7 +189,8 @@ type Config struct { // one quarter of the total physical memory in the current system. MaxBallastObjectSize int `toml:"max-ballast-object-size" json:"max-ballast-object-size"` // BallastObjectSize set the initial size of the ballast object, the unit is byte. - BallastObjectSize int `toml:"ballast-object-size" json:"ballast-object-size"` + BallastObjectSize int `toml:"ballast-object-size" json:"ballast-object-size"` + Stats Stats `toml:"stats" json:"stats"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed @@ -439,6 +440,8 @@ type Performance struct { PlanReplayerGCLease string `toml:"plan-replayer-gc-lease" json:"plan-replayer-gc-lease"` GOGC int `toml:"gogc" json:"gogc"` EnforceMPP bool `toml:"enforce-mpp" json:"enforce-mpp"` + StatsLoadConcurrency uint `toml:"stats-load-concurrency" json:"stats-load-concurrency"` + StatsLoadQueueSize uint `toml:"stats-load-queue-size" json:"stats-load-queue-size"` } // PlanCache is the PlanCache section of the config. @@ -569,6 +572,11 @@ type Experimental struct { EnableNewCharset bool `toml:"enable-new-charset" json:"-"` } +type Stats struct { + SyncLoadWait uint `toml:"sync-load-wait" json:"sync-load-wait"` + PseudoForLoadTimeout bool `toml:"pseudo-for-load-timeout" json:"pseudo-for-load-timeout"` +} + var defTiKVCfg = tikvcfg.DefaultConfig() var defaultConf = Config{ Host: DefHost, @@ -649,10 +657,12 @@ var defaultConf = Config{ MaxTxnTTL: defTiKVCfg.MaxTxnTTL, // 1hour MemProfileInterval: "1m", // TODO: set indexUsageSyncLease to 60s. - IndexUsageSyncLease: "0s", - GOGC: 100, - EnforceMPP: false, - PlanReplayerGCLease: "10m", + IndexUsageSyncLease: "0s", + GOGC: 100, + EnforceMPP: false, + PlanReplayerGCLease: "10m", + StatsLoadConcurrency: 20, + StatsLoadQueueSize: 10000, }, ProxyProtocol: ProxyProtocol{ Networks: "", @@ -711,6 +721,10 @@ var defaultConf = Config{ EnableEnumLengthLimit: true, StoresRefreshInterval: defTiKVCfg.StoresRefreshInterval, EnableForwarding: defTiKVCfg.EnableForwarding, + Stats: Stats{ + SyncLoadWait: 100, + PseudoForLoadTimeout: false, + }, } var ( diff --git a/domain/domain.go b/domain/domain.go index 116db638fd919..e9873a31b8d94 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -17,7 +17,6 @@ package domain import ( "context" "fmt" - "github.com/pingcap/tidb/session" "math/rand" "strconv" "sync" @@ -1302,14 +1301,16 @@ func (do *Domain) loadStatsWorker(ctx sessionctx.Context) { } else { logutil.BgLogger().Info("init stats info time", zap.Duration("take time", time.Since(t))) } - // TODO config concurrency - for i := 0; i < 20; i++ { - sess, err := session.CreateSession(ctx.GetStore()) - if err != nil { - logutil.BgLogger().Fatal("Create sub load session failed", zap.Error(err)) - } else { - go statsHandle.SubLoadWorker(sess) - } + // start sub load worker if concurrent-stats-load is enabled + t = time.Now() + subCtxs, err := statsHandle.InitSubSessions(ctx.GetStore()) + if err != nil { + logutil.BgLogger().Error("init sub load sessions failed", zap.Error(err)) + } else { + logutil.BgLogger().Info("init sub load sessions time", zap.Duration("take time", time.Since(t))) + } + for _, subCtx := range subCtxs { + go statsHandle.SubLoadWorker(subCtx, do.exit) } for { select { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 226b3809471f3..0a74fd3359cfb 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -16,8 +16,10 @@ package core import ( "context" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -29,9 +31,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" utilhint "github.com/pingcap/tidb/util/hint" @@ -69,6 +69,8 @@ const ( flagPrunColumnsAgain ) +var flagRulesAfterStats = []uint64{flagJoinReOrder, flagPrunColumnsAgain} + var optRuleList = []logicalOptRule{ &gcSubstituter{}, &columnPruner{}, @@ -265,14 +267,27 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if checkStableResultMode(sctx) { flag |= flagStabilizeResults } - flag1 := flag - flagJoinReOrder - flagPrunColumnsAgain - flag2 := flag & flagJoinReOrder & flagPrunColumnsAgain - logic, err := logicalOptimize(ctx, flag1, logic) + var flagBeforeStats = flag + var flagAfterStats uint64 = 0 + for _, flg := range flagRulesAfterStats { + if flagBeforeStats&flg > 0 { + flagBeforeStats -= flg + flagAfterStats |= flg + } + } + logic, err := logicalOptimize(ctx, flagBeforeStats, logic) if err != nil { return nil, 0, err } - handleNeededColumns(logic, sctx.GetSessionVars().StmtCtx) - logic, err = logicalOptimize(ctx, flag2, logic) + ok, err := SyncLoadNeededColumns(logic, sctx) + if !ok || err != nil { + if sctx.GetSessionVars().GetPseudoForLoadTimeout() { + sctx.GetSessionVars().StmtCtx.StatsLoad.Fallback = true + } else { + return nil, 0, err + } + } + logic, err = logicalOptimize(ctx, flagAfterStats, logic) if err != nil { return nil, 0, err } @@ -291,20 +306,35 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic return finalPlan, cost, nil } -func handleNeededColumns(plan LogicalPlan, stmtCtx *stmtctx.StatementContext) { +// SyncLoadNeededColumns sends column-hist request and sync-wait until timeout +func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, error) { + syncWait := sctx.GetSessionVars().GetStatsSyncWait() + if syncWait <= 0 { + return true, nil + } + statsHandle := domain.GetDomain(sctx).StatsHandle() neededColumns := collectNeededColumns(plan) - stmtCtx.StatsLoad.NeededColumnMap = neededColumns - wg := stmtCtx.StatsLoad.Wg - wg.Add(len(neededColumns)) + missingColumns := make([]model.TableColumnID, 0, len(neededColumns)) + // TODO check missing for col := range neededColumns { - handle.AppendNeededColumn(col, wg) + missingColumns = append(missingColumns, col) + } + stmtCtx := sctx.GetSessionVars().StmtCtx + stmtCtx.StatsLoad.NeededColumns = missingColumns + wg := stmtCtx.StatsLoad.Wg + wg.Add(len(missingColumns)) + waitTime := mathutil.Min(int(syncWait), int(stmtCtx.MaxExecutionTime*1000)) + var timeout = time.Duration(waitTime) + for _, col := range missingColumns { + statsHandle.AppendNeededColumn(col, wg, timeout) } - if util.WaitTimeout(wg, time.Second*10) { // TODO configurable timeout - stmtCtx.StatsLoad.Fallback = true + if util.WaitTimeout(wg, timeout) { + return false, errors.New("Fail to load stats for columns, timeout.") } - // check if all loaded from statsCache + return true, nil } +// collectNeededColumns to align with predicate-column collection func collectNeededColumns(plan LogicalPlan) map[model.TableColumnID]struct{} { var neededColumns map[model.TableColumnID]struct{} switch x := plan.(type) { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d2091108d5d74..73a49b1f1a565 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -206,7 +206,7 @@ type StatementContext struct { StatsLoad struct { // NeededColumnMap stores the columns whose stats are needed for planner. - NeededColumnMap map[model.TableColumnID]struct{} + NeededColumns []model.TableColumnID // Wg is the wait group waiting for all need columns to be loaded. Wg *sync.WaitGroup // Fallback indicates if the planner uses full-loaded stats or fallback all to pseudo/simple. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 57d6f75c25902..5a67ca8714a46 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2342,3 +2342,30 @@ func (s *SessionVars) GetSeekFactor(tbl *model.TableInfo) float64 { } return s.seekFactor } + +func (s *SessionVars) GetStatsSyncWait() uint { + if config.GetGlobalConfig().Performance.StatsLoadConcurrency <= 0 { + return 0 + } + val, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBSyncLoadWait) + if err != nil { + return config.GetGlobalConfig().Stats.SyncLoadWait + } + syncWait, err := strconv.ParseUint(val, 10, 64) + if err != nil { + return config.GetGlobalConfig().Stats.SyncLoadWait + } + return uint(syncWait) +} + +func (s *SessionVars) GetPseudoForLoadTimeout() bool { + val, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBPseudoForLoadTimeout) + if err != nil { + return config.GetGlobalConfig().Stats.PseudoForLoadTimeout + } + pseudo, err := strconv.ParseBool(val) + if err != nil { + return config.GetGlobalConfig().Stats.PseudoForLoadTimeout + } + return pseudo +} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index d66eb6c88096f..8c3cf75bb1b7e 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1873,6 +1873,22 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return "0", nil }}, + {Scope: ScopeGlobal, Name: TiDBSyncLoadWait, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Stats.SyncLoadWait), 10), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, + SetGlobal: func(s *SessionVars, val string) error { + newSyncWait := tidbOptPositiveInt32(val, int(config.GetGlobalConfig().Stats.SyncLoadWait)) + if config.GetGlobalConfig().Performance.StatsLoadConcurrency <= 0 && newSyncWait > 0 { + return errors.New("Cannot enable stats sync wait, since concurrent load is not working, please enable concurrent load and restart server.") + } + StatsSyncLoadWait.Store(uint32(newSyncWait)) + return nil + }, + }, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBPseudoForLoadTimeout, Value: strconv.FormatBool(config.GetGlobalConfig().Stats.PseudoForLoadTimeout), skipInit: true, Type: TypeBool, + SetGlobal: func(s *SessionVars, val string) error { + PseudoForLoadTimeout.Store(TiDBOptOn(val)) + return nil + }, + }, } func collectAllowFuncName4ExpressionIndex() string { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ef7c001841d13..3ecb3a9b42430 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -597,6 +597,11 @@ const ( // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" + + // TiDBSyncLoadWait indicates the time sql execution will sync-wait for stats load. + TiDBSyncLoadWait = "tidb_sync_load_wait" + // TiDBPseudoForLoadTimeout indicates whether to fallback to pseudo stats after load timeout. + TiDBPseudoForLoadTimeout = "tidb_pseudo_for_load_timeout" ) // TiDB vars that have only global scope @@ -805,6 +810,8 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + StatsSyncLoadWait = atomic.NewUint32(uint32(config.GetGlobalConfig().Stats.SyncLoadWait)) + PseudoForLoadTimeout = atomic.NewBool(config.GetGlobalConfig().Stats.PseudoForLoadTimeout) ) // TopSQL is the variable for control top sql feature. diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index bc5cff9a9084f..46f5d28c28e47 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -21,9 +21,12 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -430,3 +433,15 @@ func getFullTableName(is infoschema.InfoSchema, tblInfo *model.TableInfo) string } return fmt.Sprintf("%d", tblInfo.ID) } + +// InitSubSessions will init the sub sessions used for sub load workers. +func (h *Handle) InitSubSessions(store kv.Storage) ([]sessionctx.Context, error) { + for i := 0; i < len(h.subCtxs); i++ { + sess, err := session.CreateSession(store) + if err != nil { + return nil, err + } + h.subCtxs[i] = sess + } + return h.subCtxs, nil +} diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a4e86ba24ab5b..9e97b63bc3172 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -28,6 +28,7 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/ast" @@ -107,6 +108,11 @@ type Handle struct { // idxUsageListHead contains all the index usage collectors required by session. idxUsageListHead *SessionIndexUsageCollector + + // subCtxs holds all sessions used by sub stats load workers + subCtxs []sessionctx.Context + // HistogramNeeded buffers the histogram needs from optimizer/statistics and is consumed by stats worker. + HistogramNeeded NeededColumnsCh } func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context.Context, sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { @@ -185,6 +191,7 @@ type sessionPool interface { // NewHandle creates a Handle for update stats. func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (*Handle, error) { + cfg := config.GetGlobalConfig() handle := &Handle{ ddlEventCh: make(chan *util.Event, 100), listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, @@ -192,6 +199,8 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* feedback: statistics.NewQueryFeedbackMap(), idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)}, pool: pool, + subCtxs: make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency), + HistogramNeeded: NeededColumnsCh{ColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize), TimeoutColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize)}, } handle.lease.Store(lease) handle.pool = pool diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 378127eb96b72..674157b38760c 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -30,11 +30,8 @@ import ( "go.uber.org/zap" ) -// HistogramNeeded buffers the histogram needs from optimizer/statistics and is consumed by stats LoadStatsWorker. -var HistogramNeeded = NeededColumnsCh{NeededColumnsCh: make(chan *NeededColumnTask, 100000), TimeoutColumnsCh: make(chan *NeededColumnTask, 1000)} - type NeededColumnsCh struct { - NeededColumnsCh chan *NeededColumnTask + ColumnsCh chan *NeededColumnTask TimeoutColumnsCh chan *NeededColumnTask } @@ -45,18 +42,17 @@ type NeededColumnTask struct { Wg *sync.WaitGroup } -// NeededColumnTimeout is the milliseconds the SQL will wait for stats loading -var NeededColumnTimeout int64 = 100 - -// AppendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. It's not thread-safe. TODO -func AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup) { - toTimout := time.Now().Local().Add(time.Millisecond * time.Duration(NeededColumnTimeout)) +// AppendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. +func (h *Handle) AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, timeout time.Duration) { + toTimout := time.Now().Local().Add(timeout) colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, Wg: wg} - HistogramNeeded.NeededColumnsCh <- colTask + h.HistogramNeeded.ColumnsCh <- colTask } -// SubLoadWorker -func (h *Handle) SubLoadWorker(ctx sessionctx.Context) error { +var ErrExit = errors.New("Stop loading since domain is closed.") + +// SubLoadWorker loads hist data for each column +func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}) error { reader, err0 := h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) if err0 != nil { return err0 @@ -70,10 +66,15 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context) error { batched := 0 for { batched += 1 - err := h.handleOneTask(reader) + err := h.handleOneTask(reader, exit) if err != nil { - // TODO should behave differently for different errors - time.Sleep(500 * time.Millisecond) + switch err { + case ErrExit: + return nil + default: + time.Sleep(500 * time.Millisecond) + continue + } } if batched >= 100 { // refresh statsReader after a while for latest stats @@ -81,9 +82,9 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context) error { if err != nil { logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err)) } - // TODO will begin/commit fail? reader, err = h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) if err != nil { + // TODO will begin/commit fail? logutil.BgLogger().Error("Fail to new stats loader: ", zap.Error(err)) } batched = 0 @@ -91,8 +92,8 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context) error { } } -// handleOneTask -func (h *Handle) handleOneTask(reader *statsReader) error { +// handleOneTask handles one column task. +func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { defer func() { if r := recover(); r != nil { buf := make([]byte, 4096) @@ -102,7 +103,7 @@ func (h *Handle) handleOneTask(reader *statsReader) error { metrics.PanicCounter.WithLabelValues(metrics.LabelStatsLoadWorker).Inc() } }() - task, err0 := h.drainColTask() + task, err0 := h.drainColTask(exit) if err0 != nil && task == nil { logutil.BgLogger().Fatal("Fail to drain task for stats loading.") return err0 @@ -121,7 +122,7 @@ func (h *Handle) handleOneTask(reader *statsReader) error { } hist, err := h.readStatsForOne(col, c, reader) if err != nil { - // TODO Put task back to align with old code + h.HistogramNeeded.ColumnsCh <- task return err } if hist != nil && h.updateCachedColumn(col, hist) { @@ -130,7 +131,7 @@ func (h *Handle) handleOneTask(reader *statsReader) error { return nil } -// TODO load data via kv-get asynchronously +// readStatsForOne reads hist for one column, TODO load data via kv-get asynchronously func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, reader *statsReader) (*statistics.Column, error) { hg, err := h.histogramFromStorage(reader, col.TableID, c.ID, &c.Info.FieldType, c.Histogram.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) if err != nil { @@ -167,25 +168,25 @@ func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, } // drainColTask will hang until a column task can return. -func (h *Handle) drainColTask() (*NeededColumnTask, error) { +func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { timeout := time.Nanosecond * 100 to := timeutil.NewGoodTimer(timeout) for { to.Reset(timeout) - select { // select NeededColumnsCh firstly since the priority - case task, ok := <-HistogramNeeded.NeededColumnsCh: + select { // select ColumnsCh firstly since the priority + case task, ok := <-h.HistogramNeeded.ColumnsCh: if !ok { - return nil, errors.New("drainColTask: cannot read from a closed NeededColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from a closed ColumnsCh, maybe the chan is closed.") } if time.Now().After(task.ToTimeout) { - HistogramNeeded.TimeoutColumnsCh <- task + h.HistogramNeeded.TimeoutColumnsCh <- task continue } return task, nil case <-to.C(): to.SetRead() - select { // select TimeoutColumnsCh if there's no task from NeededColumnsCh currently - case task, ok := <-HistogramNeeded.TimeoutColumnsCh: + select { // select TimeoutColumnsCh if there's no task from ColumnsCh currently + case task, ok := <-h.HistogramNeeded.TimeoutColumnsCh: if !ok { return nil, errors.New("drainColTask: cannot read from a closed TimeoutColumnsCh, maybe the chan is closed.") } @@ -194,6 +195,10 @@ func (h *Handle) drainColTask() (*NeededColumnTask, error) { to.SetRead() continue } + case <-exit: + return nil, ErrExit + case <-exit: + return nil, ErrExit } } } From e8843051fa73529d37000bf82211ab7a3a186c32 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 1 Dec 2021 18:27:18 +0800 Subject: [PATCH 03/38] fix --- domain/domain.go | 25 ++++++++++++------------- planner/core/optimizer.go | 24 +++++++++++++----------- session/session.go | 12 ++++++++++++ statistics/handle/bootstrap.go | 15 --------------- statistics/handle/handle.go | 6 +++--- statistics/handle/handle_hist.go | 6 +++++- 6 files changed, 45 insertions(+), 43 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index e9873a31b8d94..fceeb2bac0313 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1245,7 +1245,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { // Negative stats lease indicates that it is in test, it does not need update. if do.statsLease >= 0 { do.wg.Add(1) - go do.loadStatsWorker(ctx) + go do.loadStatsWorker() } owner := do.newOwnerManager(handle.StatsPrompt, handle.StatsOwnerKey) if do.indexUsageSyncLease > 0 { @@ -1265,6 +1265,16 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { return nil } +// StartLoadStatsSubWorkers starts sub workers with new sessions to load stats concurrently +func (do *Domain) StartLoadStatsSubWorkers(ctxList []sessionctx.Context) { + statsHandle := do.StatsHandle() + for i, ctx := range ctxList { + statsHandle.SubCtxs[i] = ctx + do.wg.Add(1) + go statsHandle.SubLoadWorker(ctx, do.exit, do.wg) + } +} + func (do *Domain) newOwnerManager(prompt, ownerKey string) owner.Manager { id := do.ddl.OwnerManager().ID() var statsOwner owner.Manager @@ -1281,7 +1291,7 @@ func (do *Domain) newOwnerManager(prompt, ownerKey string) owner.Manager { return statsOwner } -func (do *Domain) loadStatsWorker(ctx sessionctx.Context) { +func (do *Domain) loadStatsWorker() { defer util.Recover(metrics.LabelDomain, "loadStatsWorker", nil, false) lease := do.statsLease if lease == 0 { @@ -1301,17 +1311,6 @@ func (do *Domain) loadStatsWorker(ctx sessionctx.Context) { } else { logutil.BgLogger().Info("init stats info time", zap.Duration("take time", time.Since(t))) } - // start sub load worker if concurrent-stats-load is enabled - t = time.Now() - subCtxs, err := statsHandle.InitSubSessions(ctx.GetStore()) - if err != nil { - logutil.BgLogger().Error("init sub load sessions failed", zap.Error(err)) - } else { - logutil.BgLogger().Info("init sub load sessions time", zap.Duration("take time", time.Since(t))) - } - for _, subCtx := range subCtxs { - go statsHandle.SubLoadWorker(subCtx, do.exit) - } for { select { case <-loadTicker.C: diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 0a74fd3359cfb..e84f621523d96 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -319,17 +319,19 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err for col := range neededColumns { missingColumns = append(missingColumns, col) } - stmtCtx := sctx.GetSessionVars().StmtCtx - stmtCtx.StatsLoad.NeededColumns = missingColumns - wg := stmtCtx.StatsLoad.Wg - wg.Add(len(missingColumns)) - waitTime := mathutil.Min(int(syncWait), int(stmtCtx.MaxExecutionTime*1000)) - var timeout = time.Duration(waitTime) - for _, col := range missingColumns { - statsHandle.AppendNeededColumn(col, wg, timeout) - } - if util.WaitTimeout(wg, timeout) { - return false, errors.New("Fail to load stats for columns, timeout.") + if len(missingColumns) > 0 { + stmtCtx := sctx.GetSessionVars().StmtCtx + stmtCtx.StatsLoad.NeededColumns = missingColumns + wg := stmtCtx.StatsLoad.Wg + wg.Add(len(missingColumns)) + waitTime := mathutil.Min(int(syncWait), int(stmtCtx.MaxExecutionTime*1000)) + var timeout = time.Duration(waitTime) + for _, col := range missingColumns { + statsHandle.AppendNeededColumn(col, wg, timeout) + } + if util.WaitTimeout(wg, timeout) { + return false, errors.New("Fail to load stats for columns, timeout.") + } } return true, nil } diff --git a/session/session.go b/session/session.go index 2228431007048..be19f5cdde957 100644 --- a/session/session.go +++ b/session/session.go @@ -2565,6 +2565,18 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } + // start sub workers for concurrent stats loading + concurrency := config.GetGlobalConfig().Performance.StatsLoadConcurrency + subCtxs := make([]sessionctx.Context, concurrency) + for i := 0; i < int(concurrency); i++ { + subSe, err := createSession(store) + if err != nil { + return nil, err + } + subCtxs[i] = subSe + } + dom.StartLoadStatsSubWorkers(subCtxs) + dom.PlanReplayerLoop() if raw, ok := store.(kv.EtcdBackend); ok { diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 46f5d28c28e47..bc5cff9a9084f 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -21,12 +21,9 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -433,15 +430,3 @@ func getFullTableName(is infoschema.InfoSchema, tblInfo *model.TableInfo) string } return fmt.Sprintf("%d", tblInfo.ID) } - -// InitSubSessions will init the sub sessions used for sub load workers. -func (h *Handle) InitSubSessions(store kv.Storage) ([]sessionctx.Context, error) { - for i := 0; i < len(h.subCtxs); i++ { - sess, err := session.CreateSession(store) - if err != nil { - return nil, err - } - h.subCtxs[i] = sess - } - return h.subCtxs, nil -} diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 9e97b63bc3172..af88d89ba9c51 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -109,8 +109,8 @@ type Handle struct { // idxUsageListHead contains all the index usage collectors required by session. idxUsageListHead *SessionIndexUsageCollector - // subCtxs holds all sessions used by sub stats load workers - subCtxs []sessionctx.Context + // SubCtxs holds all sessions used by sub stats load workers + SubCtxs []sessionctx.Context // HistogramNeeded buffers the histogram needs from optimizer/statistics and is consumed by stats worker. HistogramNeeded NeededColumnsCh } @@ -199,7 +199,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* feedback: statistics.NewQueryFeedbackMap(), idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)}, pool: pool, - subCtxs: make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency), + SubCtxs: make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency), HistogramNeeded: NeededColumnsCh{ColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize), TimeoutColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize)}, } handle.lease.Store(lease) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 674157b38760c..2e988f4ee2866 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -52,7 +52,11 @@ func (h *Handle) AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, t var ErrExit = errors.New("Stop loading since domain is closed.") // SubLoadWorker loads hist data for each column -func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}) error { +func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg sync.WaitGroup) error { + defer func() { + exitWg.Done() + logutil.BgLogger().Info("SubLoadWorker exited.") + }() reader, err0 := h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) if err0 != nil { return err0 From 0e8344ac0849c49d6e9cf249585508c7df901031 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 2 Dec 2021 15:09:04 +0800 Subject: [PATCH 04/38] fix --- planner/core/optimizer.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index e84f621523d96..a0bb6333c3c93 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -308,6 +308,9 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic // SyncLoadNeededColumns sends column-hist request and sync-wait until timeout func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, error) { + if sctx.GetSessionVars().InRestrictedSQL { + return true, nil + } syncWait := sctx.GetSessionVars().GetStatsSyncWait() if syncWait <= 0 { return true, nil From 39141280e83c555902d9d600d81d8f08f8cf7b40 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Fri, 3 Dec 2021 13:48:03 +0800 Subject: [PATCH 05/38] remove sysvars --- planner/core/optimizer.go | 4 ++-- sessionctx/variable/session.go | 27 --------------------------- sessionctx/variable/sysvar.go | 16 ---------------- sessionctx/variable/tidb_vars.go | 7 ------- 4 files changed, 2 insertions(+), 52 deletions(-) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index a0bb6333c3c93..df714d80b29d0 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -281,7 +281,7 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic } ok, err := SyncLoadNeededColumns(logic, sctx) if !ok || err != nil { - if sctx.GetSessionVars().GetPseudoForLoadTimeout() { + if config.GetGlobalConfig().Stats.PseudoForLoadTimeout { sctx.GetSessionVars().StmtCtx.StatsLoad.Fallback = true } else { return nil, 0, err @@ -311,7 +311,7 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err if sctx.GetSessionVars().InRestrictedSQL { return true, nil } - syncWait := sctx.GetSessionVars().GetStatsSyncWait() + syncWait := config.GetGlobalConfig().Stats.SyncLoadWait if syncWait <= 0 { return true, nil } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 5a67ca8714a46..57d6f75c25902 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2342,30 +2342,3 @@ func (s *SessionVars) GetSeekFactor(tbl *model.TableInfo) float64 { } return s.seekFactor } - -func (s *SessionVars) GetStatsSyncWait() uint { - if config.GetGlobalConfig().Performance.StatsLoadConcurrency <= 0 { - return 0 - } - val, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBSyncLoadWait) - if err != nil { - return config.GetGlobalConfig().Stats.SyncLoadWait - } - syncWait, err := strconv.ParseUint(val, 10, 64) - if err != nil { - return config.GetGlobalConfig().Stats.SyncLoadWait - } - return uint(syncWait) -} - -func (s *SessionVars) GetPseudoForLoadTimeout() bool { - val, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBPseudoForLoadTimeout) - if err != nil { - return config.GetGlobalConfig().Stats.PseudoForLoadTimeout - } - pseudo, err := strconv.ParseBool(val) - if err != nil { - return config.GetGlobalConfig().Stats.PseudoForLoadTimeout - } - return pseudo -} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 8c3cf75bb1b7e..d66eb6c88096f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1873,22 +1873,6 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return "0", nil }}, - {Scope: ScopeGlobal, Name: TiDBSyncLoadWait, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Stats.SyncLoadWait), 10), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, - SetGlobal: func(s *SessionVars, val string) error { - newSyncWait := tidbOptPositiveInt32(val, int(config.GetGlobalConfig().Stats.SyncLoadWait)) - if config.GetGlobalConfig().Performance.StatsLoadConcurrency <= 0 && newSyncWait > 0 { - return errors.New("Cannot enable stats sync wait, since concurrent load is not working, please enable concurrent load and restart server.") - } - StatsSyncLoadWait.Store(uint32(newSyncWait)) - return nil - }, - }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPseudoForLoadTimeout, Value: strconv.FormatBool(config.GetGlobalConfig().Stats.PseudoForLoadTimeout), skipInit: true, Type: TypeBool, - SetGlobal: func(s *SessionVars, val string) error { - PseudoForLoadTimeout.Store(TiDBOptOn(val)) - return nil - }, - }, } func collectAllowFuncName4ExpressionIndex() string { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 3ecb3a9b42430..ef7c001841d13 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -597,11 +597,6 @@ const ( // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" - - // TiDBSyncLoadWait indicates the time sql execution will sync-wait for stats load. - TiDBSyncLoadWait = "tidb_sync_load_wait" - // TiDBPseudoForLoadTimeout indicates whether to fallback to pseudo stats after load timeout. - TiDBPseudoForLoadTimeout = "tidb_pseudo_for_load_timeout" ) // TiDB vars that have only global scope @@ -810,8 +805,6 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) - StatsSyncLoadWait = atomic.NewUint32(uint32(config.GetGlobalConfig().Stats.SyncLoadWait)) - PseudoForLoadTimeout = atomic.NewBool(config.GetGlobalConfig().Stats.PseudoForLoadTimeout) ) // TopSQL is the variable for control top sql feature. From 8086b91b261089b7825c42b0fd2c98f7d815866c Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Fri, 3 Dec 2021 15:11:57 +0800 Subject: [PATCH 06/38] impl missing columns --- domain/domain.go | 2 +- planner/core/optimizer.go | 6 +----- statistics/handle/handle_hist.go | 22 +++++++++++++++++++++- statistics/histogram.go | 4 ++++ 4 files changed, 27 insertions(+), 7 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index fceeb2bac0313..66f9789924899 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1271,7 +1271,7 @@ func (do *Domain) StartLoadStatsSubWorkers(ctxList []sessionctx.Context) { for i, ctx := range ctxList { statsHandle.SubCtxs[i] = ctx do.wg.Add(1) - go statsHandle.SubLoadWorker(ctx, do.exit, do.wg) + go statsHandle.SubLoadWorker(ctx, do.exit, &do.wg) } } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index df714d80b29d0..50f1565da4e39 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -317,11 +317,7 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err } statsHandle := domain.GetDomain(sctx).StatsHandle() neededColumns := collectNeededColumns(plan) - missingColumns := make([]model.TableColumnID, 0, len(neededColumns)) - // TODO check missing - for col := range neededColumns { - missingColumns = append(missingColumns, col) - } + missingColumns := statsHandle.GenHistMissingColumns(neededColumns) if len(missingColumns) > 0 { stmtCtx := sctx.GetSessionVars().StmtCtx stmtCtx.StatsLoad.NeededColumns = missingColumns diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 2e988f4ee2866..6ea372bfca0a8 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -30,6 +30,26 @@ import ( "go.uber.org/zap" ) +// GenHistMissingColumns generates hist-missing columns based on neededColumns and statsCache. +func (h *Handle) GenHistMissingColumns(neededColumns map[model.TableColumnID]struct{}) []model.TableColumnID { + statsCache := h.statsCache.Load().(statsCache) + missingColumns := make([]model.TableColumnID, 0, len(neededColumns)) + for col := range neededColumns { + tbl, ok := statsCache.tables[col.TableID] + if !ok { + continue + } + colHist, ok := tbl.Columns[col.ColumnID] + if !ok { + continue + } + if colHist.IsHistNeeded() { + missingColumns = append(missingColumns, col) + } + } + return missingColumns +} + type NeededColumnsCh struct { ColumnsCh chan *NeededColumnTask TimeoutColumnsCh chan *NeededColumnTask @@ -52,7 +72,7 @@ func (h *Handle) AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, t var ErrExit = errors.New("Stop loading since domain is closed.") // SubLoadWorker loads hist data for each column -func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg sync.WaitGroup) error { +func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg *sync.WaitGroup) error { defer func() { exitWg.Done() logutil.BgLogger().Info("SubLoadWorker exited.") diff --git a/statistics/histogram.go b/statistics/histogram.go index a61f1d1405f59..a22f4c190bf0f 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1077,6 +1077,10 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) } +func (c *Column) IsHistNeeded() bool { + return !c.NotAccurate() && c.Histogram.NDV > 0 && c.notNullCount() == 0 +} + func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, encodedVal []byte, realtimeRowCount int64) (float64, error) { if val.IsNull() { return float64(c.NullCount), nil From 44ce671ef71f031b7d024f1117fb2bf0f43cb5ce Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 7 Dec 2021 11:17:34 +0800 Subject: [PATCH 07/38] fix to workable --- domain/domain.go | 2 +- planner/core/optimizer.go | 42 +++++++++++++++++++++++--------- statistics/handle/handle.go | 11 ++++----- statistics/handle/handle_hist.go | 27 +++++++++++--------- statistics/histogram.go | 5 ++-- 5 files changed, 55 insertions(+), 32 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 66f9789924899..f6dd3a6931bfc 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1269,7 +1269,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { func (do *Domain) StartLoadStatsSubWorkers(ctxList []sessionctx.Context) { statsHandle := do.StatsHandle() for i, ctx := range ctxList { - statsHandle.SubCtxs[i] = ctx + statsHandle.StatsLoad.SubCtxs[i] = ctx do.wg.Add(1) go statsHandle.SubLoadWorker(ctx, do.exit, &do.wg) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 50f1565da4e39..1556a923b4740 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" "math" + "sync" "time" ) @@ -322,9 +323,21 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err stmtCtx := sctx.GetSessionVars().StmtCtx stmtCtx.StatsLoad.NeededColumns = missingColumns wg := stmtCtx.StatsLoad.Wg + if wg == nil { + wg = &sync.WaitGroup{} + stmtCtx.StatsLoad.Wg = wg + } wg.Add(len(missingColumns)) - waitTime := mathutil.Min(int(syncWait), int(stmtCtx.MaxExecutionTime*1000)) - var timeout = time.Duration(waitTime) + hintMaxExecutionTime := stmtCtx.MaxExecutionTime + if hintMaxExecutionTime == 0 { + hintMaxExecutionTime = math.MaxInt + } + sessMaxExecutionTime := sctx.GetSessionVars().MaxExecutionTime + if sessMaxExecutionTime == 0 { + sessMaxExecutionTime = math.MaxInt + } + waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) + var timeout = time.Duration(waitTime) * time.Millisecond for _, col := range missingColumns { statsHandle.AppendNeededColumn(col, wg, timeout) } @@ -335,18 +348,25 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err return true, nil } -// collectNeededColumns to align with predicate-column collection +// collectNeededColumns simple implementation, TODO align with yifan func collectNeededColumns(plan LogicalPlan) map[model.TableColumnID]struct{} { - var neededColumns map[model.TableColumnID]struct{} + neededColumns := map[model.TableColumnID]struct{}{} + collectColumnsFromPlan(plan, neededColumns) + return neededColumns +} + +func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColumnID]struct{}) { + for _, child := range plan.Children() { + collectColumnsFromPlan(child, neededColumns) + } switch x := plan.(type) { - case *LogicalSelection: - exprs := x.Conditions - cols := make([]*expression.Column, 0, len(exprs)) - cols = expression.ExtractColumnsFromExpressions(cols, exprs, nil) - // TODO - default: + case *DataSource: + tblID := x.TableInfo().ID + for _, col := range x.Schema().Columns { + tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} + neededColumns[tblColID] = struct{}{} + } } - return neededColumns } // mergeContinuousSelections merge continuous selections which may occur after changing plans. diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index af88d89ba9c51..c755a0f041935 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -109,10 +109,8 @@ type Handle struct { // idxUsageListHead contains all the index usage collectors required by session. idxUsageListHead *SessionIndexUsageCollector - // SubCtxs holds all sessions used by sub stats load workers - SubCtxs []sessionctx.Context - // HistogramNeeded buffers the histogram needs from optimizer/statistics and is consumed by stats worker. - HistogramNeeded NeededColumnsCh + // statsLoad is used to load stats concurrently + StatsLoad StatsLoad } func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context.Context, sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { @@ -199,8 +197,6 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* feedback: statistics.NewQueryFeedbackMap(), idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)}, pool: pool, - SubCtxs: make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency), - HistogramNeeded: NeededColumnsCh{ColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize), TimeoutColumnsCh: make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize)}, } handle.lease.Store(lease) handle.pool = pool @@ -208,6 +204,9 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* handle.mu.ctx = ctx handle.mu.rateMap = make(errorRateDeltaMap) handle.statsCache.Store(statsCache{tables: make(map[int64]*statistics.Table)}) + handle.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency) + handle.StatsLoad.NeededColumnsCh = make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize) + handle.StatsLoad.TimeoutColumnsCh = make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize) err := handle.RefreshVars() if err != nil { return nil, err diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 6ea372bfca0a8..9e1d8b9efea78 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -43,15 +43,17 @@ func (h *Handle) GenHistMissingColumns(neededColumns map[model.TableColumnID]str if !ok { continue } - if colHist.IsHistNeeded() { + if colHist.IsHistNeeded(tbl.Pseudo) { missingColumns = append(missingColumns, col) } } return missingColumns } -type NeededColumnsCh struct { - ColumnsCh chan *NeededColumnTask +type StatsLoad struct { + sync.Mutex + SubCtxs []sessionctx.Context + NeededColumnsCh chan *NeededColumnTask TimeoutColumnsCh chan *NeededColumnTask } @@ -66,7 +68,7 @@ type NeededColumnTask struct { func (h *Handle) AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, timeout time.Duration) { toTimout := time.Now().Local().Add(timeout) colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, Wg: wg} - h.HistogramNeeded.ColumnsCh <- colTask + h.StatsLoad.NeededColumnsCh <- colTask } var ErrExit = errors.New("Stop loading since domain is closed.") @@ -146,7 +148,7 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { } hist, err := h.readStatsForOne(col, c, reader) if err != nil { - h.HistogramNeeded.ColumnsCh <- task + h.StatsLoad.NeededColumnsCh <- task return err } if hist != nil && h.updateCachedColumn(col, hist) { @@ -198,19 +200,20 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { for { to.Reset(timeout) select { // select ColumnsCh firstly since the priority - case task, ok := <-h.HistogramNeeded.ColumnsCh: + case task, ok := <-h.StatsLoad.NeededColumnsCh: if !ok { return nil, errors.New("drainColTask: cannot read from a closed ColumnsCh, maybe the chan is closed.") } if time.Now().After(task.ToTimeout) { - h.HistogramNeeded.TimeoutColumnsCh <- task + h.StatsLoad.NeededColumnsCh <- task continue } return task, nil case <-to.C(): to.SetRead() + to.Reset(timeout) select { // select TimeoutColumnsCh if there's no task from ColumnsCh currently - case task, ok := <-h.HistogramNeeded.TimeoutColumnsCh: + case task, ok := <-h.StatsLoad.TimeoutColumnsCh: if !ok { return nil, errors.New("drainColTask: cannot read from a closed TimeoutColumnsCh, maybe the chan is closed.") } @@ -218,19 +221,19 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { case <-to.C(): to.SetRead() continue + case <-exit: + return nil, ErrExit } case <-exit: return nil, ErrExit - case <-exit: - return nil, ErrExit } } } // updateCachedColumn updates the column hist to global statsCache. func (h *Handle) updateCachedColumn(col model.TableColumnID, colHist *statistics.Column) (updated bool) { - h.statsCache.Lock() - defer h.statsCache.Unlock() + h.StatsLoad.Lock() + defer h.StatsLoad.Unlock() // Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions // like `GetPartitionStats` called in `fmSketchFromStorage` would have modified the stats cache already. oldCache := h.statsCache.Load().(statsCache) diff --git a/statistics/histogram.go b/statistics/histogram.go index a22f4c190bf0f..d555e073f537f 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1071,14 +1071,15 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { if collPseudo && c.NotAccurate() { return true } + if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) } return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) } -func (c *Column) IsHistNeeded() bool { - return !c.NotAccurate() && c.Histogram.NDV > 0 && c.notNullCount() == 0 +func (c *Column) IsHistNeeded(collPseudo bool) bool { + return (!collPseudo || !c.NotAccurate()) && c.Histogram.NDV > 0 && c.notNullCount() == 0 } func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, encodedVal []byte, realtimeRowCount int64) (float64, error) { From 1783a2681ce6c0e829d74a762211c50db3ca16e2 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 7 Dec 2021 14:28:13 +0800 Subject: [PATCH 08/38] add metrics --- metrics/metrics.go | 3 +++ metrics/stats.go | 25 +++++++++++++++++++++++++ planner/core/optimizer.go | 6 ++++++ 3 files changed, 34 insertions(+) diff --git a/metrics/metrics.go b/metrics/metrics.go index 2002437667a03..b35ed11b3779d 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -97,6 +97,9 @@ func RegisterMetrics() { prometheus.MustRegister(HandleJobHistogram) prometheus.MustRegister(SignificantFeedbackCounter) prometheus.MustRegister(FastAnalyzeHistogram) + prometheus.MustRegister(SyncLoadCounter) + prometheus.MustRegister(SyncLoadTimeoutCounter) + prometheus.MustRegister(SyncLoadHistogram) prometheus.MustRegister(JobsGauge) prometheus.MustRegister(KeepAliveCounter) prometheus.MustRegister(LoadPrivilegeCounter) diff --git a/metrics/stats.go b/metrics/stats.go index a3347dd597716..d3bb36d949648 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -94,4 +94,29 @@ var ( Help: "Bucketed histogram of some stats in fast analyze.", Buckets: prometheus.ExponentialBuckets(1, 2, 16), }, []string{LblSQLType, LblType}) + + SyncLoadCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "sync_load_total", + Help: "Counter of sync load.", + }) + + SyncLoadTimeoutCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "sync_load_timeout_total", + Help: "Counter of sync load timeout.", + }) + + SyncLoadHistogram = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "sync_load_latency_millis", + Help: "Bucketed histogram of latency time (ms) of sync load.", + Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h + }) ) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 1556a923b4740..2b878f77484d0 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/lock" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" @@ -341,8 +342,13 @@ func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, err for _, col := range missingColumns { statsHandle.AppendNeededColumn(col, wg, timeout) } + metrics.SyncLoadCounter.Inc() + t := time.Now() if util.WaitTimeout(wg, timeout) { + metrics.SyncLoadTimeoutCounter.Inc() return false, errors.New("Fail to load stats for columns, timeout.") + } else { + metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) } } return true, nil From f360d21d40f6b568a75b2c7c25bb09a40085b9b3 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 8 Dec 2021 15:35:06 +0800 Subject: [PATCH 09/38] refactor sync-load --- planner/core/hist_column_collector.go | 45 +++++++++++++++ planner/core/optimizer.go | 83 +++++++-------------------- sessionctx/stmtctx/stmtctx.go | 4 +- statistics/handle/handle_hist.go | 65 +++++++++++++++------ statistics/histogram.go | 6 +- 5 files changed, 119 insertions(+), 84 deletions(-) create mode 100644 planner/core/hist_column_collector.go diff --git a/planner/core/hist_column_collector.go b/planner/core/hist_column_collector.go new file mode 100644 index 0000000000000..16c4125d76d73 --- /dev/null +++ b/planner/core/hist_column_collector.go @@ -0,0 +1,45 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import "github.com/pingcap/tidb/parser/model" + +// CollectHistColumns collects hist-needed columns from plan +func CollectHistColumns(plan LogicalPlan) []model.TableColumnID { + colMap := map[model.TableColumnID]struct{}{} + collectColumnsFromPlan(plan, colMap) + histColumns := make([]model.TableColumnID, 0, len(colMap)) + for col := range colMap { + histColumns = append(histColumns, col) + } + return histColumns +} + +func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColumnID]struct{}) { + for _, child := range plan.Children() { + collectColumnsFromPlan(child, neededColumns) + } + switch x := plan.(type) { + case *DataSource: + tblID := x.TableInfo().ID + for _, col := range x.Schema().Columns { + tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} + neededColumns[tblColID] = struct{}{} + } + // TODO collect idx columns + case *LogicalCTE: + // TODO + } +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 2b878f77484d0..1b13344e00c79 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/lock" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" @@ -34,13 +33,11 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" utilhint "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" "math" - "sync" "time" ) @@ -281,12 +278,14 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if err != nil { return nil, 0, err } - ok, err := SyncLoadNeededColumns(logic, sctx) - if !ok || err != nil { + ok := SyncLoadNeededColumns(logic, sctx) + if !ok { + err0 := errors.New("Timeout when sync-load histograms for needed columns.") if config.GetGlobalConfig().Stats.PseudoForLoadTimeout { + sctx.GetSessionVars().StmtCtx.AppendWarning(err0) sctx.GetSessionVars().StmtCtx.StatsLoad.Fallback = true } else { - return nil, 0, err + return nil, 0, err0 } } logic, err = logicalOptimize(ctx, flagAfterStats, logic) @@ -309,70 +308,28 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic } // SyncLoadNeededColumns sends column-hist request and sync-wait until timeout -func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) (bool, error) { +func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) bool { if sctx.GetSessionVars().InRestrictedSQL { - return true, nil + return true } syncWait := config.GetGlobalConfig().Stats.SyncLoadWait if syncWait <= 0 { - return true, nil - } - statsHandle := domain.GetDomain(sctx).StatsHandle() - neededColumns := collectNeededColumns(plan) - missingColumns := statsHandle.GenHistMissingColumns(neededColumns) - if len(missingColumns) > 0 { - stmtCtx := sctx.GetSessionVars().StmtCtx - stmtCtx.StatsLoad.NeededColumns = missingColumns - wg := stmtCtx.StatsLoad.Wg - if wg == nil { - wg = &sync.WaitGroup{} - stmtCtx.StatsLoad.Wg = wg - } - wg.Add(len(missingColumns)) - hintMaxExecutionTime := stmtCtx.MaxExecutionTime - if hintMaxExecutionTime == 0 { - hintMaxExecutionTime = math.MaxInt - } - sessMaxExecutionTime := sctx.GetSessionVars().MaxExecutionTime - if sessMaxExecutionTime == 0 { - sessMaxExecutionTime = math.MaxInt - } - waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) - var timeout = time.Duration(waitTime) * time.Millisecond - for _, col := range missingColumns { - statsHandle.AppendNeededColumn(col, wg, timeout) - } - metrics.SyncLoadCounter.Inc() - t := time.Now() - if util.WaitTimeout(wg, timeout) { - metrics.SyncLoadTimeoutCounter.Inc() - return false, errors.New("Fail to load stats for columns, timeout.") - } else { - metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) - } + return true } - return true, nil -} - -// collectNeededColumns simple implementation, TODO align with yifan -func collectNeededColumns(plan LogicalPlan) map[model.TableColumnID]struct{} { - neededColumns := map[model.TableColumnID]struct{}{} - collectColumnsFromPlan(plan, neededColumns) - return neededColumns -} - -func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColumnID]struct{}) { - for _, child := range plan.Children() { - collectColumnsFromPlan(child, neededColumns) + neededColumns := CollectHistColumns(plan) + stmtCtx := sctx.GetSessionVars().StmtCtx + hintMaxExecutionTime := stmtCtx.MaxExecutionTime + if hintMaxExecutionTime == 0 { + hintMaxExecutionTime = math.MaxInt } - switch x := plan.(type) { - case *DataSource: - tblID := x.TableInfo().ID - for _, col := range x.Schema().Columns { - tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} - neededColumns[tblColID] = struct{}{} - } + sessMaxExecutionTime := sctx.GetSessionVars().MaxExecutionTime + if sessMaxExecutionTime == 0 { + sessMaxExecutionTime = math.MaxInt } + waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) + var timeout = time.Duration(waitTime) * time.Millisecond + stmtCtx.StatsLoad.Timeout = timeout + return domain.GetDomain(sctx).StatsHandle().SyncLoad(stmtCtx, neededColumns, timeout) } // mergeContinuousSelections merge continuous selections which may occur after changing plans. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 73a49b1f1a565..b89ccee654ac8 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -205,7 +205,9 @@ type StatementContext struct { LogicalOptimizeTrace *tracing.LogicalOptimizeTracer StatsLoad struct { - // NeededColumnMap stores the columns whose stats are needed for planner. + // Timeout to wait for sync-load + Timeout time.Duration + // NeededColumns stores the columns whose stats are needed for planner. NeededColumns []model.TableColumnID // Wg is the wait group waiting for all need columns to be loaded. Wg *sync.WaitGroup diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 9e1d8b9efea78..ae489d15e0828 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -23,18 +23,59 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" "go.uber.org/zap" ) -// GenHistMissingColumns generates hist-missing columns based on neededColumns and statsCache. -func (h *Handle) GenHistMissingColumns(neededColumns map[model.TableColumnID]struct{}) []model.TableColumnID { +// TODO load idx histograms by need + +type StatsLoad struct { + sync.Mutex + SubCtxs []sessionctx.Context + NeededColumnsCh chan *NeededColumnTask + TimeoutColumnsCh chan *NeededColumnTask +} + +// NeededColumnTask represents one needed column with expire time. +type NeededColumnTask struct { + TableColumnID model.TableColumnID + ToTimeout time.Time + Wg *sync.WaitGroup +} + +// SyncLoad sync waits loading of neededColumns and return false if timeout +func (h *Handle) SyncLoad(sc *stmtctx.StatementContext, neededColumns []model.TableColumnID, timeout time.Duration) bool { + missingColumns := h.genHistMissingColumns(neededColumns) + if len(missingColumns) <= 0 { + return true + } + sc.StatsLoad.NeededColumns = missingColumns + sc.StatsLoad.Wg = &sync.WaitGroup{} + sc.StatsLoad.Wg.Add(len(missingColumns)) + for _, col := range missingColumns { + h.appendNeededColumn(col, sc.StatsLoad.Wg, timeout) + } + metrics.SyncLoadCounter.Inc() + t := time.Now() + if util.WaitTimeout(sc.StatsLoad.Wg, timeout) { + metrics.SyncLoadTimeoutCounter.Inc() + return false + } else { + metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) + return true + } +} + +// genHistMissingColumns generates hist-missing columns based on neededColumns and statsCache. +func (h *Handle) genHistMissingColumns(neededColumns []model.TableColumnID) []model.TableColumnID { statsCache := h.statsCache.Load().(statsCache) missingColumns := make([]model.TableColumnID, 0, len(neededColumns)) - for col := range neededColumns { + for _, col := range neededColumns { tbl, ok := statsCache.tables[col.TableID] if !ok { continue @@ -50,22 +91,8 @@ func (h *Handle) GenHistMissingColumns(neededColumns map[model.TableColumnID]str return missingColumns } -type StatsLoad struct { - sync.Mutex - SubCtxs []sessionctx.Context - NeededColumnsCh chan *NeededColumnTask - TimeoutColumnsCh chan *NeededColumnTask -} - -// NeededColumnTask represents one needed column with expire time. -type NeededColumnTask struct { - TableColumnID model.TableColumnID - ToTimeout time.Time - Wg *sync.WaitGroup -} - -// AppendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. -func (h *Handle) AppendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, timeout time.Duration) { +// appendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. +func (h *Handle) appendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, timeout time.Duration) { toTimout := time.Now().Local().Add(timeout) colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, Wg: wg} h.StatsLoad.NeededColumnsCh <- colTask diff --git a/statistics/histogram.go b/statistics/histogram.go index d555e073f537f..c82186f23f203 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -19,6 +19,7 @@ import ( "fmt" "math" "sort" + "strconv" "strings" "time" "unsafe" @@ -1071,8 +1072,11 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { if collPseudo && c.NotAccurate() { return true } - if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { + if len(sc.StatsLoad.NeededColumns) > 0 { + logutil.BgLogger().Warn("Hist for column %v should already be loaded as sync but not found.", + zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) + } HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) } return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) From a815170ee711e8e9c1f5ac9217ae499ee5825825 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 8 Dec 2021 16:44:42 +0800 Subject: [PATCH 10/38] fix concurrent issue --- metrics/metrics.go | 1 + metrics/stats.go | 9 +++++++++ statistics/handle/handle_hist.go | 30 +++++++++++++++++++++++++++++- 3 files changed, 39 insertions(+), 1 deletion(-) diff --git a/metrics/metrics.go b/metrics/metrics.go index b35ed11b3779d..a21f41a22f163 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -100,6 +100,7 @@ func RegisterMetrics() { prometheus.MustRegister(SyncLoadCounter) prometheus.MustRegister(SyncLoadTimeoutCounter) prometheus.MustRegister(SyncLoadHistogram) + prometheus.MustRegister(ReadStatsHistogram) prometheus.MustRegister(JobsGauge) prometheus.MustRegister(KeepAliveCounter) prometheus.MustRegister(LoadPrivilegeCounter) diff --git a/metrics/stats.go b/metrics/stats.go index d3bb36d949648..c4b74cf088915 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -119,4 +119,13 @@ var ( Help: "Bucketed histogram of latency time (ms) of sync load.", Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h }) + + ReadStatsHistogram = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "read_stats_latency_millis", + Help: "Bucketed histogram of latency time (ms) of stats read during sync-load.", + Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h + }) ) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index ae489d15e0828..d803e53a661e3 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -39,6 +39,7 @@ type StatsLoad struct { SubCtxs []sessionctx.Context NeededColumnsCh chan *NeededColumnTask TimeoutColumnsCh chan *NeededColumnTask + workingColMap map[model.TableColumnID]struct{} } // NeededColumnTask represents one needed column with expire time. @@ -125,7 +126,7 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitW case ErrExit: return nil default: - time.Sleep(500 * time.Millisecond) + time.Sleep(10 * time.Millisecond) continue } } @@ -162,6 +163,10 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { return err0 } col := task.TableColumnID + // to avoid duplicated handling in concurrent scenario + if !h.setWorking(col) { + return nil + } oldCache := h.statsCache.Load().(statsCache) tbl, ok := oldCache.tables[col.TableID] if !ok { @@ -173,14 +178,17 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { task.Wg.Done() return nil } + t := time.Now() hist, err := h.readStatsForOne(col, c, reader) if err != nil { h.StatsLoad.NeededColumnsCh <- task return err } + metrics.ReadStatsHistogram.Observe(float64(time.Since(t).Milliseconds())) if hist != nil && h.updateCachedColumn(col, hist) { task.Wg.Done() } + h.finishWorking(col) return nil } @@ -276,3 +284,23 @@ func (h *Handle) updateCachedColumn(col model.TableColumnID, colHist *statistics tbl.Columns[c.ID] = colHist return h.updateStatsCache(oldCache.update([]*statistics.Table{tbl}, nil, oldCache.version)) } + +func (h *Handle) setWorking(col model.TableColumnID) bool { + h.StatsLoad.Lock() + defer h.StatsLoad.Unlock() + if h.StatsLoad.workingColMap == nil { + h.StatsLoad.workingColMap = map[model.TableColumnID]struct{}{} + } + if _, ok := h.StatsLoad.workingColMap[col]; ok { + return false + } else { + h.StatsLoad.workingColMap[col] = struct{}{} + return true + } +} + +func (h *Handle) finishWorking(col model.TableColumnID) { + h.StatsLoad.Lock() + defer h.StatsLoad.Unlock() + delete(h.StatsLoad.workingColMap, col) +} From f10fffac5292f1553a2559661758a225438fbfa3 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Fri, 10 Dec 2021 11:51:42 +0800 Subject: [PATCH 11/38] add partial uts --- statistics/handle/handle_hist.go | 4 +- statistics/handle/handle_hist_test.go | 64 +++++++++++++++++++++++++++ statistics/histogram.go | 4 +- util/timeutil/timer.go | 6 +-- util/wait_group_test.go | 40 +++++++++++++++++ 5 files changed, 112 insertions(+), 6 deletions(-) create mode 100644 statistics/handle/handle_hist_test.go create mode 100644 util/wait_group_test.go diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index d803e53a661e3..c6f1d160fe621 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -159,7 +159,9 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { }() task, err0 := h.drainColTask(exit) if err0 != nil && task == nil { - logutil.BgLogger().Fatal("Fail to drain task for stats loading.") + if err0 != ErrExit { + logutil.BgLogger().Fatal("Fail to drain task for stats loading.") + } return err0 } col := task.TableColumnID diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go new file mode 100644 index 0000000000000..d0089259c8454 --- /dev/null +++ b/statistics/handle/handle_hist_test.go @@ -0,0 +1,64 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package handle_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/util/testkit" + "time" +) + +func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") + testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") + + oriLease := s.do.StatsHandle().Lease() + s.do.StatsHandle().SetLease(1) + defer func() { + s.do.StatsHandle().SetLease(oriLease) + }() + testKit.MustExec("analyze table t") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + stat := h.GetTableStats(tableInfo) + hg := stat.Columns[tableInfo.Columns[0].ID].Histogram + topn := stat.Columns[tableInfo.Columns[0].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Equals, 0) + stmtCtx := &stmtctx.StatementContext{} + neededColumns := make([]model.TableColumnID, 0, len(tableInfo.Columns)) + for _, col := range tableInfo.Columns { + neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) + } + rs := h.SyncLoad(stmtCtx, neededColumns, time.Hour) + c.Assert(rs, Equals, true) + stat = h.GetTableStats(tableInfo) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) +} diff --git a/statistics/histogram.go b/statistics/histogram.go index c82186f23f203..38db742297c34 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1069,11 +1069,11 @@ var HistogramNeededColumns = neededColumnMap{cols: map[tableColumnID]struct{}{}} // IsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it // as need histogram. func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { - if collPseudo && c.NotAccurate() { + if (sc != nil && sc.StatsLoad.Fallback) || (collPseudo && c.NotAccurate()) { return true } if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { - if len(sc.StatsLoad.NeededColumns) > 0 { + if sc.StatsLoad.Timeout > 0 { logutil.BgLogger().Warn("Hist for column %v should already be loaded as sync but not found.", zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) } diff --git a/util/timeutil/timer.go b/util/timeutil/timer.go index 65d175996bca4..138d8a120774f 100644 --- a/util/timeutil/timer.go +++ b/util/timeutil/timer.go @@ -19,9 +19,9 @@ import ( ) // GoodTimer wraps the standard time.Timer to provide more user-friendly interfaces. -// As Russ Cox suggested (here and here), the correct way to use time.Timer is: -// All the Timer operations (Timer.Stop, Timer.Reset and receiving from or draining the channel) should be done in the same goroutine. -// The program should manage an extra status showing whether it has received from the Timer's channel or not. +// As Russ Cox suggested, the correct way to use time.Timer is: +// 1. All the Timer operations (Timer.Stop, Timer.Reset and receiving from or draining the channel) should be done in the same goroutine. +// 2. The program should manage an extra status showing whether it has received from the Timer's channel or not. // **NOTE**: All the functions of GoodTimer *should* be used in the same goroutine. type GoodTimer struct { t *time.Timer // The actual timer diff --git a/util/wait_group_test.go b/util/wait_group_test.go new file mode 100644 index 0000000000000..225dd25c2a6a9 --- /dev/null +++ b/util/wait_group_test.go @@ -0,0 +1,40 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestWaitGroupTimeout(t *testing.T) { + t.Parallel() + wg := &sync.WaitGroup{} + wg.Add(1) + go func() { + wg.Done() + }() + timeout := time.Millisecond * 10 + rs := WaitTimeout(wg, timeout) + assert.Equal(t, false, rs) + wg.Add(1) + timeout = time.Nanosecond * 10 + rs = WaitTimeout(wg, timeout) + assert.Equal(t, true, rs) + wg.Done() +} From 78c6fa4ced7aa0c9c416dd2699b5935d38b3aeee Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Mon, 13 Dec 2021 13:14:31 +0800 Subject: [PATCH 12/38] refactor --- planner/core/hist_column_collector.go | 15 ++- planner/core/optimizer.go | 58 ++++++------ planner/core/rule_aggregation_elimination.go | 4 + planner/core/rule_aggregation_push_down.go | 4 + planner/core/rule_build_key_info.go | 4 + planner/core/rule_column_pruning.go | 4 + planner/core/rule_decorrelate.go | 4 + planner/core/rule_eliminate_projection.go | 4 + .../core/rule_generate_column_substitute.go | 4 + planner/core/rule_join_elimination.go | 4 + planner/core/rule_join_reorder.go | 4 + planner/core/rule_max_min_eliminate.go | 4 + planner/core/rule_partition_processor.go | 4 + planner/core/rule_predicate_push_down.go | 4 + planner/core/rule_result_reorder.go | 4 + planner/core/rule_topn_push_down.go | 4 + planner/core/stats_load_test.go | 1 + sessionctx/stmtctx/stmtctx.go | 4 +- statistics/handle/handle.go | 1 + statistics/handle/handle_hist.go | 93 +++++++++++-------- statistics/handle/handle_hist_test.go | 4 +- util/timeutil/timer.go | 39 +++----- util/wait_group.go | 34 ------- util/wait_group_test.go | 40 -------- 24 files changed, 169 insertions(+), 176 deletions(-) create mode 100644 planner/core/stats_load_test.go delete mode 100644 util/wait_group.go delete mode 100644 util/wait_group_test.go diff --git a/planner/core/hist_column_collector.go b/planner/core/hist_column_collector.go index 16c4125d76d73..7bda17aa64bde 100644 --- a/planner/core/hist_column_collector.go +++ b/planner/core/hist_column_collector.go @@ -14,7 +14,10 @@ package core -import "github.com/pingcap/tidb/parser/model" +import ( + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/model" +) // CollectHistColumns collects hist-needed columns from plan func CollectHistColumns(plan LogicalPlan) []model.TableColumnID { @@ -34,12 +37,16 @@ func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColum switch x := plan.(type) { case *DataSource: tblID := x.TableInfo().ID - for _, col := range x.Schema().Columns { + columns := expression.ExtractColumnsFromExpressions(nil, x.pushedDownConds, nil) + for _, col := range columns { tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} neededColumns[tblColID] = struct{}{} } - // TODO collect idx columns + // TODO collect idx columns? case *LogicalCTE: - // TODO + collectColumnsFromPlan(x.cte.seedPartLogicalPlan, neededColumns) + if x.cte.recursivePartLogicalPlan != nil { + collectColumnsFromPlan(x.cte.recursivePartLogicalPlan, neededColumns) + } } } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 7f5454d350b83..022085463c0b1 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -127,6 +127,7 @@ func (op *logicalOptimizeOp) recordFinalLogicalPlan(final LogicalPlan) { type logicalOptRule interface { optimize(context.Context, LogicalPlan, *logicalOptimizeOp) (LogicalPlan, error) name() string + needStats() bool } // BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node. @@ -266,29 +267,7 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if checkStableResultMode(sctx) { flag |= flagStabilizeResults } - var flagBeforeStats = flag - var flagAfterStats uint64 = 0 - for _, flg := range flagRulesAfterStats { - if flagBeforeStats&flg > 0 { - flagBeforeStats -= flg - flagAfterStats |= flg - } - } - logic, err := logicalOptimize(ctx, flagBeforeStats, logic) - if err != nil { - return nil, 0, err - } - ok := SyncLoadNeededColumns(logic, sctx) - if !ok { - err0 := errors.New("Timeout when sync-load histograms for needed columns.") - if config.GetGlobalConfig().Stats.PseudoForLoadTimeout { - sctx.GetSessionVars().StmtCtx.AppendWarning(err0) - sctx.GetSessionVars().StmtCtx.StatsLoad.Fallback = true - } else { - return nil, 0, err0 - } - } - logic, err = logicalOptimize(ctx, flagAfterStats, logic) + logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, 0, err } @@ -307,29 +286,37 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic return finalPlan, cost, nil } -// SyncLoadNeededColumns sends column-hist request and sync-wait until timeout -func SyncLoadNeededColumns(plan LogicalPlan, sctx sessionctx.Context) bool { - if sctx.GetSessionVars().InRestrictedSQL { - return true +// SyncLoadColumnFullStats sends columns' full-stats request and sync-wait until timeout +func SyncLoadColumnFullStats(plan LogicalPlan) (bool, error) { + if plan.SCtx().GetSessionVars().InRestrictedSQL { + return true, nil } syncWait := config.GetGlobalConfig().Stats.SyncLoadWait if syncWait <= 0 { - return true + return true, nil } neededColumns := CollectHistColumns(plan) - stmtCtx := sctx.GetSessionVars().StmtCtx + stmtCtx := plan.SCtx().GetSessionVars().StmtCtx hintMaxExecutionTime := stmtCtx.MaxExecutionTime if hintMaxExecutionTime == 0 { hintMaxExecutionTime = math.MaxInt } - sessMaxExecutionTime := sctx.GetSessionVars().MaxExecutionTime + sessMaxExecutionTime := plan.SCtx().GetSessionVars().MaxExecutionTime if sessMaxExecutionTime == 0 { sessMaxExecutionTime = math.MaxInt } waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) var timeout = time.Duration(waitTime) * time.Millisecond stmtCtx.StatsLoad.Timeout = timeout - return domain.GetDomain(sctx).StatsHandle().SyncLoad(stmtCtx, neededColumns, timeout) + success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncLoad(stmtCtx, neededColumns, timeout) + err := errors.New("Timeout when sync-load full stats for needed columns.") + if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { + stmtCtx.AppendWarning(err) + stmtCtx.StatsLoad.Fallback = true + return false, err + } else { + return true, nil + } } // mergeContinuousSelections merge continuous selections which may occur after changing plans. @@ -439,6 +426,7 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic }() } var err error + fullStatsLoaded := false for i, rule := range optRuleList { // The order of flags is same as the order of optRule in the list. // We use a bitmask to record which opt rules should be used. If the i-th bit is 1, it means we should @@ -447,6 +435,14 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic continue } opt.appendBeforeRuleOptimize(i, rule.name(), logic) + // sync-load full stats before the first stats-needed rule applied + if !fullStatsLoaded && rule.needStats() { + _, err = SyncLoadColumnFullStats(logic) + if err != nil { + return nil, err + } + fullStatsLoaded = true + } logic, err = rule.optimize(ctx, logic, opt) if err != nil { return nil, err diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 61d9e0f117e0d..24da4ba93e0cd 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -31,6 +31,10 @@ type aggregationEliminator struct { aggregationEliminateChecker } +func (a *aggregationEliminator) needStats() bool { + return false +} + type aggregationEliminateChecker struct { } diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 7cc0548a7f57d..682a6f1274433 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -31,6 +31,10 @@ type aggregationPushDownSolver struct { aggregationEliminateChecker } +func (a *aggregationPushDownSolver) needStats() bool { + return false +} + // isDecomposable checks if an aggregate function is decomposable. An aggregation function $F$ is decomposable // if there exist aggregation functions F_1 and F_2 such that F(S_1 union all S_2) = F_2(F_1(S_1),F_1(S_2)), // where S_1 and S_2 are two sets of values. We call S_1 and S_2 partial groups. diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index ae8a0a1d3a566..96170274a07c2 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -25,6 +25,10 @@ import ( type buildKeySolver struct{} +func (s *buildKeySolver) needStats() bool { + return false +} + func (s *buildKeySolver) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { buildKeyInfo(p) return p, nil diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 1d144c72807f9..ca97fd1f02be6 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -29,6 +29,10 @@ import ( type columnPruner struct { } +func (s *columnPruner) needStats() bool { + return false +} + func (s *columnPruner) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { err := lp.PruneColumns(lp.Schema().Columns) return lp, err diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index a8835d57448b9..6433e2d353493 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -105,6 +105,10 @@ func ExtractCorrelatedCols4PhysicalPlan(p PhysicalPlan) []*expression.Correlated // decorrelateSolver tries to convert apply plan to join plan. type decorrelateSolver struct{} +func (s *decorrelateSolver) needStats() bool { + return false +} + func (s *decorrelateSolver) aggDefaultValueMap(agg *LogicalAggregation) map[int]*expression.Constant { defaultValueMap := make(map[int]*expression.Constant, len(agg.AggFuncs)) for i, f := range agg.AggFuncs { diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 0c377bd974703..90c16d9efe3af 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -146,6 +146,10 @@ func eliminatePhysicalProjection(p PhysicalPlan) PhysicalPlan { type projectionEliminator struct { } +func (pe *projectionEliminator) needStats() bool { + return false +} + // optimize implements the logicalOptRule interface. func (pe *projectionEliminator) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { root := pe.eliminate(lp, make(map[string]*expression.Column), false, opt) diff --git a/planner/core/rule_generate_column_substitute.go b/planner/core/rule_generate_column_substitute.go index d3ec3d5c7960c..de6aaa0d686c7 100644 --- a/planner/core/rule_generate_column_substitute.go +++ b/planner/core/rule_generate_column_substitute.go @@ -27,6 +27,10 @@ import ( type gcSubstituter struct { } +func (gc *gcSubstituter) needStats() bool { + return false +} + // ExprColumnMap is used to store all expressions of indexed generated columns in a table, // and map them to the generated columns, // thus we can substitute the expression in a query to an indexed generated column. diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index a88db921d5a56..a91016230efa7 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -25,6 +25,10 @@ import ( type outerJoinEliminator struct { } +func (o *outerJoinEliminator) needStats() bool { + return false +} + // tryToEliminateOuterJoin will eliminate outer join plan base on the following rules // 1. outer join elimination: For example left outer join, if the parent only use the // columns from left table and the join key of right table(the inner table) is a unique diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 9fb38e572d228..45a7f32de5dda 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -50,6 +50,10 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression type joinReOrderSolver struct { } +func (s *joinReOrderSolver) needStats() bool { + return true +} + type jrNode struct { p LogicalPlan cumCost float64 diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index efad9c9296459..344667e66e0ae 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -34,6 +34,10 @@ import ( type maxMinEliminator struct { } +func (a *maxMinEliminator) needStats() bool { + return false +} + func (a *maxMinEliminator) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { return a.eliminateMaxMin(p), nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 7c3bbb565c69d..0bb779ef83374 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -57,6 +57,10 @@ const FullRange = -1 // partitionProcessor is here because it's easier to prune partition after predicate push down. type partitionProcessor struct{} +func (s *partitionProcessor) needStats() bool { + return false +} + func (s *partitionProcessor) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { p, err := s.rewriteDataSource(lp) return p, err diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index f59fedc25ba28..ec6d7b0b68707 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -27,6 +27,10 @@ import ( type ppdSolver struct{} +func (s *ppdSolver) needStats() bool { + return false +} + func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { _, p := lp.PredicatePushDown(nil) return p, nil diff --git a/planner/core/rule_result_reorder.go b/planner/core/rule_result_reorder.go index 7ea7d73556b4d..827d3fbe675c6 100644 --- a/planner/core/rule_result_reorder.go +++ b/planner/core/rule_result_reorder.go @@ -37,6 +37,10 @@ import ( type resultReorder struct { } +func (rs *resultReorder) needStats() bool { + return false +} + func (rs *resultReorder) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { ordered := rs.completeSort(lp) if !ordered { diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index e6234bbc3f3dc..e3b6ea51bb211 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -26,6 +26,10 @@ import ( type pushDownTopNOptimizer struct { } +func (s *pushDownTopNOptimizer) needStats() bool { + return false +} + func (s *pushDownTopNOptimizer) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { return p.pushDownTopN(nil), nil } diff --git a/planner/core/stats_load_test.go b/planner/core/stats_load_test.go new file mode 100644 index 0000000000000..9a8bc9592b0bc --- /dev/null +++ b/planner/core/stats_load_test.go @@ -0,0 +1 @@ +package core diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 03ecff5f120bb..e50a97af23abc 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -209,8 +209,8 @@ type StatementContext struct { Timeout time.Duration // NeededColumns stores the columns whose stats are needed for planner. NeededColumns []model.TableColumnID - // Wg is the wait group waiting for all need columns to be loaded. - Wg *sync.WaitGroup + // ResultCh to receive stats loading results + ResultCh chan model.TableColumnID // Fallback indicates if the planner uses full-loaded stats or fallback all to pseudo/simple. Fallback bool } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index c755a0f041935..c9a44f857ddd5 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -207,6 +207,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration, pool sessionPool) (* handle.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency) handle.StatsLoad.NeededColumnsCh = make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize) handle.StatsLoad.TimeoutColumnsCh = make(chan *NeededColumnTask, cfg.Performance.StatsLoadQueueSize) + handle.StatsLoad.workingColMap = map[model.TableColumnID][]chan model.TableColumnID{} err := handle.RefreshVars() if err != nil { return nil, err diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index c6f1d160fe621..7692800df63e5 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -25,28 +25,25 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" "go.uber.org/zap" ) -// TODO load idx histograms by need - type StatsLoad struct { sync.Mutex SubCtxs []sessionctx.Context NeededColumnsCh chan *NeededColumnTask TimeoutColumnsCh chan *NeededColumnTask - workingColMap map[model.TableColumnID]struct{} + workingColMap map[model.TableColumnID][]chan model.TableColumnID } // NeededColumnTask represents one needed column with expire time. type NeededColumnTask struct { TableColumnID model.TableColumnID ToTimeout time.Time - Wg *sync.WaitGroup + ResultCh chan model.TableColumnID } // SyncLoad sync waits loading of neededColumns and return false if timeout @@ -56,19 +53,32 @@ func (h *Handle) SyncLoad(sc *stmtctx.StatementContext, neededColumns []model.Ta return true } sc.StatsLoad.NeededColumns = missingColumns - sc.StatsLoad.Wg = &sync.WaitGroup{} - sc.StatsLoad.Wg.Add(len(missingColumns)) + sc.StatsLoad.ResultCh = make(chan model.TableColumnID, len(neededColumns)) + defer close(sc.StatsLoad.ResultCh) + resultCheckMap := map[model.TableColumnID]struct{}{} for _, col := range missingColumns { - h.appendNeededColumn(col, sc.StatsLoad.Wg, timeout) + h.appendNeededColumn(col, sc.StatsLoad.ResultCh, timeout) + resultCheckMap[col] = struct{}{} } metrics.SyncLoadCounter.Inc() + timeoutTimer := timeutil.NewWrappedTimer(timeout) + defer timeoutTimer.Stop() t := time.Now() - if util.WaitTimeout(sc.StatsLoad.Wg, timeout) { - metrics.SyncLoadTimeoutCounter.Inc() - return false - } else { - metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) - return true + for { + select { + case result, ok := <-sc.StatsLoad.ResultCh: + if ok { + delete(resultCheckMap, result) + if len(resultCheckMap) == 0 { + metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) + return true + } + } + case <-timeoutTimer.C(): + timeoutTimer.SetRead() + metrics.SyncLoadTimeoutCounter.Inc() + return false + } } } @@ -93,13 +103,13 @@ func (h *Handle) genHistMissingColumns(neededColumns []model.TableColumnID) []mo } // appendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. -func (h *Handle) appendNeededColumn(c model.TableColumnID, wg *sync.WaitGroup, timeout time.Duration) { +func (h *Handle) appendNeededColumn(c model.TableColumnID, resultCh chan model.TableColumnID, timeout time.Duration) { toTimout := time.Now().Local().Add(timeout) - colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, Wg: wg} + colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, ResultCh: resultCh} h.StatsLoad.NeededColumnsCh <- colTask } -var ErrExit = errors.New("Stop loading since domain is closed.") +var errExit = errors.New("Stop loading since domain is closed.") // SubLoadWorker loads hist data for each column func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg *sync.WaitGroup) error { @@ -123,7 +133,7 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitW err := h.handleOneTask(reader, exit) if err != nil { switch err { - case ErrExit: + case errExit: return nil default: time.Sleep(10 * time.Millisecond) @@ -159,25 +169,25 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { }() task, err0 := h.drainColTask(exit) if err0 != nil && task == nil { - if err0 != ErrExit { - logutil.BgLogger().Fatal("Fail to drain task for stats loading.") + if err0 != errExit { + logutil.BgLogger().Error("Fail to drain task for stats loading.") } return err0 } col := task.TableColumnID // to avoid duplicated handling in concurrent scenario - if !h.setWorking(col) { + if !h.setWorking(col, task.ResultCh) { return nil } oldCache := h.statsCache.Load().(statsCache) tbl, ok := oldCache.tables[col.TableID] if !ok { - task.Wg.Done() + task.ResultCh <- col return nil } c, ok := tbl.Columns[col.ColumnID] if !ok || c.Len() > 0 { - task.Wg.Done() + task.ResultCh <- col return nil } t := time.Now() @@ -188,7 +198,7 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { } metrics.ReadStatsHistogram.Observe(float64(time.Since(t).Milliseconds())) if hist != nil && h.updateCachedColumn(col, hist) { - task.Wg.Done() + task.ResultCh <- col } h.finishWorking(col) return nil @@ -232,24 +242,25 @@ func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, // drainColTask will hang until a column task can return. func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { - timeout := time.Nanosecond * 100 - to := timeutil.NewGoodTimer(timeout) + timeout := time.Millisecond + to := timeutil.NewWrappedTimer(timeout) for { to.Reset(timeout) - select { // select ColumnsCh firstly since the priority + // select NeededColumnsCh firstly, if no task, then select TimeoutColumnsCh + select { case task, ok := <-h.StatsLoad.NeededColumnsCh: if !ok { - return nil, errors.New("drainColTask: cannot read from a closed ColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from a closed NeededColumnsCh, maybe the chan is closed.") } if time.Now().After(task.ToTimeout) { - h.StatsLoad.NeededColumnsCh <- task + h.StatsLoad.TimeoutColumnsCh <- task continue } return task, nil case <-to.C(): to.SetRead() to.Reset(timeout) - select { // select TimeoutColumnsCh if there's no task from ColumnsCh currently + select { case task, ok := <-h.StatsLoad.TimeoutColumnsCh: if !ok { return nil, errors.New("drainColTask: cannot read from a closed TimeoutColumnsCh, maybe the chan is closed.") @@ -259,10 +270,10 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { to.SetRead() continue case <-exit: - return nil, ErrExit + return nil, errExit } case <-exit: - return nil, ErrExit + return nil, errExit } } } @@ -287,16 +298,17 @@ func (h *Handle) updateCachedColumn(col model.TableColumnID, colHist *statistics return h.updateStatsCache(oldCache.update([]*statistics.Table{tbl}, nil, oldCache.version)) } -func (h *Handle) setWorking(col model.TableColumnID) bool { +func (h *Handle) setWorking(col model.TableColumnID, resultCh chan model.TableColumnID) bool { h.StatsLoad.Lock() defer h.StatsLoad.Unlock() - if h.StatsLoad.workingColMap == nil { - h.StatsLoad.workingColMap = map[model.TableColumnID]struct{}{} - } - if _, ok := h.StatsLoad.workingColMap[col]; ok { + chList, ok := h.StatsLoad.workingColMap[col] + if ok { + chList = append(chList, resultCh) return false } else { - h.StatsLoad.workingColMap[col] = struct{}{} + chList = []chan model.TableColumnID{} + chList = append(chList, resultCh) + h.StatsLoad.workingColMap[col] = chList return true } } @@ -304,5 +316,10 @@ func (h *Handle) setWorking(col model.TableColumnID) bool { func (h *Handle) finishWorking(col model.TableColumnID) { h.StatsLoad.Lock() defer h.StatsLoad.Unlock() + if chList, ok := h.StatsLoad.workingColMap[col]; ok { + for _, ch := range chList { + ch <- col + } + } delete(h.StatsLoad.workingColMap, col) } diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index d0089259c8454..5f111996c2d4a 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/testkit" + "math" "time" ) @@ -55,7 +56,8 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { for _, col := range tableInfo.Columns { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } - rs := h.SyncLoad(stmtCtx, neededColumns, time.Hour) + timeout := time.Nanosecond * math.MaxInt + rs := h.SyncLoad(stmtCtx, neededColumns, timeout) c.Assert(rs, Equals, true) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram diff --git a/util/timeutil/timer.go b/util/timeutil/timer.go index 138d8a120774f..891b62f0a97c3 100644 --- a/util/timeutil/timer.go +++ b/util/timeutil/timer.go @@ -18,24 +18,26 @@ import ( "time" ) -// GoodTimer wraps the standard time.Timer to provide more user-friendly interfaces. +// WrappedTimer wraps the standard time.Timer to handle clean stop and reset. // As Russ Cox suggested, the correct way to use time.Timer is: // 1. All the Timer operations (Timer.Stop, Timer.Reset and receiving from or draining the channel) should be done in the same goroutine. // 2. The program should manage an extra status showing whether it has received from the Timer's channel or not. -// **NOTE**: All the functions of GoodTimer *should* be used in the same goroutine. -type GoodTimer struct { +// https://github.com/golang/go/issues/11513#issuecomment-157062583 +// https://groups.google.com/g/golang-dev/c/c9UUfASVPoU/m/tlbK2BpFEwAJ +// **NOTE**: All the functions of WrappedTimer *should* be used in the same goroutine. +type WrappedTimer struct { t *time.Timer // The actual timer read bool // Whether t.C has already been read from } -// NewGoodTimer creates an instance of GoodTimer. -func NewGoodTimer(d time.Duration) *GoodTimer { - return &GoodTimer{t: time.NewTimer(d)} +// NewWrappedTimer creates an instance of WrappedTimer. +func NewWrappedTimer(d time.Duration) *WrappedTimer { + return &WrappedTimer{t: time.NewTimer(d)} } // ReadC waits until it can read from the wrapped timer's channel C. // It returns the time value received from the channel C, a zero time value if the channel C has already been read from. -func (gt *GoodTimer) ReadC() time.Time { +func (gt *WrappedTimer) ReadC() time.Time { if gt.read { return time.Time{} } @@ -44,33 +46,18 @@ func (gt *GoodTimer) ReadC() time.Time { return tv } -// TryReadC waits for at most the duration d, in order to read from the wrapped timer's channel C. -// It returns the time value received from the channel C, a zero time value if the channel C has already been read from or if the timeout is reached. -func (gt *GoodTimer) TryReadC(timeout time.Duration) time.Time { - if gt.read { - return time.Time{} - } - select { - case tv := <-gt.t.C: - gt.read = true - return tv - case <-time.After(timeout): - return time.Time{} - } -} - // C returns the chan of wrapped timer for select. -func (gt *GoodTimer) C() <-chan time.Time { +func (gt *WrappedTimer) C() <-chan time.Time { return gt.t.C } // SetRead set the read flag, you must call it if timer chan is read, otherwise stop/reset will hang. -func (gt *GoodTimer) SetRead() { +func (gt *WrappedTimer) SetRead() { gt.read = true } // Reset changes the timer to expire after duration d. -func (gt *GoodTimer) Reset(d time.Duration) { +func (gt *WrappedTimer) Reset(d time.Duration) { gt.Stop() gt.t.Reset(d) gt.read = false @@ -78,7 +65,7 @@ func (gt *GoodTimer) Reset(d time.Duration) { // Stop prevents the Timer from firing. // It returns true if the call stops the timer, false if the timer has already expired or been stopped. -func (gt *GoodTimer) Stop() bool { +func (gt *WrappedTimer) Stop() bool { stopped := gt.t.Stop() if !stopped && !gt.read { // Drain the gt.t.C if it has not been read from already diff --git a/util/wait_group.go b/util/wait_group.go deleted file mode 100644 index 168d655c45b5a..0000000000000 --- a/util/wait_group.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "sync" - "time" -) - -func WaitTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { - c := make(chan struct{}) - go func() { - defer close(c) - wg.Wait() - }() - select { - case <-c: - return false // completed normally - case <-time.After(timeout): - return true // timed out - } -} diff --git a/util/wait_group_test.go b/util/wait_group_test.go deleted file mode 100644 index 225dd25c2a6a9..0000000000000 --- a/util/wait_group_test.go +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "sync" - "testing" - "time" - - "github.com/stretchr/testify/assert" -) - -func TestWaitGroupTimeout(t *testing.T) { - t.Parallel() - wg := &sync.WaitGroup{} - wg.Add(1) - go func() { - wg.Done() - }() - timeout := time.Millisecond * 10 - rs := WaitTimeout(wg, timeout) - assert.Equal(t, false, rs) - wg.Add(1) - timeout = time.Nanosecond * 10 - rs = WaitTimeout(wg, timeout) - assert.Equal(t, true, rs) - wg.Done() -} From 9dd41481170ff31601dc5d7890ca1054fefe6fad Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Mon, 13 Dec 2021 15:02:06 +0800 Subject: [PATCH 13/38] refactor and simplify --- planner/core/stats_load_test.go | 14 ++++++ statistics/handle/handle_hist.go | 42 +++++++++--------- util/timeutil/timer.go | 75 -------------------------------- 3 files changed, 34 insertions(+), 97 deletions(-) delete mode 100644 util/timeutil/timer.go diff --git a/planner/core/stats_load_test.go b/planner/core/stats_load_test.go index 9a8bc9592b0bc..27659a5c3fffa 100644 --- a/planner/core/stats_load_test.go +++ b/planner/core/stats_load_test.go @@ -1 +1,15 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package core diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 7692800df63e5..adaedf6fc9e7f 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/timeutil" "go.uber.org/zap" ) @@ -61,8 +60,8 @@ func (h *Handle) SyncLoad(sc *stmtctx.StatementContext, neededColumns []model.Ta resultCheckMap[col] = struct{}{} } metrics.SyncLoadCounter.Inc() - timeoutTimer := timeutil.NewWrappedTimer(timeout) - defer timeoutTimer.Stop() + timer := time.NewTimer(timeout) + defer timer.Stop() t := time.Now() for { select { @@ -74,8 +73,7 @@ func (h *Handle) SyncLoad(sc *stmtctx.StatementContext, neededColumns []model.Ta return true } } - case <-timeoutTimer.C(): - timeoutTimer.SetRead() + case <-timer.C: metrics.SyncLoadTimeoutCounter.Inc() return false } @@ -242,38 +240,38 @@ func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, // drainColTask will hang until a column task can return. func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { - timeout := time.Millisecond - to := timeutil.NewWrappedTimer(timeout) + // select NeededColumnsCh firstly, if no task, then select TimeoutColumnsCh for { - to.Reset(timeout) - // select NeededColumnsCh firstly, if no task, then select TimeoutColumnsCh select { + case <-exit: + return nil, errExit case task, ok := <-h.StatsLoad.NeededColumnsCh: if !ok { - return nil, errors.New("drainColTask: cannot read from a closed NeededColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") } if time.Now().After(task.ToTimeout) { h.StatsLoad.TimeoutColumnsCh <- task continue } return task, nil - case <-to.C(): - to.SetRead() - to.Reset(timeout) + case task, ok := <-h.StatsLoad.TimeoutColumnsCh: select { - case task, ok := <-h.StatsLoad.TimeoutColumnsCh: + case <-exit: + return nil, errExit + case task0, ok0 := <-h.StatsLoad.NeededColumnsCh: + if !ok0 { + return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") + } + // send task back to TimeoutColumnsCh and return the task drained from NeededColumnsCh + h.StatsLoad.TimeoutColumnsCh <- task + return task0, nil + default: if !ok { - return nil, errors.New("drainColTask: cannot read from a closed TimeoutColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from TimeoutColumnsCh, maybe the chan is closed.") } + // NeededColumnsCh is empty now, handle task from TimeoutColumnsCh return task, nil - case <-to.C(): - to.SetRead() - continue - case <-exit: - return nil, errExit } - case <-exit: - return nil, errExit } } } diff --git a/util/timeutil/timer.go b/util/timeutil/timer.go deleted file mode 100644 index 891b62f0a97c3..0000000000000 --- a/util/timeutil/timer.go +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package timeutil - -import ( - "time" -) - -// WrappedTimer wraps the standard time.Timer to handle clean stop and reset. -// As Russ Cox suggested, the correct way to use time.Timer is: -// 1. All the Timer operations (Timer.Stop, Timer.Reset and receiving from or draining the channel) should be done in the same goroutine. -// 2. The program should manage an extra status showing whether it has received from the Timer's channel or not. -// https://github.com/golang/go/issues/11513#issuecomment-157062583 -// https://groups.google.com/g/golang-dev/c/c9UUfASVPoU/m/tlbK2BpFEwAJ -// **NOTE**: All the functions of WrappedTimer *should* be used in the same goroutine. -type WrappedTimer struct { - t *time.Timer // The actual timer - read bool // Whether t.C has already been read from -} - -// NewWrappedTimer creates an instance of WrappedTimer. -func NewWrappedTimer(d time.Duration) *WrappedTimer { - return &WrappedTimer{t: time.NewTimer(d)} -} - -// ReadC waits until it can read from the wrapped timer's channel C. -// It returns the time value received from the channel C, a zero time value if the channel C has already been read from. -func (gt *WrappedTimer) ReadC() time.Time { - if gt.read { - return time.Time{} - } - tv := <-gt.t.C - gt.read = true - return tv -} - -// C returns the chan of wrapped timer for select. -func (gt *WrappedTimer) C() <-chan time.Time { - return gt.t.C -} - -// SetRead set the read flag, you must call it if timer chan is read, otherwise stop/reset will hang. -func (gt *WrappedTimer) SetRead() { - gt.read = true -} - -// Reset changes the timer to expire after duration d. -func (gt *WrappedTimer) Reset(d time.Duration) { - gt.Stop() - gt.t.Reset(d) - gt.read = false -} - -// Stop prevents the Timer from firing. -// It returns true if the call stops the timer, false if the timer has already expired or been stopped. -func (gt *WrappedTimer) Stop() bool { - stopped := gt.t.Stop() - if !stopped && !gt.read { - // Drain the gt.t.C if it has not been read from already - <-gt.t.C - } - return stopped -} From b0b7ca33ce539bdf8e0b895321da58f0a2505d03 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 14 Dec 2021 01:13:24 +0800 Subject: [PATCH 14/38] refactor --- metrics/metrics.go | 17 +++-- planner/core/optimizer.go | 13 +++- sessionctx/stmtctx/stmtctx.go | 2 + statistics/handle/handle_hist.go | 78 ++++++++++++---------- statistics/handle/handle_hist_test.go | 94 +++++++++++++++++++++++++++ statistics/handle/handle_test.go | 2 + statistics/histogram.go | 2 +- 7 files changed, 162 insertions(+), 46 deletions(-) diff --git a/metrics/metrics.go b/metrics/metrics.go index f64bf47a0e7f1..bde3ad8c117b8 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -32,15 +32,14 @@ var ( // metrics labels. const ( - LabelSession = "session" - LabelDomain = "domain" - LabelDDLOwner = "ddl-owner" - LabelDDL = "ddl" - LabelDDLWorker = "ddl-worker" - LabelDDLSyncer = "ddl-syncer" - LabelGCWorker = "gcworker" - LabelAnalyze = "analyze" - LabelStatsLoadWorker = "stats_load_worker" + LabelSession = "session" + LabelDomain = "domain" + LabelDDLOwner = "ddl-owner" + LabelDDL = "ddl" + LabelDDLWorker = "ddl-worker" + LabelDDLSyncer = "ddl-syncer" + LabelGCWorker = "gcworker" + LabelAnalyze = "analyze" LabelBatchRecvLoop = "batch-recv-loop" LabelBatchSendLoop = "batch-send-loop" diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 022085463c0b1..88c87ac1124fa 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -271,6 +271,14 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if err != nil { return nil, 0, err } + // load full stats if not loaded in logicalOptimize() + if !sctx.GetSessionVars().StmtCtx.StatsLoad.FullStatsLoaded { + _, err = SyncLoadColumnFullStats(logic) + if err != nil { + return nil, 0, err + } + sctx.GetSessionVars().StmtCtx.StatsLoad.FullStatsLoaded = true + } if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } @@ -426,7 +434,6 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic }() } var err error - fullStatsLoaded := false for i, rule := range optRuleList { // The order of flags is same as the order of optRule in the list. // We use a bitmask to record which opt rules should be used. If the i-th bit is 1, it means we should @@ -436,12 +443,12 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic } opt.appendBeforeRuleOptimize(i, rule.name(), logic) // sync-load full stats before the first stats-needed rule applied - if !fullStatsLoaded && rule.needStats() { + if !vars.StmtCtx.StatsLoad.FullStatsLoaded && rule.needStats() { _, err = SyncLoadColumnFullStats(logic) if err != nil { return nil, err } - fullStatsLoaded = true + vars.StmtCtx.StatsLoad.FullStatsLoaded = true } logic, err = rule.optimize(ctx, logic, opt) if err != nil { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index e50a97af23abc..211a7ea2011c9 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -213,6 +213,8 @@ type StatementContext struct { ResultCh chan model.TableColumnID // Fallback indicates if the planner uses full-loaded stats or fallback all to pseudo/simple. Fallback bool + // FullStatsLoaded indicates if full stats of all needed columns are loaded + FullStatsLoaded bool } } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index adaedf6fc9e7f..c73214ec40278 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -109,26 +109,26 @@ func (h *Handle) appendNeededColumn(c model.TableColumnID, resultCh chan model.T var errExit = errors.New("Stop loading since domain is closed.") +type statsReaderContext struct { + reader *statsReader + createdTime time.Time +} + // SubLoadWorker loads hist data for each column func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg *sync.WaitGroup) error { + readerCtx := &statsReaderContext{} defer func() { exitWg.Done() logutil.BgLogger().Info("SubLoadWorker exited.") - }() - reader, err0 := h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) - if err0 != nil { - return err0 - } - defer func() { - err1 := h.releaseStatsReader(reader, ctx.(sqlexec.RestrictedSQLExecutor)) - if err1 != nil && err0 == nil { - logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err1)) + if readerCtx.reader != nil { + err := h.releaseStatsReader(readerCtx.reader, ctx.(sqlexec.RestrictedSQLExecutor)) + if err != nil { + logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err)) + } } }() - batched := 0 for { - batched += 1 - err := h.handleOneTask(reader, exit) + err := h.handleOneTask(readerCtx, ctx.(sqlexec.RestrictedSQLExecutor), exit) if err != nil { switch err { case errExit: @@ -138,39 +138,27 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitW continue } } - if batched >= 100 { - // refresh statsReader after a while for latest stats - err = h.releaseStatsReader(reader, ctx.(sqlexec.RestrictedSQLExecutor)) - if err != nil { - logutil.BgLogger().Error("Fail to release stats loader: ", zap.Error(err)) - } - reader, err = h.getStatsReader(0, ctx.(sqlexec.RestrictedSQLExecutor)) - if err != nil { - // TODO will begin/commit fail? - logutil.BgLogger().Error("Fail to new stats loader: ", zap.Error(err)) - } - batched = 0 - } } } // handleOneTask handles one column task. -func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { +func (h *Handle) handleOneTask(readerCtx *statsReaderContext, ctx sqlexec.RestrictedSQLExecutor, exit chan struct{}) (err error) { defer func() { + // recover for each task, worker keeps working if r := recover(); r != nil { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] logutil.BgLogger().Error("stats loading panicked", zap.String("stack", string(buf))) - metrics.PanicCounter.WithLabelValues(metrics.LabelStatsLoadWorker).Inc() } }() - task, err0 := h.drainColTask(exit) - if err0 != nil && task == nil { - if err0 != errExit { - logutil.BgLogger().Error("Fail to drain task for stats loading.") + h.getFreshStatsReader(readerCtx, ctx.(sqlexec.RestrictedSQLExecutor)) + task, err := h.drainColTask(exit) + if err != nil { + if err != errExit { + logutil.BgLogger().Error("Fail to drain task for stats loading.", zap.Error(err)) } - return err0 + return err } col := task.TableColumnID // to avoid duplicated handling in concurrent scenario @@ -189,7 +177,7 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { return nil } t := time.Now() - hist, err := h.readStatsForOne(col, c, reader) + hist, err := h.readStatsForOne(col, c, readerCtx.reader) if err != nil { h.StatsLoad.NeededColumnsCh <- task return err @@ -202,6 +190,30 @@ func (h *Handle) handleOneTask(reader *statsReader, exit chan struct{}) error { return nil } +func (h *Handle) getFreshStatsReader(readerCtx *statsReaderContext, ctx sqlexec.RestrictedSQLExecutor) { + if readerCtx.reader == nil || readerCtx.createdTime.Add(time.Second*3).Before(time.Now()) { + if readerCtx.reader != nil { + err := h.releaseStatsReader(readerCtx.reader, ctx) + if err != nil { + logutil.BgLogger().Warn("Fail to release stats loader: ", zap.Error(err)) + } + } + for { + newReader, err := h.getStatsReader(0, ctx) + if err != nil { + logutil.BgLogger().Error("Fail to new stats loader, retry after a while.", zap.Error(err)) + time.Sleep(time.Millisecond * 10) + } else { + readerCtx.reader = newReader + readerCtx.createdTime = time.Now() + return + } + } + } else { + return + } +} + // readStatsForOne reads hist for one column, TODO load data via kv-get asynchronously func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column, reader *statsReader) (*statistics.Column, error) { hg, err := h.histogramFromStorage(reader, col.TableID, c.ID, &c.Info.FieldType, c.Histogram.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 5f111996c2d4a..1600bb7f34ac8 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -64,3 +64,97 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { topn = stat.Columns[tableInfo.Columns[2].ID].TopN c.Assert(hg.Len()+topn.Num(), Greater, 0) } + +func (s *testStatsSuite) TestConcurrentLoadHistTimeout(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") + testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") + + oriLease := s.do.StatsHandle().Lease() + s.do.StatsHandle().SetLease(1) + defer func() { + s.do.StatsHandle().SetLease(oriLease) + }() + testKit.MustExec("analyze table t") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + stat := h.GetTableStats(tableInfo) + hg := stat.Columns[tableInfo.Columns[0].ID].Histogram + topn := stat.Columns[tableInfo.Columns[0].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Equals, 0) + stmtCtx := &stmtctx.StatementContext{} + neededColumns := make([]model.TableColumnID, 0, len(tableInfo.Columns)) + for _, col := range tableInfo.Columns { + neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) + } + rs := h.SyncLoad(stmtCtx, neededColumns, 0) + c.Assert(rs, Equals, false) + stat = h.GetTableStats(tableInfo) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Equals, 0) + for { + time.Sleep(time.Millisecond * 100) + if len(h.StatsLoad.TimeoutColumnsCh)+len(h.StatsLoad.NeededColumnsCh) == 0 { + break + } + } + stat = h.GetTableStats(tableInfo) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) +} + +func (s *testStatsSuite) TestConcurrentLoadHistFail(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") + testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") + + oriLease := s.do.StatsHandle().Lease() + s.do.StatsHandle().SetLease(1) + defer func() { + s.do.StatsHandle().SetLease(oriLease) + }() + testKit.MustExec("analyze table t") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + stat := h.GetTableStats(tableInfo) + hg := stat.Columns[tableInfo.Columns[0].ID].Histogram + topn := stat.Columns[tableInfo.Columns[0].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Equals, 0) + stmtCtx := &stmtctx.StatementContext{} + neededColumns := make([]model.TableColumnID, 0, len(tableInfo.Columns)) + for _, col := range tableInfo.Columns { + neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) + } + timeout := time.Nanosecond * math.MaxInt + // TODO failpoint, and works again after failpoint + rs := h.SyncLoad(stmtCtx, neededColumns, timeout) + c.Assert(rs, Equals, true) + stat = h.GetTableStats(tableInfo) + hg = stat.Columns[tableInfo.Columns[2].ID].Histogram + topn = stat.Columns[tableInfo.Columns[2].ID].TopN + c.Assert(hg.Len()+topn.Num(), Greater, 0) +} diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 8c5f1d08a4013..a8392555c3112 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -585,6 +585,8 @@ func (s *testStatsSuite) TestLoadStats(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("set @@tidb_analyze_version=1") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") diff --git a/statistics/histogram.go b/statistics/histogram.go index 38db742297c34..9365ac394506b 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1073,7 +1073,7 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { return true } if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { - if sc.StatsLoad.Timeout > 0 { + if sc.StatsLoad.FullStatsLoaded { logutil.BgLogger().Warn("Hist for column %v should already be loaded as sync but not found.", zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) } From adb224ff0ce08a11c4a4563d71618bc0bf05b56f Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 14 Dec 2021 21:55:35 +0800 Subject: [PATCH 15/38] add ut for plan stats --- planner/core/logical_plan_builder.go | 6 - planner/core/optimizer.go | 2 +- planner/core/plan_stats_test.go | 167 ++++++++++++++++++++++++++ planner/core/stats_load_test.go | 15 --- statistics/handle/handle_hist_test.go | 2 + statistics/histogram.go | 2 +- 6 files changed, 171 insertions(+), 23 deletions(-) create mode 100644 planner/core/plan_stats_test.go delete mode 100644 planner/core/stats_load_test.go diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index c3b1239d4ffac..028bfa2c66d87 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4052,11 +4052,6 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as } else { columns = tbl.Cols() } - var statisticTable *statistics.Table - if _, ok := tbl.(table.PartitionedTable); !ok || b.ctx.GetSessionVars().UseDynamicPartitionPrune() { - statisticTable = getStatsTable(b.ctx, tbl.Meta(), tbl.Meta().ID) - } - // extract the IndexMergeHint var indexMergeHints []indexHintInfo if hints := b.TableHints(); hints != nil { @@ -4101,7 +4096,6 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as TableAsName: asName, table: tbl, tableInfo: tableInfo, - statisticTable: statisticTable, astIndexHints: tn.IndexHints, IndexHints: b.TableHints().indexHintList, indexMergeHints: indexMergeHints, diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 3aa02ec615d43..eadcb371461b1 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -339,8 +339,8 @@ func SyncLoadColumnFullStats(plan LogicalPlan) (bool, error) { var timeout = time.Duration(waitTime) * time.Millisecond stmtCtx.StatsLoad.Timeout = timeout success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncLoad(stmtCtx, neededColumns, timeout) - err := errors.New("Timeout when sync-load full stats for needed columns.") if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { + err := errors.New("Timeout when sync-load full stats for needed columns.") stmtCtx.AppendWarning(err) stmtCtx.StatsLoad.Fallback = true return false, err diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go new file mode 100644 index 0000000000000..4bfc36a97b5b1 --- /dev/null +++ b/planner/core/plan_stats_test.go @@ -0,0 +1,167 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "context" + . "github.com/pingcap/check" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" +) + +var _ = Suite(&testPlanStatsSuite{}) + +type testPlanStatsSuite struct { + *parser.Parser +} + +func (s *testPlanStatsSuite) SetUpSuite(c *C) { + s.Parser = parser.New() +} + +func (s *testPlanStatsSuite) TearDownSuite(c *C) { +} + +func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Check(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + ctx := tk.Se.(sessionctx.Context) + 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(a), key idx(b))") + tk.MustExec("insert into t values (1,1,1,1),(2,2,2,2),(3,3,3,3)") + tk.MustExec("create table pt(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)") + + oriLease := dom.StatsHandle().Lease() + dom.StatsHandle().SetLease(1) + defer func() { + dom.StatsHandle().SetLease(oriLease) + }() + tk.MustExec("analyze table t") + tk.MustExec("analyze table pt") + + testCases := []struct { + sql string + skip bool + check func(p plannercore.Plan, tableInfo *model.TableInfo) + }{ + { // DataSource + sql: "select * from t where c>1", + skip: false, + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + switch pp := p.(type) { + case *plannercore.PhysicalTableReader: + stats := pp.Stats().HistColl + c.Assert(countFullStats(stats, tableInfo.Columns[1].ID), Equals, 0) + c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Greater, 0) + default: + c.Error("unexpected plan:", pp) + } + }, + }, + { // PartitionTable TODO + sql: "select * from pt where a < 15 and c > 1", + skip: true, + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + switch pp := p.(type) { + case *plannercore.PhysicalTableReader: + stats := pp.Stats().HistColl + c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Greater, 0) + default: + c.Error("unexpected plan:", pp) + } + }, + }, + { // Join + sql: "select * from t t1 inner join t t2 on t1.b=t2.b where t1.d=3", + skip: false, + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + pp, ok := p.(plannercore.PhysicalPlan) + c.Check(ok, IsTrue) + c.Assert(countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[3].ID), Greater, 0) + c.Assert(countFullStats(pp.Children()[1].Stats().HistColl, tableInfo.Columns[3].ID), Greater, 0) + }, + }, + { // Apply + sql: "select * from t t1 where t1.b > (select count(*) from t t2 where t2.c > t1.a and t2.d>1) and t1.c>2", + skip: false, + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + pp, ok := p.(*plannercore.PhysicalProjection) + c.Check(ok, IsTrue) + pa, ok := pp.Children()[0].(*plannercore.PhysicalApply) + c.Check(ok, IsTrue) + left := pa.PhysicalHashJoin.Children()[0] + right := pa.PhysicalHashJoin.Children()[0] + c.Assert(countFullStats(left.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + c.Assert(countFullStats(right.Stats().HistColl, tableInfo.Columns[3].ID), Greater, 0) + }, + }, + { // CTE TODO + sql: "with cte(x, y) as (select d + 1, b from t where c > 1) select * from cte where x > 3", + skip: true, + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ps, ok := p.(*plannercore.PhysicalSelection) + c.Check(ok, IsTrue) + pc, ok := ps.Children()[0].(*plannercore.PhysicalCTE) + c.Check(ok, IsTrue) + pp, ok := pc.SeedPlan.(*plannercore.PhysicalProjection) + c.Check(ok, IsTrue) + c.Assert(countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + c.Assert(countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[3].ID), Equals, 0) + }, + }, + } + for _, testCase := range testCases { + if testCase.skip { + continue + } + is := dom.InfoSchema() + dom.StatsHandle().Clear() // clear statsCache + c.Assert(dom.StatsHandle().Update(is), IsNil) + stmt, err := s.ParseOneStmt(testCase.sql, "", "") + c.Check(err, IsNil) + err = executor.ResetContextOfStmt(ctx, stmt) + c.Assert(err, IsNil) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + c.Check(err, IsNil) + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + testCase.check(p, tableInfo) + } +} + +func countFullStats(stats *statistics.HistColl, colID int64) int { + for _, col := range stats.Columns { + if col.Info.ID == colID { + return col.Histogram.Len() + col.TopN.Num() + } + } + return -1 +} diff --git a/planner/core/stats_load_test.go b/planner/core/stats_load_test.go deleted file mode 100644 index 27659a5c3fffa..0000000000000 --- a/planner/core/stats_load_test.go +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package core diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 1600bb7f34ac8..9b03655017080 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -63,6 +63,8 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { hg = stat.Columns[tableInfo.Columns[2].ID].Histogram topn = stat.Columns[tableInfo.Columns[2].ID].TopN c.Assert(hg.Len()+topn.Num(), Greater, 0) + + // TODO partition table } func (s *testStatsSuite) TestConcurrentLoadHistTimeout(c *C) { diff --git a/statistics/histogram.go b/statistics/histogram.go index 9365ac394506b..2ed82f4f7423d 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1074,7 +1074,7 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { } if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { if sc.StatsLoad.FullStatsLoaded { - logutil.BgLogger().Warn("Hist for column %v should already be loaded as sync but not found.", + logutil.BgLogger().Warn("Hist for column should already be loaded as sync but not found.", zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) } HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) From 6273b9234a321e8cac35382e95d07194d654bf2f Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 11:29:34 +0800 Subject: [PATCH 16/38] fix deadlock --- statistics/handle/handle_hist.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index c73214ec40278..7802eb7c54354 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -191,7 +191,7 @@ func (h *Handle) handleOneTask(readerCtx *statsReaderContext, ctx sqlexec.Restri } func (h *Handle) getFreshStatsReader(readerCtx *statsReaderContext, ctx sqlexec.RestrictedSQLExecutor) { - if readerCtx.reader == nil || readerCtx.createdTime.Add(time.Second*3).Before(time.Now()) { + if readerCtx.reader == nil || readerCtx.createdTime.Add(h.Lease()).Before(time.Now()) { if readerCtx.reader != nil { err := h.releaseStatsReader(readerCtx.reader, ctx) if err != nil { @@ -262,7 +262,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") } if time.Now().After(task.ToTimeout) { - h.StatsLoad.TimeoutColumnsCh <- task + h.writeToTimeoutCh(task) continue } return task, nil @@ -275,7 +275,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") } // send task back to TimeoutColumnsCh and return the task drained from NeededColumnsCh - h.StatsLoad.TimeoutColumnsCh <- task + h.writeToTimeoutCh(task) return task0, nil default: if !ok { @@ -288,6 +288,16 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { } } +func (h *Handle) writeToTimeoutCh(task *NeededColumnTask) { + select { + case h.StatsLoad.TimeoutColumnsCh <- task: + default: + logutil.BgLogger().Debug("TimeoutCh is full, drop task:", zap.Int64("table", task.TableColumnID.TableID), + zap.Int64("column", task.TableColumnID.ColumnID)) + } + +} + // updateCachedColumn updates the column hist to global statsCache. func (h *Handle) updateCachedColumn(col model.TableColumnID, colHist *statistics.Column) (updated bool) { h.StatsLoad.Lock() From abc2efd946eec5056d429ce266d3fb5ce4a2ab16 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 15:35:14 +0800 Subject: [PATCH 17/38] refactor --- planner/core/hist_column_collector.go | 5 -- planner/core/optimizer.go | 59 ++----------- planner/core/plan_stats.go | 86 +++++++++++++++++++ planner/core/plan_stats_test.go | 19 ++-- planner/core/rule_aggregation_elimination.go | 4 - planner/core/rule_aggregation_push_down.go | 4 - planner/core/rule_build_key_info.go | 4 - planner/core/rule_column_pruning.go | 4 - planner/core/rule_decorrelate.go | 4 - planner/core/rule_eliminate_projection.go | 4 - .../core/rule_generate_column_substitute.go | 4 - planner/core/rule_join_elimination.go | 4 - planner/core/rule_join_reorder.go | 4 - planner/core/rule_max_min_eliminate.go | 4 - planner/core/rule_partition_processor.go | 4 - planner/core/rule_predicate_push_down.go | 4 - planner/core/rule_result_reorder.go | 4 - planner/core/rule_topn_push_down.go | 4 - sessionctx/stmtctx/stmtctx.go | 4 +- statistics/handle/handle_hist.go | 30 +++++-- statistics/handle/handle_hist_test.go | 9 +- statistics/histogram.go | 2 +- 22 files changed, 131 insertions(+), 139 deletions(-) create mode 100644 planner/core/plan_stats.go diff --git a/planner/core/hist_column_collector.go b/planner/core/hist_column_collector.go index 7bda17aa64bde..57ee22959f381 100644 --- a/planner/core/hist_column_collector.go +++ b/planner/core/hist_column_collector.go @@ -43,10 +43,5 @@ func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColum neededColumns[tblColID] = struct{}{} } // TODO collect idx columns? - case *LogicalCTE: - collectColumnsFromPlan(x.cte.seedPartLogicalPlan, neededColumns) - if x.cte.recursivePartLogicalPlan != nil { - collectColumnsFromPlan(x.cte.recursivePartLogicalPlan, neededColumns) - } } } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index eadcb371461b1..4675859715026 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -16,10 +16,8 @@ package core import ( "context" - "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -40,7 +38,6 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" "math" - "time" ) // OptimizeAstNode optimizes the query to a physical plan directly. @@ -62,10 +59,12 @@ const ( flagEliminateProjection flagMaxMinEliminate flagPredicatePushDown + flagCollectPredicateColumnsPoint flagEliminateOuterJoin flagPartitionProcessor flagPushDownAgg flagPushDownTopN + flagSyncWaitStatsLoadPoint flagJoinReOrder flagPrunColumnsAgain ) @@ -80,10 +79,12 @@ var optRuleList = []logicalOptRule{ &projectionEliminator{}, &maxMinEliminator{}, &ppdSolver{}, + &collectPredicateColumnsPoint{}, &outerJoinEliminator{}, &partitionProcessor{}, &aggregationPushDownSolver{}, &pushDownTopNOptimizer{}, + &syncWaitStatsLoadPoint{}, &joinReOrderSolver{}, &columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver } @@ -127,7 +128,6 @@ func (op *logicalOptimizeOp) recordFinalLogicalPlan(final LogicalPlan) { type logicalOptRule interface { optimize(context.Context, LogicalPlan, *logicalOptimizeOp) (LogicalPlan, error) name() string - needStats() bool } // BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node. @@ -267,18 +267,12 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if checkStableResultMode(sctx) { flag |= flagStabilizeResults } + flag |= flagCollectPredicateColumnsPoint + flag |= flagSyncWaitStatsLoadPoint logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, 0, err } - // load full stats if not loaded in logicalOptimize() - if !sctx.GetSessionVars().StmtCtx.StatsLoad.FullStatsLoaded { - _, err = SyncLoadColumnFullStats(logic) - if err != nil { - return nil, 0, err - } - sctx.GetSessionVars().StmtCtx.StatsLoad.FullStatsLoaded = true - } if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } @@ -316,39 +310,6 @@ func refineCETrace(sctx sessionctx.Context) { } } -// SyncLoadColumnFullStats sends columns' full-stats request and sync-wait until timeout -func SyncLoadColumnFullStats(plan LogicalPlan) (bool, error) { - if plan.SCtx().GetSessionVars().InRestrictedSQL { - return true, nil - } - syncWait := config.GetGlobalConfig().Stats.SyncLoadWait - if syncWait <= 0 { - return true, nil - } - neededColumns := CollectHistColumns(plan) - stmtCtx := plan.SCtx().GetSessionVars().StmtCtx - hintMaxExecutionTime := stmtCtx.MaxExecutionTime - if hintMaxExecutionTime == 0 { - hintMaxExecutionTime = math.MaxInt - } - sessMaxExecutionTime := plan.SCtx().GetSessionVars().MaxExecutionTime - if sessMaxExecutionTime == 0 { - sessMaxExecutionTime = math.MaxInt - } - waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) - var timeout = time.Duration(waitTime) * time.Millisecond - stmtCtx.StatsLoad.Timeout = timeout - success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncLoad(stmtCtx, neededColumns, timeout) - if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { - err := errors.New("Timeout when sync-load full stats for needed columns.") - stmtCtx.AppendWarning(err) - stmtCtx.StatsLoad.Fallback = true - return false, err - } else { - return true, nil - } -} - // mergeContinuousSelections merge continuous selections which may occur after changing plans. func mergeContinuousSelections(p PhysicalPlan) { if sel, ok := p.(*PhysicalSelection); ok { @@ -464,14 +425,6 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic continue } opt.appendBeforeRuleOptimize(i, rule.name(), logic) - // sync-load full stats before the first stats-needed rule applied - if !vars.StmtCtx.StatsLoad.FullStatsLoaded && rule.needStats() { - _, err = SyncLoadColumnFullStats(logic) - if err != nil { - return nil, err - } - vars.StmtCtx.StatsLoad.FullStatsLoaded = true - } logic, err = rule.optimize(ctx, logic, opt) if err != nil { return nil, err diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go new file mode 100644 index 0000000000000..aae6d7f294c65 --- /dev/null +++ b/planner/core/plan_stats.go @@ -0,0 +1,86 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "context" + "math" + "time" + + "github.com/cznic/mathutil" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" +) + +type collectPredicateColumnsPoint struct{} + +func (c collectPredicateColumnsPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { + RequestLoadColumnStats(plan) + return plan, nil +} + +func (c collectPredicateColumnsPoint) name() string { + return "collect_predicate_columns_point" +} + +type syncWaitStatsLoadPoint struct{} + +func (s syncWaitStatsLoadPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { + SyncWaitStatsLoad(plan) + return plan, nil +} + +func (s syncWaitStatsLoadPoint) name() string { + return "sync_wait_stats_load_point" +} + +// RequestLoadColumnStats send requests to stats handle +func RequestLoadColumnStats(plan LogicalPlan) { + if plan.SCtx().GetSessionVars().InRestrictedSQL { + return + } + syncWait := config.GetGlobalConfig().Stats.SyncLoadWait + if syncWait <= 0 { + return + } + stmtCtx := plan.SCtx().GetSessionVars().StmtCtx + hintMaxExecutionTime := stmtCtx.MaxExecutionTime + if hintMaxExecutionTime == 0 { + hintMaxExecutionTime = math.MaxInt + } + sessMaxExecutionTime := plan.SCtx().GetSessionVars().MaxExecutionTime + if sessMaxExecutionTime == 0 { + sessMaxExecutionTime = math.MaxInt + } + waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) + var timeout = time.Duration(waitTime) * time.Millisecond + neededColumns := CollectHistColumns(plan) + domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) +} + +// SyncWaitStatsLoad sync-wait for stats load until timeout +func SyncWaitStatsLoad(plan LogicalPlan) (bool, error) { + stmtCtx := plan.SCtx().GetSessionVars().StmtCtx + success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) + if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { + err := errors.New("Timeout when sync-load full stats for needed columns.") + stmtCtx.AppendWarning(err) + stmtCtx.StatsLoad.Fallback = true + return false, err + } else { + return true, nil + } +} diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index 4bfc36a97b5b1..8ce7c8adac9a5 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -72,8 +72,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { check func(p plannercore.Plan, tableInfo *model.TableInfo) }{ { // DataSource - sql: "select * from t where c>1", - skip: false, + sql: "select * from t where c>1", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { switch pp := p.(type) { case *plannercore.PhysicalTableReader: @@ -99,8 +98,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { }, }, { // Join - sql: "select * from t t1 inner join t t2 on t1.b=t2.b where t1.d=3", - skip: false, + sql: "select * from t t1 inner join t t2 on t1.b=t2.b where t1.d=3", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { pp, ok := p.(plannercore.PhysicalPlan) c.Check(ok, IsTrue) @@ -109,8 +107,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { }, }, { // Apply - sql: "select * from t t1 where t1.b > (select count(*) from t t2 where t2.c > t1.a and t2.d>1) and t1.c>2", - skip: false, + sql: "select * from t t1 where t1.b > (select count(*) from t t2 where t2.c > t1.a and t2.d>1) and t1.c>2", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { pp, ok := p.(*plannercore.PhysicalProjection) c.Check(ok, IsTrue) @@ -122,9 +119,8 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { c.Assert(countFullStats(right.Stats().HistColl, tableInfo.Columns[3].ID), Greater, 0) }, }, - { // CTE TODO - sql: "with cte(x, y) as (select d + 1, b from t where c > 1) select * from cte where x > 3", - skip: true, + { // CTE + sql: "with cte(x, y) as (select d + 1, b from t where c > 1) select * from cte where x < 3", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { ps, ok := p.(*plannercore.PhysicalSelection) c.Check(ok, IsTrue) @@ -132,8 +128,9 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { c.Check(ok, IsTrue) pp, ok := pc.SeedPlan.(*plannercore.PhysicalProjection) c.Check(ok, IsTrue) - c.Assert(countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) - c.Assert(countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[3].ID), Equals, 0) + reader, ok := pp.Children()[0].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(reader.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) }, }, } diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 24da4ba93e0cd..61d9e0f117e0d 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -31,10 +31,6 @@ type aggregationEliminator struct { aggregationEliminateChecker } -func (a *aggregationEliminator) needStats() bool { - return false -} - type aggregationEliminateChecker struct { } diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 682a6f1274433..7cc0548a7f57d 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -31,10 +31,6 @@ type aggregationPushDownSolver struct { aggregationEliminateChecker } -func (a *aggregationPushDownSolver) needStats() bool { - return false -} - // isDecomposable checks if an aggregate function is decomposable. An aggregation function $F$ is decomposable // if there exist aggregation functions F_1 and F_2 such that F(S_1 union all S_2) = F_2(F_1(S_1),F_1(S_2)), // where S_1 and S_2 are two sets of values. We call S_1 and S_2 partial groups. diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 96170274a07c2..ae8a0a1d3a566 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -25,10 +25,6 @@ import ( type buildKeySolver struct{} -func (s *buildKeySolver) needStats() bool { - return false -} - func (s *buildKeySolver) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { buildKeyInfo(p) return p, nil diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index ca97fd1f02be6..1d144c72807f9 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -29,10 +29,6 @@ import ( type columnPruner struct { } -func (s *columnPruner) needStats() bool { - return false -} - func (s *columnPruner) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { err := lp.PruneColumns(lp.Schema().Columns) return lp, err diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 6433e2d353493..a8835d57448b9 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -105,10 +105,6 @@ func ExtractCorrelatedCols4PhysicalPlan(p PhysicalPlan) []*expression.Correlated // decorrelateSolver tries to convert apply plan to join plan. type decorrelateSolver struct{} -func (s *decorrelateSolver) needStats() bool { - return false -} - func (s *decorrelateSolver) aggDefaultValueMap(agg *LogicalAggregation) map[int]*expression.Constant { defaultValueMap := make(map[int]*expression.Constant, len(agg.AggFuncs)) for i, f := range agg.AggFuncs { diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 90c16d9efe3af..0c377bd974703 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -146,10 +146,6 @@ func eliminatePhysicalProjection(p PhysicalPlan) PhysicalPlan { type projectionEliminator struct { } -func (pe *projectionEliminator) needStats() bool { - return false -} - // optimize implements the logicalOptRule interface. func (pe *projectionEliminator) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { root := pe.eliminate(lp, make(map[string]*expression.Column), false, opt) diff --git a/planner/core/rule_generate_column_substitute.go b/planner/core/rule_generate_column_substitute.go index de6aaa0d686c7..d3ec3d5c7960c 100644 --- a/planner/core/rule_generate_column_substitute.go +++ b/planner/core/rule_generate_column_substitute.go @@ -27,10 +27,6 @@ import ( type gcSubstituter struct { } -func (gc *gcSubstituter) needStats() bool { - return false -} - // ExprColumnMap is used to store all expressions of indexed generated columns in a table, // and map them to the generated columns, // thus we can substitute the expression in a query to an indexed generated column. diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index eeacb53f79e47..6d4a750ea4356 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -27,10 +27,6 @@ import ( type outerJoinEliminator struct { } -func (o *outerJoinEliminator) needStats() bool { - return false -} - // tryToEliminateOuterJoin will eliminate outer join plan base on the following rules // 1. outer join elimination: For example left outer join, if the parent only use the // columns from left table and the join key of right table(the inner table) is a unique diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 78a0fea4f8c2c..dd29f7d3f1f30 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -55,10 +55,6 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression type joinReOrderSolver struct { } -func (s *joinReOrderSolver) needStats() bool { - return true -} - type jrNode struct { p LogicalPlan cumCost float64 diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index fe77d1fd85f46..858f9005c2273 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -36,10 +36,6 @@ import ( type maxMinEliminator struct { } -func (a *maxMinEliminator) needStats() bool { - return false -} - func (a *maxMinEliminator) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { return a.eliminateMaxMin(p, opt), nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 0bb779ef83374..7c3bbb565c69d 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -57,10 +57,6 @@ const FullRange = -1 // partitionProcessor is here because it's easier to prune partition after predicate push down. type partitionProcessor struct{} -func (s *partitionProcessor) needStats() bool { - return false -} - func (s *partitionProcessor) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { p, err := s.rewriteDataSource(lp) return p, err diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index ec6d7b0b68707..f59fedc25ba28 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -27,10 +27,6 @@ import ( type ppdSolver struct{} -func (s *ppdSolver) needStats() bool { - return false -} - func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { _, p := lp.PredicatePushDown(nil) return p, nil diff --git a/planner/core/rule_result_reorder.go b/planner/core/rule_result_reorder.go index 827d3fbe675c6..7ea7d73556b4d 100644 --- a/planner/core/rule_result_reorder.go +++ b/planner/core/rule_result_reorder.go @@ -37,10 +37,6 @@ import ( type resultReorder struct { } -func (rs *resultReorder) needStats() bool { - return false -} - func (rs *resultReorder) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { ordered := rs.completeSort(lp) if !ordered { diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index e3b6ea51bb211..e6234bbc3f3dc 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -26,10 +26,6 @@ import ( type pushDownTopNOptimizer struct { } -func (s *pushDownTopNOptimizer) needStats() bool { - return false -} - func (s *pushDownTopNOptimizer) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { return p.pushDownTopN(nil), nil } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 275a15727edde..5aef4bd0916d2 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -214,8 +214,8 @@ type StatementContext struct { ResultCh chan model.TableColumnID // Fallback indicates if the planner uses full-loaded stats or fallback all to pseudo/simple. Fallback bool - // FullStatsLoaded indicates if full stats of all needed columns are loaded - FullStatsLoaded bool + // LoadStartTime is to record the load start time to calculate latency + LoadStartTime time.Time } } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 7802eb7c54354..465d1240b18d0 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -45,31 +45,45 @@ type NeededColumnTask struct { ResultCh chan model.TableColumnID } -// SyncLoad sync waits loading of neededColumns and return false if timeout -func (h *Handle) SyncLoad(sc *stmtctx.StatementContext, neededColumns []model.TableColumnID, timeout time.Duration) bool { +// SendLoadRequests send neededColumns requests +func (h *Handle) SendLoadRequests(sc *stmtctx.StatementContext, neededColumns []model.TableColumnID, timeout time.Duration) { missingColumns := h.genHistMissingColumns(neededColumns) if len(missingColumns) <= 0 { - return true + return } + sc.StatsLoad.Timeout = timeout sc.StatsLoad.NeededColumns = missingColumns sc.StatsLoad.ResultCh = make(chan model.TableColumnID, len(neededColumns)) - defer close(sc.StatsLoad.ResultCh) - resultCheckMap := map[model.TableColumnID]struct{}{} for _, col := range missingColumns { h.appendNeededColumn(col, sc.StatsLoad.ResultCh, timeout) + } + sc.StatsLoad.LoadStartTime = time.Now() +} + +// SyncWaitStatsLoad sync waits loading of neededColumns and return false if timeout +func (h *Handle) SyncWaitStatsLoad(sc *stmtctx.StatementContext) bool { + if len(sc.StatsLoad.NeededColumns) <= 0 { + return true + } + defer func() { + if sc.StatsLoad.ResultCh != nil { + close(sc.StatsLoad.ResultCh) + } + }() + resultCheckMap := map[model.TableColumnID]struct{}{} + for _, col := range sc.StatsLoad.NeededColumns { resultCheckMap[col] = struct{}{} } metrics.SyncLoadCounter.Inc() - timer := time.NewTimer(timeout) + timer := time.NewTimer(sc.StatsLoad.Timeout) defer timer.Stop() - t := time.Now() for { select { case result, ok := <-sc.StatsLoad.ResultCh: if ok { delete(resultCheckMap, result) if len(resultCheckMap) == 0 { - metrics.SyncLoadHistogram.Observe(float64(time.Since(t).Milliseconds())) + metrics.SyncLoadHistogram.Observe(float64(time.Since(sc.StatsLoad.LoadStartTime).Milliseconds())) return true } } diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 9b03655017080..801c99e70fedb 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -57,7 +57,8 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } timeout := time.Nanosecond * math.MaxInt - rs := h.SyncLoad(stmtCtx, neededColumns, timeout) + h.SendLoadRequests(stmtCtx, neededColumns, timeout) + rs := h.SyncWaitStatsLoad(stmtCtx) c.Assert(rs, Equals, true) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram @@ -100,7 +101,8 @@ func (s *testStatsSuite) TestConcurrentLoadHistTimeout(c *C) { for _, col := range tableInfo.Columns { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } - rs := h.SyncLoad(stmtCtx, neededColumns, 0) + h.SendLoadRequests(stmtCtx, neededColumns, 0) + rs := h.SyncWaitStatsLoad(stmtCtx) c.Assert(rs, Equals, false) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram @@ -153,7 +155,8 @@ func (s *testStatsSuite) TestConcurrentLoadHistFail(c *C) { } timeout := time.Nanosecond * math.MaxInt // TODO failpoint, and works again after failpoint - rs := h.SyncLoad(stmtCtx, neededColumns, timeout) + h.SendLoadRequests(stmtCtx, neededColumns, timeout) + rs := h.SyncWaitStatsLoad(stmtCtx) c.Assert(rs, Equals, true) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram diff --git a/statistics/histogram.go b/statistics/histogram.go index 2ed82f4f7423d..453656049c302 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1073,7 +1073,7 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { return true } if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sc != nil { - if sc.StatsLoad.FullStatsLoaded { + if sc.StatsLoad.Timeout > 0 { logutil.BgLogger().Warn("Hist for column should already be loaded as sync but not found.", zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) } From fc369a0986d7ddd223a4ffe6e932ea6c4db24fd7 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 19:20:38 +0800 Subject: [PATCH 18/38] fix --- planner/core/plan_stats.go | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index aae6d7f294c65..d0766561d6276 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -16,7 +16,6 @@ package core import ( "context" - "math" "time" "github.com/cznic/mathutil" @@ -52,20 +51,20 @@ func RequestLoadColumnStats(plan LogicalPlan) { if plan.SCtx().GetSessionVars().InRestrictedSQL { return } - syncWait := config.GetGlobalConfig().Stats.SyncLoadWait + syncWait := int64(config.GetGlobalConfig().Stats.SyncLoadWait) if syncWait <= 0 { return } stmtCtx := plan.SCtx().GetSessionVars().StmtCtx - hintMaxExecutionTime := stmtCtx.MaxExecutionTime + hintMaxExecutionTime := int64(stmtCtx.MaxExecutionTime) if hintMaxExecutionTime == 0 { - hintMaxExecutionTime = math.MaxInt + hintMaxExecutionTime = mathutil.MaxInt } - sessMaxExecutionTime := plan.SCtx().GetSessionVars().MaxExecutionTime + sessMaxExecutionTime := int64(plan.SCtx().GetSessionVars().MaxExecutionTime) if sessMaxExecutionTime == 0 { - sessMaxExecutionTime = math.MaxInt + sessMaxExecutionTime = mathutil.MaxInt } - waitTime := mathutil.Min(int(syncWait), mathutil.Min(int(hintMaxExecutionTime), int(sessMaxExecutionTime))) + waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) * time.Millisecond neededColumns := CollectHistColumns(plan) domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) From 69c6ce0e831d1a7ca1e533a683cfc997936ffc43 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 20:20:09 +0800 Subject: [PATCH 19/38] fix format --- config/config.go | 15 ++++++++------- planner/core/optimizer.go | 3 ++- planner/core/plan_stats.go | 9 ++++----- planner/core/plan_stats_test.go | 1 + statistics/handle/handle_hist.go | 26 +++++++++++++------------- statistics/handle/handle_hist_test.go | 5 +++-- statistics/histogram.go | 1 + 7 files changed, 32 insertions(+), 28 deletions(-) diff --git a/config/config.go b/config/config.go index b94ad8d7c7d94..0c4e99f362a16 100644 --- a/config/config.go +++ b/config/config.go @@ -472,13 +472,13 @@ type Performance struct { CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"` MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"` // Deprecated - MemProfileInterval string `toml:"-" json:"-"` - IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"` - PlanReplayerGCLease string `toml:"plan-replayer-gc-lease" json:"plan-replayer-gc-lease"` - GOGC int `toml:"gogc" json:"gogc"` - EnforceMPP bool `toml:"enforce-mpp" json:"enforce-mpp"` - StatsLoadConcurrency uint `toml:"stats-load-concurrency" json:"stats-load-concurrency"` - StatsLoadQueueSize uint `toml:"stats-load-queue-size" json:"stats-load-queue-size"` + MemProfileInterval string `toml:"-" json:"-"` + IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"` + PlanReplayerGCLease string `toml:"plan-replayer-gc-lease" json:"plan-replayer-gc-lease"` + GOGC int `toml:"gogc" json:"gogc"` + EnforceMPP bool `toml:"enforce-mpp" json:"enforce-mpp"` + StatsLoadConcurrency uint `toml:"stats-load-concurrency" json:"stats-load-concurrency"` + StatsLoadQueueSize uint `toml:"stats-load-queue-size" json:"stats-load-queue-size"` } // PlanCache is the PlanCache section of the config. @@ -609,6 +609,7 @@ type Experimental struct { EnableNewCharset bool `toml:"enable-new-charset" json:"-"` } +// Stats controls the stats loading and usage behavior type Stats struct { SyncLoadWait uint `toml:"sync-load-wait" json:"sync-load-wait"` PseudoForLoadTimeout bool `toml:"pseudo-for-load-timeout" json:"pseudo-for-load-timeout"` diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 4675859715026..434a97c17f4c7 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -16,6 +16,8 @@ package core import ( "context" + "math" + "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -37,7 +39,6 @@ import ( "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" "go.uber.org/zap" - "math" ) // OptimizeAstNode optimizes the query to a physical plan directly. diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index d0766561d6276..9cdd336028c19 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -71,15 +71,14 @@ func RequestLoadColumnStats(plan LogicalPlan) { } // SyncWaitStatsLoad sync-wait for stats load until timeout -func SyncWaitStatsLoad(plan LogicalPlan) (bool, error) { +func SyncWaitStatsLoad(plan LogicalPlan) bool { stmtCtx := plan.SCtx().GetSessionVars().StmtCtx success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { - err := errors.New("Timeout when sync-load full stats for needed columns.") + err := errors.New("Timeout when sync-load full stats for needed columns") stmtCtx.AppendWarning(err) stmtCtx.StatsLoad.Fallback = true - return false, err - } else { - return true, nil + return false } + return true } diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index 8ce7c8adac9a5..ae39201c74ea2 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -16,6 +16,7 @@ package core_test import ( "context" + . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser" diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 465d1240b18d0..8e129d68783e5 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -30,6 +30,7 @@ import ( "go.uber.org/zap" ) +// StatsLoad is used to load stats concurrently type StatsLoad struct { sync.Mutex SubCtxs []sessionctx.Context @@ -121,7 +122,7 @@ func (h *Handle) appendNeededColumn(c model.TableColumnID, resultCh chan model.T h.StatsLoad.NeededColumnsCh <- colTask } -var errExit = errors.New("Stop loading since domain is closed.") +var errExit = errors.New("Stop loading since domain is closed") type statsReaderContext struct { reader *statsReader @@ -129,7 +130,7 @@ type statsReaderContext struct { } // SubLoadWorker loads hist data for each column -func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg *sync.WaitGroup) error { +func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitWg *sync.WaitGroup) { readerCtx := &statsReaderContext{} defer func() { exitWg.Done() @@ -146,7 +147,7 @@ func (h *Handle) SubLoadWorker(ctx sessionctx.Context, exit chan struct{}, exitW if err != nil { switch err { case errExit: - return nil + return default: time.Sleep(10 * time.Millisecond) continue @@ -166,7 +167,7 @@ func (h *Handle) handleOneTask(readerCtx *statsReaderContext, ctx sqlexec.Restri logutil.BgLogger().Error("stats loading panicked", zap.String("stack", string(buf))) } }() - h.getFreshStatsReader(readerCtx, ctx.(sqlexec.RestrictedSQLExecutor)) + h.getFreshStatsReader(readerCtx, ctx) task, err := h.drainColTask(exit) if err != nil { if err != errExit { @@ -273,7 +274,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { return nil, errExit case task, ok := <-h.StatsLoad.NeededColumnsCh: if !ok { - return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed") } if time.Now().After(task.ToTimeout) { h.writeToTimeoutCh(task) @@ -286,14 +287,14 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { return nil, errExit case task0, ok0 := <-h.StatsLoad.NeededColumnsCh: if !ok0 { - return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed") } // send task back to TimeoutColumnsCh and return the task drained from NeededColumnsCh h.writeToTimeoutCh(task) return task0, nil default: if !ok { - return nil, errors.New("drainColTask: cannot read from TimeoutColumnsCh, maybe the chan is closed.") + return nil, errors.New("drainColTask: cannot read from TimeoutColumnsCh, maybe the chan is closed") } // NeededColumnsCh is empty now, handle task from TimeoutColumnsCh return task, nil @@ -337,14 +338,13 @@ func (h *Handle) setWorking(col model.TableColumnID, resultCh chan model.TableCo defer h.StatsLoad.Unlock() chList, ok := h.StatsLoad.workingColMap[col] if ok { - chList = append(chList, resultCh) + h.StatsLoad.workingColMap[col] = append(chList, resultCh) return false - } else { - chList = []chan model.TableColumnID{} - chList = append(chList, resultCh) - h.StatsLoad.workingColMap[col] = chList - return true } + chList = []chan model.TableColumnID{} + chList = append(chList, resultCh) + h.StatsLoad.workingColMap[col] = chList + return true } func (h *Handle) finishWorking(col model.TableColumnID) { diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 801c99e70fedb..818fd2e6cbf65 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -15,12 +15,13 @@ package handle_test import ( + "math" + "time" + . "github.com/pingcap/check" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/testkit" - "math" - "time" ) func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { diff --git a/statistics/histogram.go b/statistics/histogram.go index db24e18a2bc4c..9c21394197281 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1087,6 +1087,7 @@ func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) bool { return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) } +// IsHistNeeded checks if this column needs histogram to be loaded func (c *Column) IsHistNeeded(collPseudo bool) bool { return (!collPseudo || !c.NotAccurate()) && c.Histogram.NDV > 0 && c.notNullCount() == 0 } From 75c43176fbba9b05d539ea40da44902a937e4cb2 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 20:41:47 +0800 Subject: [PATCH 20/38] fix check --- statistics/handle/handle_hist_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 818fd2e6cbf65..bc9f7a2533926 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -15,9 +15,9 @@ package handle_test import ( - "math" "time" + "github.com/cznic/mathutil" . "github.com/pingcap/check" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -57,7 +57,7 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { for _, col := range tableInfo.Columns { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } - timeout := time.Nanosecond * math.MaxInt + timeout := time.Nanosecond * mathutil.MaxInt h.SendLoadRequests(stmtCtx, neededColumns, timeout) rs := h.SyncWaitStatsLoad(stmtCtx) c.Assert(rs, Equals, true) @@ -154,7 +154,7 @@ func (s *testStatsSuite) TestConcurrentLoadHistFail(c *C) { for _, col := range tableInfo.Columns { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } - timeout := time.Nanosecond * math.MaxInt + timeout := time.Nanosecond * mathutil.MaxInt // TODO failpoint, and works again after failpoint h.SendLoadRequests(stmtCtx, neededColumns, timeout) rs := h.SyncWaitStatsLoad(stmtCtx) From c3efa75d8ae993b1f43683a123c438df8cc30543 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 21:19:45 +0800 Subject: [PATCH 21/38] fix --- statistics/histogram.go | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index 9c21394197281..9947ee6dfa671 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1073,16 +1073,18 @@ func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) bool { if collPseudo && c.NotAccurate() { return true } - stmtctx := sctx.GetSessionVars().StmtCtx - if stmtctx != nil && stmtctx.StatsLoad.Fallback { - return true - } - if c.Histogram.NDV > 0 && c.notNullCount() == 0 && sctx != nil && stmtctx != nil { - if stmtctx.StatsLoad.Timeout > 0 { - logutil.BgLogger().Warn("Hist for column should already be loaded as sync but not found.", - zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) + if sctx != nil { + stmtctx := sctx.GetSessionVars().StmtCtx + if stmtctx != nil && stmtctx.StatsLoad.Fallback { + return true + } + if c.Histogram.NDV > 0 && c.notNullCount() == 0 && stmtctx != nil { + if stmtctx.StatsLoad.Timeout > 0 { + logutil.BgLogger().Warn("Hist for column should already be loaded as sync but not found.", + zap.String(strconv.FormatInt(c.Info.ID, 10), c.Info.Name.O)) + } + HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) } - HistogramNeededColumns.insert(tableColumnID{TableID: c.PhysicalID, ColumnID: c.Info.ID}) } return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) } From 5a5b0dc4644f43fbcd51b5f5e287cd23c3df86d0 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 15 Dec 2021 23:29:50 +0800 Subject: [PATCH 22/38] separate test suite --- statistics/handle/handle_hist_test.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index bc9f7a2533926..ea194451a8ea8 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -24,7 +24,13 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { +var _ = Suite(&testLoadHistSuite{}) + +type testLoadHistSuite struct { + testSuiteBase +} + +func (s *testLoadHistSuite) TestConcurrentLoadHist(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -69,7 +75,7 @@ func (s *testStatsSuite) TestConcurrentLoadHist(c *C) { // TODO partition table } -func (s *testStatsSuite) TestConcurrentLoadHistTimeout(c *C) { +func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -121,7 +127,7 @@ func (s *testStatsSuite) TestConcurrentLoadHistTimeout(c *C) { c.Assert(hg.Len()+topn.Num(), Greater, 0) } -func (s *testStatsSuite) TestConcurrentLoadHistFail(c *C) { +func (s *testLoadHistSuite) TestConcurrentLoadHistFail(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") From 1ccb9bc4dbd6838730cb943da193e17940eaa249 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 16 Dec 2021 01:50:01 +0800 Subject: [PATCH 23/38] fix for partition table --- planner/core/hist_column_collector.go | 2 +- planner/core/logical_plan_builder.go | 1 + planner/core/optimizer.go | 4 ++-- planner/core/plan_stats_test.go | 17 ++++++++--------- planner/core/rule_partition_processor.go | 3 --- planner/core/stats.go | 2 +- 6 files changed, 13 insertions(+), 16 deletions(-) diff --git a/planner/core/hist_column_collector.go b/planner/core/hist_column_collector.go index 57ee22959f381..4c372190cea5e 100644 --- a/planner/core/hist_column_collector.go +++ b/planner/core/hist_column_collector.go @@ -36,7 +36,7 @@ func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColum } switch x := plan.(type) { case *DataSource: - tblID := x.TableInfo().ID + tblID := x.physicalTableID columns := expression.ExtractColumnsFromExpressions(nil, x.pushedDownConds, nil) for _, col := range columns { tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 028bfa2c66d87..84149462965e1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4096,6 +4096,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as TableAsName: asName, table: tbl, tableInfo: tableInfo, + physicalTableID: tableInfo.ID, astIndexHints: tn.IndexHints, IndexHints: b.TableHints().indexHintList, indexMergeHints: indexMergeHints, diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 434a97c17f4c7..215428dc40dc2 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -60,9 +60,9 @@ const ( flagEliminateProjection flagMaxMinEliminate flagPredicatePushDown - flagCollectPredicateColumnsPoint flagEliminateOuterJoin flagPartitionProcessor + flagCollectPredicateColumnsPoint flagPushDownAgg flagPushDownTopN flagSyncWaitStatsLoadPoint @@ -80,9 +80,9 @@ var optRuleList = []logicalOptRule{ &projectionEliminator{}, &maxMinEliminator{}, &ppdSolver{}, - &collectPredicateColumnsPoint{}, &outerJoinEliminator{}, &partitionProcessor{}, + &collectPredicateColumnsPoint{}, &aggregationPushDownSolver{}, &pushDownTopNOptimizer{}, &syncWaitStatsLoadPoint{}, diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index ae39201c74ea2..a12c4a270b352 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -55,9 +55,11 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { ctx := tk.Se.(sessionctx.Context) tk.MustExec("drop table if exists t") tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") tk.MustExec("create table t(a int, b int, c int, d int, primary key(a), key idx(b))") tk.MustExec("insert into t values (1,1,1,1),(2,2,2,2),(3,3,3,3)") tk.MustExec("create table pt(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)") + tk.MustExec("insert into pt values (1,1,1),(2,2,2),(13,13,13),(14,14,14),(25,25,25),(36,36,36)") oriLease := dom.StatsHandle().Lease() dom.StatsHandle().SetLease(1) @@ -85,16 +87,13 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { } }, }, - { // PartitionTable TODO - sql: "select * from pt where a < 15 and c > 1", - skip: true, + { // PartitionTable + sql: "select * from pt where a < 15 and c > 1", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { - switch pp := p.(type) { - case *plannercore.PhysicalTableReader: - stats := pp.Stats().HistColl - c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Greater, 0) - default: - c.Error("unexpected plan:", pp) + pua, ok := p.(*plannercore.PhysicalUnionAll) + c.Check(ok, IsTrue) + for _, child := range pua.Children() { + c.Assert(countFullStats(child.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) } }, }, diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index bb57b0fac33da..1264a47ac97bc 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -1415,9 +1415,6 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. newDataSource.id = ds.id - if !ds.ctx.GetSessionVars().UseDynamicPartitionPrune() { - newDataSource.statisticTable = getStatsTable(ds.SCtx(), ds.table.Meta(), pi.Definitions[i].ID) - } err := s.resolveOptimizeHint(&newDataSource, pi.Definitions[i].Name) partitionNameSet.Insert(pi.Definitions[i].Name.L) if err != nil { diff --git a/planner/core/stats.go b/planner/core/stats.go index 2e7fd14a67b8d..d3f23427b2f40 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -226,7 +226,7 @@ func (ds *DataSource) initStats(colGroups [][]*expression.Column) { return } if ds.statisticTable == nil { - ds.statisticTable = getStatsTable(ds.ctx, ds.tableInfo, ds.table.Meta().ID) + ds.statisticTable = getStatsTable(ds.ctx, ds.tableInfo, ds.physicalTableID) } tableStats := &property.StatsInfo{ RowCount: float64(ds.statisticTable.Count), From f6dba589ec2701475ddcd3da98c6b03877d73612 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Sat, 18 Dec 2021 11:56:23 +0800 Subject: [PATCH 24/38] move configs to sysvars --- config/config.go | 13 +------------ domain/sysvar_cache.go | 2 ++ planner/core/plan_stats.go | 21 ++++++++++++--------- sessionctx/variable/session.go | 3 +++ sessionctx/variable/sysvar.go | 15 +++++++++++++++ sessionctx/variable/tidb_vars.go | 8 ++++++++ 6 files changed, 41 insertions(+), 21 deletions(-) diff --git a/config/config.go b/config/config.go index 1db864af5ec5b..d320242bc4340 100644 --- a/config/config.go +++ b/config/config.go @@ -190,8 +190,7 @@ type Config struct { // one quarter of the total physical memory in the current system. MaxBallastObjectSize int `toml:"max-ballast-object-size" json:"max-ballast-object-size"` // BallastObjectSize set the initial size of the ballast object, the unit is byte. - BallastObjectSize int `toml:"ballast-object-size" json:"ballast-object-size"` - Stats Stats `toml:"stats" json:"stats"` + BallastObjectSize int `toml:"ballast-object-size" json:"ballast-object-size"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed @@ -621,12 +620,6 @@ type Experimental struct { EnableNewCharset bool `toml:"enable-new-charset" json:"-"` } -// Stats controls the stats loading and usage behavior -type Stats struct { - SyncLoadWait uint `toml:"sync-load-wait" json:"sync-load-wait"` - PseudoForLoadTimeout bool `toml:"pseudo-for-load-timeout" json:"pseudo-for-load-timeout"` -} - var defTiKVCfg = tikvcfg.DefaultConfig() var defaultConf = Config{ Host: DefHost, @@ -775,10 +768,6 @@ var defaultConf = Config{ EnableEnumLengthLimit: true, StoresRefreshInterval: defTiKVCfg.StoresRefreshInterval, EnableForwarding: defTiKVCfg.EnableForwarding, - Stats: Stats{ - SyncLoadWait: 100, - PseudoForLoadTimeout: false, - }, } var ( diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index d89ba88a76ee0..8c0c530e1a998 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -240,6 +240,8 @@ func (do *Domain) checkEnableServerGlobalVar(name, sVal string) { break } storekv.StoreLimit.Store(val) + case variable.TiDBStatsLoadPseudoTimeout: + variable.StatsLoadPseudoTimeout.Store(variable.TiDBOptOn(sVal)) } if err != nil { logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err)) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 9cdd336028c19..9cf3a2a9b8d92 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -20,8 +20,8 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/sessionctx/variable" ) type collectPredicateColumnsPoint struct{} @@ -38,8 +38,8 @@ func (c collectPredicateColumnsPoint) name() string { type syncWaitStatsLoadPoint struct{} func (s syncWaitStatsLoadPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { - SyncWaitStatsLoad(plan) - return plan, nil + _, err := SyncWaitStatsLoad(plan) + return plan, err } func (s syncWaitStatsLoadPoint) name() string { @@ -51,7 +51,7 @@ func RequestLoadColumnStats(plan LogicalPlan) { if plan.SCtx().GetSessionVars().InRestrictedSQL { return } - syncWait := int64(config.GetGlobalConfig().Stats.SyncLoadWait) + syncWait := int64(plan.SCtx().GetSessionVars().StatsLoadSyncWait) if syncWait <= 0 { return } @@ -71,14 +71,17 @@ func RequestLoadColumnStats(plan LogicalPlan) { } // SyncWaitStatsLoad sync-wait for stats load until timeout -func SyncWaitStatsLoad(plan LogicalPlan) bool { +func SyncWaitStatsLoad(plan LogicalPlan) (bool, error) { stmtCtx := plan.SCtx().GetSessionVars().StmtCtx success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) - if !success && config.GetGlobalConfig().Stats.PseudoForLoadTimeout { - err := errors.New("Timeout when sync-load full stats for needed columns") + if success { + return true, nil + } + err := errors.New("Timeout when sync-load full stats for needed columns") + if variable.StatsLoadPseudoTimeout.Load() { stmtCtx.AppendWarning(err) stmtCtx.StatsLoad.Fallback = true - return false + return false, nil } - return true + return false, err } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8169eaa5c2d66..b889cb9459036 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -969,6 +969,9 @@ type SessionVars struct { // EnablePaging indicates whether enable paging in coprocessor requests. EnablePaging bool + + // StatsLoadSyncWait indicates how long to wait for stats load before timeout. + StatsLoadSyncWait int } // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index fc7ce09cae6a7..63917526a982a 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1331,6 +1331,21 @@ var defaultSysVars = []*SysVar{ s.EnablePaging = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStatsLoadSyncWait, Value: strconv.Itoa(DefTiDBStatsLoadSyncWait), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + SetSession: func(s *SessionVars, val string) error { + s.StatsLoadSyncWait = tidbOptInt(val, DefTiDBStatsLoadSyncWait) + return nil + }, + }, + {Scope: ScopeGlobal, Name: TiDBStatsLoadPseudoTimeout, Value: BoolToOnOff(DefTiDBStatsLoadPseudoTimeout), skipInit: true, Type: TypeBool, + GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatBool(StatsLoadPseudoTimeout.Load()), nil + }, + SetGlobal: func(s *SessionVars, val string) error { + StatsLoadPseudoTimeout.Store(TiDBOptOn(val)) + 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..ae25e70846a7b 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -603,6 +603,9 @@ const ( // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" + + // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. + TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" ) // TiDB vars that have only global scope @@ -622,6 +625,8 @@ const ( TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" // TiDBEnableHistoricalStats enables the historical statistics feature (default off) TiDBEnableHistoricalStats = "tidb_enable_historical_stats" + // TiDBStatsLoadPseudoTimeout indicates whether to fallback to pseudo stats after load timeout. + TiDBStatsLoadPseudoTimeout = "tidb_stats_load_pseudo_timeout" ) // TiDB intentional limits @@ -779,6 +784,8 @@ const ( DefTiDBRegardNULLAsPoint = true DefEnablePlacementCheck = true DefTimestamp = "0" + DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadPseudoTimeout = false ) // Process global variables. @@ -814,6 +821,7 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) ) // TopSQL is the variable for control top sql feature. From 00fb138fd53d8d7299e137c980b985c514fc8fff Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 23 Dec 2021 16:20:29 +0800 Subject: [PATCH 25/38] disable sync-load by default and make waitTime configurable both in session & global --- domain/sysvar_cache.go | 7 +++++++ planner/core/plan_stats.go | 2 +- planner/core/plan_stats_test.go | 1 + sessionctx/variable/session.go | 3 ++- sessionctx/variable/sysvar.go | 9 ++++++++- sessionctx/variable/tidb_vars.go | 3 ++- 6 files changed, 21 insertions(+), 4 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 8e8ee67cb91a6..2252551522009 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.TiDBStatsLoadSyncWait: + var val int64 + val, err = strconv.ParseInt(sVal, 10, 64) + if err != nil { + break + } + variable.StatsLoadSyncWait.Store(val) case variable.TiDBStatsLoadPseudoTimeout: variable.StatsLoadPseudoTimeout.Store(variable.TiDBOptOn(sVal)) } diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 9cf3a2a9b8d92..eef0a9af447b0 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -51,7 +51,7 @@ func RequestLoadColumnStats(plan LogicalPlan) { if plan.SCtx().GetSessionVars().InRestrictedSQL { return } - syncWait := int64(plan.SCtx().GetSessionVars().StatsLoadSyncWait) + syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait if syncWait <= 0 { return } diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index a12c4a270b352..1761ab8e7ad12 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -56,6 +56,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("set @@tidb_analyze_version=2") tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") + tk.MustExec("set @@session.tidb_stats_load_sync_wait =1000") tk.MustExec("create table t(a int, b int, c int, d int, primary key(a), key idx(b))") tk.MustExec("insert into t values (1,1,1,1),(2,2,2,2),(3,3,3,3)") tk.MustExec("create table pt(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)") diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index af1be194447c9..e9e1e386bd165 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -981,7 +981,7 @@ type SessionVars struct { StmtStats *stmtstats.StatementStats // StatsLoadSyncWait indicates how long to wait for stats load before timeout. - StatsLoadSyncWait int + StatsLoadSyncWait int64 } // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. @@ -1217,6 +1217,7 @@ func NewSessionVars() *SessionVars { EnablePlacementChecks: DefEnablePlacementCheck, Rng: utilMath.NewWithTime(), StmtStats: stmtstats.CreateStatementStats(), + StatsLoadSyncWait: StatsLoadSyncWait.Load(), } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 63430cdeaceba..0a49f8de4d928 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1327,7 +1327,14 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStatsLoadSyncWait, Value: strconv.Itoa(DefTiDBStatsLoadSyncWait), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.StatsLoadSyncWait = tidbOptInt(val, DefTiDBStatsLoadSyncWait) + s.StatsLoadSyncWait = tidbOptInt64(val, DefTiDBStatsLoadSyncWait) + return nil + }, + GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatInt(StatsLoadSyncWait.Load(), 10), nil + }, + SetGlobal: func(s *SessionVars, val string) error { + StatsLoadSyncWait.Store(tidbOptInt64(val, DefTiDBStatsLoadSyncWait)) return nil }, }, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 58693c053fded..3c6bc86bfb236 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -784,7 +784,7 @@ const ( DefTiDBRegardNULLAsPoint = true DefEnablePlacementCheck = true DefTimestamp = "0" - DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadSyncWait = 0 DefTiDBStatsLoadPseudoTimeout = false ) @@ -821,6 +821,7 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) ) From f0473a9ea8ad6b6d133224cf3d6cd2c3454f7108 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 23 Dec 2021 20:55:54 +0800 Subject: [PATCH 26/38] add debug log --- planner/core/plan_stats.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index eef0a9af447b0..8e276df9cff1b 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -16,12 +16,16 @@ package core import ( "context" + "strings" "time" "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) type collectPredicateColumnsPoint struct{} @@ -67,6 +71,16 @@ func RequestLoadColumnStats(plan LogicalPlan) { waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) * time.Millisecond neededColumns := CollectHistColumns(plan) + if config.GetGlobalConfig().Log.Level == "debug" && len(neededColumns) > 0 { + neededColInfos := make([]string, len(neededColumns)) + is := domain.GetDomain(plan.SCtx()).InfoSchema() + for i, col := range neededColumns { + table, _ := is.TableByID(col.TableID) + colInfo := FindColumnInfoByID(table.Meta().Columns, col.ColumnID) + neededColInfos[i] = table.Meta().Name.L + "." + colInfo.Name.L + } + logutil.BgLogger().Debug("Full stats are needed:", zap.String("columns", strings.Join(neededColInfos, ","))) + } domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) } From 807faede6889235d2726bf611b1ee453e8ebe6e7 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 28 Dec 2021 11:57:15 +0800 Subject: [PATCH 27/38] integration and comments --- planner/core/hist_column_collector.go | 47 ------------------------- planner/core/plan_stats.go | 2 +- planner/core/plan_stats_test.go | 2 +- statistics/handle/handle_hist.go | 3 ++ statistics/handle/handle_hist_test.go | 50 ++------------------------- 5 files changed, 7 insertions(+), 97 deletions(-) delete mode 100644 planner/core/hist_column_collector.go diff --git a/planner/core/hist_column_collector.go b/planner/core/hist_column_collector.go deleted file mode 100644 index 4c372190cea5e..0000000000000 --- a/planner/core/hist_column_collector.go +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package core - -import ( - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/parser/model" -) - -// CollectHistColumns collects hist-needed columns from plan -func CollectHistColumns(plan LogicalPlan) []model.TableColumnID { - colMap := map[model.TableColumnID]struct{}{} - collectColumnsFromPlan(plan, colMap) - histColumns := make([]model.TableColumnID, 0, len(colMap)) - for col := range colMap { - histColumns = append(histColumns, col) - } - return histColumns -} - -func collectColumnsFromPlan(plan LogicalPlan, neededColumns map[model.TableColumnID]struct{}) { - for _, child := range plan.Children() { - collectColumnsFromPlan(child, neededColumns) - } - switch x := plan.(type) { - case *DataSource: - tblID := x.physicalTableID - columns := expression.ExtractColumnsFromExpressions(nil, x.pushedDownConds, nil) - for _, col := range columns { - tblColID := model.TableColumnID{TableID: tblID, ColumnID: col.ID} - neededColumns[tblColID] = struct{}{} - } - // TODO collect idx columns? - } -} diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 8e276df9cff1b..8f916c08aeae5 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -70,7 +70,7 @@ func RequestLoadColumnStats(plan LogicalPlan) { } waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) * time.Millisecond - neededColumns := CollectHistColumns(plan) + _, neededColumns := CollectColumnStatsUsage(plan) if config.GetGlobalConfig().Log.Level == "debug" && len(neededColumns) > 0 { neededColInfos := make([]string, len(neededColumns)) is := domain.GetDomain(plan.SCtx()).InfoSchema() diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index 1761ab8e7ad12..bbe9b85640048 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -54,7 +54,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { tk.MustExec("use test") ctx := tk.Se.(sessionctx.Context) tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") tk.MustExec("set @@session.tidb_stats_load_sync_wait =1000") tk.MustExec("create table t(a int, b int, c int, d int, primary key(a), key idx(b))") diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 8e129d68783e5..de16bbc2f60d7 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -276,6 +276,8 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { if !ok { return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed") } + // if the task has already timeout, no sql is sync-waiting for it, + // so do not handle it just now, put it to another channel with lower priority if time.Now().After(task.ToTimeout) { h.writeToTimeoutCh(task) continue @@ -303,6 +305,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { } } +// writeToTimeoutCh writes in a nonblocking way, and if the channel queue is full, it's ok to drop the task, since timeout task is not that critical. func (h *Handle) writeToTimeoutCh(task *NeededColumnTask) { select { case h.StatsLoad.TimeoutColumnsCh <- task: diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index ea194451a8ea8..741d65cad05cf 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -35,7 +35,7 @@ func (s *testLoadHistSuite) TestConcurrentLoadHist(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") - testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") @@ -71,8 +71,6 @@ func (s *testLoadHistSuite) TestConcurrentLoadHist(c *C) { hg = stat.Columns[tableInfo.Columns[2].ID].Histogram topn = stat.Columns[tableInfo.Columns[2].ID].TopN c.Assert(hg.Len()+topn.Num(), Greater, 0) - - // TODO partition table } func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { @@ -80,7 +78,7 @@ func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") - testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") @@ -126,47 +124,3 @@ func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { topn = stat.Columns[tableInfo.Columns[2].ID].TopN c.Assert(hg.Len()+topn.Num(), Greater, 0) } - -func (s *testLoadHistSuite) TestConcurrentLoadHistFail(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("set @@tidb_analyze_version=2") - testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") - testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") - - oriLease := s.do.StatsHandle().Lease() - s.do.StatsHandle().SetLease(1) - defer func() { - s.do.StatsHandle().SetLease(oriLease) - }() - testKit.MustExec("analyze table t") - - is := s.do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tableInfo := tbl.Meta() - h := s.do.StatsHandle() - stat := h.GetTableStats(tableInfo) - hg := stat.Columns[tableInfo.Columns[0].ID].Histogram - topn := stat.Columns[tableInfo.Columns[0].ID].TopN - c.Assert(hg.Len()+topn.Num(), Greater, 0) - hg = stat.Columns[tableInfo.Columns[2].ID].Histogram - topn = stat.Columns[tableInfo.Columns[2].ID].TopN - c.Assert(hg.Len()+topn.Num(), Equals, 0) - stmtCtx := &stmtctx.StatementContext{} - neededColumns := make([]model.TableColumnID, 0, len(tableInfo.Columns)) - for _, col := range tableInfo.Columns { - neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) - } - timeout := time.Nanosecond * mathutil.MaxInt - // TODO failpoint, and works again after failpoint - h.SendLoadRequests(stmtCtx, neededColumns, timeout) - rs := h.SyncWaitStatsLoad(stmtCtx) - c.Assert(rs, Equals, true) - stat = h.GetTableStats(tableInfo) - hg = stat.Columns[tableInfo.Columns[2].ID].Histogram - topn = stat.Columns[tableInfo.Columns[2].ID].TopN - c.Assert(hg.Len()+topn.Num(), Greater, 0) -} From 4bb34b0a4ba7a4007c904d1eeefa2c279077e2b2 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Tue, 28 Dec 2021 13:58:49 +0800 Subject: [PATCH 28/38] fix test --- statistics/handle/handle_test.go | 2 +- statistics/handle/update_test.go | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index a298641e631e7..f80619b44515b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -586,7 +586,7 @@ func (s *testStatsSuite) TestLoadStats(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") - testKit.MustExec("set @@tidb_analyze_version=1") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 23a061cd96f6a..ad4f72ef86581 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -1464,6 +1464,8 @@ func (s *testStatsSuite) TestLogDetailedInfo(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), primary key(a), index idx(b), index idx_ba(b,a), index idx_bc(b,c))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d, %d)", i, i, i)) From 76d832f811516a1bfce58fdde88af609c162c007 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 29 Dec 2021 16:18:17 +0800 Subject: [PATCH 29/38] fix conflict & enable sync-load by default --- planner/core/plan_stats.go | 2 +- sessionctx/variable/tidb_vars.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 8f916c08aeae5..26ba03d188bc6 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -70,7 +70,7 @@ func RequestLoadColumnStats(plan LogicalPlan) { } waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) * time.Millisecond - _, neededColumns := CollectColumnStatsUsage(plan) + _, neededColumns := CollectColumnStatsUsage(plan, false, true) if config.GetGlobalConfig().Log.Level == "debug" && len(neededColumns) > 0 { neededColInfos := make([]string, len(neededColumns)) is := domain.GetDomain(plan.SCtx()).InfoSchema() diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 88d96d29f9ccf..ca20b7c446acb 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -785,7 +785,7 @@ const ( DefEnablePlacementCheck = true DefTimestamp = "0" DefTiDBEnableIndexMerge = true - DefTiDBStatsLoadSyncWait = 0 + DefTiDBStatsLoadSyncWait = 100 DefTiDBStatsLoadPseudoTimeout = false ) From 59181326a548ff66bf4720c9aa1ae488f54f2f64 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 29 Dec 2021 19:27:25 +0800 Subject: [PATCH 30/38] fix ut --- statistics/handle/update_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 447c3003ae216..1b9862df2ed3d 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -1467,6 +1467,7 @@ func (s *testStatsSuite) TestLogDetailedInfo(c *C) { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("set @@session.tidb_stats_load_sync_wait =0") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), primary key(a), index idx(b), index idx_ba(b,a), index idx_bc(b,c))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d, %d)", i, i, i)) From da39ca2e447c322d5a27837148701db1bcc04359 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 29 Dec 2021 23:05:41 +0800 Subject: [PATCH 31/38] fix write channel timeout and add ut --- planner/core/plan_stats.go | 39 +++++++++++++------ planner/core/plan_stats_test.go | 64 ++++++++++++++++++++++++++++++++ statistics/handle/handle_hist.go | 39 ++++++++++++------- 3 files changed, 117 insertions(+), 25 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 26ba03d188bc6..e8818dba80e74 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -16,6 +16,7 @@ package core import ( "context" + "github.com/pingcap/tidb/sessionctx/stmtctx" "strings" "time" @@ -31,8 +32,8 @@ import ( type collectPredicateColumnsPoint struct{} func (c collectPredicateColumnsPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { - RequestLoadColumnStats(plan) - return plan, nil + err := RequestLoadColumnStats(plan) + return plan, err } func (c collectPredicateColumnsPoint) name() string { @@ -50,26 +51,28 @@ func (s syncWaitStatsLoadPoint) name() string { return "sync_wait_stats_load_point" } +const maxDuration = 1<<63 - 1 + // RequestLoadColumnStats send requests to stats handle -func RequestLoadColumnStats(plan LogicalPlan) { +func RequestLoadColumnStats(plan LogicalPlan) error { if plan.SCtx().GetSessionVars().InRestrictedSQL { - return + return nil } - syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait + syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait * time.Millisecond.Nanoseconds() if syncWait <= 0 { - return + return nil } stmtCtx := plan.SCtx().GetSessionVars().StmtCtx hintMaxExecutionTime := int64(stmtCtx.MaxExecutionTime) if hintMaxExecutionTime == 0 { - hintMaxExecutionTime = mathutil.MaxInt + hintMaxExecutionTime = maxDuration } sessMaxExecutionTime := int64(plan.SCtx().GetSessionVars().MaxExecutionTime) if sessMaxExecutionTime == 0 { - sessMaxExecutionTime = mathutil.MaxInt + sessMaxExecutionTime = maxDuration } waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) - var timeout = time.Duration(waitTime) * time.Millisecond + var timeout = time.Duration(waitTime) _, neededColumns := CollectColumnStatsUsage(plan, false, true) if config.GetGlobalConfig().Log.Level == "debug" && len(neededColumns) > 0 { neededColInfos := make([]string, len(neededColumns)) @@ -81,21 +84,33 @@ func RequestLoadColumnStats(plan LogicalPlan) { } logutil.BgLogger().Debug("Full stats are needed:", zap.String("columns", strings.Join(neededColInfos, ","))) } - domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) + err := domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) + if err != nil { + return handleTimeout(stmtCtx) + } + return nil } // SyncWaitStatsLoad sync-wait for stats load until timeout func SyncWaitStatsLoad(plan LogicalPlan) (bool, error) { stmtCtx := plan.SCtx().GetSessionVars().StmtCtx + if stmtCtx.StatsLoad.Fallback { + return false, nil + } success := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) if success { return true, nil } + err := handleTimeout(stmtCtx) + return false, err +} + +func handleTimeout(stmtCtx *stmtctx.StatementContext) error { err := errors.New("Timeout when sync-load full stats for needed columns") if variable.StatsLoadPseudoTimeout.Load() { stmtCtx.AppendWarning(err) stmtCtx.StatsLoad.Fallback = true - return false, nil + return nil } - return false, err + return err } diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index bbe9b85640048..d2209e9af3e32 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -16,8 +16,11 @@ package core_test import ( "context" + "fmt" + "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" @@ -163,3 +166,64 @@ func countFullStats(stats *statistics.HistColl, colID int64) int { } return -1 } + +func (s *testPlanStatsSuite) TestPlanStatsLoadTimeout(c *C) { + originConfig := config.GetGlobalConfig() + newConfig := config.NewConfig() + newConfig.Performance.StatsLoadConcurrency = 0 + newConfig.Performance.StatsLoadQueueSize = 1 + config.StoreGlobalConfig(newConfig) + defer config.StoreGlobalConfig(originConfig) + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Check(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + originalVal1 := tk.MustQuery("select @@tidb_stats_load_pseudo_timeout").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_stats_load_pseudo_timeout = %v", originalVal1)) + }() + + ctx := tk.Se.(sessionctx.Context) + tk.MustExec("drop table if exists t") + tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 1") + tk.MustExec("create table t(a int, b int, c int, primary key(a))") + tk.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") + + oriLease := dom.StatsHandle().Lease() + dom.StatsHandle().SetLease(1) + defer func() { + dom.StatsHandle().SetLease(oriLease) + }() + tk.MustExec("analyze table t") + is := dom.InfoSchema() + c.Assert(dom.StatsHandle().Update(is), IsNil) + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + neededColumn := model.TableColumnID{TableID: tableInfo.ID, ColumnID: tableInfo.Columns[0].ID} + resultCh := make(chan model.TableColumnID, 1) + timeout := time.Duration(1<<63 - 1) + dom.StatsHandle().AppendNeededColumn(neededColumn, resultCh, timeout) + stmt, err := s.ParseOneStmt("select * from t where c>1", "", "") + c.Check(err, IsNil) + tk.MustExec("set global tidb_stats_load_pseudo_timeout=false") + _, _, err = planner.Optimize(context.TODO(), ctx, stmt, is) + c.Check(err, NotNil) + tk.MustExec("set global tidb_stats_load_pseudo_timeout=true") + plan, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + c.Check(err, IsNil) + switch pp := plan.(type) { + case *plannercore.PhysicalTableReader: + stats := pp.Stats().HistColl + c.Assert(countFullStats(stats, tableInfo.Columns[0].ID), Greater, 0) + c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Equals, 0) + default: + c.Error("unexpected plan:", pp) + } +} diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index de16bbc2f60d7..57a2994a6c5aa 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -47,18 +47,22 @@ type NeededColumnTask struct { } // SendLoadRequests send neededColumns requests -func (h *Handle) SendLoadRequests(sc *stmtctx.StatementContext, neededColumns []model.TableColumnID, timeout time.Duration) { +func (h *Handle) SendLoadRequests(sc *stmtctx.StatementContext, neededColumns []model.TableColumnID, timeout time.Duration) error { missingColumns := h.genHistMissingColumns(neededColumns) if len(missingColumns) <= 0 { - return + return nil } sc.StatsLoad.Timeout = timeout sc.StatsLoad.NeededColumns = missingColumns sc.StatsLoad.ResultCh = make(chan model.TableColumnID, len(neededColumns)) for _, col := range missingColumns { - h.appendNeededColumn(col, sc.StatsLoad.ResultCh, timeout) + err := h.AppendNeededColumn(col, sc.StatsLoad.ResultCh, timeout) + if err != nil { + return err + } } sc.StatsLoad.LoadStartTime = time.Now() + return nil } // SyncWaitStatsLoad sync waits loading of neededColumns and return false if timeout @@ -115,11 +119,11 @@ func (h *Handle) genHistMissingColumns(neededColumns []model.TableColumnID) []mo return missingColumns } -// appendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. -func (h *Handle) appendNeededColumn(c model.TableColumnID, resultCh chan model.TableColumnID, timeout time.Duration) { +// AppendNeededColumn appends needed column to ch, if exists, do not append the duplicated one. +func (h *Handle) AppendNeededColumn(c model.TableColumnID, resultCh chan model.TableColumnID, timeout time.Duration) error { toTimout := time.Now().Local().Add(timeout) colTask := &NeededColumnTask{TableColumnID: c, ToTimeout: toTimout, ResultCh: resultCh} - h.StatsLoad.NeededColumnsCh <- colTask + return h.writeToChanWithTimeout(h.StatsLoad.NeededColumnsCh, colTask, timeout) } var errExit = errors.New("Stop loading since domain is closed") @@ -279,7 +283,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { // if the task has already timeout, no sql is sync-waiting for it, // so do not handle it just now, put it to another channel with lower priority if time.Now().After(task.ToTimeout) { - h.writeToTimeoutCh(task) + h.writeToChanNonblocking(h.StatsLoad.TimeoutColumnsCh, task) continue } return task, nil @@ -292,7 +296,7 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { return nil, errors.New("drainColTask: cannot read from NeededColumnsCh, maybe the chan is closed") } // send task back to TimeoutColumnsCh and return the task drained from NeededColumnsCh - h.writeToTimeoutCh(task) + h.writeToChanNonblocking(h.StatsLoad.TimeoutColumnsCh, task) return task0, nil default: if !ok { @@ -305,15 +309,24 @@ func (h *Handle) drainColTask(exit chan struct{}) (*NeededColumnTask, error) { } } -// writeToTimeoutCh writes in a nonblocking way, and if the channel queue is full, it's ok to drop the task, since timeout task is not that critical. -func (h *Handle) writeToTimeoutCh(task *NeededColumnTask) { +// writeToChanNonblocking writes in a nonblocking way, and if the channel queue is full, it's ok to drop the task. +func (h *Handle) writeToChanNonblocking(taskCh chan *NeededColumnTask, task *NeededColumnTask) { select { - case h.StatsLoad.TimeoutColumnsCh <- task: + case taskCh <- task: default: - logutil.BgLogger().Debug("TimeoutCh is full, drop task:", zap.Int64("table", task.TableColumnID.TableID), - zap.Int64("column", task.TableColumnID.ColumnID)) } +} +// writeToChanWithTimeout writes a task to a channel and blocks until timeout. +func (h *Handle) writeToChanWithTimeout(taskCh chan *NeededColumnTask, task *NeededColumnTask, timeout time.Duration) error { + timer := time.NewTimer(timeout) + defer timer.Stop() + select { + case taskCh <- task: + case <-timer.C: + return errors.New("Channel is full and timeout writing to channel.") + } + return nil } // updateCachedColumn updates the column hist to global statsCache. From 49a8e591d605c0f73806f738d4cf4ed0b362ad91 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Wed, 29 Dec 2021 23:22:28 +0800 Subject: [PATCH 32/38] fix gofmt --- sessionctx/variable/tidb_vars.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 185d98d9543aa..6a87a75847d02 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -810,6 +810,6 @@ var ( MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) - StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) - StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) + StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) + StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) ) From f7e07cd12c97e2f781871f12de1e95c93f3651ce Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 30 Dec 2021 00:40:30 +0800 Subject: [PATCH 33/38] fix gofmt --- statistics/handle/handle_hist.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 57a2994a6c5aa..d7b407e5c5b20 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -324,7 +324,7 @@ func (h *Handle) writeToChanWithTimeout(taskCh chan *NeededColumnTask, task *Nee select { case taskCh <- task: case <-timer.C: - return errors.New("Channel is full and timeout writing to channel.") + return errors.New("Channel is full and timeout writing to channel") } return nil } From a9b902c546a5351f97919119ca8d2fc9ea42f8ae Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 30 Dec 2021 00:51:00 +0800 Subject: [PATCH 34/38] fix gofmt --- planner/core/plan_stats.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index e8818dba80e74..fd0180d085b37 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -16,7 +16,6 @@ package core import ( "context" - "github.com/pingcap/tidb/sessionctx/stmtctx" "strings" "time" @@ -24,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" From 1273760c7010337e87851bba7e24e99acfd7ad1a Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 30 Dec 2021 16:53:27 +0800 Subject: [PATCH 35/38] disable syncload by default --- planner/core/plan_stats.go | 14 -------------- sessionctx/variable/tidb_vars.go | 2 +- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index fd0180d085b37..81a1ae971e45a 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -16,17 +16,13 @@ package core import ( "context" - "strings" "time" "github.com/cznic/mathutil" "github.com/pingcap/errors" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" ) type collectPredicateColumnsPoint struct{} @@ -74,16 +70,6 @@ func RequestLoadColumnStats(plan LogicalPlan) error { waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) _, neededColumns := CollectColumnStatsUsage(plan, false, true) - if config.GetGlobalConfig().Log.Level == "debug" && len(neededColumns) > 0 { - neededColInfos := make([]string, len(neededColumns)) - is := domain.GetDomain(plan.SCtx()).InfoSchema() - for i, col := range neededColumns { - table, _ := is.TableByID(col.TableID) - colInfo := FindColumnInfoByID(table.Meta().Columns, col.ColumnID) - neededColInfos[i] = table.Meta().Name.L + "." + colInfo.Name.L - } - logutil.BgLogger().Debug("Full stats are needed:", zap.String("columns", strings.Join(neededColInfos, ","))) - } err := domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) if err != nil { return handleTimeout(stmtCtx) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ea5c4f7eaae02..457c69ddaec52 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -790,7 +790,7 @@ const ( DefTiDBEnableIndexMerge = true DefTiDBPersistAnalyzeOptions = true DefTiDBEnableColumnTracking = true - DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadSyncWait = 0 DefTiDBStatsLoadPseudoTimeout = false ) From 53e759c53544dd8c078dbb6cc868afa945e5ceb4 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 30 Dec 2021 17:28:38 +0800 Subject: [PATCH 36/38] integration with columnTracking --- planner/core/optimizer.go | 5 ----- planner/core/plan_stats.go | 39 ++++++++++++++++++++++---------------- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 68eb8760809cd..5cdd693a7e77e 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -261,11 +261,6 @@ 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. - 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/plan_stats.go b/planner/core/plan_stats.go index 81a1ae971e45a..720804296d8f2 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -21,6 +21,8 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" ) @@ -28,8 +30,21 @@ import ( type collectPredicateColumnsPoint struct{} func (c collectPredicateColumnsPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { - err := RequestLoadColumnStats(plan) - return plan, err + if plan.SCtx().GetSessionVars().InRestrictedSQL { + return plan, nil + } + predicateNeeded := variable.EnableColumnTracking.Load() + syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait * time.Millisecond.Nanoseconds() + histNeeded := syncWait > 0 + predicateColumns, histNeededColumns := CollectColumnStatsUsage(plan, predicateNeeded, histNeeded) + if len(predicateColumns) > 0 { + plan.SCtx().UpdateColStatsUsage(predicateColumns) + } + if len(histNeededColumns) > 0 { + err := RequestLoadColumnStats(plan.SCtx(), histNeededColumns, syncWait) + return plan, err + } + return plan, nil } func (c collectPredicateColumnsPoint) name() string { @@ -50,27 +65,19 @@ func (s syncWaitStatsLoadPoint) name() string { const maxDuration = 1<<63 - 1 // RequestLoadColumnStats send requests to stats handle -func RequestLoadColumnStats(plan LogicalPlan) error { - if plan.SCtx().GetSessionVars().InRestrictedSQL { - return nil - } - syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait * time.Millisecond.Nanoseconds() - if syncWait <= 0 { - return nil - } - stmtCtx := plan.SCtx().GetSessionVars().StmtCtx +func RequestLoadColumnStats(ctx sessionctx.Context, neededColumns []model.TableColumnID, syncWait int64) error { + stmtCtx := ctx.GetSessionVars().StmtCtx hintMaxExecutionTime := int64(stmtCtx.MaxExecutionTime) - if hintMaxExecutionTime == 0 { + if hintMaxExecutionTime <= 0 { hintMaxExecutionTime = maxDuration } - sessMaxExecutionTime := int64(plan.SCtx().GetSessionVars().MaxExecutionTime) - if sessMaxExecutionTime == 0 { + sessMaxExecutionTime := int64(ctx.GetSessionVars().MaxExecutionTime) + if sessMaxExecutionTime <= 0 { sessMaxExecutionTime = maxDuration } waitTime := mathutil.MinInt64(syncWait, mathutil.MinInt64(hintMaxExecutionTime, sessMaxExecutionTime)) var timeout = time.Duration(waitTime) - _, neededColumns := CollectColumnStatsUsage(plan, false, true) - err := domain.GetDomain(plan.SCtx()).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) + err := domain.GetDomain(ctx).StatsHandle().SendLoadRequests(stmtCtx, neededColumns, timeout) if err != nil { return handleTimeout(stmtCtx) } From b8c99b4b23d99d4cbe927a5264c460a9122e1be8 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Thu, 30 Dec 2021 19:46:07 +0800 Subject: [PATCH 37/38] add config range validation --- config/config.go | 20 ++++++++++++++++++-- config/config_test.go | 21 +++++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/config/config.go b/config/config.go index d320242bc4340..1727fae20698f 100644 --- a/config/config.go +++ b/config/config.go @@ -65,6 +65,14 @@ const ( DefTableColumnCountLimit = 1017 // DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table DefMaxOfTableColumnCountLimit = 4096 + // DefStatsLoadConcurrencyLimit is limit of the concurrency of stats-load + DefStatsLoadConcurrencyLimit = 1 + // DefMaxOfStatsLoadConcurrencyLimit is maximum limitation of the concurrency of stats-load + DefMaxOfStatsLoadConcurrencyLimit = 128 + // DefStatsLoadQueueSizeLimit is limit of the size of stats-load request queue + DefStatsLoadQueueSizeLimit = 1 + // DefMaxOfStatsLoadQueueSizeLimit is maximum limitation of the size of stats-load request queue + DefMaxOfStatsLoadQueueSizeLimit = 100000 ) // Valid config maps @@ -708,8 +716,8 @@ var defaultConf = Config{ GOGC: 100, EnforceMPP: false, PlanReplayerGCLease: "10m", - StatsLoadConcurrency: 20, - StatsLoadQueueSize: 10000, + StatsLoadConcurrency: 5, + StatsLoadQueueSize: 1000, }, ProxyProtocol: ProxyProtocol{ Networks: "", @@ -1005,6 +1013,14 @@ func (c *Config) Valid() error { c.Security.SpilledFileEncryptionMethod, SpilledFileEncryptionMethodPlaintext, SpilledFileEncryptionMethodAES128CTR) } + // check stats load config + if c.Performance.StatsLoadConcurrency < DefStatsLoadConcurrencyLimit || c.Performance.StatsLoadConcurrency > DefMaxOfStatsLoadConcurrencyLimit { + return fmt.Errorf("stats-load-concurrency should be [%d, %d]", DefStatsLoadConcurrencyLimit, DefMaxOfStatsLoadConcurrencyLimit) + } + if c.Performance.StatsLoadQueueSize < DefStatsLoadQueueSizeLimit || c.Performance.StatsLoadQueueSize > DefMaxOfStatsLoadQueueSizeLimit { + return fmt.Errorf("stats-load-queue-size should be [%d, %d]", DefStatsLoadQueueSizeLimit, DefMaxOfStatsLoadQueueSizeLimit) + } + // test log level l := zap.NewAtomicLevel() return l.UnmarshalText([]byte(c.Log.Level)) diff --git a/config/config_test.go b/config/config_test.go index 25c79dc40ebe3..6515d76c56453 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -674,3 +674,24 @@ func TestConfigExample(t *testing.T) { } } } + +func TestStatsLoadLimit(t *testing.T) { + conf := NewConfig() + checkConcurrencyValid := func(concurrency int, shouldBeValid bool) { + conf.Performance.StatsLoadConcurrency = uint(concurrency) + require.Equal(t, shouldBeValid, conf.Valid() == nil) + } + checkConcurrencyValid(DefStatsLoadConcurrencyLimit, true) + checkConcurrencyValid(DefStatsLoadConcurrencyLimit-1, false) + checkConcurrencyValid(DefMaxOfStatsLoadConcurrencyLimit, true) + checkConcurrencyValid(DefMaxOfStatsLoadConcurrencyLimit+1, false) + conf = NewConfig() + checkQueueSizeValid := func(queueSize int, shouldBeValid bool) { + conf.Performance.StatsLoadQueueSize = uint(queueSize) + require.Equal(t, shouldBeValid, conf.Valid() == nil) + } + checkQueueSizeValid(DefStatsLoadQueueSizeLimit, true) + checkQueueSizeValid(DefStatsLoadQueueSizeLimit-1, false) + checkQueueSizeValid(DefMaxOfStatsLoadQueueSizeLimit, true) + checkQueueSizeValid(DefMaxOfStatsLoadQueueSizeLimit+1, false) +} From 8abdb0675354227efb4e64848fa03daa44c76e01 Mon Sep 17 00:00:00 2001 From: wuxiaoju Date: Fri, 31 Dec 2021 09:13:12 +0800 Subject: [PATCH 38/38] fix for recursive CTE and add more uts --- planner/core/plan_stats.go | 5 +- planner/core/plan_stats_test.go | 75 ++++++++++++++++++++++++--- statistics/handle/handle_hist.go | 1 + statistics/handle/handle_hist_test.go | 4 +- 4 files changed, 77 insertions(+), 8 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 720804296d8f2..f1100061b9f3b 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -40,7 +40,7 @@ func (c collectPredicateColumnsPoint) optimize(ctx context.Context, plan Logical if len(predicateColumns) > 0 { plan.SCtx().UpdateColStatsUsage(predicateColumns) } - if len(histNeededColumns) > 0 { + if histNeeded && len(histNeededColumns) > 0 { err := RequestLoadColumnStats(plan.SCtx(), histNeededColumns, syncWait) return plan, err } @@ -54,6 +54,9 @@ func (c collectPredicateColumnsPoint) name() string { type syncWaitStatsLoadPoint struct{} func (s syncWaitStatsLoadPoint) optimize(ctx context.Context, plan LogicalPlan, op *logicalOptimizeOp) (LogicalPlan, error) { + if plan.SCtx().GetSessionVars().InRestrictedSQL { + return plan, nil + } _, err := SyncWaitStatsLoad(plan) return plan, err } diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index d2209e9af3e32..63396ab7ce3f0 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -59,7 +59,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") - tk.MustExec("set @@session.tidb_stats_load_sync_wait =1000") + tk.MustExec("set @@session.tidb_stats_load_sync_wait =9999999") tk.MustExec("create table t(a int, b int, c int, d int, primary key(a), key idx(b))") tk.MustExec("insert into t values (1,1,1,1),(2,2,2,2),(3,3,3,3)") tk.MustExec("create table pt(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)") @@ -123,6 +123,56 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { c.Assert(countFullStats(right.Stats().HistColl, tableInfo.Columns[3].ID), Greater, 0) }, }, + { // > Any + sql: "select * from t where t.b > any(select d from t where t.c > 2)", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ph, ok := p.(*plannercore.PhysicalHashJoin) + c.Check(ok, IsTrue) + ptr, ok := ph.Children()[0].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, + { // in + sql: "select * from t where t.b in (select d from t where t.c > 2)", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ph, ok := p.(*plannercore.PhysicalHashJoin) + c.Check(ok, IsTrue) + ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, + { // not in + sql: "select * from t where t.b not in (select d from t where t.c > 2)", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ph, ok := p.(*plannercore.PhysicalHashJoin) + c.Check(ok, IsTrue) + ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, + { // exists + sql: "select * from t t1 where exists (select * from t t2 where t1.b > t2.d and t2.c>1)", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ph, ok := p.(*plannercore.PhysicalHashJoin) + c.Check(ok, IsTrue) + ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, + { // not exists + sql: "select * from t t1 where not exists (select * from t t2 where t1.b > t2.d and t2.c>1)", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + ph, ok := p.(*plannercore.PhysicalHashJoin) + c.Check(ok, IsTrue) + ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, { // CTE sql: "with cte(x, y) as (select d + 1, b from t where c > 1) select * from cte where x < 3", check: func(p plannercore.Plan, tableInfo *model.TableInfo) { @@ -137,6 +187,18 @@ func (s *testPlanStatsSuite) TestPlanStatsLoad(c *C) { c.Assert(countFullStats(reader.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) }, }, + { // recursive CTE + sql: "with recursive cte(x, y) as (select a, b from t where c > 1 union select x + 1, y from cte where x < 5) select * from cte", + check: func(p plannercore.Plan, tableInfo *model.TableInfo) { + pc, ok := p.(*plannercore.PhysicalCTE) + c.Check(ok, IsTrue) + pp, ok := pc.SeedPlan.(*plannercore.PhysicalProjection) + c.Check(ok, IsTrue) + reader, ok := pp.Children()[0].(*plannercore.PhysicalTableReader) + c.Check(ok, IsTrue) + c.Assert(countFullStats(reader.Stats().HistColl, tableInfo.Columns[2].ID), Greater, 0) + }, + }, } for _, testCase := range testCases { if testCase.skip { @@ -170,7 +232,7 @@ func countFullStats(stats *statistics.HistColl, colID int64) int { func (s *testPlanStatsSuite) TestPlanStatsLoadTimeout(c *C) { originConfig := config.GetGlobalConfig() newConfig := config.NewConfig() - newConfig.Performance.StatsLoadConcurrency = 0 + newConfig.Performance.StatsLoadConcurrency = 0 // no worker to consume channel newConfig.Performance.StatsLoadQueueSize = 1 config.StoreGlobalConfig(newConfig) defer config.StoreGlobalConfig(originConfig) @@ -191,6 +253,7 @@ func (s *testPlanStatsSuite) TestPlanStatsLoadTimeout(c *C) { ctx := tk.Se.(sessionctx.Context) tk.MustExec("drop table if exists t") tk.MustExec("set @@session.tidb_analyze_version=2") + // since queue full, make sync-wait return as timeout as soon as possible tk.MustExec("set @@session.tidb_stats_load_sync_wait = 1") tk.MustExec("create table t(a int, b int, c int, primary key(a))") tk.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") @@ -209,20 +272,20 @@ func (s *testPlanStatsSuite) TestPlanStatsLoadTimeout(c *C) { neededColumn := model.TableColumnID{TableID: tableInfo.ID, ColumnID: tableInfo.Columns[0].ID} resultCh := make(chan model.TableColumnID, 1) timeout := time.Duration(1<<63 - 1) - dom.StatsHandle().AppendNeededColumn(neededColumn, resultCh, timeout) + dom.StatsHandle().AppendNeededColumn(neededColumn, resultCh, timeout) // make channel queue full stmt, err := s.ParseOneStmt("select * from t where c>1", "", "") c.Check(err, IsNil) tk.MustExec("set global tidb_stats_load_pseudo_timeout=false") _, _, err = planner.Optimize(context.TODO(), ctx, stmt, is) - c.Check(err, NotNil) + c.Check(err, NotNil) // fail sql for timeout when pseudo=false tk.MustExec("set global tidb_stats_load_pseudo_timeout=true") plan, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) - c.Check(err, IsNil) + c.Check(err, IsNil) // not fail sql for timeout when pseudo=true switch pp := plan.(type) { case *plannercore.PhysicalTableReader: stats := pp.Stats().HistColl c.Assert(countFullStats(stats, tableInfo.Columns[0].ID), Greater, 0) - c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Equals, 0) + c.Assert(countFullStats(stats, tableInfo.Columns[2].ID), Equals, 0) // pseudo stats default: c.Error("unexpected plan:", pp) } diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index d7b407e5c5b20..e0cb34d88f171 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -74,6 +74,7 @@ func (h *Handle) SyncWaitStatsLoad(sc *stmtctx.StatementContext) bool { if sc.StatsLoad.ResultCh != nil { close(sc.StatsLoad.ResultCh) } + sc.StatsLoad.NeededColumns = nil }() resultCheckMap := map[model.TableColumnID]struct{}{} for _, col := range sc.StatsLoad.NeededColumns { diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index 741d65cad05cf..4443a7d36e2ab 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -79,6 +79,7 @@ func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_stats_load_sync_wait =9999999") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") @@ -106,13 +107,14 @@ func (s *testLoadHistSuite) TestConcurrentLoadHistTimeout(c *C) { for _, col := range tableInfo.Columns { neededColumns = append(neededColumns, model.TableColumnID{TableID: tableInfo.ID, ColumnID: col.ID}) } - h.SendLoadRequests(stmtCtx, neededColumns, 0) + h.SendLoadRequests(stmtCtx, neededColumns, 0) // set timeout to 0 so task will go to timeout channel rs := h.SyncWaitStatsLoad(stmtCtx) c.Assert(rs, Equals, false) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram topn = stat.Columns[tableInfo.Columns[2].ID].TopN c.Assert(hg.Len()+topn.Num(), Equals, 0) + // wait for timeout task to be handled for { time.Sleep(time.Millisecond * 100) if len(h.StatsLoad.TimeoutColumnsCh)+len(h.StatsLoad.NeededColumnsCh) == 0 {