Skip to content

Commit

Permalink
sessionctx/variable: Remove "advanced" memory sysvars (pingcap#32724)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored Mar 9, 2022
1 parent 8ab1f65 commit 0cf2153
Show file tree
Hide file tree
Showing 8 changed files with 23 additions and 119 deletions.
27 changes: 23 additions & 4 deletions sessionctx/variable/removed.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,30 @@ package variable
// This helps ensure some compatibility for applications while being
// careful not to return dummy data.

const (
tiDBEnableAlterPlacement = "tidb_enable_alter_placement"
tiDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin"
tiDBMemQuotaMergeJoin = "tidb_mem_quota_mergejoin"
tiDBMemQuotaSort = "tidb_mem_quota_sort"
tiDBMemQuotaTopn = "tidb_mem_quota_topn"
tiDBMemQuotaIndexLookupReader = "tidb_mem_quota_indexlookupreader"
tiDBMemQuotaIndexLookupJoin = "tidb_mem_quota_indexlookupjoin"
tiDBEnableGlobalTemporaryTable = "tidb_enable_global_temporary_table"
tiDBSlowLogMasking = "tidb_slow_log_masking"
placementChecks = "placement_checks"
)

var removedSysVars = map[string]string{
TiDBEnableAlterPlacement: "alter placement is now always enabled",
TiDBEnableGlobalTemporaryTable: "temporary table support is now always enabled",
TiDBSlowLogMasking: "use tidb_redact_log instead",
PlacementChecks: "placement_checks is removed and use tidb_placement_mode instead",
tiDBEnableAlterPlacement: "alter placement is now always enabled",
tiDBEnableGlobalTemporaryTable: "temporary table support is now always enabled",
tiDBSlowLogMasking: "use tidb_redact_log instead",
placementChecks: "placement_checks is removed and use tidb_placement_mode instead",
tiDBMemQuotaHashJoin: "use tidb_mem_quota_query instead",
tiDBMemQuotaMergeJoin: "use tidb_mem_quota_query instead",
tiDBMemQuotaSort: "use tidb_mem_quota_query instead",
tiDBMemQuotaTopn: "use tidb_mem_quota_query instead",
tiDBMemQuotaIndexLookupReader: "use tidb_mem_quota_query instead",
tiDBMemQuotaIndexLookupJoin: "use tidb_mem_quota_query instead",
}

// IsRemovedSysVar returns true if the sysvar has been removed
Expand Down
30 changes: 0 additions & 30 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1265,16 +1265,6 @@ func NewSessionVars() *SessionVars {
vars.MemQuota = MemQuota{
MemQuotaQuery: config.GetGlobalConfig().MemQuotaQuery,
MemQuotaApplyCache: DefTiDBMemQuotaApplyCache,

// The variables below do not take any effect anymore, it's remaining for compatibility.
// TODO: remove them in v4.1
MemQuotaHashJoin: DefTiDBMemQuotaHashJoin,
MemQuotaMergeJoin: DefTiDBMemQuotaMergeJoin,
MemQuotaSort: DefTiDBMemQuotaSort,
MemQuotaTopn: DefTiDBMemQuotaTopn,
MemQuotaIndexLookupReader: DefTiDBMemQuotaIndexLookupReader,
MemQuotaIndexLookupJoin: DefTiDBMemQuotaIndexLookupJoin,
MemQuotaDistSQL: DefTiDBMemQuotaDistSQL,
}
vars.BatchSize = BatchSize{
IndexJoinBatchSize: DefIndexJoinBatchSize,
Expand Down Expand Up @@ -1548,9 +1538,6 @@ func (s *SessionVars) GetSystemVar(name string) (string, bool) {
} else if name == ErrorCount {
return strconv.Itoa(int(s.SysErrorCount)), true
}
if name == TiDBSlowLogMasking {
name = TiDBRedactLog
}
if val, ok := s.stmtVars[name]; ok {
return val, ok
}
Expand Down Expand Up @@ -1900,23 +1887,6 @@ type MemQuota struct {
MemQuotaQuery int64
// MemQuotaApplyCache defines the memory capacity for apply cache.
MemQuotaApplyCache int64

// The variables below do not take any effect anymore, it's remaining for compatibility.
// TODO: remove them in v4.1
// MemQuotaHashJoin defines the memory quota for a hash join executor.
MemQuotaHashJoin int64
// MemQuotaMergeJoin defines the memory quota for a merge join executor.
MemQuotaMergeJoin int64
// MemQuotaSort defines the memory quota for a sort executor.
MemQuotaSort int64
// MemQuotaTopn defines the memory quota for a top n executor.
MemQuotaTopn int64
// MemQuotaIndexLookupReader defines the memory quota for a index lookup reader executor.
MemQuotaIndexLookupReader int64
// MemQuotaIndexLookupJoin defines the memory quota for a index lookup join executor.
MemQuotaIndexLookupJoin int64
// MemQuotaDistSQL defines the memory quota for all operators in DistSQL layer like co-processor and selectResult.
MemQuotaDistSQL int64
}

// BatchSize defines batch size values.
Expand Down
6 changes: 0 additions & 6 deletions sessionctx/variable/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,6 @@ func TestSetSystemVariable(t *testing.T) {
{variable.TiDBOptAggPushDown, "1", false},
{variable.TiDBOptDistinctAggPushDown, "1", false},
{variable.TiDBMemQuotaQuery, "1024", false},
{variable.TiDBMemQuotaHashJoin, "1024", false},
{variable.TiDBMemQuotaMergeJoin, "1024", false},
{variable.TiDBMemQuotaSort, "1024", false},
{variable.TiDBMemQuotaTopn, "1024", false},
{variable.TiDBMemQuotaIndexLookupReader, "1024", false},
{variable.TiDBMemQuotaIndexLookupJoin, "1024", false},
{variable.TiDBMemQuotaApplyCache, "1024", false},
{variable.TiDBEnableStmtSummary, "1", true}, // now global only
}
Expand Down
45 changes: 0 additions & 45 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -542,48 +542,6 @@ var defaultSysVars = []*SysVar{
s.MemQuotaQuery = TidbOptInt64(val, config.GetGlobalConfig().MemQuotaQuery)
return nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaHashJoin = TidbOptInt64(val, DefTiDBMemQuotaHashJoin)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaHashJoin, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaMergeJoin = TidbOptInt64(val, DefTiDBMemQuotaMergeJoin)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaMergeJoin, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaSort = TidbOptInt64(val, DefTiDBMemQuotaSort)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaSort, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaTopn = TidbOptInt64(val, DefTiDBMemQuotaTopn)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaTopn, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaIndexLookupReader = TidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaIndexLookupReader, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
{Scope: ScopeSession, Name: TiDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.MemQuotaIndexLookupJoin = TidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin)
return nil
}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
appendDeprecationWarning(vars, TiDBMemQuotaIndexLookupJoin, TiDBMemQuotaQuery)
return normalizedValue, nil
}},
// Deprecated: tidb_enable_streaming
{Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, skipInit: true, Hidden: true, SetSession: func(s *SessionVars, val string) error {
s.EnableStreaming = TiDBOptOn(val)
Expand Down Expand Up @@ -1499,9 +1457,6 @@ const (
SkipNameResolve = "skip_name_resolve"
// ForeignKeyChecks is the name for 'foreign_key_checks' system variable.
ForeignKeyChecks = "foreign_key_checks"
// PlacementChecks is the name for 'placement_checks' system variable.
// Deprecated: use tidb_placement_mode instead
PlacementChecks = "placement_checks"
// SQLSafeUpdates is the name for 'sql_safe_updates' system variable.
SQLSafeUpdates = "sql_safe_updates"
// WarningCount is the name for 'warning_count' system variable.
Expand Down
26 changes: 0 additions & 26 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,15 +115,6 @@ const (
TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes.
TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache"

// TODO: remove them below sometime, it should have only one Quota(TiDBMemQuotaQuery).

TiDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin" // Bytes.
TiDBMemQuotaMergeJoin = "tidb_mem_quota_mergejoin" // Bytes.
TiDBMemQuotaSort = "tidb_mem_quota_sort" // Bytes.
TiDBMemQuotaTopn = "tidb_mem_quota_topn" // Bytes.
TiDBMemQuotaIndexLookupReader = "tidb_mem_quota_indexlookupreader" // Bytes.
TiDBMemQuotaIndexLookupJoin = "tidb_mem_quota_indexlookupjoin" // Bytes.

// TiDBGeneralLog is used to log every query in the server in info level.
TiDBGeneralLog = "tidb_general_log"

Expand Down Expand Up @@ -419,10 +410,6 @@ const (
// TiDBEnablePointGetCache is used to control whether to enable the point get cache for special scenario.
TiDBEnablePointGetCache = "tidb_enable_point_get_cache"

// TiDBEnableAlterPlacement is used to control whether to enable alter table partition.
// Deprecated: It is removed and do not use it again
TiDBEnableAlterPlacement = "tidb_enable_alter_placement"

// TiDBPlacementMode is used to control the mode for placement
TiDBPlacementMode = "tidb_placement_mode"

Expand Down Expand Up @@ -533,10 +520,6 @@ const (
// TiDBPartitionPruneMode indicates the partition prune mode used.
TiDBPartitionPruneMode = "tidb_partition_prune_mode"

// TiDBSlowLogMasking is deprecated and a alias of TiDBRedactLog.
// Deprecated: use TiDBRedactLog instead.
TiDBSlowLogMasking = "tidb_slow_log_masking"

// TiDBRedactLog indicates that whether redact log.
TiDBRedactLog = "tidb_redact_log"

Expand Down Expand Up @@ -594,8 +577,6 @@ const (
// TiDBTopSQLMaxMetaCount indicates the max capacity of the collect meta per second.
TiDBTopSQLMaxMetaCount = "tidb_top_sql_max_meta_count"

// TiDBEnableGlobalTemporaryTable indicates whether to enable global temporary table
TiDBEnableGlobalTemporaryTable = "tidb_enable_global_temporary_table"
// TiDBEnableLocalTxn indicates whether to enable Local Txn.
TiDBEnableLocalTxn = "tidb_enable_local_txn"
// TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client.
Expand Down Expand Up @@ -718,13 +699,6 @@ const (
DefWaitTimeout = 28800
DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB.
DefTiDBMemQuotaBindCache = 64 << 20 // 64MB.
DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB.
DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB.
DefTiDBMemQuotaSort = 32 << 30 // 32GB.
DefTiDBMemQuotaTopn = 32 << 30 // 32GB.
DefTiDBMemQuotaIndexLookupReader = 32 << 30 // 32GB.
DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB.
DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB.
DefTiDBGeneralLog = false
DefTiDBPProfSQLCPU = 0
DefTiDBRetryLimit = 10
Expand Down
6 changes: 0 additions & 6 deletions sessionctx/variable/varsutil_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,12 +82,6 @@ func TestNewSessionVars(t *testing.T) {
require.Equal(t, DefMaxChunkSize, vars.MaxChunkSize)
require.Equal(t, DefDMLBatchSize, vars.DMLBatchSize)
require.Equal(t, config.GetGlobalConfig().MemQuotaQuery, vars.MemQuotaQuery)
require.Equal(t, int64(DefTiDBMemQuotaHashJoin), vars.MemQuotaHashJoin)
require.Equal(t, int64(DefTiDBMemQuotaMergeJoin), vars.MemQuotaMergeJoin)
require.Equal(t, int64(DefTiDBMemQuotaSort), vars.MemQuotaSort)
require.Equal(t, int64(DefTiDBMemQuotaTopn), vars.MemQuotaTopn)
require.Equal(t, int64(DefTiDBMemQuotaIndexLookupReader), vars.MemQuotaIndexLookupReader)
require.Equal(t, int64(DefTiDBMemQuotaIndexLookupJoin), vars.MemQuotaIndexLookupJoin)
require.Equal(t, int64(DefTiDBMemQuotaApplyCache), vars.MemQuotaApplyCache)
require.Equal(t, DefOptWriteRowID, vars.AllowWriteRowID)
require.Equal(t, DefTiDBOptJoinReorderThreshold, vars.TiDBOptJoinReorderThreshold)
Expand Down
1 change: 0 additions & 1 deletion util/sem/sem.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,6 @@ func IsInvisibleSysVar(varNameInLower string) bool {
variable.TiDBMemoryUsageAlarmRatio,
variable.TiDBRedactLog,
variable.TiDBRestrictedReadOnly,
variable.TiDBSlowLogMasking,
variable.TiDBTopSQLMaxTimeSeriesCount,
variable.TiDBTopSQLMaxMetaCount:
return true
Expand Down
1 change: 0 additions & 1 deletion util/sem/sem_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,6 @@ func TestIsInvisibleSysVar(t *testing.T) {
assert.True(IsInvisibleSysVar(variable.TiDBEnableTelemetry))
assert.True(IsInvisibleSysVar(variable.TiDBRowFormatVersion))
assert.True(IsInvisibleSysVar(variable.TiDBRedactLog))
assert.True(IsInvisibleSysVar(variable.TiDBSlowLogMasking))
assert.True(IsInvisibleSysVar(variable.TiDBTopSQLMaxTimeSeriesCount))
assert.True(IsInvisibleSysVar(variable.TiDBTopSQLMaxTimeSeriesCount))
}

0 comments on commit 0cf2153

Please sign in to comment.