Skip to content

Commit

Permalink
Merge branch 'master' into options-runtime-stats
Browse files Browse the repository at this point in the history
  • Loading branch information
AndreMouche committed May 13, 2021
2 parents 18456cd + acf2e82 commit 9769283
Show file tree
Hide file tree
Showing 20 changed files with 229 additions and 102 deletions.
42 changes: 36 additions & 6 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)")
Expand All @@ -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)")
Expand All @@ -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")
}
Expand Down
5 changes: 4 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
7 changes: 7 additions & 0 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
5 changes: 0 additions & 5 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
7 changes: 7 additions & 0 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,14 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -563,6 +565,11 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J
if tableInfo == nil || jobInfo == nil {
return nil, nil, errors.Errorf("Can't find dropped/truncated table: %v in DDL history jobs", tableName.Name)
}
// Dropping local temporary tables won't appear in DDL jobs.
if tableInfo.TempTableType == model.TempTableGlobal {
msg := mysql.Message("Recover/flashback table is not supported on temporary tables", nil)
return nil, nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg)
}
return jobInfo, tableInfo, nil
}

Expand Down
1 change: 1 addition & 0 deletions executor/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
71 changes: 41 additions & 30 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5697,24 +5697,15 @@ func (s *testRecoverTable) TearDownSuite(c *C) {
s.dom.Close()
}

func (s *testRecoverTable) TestRecoverTable(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")
c.Assert(err, IsNil)
}()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create table t_recover (a int);")
defer func(originGC bool) {
func (s *testRecoverTable) mockGC(tk *testkit.TestKit) (string, string, string, func()) {
originGC := ddl.IsEmulatorGCEnable()
resetGC := func() {
if originGC {
ddl.EmulatorGCEnable()
} else {
ddl.EmulatorGCDisable()
}
}(ddl.IsEmulatorGCEnable())
}

// disable emulator GC.
// Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl.
Expand All @@ -5727,6 +5718,23 @@ func (s *testRecoverTable) TestRecoverTable(c *C) {
UPDATE variable_value = '%[1]s'`
// clear GC variables first.
tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )")
return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC
}

func (s *testRecoverTable) TestRecoverTable(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")
c.Assert(err, IsNil)
}()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create table t_recover (a int);")

timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC := s.mockGC(tk)
defer resetGC()

tk.MustExec("insert into t_recover values (1),(2),(3)")
tk.MustExec("drop table t_recover")
Expand Down Expand Up @@ -5819,24 +5827,10 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) {
tk.MustExec("use test_flashback")
tk.MustExec("drop table if exists t_flashback")
tk.MustExec("create table t_flashback (a int);")
defer func(originGC bool) {
if originGC {
ddl.EmulatorGCEnable()
} else {
ddl.EmulatorGCDisable()
}
}(ddl.IsEmulatorGCEnable())

// Disable emulator GC.
// Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl.
ddl.EmulatorGCDisable()
gcTimeFormat := "20060102-15:04:05 -0700 MST"
timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat)
safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '')
ON DUPLICATE KEY
UPDATE variable_value = '%[1]s'`
// Clear GC variables first.
tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )")
timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk)
defer resetGC()

// Set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))
// Set GC enable.
Expand Down Expand Up @@ -5939,6 +5933,23 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) {
tk.MustQuery("select a from t order by a").Check(testkit.Rows("1", "2", "3"))
}

func (s *testRecoverTable) TestRecoverTempTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create global temporary table t_recover (a int) on commit delete rows;")

timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk)
defer resetGC()
// Set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))

tk.MustExec("drop table t_recover")
tk.MustGetErrCode("recover table t_recover;", errno.ErrUnsupportedDDLOperation)
tk.MustGetErrCode("flashback table t_recover;", errno.ErrUnsupportedDDLOperation)
}

func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) {
tk1 := testkit.NewTestKit(c, s.store)
tk1.MustExec("drop database if exists ps_text")
Expand Down
6 changes: 6 additions & 0 deletions executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 4 additions & 0 deletions executor/grant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
43 changes: 38 additions & 5 deletions executor/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand All @@ -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
Expand Down
3 changes: 3 additions & 0 deletions executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -531,6 +531,9 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres
chk.AppendInt64(schemaColIdx, handle.IntValue())
return true, nil
}
if types.NeedRestoredData(col.RetType) {
return false, nil
}
// Try to decode common handle.
if mysql.HasPriKeyFlag(col.RetType.Flag) {
for i, hid := range pkCols {
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Loading

0 comments on commit 9769283

Please sign in to comment.