Skip to content

Commit

Permalink
*: Revert "#19008" and "#18788" (#21599)
Browse files Browse the repository at this point in the history
  • Loading branch information
XuHuaiyu authored Dec 14, 2020
1 parent 49b926e commit 5acff8d
Show file tree
Hide file tree
Showing 27 changed files with 213 additions and 941 deletions.
7 changes: 0 additions & 7 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,6 @@ const (
DefIndexLimit = 64
// DefMaxOfIndexLimit is the maximum limitation of index on a single table for TiDB.
DefMaxOfIndexLimit = 64 * 8
// DefMinQuotaStatistics is the minimum statistic memory quota(in bytes).
DefMinQuotaStatistics = 32 << 30
// DefPort is the default port of TiDB
DefPort = 4000
// DefStatusPort is the default status port of TiDB
Expand Down Expand Up @@ -105,7 +103,6 @@ type Config struct {
TempStoragePath string `toml:"tmp-storage-path" json:"tmp-storage-path"`
OOMAction string `toml:"oom-action" json:"oom-action"`
MemQuotaQuery int64 `toml:"mem-quota-query" json:"mem-quota-query"`
MemQuotaStatistics int64 `toml:"mem-quota-statistics" json:"mem-quota-statistics"`
NestedLoopJoinCacheCapacity int64 `toml:"nested-loop-join-cache-capacity" json:"nested-loop-join-cache-capacity"`
// TempStorageQuota describe the temporary storage Quota during query exector when OOMUseTmpStorage is enabled
// If the quota exceed the capacity of the TempStoragePath, the tidb-server would exit with fatal error
Expand Down Expand Up @@ -644,7 +641,6 @@ var defaultConf = Config{
TempStoragePath: tempStorageDirName,
OOMAction: OOMActionCancel,
MemQuotaQuery: 1 << 30,
MemQuotaStatistics: 32 << 30,
NestedLoopJoinCacheCapacity: 20971520,
EnableStreaming: false,
EnableBatchDML: false,
Expand Down Expand Up @@ -1002,9 +998,6 @@ func (c *Config) Valid() error {
if c.PreparedPlanCache.MemoryGuardRatio < 0 || c.PreparedPlanCache.MemoryGuardRatio > 1 {
return fmt.Errorf("memory-guard-ratio in [prepared-plan-cache] must be NOT less than 0 and more than 1")
}
if c.MemQuotaStatistics < DefMinQuotaStatistics {
return fmt.Errorf("memory-quota-statistics should be greater than %dB", DefMinQuotaStatistics)
}
if len(c.IsolationRead.Engines) < 1 {
return fmt.Errorf("the number of [isolation-read]engines for isolation read should be at least 1")
}
Expand Down
4 changes: 0 additions & 4 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -34,10 +34,6 @@ token-limit = 1000
# The maximum memory available for a single SQL statement. Default: 1GB
mem-quota-query = 1073741824

# The maximum memory limitation for statistics. Default: 32GB
# This value must not be less than 32GB.
mem-quota-statistics = 34359738368

# The maximum number available of a NLJ cache for a single SQL statement. Default: 20MB
nested-loop-join-cache-capacity = 20971520

Expand Down
2 changes: 0 additions & 2 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,6 @@ server-version = "test_version"
repair-mode = true
max-server-connections = 200
mem-quota-query = 10000
mem-quota-statistics = 10000
nested-loop-join-cache-capacity = 100
max-index-length = 3080
index-limit = 70
Expand Down Expand Up @@ -261,7 +260,6 @@ spilled-file-encryption-method = "plaintext"
c.Assert(conf.RepairMode, Equals, true)
c.Assert(conf.MaxServerConnections, Equals, uint32(200))
c.Assert(conf.MemQuotaQuery, Equals, int64(10000))
c.Assert(conf.MemQuotaStatistics, Equals, int64(10000))
c.Assert(conf.NestedLoopJoinCacheCapacity, Equals, int64(100))
c.Assert(conf.IsolationRead.Engines, DeepEquals, []string{"tiflash"})
c.Assert(conf.MaxIndexLength, Equals, 3080)
Expand Down
3 changes: 1 addition & 2 deletions config/config_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@ func (s *testConfigSuite) TestMergeConfigItems(c *C) {
newConf.Performance.PseudoEstimateRatio = 123
newConf.OOMAction = "panic"
newConf.MemQuotaQuery = 123
newConf.MemQuotaStatistics = 123
newConf.TiKVClient.StoreLimit = 123

// rejected
Expand All @@ -67,7 +66,7 @@ func (s *testConfigSuite) TestMergeConfigItems(c *C) {

as, rs := MergeConfigItems(oldConf, newConf)
c.Assert(len(as), Equals, 10)
c.Assert(len(rs), Equals, 4)
c.Assert(len(rs), Equals, 3)
for _, a := range as {
_, ok := dynamicConfigItems[a]
c.Assert(ok, IsTrue)
Expand Down
4 changes: 2 additions & 2 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,14 +197,14 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) {
c.Assert(width, Equals, int32(4))

// Test very large cmsketch
tk.MustExec(fmt.Sprintf("analyze table t with %d cmsketch width, %d cmsketch depth", statistics.CMSketchSizeLimit, 1))
tk.MustExec(fmt.Sprintf("analyze table t with %d cmsketch width, %d cmsketch depth", core.CMSketchSizeLimit, 1))
tbl = s.dom.StatsHandle().GetTableStats(tableInfo)
col = tbl.Columns[1]
c.Assert(col.Len(), Equals, 20)
c.Assert(len(col.TopN.TopN), Equals, 1)
width, depth = col.CMSketch.GetWidthAndDepth()
c.Assert(depth, Equals, int32(1))
c.Assert(width, Equals, int32(statistics.CMSketchSizeLimit))
c.Assert(width, Equals, int32(core.CMSketchSizeLimit))

// Test very large cmsketch
tk.MustExec("analyze table t with 20480 cmsketch width, 50 cmsketch depth")
Expand Down
4 changes: 2 additions & 2 deletions executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ func (s *testInfoschemaTableSerialSuite) TestDataForTableStatsField(c *C) {
defer func() { executor.TableStatsCacheExpiry = oldExpiryTime }()
do := s.dom
h := do.StatsHandle()
h.Clear4Test()
h.Clear()
is := do.InfoSchema()
tk := testkit.NewTestKit(c, s.store)

Expand Down Expand Up @@ -414,7 +414,7 @@ func (s *testInfoschemaTableSerialSuite) TestPartitionsTable(c *C) {
defer func() { executor.TableStatsCacheExpiry = oldExpiryTime }()
do := s.dom
h := do.StatsHandle()
h.Clear4Test()
h.Clear()
is := do.InfoSchema()

tk := testkit.NewTestKit(c, s.store)
Expand Down
2 changes: 1 addition & 1 deletion executor/simple_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,7 +600,7 @@ func (s *testSuite3) TestDropStats(c *C) {
c.Assert(err, IsNil)
tableInfo := tbl.Meta()
h := do.StatsHandle()
h.Clear4Test()
h.Clear()
testKit.MustExec("analyze table t")
statsTbl := h.GetTableStats(tableInfo)
c.Assert(statsTbl.Pseudo, IsFalse)
Expand Down
4 changes: 2 additions & 2 deletions planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,7 +496,7 @@ func (s *testAnalyzeSuite) TestNullCount(c *C) {
testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...))
}
h := dom.StatsHandle()
h.Clear4Test()
h.Clear()
c.Assert(h.Update(dom.InfoSchema()), IsNil)
for i := 2; i < 4; i++ {
s.testData.OnRecord(func() {
Expand Down Expand Up @@ -552,7 +552,7 @@ func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) {
tk.MustExec("analyze table t with 2 buckets")
// Force using the histogram to estimate.
tk.MustExec("update mysql.stats_histograms set stats_ver = 0")
dom.StatsHandle().Clear4Test()
dom.StatsHandle().Clear()
dom.StatsHandle().Update(dom.InfoSchema())
// Using the histogram (a, b) to estimate `a = 5` will get 1.22, while using the CM Sketch to estimate
// the `a = 5 and c = 5` will get 10, it is not consistent.
Expand Down
2 changes: 1 addition & 1 deletion planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1246,7 +1246,7 @@ func getColumnRangeCounts(sc *stmtctx.StatementContext, colID int64, ranges []*r
for i, ran := range ranges {
if idxID >= 0 {
idxHist := histColl.Indices[idxID]
if idxHist == nil || idxHist.IsInvalid(sc, false) {
if idxHist == nil || idxHist.IsInvalid(false) {
return nil, false
}
count, err = histColl.GetRowCountByIndexRanges(sc, idxID, []*ranger.Range{ran})
Expand Down
38 changes: 29 additions & 9 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package core
import (
"bytes"
"context"
"encoding/binary"
"fmt"
"strings"
"time"
Expand All @@ -41,7 +42,7 @@ import (
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/parser_driver"
driver "github.com/pingcap/tidb/types/parser_driver"
util2 "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
Expand Down Expand Up @@ -1705,25 +1706,44 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as
return p, nil
}

// CMSketchSizeLimit indicates the size limit of CMSketch.
var CMSketchSizeLimit = kv.TxnEntrySizeLimit / binary.MaxVarintLen32

var analyzeOptionLimit = map[ast.AnalyzeOptionType]uint64{
ast.AnalyzeOptNumBuckets: 1024,
ast.AnalyzeOptNumTopN: 1024,
ast.AnalyzeOptCMSketchWidth: CMSketchSizeLimit,
ast.AnalyzeOptCMSketchDepth: CMSketchSizeLimit,
ast.AnalyzeOptNumSamples: 100000,
}

var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{
ast.AnalyzeOptNumBuckets: 256,
ast.AnalyzeOptNumTopN: 20,
ast.AnalyzeOptCMSketchWidth: 2048,
ast.AnalyzeOptCMSketchDepth: 5,
ast.AnalyzeOptNumSamples: 10000,
}

func handleAnalyzeOptions(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uint64, error) {
optMap := make(map[ast.AnalyzeOptionType]uint64, len(statistics.AnalyzeOptionDefault))
for key, val := range statistics.AnalyzeOptionDefault {
optMap := make(map[ast.AnalyzeOptionType]uint64, len(analyzeOptionDefault))
for key, val := range analyzeOptionDefault {
optMap[key] = val
}
for _, opt := range opts {
if opt.Type == ast.AnalyzeOptNumTopN {
if opt.Value > statistics.AnalyzeOptionLimit[opt.Type] {
return nil, errors.Errorf("value of analyze option %s should not larger than %d", ast.AnalyzeOptionString[opt.Type], statistics.AnalyzeOptionLimit[opt.Type])
if opt.Value > analyzeOptionLimit[opt.Type] {
return nil, errors.Errorf("value of analyze option %s should not larger than %d", ast.AnalyzeOptionString[opt.Type], analyzeOptionLimit[opt.Type])
}
} else {
if opt.Value == 0 || opt.Value > statistics.AnalyzeOptionLimit[opt.Type] {
return nil, errors.Errorf("value of analyze option %s should be positive and not larger than %d", ast.AnalyzeOptionString[opt.Type], statistics.AnalyzeOptionLimit[opt.Type])
if opt.Value == 0 || opt.Value > analyzeOptionLimit[opt.Type] {
return nil, errors.Errorf("value of analyze option %s should be positive and not larger than %d", ast.AnalyzeOptionString[opt.Type], analyzeOptionLimit[opt.Type])
}
}
optMap[opt.Type] = opt.Value
}
if optMap[ast.AnalyzeOptCMSketchWidth]*optMap[ast.AnalyzeOptCMSketchDepth] > statistics.CMSketchSizeLimit {
return nil, errors.Errorf("cm sketch size(depth * width) should not larger than %d", statistics.CMSketchSizeLimit)
if optMap[ast.AnalyzeOptCMSketchWidth]*optMap[ast.AnalyzeOptCMSketchDepth] > CMSketchSizeLimit {
return nil, errors.Errorf("cm sketch size(depth * width) should not larger than %d", CMSketchSizeLimit)
}
return optMap, nil
}
Expand Down
6 changes: 1 addition & 5 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -937,7 +937,6 @@ func NewSessionVars() *SessionVars {
}
vars.MemQuota = MemQuota{
MemQuotaQuery: config.GetGlobalConfig().MemQuotaQuery,
MemQuotaStatistics: config.GetGlobalConfig().MemQuotaStatistics,
NestedLoopJoinCacheCapacity: config.GetGlobalConfig().NestedLoopJoinCacheCapacity,

// The variables below do not take any effect anymore, it's remaining for compatibility.
Expand Down Expand Up @@ -1419,8 +1418,6 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.InitChunkSize = tidbOptPositiveInt32(val, DefInitChunkSize)
case TIDBMemQuotaQuery:
s.MemQuotaQuery = tidbOptInt64(val, config.GetGlobalConfig().MemQuotaQuery)
case TIDBMemQuotaStatistics:
s.MemQuotaStatistics = tidbOptInt64(val, config.GetGlobalConfig().MemQuotaStatistics)
case TIDBNestedLoopJoinCacheCapacity:
s.NestedLoopJoinCacheCapacity = tidbOptInt64(val, config.GetGlobalConfig().NestedLoopJoinCacheCapacity)
case TIDBMemQuotaHashJoin:
Expand Down Expand Up @@ -1936,8 +1933,7 @@ func (c *Concurrency) UnionConcurrency() int {
type MemQuota struct {
// MemQuotaQuery defines the memory quota for a query.
MemQuotaQuery int64
// MemQuotaStatistics defines the memory quota for the statistic Cache.
MemQuotaStatistics int64

// NestedLoopJoinCacheCapacity defines the memory capacity for apply cache.
NestedLoopJoinCacheCapacity int64

Expand Down
1 change: 0 additions & 1 deletion sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -999,7 +999,6 @@ var defaultSysVars = []*SysVar{
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: BoolOff, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: BoolOff, Type: TypeBool},
{Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64},
{Scope: ScopeGlobal, Name: TIDBMemQuotaStatistics, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaStatistics, 10), Type: TypeInt, MinValue: int64(32 << 30), MaxValue: math.MaxInt64},
{Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64},
{Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64},
{Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64},
Expand Down
1 change: 0 additions & 1 deletion sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,6 @@ const (
// The following session variables controls the memory quota during query execution.
// "tidb_mem_quota_query": control the memory quota of a query.
TIDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes.
TIDBMemQuotaStatistics = "tidb_mem_quota_statistics"
TIDBNestedLoopJoinCacheCapacity = "tidb_nested_loop_join_cache_capacity"
// TODO: remove them below sometime, it should have only one Quota(TIDBMemQuotaQuery).
TIDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin" // Bytes.
Expand Down
1 change: 0 additions & 1 deletion sessionctx/variable/varsutil_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,6 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) {
c.Assert(vars.MaxChunkSize, Equals, DefMaxChunkSize)
c.Assert(vars.DMLBatchSize, Equals, DefDMLBatchSize)
c.Assert(vars.MemQuotaQuery, Equals, config.GetGlobalConfig().MemQuotaQuery)
c.Assert(vars.MemQuotaStatistics, Equals, config.GetGlobalConfig().MemQuotaStatistics)
c.Assert(vars.MemQuotaHashJoin, Equals, int64(DefTiDBMemQuotaHashJoin))
c.Assert(vars.MemQuotaMergeJoin, Equals, int64(DefTiDBMemQuotaMergeJoin))
c.Assert(vars.MemQuotaSort, Equals, int64(DefTiDBMemQuotaSort))
Expand Down
Loading

0 comments on commit 5acff8d

Please sign in to comment.