Skip to content

Commit

Permalink
Merge branch 'dev_partitionTest' of github.com:zhuo-zhi/tidb into dev…
Browse files Browse the repository at this point in the history
…_partitionTest
  • Loading branch information
zhuo-zhi committed May 13, 2021
2 parents 0449db5 + 4bee259 commit 3dd9819
Show file tree
Hide file tree
Showing 92 changed files with 1,787 additions and 558 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
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
4 changes: 3 additions & 1 deletion ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,9 @@ func LoadGlobalVars(ctx sessionctx.Context, varNames []string) error {
for _, row := range rows {
varName := row.GetString(0)
varValue := row.GetString(1)
variable.SetLocalSystemVar(varName, varValue)
if err = ctx.GetSessionVars().SetSystemVar(varName, varValue); err != nil {
return err
}
}
}
return nil
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
7 changes: 6 additions & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1133,7 +1133,7 @@ Schema has changed

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

["planner:8122"]
Expand Down Expand Up @@ -1281,6 +1281,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
2 changes: 1 addition & 1 deletion executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,7 +479,7 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe
panic(fmt.Sprintf("Error happened when buildGroupConcat: %s", err.Error()))
}
var s string
s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen)
if err != nil {
panic(fmt.Sprintf("Error happened when buildGroupConcat: no system variable named '%s'", variable.GroupConcatMaxLen))
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {

func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) {
sessionVars := ctx.GetSessionVars()
concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency)
concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency)
if err != nil {
return 0, err
}
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
2 changes: 1 addition & 1 deletion executor/checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,7 @@ func (c *checksumContext) HandleResponse(update *tipb.ChecksumResponse) {

func getChecksumTableConcurrency(ctx sessionctx.Context) (int, error) {
sessionVars := ctx.GetSessionVars()
concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency)
concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency)
if err != nil {
return 0, err
}
Expand Down
8 changes: 6 additions & 2 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -311,8 +311,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
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
5 changes: 5 additions & 0 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/parser/auth"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/israce"
Expand All @@ -38,6 +39,10 @@ type mockSessionManager1 struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo {
return nil
}

// ShowProcessList implements the SessionManager.ShowProcessList interface.
func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
Expand Down
40 changes: 39 additions & 1 deletion executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
infoschema.TableClientErrorsSummaryByUser,
infoschema.TableClientErrorsSummaryByHost:
err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O)
case infoschema.TableTiDBTrx:
e.setDataForTiDBTrx(sctx)
case infoschema.ClusterTableTiDBTrx:
err = e.setDataForClusterTiDBTrx(sctx)
}
if err != nil {
return nil, err
Expand Down Expand Up @@ -1675,7 +1679,7 @@ func (e *memtableRetriever) setDataFromSessionVar(ctx sessionctx.Context) error
sessionVars := ctx.GetSessionVars()
for _, v := range variable.GetSysVars() {
var value string
value, err = variable.GetSessionSystemVar(sessionVars, v.Name)
value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name)
if err != nil {
return err
}
Expand Down Expand Up @@ -2011,6 +2015,40 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context
return nil
}

func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) {
sm := ctx.GetSessionManager()
if sm == nil {
return
}

loginUser := ctx.GetSessionVars().User
var hasProcessPriv bool
if pm := privilege.GetPrivilegeManager(ctx); pm != nil {
if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) {
hasProcessPriv = true
}
}
infoList := sm.ShowTxnList()
for _, info := range infoList {
// If you have the PROCESS privilege, you can see all running transactions.
// Otherwise, you can see only your own transactions.
if !hasProcessPriv && loginUser != nil && info.Username != loginUser.Username {
continue
}
e.rows = append(e.rows, info.ToDatum())
}
}

func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) error {
e.setDataForTiDBTrx(ctx)
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
e.rows = rows
return nil
}

type hugeMemTableRetriever struct {
dummyCloser
table *model.TableInfo
Expand Down
5 changes: 5 additions & 0 deletions executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/session"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
Expand Down Expand Up @@ -728,6 +729,10 @@ type mockSessionManager struct {
serverID uint64
}

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

func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
return sm.processInfoMap
}
Expand Down
Loading

0 comments on commit 3dd9819

Please sign in to comment.