From c8637a814138ac256357d9fb54fcc1149d8eaf95 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 16 Mar 2022 11:01:53 +0800 Subject: [PATCH 1/5] executor: set kvRanges in table_reader when batch_cop=2 for partition table (#33112) close pingcap/tidb#33111 --- executor/table_reader.go | 1 + executor/tiflash_test.go | 6 ++++++ store/mockstore/unistore/cophandler/mpp.go | 6 +++++- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/executor/table_reader.go b/executor/table_reader.go index adae20d401efa..3b20bf90c98d2 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -344,6 +344,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex } partitionIDAndRanges := make([]kv.PartitionIDAndRanges, 0, len(pids)) for i, kvRange := range kvRanges { + e.kvRanges = append(e.kvRanges, kvRange...) partitionIDAndRanges = append(partitionIDAndRanges, kv.PartitionIDAndRanges{ ID: pids[i], KeyRanges: kvRange, diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 13a3cc3bbf432..e70c717c0227a 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -109,6 +109,12 @@ func TestReadPartitionTable(t *testing.T) { tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("5")) tk.MustExec("insert into t values(6,0)") tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) + // test dynamic prune + union scan + tk.MustExec("set tidb_partition_prune_mode=dynamic") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) + // test dynamic prune + batch cop + union scan + tk.MustExec("set tidb_allow_batch_cop=2") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) tk.MustExec("commit") } diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index fcab223f170fa..2b13319e43f08 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -101,7 +101,11 @@ func (b *mppExecBuilder) buildMPPPartitionTableScan(pb *tipb.PartitionTableScan) kvRanges: ranges, dbReader: b.dbReader, } - for _, col := range pb.Columns { + for i, col := range pb.Columns { + if col.ColumnId == model.ExtraPhysTblID { + ts.physTblIDColIdx = new(int) + *ts.physTblIDColIdx = i + } ft := fieldTypeFromPBColumn(col) ts.fieldTypes = append(ts.fieldTypes, ft) } From d981c0e06a1397def2f4b3863aa532da5195a07d Mon Sep 17 00:00:00 2001 From: CbcWestwolf <1004626265@qq.com> Date: Wed, 16 Mar 2022 11:15:52 +0800 Subject: [PATCH 2/5] sysvar: add INSTANCE scope for system variable (#32888) ref pingcap/tidb#32887 --- errno/errcode.go | 1 + errno/errname.go | 1 + errors.toml | 5 +++ executor/errors.go | 1 + executor/set.go | 8 ++++ executor/set_test.go | 27 +++++++++--- planner/core/expression_rewriter.go | 2 +- session/session.go | 7 ++++ session/session_test.go | 51 +++++++++++++++++++++++ sessionctx/variable/session.go | 5 +++ sessionctx/variable/sysvar.go | 9 ++-- sessionctx/variable/sysvar_test.go | 64 ++++++++++++++++++++++++++++- sessionctx/variable/tidb_vars.go | 4 ++ sessionctx/variable/variable.go | 9 +++- 14 files changed, 183 insertions(+), 11 deletions(-) diff --git a/errno/errcode.go b/errno/errcode.go index e926b9ffe53df..e061f0c25a827 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1016,6 +1016,7 @@ const ( ErrInconsistentHandle = 8139 ErrInconsistentIndexedValue = 8140 ErrAssertionFailed = 8141 + ErrInstanceScope = 8142 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 diff --git a/errno/errname.go b/errno/errname.go index ab44d3bd18860..503578504b44c 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1014,6 +1014,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrInconsistentHandle: mysql.Message("writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v", []int{2, 3, 4, 5}), ErrInconsistentIndexedValue: mysql.Message("writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s}", []int{3, 4}), ErrAssertionFailed: mysql.Message("assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v", []int{0}), + ErrInstanceScope: mysql.Message("modifying %s will require SET GLOBAL in a future version of TiDB", nil), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index c1c504ac9c47f..43bffcb5ff584 100644 --- a/errors.toml +++ b/errors.toml @@ -1371,6 +1371,11 @@ error = ''' data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v ''' +["executor:8142"] +error = ''' +modifying %s will require SET GLOBAL in a future version of TiDB +''' + ["executor:8212"] error = ''' Failed to split region ranges: %s diff --git a/executor/errors.go b/executor/errors.go index 1e9c2b8969b00..b43496d407c4a 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -53,6 +53,7 @@ var ( ErrIllegalPrivilegeLevel = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalPrivilegeLevel) ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges) ErrViewInvalid = dbterror.ClassExecutor.NewStd(mysql.ErrViewInvalid) + ErrInstanceScope = dbterror.ClassExecutor.NewStd(mysql.ErrInstanceScope) ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) diff --git a/executor/set.go b/executor/set.go index 65e21d470cc78..a79055abb5dbe 100644 --- a/executor/set.go +++ b/executor/set.go @@ -115,6 +115,14 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres } return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } + + if sysVar.HasInstanceScope() && !v.IsGlobal && sessionVars.EnableLegacyInstanceScope { + // For backward compatibility we will change the v.IsGlobal to true, + // and append a warning saying this will not be supported in future. + v.IsGlobal = true + sessionVars.StmtCtx.AppendWarning(ErrInstanceScope.GenWithStackByArgs(sysVar.Name)) + } + if v.IsGlobal { valStr, err := e.getVarValue(v, sysVar) if err != nil { diff --git a/executor/set_test.go b/executor/set_test.go index 2cb9eec98102c..42cd1340f09c0 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -416,19 +416,19 @@ func TestSetVar(t *testing.T) { tk.MustQuery(`select @@global.tidb_enable_parallel_apply`).Check(testkit.Rows("0")) tk.MustQuery(`select @@tidb_enable_parallel_apply`).Check(testkit.Rows("1")) - tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0")) + tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0")) tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF")) tk.MustExec("set tidb_general_log = 1") - tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("1")) + tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("1")) tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log ON")) tk.MustExec("set tidb_general_log = 0") - tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0")) + tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0")) tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF")) tk.MustExec("set tidb_general_log = on") - tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("1")) + tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("1")) tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log ON")) tk.MustExec("set tidb_general_log = off") - tk.MustQuery(`select @@session.tidb_general_log;`).Check(testkit.Rows("0")) + tk.MustQuery(`select @@global.tidb_general_log;`).Check(testkit.Rows("0")) tk.MustQuery(`show variables like 'tidb_general_log';`).Check(testkit.Rows("tidb_general_log OFF")) require.Error(t, tk.ExecToErr("set tidb_general_log = abc")) require.Error(t, tk.ExecToErr("set tidb_general_log = 123")) @@ -803,6 +803,7 @@ func TestValidateSetVar(t *testing.T) { tk.MustQuery("select @@tidb_constraint_check_in_place;").Check(testkit.Rows("1")) tk.MustExec("set @@tidb_general_log=0;") + tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) tk.MustQuery("select @@tidb_general_log;").Check(testkit.Rows("0")) tk.MustExec("set @@tidb_pprof_sql_cpu=1;") @@ -1574,3 +1575,19 @@ func TestSetTopSQLVariables(t *testing.T) { tk.MustQuery("show variables like '%top_sql%'").Check(testkit.Rows("tidb_enable_top_sql OFF", "tidb_top_sql_max_meta_count 5000", "tidb_top_sql_max_time_series_count 20")) tk.MustQuery("show global variables like '%top_sql%'").Check(testkit.Rows("tidb_enable_top_sql OFF", "tidb_top_sql_max_meta_count 5000", "tidb_top_sql_max_time_series_count 20")) } + +func TestInstanceScopeSwitching(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // enable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 1") + tk.MustExec("set tidb_general_log = 1") + tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) + + // disable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 0") + tk.MustGetErrCode("set tidb_general_log = 1", errno.ErrGlobalVariable) +} diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 663fb89e4657d..427a0633aa9ba 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1291,7 +1291,7 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { er.b.visitInfo = appendDynamicVisitInfo(er.b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err) } if v.ExplicitScope && !sysVar.HasNoneScope() { - if v.IsGlobal && !sysVar.HasGlobalScope() { + if v.IsGlobal && !(sysVar.HasGlobalScope() || sysVar.HasInstanceScope()) { er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") return } diff --git a/session/session.go b/session/session.go index db3b933a713e8..2ea6c6bf1f5ca 100644 --- a/session/session.go +++ b/session/session.go @@ -1331,6 +1331,7 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { } // SetGlobalSysVar implements GlobalVarAccessor.SetGlobalSysVar interface. +// it is called (but skipped) when setting instance scope func (s *session) SetGlobalSysVar(name, value string) (err error) { sv := variable.GetSysVar(name) if sv == nil { @@ -1342,6 +1343,9 @@ func (s *session) SetGlobalSysVar(name, value string) (err error) { if err = sv.SetGlobalFromHook(s.sessionVars, value, false); err != nil { return err } + if sv.HasInstanceScope() { // skip for INSTANCE scope + return nil + } if sv.GlobalConfigName != "" { domain.GetDomain(s).NotifyGlobalConfigChange(sv.GlobalConfigName, variable.OnOffToTrueFalse(value)) } @@ -1358,6 +1362,9 @@ func (s *session) SetGlobalSysVarOnly(name, value string) (err error) { if err = sv.SetGlobalFromHook(s.sessionVars, value, true); err != nil { return err } + if sv.HasInstanceScope() { // skip for INSTANCE scope + return nil + } return s.replaceGlobalVariablesTableValue(context.TODO(), sv.Name, value) } diff --git a/session/session_test.go b/session/session_test.go index 72d60d328c7b7..e68f68c9ba252 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -764,6 +764,42 @@ func (s *testSessionSuite) TestUpgradeSysvars(c *C) { c.Assert(v, Equals, "OFF") // the default value is restored. } +func (s *testSessionSuite) TestSetInstanceSysvarBySetGlobalSysVar(c *C) { + varName := "tidb_general_log" + defaultValue := "OFF" // This is the default value for tidb_general_log + + tk := testkit.NewTestKitWithInit(c, s.store) + se := tk.Se.(variable.GlobalVarAccessor) + + // Get globalSysVar twice and get the same default value + v, err := se.GetGlobalSysVar(varName) + c.Assert(err, IsNil) + c.Assert(v, Equals, defaultValue) + v, err = se.GetGlobalSysVar(varName) + c.Assert(err, IsNil) + c.Assert(v, Equals, defaultValue) + + // session.GetGlobalSysVar would not get the value which session.SetGlobalSysVar writes, + // because SetGlobalSysVar calls SetGlobalFromHook, which uses TiDBGeneralLog's SetGlobal, + // but GetGlobalSysVar could not access TiDBGeneralLog's GetGlobal. + + // set to "1" + err = se.SetGlobalSysVar(varName, "ON") + c.Assert(err, IsNil) + v, err = se.GetGlobalSysVar(varName) + tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("1")) + c.Assert(err, IsNil) + c.Assert(v, Equals, defaultValue) + + // set back to "0" + err = se.SetGlobalSysVar(varName, defaultValue) + c.Assert(err, IsNil) + v, err = se.GetGlobalSysVar(varName) + tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("0")) + c.Assert(err, IsNil) + c.Assert(v, Equals, defaultValue) +} + func (s *testSessionSuite) TestMatchIdentity(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("CREATE USER `useridentity`@`%`") @@ -6091,3 +6127,18 @@ func (s *testSessionSuite) TestSysdateIsNow(c *C) { tk.MustQuery("show variables like '%tidb_sysdate_is_now%'").Check(testkit.Rows("tidb_sysdate_is_now ON")) c.Assert(tk.Se.GetSessionVars().SysdateIsNow, IsTrue) } + +func (s *testSessionSuite) TestEnableLegacyInstanceScope(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // enable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 1") + tk.MustExec("set tidb_general_log = 1") + tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) + c.Assert(tk.Se.GetSessionVars().EnableLegacyInstanceScope, IsTrue) + + // disable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 0") + tk.MustGetErrCode("set tidb_general_log = 1", errno.ErrGlobalVariable) + c.Assert(tk.Se.GetSessionVars().EnableLegacyInstanceScope, IsFalse) +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 56145e291b0a0..3044150faae87 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -992,6 +992,10 @@ type SessionVars struct { // EnablePaging indicates whether enable paging in coprocessor requests. EnablePaging bool + // EnableLegacyInstanceScope says if SET SESSION can be used to set an instance + // scope variable. The default is TRUE. + EnableLegacyInstanceScope bool + // ReadConsistency indicates the read consistency requirement. ReadConsistency ReadConsistencyLevel @@ -1242,6 +1246,7 @@ func NewSessionVars() *SessionVars { MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, Rng: utilMath.NewWithTime(), StatsLoadSyncWait: StatsLoadSyncWait.Load(), + EnableLegacyInstanceScope: DefEnableLegacyInstanceScope, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cdaa19a96fba0..248ca58669374 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -707,11 +707,10 @@ var defaultSysVars = []*SysVar{ } return nil }}, - /* The following variable is defined as session scope but is actually server scope. */ - {Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeInstance, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetGlobal: func(s *SessionVars, val string) error { ProcessGeneralLog.Store(TiDBOptOn(val)) return nil - }, GetSession: func(s *SessionVars) (string, error) { + }, GetGlobal: func(s *SessionVars) (string, error) { return BoolToOnOff(ProcessGeneralLog.Load()), nil }}, {Scope: ScopeSession, Name: TiDBLogFileMaxDays, Value: strconv.Itoa(config.GetGlobalConfig().Log.File.MaxDays), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, skipInit: true, SetSession: func(s *SessionVars, val string) error { @@ -1266,6 +1265,10 @@ var defaultSysVars = []*SysVar{ s.EnablePaging = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableLegacyInstanceScope, Value: BoolToOnOff(DefEnableLegacyInstanceScope), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnableLegacyInstanceScope = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal, Name: TiDBPersistAnalyzeOptions, Value: BoolToOnOff(DefTiDBPersistAnalyzeOptions), skipInit: true, Type: TypeBool, GetGlobal: func(s *SessionVars) (string, error) { return BoolToOnOff(PersistAnalyzeOptions.Load()), nil diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index e67fa600cbcf2..0d694d8b9af88 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -246,22 +246,32 @@ func TestScope(t *testing.T) { sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.True(t, sv.HasSessionScope()) require.True(t, sv.HasGlobalScope()) + require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.False(t, sv.HasSessionScope()) require.True(t, sv.HasGlobalScope()) + require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.True(t, sv.HasSessionScope()) require.False(t, sv.HasGlobalScope()) + require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.False(t, sv.HasSessionScope()) require.False(t, sv.HasGlobalScope()) + require.False(t, sv.HasInstanceScope()) require.True(t, sv.HasNoneScope()) + + sv = SysVar{Scope: ScopeInstance, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + require.False(t, sv.HasSessionScope()) + require.False(t, sv.HasGlobalScope()) + require.True(t, sv.HasInstanceScope()) + require.False(t, sv.HasNoneScope()) } func TestBuiltInCase(t *testing.T) { @@ -696,7 +706,7 @@ func TestSettersandGetters(t *testing.T) { require.Nil(t, sv.SetSession) require.Nil(t, sv.GetSession) } - if !sv.HasGlobalScope() { + if !sv.HasGlobalScope() && !sv.HasInstanceScope() { require.Nil(t, sv.SetGlobal) if sv.Name == Timestamp { // The Timestamp sysvar will have GetGlobal func even though it does not have global scope. @@ -845,6 +855,58 @@ func TestDefaultCharsetAndCollation(t *testing.T) { require.Equal(t, val, mysql.DefaultCollationName) } +func TestInstanceScope(t *testing.T) { + // Instance scope used to be settable via "SET SESSION", which is weird to any MySQL user. + // It is now settable via SET GLOBAL, but to work correctly a sysvar can only ever + // be INSTANCE scoped or GLOBAL scoped, never *both* at the same time (at least for now). + // Otherwise the semantics are confusing to users for how precedence applies. + + for _, sv := range GetSysVars() { + require.False(t, sv.HasGlobalScope() && sv.HasInstanceScope(), "sysvar %s has both instance and global scope", sv.Name) + if sv.HasInstanceScope() { + require.NotNil(t, sv.GetGlobal) + require.NotNil(t, sv.SetGlobal) + } + } + + count := len(GetSysVars()) + sv := SysVar{Scope: ScopeInstance, Name: "newinstancesysvar", Value: On, Type: TypeBool, + SetGlobal: func(s *SessionVars, val string) error { + return fmt.Errorf("set should fail") + }, + GetGlobal: func(s *SessionVars) (string, error) { + return "", fmt.Errorf("get should fail") + }, + } + + RegisterSysVar(&sv) + require.Len(t, GetSysVars(), count+1) + + sysVar := GetSysVar("newinstancesysvar") + require.NotNil(t, sysVar) + + vars := NewSessionVars() + + // It is a boolean, try to set it to a bogus value + _, err := sysVar.Validate(vars, "ABCD", ScopeInstance) + require.Error(t, err) + + // Boolean oN or 1 converts to canonical ON or OFF + normalizedVal, err := sysVar.Validate(vars, "oN", ScopeInstance) + require.Equal(t, "ON", normalizedVal) + require.NoError(t, err) + normalizedVal, err = sysVar.Validate(vars, "0", ScopeInstance) + require.Equal(t, "OFF", normalizedVal) + require.NoError(t, err) + + err = sysVar.SetGlobalFromHook(vars, "OFF", true) // default is on + require.Equal(t, "set should fail", err.Error()) + + // Test unregistration restores previous count + UnregisterSysVar("newinstancesysvar") + require.Equal(t, len(GetSysVars()), count) +} + func TestIndexMergeSwitcher(t *testing.T) { vars := NewSessionVars() vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 8ce847f9e1dbe..4caa7dc410f7e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -593,6 +593,9 @@ const ( // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" + // TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION. + TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope" + // TiDBTableCacheLease indicates the read lock lease of a cached table. TiDBTableCacheLease = "tidb_table_cache_lease" @@ -795,6 +798,7 @@ const ( DefTiDBStmtSummaryMaxSQLLength = 4096 DefTiDBCapturePlanBaseline = Off DefTiDBEnableIndexMerge = true + DefEnableLegacyInstanceScope = true DefTiDBTableCacheLease = 3 // 3s DefTiDBPersistAnalyzeOptions = true DefTiDBEnableColumnTracking = false diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go index ea276da0e3b65..7989f50fe2e8e 100644 --- a/sessionctx/variable/variable.go +++ b/sessionctx/variable/variable.go @@ -38,6 +38,8 @@ const ( ScopeGlobal ScopeFlag = 1 << 0 // ScopeSession means the system variable can only be changed in current session. ScopeSession ScopeFlag = 1 << 1 + // ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers. + ScopeInstance ScopeFlag = 1 << 2 // TypeStr is the default TypeStr TypeFlag = 0 @@ -248,6 +250,11 @@ func (sv *SysVar) HasGlobalScope() bool { return sv.Scope&ScopeGlobal != 0 } +// HasInstanceScope returns true if the scope for the sysVar includes instance +func (sv *SysVar) HasInstanceScope() bool { + return sv.Scope&ScopeInstance != 0 +} + // Validate checks if system variable satisfies specific restriction. func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { // Check that the scope is correct first. @@ -299,7 +306,7 @@ func (sv *SysVar) validateScope(scope ScopeFlag) error { if sv.ReadOnly || sv.Scope == ScopeNone { return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") } - if scope == ScopeGlobal && !sv.HasGlobalScope() { + if scope == ScopeGlobal && !(sv.HasGlobalScope() || sv.HasInstanceScope()) { return errLocalVariable.FastGenByArgs(sv.Name) } if scope == ScopeSession && !sv.HasSessionScope() { From ef8c858347514674a3bfd50b65107e2c21ff0441 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 16 Mar 2022 11:49:52 +0800 Subject: [PATCH 3/5] ddl: migrate test-infra to testify for ddl.testDBSuite4 (#33113) close pingcap/tidb#32964 --- ddl/column_modify_test.go | 12 ++ ddl/db_legacy_test.go | 442 +------------------------------------- ddl/db_test.go | 208 ++++++++++++++++++ ddl/index_modify_test.go | 38 ++++ ddl/table_modify_test.go | 196 +++++++++++++++++ 5 files changed, 455 insertions(+), 441 deletions(-) diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index d4fcf609182d7..654329c43f3b3 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -897,3 +897,15 @@ func TestTransactionWithWriteOnlyColumn(t *testing.T) { require.NoError(t, checkErr) tk.MustQuery("select a from t1").Check(testkit.Rows("2")) } + +func TestColumnCheck(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists column_check") + tk.MustExec("create table column_check (pk int primary key, a int check (a > 1))") + defer tk.MustExec("drop table if exists column_check") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|8231|CONSTRAINT CHECK is not supported")) +} diff --git a/ddl/db_legacy_test.go b/ddl/db_legacy_test.go index 705a5c9caa8f2..35ee20685b471 100644 --- a/ddl/db_legacy_test.go +++ b/ddl/db_legacy_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -49,10 +48,8 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" ntestkit "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/dbterror" - "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" @@ -78,7 +75,6 @@ const ( waitForCleanDataInterval = time.Millisecond * 100 ) -var _ = Suite(&testDBSuite4{&testDBSuite{}}) var _ = Suite(&testDBSuite5{&testDBSuite{}}) var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) @@ -143,7 +139,6 @@ func (s *testDBSuite) TearDownSuite(c *C) { tearDownSuite(s, c) } -type testDBSuite4 struct{ *testDBSuite } type testDBSuite5 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } @@ -369,7 +364,7 @@ func (s *testSerialDBSuite) TestDropTableOnTiKVDiskFull(c *C) { } // TestCancelDropIndex tests cancel ddl job which type is drop primary key. -func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { +func (s *testDBSuite5) TestCancelDropPrimaryKey(c *C) { idxName := "primary" addIdxSQL := "alter table t add primary key idx_c2 (c2);" dropIdxSQL := "alter table t drop primary key;" @@ -575,13 +570,6 @@ func (s *testDBSuite5) TestParallelDropSchemaAndDropTable(c *C) { c.Assert(err, IsNil) } -func (s *testDBSuite4) TestAlterLock(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - tk.MustExec("create table t_index_lock (c1 int, c2 int, C3 int)") - tk.MustExec("alter table t_index_lock add index (c1, c2), lock=none") -} - func (s *testDBSuite5) TestAddMultiColumnsIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use " + s.schemaName) @@ -748,28 +736,6 @@ func (s *testDBSuite5) TestAlterPrimaryKey(c *C) { tk.MustGetErrCode("alter table test_add_pk add primary key (c, b, a)", errno.ErrInvalidUseOfNull) } -func (s *testDBSuite4) TestAddIndexWithDupCols(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - err1 := infoschema.ErrColumnExists.GenWithStackByArgs("b") - err2 := infoschema.ErrColumnExists.GenWithStackByArgs("B") - - tk.MustExec("create table test_add_index_with_dup (a int, b int)") - _, err := tk.Exec("create index c on test_add_index_with_dup(b, a, b)") - c.Check(errors.Cause(err1).(*terror.Error).Equal(err), Equals, true) - - _, err = tk.Exec("create index c on test_add_index_with_dup(b, a, B)") - c.Check(errors.Cause(err2).(*terror.Error).Equal(err), Equals, true) - - _, err = tk.Exec("alter table test_add_index_with_dup add index c (b, a, b)") - c.Check(errors.Cause(err1).(*terror.Error).Equal(err), Equals, true) - - _, err = tk.Exec("alter table test_add_index_with_dup add index c (b, a, B)") - c.Check(errors.Cause(err2).(*terror.Error).Equal(err), Equals, true) - - tk.MustExec("drop table test_add_index_with_dup") -} - func (s *testDBSuite5) TestCreateIndexType(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use " + s.schemaName) @@ -1014,36 +980,6 @@ func (s *testSerialDBSuite) TestTruncateTable(c *C) { } -func (s *testDBSuite4) TestComment(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - tk.MustExec("drop table if exists ct, ct1") - - validComment := strings.Repeat("a", 1024) - invalidComment := strings.Repeat("b", 1025) - - tk.MustExec("create table ct (c int, d int, e int, key (c) comment '" + validComment + "')") - tk.MustExec("create index i on ct (d) comment '" + validComment + "'") - tk.MustExec("alter table ct add key (e) comment '" + validComment + "'") - - tk.MustGetErrCode("create table ct1 (c int, key (c) comment '"+invalidComment+"')", errno.ErrTooLongIndexComment) - tk.MustGetErrCode("create index i1 on ct (d) comment '"+invalidComment+"b"+"'", errno.ErrTooLongIndexComment) - tk.MustGetErrCode("alter table ct add key (e) comment '"+invalidComment+"'", errno.ErrTooLongIndexComment) - - tk.MustExec("set @@sql_mode=''") - tk.MustExec("create table ct1 (c int, d int, e int, key (c) comment '" + invalidComment + "')") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1688|Comment for index 'c' is too long (max = 1024)")) - tk.MustExec("create index i1 on ct1 (d) comment '" + invalidComment + "b" + "'") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1688|Comment for index 'i1' is too long (max = 1024)")) - tk.MustExec("alter table ct1 add key (e) comment '" + invalidComment + "'") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1688|Comment for index 'e' is too long (max = 1024)")) - - tk.MustExec("drop table if exists ct, ct1") -} - func (s *testSerialDBSuite) TestRebaseAutoID(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { @@ -1127,32 +1063,6 @@ func (s *testDBSuite5) TestCheckColumnDefaultValue(c *C) { c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, `null`) } -func (s *testDBSuite4) TestCheckTooBigFieldLength(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists tr_01;") - tk.MustExec("create table tr_01 (id int, name varchar(20000), purchased date ) default charset=utf8 collate=utf8_bin;") - - tk.MustExec("drop table if exists tr_02;") - tk.MustExec("create table tr_02 (id int, name varchar(16000), purchased date ) default charset=utf8mb4 collate=utf8mb4_bin;") - - tk.MustExec("drop table if exists tr_03;") - tk.MustExec("create table tr_03 (id int, name varchar(65534), purchased date ) default charset=latin1;") - - tk.MustExec("drop table if exists tr_04;") - tk.MustExec("create table tr_04 (a varchar(20000) ) default charset utf8;") - tk.MustGetErrCode("alter table tr_04 add column b varchar(20000) charset utf8mb4;", errno.ErrTooBigFieldlength) - tk.MustGetErrCode("alter table tr_04 convert to character set utf8mb4;", errno.ErrTooBigFieldlength) - tk.MustGetErrCode("create table tr (id int, name varchar(30000), purchased date ) default charset=utf8 collate=utf8_bin;", errno.ErrTooBigFieldlength) - tk.MustGetErrCode("create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8_bin;", errno.ErrTooBigFieldlength) - tk.MustGetErrCode("create table tr (id int, name varchar(65536), purchased date ) default charset=latin1;", errno.ErrTooBigFieldlength) - - tk.MustExec("drop table if exists tr_05;") - tk.MustExec("create table tr_05 (a varchar(16000) charset utf8);") - tk.MustExec("alter table tr_05 modify column a varchar(16000) charset utf8;") - tk.MustExec("alter table tr_05 modify column a varchar(16000) charset utf8mb4;") -} - func (s *testDBSuite5) TestCheckConvertToCharacter(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1169,96 +1079,6 @@ func (s *testDBSuite5) TestCheckConvertToCharacter(c *C) { c.Assert(t.Cols()[0].Charset, Equals, "binary") } -func (s *testDBSuite4) TestIfNotExists(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int key);") - - // ADD COLUMN - sql := "alter table t1 add column b int" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrDupFieldName) - tk.MustExec("alter table t1 add column if not exists b int") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1060|Duplicate column name 'b'")) - - // ADD INDEX - sql = "alter table t1 add index idx_b (b)" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrDupKeyName) - tk.MustExec("alter table t1 add index if not exists idx_b (b)") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1061|index already exist idx_b")) - - // CREATE INDEX - sql = "create index idx_b on t1 (b)" - tk.MustGetErrCode(sql, errno.ErrDupKeyName) - tk.MustExec("create index if not exists idx_b on t1 (b)") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1061|index already exist idx_b")) - - // ADD PARTITION - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") - sql = "alter table t2 add partition (partition p2 values less than (30))" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrSameNamePartition) - tk.MustExec("alter table t2 add partition if not exists (partition p2 values less than (30))") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1517|Duplicate partition name p2")) -} - -func (s *testDBSuite4) TestIfExists(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int key, b int);") - - // DROP COLUMN - sql := "alter table t1 drop column b" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrCantDropFieldOrKey) - tk.MustExec("alter table t1 drop column if exists b") // only `a` exists now - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|Can't DROP 'b'; check that column/key exists")) - - // CHANGE COLUMN - sql = "alter table t1 change column b c int" - tk.MustGetErrCode(sql, errno.ErrBadField) - tk.MustExec("alter table t1 change column if exists b c int") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1054|Unknown column 'b' in 't1'")) - tk.MustExec("alter table t1 change column if exists a c int") // only `c` exists now - - // MODIFY COLUMN - sql = "alter table t1 modify column a bigint" - tk.MustGetErrCode(sql, errno.ErrBadField) - tk.MustExec("alter table t1 modify column if exists a bigint") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1054|Unknown column 'a' in 't1'")) - tk.MustExec("alter table t1 modify column if exists c bigint") // only `c` exists now - - // DROP INDEX - tk.MustExec("alter table t1 add index idx_c (c)") - sql = "alter table t1 drop index idx_c" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrCantDropFieldOrKey) - tk.MustExec("alter table t1 drop index if exists idx_c") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index idx_c doesn't exist")) - - // DROP PARTITION - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2 (a int key) partition by range(a) (partition pNeg values less than (0), partition p0 values less than (10), partition p1 values less than (20))") - sql = "alter table t2 drop partition p1" - tk.MustExec(sql) - tk.MustGetErrCode(sql, errno.ErrDropPartitionNonExistent) - tk.MustExec("alter table t2 drop partition if exists p1") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to DROP")) -} - func (s *testDBSuite5) TestModifyGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test;") @@ -1744,202 +1564,6 @@ func (s *testSerialDBSuite) TestSkipSchemaChecker(c *C) { c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue) } -// TestConcurrentLockTables test concurrent lock/unlock tables. -func (s *testDBSuite4) TestConcurrentLockTables(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) - tk2 := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - defer tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int)") - tk2.MustExec("use test") - - // Test concurrent lock tables read. - sql1 := "lock tables t1 read" - sql2 := "lock tables t1 read" - s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { - c.Assert(err1, IsNil) - c.Assert(err2, IsNil) - }) - tk.MustExec("unlock tables") - tk2.MustExec("unlock tables") - - // Test concurrent lock tables write. - sql1 = "lock tables t1 write" - sql2 = "lock tables t1 write" - s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { - c.Assert(err1, IsNil) - c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) - }) - tk.MustExec("unlock tables") - tk2.MustExec("unlock tables") - - // Test concurrent lock tables write local. - sql1 = "lock tables t1 write local" - sql2 = "lock tables t1 write local" - s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { - c.Assert(err1, IsNil) - c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) - }) - - tk.MustExec("unlock tables") - tk2.MustExec("unlock tables") -} - -func (s *testDBSuite4) TestLockTableReadOnly(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) - tk2 := testkit.NewTestKit(c, s.store) - tk2.MustExec("use test") - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1,t2") - defer func() { - tk.MustExec("alter table t1 read write") - tk.MustExec("alter table t2 read write") - tk.MustExec("drop table if exists t1,t2") - }() - tk.MustExec("create table t1 (a int key, b int)") - tk.MustExec("create table t2 (a int key)") - - tk.MustExec("alter table t1 read only") - tk.MustQuery("select * from t1") - tk2.MustQuery("select * from t1") - _, err := tk.Exec("insert into t1 set a=1, b=2") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - _, err = tk.Exec("update t1 set a=1") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - _, err = tk.Exec("delete from t1") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - - _, err = tk2.Exec("insert into t1 set a=1, b=2") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - _, err = tk2.Exec("update t1 set a=1") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - _, err = tk2.Exec("delete from t1") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - tk2.MustExec("alter table t1 read only") - _, err = tk2.Exec("insert into t1 set a=1, b=2") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) - tk.MustExec("alter table t1 read write") - - tk.MustExec("lock tables t1 read") - c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - tk.MustExec("lock tables t1 write") - c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - tk.MustExec("lock tables t1 write local") - c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) - tk.MustExec("unlock tables") - - tk.MustExec("alter table t1 read only") - c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked), IsTrue) - c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked), IsTrue) - tk.MustExec("admin cleanup table lock t1") - tk2.MustExec("insert into t1 set a=1, b=2") - - tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1") - tk.MustExec("begin pessimistic") - tk.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) - tk2.MustExec("update t1 set b = 3") - tk2.MustExec("alter table t1 read only") - tk2.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 3")) - tk.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) - tk.MustExec("update t1 set b = 4") - c.Assert(terror.ErrorEqual(tk.ExecToErr("commit"), domain.ErrInfoSchemaChanged), IsTrue) - tk2.MustExec("alter table t1 read write") -} - -type checkRet func(c *C, err1, err2 error) - -func (s *testDBSuite4) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { - callback := &ddl.TestDDLCallback{} - times := 0 - callback.OnJobRunBeforeExported = func(job *model.Job) { - if times != 0 { - return - } - var qLen int - for { - err := kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { - jobs, err1 := admin.GetDDLJobs(txn) - if err1 != nil { - return err1 - } - qLen = len(jobs) - return nil - }) - c.Assert(err, IsNil) - if qLen == 2 { - break - } - time.Sleep(5 * time.Millisecond) - } - times++ - } - d := s.dom.DDL() - originalCallback := d.GetHook() - defer d.SetHook(originalCallback) - d.SetHook(callback) - - var wg util.WaitGroupWrapper - var err1 error - var err2 error - ch := make(chan struct{}) - // Make sure the sql1 is put into the DDLJobQueue. - go func() { - var qLen int - for { - err := kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { - jobs, err3 := admin.GetDDLJobs(txn) - if err3 != nil { - return err3 - } - qLen = len(jobs) - return nil - }) - c.Assert(err, IsNil) - if qLen == 1 { - // Make sure sql2 is executed after the sql1. - close(ch) - break - } - time.Sleep(5 * time.Millisecond) - } - }() - wg.Run(func() { - _, err1 = se1.Execute(context.Background(), sql1) - }) - wg.Run(func() { - <-ch - _, err2 = se2.Execute(context.Background(), sql2) - }) - - wg.Wait() - f(c, err1, err2) -} - -func (s *testDBSuite4) TestColumnCheck(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - tk.MustExec("drop table if exists column_check") - tk.MustExec("create table column_check (pk int primary key, a int check (a > 1))") - defer tk.MustExec("drop table if exists column_check") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|8231|CONSTRAINT CHECK is not supported")) -} - func (s *testDBSuite5) TestAlterCheck(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use " + s.schemaName) @@ -2367,70 +1991,6 @@ func (s *testSerialDBSuite) TestColumnTypeChangeGenUniqueChangingName(c *C) { tk.MustExec("drop table if exists t") } -func (s *testDBSuite4) TestGeneratedColumnWindowFunction(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("DROP TABLE IF EXISTS t") - tk.MustGetErrCode("CREATE TABLE t (a INT , b INT as (ROW_NUMBER() OVER (ORDER BY a)))", errno.ErrWindowInvalidWindowFuncUse) - tk.MustGetErrCode("CREATE TABLE t (a INT , index idx ((ROW_NUMBER() OVER (ORDER BY a))))", errno.ErrWindowInvalidWindowFuncUse) -} - -func (s *testDBSuite4) TestAnonymousIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("DROP TABLE IF EXISTS t") - tk.MustExec("create table t(bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb int, b int);") - tk.MustExec("alter table t add index bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb(b);") - tk.MustExec("alter table t add index (bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb);") - res := tk.MustQuery("show index from t where key_name='bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb';") - c.Assert(len(res.Rows()), Equals, 1) - res = tk.MustQuery("show index from t where key_name='bbbbbbbbbbbbbbbbbbbbbbbbbbbbbb_2';") - c.Assert(len(res.Rows()), Equals, 1) -} - -func (s *testDBSuite4) TestUnsupportedAlterTableOption(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("DROP TABLE IF EXISTS t") - tk.MustExec("create table t(a char(10) not null,b char(20)) shard_row_id_bits=6;") - tk.MustGetErrCode("alter table t pre_split_regions=6;", errno.ErrUnsupportedDDLOperation) -} - -func (s *testDBSuite4) TestCreateTableWithDecimalWithDoubleZero(c *C) { - tk := testkit.NewTestKit(c, s.store) - - checkType := func(db, table, field string) { - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() - tableInfo, err := is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) - c.Assert(err, IsNil) - tblInfo := tableInfo.Meta() - for _, col := range tblInfo.Columns { - if col.Name.L == field { - c.Assert(col.Flen, Equals, 10) - } - } - } - - tk.MustExec("use test") - tk.MustExec("drop table if exists tt") - tk.MustExec("create table tt(d decimal(0, 0))") - checkType("test", "tt", "d") - - tk.MustExec("drop table tt") - tk.MustExec("create table tt(a int)") - tk.MustExec("alter table tt add column d decimal(0, 0)") - checkType("test", "tt", "d") - - /* - Currently not support change column to decimal - tk.MustExec("drop table tt") - tk.MustExec("create table tt(d int)") - tk.MustExec("alter table tt change column d d decimal(0, 0)") - checkType("test", "tt", "d") - */ -} - // Close issue #24172. // See https://github.com/pingcap/tidb/issues/24172 func (s *testSerialDBSuite) TestCancelJobWriteConflict(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index c66fb757e6180..7ce3a4ea396af 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -22,9 +22,11 @@ import ( "github.com/pingcap/tidb/ddl" ddlutil "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" @@ -520,3 +522,209 @@ func TestCreateTableIgnoreCheckConstraint(t *testing.T) { " `enable` tinyint(1) DEFAULT NULL\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) } + +func TestAlterLock(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t_index_lock (c1 int, c2 int, C3 int)") + tk.MustExec("alter table t_index_lock add index (c1, c2), lock=none") +} + +func TestComment(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + validComment := strings.Repeat("a", 1024) + invalidComment := strings.Repeat("b", 1025) + + tk.MustExec("create table ct (c int, d int, e int, key (c) comment '" + validComment + "')") + tk.MustExec("create index i on ct (d) comment '" + validComment + "'") + tk.MustExec("alter table ct add key (e) comment '" + validComment + "'") + + tk.MustGetErrCode("create table ct1 (c int, key (c) comment '"+invalidComment+"')", errno.ErrTooLongIndexComment) + tk.MustGetErrCode("create index i1 on ct (d) comment '"+invalidComment+"b"+"'", errno.ErrTooLongIndexComment) + tk.MustGetErrCode("alter table ct add key (e) comment '"+invalidComment+"'", errno.ErrTooLongIndexComment) + + tk.MustExec("set @@sql_mode=''") + tk.MustExec("create table ct1 (c int, d int, e int, key (c) comment '" + invalidComment + "')") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1688|Comment for index 'c' is too long (max = 1024)")) + tk.MustExec("create index i1 on ct1 (d) comment '" + invalidComment + "b" + "'") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1688|Comment for index 'i1' is too long (max = 1024)")) + tk.MustExec("alter table ct1 add key (e) comment '" + invalidComment + "'") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1688|Comment for index 'e' is too long (max = 1024)")) +} + +func TestIfNotExists(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a int key)") + + // ADD COLUMN + sql := "alter table t1 add column b int" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrDupFieldName) + tk.MustExec("alter table t1 add column if not exists b int") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1060|Duplicate column name 'b'")) + + // ADD INDEX + sql = "alter table t1 add index idx_b (b)" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrDupKeyName) + tk.MustExec("alter table t1 add index if not exists idx_b (b)") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1061|index already exist idx_b")) + + // CREATE INDEX + sql = "create index idx_b on t1 (b)" + tk.MustGetErrCode(sql, errno.ErrDupKeyName) + tk.MustExec("create index if not exists idx_b on t1 (b)") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1061|index already exist idx_b")) + + // ADD PARTITION + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") + sql = "alter table t2 add partition (partition p2 values less than (30))" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrSameNamePartition) + tk.MustExec("alter table t2 add partition if not exists (partition p2 values less than (30))") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1517|Duplicate partition name p2")) +} + +func TestIfExists(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a int key, b int);") + + // DROP COLUMN + sql := "alter table t1 drop column b" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrCantDropFieldOrKey) + tk.MustExec("alter table t1 drop column if exists b") // only `a` exists now + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1091|Can't DROP 'b'; check that column/key exists")) + + // CHANGE COLUMN + sql = "alter table t1 change column b c int" + tk.MustGetErrCode(sql, errno.ErrBadField) + tk.MustExec("alter table t1 change column if exists b c int") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1054|Unknown column 'b' in 't1'")) + tk.MustExec("alter table t1 change column if exists a c int") // only `c` exists now + + // MODIFY COLUMN + sql = "alter table t1 modify column a bigint" + tk.MustGetErrCode(sql, errno.ErrBadField) + tk.MustExec("alter table t1 modify column if exists a bigint") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1054|Unknown column 'a' in 't1'")) + tk.MustExec("alter table t1 modify column if exists c bigint") // only `c` exists now + + // DROP INDEX + tk.MustExec("alter table t1 add index idx_c (c)") + sql = "alter table t1 drop index idx_c" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrCantDropFieldOrKey) + tk.MustExec("alter table t1 drop index if exists idx_c") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1091|index idx_c doesn't exist")) + + // DROP PARTITION + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2 (a int key) partition by range(a) (partition pNeg values less than (0), partition p0 values less than (10), partition p1 values less than (20))") + sql = "alter table t2 drop partition p1" + tk.MustExec(sql) + tk.MustGetErrCode(sql, errno.ErrDropPartitionNonExistent) + tk.MustExec("alter table t2 drop partition if exists p1") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1507|Error in list of partitions to DROP")) +} + +func TestCheckTooBigFieldLength(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table tr_01 (id int, name varchar(20000), purchased date ) default charset=utf8 collate=utf8_bin;") + + tk.MustExec("drop table if exists tr_02;") + tk.MustExec("create table tr_02 (id int, name varchar(16000), purchased date ) default charset=utf8mb4 collate=utf8mb4_bin;") + + tk.MustExec("drop table if exists tr_03;") + tk.MustExec("create table tr_03 (id int, name varchar(65534), purchased date ) default charset=latin1;") + + tk.MustExec("drop table if exists tr_04;") + tk.MustExec("create table tr_04 (a varchar(20000) ) default charset utf8;") + tk.MustGetErrCode("alter table tr_04 add column b varchar(20000) charset utf8mb4;", errno.ErrTooBigFieldlength) + tk.MustGetErrCode("alter table tr_04 convert to character set utf8mb4;", errno.ErrTooBigFieldlength) + tk.MustGetErrCode("create table tr (id int, name varchar(30000), purchased date ) default charset=utf8 collate=utf8_bin;", errno.ErrTooBigFieldlength) + tk.MustGetErrCode("create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8_bin;", errno.ErrTooBigFieldlength) + tk.MustGetErrCode("create table tr (id int, name varchar(65536), purchased date ) default charset=latin1;", errno.ErrTooBigFieldlength) + + tk.MustExec("drop table if exists tr_05;") + tk.MustExec("create table tr_05 (a varchar(16000) charset utf8);") + tk.MustExec("alter table tr_05 modify column a varchar(16000) charset utf8;") + tk.MustExec("alter table tr_05 modify column a varchar(16000) charset utf8mb4;") +} + +func TestGeneratedColumnWindowFunction(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustGetErrCode("CREATE TABLE t (a INT , b INT as (ROW_NUMBER() OVER (ORDER BY a)))", errno.ErrWindowInvalidWindowFuncUse) + tk.MustGetErrCode("CREATE TABLE t (a INT , index idx ((ROW_NUMBER() OVER (ORDER BY a))))", errno.ErrWindowInvalidWindowFuncUse) +} + +func TestCreateTableWithDecimalWithDoubleZero(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + checkType := func(db, table, field string) { + ctx := tk.Session().(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) + require.NoError(t, err) + tblInfo := tableInfo.Meta() + for _, col := range tblInfo.Columns { + if col.Name.L == field { + require.Equal(t, 10, col.Flen) + } + } + } + + tk.MustExec("use test") + tk.MustExec("drop table if exists tt") + tk.MustExec("create table tt(d decimal(0, 0))") + checkType("test", "tt", "d") + + tk.MustExec("drop table tt") + tk.MustExec("create table tt(a int)") + tk.MustExec("alter table tt add column d decimal(0, 0)") + checkType("test", "tt", "d") + + tk.MustExec("drop table tt") + tk.MustExec("create table tt(d int)") + tk.MustExec("alter table tt change column d d decimal(0, 0)") + checkType("test", "tt", "d") +} diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 8d9c5f539552b..45410b3a4c4bc 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/ddl" testddlutil "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -1308,3 +1309,40 @@ func TestAddMultiColumnsIndexClusterIndex(t *testing.T) { tk.MustExec("admin check index t idx;") tk.MustExec("admin check table t;") } + +func TestAddIndexWithDupCols(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + err1 := infoschema.ErrColumnExists.GenWithStackByArgs("b") + err2 := infoschema.ErrColumnExists.GenWithStackByArgs("B") + + tk.MustExec("create table test_add_index_with_dup (a int, b int)") + err := tk.ExecToErr("create index c on test_add_index_with_dup(b, a, b)") + require.ErrorIs(t, err, errors.Cause(err1)) + err = tk.ExecToErr("create index c on test_add_index_with_dup(b, a, B)") + require.ErrorIs(t, err, errors.Cause(err2)) + err = tk.ExecToErr("alter table test_add_index_with_dup add index c (b, a, b)") + require.ErrorIs(t, err, errors.Cause(err1)) + err = tk.ExecToErr("alter table test_add_index_with_dup add index c (b, a, B)") + require.ErrorIs(t, err, errors.Cause(err2)) + + tk.MustExec("drop table test_add_index_with_dup") +} + +func TestAnonymousIndex(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec("create table t(bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb int, b int)") + tk.MustExec("alter table t add index bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb(b)") + tk.MustExec("alter table t add index (bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb)") + rows := tk.MustQuery("show index from t where key_name='bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb'").Rows() + require.Len(t, rows, 1) + rows = tk.MustQuery("show index from t where key_name='bbbbbbbbbbbbbbbbbbbbbbbbbbbbbb_2'").Rows() + require.Len(t, rows, 1) +} diff --git a/ddl/table_modify_test.go b/ddl/table_modify_test.go index 7e975c138335d..30e2d4d3d45c9 100644 --- a/ddl/table_modify_test.go +++ b/ddl/table_modify_test.go @@ -15,13 +15,22 @@ package ddl_test import ( + "context" "testing" "time" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/israce" "github.com/stretchr/testify/require" ) @@ -103,3 +112,190 @@ func TestCreateTable(t *testing.T) { tk.MustExec("drop table x;") tk.MustExec("drop table y;") } + +func TestLockTableReadOnly(t *testing.T) { + if israce.RaceEnabled { + t.Skip("skip race test") + } + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tableModifyLease) + defer clean() + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1,t2") + defer func() { + tk1.MustExec("alter table t1 read write") + tk1.MustExec("alter table t2 read write") + tk1.MustExec("drop table if exists t1,t2") + }() + tk1.MustExec("create table t1 (a int key, b int)") + tk1.MustExec("create table t2 (a int key)") + + tk1.MustExec("alter table t1 read only") + tk1.MustQuery("select * from t1") + tk2.MustQuery("select * from t1") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("insert into t1 set a=1, b=2"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk1.ExecToErr("update t1 set a=1"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk1.ExecToErr("delete from t1"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("insert into t1 set a=1, b=2"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("update t1 set a=1"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("delete from t1"), infoschema.ErrTableLocked)) + + tk2.MustExec("alter table t1 read only") + require.True(t, terror.ErrorEqual(tk2.ExecToErr("insert into t1 set a=1, b=2"), infoschema.ErrTableLocked)) + + tk1.MustExec("alter table t1 read write") + tk1.MustExec("lock tables t1 read") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + tk1.MustExec("lock tables t1 write") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + tk1.MustExec("lock tables t1 write local") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked)) + tk1.MustExec("unlock tables") + + tk1.MustExec("alter table t1 read only") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk1.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk1.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked)) + require.True(t, terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked)) + tk1.MustExec("admin cleanup table lock t1") + tk2.MustExec("insert into t1 set a=1, b=2") + + tk1.MustExec("set tidb_enable_amend_pessimistic_txn = 1") + tk1.MustExec("begin pessimistic") + tk1.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) + tk2.MustExec("update t1 set b = 3") + tk2.MustExec("alter table t1 read only") + tk2.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 3")) + tk1.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) + tk1.MustExec("update t1 set b = 4") + require.True(t, terror.ErrorEqual(tk1.ExecToErr("commit"), domain.ErrInfoSchemaChanged)) + tk2.MustExec("alter table t1 read write") +} + +// TestConcurrentLockTables test concurrent lock/unlock tables. +func TestConcurrentLockTables(t *testing.T) { + if israce.RaceEnabled { + t.Skip("skip race test") + } + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, tableModifyLease) + defer clean() + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("create table t1 (a int)") + + // Test concurrent lock tables read. + sql1 := "lock tables t1 read" + sql2 := "lock tables t1 read" + testParallelExecSQL(t, store, dom, sql1, sql2, tk1.Session(), tk2.Session(), func(t *testing.T, err1, err2 error) { + require.NoError(t, err1) + require.NoError(t, err2) + }) + tk1.MustExec("unlock tables") + tk2.MustExec("unlock tables") + + // Test concurrent lock tables write. + sql1 = "lock tables t1 write" + sql2 = "lock tables t1 write" + testParallelExecSQL(t, store, dom, sql1, sql2, tk1.Session(), tk2.Session(), func(t *testing.T, err1, err2 error) { + require.NoError(t, err1) + require.True(t, terror.ErrorEqual(err2, infoschema.ErrTableLocked)) + }) + tk1.MustExec("unlock tables") + tk2.MustExec("unlock tables") + + // Test concurrent lock tables write local. + sql1 = "lock tables t1 write local" + sql2 = "lock tables t1 write local" + testParallelExecSQL(t, store, dom, sql1, sql2, tk1.Session(), tk2.Session(), func(t *testing.T, err1, err2 error) { + require.NoError(t, err1) + require.True(t, terror.ErrorEqual(err2, infoschema.ErrTableLocked)) + }) + + tk1.MustExec("unlock tables") + tk2.MustExec("unlock tables") +} + +func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql1, sql2 string, se1, se2 session.Session, f func(t *testing.T, err1, err2 error)) { + callback := &ddl.TestDDLCallback{} + times := 0 + callback.OnJobRunBeforeExported = func(job *model.Job) { + if times != 0 { + return + } + var qLen int + for { + err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + jobs, err1 := admin.GetDDLJobs(txn) + if err1 != nil { + return err1 + } + qLen = len(jobs) + return nil + }) + require.NoError(t, err) + if qLen == 2 { + break + } + time.Sleep(5 * time.Millisecond) + } + times++ + } + d := dom.DDL() + originalCallback := d.GetHook() + defer d.SetHook(originalCallback) + d.SetHook(callback) + + var wg util.WaitGroupWrapper + var err1 error + var err2 error + ch := make(chan struct{}) + // Make sure the sql1 is put into the DDLJobQueue. + go func() { + var qLen int + for { + err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + jobs, err3 := admin.GetDDLJobs(txn) + if err3 != nil { + return err3 + } + qLen = len(jobs) + return nil + }) + require.NoError(t, err) + if qLen == 1 { + // Make sure sql2 is executed after the sql1. + close(ch) + break + } + time.Sleep(5 * time.Millisecond) + } + }() + wg.Run(func() { + _, err1 = se1.Execute(context.Background(), sql1) + }) + wg.Run(func() { + <-ch + _, err2 = se2.Execute(context.Background(), sql2) + }) + + wg.Wait() + f(t, err1, err2) +} + +func TestUnsupportedAlterTableOption(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, tableModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a char(10) not null,b char(20)) shard_row_id_bits=6") + tk.MustGetErrCode("alter table t pre_split_regions=6", errno.ErrUnsupportedDDLOperation) +} From 2dd0074e4ecebebba85b2726cc09f6f0c7e56005 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 16 Mar 2022 12:05:52 +0800 Subject: [PATCH 4/5] executor: fix wrong result of delete multiple tables using left join (#33055) close pingcap/tidb#31321 --- executor/delete.go | 3 +++ executor/executor_test.go | 30 ++++++++++++++++++++++++++++++ executor/update.go | 4 ++-- 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/executor/delete.go b/executor/delete.go index c859b41a83c75..c327236ee0f18 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -159,6 +159,9 @@ func (e *DeleteExec) doBatchDelete(ctx context.Context) error { func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []plannercore.TblColPosInfo, joinedRow []types.Datum) error { // iterate all the joined tables, and got the copresonding rows in joinedRow. for _, info := range colPosInfos { + if unmatchedOuterRow(info, joinedRow) { + continue + } if tblRowMap[info.TblID] == nil { tblRowMap[info.TblID] = kv.NewHandleMap() } diff --git a/executor/executor_test.go b/executor/executor_test.go index 2c61049d0eda8..f3d46e60d0169 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8521,3 +8521,33 @@ func (s *testSuite1) TestBitColumnIn(c *C) { tk.MustExec("insert into t values (65)") tk.MustQuery("select * from t where id not in (-1,2)").Check(testkit.Rows("\x00A")) } + +func (s *testSuite) TestDeleteWithMulTbl(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // Delete multiple tables from left joined table. + // The result of left join is (3, null, null). + // Because rows in t2 are not matched, so no row will be deleted in t2. + // But row in t1 is matched, so it should be deleted. + tk.MustExec("use test;") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c1 int);") + tk.MustExec("create table t2 (c1 int primary key, c2 int);") + tk.MustExec("insert into t1 values(3);") + tk.MustExec("insert into t2 values(2, 2);") + tk.MustExec("insert into t2 values(0, 0);") + tk.MustExec("delete from t1, t2 using t1 left join t2 on t1.c1 = t2.c2;") + tk.MustQuery("select * from t1 order by c1;").Check(testkit.Rows()) + tk.MustQuery("select * from t2 order by c1;").Check(testkit.Rows("0 0", "2 2")) + + // Rows in both t1 and t2 are matched, so will be deleted even if it's null. + // NOTE: The null values are not generated by join. + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c1 int);") + tk.MustExec("create table t2 (c2 int);") + tk.MustExec("insert into t1 values(null);") + tk.MustExec("insert into t2 values(null);") + tk.MustExec("delete from t1, t2 using t1 join t2 where t1.c1 is null;") + tk.MustQuery("select * from t1;").Check(testkit.Rows()) + tk.MustQuery("select * from t2;").Check(testkit.Rows()) +} diff --git a/executor/update.go b/executor/update.go index 7173897757fc2..ed5c76f44c278 100644 --- a/executor/update.go +++ b/executor/update.go @@ -96,7 +96,7 @@ func (e *UpdateExec) prepare(row []types.Datum) (err error) { break } } - if e.unmatchedOuterRow(content, row) { + if unmatchedOuterRow(content, row) { updatable = false } e.tableUpdatable = append(e.tableUpdatable, updatable) @@ -211,7 +211,7 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema, row, n // the inner handle field is filled with a NULL value. // // This fixes: https://github.com/pingcap/tidb/issues/7176. -func (e *UpdateExec) unmatchedOuterRow(tblPos plannercore.TblColPosInfo, waitUpdateRow []types.Datum) bool { +func unmatchedOuterRow(tblPos plannercore.TblColPosInfo, waitUpdateRow []types.Datum) bool { firstHandleIdx := tblPos.HandleCols.GetCol(0) return waitUpdateRow[firstHandleIdx.Index].IsNull() } From 0f343d3a4094460a97668561d9ad15bc72be4883 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Wed, 16 Mar 2022 12:55:52 +0800 Subject: [PATCH 5/5] expression, types: use time.Add() to calculate subtime (#32903) close pingcap/tidb#31868 --- expression/builtin_time.go | 27 ++++++------------------ expression/builtin_time_test.go | 2 ++ expression/builtin_time_vec_generated.go | 14 ++---------- expression/generator/time_vec.go | 14 ++---------- expression/integration_serial_test.go | 7 ++++++ 5 files changed, 19 insertions(+), 45 deletions(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index ee865f440eea0..d9cdcb9504e4b 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -5391,21 +5391,16 @@ func strDatetimeSubDuration(sc *stmtctx.StatementContext, d string, arg1 types.D sc.AppendWarning(err) return "", true, nil } - arg1time, err := arg1.ConvertToTime(sc, uint8(types.GetFsp(arg1.String()))) + resultTime, err := arg0.Add(sc, arg1.Neg()) if err != nil { return "", false, err } - tmpDuration := arg0.Sub(sc, &arg1time) fsp := types.MaxFsp - if tmpDuration.MicroSecond() == 0 { + if resultTime.Microsecond() == 0 { fsp = types.MinFsp } - resultDuration, err := tmpDuration.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return "", false, err - } - resultDuration.SetFsp(fsp) - return resultDuration.String(), false, nil + resultTime.SetFsp(fsp) + return resultTime.String(), false, nil } // strDurationSubDuration subtracts duration from duration string, returns a string value. @@ -6476,12 +6471,7 @@ func (b *builtinSubDatetimeAndDurationSig) evalTime(row chunk.Row) (types.Time, return types.ZeroDatetime, isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - arg1time, err := arg1.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return arg1time, true, err - } - tmpDuration := arg0.Sub(sc, &arg1time) - result, err := tmpDuration.ConvertToTime(sc, arg0.Type()) + result, err := arg0.Add(sc, arg1.Neg()) return result, err != nil, err } @@ -6521,12 +6511,7 @@ func (b *builtinSubDatetimeAndStringSig) evalTime(row chunk.Row) (types.Time, bo } return types.ZeroDatetime, true, err } - arg1time, err := arg1.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return types.ZeroDatetime, true, err - } - tmpDuration := arg0.Sub(sc, &arg1time) - result, err := tmpDuration.ConvertToTime(sc, mysql.TypeDatetime) + result, err := arg0.Add(sc, arg1.Neg()) return result, err != nil, err } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 256f9e098e2e9..4d9c610400565 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -1036,6 +1036,8 @@ func TestSubTimeSig(t *testing.T) { {"110:00:00", "1 02:00:00", "84:00:00"}, {"2017-01-01 01:01:01.11", "01:01:01.11111", "2016-12-31 23:59:59.998890"}, {"2007-12-31 23:59:59.999999", "1 1:1:1.000002", "2007-12-30 22:58:58.999997"}, + {"1000-01-01 01:00:00.000000", "00:00:00.000001", "1000-01-01 00:59:59.999999"}, + {"1000-01-01 01:00:00.000001", "00:00:00.000001", "1000-01-01 01:00:00"}, {"1", "xxcvadfgasd", ""}, {"xxcvadfgasd", "1", ""}, } diff --git a/expression/builtin_time_vec_generated.go b/expression/builtin_time_vec_generated.go index 4081890fb23c3..87a89e8208cc6 100644 --- a/expression/builtin_time_vec_generated.go +++ b/expression/builtin_time_vec_generated.go @@ -677,12 +677,7 @@ func (b *builtinSubDatetimeAndDurationSig) vecEvalTime(input *chunk.Chunk, resul sc := b.ctx.GetSessionVars().StmtCtx arg1Duration := types.Duration{Duration: arg1, Fsp: -1} - arg1time, err := arg1Duration.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return err - } - tmpDuration := arg0.Sub(sc, &arg1time) - output, err := tmpDuration.ConvertToTime(sc, arg0.Type()) + output, err := arg0.Add(sc, arg1Duration.Neg()) if err != nil { return err @@ -751,12 +746,7 @@ func (b *builtinSubDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result } return err } - arg1time, err := arg1Duration.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return err - } - tmpDuration := arg0.Sub(sc, &arg1time) - output, err := tmpDuration.ConvertToTime(sc, mysql.TypeDatetime) + output, err := arg0.Add(sc, arg1Duration.Neg()) if err != nil { return err diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go index 86a8f6703d8d8..5d18b9a4a17b4 100644 --- a/expression/generator/time_vec.go +++ b/expression/generator/time_vec.go @@ -175,12 +175,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result {{ else }} sc := b.ctx.GetSessionVars().StmtCtx arg1Duration := types.Duration{Duration: arg1, Fsp: -1} - arg1time, err := arg1Duration.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return err - } - tmpDuration := arg0.Sub(sc, &arg1time) - output, err := tmpDuration.ConvertToTime(sc, arg0.Type()) + output, err := arg0.Add(sc, arg1Duration.Neg()) {{ end }} if err != nil { return err @@ -205,12 +200,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result } return err } - arg1time, err := arg1Duration.ConvertToTime(sc, mysql.TypeDatetime) - if err != nil { - return err - } - tmpDuration := arg0.Sub(sc, &arg1time) - output, err := tmpDuration.ConvertToTime(sc, mysql.TypeDatetime) + output, err := arg0.Add(sc, arg1Duration.Neg()) {{ end }} if err != nil { return err diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index 720065e9baa38..b1eaf4b2596f2 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -2115,6 +2115,13 @@ func TestTimeBuiltin(t *testing.T) { tk.MustQuery("select subtime(cast('10:10:10' as time), cast('9:10:10' as time))").Check(testkit.Rows("01:00:00")) tk.MustQuery("select subtime('10:10:10', cast('9:10:10' as time))").Check(testkit.Rows("01:00:00")) + // SUBTIME issue #31868 + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a DATETIME(6))") + tk.MustExec(`insert into t values ("1000-01-01 01:00:00.000000"), ("1000-01-01 01:00:00.000001")`) + tk.MustQuery(`SELECT SUBTIME(a, '00:00:00.000001') FROM t ORDER BY a;`).Check(testkit.Rows("1000-01-01 00:59:59.999999", "1000-01-01 01:00:00.000000")) + tk.MustQuery(`SELECT SUBTIME(a, '10:00:00.000001') FROM t ORDER BY a;`).Check(testkit.Rows("0999-12-31 14:59:59.999999", "0999-12-31 15:00:00.000000")) + // ADDTIME & SUBTIME issue #5966 tk.MustExec("drop table if exists t") tk.MustExec("create table t(a datetime, b timestamp, c time, d date, e bit(1))")