diff --git a/errors.toml b/errors.toml index ce6ce2e582..1c6903d190 100644 --- a/errors.toml +++ b/errors.toml @@ -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 diff --git a/executor/set_test.go b/executor/set_test.go index b258cec1ca..c4dcc1b608 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -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) diff --git a/session/session_test.go b/session/session_test.go index 1328fdbce2..61e90f6f4b 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -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) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 93b3c1c9ce..fe3af2d336 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -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}, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index cd076bc50f..772882153a 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -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) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9a51c17e2c..ff8650317d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -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. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 33400361d1..2a5fd6360c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -705,6 +705,7 @@ const ( DefTiDBTopSQLMaxStatementCount = 200 DefTiDBTopSQLReportIntervalSeconds = 60 DefTiDBEnableGlobalTemporaryTable = false + DefTMPTableSize = 16777216 ) // Process global variables. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 78a5b5c492..41f379a9d6 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -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)) diff --git a/table/table.go b/table/table.go index b39304adca..f76210a9bd 100644 --- a/table/table.go +++ b/table/table.go @@ -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. diff --git a/table/tables/tables.go b/table/tables/tables.go index 003e2aa408..c78d1909cf 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -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 +} diff --git a/util/tableutil/tableutil.go b/util/tableutil/tableutil.go index 11cbe626dc..bf5d7caac2 100644 --- a/util/tableutil/tableutil.go +++ b/util/tableutil/tableutil.go @@ -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.