diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index fe6bca7dc4563..405ada57f15ec 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -969,9 +969,9 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test") tk.MustExec("use test") + tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1))") tk.MustExec("insert into t (a) values (1)") @@ -980,42 +980,72 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { res := tk.MustQuery("select * from t use index(idx) where c > 1") tk.MustQuery("select * from t ignore index(idx) where c > 1").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1), d int as (c + 1))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a+1), c varchar(20) as (b*2), d float as (a*23+b-1+length(c)))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 4 25")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(10), b float as (length(a)+123), c varchar(20) as (right(a, 2)), d float as (b+b-7+1-3+3*ASCII(c)))") tk.MustExec("insert into t (a) values ('adorable')") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("adorable 131 le 577")) // 131+131-7+1-3+3*108 - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a), c int(10) as (a+b), d float as (a+b+c), e decimal as (a+b+c+d))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 1 2 4 8")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a bigint, b bigint as (a+1) virtual, c bigint as (b+1) virtual)") @@ -1025,7 +1055,7 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk.MustExec("alter table t add column(d bigint as (c+1) virtual)") tk.MustExec("alter table t add index idx_d(d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx_d) where d > 2") + res := tk.MustQuery("select * from t use index(idx_d) where d > 2") tk.MustQuery("select * from t ignore index(idx_d) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b03b4ca66f536..e6d77c9e674e9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1617,7 +1617,10 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T return errors.Trace(err) } if err = checkAddPartitionTooManyPartitions(uint64(len(tbInfo.Partition.Definitions))); err != nil { - return errors.Trace(err) + return err + } + if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { + return err } switch tbInfo.Partition.Type { diff --git a/ddl/partition.go b/ddl/partition.go index 4cc71eb1c8d74..0cafa9d2ff525 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1476,6 +1476,13 @@ func checkAddPartitionTooManyPartitions(piDefs uint64) error { return nil } +func checkAddPartitionOnTemporaryMode(tbInfo *model.TableInfo) error { + if tbInfo.Partition != nil && tbInfo.TempTableType != model.TempTableNone { + return ErrPartitionNoTemporary + } + return nil +} + func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { return ast.ErrNoParts.GenWithStackByArgs("partitions") diff --git a/ddl/table.go b/ddl/table.go index acd209a7bb0da..668de3ac41c05 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -56,11 +56,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) job.State = model.JobStateCancelled return ver, errors.Trace(err) } - if tbInfo.Partition != nil && (tbInfo.TempTableType == model.TempTableGlobal || tbInfo.TempTableType == model.TempTableLocal) { - // unsupported ddl, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(ErrPartitionNoTemporary) - } tbInfo.State = model.StateNone err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) diff --git a/errors.toml b/errors.toml index a54913fa1bd2c..926823909f96e 100644 --- a/errors.toml +++ b/errors.toml @@ -501,6 +501,11 @@ error = ''' Deadlock found when trying to get lock; try restarting transaction ''' +["executor:1221"] +error = ''' +Incorrect usage of %s and %s +''' + ["executor:1242"] error = ''' Subquery returns more than 1 row diff --git a/executor/errors.go b/executor/errors.go index 7f3345659e4f9..ad8104a96e7ee 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -42,6 +42,7 @@ var ( ErrTableaccessDenied = dbterror.ClassExecutor.NewStd(mysql.ErrTableaccessDenied) ErrBadDB = dbterror.ClassExecutor.NewStd(mysql.ErrBadDB) ErrWrongObject = dbterror.ClassExecutor.NewStd(mysql.ErrWrongObject) + ErrWrongUsage = dbterror.ClassExecutor.NewStd(mysql.ErrWrongUsage) ErrRoleNotGranted = dbterror.ClassPrivilege.NewStd(mysql.ErrRoleNotGranted) ErrDeadlock = dbterror.ClassExecutor.NewStd(mysql.ErrLockDeadlock) ErrQueryInterrupted = dbterror.ClassExecutor.NewStd(mysql.ErrQueryInterrupted) diff --git a/executor/grant.go b/executor/grant.go index 6c715758b6c7f..86620cc0124b1 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -473,6 +473,12 @@ func (e *GrantExec) grantDBLevel(priv *ast.PrivElem, user *ast.UserSpec, interna if priv.Priv == mysql.UsagePriv { return nil } + for _, v := range mysql.StaticGlobalOnlyPrivs { + if v == priv.Priv { + return ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES") + } + } + dbName := e.Level.DBName if len(dbName) == 0 { dbName = e.ctx.GetSessionVars().CurrentDB diff --git a/executor/grant_test.go b/executor/grant_test.go index bb720b48b730b..13686494feddc 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -89,6 +89,10 @@ func (s *testSuite3) TestGrantDBScope(c *C) { sql := fmt.Sprintf("SELECT %s FROM mysql.DB WHERE User=\"testDB1\" and host=\"localhost\" and db=\"test\";", mysql.Priv2UserCol[v]) tk.MustQuery(sql).Check(testkit.Rows("Y")) } + + // Grant in wrong scope. + _, err := tk.Exec(` grant create user on test.* to 'testDB1'@'localhost';`) + c.Assert(terror.ErrorEqual(err, executor.ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES")), IsTrue) } func (s *testSuite3) TestWithGrantOption(c *C) { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 0c9e63129ebf1..5be39c3a04d54 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -494,12 +494,45 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { for _, t := range tests { for i := range t.partitions { sql := fmt.Sprintf(t.sql, tables[i]) - c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue) - tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + tk.MustPartition(sql, t.partitions[i]).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) } } } +func (s *partitionTableSuite) TestAddDropPartitions(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_add_drop_partition") + tk.MustExec("use test_add_drop_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int) partition by range(a) ( + partition p0 values less than (5), + partition p1 values less than (10), + partition p2 values less than (15))`) + tk.MustExec(`insert into t values (2), (7), (12)`) + tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) + tk.MustPartition(`select * from t where a < 8`, "p0,p1").Sort().Check(testkit.Rows("2", "7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "2", "7")) + + // remove p0 + tk.MustExec(`alter table t drop partition p0`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "7")) + + // add 2 more partitions + tk.MustExec(`alter table t add partition (partition p3 values less than (20))`) + tk.MustExec(`alter table t add partition (partition p4 values less than (40))`) + tk.MustExec(`insert into t values (15), (25)`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") @@ -511,15 +544,15 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") // list partition table - tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( partition p0 values in (1, 2, 3, 4), partition p1 values in (5, 6, 7, 8), partition p2 values in (9, 10, 11, 12));`) // range partition table tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( - partition p0 values less than(300), - partition p1 values less than (500), + partition p0 values less than(300), + partition p1 values less than (500), partition p2 values less than(1100));`) // hash partition table diff --git a/go.mod b/go.mod index cd8becf5a757c..b2bd664da13f3 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c + github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index 4e1030039a04f..c42d129ae5b8b 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c h1:GLFd+wBN7EsV6ad/tVGFCD37taOyzIMVs3SdiWZF18I= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b h1:eLuDQ6eJCEKCbGwhGrkjzagwev1GJGU2Y2kFkAsBzV0= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= diff --git a/store/tikv/config/config_test.go b/store/tikv/config/config_test.go index f79f2d09c22c8..a47e1e7e5030b 100644 --- a/store/tikv/config/config_test.go +++ b/store/tikv/config/config_test.go @@ -34,20 +34,19 @@ func (s *testConfigSuite) TestParsePath(c *C) { } func (s *testConfigSuite) TestTxnScopeValue(c *C) { - - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`), IsNil) isGlobal, v := GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "bj") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsTrue) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) } diff --git a/types/json/binary_functions_test.go b/types/json/binary_functions_test.go index 8191638f7b2f6..12a9c8ece8ae2 100644 --- a/types/json/binary_functions_test.go +++ b/types/json/binary_functions_test.go @@ -35,6 +35,6 @@ func (s *testJSONFuncSuite) TestdecodeEscapedUnicode(c *C) { func BenchmarkDecodeEscapedUnicode(b *testing.B) { for i := 0; i < b.N; i++ { in := "597d" - decodeEscapedUnicode([]byte(in)) + _, _, _ = decodeEscapedUnicode([]byte(in)) } } diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 06eb826c56b78..d6d6e41bb6e9f 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -256,14 +256,16 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { } // MustPartition checks if the result execution plan must read specific partitions. -func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) bool { +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) + ok := false for i := range rs.rows { if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { - return true + ok = true } } - return false + tk.c.Assert(ok, check.IsTrue) + return tk.MustQuery(sql, args...) } // MustUseIndex checks if the result execution plan contains specific index(es).