*: add global system variable tmp_table_size (#24827)
This commit is contained in:
@ -1381,6 +1381,11 @@ error = '''
|
||||
Unknown column '%-.192s' in '%-.192s'
|
||||
'''
|
||||
|
||||
["table:1114"]
|
||||
error = '''
|
||||
The table '%-.192s' is full
|
||||
'''
|
||||
|
||||
["table:1192"]
|
||||
error = '''
|
||||
Can't execute the given command because you have active locked tables or an active transaction
|
||||
|
||||
@ -884,9 +884,9 @@ func (s *testSuite5) TestValidateSetVar(c *C) {
|
||||
result = tk.MustQuery("select @@tmp_table_size;")
|
||||
result.Check(testkit.Rows("167772161"))
|
||||
|
||||
tk.MustExec("set @@tmp_table_size=18446744073709551615")
|
||||
tk.MustExec("set @@tmp_table_size=9223372036854775807")
|
||||
result = tk.MustQuery("select @@tmp_table_size;")
|
||||
result.Check(testkit.Rows("18446744073709551615"))
|
||||
result.Check(testkit.Rows("9223372036854775807"))
|
||||
|
||||
_, err = tk.Exec("set @@tmp_table_size=18446744073709551616")
|
||||
c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue)
|
||||
|
||||
@ -37,6 +37,7 @@ import (
|
||||
"github.com/pingcap/tidb/config"
|
||||
"github.com/pingcap/tidb/ddl/placement"
|
||||
"github.com/pingcap/tidb/domain"
|
||||
"github.com/pingcap/tidb/errno"
|
||||
"github.com/pingcap/tidb/executor"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/meta/autoid"
|
||||
@ -4532,6 +4533,36 @@ func (s *testSessionSuite) TestInTxnPSProtoPointGet(c *C) {
|
||||
tk.MustExec("commit")
|
||||
}
|
||||
|
||||
func (s *testSessionSuite) TestTMPTableSize(c *C) {
|
||||
// Test the @@tmp_table_size system variable.
|
||||
tk := testkit.NewTestKit(c, s.store)
|
||||
tk.MustExec("use test")
|
||||
tk.MustExec("set tidb_enable_global_temporary_table=on")
|
||||
tk.MustExec("create global temporary table t (c1 int, c2 varchar(512)) on commit delete rows")
|
||||
|
||||
tk.MustQuery("select @@global.tmp_table_size").Check(testkit.Rows(strconv.Itoa(variable.DefTMPTableSize)))
|
||||
c.Assert(tk.Se.GetSessionVars().TMPTableSize, Equals, int64(variable.DefTMPTableSize))
|
||||
|
||||
// Min value 1024, so the result is change to 1024, with a warning.
|
||||
tk.MustExec("set @@global.tmp_table_size = 123")
|
||||
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tmp_table_size value: '123'"))
|
||||
|
||||
// Change the session scope value.
|
||||
tk.MustExec("set @@session.tmp_table_size = 2097152")
|
||||
c.Assert(tk.Se.GetSessionVars().TMPTableSize, Equals, int64(2097152))
|
||||
|
||||
// Check in another sessin, change session scope value does not affect the global scope.
|
||||
tk1 := testkit.NewTestKit(c, s.store)
|
||||
tk1.MustQuery("select @@global.tmp_table_size").Check(testkit.Rows("1024"))
|
||||
|
||||
// The value is now 1024, check the error when table size exceed it.
|
||||
tk.MustExec("set @@session.tmp_table_size = 1024")
|
||||
tk.MustExec("begin")
|
||||
tk.MustExec("insert into t values (1, repeat('x', 512))")
|
||||
tk.MustExec("insert into t values (1, repeat('x', 512))")
|
||||
tk.MustGetErrCode("insert into t values (1, repeat('x', 512))", errno.ErrRecordFileFull)
|
||||
}
|
||||
|
||||
func (s *testSessionSuite) TestTiDBEnableGlobalTemporaryTable(c *C) {
|
||||
// Test the @@tidb_enable_global_temporary_table system variable.
|
||||
tk := testkit.NewTestKit(c, s.store)
|
||||
|
||||
@ -83,7 +83,6 @@ var noopSysVars = []*SysVar{
|
||||
{Scope: ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"},
|
||||
{Scope: ScopeGlobal, Name: "server_id", Value: "0"},
|
||||
{Scope: ScopeGlobal, Name: "innodb_flushing_avg_loops", Value: "30"},
|
||||
{Scope: ScopeGlobal | ScopeSession, Name: TmpTableSize, Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true, IsHintUpdatable: true},
|
||||
{Scope: ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"},
|
||||
{Scope: ScopeGlobal | ScopeSession, Name: "preload_buffer_size", Value: "32768"},
|
||||
{Scope: ScopeGlobal, Name: CheckProxyUsers, Value: Off, Type: TypeBool},
|
||||
|
||||
@ -836,6 +836,12 @@ type SessionVars struct {
|
||||
// see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth
|
||||
CTEMaxRecursionDepth int
|
||||
|
||||
// The temporary table size threshold
|
||||
// In MySQL, when a temporary table exceed this size, it spills to disk.
|
||||
// In TiDB, as we do not support spill to disk for now, an error is reported.
|
||||
// See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_tmp_table_size
|
||||
TMPTableSize int64
|
||||
|
||||
// EnableGlobalTemporaryTable indicates whether to enable global temporary table
|
||||
EnableGlobalTemporaryTable bool
|
||||
}
|
||||
@ -1041,6 +1047,7 @@ func NewSessionVars() *SessionVars {
|
||||
EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin,
|
||||
AllowFallbackToTiKV: make(map[kv.StoreType]struct{}),
|
||||
CTEMaxRecursionDepth: DefCTEMaxRecursionDepth,
|
||||
TMPTableSize: DefTMPTableSize,
|
||||
EnableGlobalTemporaryTable: DefTiDBEnableGlobalTemporaryTable,
|
||||
}
|
||||
vars.KVVars = tikvstore.NewVariables(&vars.Killed)
|
||||
|
||||
@ -1670,8 +1670,14 @@ var defaultSysVars = []*SysVar{
|
||||
{Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64},
|
||||
{Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64},
|
||||
{Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true},
|
||||
{Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "PHYSICAL", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}},
|
||||
{Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "LEGACY", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}},
|
||||
|
||||
// See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_tmp_table_size
|
||||
{Scope: ScopeGlobal | ScopeSession, Name: TMPTableSize, Value: strconv.Itoa(DefTMPTableSize), Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, IsHintUpdatable: true, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error {
|
||||
s.TMPTableSize = tidbOptInt64(val, DefTMPTableSize)
|
||||
return nil
|
||||
}},
|
||||
// variable for top SQL feature.
|
||||
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) {
|
||||
return BoolToOnOff(TopSQLVariable.Enable.Load()), nil
|
||||
@ -1854,8 +1860,8 @@ const (
|
||||
MaxConnectErrors = "max_connect_errors"
|
||||
// TableDefinitionCache is the name for 'table_definition_cache' system variable.
|
||||
TableDefinitionCache = "table_definition_cache"
|
||||
// TmpTableSize is the name for 'tmp_table_size' system variable.
|
||||
TmpTableSize = "tmp_table_size"
|
||||
// TMPTableSize is the name for 'tmp_table_size' system variable.
|
||||
TMPTableSize = "tmp_table_size"
|
||||
// Timestamp is the name for 'timestamp' system variable.
|
||||
Timestamp = "timestamp"
|
||||
// ConnectTimeout is the name for 'connect_timeout' system variable.
|
||||
|
||||
@ -705,6 +705,7 @@ const (
|
||||
DefTiDBTopSQLMaxStatementCount = 200
|
||||
DefTiDBTopSQLReportIntervalSeconds = 60
|
||||
DefTiDBEnableGlobalTemporaryTable = false
|
||||
DefTMPTableSize = 16777216
|
||||
)
|
||||
|
||||
// Process global variables.
|
||||
|
||||
@ -104,6 +104,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) {
|
||||
c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep))
|
||||
c.Assert(vars.AnalyzeVersion, Equals, DefTiDBAnalyzeVersion)
|
||||
c.Assert(vars.CTEMaxRecursionDepth, Equals, DefCTEMaxRecursionDepth)
|
||||
c.Assert(vars.TMPTableSize, Equals, int64(DefTMPTableSize))
|
||||
|
||||
assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota))
|
||||
assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.BatchSize))
|
||||
|
||||
@ -97,6 +97,8 @@ var (
|
||||
ErrSequenceHasRunOut = dbterror.ClassTable.NewStd(mysql.ErrSequenceRunOut)
|
||||
// ErrRowDoesNotMatchGivenPartitionSet returns when the destination partition conflict with the partition selection.
|
||||
ErrRowDoesNotMatchGivenPartitionSet = dbterror.ClassTable.NewStd(mysql.ErrRowDoesNotMatchGivenPartitionSet)
|
||||
// ErrTempTableFull returns a table is full error, it's used by temporary table now.
|
||||
ErrTempTableFull = dbterror.ClassTable.NewStd(mysql.ErrRecordFileFull)
|
||||
)
|
||||
|
||||
// RecordIterFunc is used for low-level record iteration.
|
||||
|
||||
@ -324,7 +324,12 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context,
|
||||
defer memBuffer.Cleanup(sh)
|
||||
|
||||
if m := t.Meta(); m.TempTableType == model.TempTableGlobal {
|
||||
addTemporaryTable(sctx, m)
|
||||
if tmpTable := addTemporaryTable(sctx, m); tmpTable != nil {
|
||||
if tmpTable.GetSize() > sctx.GetSessionVars().TMPTableSize {
|
||||
return table.ErrTempTableFull.GenWithStackByArgs(m.Name.O)
|
||||
}
|
||||
defer handleTempTableSize(tmpTable, txn.Size(), txn)
|
||||
}
|
||||
}
|
||||
|
||||
var colIDs, binlogColIDs []int64
|
||||
@ -589,9 +594,20 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column {
|
||||
return pkCols
|
||||
}
|
||||
|
||||
func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) {
|
||||
func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) tableutil.TempTable {
|
||||
tempTable := sctx.GetSessionVars().GetTemporaryTable(tblInfo)
|
||||
tempTable.SetModified(true)
|
||||
return tempTable
|
||||
}
|
||||
|
||||
// The size of a temporary table is calculated by accumulating the transaction size delta.
|
||||
func handleTempTableSize(t tableutil.TempTable, txnSizeBefore int, txn kv.Transaction) {
|
||||
txnSizeNow := txn.Size()
|
||||
delta := txnSizeNow - txnSizeBefore
|
||||
|
||||
oldSize := t.GetSize()
|
||||
newSize := oldSize + int64(delta)
|
||||
t.SetSize(newSize)
|
||||
}
|
||||
|
||||
// AddRecord implements table.Table AddRecord interface.
|
||||
@ -607,7 +623,12 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts .
|
||||
}
|
||||
|
||||
if m := t.Meta(); m.TempTableType == model.TempTableGlobal {
|
||||
addTemporaryTable(sctx, m)
|
||||
if tmpTable := addTemporaryTable(sctx, m); tmpTable != nil {
|
||||
if tmpTable.GetSize() > sctx.GetSessionVars().TMPTableSize {
|
||||
return nil, table.ErrTempTableFull.GenWithStackByArgs(m.Name.O)
|
||||
}
|
||||
defer handleTempTableSize(tmpTable, txn.Size(), txn)
|
||||
}
|
||||
}
|
||||
|
||||
var ctx context.Context
|
||||
@ -1008,8 +1029,17 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type
|
||||
return err
|
||||
}
|
||||
|
||||
txn, err := ctx.Txn(true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if m := t.Meta(); m.TempTableType == model.TempTableGlobal {
|
||||
addTemporaryTable(ctx, m)
|
||||
if tmpTable := addTemporaryTable(ctx, m); tmpTable != nil {
|
||||
if tmpTable.GetSize() > ctx.GetSessionVars().TMPTableSize {
|
||||
return table.ErrTempTableFull.GenWithStackByArgs(m.Name.O)
|
||||
}
|
||||
defer handleTempTableSize(tmpTable, txn.Size(), txn)
|
||||
}
|
||||
}
|
||||
|
||||
// The table has non-public column and this column is doing the operation of "modify/change column".
|
||||
@ -1785,6 +1815,8 @@ type TemporaryTable struct {
|
||||
stats *statistics.Table
|
||||
// The autoID allocator of this table.
|
||||
autoIDAllocator autoid.Allocator
|
||||
// Table size.
|
||||
size int64
|
||||
}
|
||||
|
||||
// TempTableFromMeta builds a TempTable from model.TableInfo.
|
||||
@ -1815,3 +1847,13 @@ func (t *TemporaryTable) GetModified() bool {
|
||||
func (t *TemporaryTable) GetStats() interface{} {
|
||||
return t.stats
|
||||
}
|
||||
|
||||
// GetSize gets the table size.
|
||||
func (t *TemporaryTable) GetSize() int64 {
|
||||
return t.size
|
||||
}
|
||||
|
||||
// SetSize sets the table size.
|
||||
func (t *TemporaryTable) SetSize(v int64) {
|
||||
t.size = v
|
||||
}
|
||||
|
||||
@ -33,6 +33,9 @@ type TempTable interface {
|
||||
// The stats of this table (*statistics.Table).
|
||||
// Define the return type as interface{} here to avoid cycle imports.
|
||||
GetStats() interface{}
|
||||
|
||||
GetSize() int64
|
||||
SetSize(int64)
|
||||
}
|
||||
|
||||
// TempTableFromMeta builds a TempTable from *model.TableInfo.
|
||||
|
||||
Reference in New Issue
Block a user