unload fmsketch (#35098)

Signed-off-by: yisaer <disxiaofei@163.com>

Co-authored-by: Ti Chi Robot <ti-community-prow-bot@tidb.io>
This commit is contained in:
Song Gao
2022-06-06 14:38:07 +08:00
committed by GitHub
parent b2b7e311de
commit eca9f2d6be
5 changed files with 27 additions and 9 deletions

View File

@ -641,6 +641,8 @@ type Performance struct {
// ForcePriority, MemoryUsageAlarmRatio are deprecated.
ForcePriority string `toml:"force-priority" json:"force-priority"`
MemoryUsageAlarmRatio float64 `toml:"memory-usage-alarm-ratio" json:"memory-usage-alarm-ratio"`
EnableLoadFMSketch bool `toml:"enable-load-fmsketch" json:"enable-load-fmsketch"`
}
// PlanCache is the PlanCache section of the config.
@ -864,6 +866,7 @@ var defaultConf = Config{
StatsLoadQueueSize: 1000,
EnableStatsCacheMemQuota: false,
RunAutoAnalyze: true,
EnableLoadFMSketch: false,
},
ProxyProtocol: ProxyProtocol{
Networks: "",

View File

@ -736,6 +736,7 @@ enable-global-kill = true
[performance]
txn-total-size-limit=2000
tcp-no-delay = false
enable-load-fmsketch = true
[tikv-client]
commit-timeout="41s"
max-batch-size=128
@ -822,6 +823,7 @@ grpc-max-send-msg-size = 40960
require.Equal(t, uint(2048), conf.Status.GRPCConcurrentStreams)
require.Equal(t, 10240, conf.Status.GRPCInitialWindowSize)
require.Equal(t, 40960, conf.Status.GRPCMaxSendMsgSize)
require.True(t, conf.Performance.EnableLoadFMSketch)
err = f.Truncate(0)
require.NoError(t, err)

View File

@ -19,6 +19,7 @@ import (
"strconv"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
@ -384,6 +385,7 @@ func (h *Handle) initStatsBuckets(cache *statsCache) error {
// InitStats will init the stats cache using full load strategy.
func (h *Handle) InitStats(is infoschema.InfoSchema) (err error) {
loadFMSketch := config.GetGlobalConfig().Performance.EnableLoadFMSketch
h.mu.Lock()
defer func() {
_, err1 := h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), "commit")
@ -408,9 +410,11 @@ func (h *Handle) InitStats(is infoschema.InfoSchema) (err error) {
if err != nil {
return err
}
err = h.initStatsFMSketch(&cache)
if err != nil {
return err
if loadFMSketch {
err = h.initStatsFMSketch(&cache)
if err != nil {
return err
}
}
err = h.initStatsBuckets(&cache)
if err != nil {

View File

@ -644,6 +644,7 @@ func (h *Handle) LoadNeededHistograms() (err error) {
err = err1
}
}()
loadFMSketch := config.GetGlobalConfig().Performance.EnableLoadFMSketch
for _, col := range cols {
oldCache := h.statsCache.Load().(statsCache)
@ -664,9 +665,12 @@ func (h *Handle) LoadNeededHistograms() (err error) {
if err != nil {
return errors.Trace(err)
}
fms, err := h.fmSketchFromStorage(reader, col.TableID, 0, col.ColumnID)
if err != nil {
return errors.Trace(err)
var fms *statistics.FMSketch
if loadFMSketch {
fms, err = h.fmSketchFromStorage(reader, col.TableID, 0, col.ColumnID)
if err != nil {
return 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 {

View File

@ -20,6 +20,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
@ -249,6 +250,7 @@ func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column,
failpoint.Return(nil, errors.New("gofail ReadStatsForOne error"))
}
})
loadFMSketch := config.GetGlobalConfig().Performance.EnableLoadFMSketch
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)
@ -257,9 +259,12 @@ func (h *Handle) readStatsForOne(col model.TableColumnID, c *statistics.Column,
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)
var fms *statistics.FMSketch
if loadFMSketch {
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 {