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:
@ -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: "",
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
Reference in New Issue
Block a user