Skip to content

Commit

Permalink
Merge branch 'master' into init-cte-max
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid committed May 13, 2021
2 parents c7f5799 + f2c2fbd commit 8dd0d93
Show file tree
Hide file tree
Showing 89 changed files with 1,765 additions and 448 deletions.
5 changes: 5 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/metrics"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -70,6 +71,10 @@ type mockSessionManager struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
for _, item := range msm.PS {
Expand Down
6 changes: 3 additions & 3 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,9 @@ const (
DefHost = "0.0.0.0"
// DefStatusHost is the default status host of TiDB
DefStatusHost = "0.0.0.0"
// Def TableColumnCountLimit is limit of the number of columns in a table
// DefTableColumnCountLimit is limit of the number of columns in a table
DefTableColumnCountLimit = 1017
// Def TableColumnCountLimit is maximum limitation of the number of columns in a table
// DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table
DefMaxOfTableColumnCountLimit = 4096
)

Expand All @@ -73,7 +73,7 @@ var (
"tikv": true,
"unistore": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
// CheckTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
checkBeforeDropLDFlag = "None"
Expand Down
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 domain/domain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv"
Expand Down Expand Up @@ -241,6 +242,10 @@ type mockSessionManager struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
for _, item := range msm.PS {
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1020,7 +1020,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrBuildExecutor: mysql.Message("Failed to build executor", nil),
ErrBatchInsertFail: mysql.Message("Batch insert failed, please clean the table and try again.", nil),
ErrGetStartTS: mysql.Message("Can not get start ts", nil),
ErrPrivilegeCheckFail: mysql.Message("privilege check fail", nil), // this error message should begin lowercased to be compatible with the test
ErrPrivilegeCheckFail: mysql.Message("privilege check for '%s' fail", nil), // this error message should begin lowercased to be compatible with the test
ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil),
ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil),
ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil),
Expand Down
12 changes: 11 additions & 1 deletion 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 Expand Up @@ -1133,7 +1138,7 @@ Schema has changed

["planner:8121"]
error = '''
privilege check fail
privilege check for '%s' fail
'''

["planner:8122"]
Expand Down Expand Up @@ -1281,6 +1286,11 @@ error = '''
Unknown SEQUENCE: '%-.300s'
'''

["schema:8003"]
error = '''
TiDB admin check table failed.
'''

["schema:8020"]
error = '''
Table '%s' was locked in %s by %v
Expand Down
3 changes: 3 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import (
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stmtsummary"
"github.com/pingcap/tidb/util/stringutil"

"go.uber.org/zap"
"go.uber.org/zap/zapcore"
)
Expand Down Expand Up @@ -377,6 +378,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) {
if txn.Valid() {
txnStartTS = txn.StartTS()
}

return &recordSet{
executor: e,
stmt: a,
Expand Down Expand Up @@ -590,6 +592,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error {
}
e, err = a.handlePessimisticLockError(ctx, err)
if err != nil {
// todo: Report deadlock
if ErrDeadlock.Equal(err) {
metrics.StatementDeadlockDetectDuration.Observe(time.Since(startLocking).Seconds())
}
Expand Down
16 changes: 16 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,22 @@ func (s *testSuite5) TestAdminCheckIndex(c *C) {
check()
}

func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists temporary_admin_test;")
tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;")
tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);")
tk.MustGetErrCode("admin check table temporary_admin_test;", mysql.ErrAdminCheckTable)
tk.MustExec("drop table if exists temporary_admin_test;")

tk.MustExec("drop table if exists non_temporary_admin_test;")
tk.MustExec("create table non_temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2));")
tk.MustExec("insert non_temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);")
tk.MustExec("admin check table non_temporary_admin_test;")
tk.MustExec("drop table if exists non_temporary_admin_test;")
}

func (s *testSuite5) TestAdminRecoverIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
4 changes: 3 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1531,7 +1531,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TablePlacementPolicy),
strings.ToLower(infoschema.TableClientErrorsSummaryGlobal),
strings.ToLower(infoschema.TableClientErrorsSummaryByUser),
strings.ToLower(infoschema.TableClientErrorsSummaryByHost):
strings.ToLower(infoschema.TableClientErrorsSummaryByHost),
strings.ToLower(infoschema.TableTiDBTrx),
strings.ToLower(infoschema.ClusterTableTiDBTrx):
return &MemTableReaderExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()),
table: v.Table,
Expand Down
15 changes: 13 additions & 2 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 @@ -311,8 +313,12 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx
if isSystemTable(tn.Schema.L, tn.Name.L) {
return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L)
}

if obt == tableObject && config.CheckTableBeforeDrop {
tableInfo, err := e.is.TableByName(tn.Schema, tn.Name)
if err != nil {
return err
}
tempTableType := tableInfo.Meta().TempTableType
if obt == tableObject && config.CheckTableBeforeDrop && tempTableType == model.TempTableNone {
logutil.BgLogger().Warn("admin check table before drop",
zap.String("database", fullti.Schema.O),
zap.String("table", fullti.Name.O),
Expand Down Expand Up @@ -559,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
5 changes: 5 additions & 0 deletions executor/executor_pkg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/executor/aggfuncs"
"github.com/pingcap/tidb/expression"
plannerutil "github.com/pingcap/tidb/planner/util"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -60,6 +61,10 @@ type mockSessionManager struct {
serverID uint64
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

// ShowProcessList implements the SessionManager.ShowProcessList interface.
func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
Expand Down
Loading

0 comments on commit 8dd0d93

Please sign in to comment.