diff --git a/Makefile b/Makefile index 68a622857ccee..20bd554472bbc 100644 --- a/Makefile +++ b/Makefile @@ -363,15 +363,16 @@ endif # make bench-daily TO=/path/to/file.json .PHONY: bench-daily bench-daily: - go test github.com/pingcap/tidb/pkg/session -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/distsql -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test github.com/pingcap/tidb/pkg/executor -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test github.com/pingcap/tidb/pkg/executor/test/splittest -run TestBenchDaily -bench Ignore --outfile bench_daily.json - go test github.com/pingcap/tidb/pkg/tablecodec -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test github.com/pingcap/tidb/pkg/expression -run TestBenchDaily -bench Ignore --outfile bench_daily.json - go test github.com/pingcap/tidb/pkg/util/rowcodec -run TestBenchDaily -bench Ignore --outfile bench_daily.json - go test github.com/pingcap/tidb/pkg/util/codec -run TestBenchDaily -bench Ignore --outfile bench_daily.json - go test github.com/pingcap/tidb/pkg/distsql -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/planner/core/tests/partition -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/session -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test github.com/pingcap/tidb/pkg/statistics -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/tablecodec -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/util/codec -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test github.com/pingcap/tidb/pkg/util/rowcodec -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test github.com/pingcap/tidb/pkg/util/benchdaily -run TestBenchDaily -bench Ignore \ -date `git log -n1 --date=unix --pretty=format:%cd` \ -commit `git log -n1 --pretty=format:%h` \ diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index ac5578c0f231a..057f7985216e4 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -3069,11 +3069,11 @@ func TestIssue52680(t *testing.T) { testSteps := []struct { sql string - expect meta.AutoIDGroup + expect model.AutoIDGroup }{ - {sql: "", expect: meta.AutoIDGroup{RowID: 0, IncrementID: 4000, RandomID: 0}}, - {sql: "drop table issue52680", expect: meta.AutoIDGroup{RowID: 0, IncrementID: 0, RandomID: 0}}, - {sql: "recover table issue52680", expect: meta.AutoIDGroup{RowID: 0, IncrementID: 4000, RandomID: 0}}, + {sql: "", expect: model.AutoIDGroup{RowID: 0, IncrementID: 4000, RandomID: 0}}, + {sql: "drop table issue52680", expect: model.AutoIDGroup{RowID: 0, IncrementID: 0, RandomID: 0}}, + {sql: "recover table issue52680", expect: model.AutoIDGroup{RowID: 0, IncrementID: 4000, RandomID: 0}}, } for _, step := range testSteps { if step.sql != "" { diff --git a/pkg/ddl/ddl.go b/pkg/ddl/ddl.go index b03b2a0c0c9cc..b6a5134f8f447 100644 --- a/pkg/ddl/ddl.go +++ b/pkg/ddl/ddl.go @@ -51,7 +51,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/owner" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -78,9 +77,6 @@ const ( reorgWorkerCnt = 10 generalWorkerCnt = 10 - - // checkFlagIndexInJobArgs is the recoverCheckFlag index used in RecoverTable/RecoverSchema job arg list. - checkFlagIndexInJobArgs = 1 ) const ( @@ -1033,16 +1029,16 @@ func (d *ddl) cleanDeadTableLock(unlockTables []model.TableLockTpInfo, se model. if len(unlockTables) == 0 { return nil } - arg := &LockTablesArg{ + args := &model.LockTablesArgs{ UnlockTables: unlockTables, SessionInfo: se, } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: unlockTables[0].SchemaID, TableID: unlockTables[0].TableID, Type: model.ActionUnlockTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, } ctx, err := d.sessPool.Get() @@ -1050,7 +1046,7 @@ func (d *ddl) cleanDeadTableLock(unlockTables []model.TableLockTpInfo, se model. return err } defer d.sessPool.Put(ctx) - err = d.executor.DoDDLJob(ctx, job) + err = d.executor.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -1100,30 +1096,6 @@ func (d *ddl) SwitchMDL(enable bool) error { return nil } -// RecoverInfo contains information needed by DDL.RecoverTable. -type RecoverInfo struct { - SchemaID int64 - TableInfo *model.TableInfo - DropJobID int64 - SnapshotTS uint64 - AutoIDs meta.AutoIDGroup - OldSchemaName string - OldTableName string -} - -// RecoverSchemaInfo contains information needed by DDL.RecoverSchema. -type RecoverSchemaInfo struct { - *model.DBInfo - RecoverTabsInfo []*RecoverInfo - // LoadTablesOnExecute is the new logic to avoid a large RecoverTabsInfo can't be - // persisted. If it's true, DDL owner will recover RecoverTabsInfo instead of the - // job submit node. - LoadTablesOnExecute bool - DropJobID int64 - SnapshotTS uint64 - OldSchemaName pmodel.CIStr -} - // delayForAsyncCommit sleeps `SafeWindow + AllowedClockDrift` before a DDL job finishes. // It should be called before any DDL that could break data consistency. // This provides a safe window for async commit and 1PC to commit with an old schema. diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index cb3a92e241ca0..1624f81b62459 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -104,8 +104,8 @@ type Executor interface { CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) - RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo) (err error) - RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *RecoverSchemaInfo) error + RecoverTable(ctx sessionctx.Context, recoverTableInfo *model.RecoverTableInfo) (err error) + RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *model.RecoverSchemaInfo) error DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) CreateIndex(ctx sessionctx.Context, stmt *ast.CreateIndexStmt) error DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) error @@ -794,7 +794,7 @@ func (e *executor) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt return nil } -func (e *executor) RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *RecoverSchemaInfo) error { +func (e *executor) RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *model.RecoverSchemaInfo) error { involvedSchemas := []model.InvolvingSchemaInfo{{ Database: recoverSchemaInfo.DBInfo.Name.L, Table: model.InvolvingAll, @@ -807,14 +807,19 @@ func (e *executor) RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *Reco } recoverSchemaInfo.State = model.StateNone job := &model.Job{ + Version: model.GetJobVerInUse(), Type: model.ActionRecoverSchema, BinlogInfo: &model.HistoryInfo{}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, - Args: []any{recoverSchemaInfo, recoverCheckFlagNone}, InvolvingSchemaInfo: involvedSchemas, SQLMode: ctx.GetSessionVars().SQLMode, } - err := e.DoDDLJob(ctx, job) + + args := &model.RecoverArgs{ + RecoverInfo: recoverSchemaInfo, + CheckFlag: recoverCheckFlagNone, + } + err := e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -1431,9 +1436,9 @@ func (e *executor) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) return errors.Trace(err) } -func (e *executor) RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo) (err error) { +func (e *executor) RecoverTable(ctx sessionctx.Context, recoverTableInfo *model.RecoverTableInfo) (err error) { is := e.infoCache.GetLatest() - schemaID, tbInfo := recoverInfo.SchemaID, recoverInfo.TableInfo + schemaID, tbInfo := recoverTableInfo.SchemaID, recoverTableInfo.TableInfo // Check schema exist. schema, ok := is.SchemaByID(schemaID) if !ok { @@ -1449,28 +1454,33 @@ func (e *executor) RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo // for "flashback table xxx to yyy" // Note: this case only allow change table name, schema remains the same. var involvedSchemas []model.InvolvingSchemaInfo - if recoverInfo.OldTableName != tbInfo.Name.L { + if recoverTableInfo.OldTableName != tbInfo.Name.L { involvedSchemas = []model.InvolvingSchemaInfo{ - {Database: schema.Name.L, Table: recoverInfo.OldTableName}, + {Database: schema.Name.L, Table: recoverTableInfo.OldTableName}, {Database: schema.Name.L, Table: tbInfo.Name.L}, } } tbInfo.State = model.StateNone job := &model.Job{ - SchemaID: schemaID, - TableID: tbInfo.ID, - SchemaName: schema.Name.L, - TableName: tbInfo.Name.L, - + Version: model.GetJobVerInUse(), + SchemaID: schemaID, + TableID: tbInfo.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, Type: model.ActionRecoverTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{recoverInfo, recoverCheckFlagNone}, InvolvingSchemaInfo: involvedSchemas, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - err = e.DoDDLJob(ctx, job) + + args := &model.RecoverArgs{ + RecoverInfo: &model.RecoverSchemaInfo{ + RecoverTableInfos: []*model.RecoverTableInfo{recoverTableInfo}, + }, + CheckFlag: recoverCheckFlagNone} + err = e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -2181,17 +2191,18 @@ func (e *executor) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal return err } job := &model.Job{ + Version: model.GetJobVerInUse(), Type: model.ActionShardRowID, SchemaID: schema.ID, TableID: tbInfo.ID, SchemaName: schema.Name.L, TableName: tbInfo.Name.L, BinlogInfo: &model.HistoryInfo{}, - Args: []any{uVal}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - err = e.DoDDLJob(ctx, job) + args := &model.ShardRowIDArgs{ShardRowIDBits: uVal} + err = e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -3550,18 +3561,20 @@ func (e *executor) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: schema.ID, TableID: tb.Meta().ID, SchemaName: schema.Name.L, TableName: tb.Meta().Name.L, Type: model.ActionModifyTableAutoIDCache, BinlogInfo: &model.HistoryInfo{}, - Args: []any{newCache}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - - err = e.DoDDLJob(ctx, job) + args := &model.ModifyTableAutoIDCacheArgs{ + NewCache: newCache, + } + err = e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -3738,18 +3751,22 @@ func (e *executor) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.I } job = &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: schema.ID, TableID: tableID, SchemaName: schema.Name.L, TableName: tableName, Type: model.ActionAlterTTLInfo, BinlogInfo: &model.HistoryInfo{}, - Args: []any{ttlInfo, ttlEnable, ttlCronJobSchedule}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - - err = e.DoDDLJob(ctx, job) + args := &model.AlterTTLInfoArgs{ + TTLInfo: ttlInfo, + TTLEnable: ttlEnable, + TTLCronJobSchedule: ttlCronJobSchedule, + } + err = e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -3772,6 +3789,7 @@ func (e *executor) AlterTableRemoveTTL(ctx sessionctx.Context, ident ast.Ident) if tblInfo.TTLInfo != nil { job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: schema.ID, TableID: tableID, SchemaName: schema.Name.L, @@ -3781,7 +3799,7 @@ func (e *executor) AlterTableRemoveTTL(ctx sessionctx.Context, ident ast.Ident) CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - err = e.DoDDLJob(ctx, job) + err = e.doDDLJob2(ctx, job, &model.EmptyArgs{}) return errors.Trace(err) } @@ -5296,24 +5314,24 @@ func (e *executor) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) } unlockTables := ctx.GetAllTableLocks() - arg := &LockTablesArg{ + args := &model.LockTablesArgs{ LockTables: lockTables, UnlockTables: unlockTables, SessionInfo: sessionInfo, } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: lockTables[0].SchemaID, TableID: lockTables[0].TableID, Type: model.ActionLockTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, InvolvingSchemaInfo: involveSchemaInfo, SQLMode: ctx.GetSessionVars().SQLMode, } // AddTableLock here is avoiding this job was executed successfully but the session was killed before return. ctx.AddTableLock(lockTables) - err := e.DoDDLJob(ctx, job) + err := e.doDDLJob2(ctx, job, args) if err == nil { ctx.ReleaseTableLocks(unlockTables) ctx.AddTableLock(lockTables) @@ -5326,7 +5344,7 @@ func (e *executor) UnlockTables(ctx sessionctx.Context, unlockTables []model.Tab if len(unlockTables) == 0 { return nil } - arg := &LockTablesArg{ + args := &model.LockTablesArgs{ UnlockTables: unlockTables, SessionInfo: model.SessionInfo{ ServerID: e.uuid, @@ -5351,17 +5369,17 @@ func (e *executor) UnlockTables(ctx sessionctx.Context, unlockTables []model.Tab }) } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: unlockTables[0].SchemaID, TableID: unlockTables[0].TableID, Type: model.ActionUnlockTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, InvolvingSchemaInfo: involveSchemaInfo, SQLMode: ctx.GetSessionVars().SQLMode, } - err := e.DoDDLJob(ctx, job) + err := e.doDDLJob2(ctx, job, args) if err == nil { ctx.ReleaseAllTableLocks() } @@ -5418,37 +5436,27 @@ func (e *executor) CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableN return nil } - arg := &LockTablesArg{ + args := &model.LockTablesArgs{ UnlockTables: cleanupTables, IsCleanup: true, } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: cleanupTables[0].SchemaID, TableID: cleanupTables[0].TableID, Type: model.ActionUnlockTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, InvolvingSchemaInfo: involvingSchemaInfo, SQLMode: ctx.GetSessionVars().SQLMode, } - err := e.DoDDLJob(ctx, job) + err := e.doDDLJob2(ctx, job, args) if err == nil { ctx.ReleaseTableLocks(cleanupTables) } return errors.Trace(err) } -// LockTablesArg is the argument for LockTables, export for test. -type LockTablesArg struct { - LockTables []model.TableLockTpInfo - IndexOfLock int - UnlockTables []model.TableLockTpInfo - IndexOfUnlock int - SessionInfo model.SessionInfo - IsCleanup bool -} - func (e *executor) RepairTable(ctx sessionctx.Context, createStmt *ast.CreateTableStmt) error { // Existence of DB and table has been checked in the preprocessor. oldTableInfo, ok := (ctx.Value(domainutil.RepairedTable)).(*model.TableInfo) @@ -5513,17 +5521,19 @@ func (e *executor) RepairTable(ctx sessionctx.Context, createStmt *ast.CreateTab newTableInfo.State = model.StateNone job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: oldDBInfo.ID, TableID: newTableInfo.ID, SchemaName: oldDBInfo.Name.L, TableName: newTableInfo.Name.L, Type: model.ActionRepairTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{newTableInfo}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - err = e.DoDDLJob(ctx, job) + + args := &model.RepairTableArgs{TableInfo: newTableInfo} + err = e.doDDLJob2(ctx, job, args) if err == nil { // Remove the old TableInfo from repairInfo before domain reload. domainutil.RepairInfo.RemoveFromRepairInfo(oldDBInfo.Name.L, oldTableInfo.Name.L) @@ -5593,18 +5603,21 @@ func (e *executor) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequence } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: db.ID, TableID: tbl.Meta().ID, SchemaName: db.Name.L, TableName: tbl.Meta().Name.L, Type: model.ActionAlterSequence, BinlogInfo: &model.HistoryInfo{}, - Args: []any{ident, stmt.SeqOptions}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - - err = e.DoDDLJob(ctx, job) + args := &model.AlterSequenceArgs{ + Ident: ident, + SeqOptions: stmt.SeqOptions, + } + err = e.doDDLJob2(ctx, job, args) return errors.Trace(err) } @@ -6092,18 +6105,18 @@ func (e *executor) AlterTableCache(sctx sessionctx.Context, ti ast.Ident) (err e sctx.SetValue(sessionctx.QueryString, ddlQuery) job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: schema.ID, SchemaName: schema.Name.L, TableName: t.Meta().Name.L, TableID: t.Meta().ID, Type: model.ActionAlterCacheTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{}, CDCWriteSource: sctx.GetSessionVars().CDCWriteSource, SQLMode: sctx.GetSessionVars().SQLMode, } - return e.DoDDLJob(sctx, job) + return e.doDDLJob2(sctx, job, &model.EmptyArgs{}) } func checkCacheTableSize(store kv.Storage, tableID int64) (bool, error) { @@ -6152,18 +6165,17 @@ func (e *executor) AlterTableNoCache(ctx sessionctx.Context, ti ast.Ident) (err } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: schema.ID, SchemaName: schema.Name.L, TableName: t.Meta().Name.L, TableID: t.Meta().ID, Type: model.ActionAlterNoCacheTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, SQLMode: ctx.GetSessionVars().SQLMode, } - - return e.DoDDLJob(ctx, job) + return e.doDDLJob2(ctx, job, &model.EmptyArgs{}) } func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr, constr *ast.Constraint) error { diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 3a79da64d49fa..333126ca33ba2 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -186,9 +186,7 @@ func testAddIndex(t *testing.T, tp testAddIndexType, createTableSQL, idxTp strin tk.MustExec("set global tidb_scatter_region = 0") }() } - if isTestPartition { - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") - } else if (testClusteredIndex & tp) > 0 { + if (testClusteredIndex & tp) > 0 { tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn } tk.MustExec("drop table if exists test_add_index") diff --git a/pkg/ddl/job_worker.go b/pkg/ddl/job_worker.go index c6fc8b723bde2..7bd921c067cdc 100644 --- a/pkg/ddl/job_worker.go +++ b/pkg/ddl/job_worker.go @@ -393,15 +393,11 @@ func (w *worker) deleteDDLJob(job *model.Job) error { } func finishRecoverTable(w *worker, job *model.Job) error { - var ( - recoverInfo *RecoverInfo - recoverTableCheckFlag int64 - ) - err := job.DecodeArgs(&recoverInfo, &recoverTableCheckFlag) + args, err := model.GetRecoverArgs(job) if err != nil { return errors.Trace(err) } - if recoverTableCheckFlag == recoverCheckFlagEnableGC { + if args.CheckFlag == recoverCheckFlagEnableGC { err = enableGC(w) if err != nil { return errors.Trace(err) @@ -411,15 +407,11 @@ func finishRecoverTable(w *worker, job *model.Job) error { } func finishRecoverSchema(w *worker, job *model.Job) error { - var ( - recoverSchemaInfo *RecoverSchemaInfo - recoverSchemaCheckFlag int64 - ) - err := job.DecodeArgs(&recoverSchemaInfo, &recoverSchemaCheckFlag) + args, err := model.GetRecoverArgs(job) if err != nil { return errors.Trace(err) } - if recoverSchemaCheckFlag == recoverCheckFlagEnableGC { + if args.CheckFlag == recoverCheckFlagEnableGC { err = enableGC(w) if err != nil { return errors.Trace(err) diff --git a/pkg/ddl/metabuild_test.go b/pkg/ddl/metabuild_test.go index b1bc2b6356d5e..2f43896176360 100644 --- a/pkg/ddl/metabuild_test.go +++ b/pkg/ddl/metabuild_test.go @@ -117,16 +117,6 @@ func TestNewMetaBuildContextWithSctx(t *testing.T) { return ctx.GetPreSplitRegions() }, }, - { - field: "enableTablePartitionMode", - setSctx: func(val any) { - sessVars.EnableTablePartition = val.(string) - }, - testVals: []any{variable.Off, variable.On, "AUTO"}, - getter: func(ctx *metabuild.Context) any { - return ctx.GetEnableTablePartitionMode() - }, - }, { field: "suppressTooLongIndexErr", extra: func() { diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 79adea67e010b..ecf204f2ac93d 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -508,11 +508,6 @@ func buildTablePartitionInfo(ctx *metabuild.Context, s *ast.PartitionOptions, tb return nil } - if strings.EqualFold(ctx.GetEnableTablePartitionMode(), "OFF") { - ctx.AppendWarning(dbterror.ErrTablePartitionDisabled) - return nil - } - var enable bool switch s.Tp { case pmodel.PartitionTypeRange: @@ -2840,7 +2835,7 @@ func (w *worker) onExchangeTablePartition(jobCtx *jobContext, t *meta.Meta, job // Set both tables to the maximum auto IDs between normal table and partitioned table. // TODO: Fix the issue of big transactions during EXCHANGE PARTITION with AutoID. // Similar to https://github.com/pingcap/tidb/issues/46904 - newAutoIDs := meta.AutoIDGroup{ + newAutoIDs := model.AutoIDGroup{ RowID: mathutil.Max(ptAutoIDs.RowID, ntAutoIDs.RowID), IncrementID: mathutil.Max(ptAutoIDs.IncrementID, ntAutoIDs.IncrementID), RandomID: mathutil.Max(ptAutoIDs.RandomID, ntAutoIDs.RandomID), diff --git a/pkg/ddl/schema.go b/pkg/ddl/schema.go index d5f5001db55f1..ad34f3d92ea76 100644 --- a/pkg/ddl/schema.go +++ b/pkg/ddl/schema.go @@ -230,15 +230,14 @@ func onDropSchema(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, } func (w *worker) onRecoverSchema(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) { - var ( - recoverSchemaInfo *RecoverSchemaInfo - recoverSchemaCheckFlag int64 - ) - if err := job.DecodeArgs(&recoverSchemaInfo, &recoverSchemaCheckFlag); err != nil { + args, err := model.GetRecoverArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } + recoverSchemaInfo := args.RecoverInfo + schemaInfo := recoverSchemaInfo.DBInfo // check GC and safe point gcEnable, err := checkGCEnable(w) @@ -251,10 +250,12 @@ func (w *worker) onRecoverSchema(jobCtx *jobContext, t *meta.Meta, job *model.Jo // none -> write only // check GC enable and update flag. if gcEnable { - job.Args[checkFlagIndexInJobArgs] = recoverCheckFlagEnableGC + args.CheckFlag = recoverCheckFlagEnableGC } else { - job.Args[checkFlagIndexInJobArgs] = recoverCheckFlagDisableGC + args.CheckFlag = recoverCheckFlagDisableGC } + job.FillArgs(args) + schemaInfo.State = model.StateWriteOnly job.SchemaState = model.StateWriteOnly case model.StateWriteOnly: @@ -268,7 +269,7 @@ func (w *worker) onRecoverSchema(jobCtx *jobContext, t *meta.Meta, job *model.Jo } } - recoverTbls := recoverSchemaInfo.RecoverTabsInfo + recoverTbls := recoverSchemaInfo.RecoverTableInfos if recoverSchemaInfo.LoadTablesOnExecute { sid := recoverSchemaInfo.DBInfo.ID snap := w.store.GetSnapshot(kv.NewVersion(recoverSchemaInfo.SnapshotTS)) @@ -278,14 +279,14 @@ func (w *worker) onRecoverSchema(jobCtx *jobContext, t *meta.Meta, job *model.Jo job.State = model.JobStateCancelled return ver, errors.Trace(err2) } - recoverTbls = make([]*RecoverInfo, 0, len(tables)) + recoverTbls = make([]*model.RecoverTableInfo, 0, len(tables)) for _, tblInfo := range tables { autoIDs, err3 := snapMeta.GetAutoIDAccessors(sid, tblInfo.ID).Get() if err3 != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err3) } - recoverTbls = append(recoverTbls, &RecoverInfo{ + recoverTbls = append(recoverTbls, &model.RecoverTableInfo{ SchemaID: sid, TableInfo: tblInfo, DropJobID: recoverSchemaInfo.DropJobID, diff --git a/pkg/ddl/schema_version.go b/pkg/ddl/schema_version.go index b0941b56bc290..6f35d0e60944d 100644 --- a/pkg/ddl/schema_version.go +++ b/pkg/ddl/schema_version.go @@ -266,17 +266,11 @@ func SetSchemaDiffForCreateTable(diff *model.SchemaDiff, job *model.Job) error { // SetSchemaDiffForRecoverSchema set SchemaDiff for ActionRecoverSchema. func SetSchemaDiffForRecoverSchema(diff *model.SchemaDiff, job *model.Job) error { - var ( - recoverSchemaInfo *RecoverSchemaInfo - recoverSchemaCheckFlag int64 - ) - err := job.DecodeArgs(&recoverSchemaInfo, &recoverSchemaCheckFlag) + args, err := model.GetRecoverArgs(job) if err != nil { return errors.Trace(err) } - // Reserved recoverSchemaCheckFlag value for gc work judgment. - job.Args[checkFlagIndexInJobArgs] = recoverSchemaCheckFlag - recoverTabsInfo := recoverSchemaInfo.RecoverTabsInfo + recoverTabsInfo := args.RecoverTableInfos() diff.AffectedOpts = make([]*model.AffectedOption, len(recoverTabsInfo)) for i := range recoverTabsInfo { diff.AffectedOpts[i] = &model.AffectedOption{ diff --git a/pkg/ddl/schematracker/checker.go b/pkg/ddl/schematracker/checker.go index c2112a12c3174..1c666a026d352 100644 --- a/pkg/ddl/schematracker/checker.go +++ b/pkg/ddl/schematracker/checker.go @@ -233,7 +233,7 @@ func (d *Checker) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) } // RecoverSchema implements the DDL interface. -func (*Checker) RecoverSchema(_ sessionctx.Context, _ *ddl.RecoverSchemaInfo) (err error) { +func (*Checker) RecoverSchema(_ sessionctx.Context, _ *model.RecoverSchemaInfo) (err error) { return nil } @@ -288,7 +288,7 @@ func (d *Checker) DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (er } // RecoverTable implements the DDL interface. -func (*Checker) RecoverTable(_ sessionctx.Context, _ *ddl.RecoverInfo) (err error) { +func (*Checker) RecoverTable(_ sessionctx.Context, _ *model.RecoverTableInfo) (err error) { //TODO implement me panic("implement me") } diff --git a/pkg/ddl/schematracker/dm_tracker.go b/pkg/ddl/schematracker/dm_tracker.go index 9f90ce5c8c396..c74b42583fd75 100644 --- a/pkg/ddl/schematracker/dm_tracker.go +++ b/pkg/ddl/schematracker/dm_tracker.go @@ -319,7 +319,7 @@ func (d *SchemaTracker) DropTable(_ sessionctx.Context, stmt *ast.DropTableStmt) } // RecoverTable implements the DDL interface, which is no-op in DM's case. -func (*SchemaTracker) RecoverTable(_ sessionctx.Context, _ *ddl.RecoverInfo) (err error) { +func (*SchemaTracker) RecoverTable(_ sessionctx.Context, _ *model.RecoverTableInfo) (err error) { return nil } @@ -329,7 +329,7 @@ func (*SchemaTracker) FlashbackCluster(_ sessionctx.Context, _ uint64) (err erro } // RecoverSchema implements the DDL interface, which is no-op in DM's case. -func (*SchemaTracker) RecoverSchema(_ sessionctx.Context, _ *ddl.RecoverSchemaInfo) (err error) { +func (*SchemaTracker) RecoverSchema(_ sessionctx.Context, _ *model.RecoverSchemaInfo) (err error) { return nil } diff --git a/pkg/ddl/sequence.go b/pkg/ddl/sequence.go index 76fe7c60320e5..9e4db966619e4 100644 --- a/pkg/ddl/sequence.go +++ b/pkg/ddl/sequence.go @@ -234,15 +234,13 @@ func alterSequenceOptions(sequenceOptions []*ast.SequenceOption, ident ast.Ident func onAlterSequence(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) { schemaID := job.SchemaID - var ( - sequenceOpts []*ast.SequenceOption - ident ast.Ident - ) - if err := job.DecodeArgs(&ident, &sequenceOpts); err != nil { + args, err := model.GetAlterSequenceArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } + ident, sequenceOpts := args.Ident, args.SeqOptions // Get the old tableInfo. tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, schemaID) diff --git a/pkg/ddl/table.go b/pkg/ddl/table.go index 1913b7577c5cc..89b2fa76a27b6 100644 --- a/pkg/ddl/table.go +++ b/pkg/ddl/table.go @@ -138,15 +138,13 @@ func onDropTableOrView(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver in } func (w *worker) onRecoverTable(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, err error) { - var ( - recoverInfo *RecoverInfo - recoverTableCheckFlag int64 - ) - if err = job.DecodeArgs(&recoverInfo, &recoverTableCheckFlag); err != nil { + args, err := model.GetRecoverArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } + recoverInfo := args.RecoverTableInfos()[0] schemaID := recoverInfo.SchemaID tblInfo := recoverInfo.TableInfo @@ -199,10 +197,11 @@ func (w *worker) onRecoverTable(jobCtx *jobContext, t *meta.Meta, job *model.Job // none -> write only // check GC enable and update flag. if gcEnable { - job.Args[checkFlagIndexInJobArgs] = recoverCheckFlagEnableGC + args.CheckFlag = recoverCheckFlagEnableGC } else { - job.Args[checkFlagIndexInJobArgs] = recoverCheckFlagDisableGC + args.CheckFlag = recoverCheckFlagDisableGC } + job.FillArgs(args) job.SchemaState = model.StateWriteOnly tblInfo.State = model.StateWriteOnly @@ -243,7 +242,7 @@ func (w *worker) onRecoverTable(jobCtx *jobContext, t *meta.Meta, job *model.Job return ver, nil } -func (w *worker) recoverTable(t *meta.Meta, job *model.Job, recoverInfo *RecoverInfo) (ver int64, err error) { +func (w *worker) recoverTable(t *meta.Meta, job *model.Job, recoverInfo *model.RecoverTableInfo) (ver int64, err error) { var tids []int64 if recoverInfo.TableInfo.GetPartitionInfo() != nil { tids = getPartitionIDs(recoverInfo.TableInfo) @@ -288,6 +287,8 @@ func (w *worker) recoverTable(t *meta.Meta, job *model.Job, recoverInfo *Recover job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to update the label rule to PD") } + + // TODO(joechenrh): tid is used in SerSchemaDiffForDropTable, remove this after refactor done. job.CtxVars = []any{tids} return ver, nil } @@ -659,8 +660,8 @@ func onRebaseAutoID(jobCtx *jobContext, t *meta.Meta, job *model.Job, tp autoid. } func onModifyTableAutoIDCache(jobCtx *jobContext, t *meta.Meta, job *model.Job) (int64, error) { - var cache int64 - if err := job.DecodeArgs(&cache); err != nil { + args, err := model.GetModifyTableAutoIDCacheArgs(job) + if err != nil { job.State = model.JobStateCancelled return 0, errors.Trace(err) } @@ -670,7 +671,7 @@ func onModifyTableAutoIDCache(jobCtx *jobContext, t *meta.Meta, job *model.Job) return 0, errors.Trace(err) } - tblInfo.AutoIDCache = cache + tblInfo.AutoIDCache = args.NewCache ver, err := updateVersionAndTableInfo(jobCtx, t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) @@ -680,12 +681,13 @@ func onModifyTableAutoIDCache(jobCtx *jobContext, t *meta.Meta, job *model.Job) } func (w *worker) onShardRowID(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) { - var shardRowIDBits uint64 - err := job.DecodeArgs(&shardRowIDBits) + args, err := model.GetShardRowIDArgs(job) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } + + shardRowIDBits := args.ShardRowIDBits tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { job.State = model.JobStateCancelled @@ -1337,18 +1339,19 @@ type schemaIDAndTableInfo struct { func onRepairTable(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) { schemaID := job.SchemaID - tblInfo := &model.TableInfo{} - if err := job.DecodeArgs(tblInfo); err != nil { + args, err := model.GetRepairTableArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } + tblInfo := args.TableInfo tblInfo.State = model.StateNone // Check the old DB and old table exist. - _, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + _, err = GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { return ver, errors.Trace(err) } diff --git a/pkg/ddl/table_lock.go b/pkg/ddl/table_lock.go index a2643bf937f67..ef6d6e6a13fae 100644 --- a/pkg/ddl/table_lock.go +++ b/pkg/ddl/table_lock.go @@ -24,28 +24,28 @@ import ( ) func onLockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, err error) { - arg := &LockTablesArg{} - if err := job.DecodeArgs(arg); err != nil { + args, err := model.GetLockTablesArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } // Unlock table first. - if arg.IndexOfUnlock < len(arg.UnlockTables) { - return unlockTables(jobCtx, t, job, arg) + if args.IndexOfUnlock < len(args.UnlockTables) { + return unlockTables(jobCtx, t, job, args) } // Check table locked by other, this can be only checked at the first time. - if arg.IndexOfLock == 0 { - for i, tl := range arg.LockTables { + if args.IndexOfLock == 0 { + for i, tl := range args.LockTables { job.SchemaID = tl.SchemaID job.TableID = tl.TableID tbInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { return ver, err } - err = checkTableLocked(tbInfo, arg.LockTables[i].Tp, arg.SessionInfo) + err = checkTableLocked(tbInfo, args.LockTables[i].Tp, args.SessionInfo) if err != nil { // If any request table was locked by other session, just cancel this job. // No need to rolling back the unlocked tables, MySQL will release the lock first @@ -57,15 +57,15 @@ func onLockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, } // Lock tables. - if arg.IndexOfLock < len(arg.LockTables) { - job.SchemaID = arg.LockTables[arg.IndexOfLock].SchemaID - job.TableID = arg.LockTables[arg.IndexOfLock].TableID + if args.IndexOfLock < len(args.LockTables) { + job.SchemaID = args.LockTables[args.IndexOfLock].SchemaID + job.TableID = args.LockTables[args.IndexOfLock].TableID var tbInfo *model.TableInfo tbInfo, err = GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { return ver, err } - err = lockTable(tbInfo, arg.IndexOfLock, arg) + err = lockTable(tbInfo, args.IndexOfLock, args) if err != nil { job.State = model.JobStateCancelled return ver, err @@ -86,9 +86,9 @@ func onLockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, if err != nil { return ver, errors.Trace(err) } - arg.IndexOfLock++ - job.Args = []any{arg} - if arg.IndexOfLock == len(arg.LockTables) { + args.IndexOfLock++ + job.FillArgs(args) + if args.IndexOfLock == len(args.LockTables) { // Finish this job. job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, nil) } @@ -112,12 +112,12 @@ func findSessionInfoIndex(sessions []model.SessionInfo, sessionInfo model.Sessio } // lockTable uses to check table locked and acquire the table lock for the request session. -func lockTable(tbInfo *model.TableInfo, idx int, arg *LockTablesArg) error { +func lockTable(tbInfo *model.TableInfo, idx int, args *model.LockTablesArgs) error { if !tbInfo.IsLocked() { tbInfo.Lock = &model.TableLockInfo{ - Tp: arg.LockTables[idx].Tp, + Tp: args.LockTables[idx].Tp, } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, args.SessionInfo) return nil } // If the state of the lock is in pre-lock, then the lock must be locked by the current request. So we can just return here. @@ -125,14 +125,14 @@ func lockTable(tbInfo *model.TableInfo, idx int, arg *LockTablesArg) error { if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if (tbInfo.Lock.Tp == pmodel.TableLockRead && arg.LockTables[idx].Tp == pmodel.TableLockRead) || - (tbInfo.Lock.Tp == pmodel.TableLockReadOnly && arg.LockTables[idx].Tp == pmodel.TableLockReadOnly) { - sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) + if (tbInfo.Lock.Tp == pmodel.TableLockRead && args.LockTables[idx].Tp == pmodel.TableLockRead) || + (tbInfo.Lock.Tp == pmodel.TableLockReadOnly && args.LockTables[idx].Tp == pmodel.TableLockReadOnly) { + sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, args.SessionInfo) // repeat lock. if sessionIndex >= 0 { return nil } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, args.SessionInfo) return nil } @@ -168,24 +168,24 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp pmodel.TableLockType, sess } // unlockTables uses unlock a batch of table lock one by one. -func unlockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job, arg *LockTablesArg) (ver int64, err error) { - if arg.IndexOfUnlock >= len(arg.UnlockTables) { +func unlockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job, args *model.LockTablesArgs) (ver int64, err error) { + if args.IndexOfUnlock >= len(args.UnlockTables) { return ver, nil } - job.SchemaID = arg.UnlockTables[arg.IndexOfUnlock].SchemaID - job.TableID = arg.UnlockTables[arg.IndexOfUnlock].TableID + job.SchemaID = args.UnlockTables[args.IndexOfUnlock].SchemaID + job.TableID = args.UnlockTables[args.IndexOfUnlock].TableID tbInfo, err := getTableInfo(t, job.TableID, job.SchemaID) if err != nil { if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { // The table maybe has been dropped. just ignore this err and go on. - arg.IndexOfUnlock++ - job.Args = []any{arg} + args.IndexOfUnlock++ + job.FillArgs(args) return ver, nil } return ver, err } - needUpdateTableInfo := unlockTable(tbInfo, arg) + needUpdateTableInfo := unlockTable(tbInfo, args) if needUpdateTableInfo { ver, err = updateVersionAndTableInfo(jobCtx, t, job, tbInfo, true) if err != nil { @@ -193,22 +193,22 @@ func unlockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job, arg *LockTab } } - arg.IndexOfUnlock++ - job.Args = []any{arg} + args.IndexOfUnlock++ + job.FillArgs(args) return ver, nil } // unlockTable uses to unlock table lock that hold by the session. -func unlockTable(tbInfo *model.TableInfo, arg *LockTablesArg) (needUpdateTableInfo bool) { +func unlockTable(tbInfo *model.TableInfo, args *model.LockTablesArgs) (needUpdateTableInfo bool) { if !tbInfo.IsLocked() { return false } - if arg.IsCleanup { + if args.IsCleanup { tbInfo.Lock = nil return true } - sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) + sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, args.SessionInfo) if sessionIndex < 0 { // When session clean table lock, session maybe send unlock table even the table lock maybe not hold by the session. // so just ignore and return here. @@ -224,15 +224,15 @@ func unlockTable(tbInfo *model.TableInfo, arg *LockTablesArg) (needUpdateTableIn } func onUnlockTables(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, err error) { - arg := &LockTablesArg{} - if err := job.DecodeArgs(arg); err != nil { + args, err := model.GetLockTablesArgs(job) + if err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } - ver, err = unlockTables(jobCtx, t, job, arg) - if arg.IndexOfUnlock == len(arg.UnlockTables) { + ver, err = unlockTables(jobCtx, t, job, args) + if args.IndexOfUnlock == len(args.UnlockTables) { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nil) } return ver, err diff --git a/pkg/ddl/table_test.go b/pkg/ddl/table_test.go index 78539632c4835..9a8157e3ef8de 100644 --- a/pkg/ddl/table_test.go +++ b/pkg/ddl/table_test.go @@ -115,7 +115,7 @@ func testLockTable( tblInfo *model.TableInfo, lockTp pmodel.TableLockType, ) *model.Job { - arg := &ddl.LockTablesArg{ + args := &model.LockTablesArgs{ LockTables: []model.TableLockTpInfo{{SchemaID: newSchemaID, TableID: tblInfo.ID, Tp: lockTp}}, SessionInfo: model.SessionInfo{ ServerID: uuid, @@ -123,17 +123,17 @@ func testLockTable( }, } job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: newSchemaID, TableID: tblInfo.ID, Type: model.ActionLockTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ {Database: schemaName.L, Table: tblInfo.Name.L}, }, } ctx.SetValue(sessionctx.QueryString, "skip") - err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapper(job, true)) + err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, args, true)) require.NoError(t, err) v := getSchemaVer(t, ctx) @@ -403,17 +403,17 @@ func testAlterCacheTable( tblInfo *model.TableInfo, ) *model.Job { job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: newSchemaID, TableID: tblInfo.ID, Type: model.ActionAlterCacheTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{}, InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ {Database: newSchemaName.L, Table: tblInfo.Name.L}, }, } ctx.SetValue(sessionctx.QueryString, "skip") - err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapper(job, true)) + err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, &model.EmptyArgs{}, true)) require.NoError(t, err) v := getSchemaVer(t, ctx) @@ -430,17 +430,17 @@ func testAlterNoCacheTable( tblInfo *model.TableInfo, ) *model.Job { job := &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: newSchemaID, TableID: tblInfo.ID, Type: model.ActionAlterNoCacheTable, BinlogInfo: &model.HistoryInfo{}, - Args: []any{}, InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ {Database: newSchemaName.L, Table: tblInfo.Name.L}, }, } ctx.SetValue(sessionctx.QueryString, "skip") - require.NoError(t, d.DoDDLJobWrapper(ctx, ddl.NewJobWrapper(job, true))) + require.NoError(t, d.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, &model.EmptyArgs{}, true))) v := getSchemaVer(t, ctx) checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v}) @@ -577,20 +577,23 @@ func TestAlterTTL(t *testing.T) { } job = &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: dbInfo.ID, SchemaName: dbInfo.Name.L, TableID: tblInfo.ID, TableName: tblInfo.Name.L, Type: model.ActionAlterTTLInfo, BinlogInfo: &model.HistoryInfo{}, - Args: []any{&model.TTLInfo{ + } + ctx.SetValue(sessionctx.QueryString, "skip") + args := &model.AlterTTLInfoArgs{ + TTLInfo: &model.TTLInfo{ ColumnName: tblInfo.Columns[1].Name, IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitYear), - }}, + }, } - ctx.SetValue(sessionctx.QueryString, "skip") - require.NoError(t, de.DoDDLJobWrapper(ctx, ddl.NewJobWrapper(job, true))) + require.NoError(t, de.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, args, true))) v := getSchemaVer(t, ctx) checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: nil}) @@ -602,16 +605,16 @@ func TestAlterTTL(t *testing.T) { // submit a ddl job to modify ttlEnabled job = &model.Job{ + Version: model.GetJobVerInUse(), SchemaID: dbInfo.ID, SchemaName: dbInfo.Name.L, TableID: tblInfo.ID, TableName: tblInfo.Name.L, Type: model.ActionAlterTTLRemove, BinlogInfo: &model.HistoryInfo{}, - Args: []any{true}, } ctx.SetValue(sessionctx.QueryString, "skip") - require.NoError(t, de.DoDDLJobWrapper(ctx, ddl.NewJobWrapper(job, true))) + require.NoError(t, de.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, &model.EmptyArgs{}, true))) v = getSchemaVer(t, ctx) checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: nil}) diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index f6fe1952ee41a..52d8511096ccf 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -275,7 +275,6 @@ func TestCreateTableWithPartition(t *testing.T) { partition p4 values less than (18446744073709551000 + 10) );`) - tk.MustExec("set @@tidb_enable_table_partition = 1") tk.MustExec(`create table t30 ( a int, b varchar(20), @@ -349,7 +348,6 @@ func TestCreateTableWithHashPartition(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists employees;") - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec(` create table employees ( id int not null, @@ -2251,7 +2249,6 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { // Test truncate table partition reassigns new partitionIDs. tk.MustExec("drop table if exists t5;") - tk.MustExec("set @@session.tidb_enable_table_partition=1;") tk.MustExec(`create table t5( id int, name varchar(50), purchased date @@ -2276,7 +2273,6 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { newPID := newTblInfo.Meta().Partition.Definitions[0].ID require.True(t, oldPID != newPID) - tk.MustExec("set @@session.tidb_enable_table_partition = 1;") tk.MustExec("drop table if exists clients;") tk.MustExec(`create table clients ( id int, @@ -2392,7 +2388,6 @@ func testPartitionAddIndexOrPK(t *testing.T, tk *testkit.TestKit, key string) { testPartitionAddIndex(tk, t, key) // test hash partition table. - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") tk.MustExec("drop table if exists partition_add_idx") tk.MustExec(`create table partition_add_idx ( id int not null, @@ -2403,7 +2398,6 @@ func testPartitionAddIndexOrPK(t *testing.T, tk *testkit.TestKit, key string) { // Test hash partition for pr 10475. tk.MustExec("drop table if exists t1") defer tk.MustExec("drop table if exists t1") - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") tk.MustExec("create table t1 (a int, b int, unique key(a)) partition by hash(a) partitions 5;") tk.MustExec("insert into t1 values (0,0),(1,1),(2,2),(3,3);") tk.MustExec(fmt.Sprintf("alter table t1 add %s idx(a)", key)) @@ -2549,7 +2543,6 @@ func TestPartitionErrorCode(t *testing.T) { tk1 := testkit.NewTestKit(t, store) // add partition - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec("drop database if exists test_db_with_partition") tk.MustExec("create database test_db_with_partition") tk.MustExec("use test_db_with_partition") diff --git a/pkg/ddl/ttl.go b/pkg/ddl/ttl.go index 7b130bee1718d..cc993842c492c 100644 --- a/pkg/ddl/ttl.go +++ b/pkg/ddl/ttl.go @@ -51,14 +51,12 @@ func onTTLInfoRemove(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int6 func onTTLInfoChange(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, err error) { // at least one for them is not nil - var ttlInfo *model.TTLInfo - var ttlInfoEnable *bool - var ttlInfoJobInterval *string - - if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable, &ttlInfoJobInterval); err != nil { + args, err := model.GetAlterTTLInfoArgs(job) + if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } + ttlInfo, ttlInfoEnable, ttlInfoJobInterval := args.TTLInfo, args.TTLEnable, args.TTLCronJobSchedule tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index d69072538296f..ba6eb9a3c36b0 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -249,7 +249,6 @@ go_library( "@com_github_docker_go_units//:go-units", "@com_github_gogo_protobuf//proto", "@com_github_google_uuid//:uuid", - "@com_github_ngaut_pools//:pools", "@com_github_opentracing_basictracer_go//:basictracer-go", "@com_github_opentracing_opentracing_go//:opentracing-go", "@com_github_pingcap_errors//:errors", diff --git a/pkg/executor/ddl.go b/pkg/executor/ddl.go index 1d5263871f2c3..24038b07db878 100644 --- a/pkg/executor/ddl.go +++ b/pkg/executor/ddl.go @@ -422,7 +422,7 @@ func (e *DDLExec) executeRecoverTable(s *ast.RecoverTableStmt) error { return err } - recoverInfo := &ddl.RecoverInfo{ + recoverInfo := &model.RecoverTableInfo{ SchemaID: job.SchemaID, TableInfo: tblInfo, DropJobID: job.ID, @@ -473,9 +473,11 @@ func (e *DDLExec) getRecoverTableByJobID(s *ast.RecoverTableStmt, dom *domain.Do fmt.Sprintf("(Table ID %d)", job.TableID), ) } - // Return the cloned meta here, since meta will be modified later. - // This may corrupt the infocache. - return job, table.Meta().Clone(), nil + // We can't return the meta directly since it will be modified outside, which may corrupt the infocache. + // Since only State field is changed, return a shallow copy is enough. + // see https://github.com/pingcap/tidb/issues/55462 + tblInfo := *table.Meta() + return job, &tblInfo, nil } // GetDropOrTruncateTableInfoFromJobs gets the dropped/truncated table information from DDL jobs, @@ -540,7 +542,12 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J if tableInfo.TempTableType == model.TempTableGlobal { return nil, nil, exeerrors.ErrUnsupportedFlashbackTmpTable } - return jobInfo, tableInfo, nil + + // We can't return the meta directly since it will be modified outside, which may corrupt the infocache. + // Since only State field is changed, return a shallow copy is enough. + // see https://github.com/pingcap/tidb/issues/55462 + tblInfo := *tableInfo + return jobInfo, &tblInfo, nil } func (e *DDLExec) executeFlashBackCluster(s *ast.FlashBackToTimestampStmt) error { @@ -579,7 +586,7 @@ func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { return err } - recoverInfo := &ddl.RecoverInfo{ + recoverInfo := &model.RecoverTableInfo{ SchemaID: job.SchemaID, TableInfo: tblInfo, DropJobID: job.ID, @@ -620,7 +627,7 @@ func (e *DDLExec) executeFlashbackDatabase(s *ast.FlashBackDatabaseStmt) error { return err } -func (e *DDLExec) getRecoverDBByName(schemaName pmodel.CIStr) (recoverSchemaInfo *ddl.RecoverSchemaInfo, err error) { +func (e *DDLExec) getRecoverDBByName(schemaName pmodel.CIStr) (recoverSchemaInfo *model.RecoverSchemaInfo, err error) { txn, err := e.Ctx().Txn(true) if err != nil { return nil, err @@ -654,7 +661,7 @@ func (e *DDLExec) getRecoverDBByName(schemaName pmodel.CIStr) (recoverSchemaInfo if schemaInfo.Name.L != schemaName.L { continue } - recoverSchemaInfo = &ddl.RecoverSchemaInfo{ + recoverSchemaInfo = &model.RecoverSchemaInfo{ DBInfo: schemaInfo, LoadTablesOnExecute: true, DropJobID: job.ID, diff --git a/pkg/executor/show_placement.go b/pkg/executor/show_placement.go index b31bf6d564861..2c877685b2ca8 100644 --- a/pkg/executor/show_placement.go +++ b/pkg/executor/show_placement.go @@ -361,19 +361,16 @@ func (e *ShowExec) fetchAllTablePlacements(ctx context.Context, scheduleState ma dbs := e.is.AllSchemaNames() slices.SortFunc(dbs, func(i, j pmodel.CIStr) int { return cmp.Compare(i.O, j.O) }) - for _, dbName := range dbs { + tbls := e.is.ListTablesWithSpecialAttribute(infoschema.AllSpecialAttribute) + for _, db := range tbls { tableRowSets := make([]tableRowSet, 0) - tblInfos, err := e.is.SchemaTableInfos(ctx, dbName) - if err != nil { - return errors.Trace(err) - } - for _, tblInfo := range tblInfos { - if checker != nil && !checker.RequestVerification(activeRoles, dbName.O, tblInfo.Name.O, "", mysql.AllPrivMask) { + for _, tblInfo := range db.TableInfos { + if checker != nil && !checker.RequestVerification(activeRoles, db.DBName.O, tblInfo.Name.O, "", mysql.AllPrivMask) { continue } var rows [][]any - ident := ast.Ident{Schema: dbName, Name: tblInfo.Name} + ident := ast.Ident{Schema: db.DBName, Name: tblInfo.Name} tblPlacement, err := e.getTablePlacement(tblInfo) if err != nil { return err diff --git a/pkg/executor/show_placement_test.go b/pkg/executor/show_placement_test.go index d6c7f4c10bcb7..178f517d1bf57 100644 --- a/pkg/executor/show_placement_test.go +++ b/pkg/executor/show_placement_test.go @@ -175,9 +175,9 @@ func TestShowPlacementPrivilege(t *testing.T) { tk1.MustQuery("show placement").Check(testkit.Rows( "POLICY p1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" NULL", "DATABASE db2 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" PENDING", - "TABLE db2.t1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" PENDING", "TABLE test.t1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" PENDING", "TABLE test.t3 PARTITION p1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" PENDING", + "TABLE db2.t1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" PENDING", )) } diff --git a/pkg/executor/show_stats_test.go b/pkg/executor/show_stats_test.go index 7d2628280036e..94bb106064e75 100644 --- a/pkg/executor/show_stats_test.go +++ b/pkg/executor/show_stats_test.go @@ -235,7 +235,6 @@ func TestShowPartitionStats(t *testing.T) { tk := testkit.NewTestKit(t, store) testkit.WithPruneMode(tk, variable.Static, func() { - tk.MustExec("set @@session.tidb_enable_table_partition=1") // Version2 is tested in TestGlobalStatsData1/2/3 and TestAnalyzeGlobalStatsWithOpts. tk.MustExec("set @@session.tidb_analyze_version=1") tk.MustExec("use test") diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 07399831cc85e..4e3872b48fc1e 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -26,7 +26,6 @@ import ( "syscall" "time" - "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/placement" @@ -124,16 +123,6 @@ type userInfo struct { authString string } -// clearSysSession close the session does not return the session. -// Since the environment variables in the session are changed, the session object is not returned. -func clearSysSession(ctx context.Context, sctx sessionctx.Context) { - if sctx == nil { - return - } - _, _ = sctx.GetSQLExecutor().ExecuteInternal(ctx, "rollback") - sctx.(pools.Resource).Close() -} - // Next implements the Executor Next interface. func (e *SimpleExec) Next(ctx context.Context, _ *chunk.Chunk) (err error) { if e.done { @@ -1731,10 +1720,10 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) } sysSession, err := e.GetSysSession() - defer clearSysSession(ctx, sysSession) if err != nil { return err } + defer e.ReleaseSysSession(ctx, sysSession) sqlExecutor := sysSession.GetSQLExecutor() // session isolation level changed to READ-COMMITTED. // When tidb is at the RR isolation level, executing `begin` will obtain a consistent state. @@ -2461,10 +2450,10 @@ func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, na func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnPrivilege) sysSession, err := e.GetSysSession() - defer clearSysSession(ctx, sysSession) if err != nil { return err } + defer e.ReleaseSysSession(ctx, sysSession) sqlExecutor := sysSession.GetSQLExecutor() // session isolation level changed to READ-COMMITTED. diff --git a/pkg/executor/sortexec/parallel_sort_spill_test.go b/pkg/executor/sortexec/parallel_sort_spill_test.go index 44378dcf0613b..0b0791b0a7fda 100644 --- a/pkg/executor/sortexec/parallel_sort_spill_test.go +++ b/pkg/executor/sortexec/parallel_sort_spill_test.go @@ -81,9 +81,9 @@ func TestParallelSortSpillDisk(t *testing.T) { ctx := mock.NewContext() sortCase := &testutil.SortCase{Rows: 10000, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") ctx.GetSessionVars().InitChunkSize = 32 @@ -113,13 +113,13 @@ func TestParallelSortSpillDiskFailpoint(t *testing.T) { ctx := mock.NewContext() sortCase := &testutil.SortCase{Rows: 10000, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail") - failpoint.Enable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError", `return(true)`)) defer failpoint.Disable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError") ctx.GetSessionVars().InitChunkSize = 32 diff --git a/pkg/executor/sortexec/parallel_sort_test.go b/pkg/executor/sortexec/parallel_sort_test.go index 0fda9162f6495..8830f58031c3b 100644 --- a/pkg/executor/sortexec/parallel_sort_test.go +++ b/pkg/executor/sortexec/parallel_sort_test.go @@ -116,8 +116,10 @@ func TestParallelSort(t *testing.T) { rowNum := 30000 nvd := 100 // we have two column and should ensure that nvd*nvd is less than rowNum. sortCase := &testutil.SortCase{Rows: rowNum, OrderByIdx: []int{0, 1}, Ndvs: []int{nvd, nvd}, Ctx: ctx} - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`) - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") schema := expression.NewSchema(sortCase.Columns()...) dataSource := buildDataSource(sortCase, schema) @@ -134,9 +136,12 @@ func TestFailpoint(t *testing.T) { sortCase := &testutil.SortCase{Rows: rowNum, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} schema := expression.NewSchema(sortCase.Columns()...) dataSource := buildDataSource(sortCase, schema) - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail", `return(true)`) - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`) - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") testNum := 30 exe := buildSortExec(sortCase, dataSource) diff --git a/pkg/executor/sortexec/sort_spill_test.go b/pkg/executor/sortexec/sort_spill_test.go index a32c183590f5c..0546fd72dce55 100644 --- a/pkg/executor/sortexec/sort_spill_test.go +++ b/pkg/executor/sortexec/sort_spill_test.go @@ -350,7 +350,8 @@ func TestUnparallelSortSpillDisk(t *testing.T) { ctx := mock.NewContext() sortCase := &testutil.SortCase{Rows: 2048, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") for i := 0; i < 50; i++ { onePartitionAndAllDataInMemoryCase(t, ctx, sortCase) @@ -376,7 +377,8 @@ func TestFallBackAction(t *testing.T) { ctx.GetSessionVars().StmtCtx.MemTracker.AttachTo(ctx.GetSessionVars().MemTracker) sortCase := &testutil.SortCase{Rows: 2048, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} - failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SignalCheckpointForSort") schema := expression.NewSchema(sortCase.Columns()...) dataSource := buildDataSource(sortCase, schema) diff --git a/pkg/executor/sortexec/topn_spill_test.go b/pkg/executor/sortexec/topn_spill_test.go index 179d016dd1667..159e33e7c0876 100644 --- a/pkg/executor/sortexec/topn_spill_test.go +++ b/pkg/executor/sortexec/topn_spill_test.go @@ -435,9 +435,13 @@ func TestTopNSpillDiskFailpoint(t *testing.T) { topNCase := &testutil.SortCase{Rows: totalRowNum, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/TopNRandomFail", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/TopNRandomFail") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError", `return(true)`)) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError") ctx.GetSessionVars().InitChunkSize = 32 ctx.GetSessionVars().MaxChunkSize = 32 @@ -474,11 +478,6 @@ func TestTopNSpillDiskFailpoint(t *testing.T) { topNFailPointTest(t, nil, topNCase, dataSource, 0, count, inMemoryThenSpillHardLimit, ctx.GetSessionVars().MemTracker) topNFailPointTest(t, exe, topNCase, dataSource, offset, count, inMemoryThenSpillHardLimit, ctx.GetSessionVars().MemTracker) } - - failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/SlowSomeWorkers") - failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/TopNRandomFail") - failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/ParallelSortRandomFail") - failpoint.Disable("github.com/pingcap/tidb/pkg/util/chunk/ChunkInDiskError") } func TestIssue54206(t *testing.T) { diff --git a/pkg/executor/test/splittest/split_table_test.go b/pkg/executor/test/splittest/split_table_test.go index 339fc029d5c9d..938df7a5ca6ed 100644 --- a/pkg/executor/test/splittest/split_table_test.go +++ b/pkg/executor/test/splittest/split_table_test.go @@ -215,7 +215,6 @@ func TestShowTableRegion(t *testing.T) { // Test show table regions for partition table when disable split region when create table. atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) tk.MustExec("drop table if exists partition_t;") - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3") re = tk.MustQuery("show table partition_t regions") rows = re.Rows() diff --git a/pkg/executor/test/tiflashtest/BUILD.bazel b/pkg/executor/test/tiflashtest/BUILD.bazel index 7713977d4ca95..cbba6bb46361a 100644 --- a/pkg/executor/test/tiflashtest/BUILD.bazel +++ b/pkg/executor/test/tiflashtest/BUILD.bazel @@ -9,7 +9,7 @@ go_test( ], flaky = True, race = "on", - shard_count = 44, + shard_count = 45, deps = [ "//pkg/config", "//pkg/domain", diff --git a/pkg/executor/test/tiflashtest/tiflash_test.go b/pkg/executor/test/tiflashtest/tiflash_test.go index e19bcea99a403..e210f120e6a70 100644 --- a/pkg/executor/test/tiflashtest/tiflash_test.go +++ b/pkg/executor/test/tiflashtest/tiflash_test.go @@ -2117,3 +2117,35 @@ func TestMppTableReaderCacheForSingleSQL(t *testing.T) { require.Equal(t, tc.expectMissNum, missNum.Load()) } } + +func TestIndexMergeCarePreferTiflash(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(1)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` (" + + "`i` bigint(20) NOT NULL, " + + "`w` varchar(32) NOT NULL," + + "`l` varchar(32) NOT NULL," + + "`a` tinyint(4) NOT NULL DEFAULT '0'," + + "`m` int(11) NOT NULL DEFAULT '0'," + + "`s` int(11) NOT NULL DEFAULT '0'," + + "PRIMARY KEY (`i`) /*T![clustered_index] NONCLUSTERED */," + + "KEY `idx_win_user_site_code` (`w`,`m`)," + + "KEY `idx_lose_user_site_code` (`l`,`m`)," + + "KEY `idx_win_site_code_status` (`w`,`a`)," + + "KEY `idx_lose_site_code_status` (`l`,`a`)" + + ")") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustQuery("explain format=\"brief\" SELECT" + + " /*+ read_from_storage(tiflash[a]) */ a.i FROM t a WHERE a.s = 0 AND a.a NOT IN (-1, 0) AND m >= 1726910326 AND m <= 1726910391 AND ( a.w IN ('1123') OR a.l IN ('1123'))").Check( + testkit.Rows("TableReader 0.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 0.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 0.00 mpp[tiflash] test.t.i", + " └─Selection 0.00 mpp[tiflash] ge(test.t.m, 1726910326), le(test.t.m, 1726910391), not(in(test.t.a, -1, 0)), or(eq(test.t.w, \"1123\"), eq(test.t.l, \"1123\"))", + " └─TableFullScan 10.00 mpp[tiflash] table:a pushed down filter:eq(test.t.s, 0), keep order:false, stats:pseudo")) +} diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index 7f6a8fc268253..27630916979db 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -274,7 +274,7 @@ func updateAutoIDForExchangePartition(store kv.Storage, ptSchemaID, ptID, ntSche } // Set both tables to the maximum auto IDs between normal table and partitioned table. - newAutoIDs := meta.AutoIDGroup{ + newAutoIDs := model.AutoIDGroup{ RowID: max(ptAutoIDs.RowID, ntAutoIDs.RowID), IncrementID: max(ptAutoIDs.IncrementID, ntAutoIDs.IncrementID), RandomID: max(ptAutoIDs.RandomID, ntAutoIDs.RandomID), diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index 2f7b611c05f13..64e534ed55a3b 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -308,6 +308,19 @@ func (is *infoSchema) TableByID(_ stdctx.Context, id int64) (val table.Table, ok return slice[idx], true } +func (is *infoSchema) SchemaNameByTableID(tableID int64) (schemaName pmodel.CIStr, ok bool) { + tbl, ok := is.TableByID(stdctx.Background(), tableID) + if !ok { + return + } + db, ok := is.SchemaByID(tbl.Meta().DBID) + if !ok { + return + } + + return db.Name, true +} + // TableInfoByID implements InfoSchema.TableInfoByID func (is *infoSchema) TableInfoByID(id int64) (*model.TableInfo, bool) { tbl, ok := is.TableByID(stdctx.Background(), id) diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index d79af4fc72e1e..88d570940537c 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -244,6 +244,41 @@ func TestBasic(t *testing.T) { tbls, err := is.SchemaTableInfos(context.Background(), schema.Name) require.NoError(t, err) require.Equal(t, 1, len(tbls)) + + // Test SchemaNameByTableID + tests := []struct { + name string + tableID int64 + wantSchema pmodel.CIStr + wantOK bool + }{ + { + name: "valid table ID", + tableID: tbID, + wantSchema: dbName, + wantOK: true, + }, + { + name: "non-existent table ID", + tableID: tbID + 1, + wantSchema: pmodel.CIStr{}, + wantOK: false, + }, + { + name: "invalid table ID (negative)", + tableID: -1, + wantSchema: pmodel.CIStr{}, + wantOK: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + gotSchema, gotOK := is.SchemaNameByTableID(tt.tableID) + require.Equal(t, tt.wantOK, gotOK) + require.Equal(t, tt.wantSchema, gotSchema) + }) + } } func TestMockInfoSchema(t *testing.T) { diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index 88ba5fc7d88cc..1d4f2fd3caff0 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -589,6 +589,16 @@ func (is *infoschemaV2) CloneAndUpdateTS(startTS uint64) *infoschemaV2 { return &tmp } +func (is *infoschemaV2) searchTableItemByID(tableID int64) (tableItem, bool) { + eq := func(a, b *tableItem) bool { return a.tableID == b.tableID } + return search( + is.byID, + is.infoSchema.schemaMetaVersion, + tableItem{tableID: tableID, schemaVersion: math.MaxInt64}, + eq, + ) +} + // TableByID implements the InfoSchema interface. // As opposed to TableByName, TableByID will not refill cache when schema cache miss, // unless the caller changes the behavior by passing a context use WithRefillOption. @@ -604,8 +614,7 @@ func (is *infoschemaV2) TableByID(ctx context.Context, id int64) (val table.Tabl return tbl, true } - eq := func(a, b *tableItem) bool { return a.tableID == b.tableID } - itm, ok := search(is.byID, is.infoSchema.schemaMetaVersion, tableItem{tableID: id, schemaVersion: math.MaxInt64}, eq) + itm, ok := is.searchTableItemByID(id) if !ok { return nil, false } @@ -646,6 +655,19 @@ func (is *infoschemaV2) TableByID(ctx context.Context, id int64) (val table.Tabl return ret, true } +func (is *infoschemaV2) SchemaNameByTableID(tableID int64) (schemaName pmodel.CIStr, ok bool) { + if !tableIDIsValid(tableID) { + return + } + + itm, ok := is.searchTableItemByID(tableID) + if !ok { + return + } + + return itm.dbName, true +} + // TableItem is exported from tableItem. type TableItem struct { DBName pmodel.CIStr diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index 8ba6faeff1b70..f6ad9770691c2 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -121,6 +121,43 @@ func TestV2Basic(t *testing.T) { require.Equal(t, 0, len(tblInfos)) require.Equal(t, int64(2), is.SchemaMetaVersion()) + + // Test SchemaNameByTableID + schemaNameByTableIDTests := []struct { + name string + tableID int64 + wantSchema pmodel.CIStr + wantOK bool + }{ + { + name: "valid table ID", + tableID: tblInfo.ID, + wantSchema: schemaName, + wantOK: true, + }, + { + name: "non-existent table ID", + tableID: tblInfo.ID + 1, + wantSchema: pmodel.CIStr{}, + wantOK: false, + }, + { + name: "invalid table ID (negative)", + tableID: -1, + wantSchema: pmodel.CIStr{}, + wantOK: false, + }, + } + + for _, tt := range schemaNameByTableIDTests { + t.Run(tt.name, func(t *testing.T) { + gotSchema, gotOK := is.SchemaNameByTableID(tt.tableID) + + require.Equal(t, tt.wantOK, gotOK) + require.Equal(t, tt.wantSchema, gotSchema) + }) + } + // TODO: support FindTableByPartitionID. } diff --git a/pkg/infoschema/interface.go b/pkg/infoschema/interface.go index e6922e5e1ac53..da6f15f34956b 100644 --- a/pkg/infoschema/interface.go +++ b/pkg/infoschema/interface.go @@ -30,6 +30,7 @@ type InfoSchema interface { context.MetaOnlyInfoSchema TableByName(ctx stdctx.Context, schema, table pmodel.CIStr) (table.Table, error) TableByID(ctx stdctx.Context, id int64) (table.Table, bool) + SchemaNameByTableID(tableID int64) (pmodel.CIStr, bool) FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) ListTablesWithSpecialAttribute(filter specialAttributeFilter) []tableInfoResult base() *infoSchema diff --git a/pkg/lightning/backend/local/BUILD.bazel b/pkg/lightning/backend/local/BUILD.bazel index ce561f9ef22c1..0c2f1811a34c9 100644 --- a/pkg/lightning/backend/local/BUILD.bazel +++ b/pkg/lightning/backend/local/BUILD.bazel @@ -60,6 +60,7 @@ go_library( "//pkg/util/compress", "//pkg/util/engine", "//pkg/util/hack", + "//pkg/util/intest", "//pkg/util/logutil", "//pkg/util/mathutil", "//pkg/util/ranger", diff --git a/pkg/lightning/backend/local/local.go b/pkg/lightning/backend/local/local.go index 23522f4c0ddc2..ddcdfde36a681 100644 --- a/pkg/lightning/backend/local/local.go +++ b/pkg/lightning/backend/local/local.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/engine" + "github.com/pingcap/tidb/pkg/util/intest" tikvclient "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" pdhttp "github.com/tikv/pd/client/http" @@ -867,12 +868,8 @@ func getRegionSplitKeys( return keys, err } -// prepareAndSendJob will read the engine to get estimated key range, -// then split and scatter regions for these range and send region jobs to jobToWorkerCh. -// NOTE when ctx is Done, this function will NOT return error even if it hasn't sent -// all the jobs to jobToWorkerCh. This is because the "first error" can only be -// found by checking the work group LATER, we don't want to return an error to -// seize the "first" error. +// prepareAndSendJob will read the engine to get estimated key range, then split +// and scatter regions for these range and send region jobs to jobToWorkerCh. func (local *Backend) prepareAndSendJob( ctx context.Context, engine common.Engine, @@ -982,9 +979,7 @@ func (local *Backend) generateAndSendJob( case <-egCtx.Done(): // this job is not put into jobToWorkerCh job.done(jobWg) - // if the context is canceled, it means worker has error, the first error can be - // found by worker's error group LATER. if this function returns an error it will - // seize the "first error". + // if the context is canceled, it means worker has error. return nil case jobToWorkerCh <- job: } @@ -1077,12 +1072,14 @@ func (local *Backend) generateJobForRange( } // startWorker creates a worker that reads from the job channel and processes. -// startWorker will return nil if it's expected to stop, where the only case is -// the context canceled. It will return not nil error when it actively stops. -// startWorker must Done the jobWg if it does not put the job into jobOutCh. +// startWorker will return nil if it's expected to stop, where the cases are all +// jobs are finished or the context canceled because other components report +// error. It will return not nil error when it actively stops. startWorker must +// call job.done() if it does not put the job into jobOutCh. func (local *Backend) startWorker( ctx context.Context, jobInCh, jobOutCh chan *regionJob, + afterExecuteJob func([]*metapb.Peer), jobWg *sync.WaitGroup, ) error { metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Set(0) @@ -1092,17 +1089,29 @@ func (local *Backend) startWorker( return nil case job, ok := <-jobInCh: if !ok { - // In fact we don't use close input channel to notify worker to - // exit, because there's a cycle in workflow. return nil } + var peers []*metapb.Peer + // in unit test, we may not have the real peers + if job.region != nil && job.region.Region != nil { + peers = job.region.Region.GetPeers() + } metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Inc() err := local.executeJob(ctx, job) metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Dec() + + if afterExecuteJob != nil { + afterExecuteJob(peers) + } switch job.stage { case regionScanned, wrote, ingested: - jobOutCh <- job + select { + case <-ctx.Done(): + job.done(jobWg) + return nil + case jobOutCh <- job: + } case needRescan: jobs, err2 := local.generateJobForRange( ctx, @@ -1126,7 +1135,12 @@ func (local *Backend) startWorker( } for _, j := range jobs { j.lastRetryableErr = job.lastRetryableErr - jobOutCh <- j + select { + case <-ctx.Done(): + j.done(jobWg) + // don't exit here, we mark done for each job and exit in the outer loop + case jobOutCh <- j: + } } } @@ -1364,63 +1378,100 @@ func (local *Backend) doImport( regionSplitSize, regionSplitKeyCnt int64, ) error { /* - [prepareAndSendJob]-----jobToWorkerCh--->[workers] - ^ | - | jobFromWorkerCh - | | - | v - [regionJobRetryer]<--[dispatchJobGoroutine]-->done + [prepareAndSendJob]---jobToWorkerCh->[storeBalancer(optional)]->[workers] + ^ | + | jobFromWorkerCh + | | + | v + [regionJobRetryer]<-------------[dispatchJobGoroutine]-->done */ + // Above is the happy path workflow of region jobs. A job is generated by + // prepareAndSendJob and terminated to "done" state by dispatchJobGoroutine. We + // maintain an invariant that the number of generated jobs (after job.ref()) + // minus the number of "done" jobs (after job.done()) equals to jobWg. So we can + // use jobWg to wait for all jobs to be finished. + // + // To handle the error case, we still maintain the invariant, but the workflow + // becomes a bit more complex. When an error occurs, the owner components of a + // job need to convert the job to "done" state, or send all its owned jobs to + // next components. The exit order is important because if the next component is + // exited before the owner component, deadlock will happen. + // + // All components are spawned by workGroup so the main goroutine can wait all + // components to exit. Component exit order in happy path is: + // + // 1. prepareAndSendJob is finished, its goroutine will wait all jobs are + // finished by jobWg.Wait(). Then it will exit and close the output channel of + // workers. + // + // 2. one-by-one, when every component see its input channel is closed, it knows + // the workflow is finished. It will exit and (except for workers) close the + // output channel which is the input channel of the next component. + // + // 3. Now all components are exited, the main goroutine can exit after + // workGroup.Wait(). + // + // Component exit order in error case is: + // + // 1. The error component exits and causes workGroup's context to be canceled. + // + // 2. All other components will exit because of the canceled context. No need to + // close channels. + // + // 3. the main goroutine can see the error and exit after workGroup.Wait(). var ( - ctx2, workerCancel = context.WithCancel(ctx) - // workerCtx.Done() means workflow is canceled by error. It may be caused - // by calling workerCancel() or workers in workGroup meets error. - workGroup, workerCtx = util.NewErrorGroupWithRecoverWithCtx(ctx2) - firstErr common.OnceError + workGroup, workerCtx = util.NewErrorGroupWithRecoverWithCtx(ctx) // jobToWorkerCh and jobFromWorkerCh are unbuffered so jobs will not be // owned by them. jobToWorkerCh = make(chan *regionJob) jobFromWorkerCh = make(chan *regionJob) - // jobWg tracks the number of jobs in this workflow. - // prepareAndSendJob, workers and regionJobRetryer can own jobs. - // When cancel on error, the goroutine of above three components have - // responsibility to Done jobWg of their owning jobs. - jobWg sync.WaitGroup - dispatchJobGoroutine = make(chan struct{}) + jobWg sync.WaitGroup + balancer *storeBalancer ) - defer workerCancel() + + // storeBalancer does not have backpressure, it should not be used with external + // engine to avoid OOM. + if _, ok := engine.(*Engine); ok { + balancer = newStoreBalancer(jobToWorkerCh, &jobWg) + workGroup.Go(func() error { + return balancer.run(workerCtx) + }) + } failpoint.Inject("injectVariables", func() { jobToWorkerCh = testJobToWorkerCh testJobWg = &jobWg }) - retryer := startRegionJobRetryer(workerCtx, jobToWorkerCh, &jobWg) + retryer := newRegionJobRetryer(workerCtx, jobToWorkerCh, &jobWg) + workGroup.Go(func() error { + retryer.run() + return nil + }) - // dispatchJobGoroutine handles processed job from worker, it will only exit - // when jobFromWorkerCh is closed to avoid worker is blocked on sending to - // jobFromWorkerCh. - defer func() { - // use defer to close jobFromWorkerCh after all workers are exited - close(jobFromWorkerCh) - <-dispatchJobGoroutine - }() - go func() { - defer close(dispatchJobGoroutine) + // dispatchJobGoroutine + workGroup.Go(func() error { + var ( + job *regionJob + ok bool + ) for { - job, ok := <-jobFromWorkerCh + select { + case <-workerCtx.Done(): + return nil + case job, ok = <-jobFromWorkerCh: + } if !ok { - return + retryer.close() + return nil } switch job.stage { case regionScanned, wrote: job.retryCount++ if job.retryCount > maxWriteAndIngestRetryTimes { - firstErr.Set(job.lastRetryableErr) - workerCancel() job.done(&jobWg) - continue + return job.lastRetryableErr } // max retry backoff time: 2+4+8+16+30*26=810s sleepSecond := math.Pow(2, float64(job.retryCount)) @@ -1444,7 +1495,7 @@ func (local *Backend) doImport( panic("should not reach here") } } - }() + }) failpoint.Inject("skipStartWorker", func() { failpoint.Goto("afterStartWorker") @@ -1452,7 +1503,13 @@ func (local *Backend) doImport( for i := 0; i < local.WorkerConcurrency; i++ { workGroup.Go(func() error { - return local.startWorker(workerCtx, jobToWorkerCh, jobFromWorkerCh, &jobWg) + toCh := jobToWorkerCh + var afterExecuteJob func([]*metapb.Peer) + if balancer != nil { + toCh = balancer.innerJobToWorkerCh + afterExecuteJob = balancer.releaseStoreLoad + } + return local.startWorker(workerCtx, toCh, jobFromWorkerCh, afterExecuteJob, &jobWg) }) } @@ -1473,16 +1530,28 @@ func (local *Backend) doImport( } jobWg.Wait() - workerCancel() + if balancer != nil { + intest.AssertFunc(func() bool { + allZero := true + balancer.storeLoadMap.Range(func(_, value any) bool { + if value.(int) != 0 { + allZero = false + return false + } + return true + }) + return allZero + }) + } + close(jobFromWorkerCh) return nil }) - if err := workGroup.Wait(); err != nil { - if !common.IsContextCanceledError(err) { - log.FromContext(ctx).Error("do import meets error", zap.Error(err)) - } - firstErr.Set(err) + + err := workGroup.Wait() + if err != nil && !common.IsContextCanceledError(err) { + log.FromContext(ctx).Error("do import meets error", zap.Error(err)) } - return firstErr.Get() + return err } // GetImportedKVCount returns the number of imported KV pairs of some engine. diff --git a/pkg/lightning/backend/local/local_test.go b/pkg/lightning/backend/local/local_test.go index 52b13c03e9ad2..0bdc736e6ac9e 100644 --- a/pkg/lightning/backend/local/local_test.go +++ b/pkg/lightning/backend/local/local_test.go @@ -1092,7 +1092,7 @@ func TestLocalWriteAndIngestPairsFailFast(t *testing.T) { jobCh := make(chan *regionJob, 1) jobCh <- ®ionJob{} jobOutCh := make(chan *regionJob, 1) - err := bak.startWorker(context.Background(), jobCh, jobOutCh, nil) + err := bak.startWorker(context.Background(), jobCh, jobOutCh, nil, nil) require.Error(t, err) require.Regexp(t, "the remaining storage capacity of TiKV.*", err.Error()) require.Len(t, jobCh, 0) @@ -1293,7 +1293,7 @@ func TestCheckPeersBusy(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - err := local.startWorker(ctx, jobCh, jobOutCh, nil) + err := local.startWorker(ctx, jobCh, jobOutCh, nil, nil) require.NoError(t, err) }() @@ -1400,7 +1400,7 @@ func TestNotLeaderErrorNeedUpdatePeers(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - err := local.startWorker(ctx, jobCh, jobOutCh, &jobWg) + err := local.startWorker(ctx, jobCh, jobOutCh, nil, &jobWg) require.NoError(t, err) }() @@ -1500,7 +1500,7 @@ func TestPartialWriteIngestErrorWontPanic(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - err := local.startWorker(ctx, jobCh, jobOutCh, &jobWg) + err := local.startWorker(ctx, jobCh, jobOutCh, nil, &jobWg) require.NoError(t, err) }() @@ -1620,7 +1620,7 @@ func TestPartialWriteIngestBusy(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - err := local.startWorker(ctx, jobCh, jobOutCh, &jobWg) + err := local.startWorker(ctx, jobCh, jobOutCh, nil, &jobWg) require.NoError(t, err) }() @@ -2116,12 +2116,30 @@ func TestRegionJobResetRetryCounter(t *testing.T) { ingestData: &Engine{}, injected: getNeedRescanWhenIngestBehaviour(), retryCount: maxWriteAndIngestRetryTimes, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 1, StoreId: 1}, + {Id: 2, StoreId: 2}, + {Id: 3, StoreId: 3}, + }, + }, + }, }, { keyRange: common.Range{Start: []byte{'c', '2'}, End: []byte{'d'}}, ingestData: &Engine{}, injected: getSuccessInjectedBehaviour(), retryCount: maxWriteAndIngestRetryTimes, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 4, StoreId: 4}, + {Id: 5, StoreId: 5}, + {Id: 6, StoreId: 6}, + }, + }, + }, }, }, }, @@ -2131,6 +2149,15 @@ func TestRegionJobResetRetryCounter(t *testing.T) { keyRange: common.Range{Start: []byte{'c'}, End: []byte{'c', '2'}}, ingestData: &Engine{}, injected: getSuccessInjectedBehaviour(), + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 7, StoreId: 7}, + {Id: 8, StoreId: 8}, + {Id: 9, StoreId: 9}, + }, + }, + }, }, }, }, diff --git a/pkg/lightning/backend/local/region_job.go b/pkg/lightning/backend/local/region_job.go index 134a2db13f440..2e90ea8ee0755 100644 --- a/pkg/lightning/backend/local/region_job.go +++ b/pkg/lightning/backend/local/region_job.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "io" + "math" "strings" "sync" "time" @@ -39,7 +40,9 @@ import ( "github.com/pingcap/tidb/pkg/lightning/config" "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/lightning/metric" + util2 "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/codec" + "github.com/pingcap/tidb/pkg/util/intest" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" "google.golang.org/grpc" @@ -913,30 +916,35 @@ type regionJobRetryer struct { putBackCh chan<- *regionJob reload chan struct{} jobWg *sync.WaitGroup + + ctx context.Context + cancel context.CancelFunc } -// startRegionJobRetryer starts a new regionJobRetryer and it will run in -// background to put the job back to `putBackCh` when job's waitUntil is reached. -// Cancel the `ctx` will stop retryer and `jobWg.Done` will be trigger for jobs -// that are not put back yet. -func startRegionJobRetryer( - ctx context.Context, +// newRegionJobRetryer creates a regionJobRetryer. regionJobRetryer.run is +// expected to be called soon. +func newRegionJobRetryer( + workerCtx context.Context, putBackCh chan<- *regionJob, jobWg *sync.WaitGroup, ) *regionJobRetryer { + ctx, cancel := context.WithCancel(workerCtx) ret := ®ionJobRetryer{ putBackCh: putBackCh, reload: make(chan struct{}, 1), jobWg: jobWg, + ctx: ctx, + cancel: cancel, } ret.protectedQueue.q = make(regionJobRetryHeap, 0, 16) - go ret.run(ctx) return ret } -// run is only internally used, caller should not use it. -func (q *regionJobRetryer) run(ctx context.Context) { - defer q.close() +// run occupies the goroutine and starts the retry loop. Cancel the `ctx` will +// stop retryer and `jobWg.Done` will be trigger for jobs that are not put back +// yet. It should only be used in error case. +func (q *regionJobRetryer) run() { + defer q.cleanupUnprocessedJobs() for { var front *regionJob @@ -949,7 +957,7 @@ func (q *regionJobRetryer) run(ctx context.Context) { switch { case front != nil: select { - case <-ctx.Done(): + case <-q.ctx.Done(): return case <-q.reload: case <-time.After(time.Until(front.waitUntil)): @@ -962,7 +970,7 @@ func (q *regionJobRetryer) run(ctx context.Context) { // hold the lock of toPutBack to make sending to putBackCh and // resetting toPutBack atomic w.r.t. regionJobRetryer.close select { - case <-ctx.Done(): + case <-q.ctx.Done(): q.protectedToPutBack.mu.Unlock() return case q.putBackCh <- q.protectedToPutBack.toPutBack: @@ -973,7 +981,7 @@ func (q *regionJobRetryer) run(ctx context.Context) { default: // len(q.q) == 0 select { - case <-ctx.Done(): + case <-q.ctx.Done(): return case <-q.reload: } @@ -981,8 +989,25 @@ func (q *regionJobRetryer) run(ctx context.Context) { } } -// close is only internally used, caller should not use it. +// close stops the retryer. It should only be used in the happy path where all +// jobs are finished. func (q *regionJobRetryer) close() { + q.cancel() + close(q.putBackCh) + intest.AssertFunc(func() bool { + q.protectedToPutBack.mu.Lock() + defer q.protectedToPutBack.mu.Unlock() + return q.protectedToPutBack.toPutBack == nil + }, "toPutBack should be nil considering it's happy path") + intest.AssertFunc(func() bool { + q.protectedQueue.mu.Lock() + defer q.protectedQueue.mu.Unlock() + return len(q.protectedQueue.q) == 0 + }, "queue should be empty considering it's happy path") +} + +// cleanupUnprocessedJobs is only internally used, caller should not use it. +func (q *regionJobRetryer) cleanupUnprocessedJobs() { q.protectedClosed.mu.Lock() defer q.protectedClosed.mu.Unlock() q.protectedClosed.closed = true @@ -1013,3 +1038,204 @@ func (q *regionJobRetryer) push(job *regionJob) bool { } return true } + +// storeBalancer is used to balance the store load when sending region jobs to +// worker. Internally it maintains a large enough buffer to hold all region jobs, +// and pick the job related to stores that has the least load to send to worker. +// Because it does not have backpressure, it should not be used with external +// engine to avoid OOM. +type storeBalancer struct { + // map[int]*regionJob + jobs sync.Map + jobIdx int + jobWg *sync.WaitGroup + + jobToWorkerCh <-chan *regionJob + innerJobToWorkerCh chan *regionJob + + wakeSendToWorker chan struct{} + + // map[uint64]int. 0 can appear in the map after it's decremented to 0. + storeLoadMap sync.Map +} + +func newStoreBalancer( + jobToWorkerCh <-chan *regionJob, + jobWg *sync.WaitGroup, +) *storeBalancer { + return &storeBalancer{ + jobToWorkerCh: jobToWorkerCh, + innerJobToWorkerCh: make(chan *regionJob), + wakeSendToWorker: make(chan struct{}, 1), + jobWg: jobWg, + } +} + +func (b *storeBalancer) run(workerCtx context.Context) error { + // all goroutine will not return error except panic, so we make use of + // ErrorGroupWithRecover. + wg, ctx2 := util2.NewErrorGroupWithRecoverWithCtx(workerCtx) + sendToWorkerCtx, cancelSendToWorker := context.WithCancel(ctx2) + wg.Go(func() error { + b.runReadToWorkerCh(ctx2) + cancelSendToWorker() + return nil + }) + wg.Go(func() error { + b.runSendToWorker(sendToWorkerCtx) + return nil + }) + + if err := wg.Wait(); err != nil { + return err + } + + b.jobs.Range(func(_, value any) bool { + value.(*regionJob).done(b.jobWg) + return true + }) + return nil +} + +func (b *storeBalancer) runReadToWorkerCh(workerCtx context.Context) { + for { + select { + case <-workerCtx.Done(): + return + case job, ok := <-b.jobToWorkerCh: + if !ok { + close(b.innerJobToWorkerCh) + return + } + b.jobs.Store(b.jobIdx, job) + b.jobIdx++ + + select { + case b.wakeSendToWorker <- struct{}{}: + default: + } + } + } +} + +func (b *storeBalancer) jobLen() int { + cnt := 0 + b.jobs.Range(func(_, _ any) bool { + cnt++ + return true + }) + return cnt +} + +func (b *storeBalancer) runSendToWorker(workerCtx context.Context) { + for { + select { + case <-workerCtx.Done(): + return + case <-b.wakeSendToWorker: + } + + remainJobCnt := b.jobLen() + for i := 0; i < remainJobCnt; i++ { + j := b.pickJob() + if j == nil { + // j can be nil if it's executed after the jobs.Store of runReadToWorkerCh + // and before the sending to wakeSendToWorker of runReadToWorkerCh. + break + } + + // after the job is picked and before the job is sent to worker, the score may + // have changed so we should pick again to get the optimal job. However for + // simplicity we don't do it. The optimal job will be picked in the next round. + select { + case <-workerCtx.Done(): + j.done(b.jobWg) + return + case b.innerJobToWorkerCh <- j: + } + } + } +} + +func (b *storeBalancer) pickJob() *regionJob { + var ( + best *regionJob + bestIdx = -1 + minScore = math.MaxInt64 + ) + b.jobs.Range(func(key, value any) bool { + idx := key.(int) + job := value.(*regionJob) + + score := 0 + // in unit tests, the fields of job may not set + if job.region == nil || job.region.Region == nil { + best = job + bestIdx = idx + return false + } + + for _, p := range job.region.Region.Peers { + if v, ok := b.storeLoadMap.Load(p.StoreId); ok { + score += v.(int) + } + } + + if score == 0 { + best = job + bestIdx = idx + return false + } + if score < minScore { + minScore = score + best = job + bestIdx = idx + } + return true + }) + if bestIdx == -1 { + return nil + } + + b.jobs.Delete(bestIdx) + // in unit tests, the fields of job may not set + if best.region == nil || best.region.Region == nil { + return best + } + + for _, p := range best.region.Region.Peers { + retry: + val, loaded := b.storeLoadMap.LoadOrStore(p.StoreId, 1) + if !loaded { + continue + } + + old := val.(int) + if !b.storeLoadMap.CompareAndSwap(p.StoreId, old, old+1) { + // retry the whole check because the entry may have been deleted + goto retry + } + } + return best +} + +func (b *storeBalancer) releaseStoreLoad(peers []*metapb.Peer) { + for _, p := range peers { + retry: + val, ok := b.storeLoadMap.Load(p.StoreId) + if !ok { + intest.Assert(false, + "missing key in storeLoadMap. key: %d", + p.StoreId, + ) + log.L().Error("missing key in storeLoadMap", + zap.Uint64("storeID", p.StoreId)) + continue + } + + old := val.(int) + if !b.storeLoadMap.CompareAndSwap(p.StoreId, old, old-1) { + goto retry + } + } +} diff --git a/pkg/lightning/backend/local/region_job_test.go b/pkg/lightning/backend/local/region_job_test.go index c3ed8048863ec..907e6a72d1033 100644 --- a/pkg/lightning/backend/local/region_job_test.go +++ b/pkg/lightning/backend/local/region_job_test.go @@ -16,6 +16,7 @@ package local import ( "context" + "math/rand" "sync" "testing" "time" @@ -187,9 +188,14 @@ func TestRegionJobRetryer(t *testing.T) { putBackCh = make(chan *regionJob, 10) jobWg sync.WaitGroup ctx, cancel = context.WithCancel(context.Background()) + done = make(chan struct{}) ) - retryer := startRegionJobRetryer(ctx, putBackCh, &jobWg) + retryer := newRegionJobRetryer(ctx, putBackCh, &jobWg) require.Len(t, putBackCh, 0) + go func() { + defer close(done) + retryer.run() + }() for i := 0; i < 8; i++ { go func() { @@ -226,6 +232,7 @@ func TestRegionJobRetryer(t *testing.T) { cancel() jobWg.Wait() + <-done ok = retryer.push(job) require.False(t, ok) @@ -234,7 +241,12 @@ func TestRegionJobRetryer(t *testing.T) { ctx, cancel = context.WithCancel(context.Background()) putBackCh = make(chan *regionJob) - retryer = startRegionJobRetryer(ctx, putBackCh, &jobWg) + retryer = newRegionJobRetryer(ctx, putBackCh, &jobWg) + done = make(chan struct{}) + go func() { + defer close(done) + retryer.run() + }() job = ®ionJob{ keyRange: common.Range{ @@ -258,6 +270,29 @@ func TestRegionJobRetryer(t *testing.T) { require.True(t, ok) cancel() jobWg.Wait() + <-done + + // test when close successfully, regionJobRetryer should close the putBackCh + ctx = context.Background() + putBackCh = make(chan *regionJob) + retryer = newRegionJobRetryer(ctx, putBackCh, &jobWg) + done = make(chan struct{}) + go func() { + defer close(done) + retryer.run() + }() + + job = ®ionJob{ + keyRange: common.Range{ + Start: []byte("123"), + }, + waitUntil: time.Now().Add(-time.Second), + } + ok = retryer.push(job) + require.True(t, ok) + <-putBackCh + retryer.close() + <-putBackCh } func TestNewRegionJobs(t *testing.T) { @@ -339,3 +374,209 @@ func TestNewRegionJobs(t *testing.T) { } } } + +func mockWorkerReadJob( + t *testing.T, + b *storeBalancer, + jobs []*regionJob, + jobToWorkerCh chan<- *regionJob, +) []*regionJob { + ret := make([]*regionJob, len(jobs)) + jobToWorkerCh <- jobs[0] + require.Eventually(t, func() bool { + // wait runSendToWorker goroutine is blocked at sending + return b.jobLen() == 0 + }, time.Second, 10*time.Millisecond) + + for _, job := range jobs[1:] { + jobToWorkerCh <- job + } + require.Eventually(t, func() bool { + // rest are waiting to be picked + return b.jobLen() == len(jobs)-1 + }, time.Second, 10*time.Millisecond) + for i := range ret { + got := <-b.innerJobToWorkerCh + ret[i] = got + } + return ret +} + +func checkStoreScoreZero(t *testing.T, b *storeBalancer) { + b.storeLoadMap.Range(func(_, value any) bool { + require.Equal(t, 0, value.(int)) + return true + }) +} + +func TestStoreBalancerPick(t *testing.T) { + jobToWorkerCh := make(chan *regionJob) + jobWg := sync.WaitGroup{} + ctx := context.Background() + + b := newStoreBalancer(jobToWorkerCh, &jobWg) + done := make(chan struct{}) + go func() { + defer close(done) + err := b.run(ctx) + require.NoError(t, err) + }() + + job := ®ionJob{ + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 1, StoreId: 1}, + {Id: 2, StoreId: 2}, + }, + }, + }, + } + + // the worker can get the job just sent to storeBalancer + got := mockWorkerReadJob(t, b, []*regionJob{job}, jobToWorkerCh) + require.Equal(t, []*regionJob{job}, got) + + // mimic the worker is handled the job and storeBalancer release it + b.releaseStoreLoad(job.region.Region.GetPeers()) + checkStoreScoreZero(t, b) + + busyStoreJob := ®ionJob{ + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 3, StoreId: 2}, + {Id: 4, StoreId: 2}, + }, + }, + }, + } + idleStoreJob := ®ionJob{ + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {Id: 5, StoreId: 3}, + {Id: 6, StoreId: 4}, + }, + }, + }, + } + + // now the worker should get the job in specific order. The first job is already + // picked and can't be dynamically changed by design, so the order is job, + // idleStoreJob, busyStoreJob + got = mockWorkerReadJob(t, b, []*regionJob{job, busyStoreJob, idleStoreJob}, jobToWorkerCh) + require.Equal(t, []*regionJob{job, idleStoreJob, busyStoreJob}, got) + // mimic the worker finished the job in different order + jonDone := make(chan struct{}, 3) + go func() { + b.releaseStoreLoad(idleStoreJob.region.Region.GetPeers()) + jonDone <- struct{}{} + }() + go func() { + b.releaseStoreLoad(job.region.Region.GetPeers()) + jonDone <- struct{}{} + }() + go func() { + b.releaseStoreLoad(busyStoreJob.region.Region.GetPeers()) + jonDone <- struct{}{} + }() + + for i := 0; i < 3; i++ { + <-jonDone + } + checkStoreScoreZero(t, b) + + close(jobToWorkerCh) + <-done +} + +func mockRegionJob4Balance(t *testing.T, cnt int) []*regionJob { + seed := time.Now().UnixNano() + t.Logf("seed: %d", seed) + r := rand.New(rand.NewSource(seed)) + ret := make([]*regionJob, cnt) + for i := range ret { + ret[i] = ®ionJob{ + region: &split.RegionInfo{ + Region: &metapb.Region{ + Peers: []*metapb.Peer{ + {StoreId: uint64(r.Intn(10))}, + {StoreId: uint64(r.Intn(10))}, + }, + }, + }, + } + } + + return ret +} + +func TestCancelBalancer(t *testing.T) { + jobToWorkerCh := make(chan *regionJob) + jobWg := sync.WaitGroup{} + ctx, cancel := context.WithCancel(context.Background()) + + b := newStoreBalancer(jobToWorkerCh, &jobWg) + done := make(chan struct{}) + go func() { + defer close(done) + err := b.run(ctx) + require.NoError(t, err) + }() + + jobs := mockRegionJob4Balance(t, 20) + for _, job := range jobs { + jobWg.Add(1) + jobToWorkerCh <- job + } + + cancel() + <-done + jobWg.Wait() +} + +func TestStoreBalancerNoRace(t *testing.T) { + jobToWorkerCh := make(chan *regionJob) + jobFromWorkerCh := make(chan *regionJob) + jobWg := sync.WaitGroup{} + ctx, cancel := context.WithCancel(context.Background()) + + b := newStoreBalancer(jobToWorkerCh, &jobWg) + done := make(chan struct{}) + go func() { + defer close(done) + err := b.run(ctx) + require.NoError(t, err) + }() + + cnt := 200 + done2 := make(chan struct{}) + + jobs := mockRegionJob4Balance(t, cnt) + for _, job := range jobs { + jobWg.Add(1) + jobToWorkerCh <- job + } + + go func() { + // mimic that worker handles the job and send back to storeBalancer concurrently + for j := range b.innerJobToWorkerCh { + j := j + go func() { + b.releaseStoreLoad(j.region.Region.GetPeers()) + j.done(&jobWg) + }() + } + close(done2) + }() + + jobWg.Wait() + checkStoreScoreZero(t, b) + + cancel() + <-done + close(b.innerJobToWorkerCh) + <-done2 + require.Len(t, jobFromWorkerCh, 0) +} diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index 8af50ca8a5547..f45102e122ee0 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -833,7 +833,7 @@ func (m *Meta) GetSchemaCacheSize() (size uint64, isNull bool, err error) { // CreateTableAndSetAutoID creates a table with tableInfo in database, // and rebases the table autoID. -func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, autoIDs AutoIDGroup) error { +func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, autoIDs model.AutoIDGroup) error { err := m.CreateTableOrView(dbID, tableInfo) if err != nil { return errors.Trace(err) diff --git a/pkg/meta/meta_autoid.go b/pkg/meta/meta_autoid.go index e5071f5dbe899..d4ec69fcd5b60 100644 --- a/pkg/meta/meta_autoid.go +++ b/pkg/meta/meta_autoid.go @@ -93,8 +93,8 @@ func (a *autoIDAccessor) CopyTo(databaseID, tableID int64) error { // AutoIDAccessors represents all the auto IDs of a table. type AutoIDAccessors interface { - Get() (AutoIDGroup, error) - Put(autoIDs AutoIDGroup) error + Get() (model.AutoIDGroup, error) + Put(autoIDs model.AutoIDGroup) error Del() error AccessorPicker @@ -117,7 +117,7 @@ type autoIDAccessors struct { const sepAutoIncVer = model.TableInfoVersion5 // Get implements the interface AutoIDAccessors. -func (a *autoIDAccessors) Get() (autoIDs AutoIDGroup, err error) { +func (a *autoIDAccessors) Get() (autoIDs model.AutoIDGroup, err error) { if autoIDs.RowID, err = a.RowID().Get(); err != nil { return autoIDs, err } @@ -131,7 +131,7 @@ func (a *autoIDAccessors) Get() (autoIDs AutoIDGroup, err error) { } // Put implements the interface AutoIDAccessors. -func (a *autoIDAccessors) Put(autoIDs AutoIDGroup) error { +func (a *autoIDAccessors) Put(autoIDs model.AutoIDGroup) error { if err := a.RowID().Put(autoIDs.RowID); err != nil { return err } @@ -197,10 +197,3 @@ func NewAutoIDAccessors(m *Meta, databaseID, tableID int64) AutoIDAccessors { }, } } - -// AutoIDGroup represents a group of auto IDs of a specific table. -type AutoIDGroup struct { - RowID int64 - IncrementID int64 - RandomID int64 -} diff --git a/pkg/meta/meta_test.go b/pkg/meta/meta_test.go index bb59822466250..8b4e6cea5bb18 100644 --- a/pkg/meta/meta_test.go +++ b/pkg/meta/meta_test.go @@ -373,7 +373,7 @@ func TestMeta(t *testing.T) { ID: 3, Name: pmodel.NewCIStr("tbl3"), } - err = m.CreateTableAndSetAutoID(1, tbInfo3, meta.AutoIDGroup{RowID: 123, IncrementID: 0}) + err = m.CreateTableAndSetAutoID(1, tbInfo3, model.AutoIDGroup{RowID: 123, IncrementID: 0}) require.NoError(t, err) id, err := m.GetAutoIDAccessors(1, tbInfo3.ID).RowID().Get() require.NoError(t, err) diff --git a/pkg/meta/metabuild/context.go b/pkg/meta/metabuild/context.go index 1e5820a4c41eb..faa2056cedbd2 100644 --- a/pkg/meta/metabuild/context.go +++ b/pkg/meta/metabuild/context.go @@ -84,13 +84,6 @@ func WithPreSplitRegions(regions uint64) Option { }) } -// WithEnableTablePartitionMode sets the enable table partition mode. -func WithEnableTablePartitionMode(mode string) Option { - return funcOpt(func(ctx *Context) { - ctx.enableTablePartitionMode = mode - }) -} - // WithSuppressTooLongIndexErr sets whether to suppress too long index error. func WithSuppressTooLongIndexErr(suppress bool) Option { return funcOpt(func(ctx *Context) { @@ -113,7 +106,6 @@ type Context struct { clusteredIndexDefMode variable.ClusteredIndexDefMode shardRowIDBits uint64 preSplitRegions uint64 - enableTablePartitionMode string suppressTooLongIndexErr bool is infoschemactx.MetaOnlyInfoSchema } @@ -125,7 +117,6 @@ func NewContext(opts ...Option) *Context { primaryKeyRequired: false, clusteredIndexDefMode: variable.DefTiDBEnableClusteredIndex, shardRowIDBits: variable.DefShardRowIDBits, - enableTablePartitionMode: variable.On, preSplitRegions: variable.DefPreSplitRegions, suppressTooLongIndexErr: false, } @@ -191,11 +182,6 @@ func (ctx *Context) GetPreSplitRegions() uint64 { return ctx.preSplitRegions } -// GetEnableTablePartitionMode returns the enable table partition mode. -func (ctx *Context) GetEnableTablePartitionMode() string { - return ctx.enableTablePartitionMode -} - // SuppressTooLongIndexErr returns whether suppress too long index error. func (ctx *Context) SuppressTooLongIndexErr() bool { return ctx.suppressTooLongIndexErr diff --git a/pkg/meta/metabuild/context_test.go b/pkg/meta/metabuild/context_test.go index c0b2b8a6bd916..99a20961fe3a8 100644 --- a/pkg/meta/metabuild/context_test.go +++ b/pkg/meta/metabuild/context_test.go @@ -116,16 +116,6 @@ func TestMetaBuildContext(t *testing.T) { }, testVals: []any{uint64(123), uint64(456)}, }, - { - name: "enableTablePartitionMode", - getter: func(ctx *metabuild.Context) any { - return ctx.GetEnableTablePartitionMode() - }, - checkDefault: variable.On, - option: func(val any) metabuild.Option { - return metabuild.WithEnableTablePartitionMode(val.(string)) - }, - }, { name: "suppressTooLongIndexErr", getter: func(ctx *metabuild.Context) any { diff --git a/pkg/meta/model/BUILD.bazel b/pkg/meta/model/BUILD.bazel index cf69f180a136b..711ff655c24f6 100644 --- a/pkg/meta/model/BUILD.bazel +++ b/pkg/meta/model/BUILD.bazel @@ -45,7 +45,7 @@ go_test( ], embed = [":model"], flaky = True, - shard_count = 47, + shard_count = 50, deps = [ "//pkg/parser/ast", "//pkg/parser/charset", diff --git a/pkg/meta/model/job_args.go b/pkg/meta/model/job_args.go index f62b3fa6f0c7c..7191b5161c60f 100644 --- a/pkg/meta/model/job_args.go +++ b/pkg/meta/model/job_args.go @@ -23,6 +23,37 @@ import ( "github.com/pingcap/tidb/pkg/util/intest" ) +// AutoIDGroup represents a group of auto IDs of a specific table. +type AutoIDGroup struct { + RowID int64 + IncrementID int64 + RandomID int64 +} + +// RecoverTableInfo contains information needed by DDL.RecoverTable. +type RecoverTableInfo struct { + SchemaID int64 + TableInfo *TableInfo + DropJobID int64 + SnapshotTS uint64 + AutoIDs AutoIDGroup + OldSchemaName string + OldTableName string +} + +// RecoverSchemaInfo contains information needed by DDL.RecoverSchema. +type RecoverSchemaInfo struct { + *DBInfo + RecoverTableInfos []*RecoverTableInfo + // LoadTablesOnExecute is the new logic to avoid a large RecoverTabsInfo can't be + // persisted. If it's true, DDL owner will recover RecoverTabsInfo instead of the + // job submit node. + LoadTablesOnExecute bool + DropJobID int64 + SnapshotTS uint64 + OldSchemaName pmodel.CIStr +} + // getOrDecodeArgsV2 get the argsV2 from job, if the argsV2 is nil, decode rawArgsV2 // and fill argsV2. func getOrDecodeArgsV2[T JobArgs](job *Job) (T, error) { @@ -55,6 +86,11 @@ type FinishedJobArgs interface { fillFinishedJob(job *Job) } +// EmptyArgs is the args for ddl job with no args. +type EmptyArgs struct{} + +func (*EmptyArgs) fillJob(*Job) {} + // CreateSchemaArgs is the arguments for create schema job. type CreateSchemaArgs struct { DBInfo *DBInfo `json:"db_info,omitempty"` @@ -997,6 +1033,123 @@ func GetRenameTablesArgs(job *Job) (*RenameTablesArgs, error) { return getOrDecodeArgsV2[*RenameTablesArgs](job) } +// AlterSequenceArgs is the arguments for alter sequence ddl job. +type AlterSequenceArgs struct { + Ident ast.Ident `json:"ident,omitempty"` + SeqOptions []*ast.SequenceOption `json:"seq_options,omitempty"` +} + +func (a *AlterSequenceArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + job.Args = []any{a.Ident, a.SeqOptions} + } else { + job.Args = []any{a} + } +} + +// GetAlterSequenceArgs gets the args for alter Sequence ddl job. +func GetAlterSequenceArgs(job *Job) (*AlterSequenceArgs, error) { + if job.Version == JobVersion1 { + var ( + ident ast.Ident + seqOptions []*ast.SequenceOption + ) + if err := job.DecodeArgs(&ident, &seqOptions); err != nil { + return nil, errors.Trace(err) + } + return &AlterSequenceArgs{ + Ident: ident, + SeqOptions: seqOptions, + }, nil + } + + return getOrDecodeArgsV2[*AlterSequenceArgs](job) +} + +// ModifyTableAutoIDCacheArgs is the arguments for Modify Table AutoID Cache ddl job. +type ModifyTableAutoIDCacheArgs struct { + NewCache int64 `json:"new_cache,omitempty"` +} + +func (a *ModifyTableAutoIDCacheArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + job.Args = []any{a.NewCache} + } else { + job.Args = []any{a} + } +} + +// GetModifyTableAutoIDCacheArgs gets the args for modify table autoID cache ddl job. +func GetModifyTableAutoIDCacheArgs(job *Job) (*ModifyTableAutoIDCacheArgs, error) { + if job.Version == JobVersion1 { + var newCache int64 + if err := job.DecodeArgs(&newCache); err != nil { + return nil, errors.Trace(err) + } + return &ModifyTableAutoIDCacheArgs{ + NewCache: newCache, + }, nil + } + + return getOrDecodeArgsV2[*ModifyTableAutoIDCacheArgs](job) +} + +// ShardRowIDArgs is the arguments for shard row ID ddl job. +type ShardRowIDArgs struct { + ShardRowIDBits uint64 `json:"shard_row_id_bits,omitempty"` +} + +func (a *ShardRowIDArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + job.Args = []any{a.ShardRowIDBits} + } else { + job.Args = []any{a} + } +} + +// GetShardRowIDArgs gets the args for shard row ID ddl job. +func GetShardRowIDArgs(job *Job) (*ShardRowIDArgs, error) { + if job.Version == JobVersion1 { + var val uint64 + if err := job.DecodeArgs(&val); err != nil { + return nil, errors.Trace(err) + } + return &ShardRowIDArgs{ + ShardRowIDBits: val, + }, nil + } + + return getOrDecodeArgsV2[*ShardRowIDArgs](job) +} + +// AlterTTLInfoArgs is the arguments for alter ttl info job. +type AlterTTLInfoArgs struct { + TTLInfo *TTLInfo `json:"ttl_info,omitempty"` + TTLEnable *bool `json:"ttl_enable,omitempty"` + TTLCronJobSchedule *string `json:"ttl_cron_job_schedule,omitempty"` +} + +func (a *AlterTTLInfoArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + job.Args = []any{a.TTLInfo, a.TTLEnable, a.TTLCronJobSchedule} + } else { + job.Args = []any{a} + } +} + +// GetAlterTTLInfoArgs gets the args for alter ttl info job. +func GetAlterTTLInfoArgs(job *Job) (*AlterTTLInfoArgs, error) { + if job.Version == JobVersion1 { + args := &AlterTTLInfoArgs{} + if err := job.DecodeArgs(&args.TTLInfo, &args.TTLEnable, &args.TTLCronJobSchedule); err != nil { + return nil, errors.Trace(err) + } + return args, nil + } + + return getOrDecodeArgsV2[*AlterTTLInfoArgs](job) +} + // GetCheckConstraintArgs gets the AlterCheckConstraint args. func GetCheckConstraintArgs(job *Job) (*CheckConstraintArgs, error) { if job.Version == JobVersion1 { @@ -1045,6 +1198,117 @@ func GetAddCheckConstraintArgs(job *Job) (*AddCheckConstraintArgs, error) { return getOrDecodeArgsV2[*AddCheckConstraintArgs](job) } +// LockTablesArgs is the argument for LockTables. +type LockTablesArgs struct { + LockTables []TableLockTpInfo `json:"lock_tables,omitempty"` + IndexOfLock int `json:"index_of_lock,omitempty"` + UnlockTables []TableLockTpInfo `json:"unlock_tables,omitempty"` + IndexOfUnlock int `json:"index_of_unlock,omitempty"` + SessionInfo SessionInfo `json:"session_info,omitempty"` + IsCleanup bool `json:"is_cleanup:omitempty"` +} + +func (a *LockTablesArgs) fillJob(job *Job) { + job.Args = []any{a} +} + +// GetLockTablesArgs get the LockTablesArgs argument. +func GetLockTablesArgs(job *Job) (*LockTablesArgs, error) { + var args *LockTablesArgs + var err error + + if job.Version == JobVersion1 { + err = job.DecodeArgs(&args) + } else { + args, err = getOrDecodeArgsV2[*LockTablesArgs](job) + } + + if err != nil { + return nil, errors.Trace(err) + } + return args, nil +} + +// RepairTableArgs is the argument for repair table +type RepairTableArgs struct { + *TableInfo `json:"table_info"` +} + +func (a *RepairTableArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + job.Args = []any{a.TableInfo} + return + } + job.Args = []any{a} +} + +// GetRepairTableArgs get the repair table args. +func GetRepairTableArgs(job *Job) (*RepairTableArgs, error) { + if job.Version == JobVersion1 { + var tblInfo *TableInfo + if err := job.DecodeArgs(&tblInfo); err != nil { + return nil, errors.Trace(err) + } + return &RepairTableArgs{tblInfo}, nil + } + + return getOrDecodeArgsV2[*RepairTableArgs](job) +} + +// RecoverArgs is the argument for recover table/schema. +type RecoverArgs struct { + RecoverInfo *RecoverSchemaInfo `json:"recover_info,omitempty"` + CheckFlag int64 `json:"check_flag,omitempty"` +} + +func (a *RecoverArgs) fillJob(job *Job) { + if job.Version == JobVersion1 { + if job.Type == ActionRecoverTable { + job.Args = []any{a.RecoverTableInfos()[0], a.CheckFlag} + } else { + job.Args = []any{a.RecoverInfo, a.CheckFlag} + } + return + } + job.Args = []any{a} +} + +// RecoverTableInfos get all the recover infos. +func (a *RecoverArgs) RecoverTableInfos() []*RecoverTableInfo { + return a.RecoverInfo.RecoverTableInfos +} + +// GetRecoverArgs get the recover table/schema args. +func GetRecoverArgs(job *Job) (*RecoverArgs, error) { + if job.Version == JobVersion1 { + var ( + recoverTableInfo *RecoverTableInfo + recoverSchemaInfo = &RecoverSchemaInfo{} + recoverCheckFlag int64 + ) + + if job.Type == ActionRecoverTable { + err := job.DecodeArgs(&recoverTableInfo, &recoverCheckFlag) + if err != nil { + return nil, errors.Trace(err) + } + recoverSchemaInfo.RecoverTableInfos = []*RecoverTableInfo{recoverTableInfo} + } else { + err := job.DecodeArgs(recoverSchemaInfo, &recoverCheckFlag) + if err != nil { + return nil, errors.Trace(err) + } + } + + return &RecoverArgs{ + RecoverInfo: recoverSchemaInfo, + CheckFlag: recoverCheckFlag, + }, nil + } + + return getOrDecodeArgsV2[*RecoverArgs](job) +} + // PlacementPolicyArgs is the argument for create/alter/drop placement policy type PlacementPolicyArgs struct { Policy *PolicyInfo `json:"policy,omitempty"` diff --git a/pkg/meta/model/job_args_test.go b/pkg/meta/model/job_args_test.go index 91dd775a6feb9..338f92ad9cfc2 100644 --- a/pkg/meta/model/job_args_test.go +++ b/pkg/meta/model/job_args_test.go @@ -499,6 +499,31 @@ func TestDropColumnArgs(t *testing.T) { } } +func TestGetAlterSequenceArgs(t *testing.T) { + inArgs := &AlterSequenceArgs{ + Ident: ast.Ident{ + Schema: model.NewCIStr("test_db"), + Name: model.NewCIStr("test_t"), + }, + SeqOptions: []*ast.SequenceOption{ + { + Tp: ast.SequenceOptionIncrementBy, + IntValue: 7527, + }, { + Tp: ast.SequenceCache, + IntValue: 9528, + }, + }, + } + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionAlterSequence))) + args, err := GetAlterSequenceArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs, args) + } +} + func TestGetRebaseAutoIDArgs(t *testing.T) { inArgs := &RebaseAutoIDArgs{ NewBase: 9527, @@ -562,6 +587,33 @@ func TestGetAddForeignKeyArgs(t *testing.T) { } } +func TestGetModifyTableAutoIDCacheArgs(t *testing.T) { + inArgs := &ModifyTableAutoIDCacheArgs{ + NewCache: 7527, + } + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionModifyTableAutoIDCache))) + args, err := GetModifyTableAutoIDCacheArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs, args) + } +} + +func TestGetShardRowIDArgs(t *testing.T) { + inArgs := &ShardRowIDArgs{ + ShardRowIDBits: 101, + } + + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionShardRowID))) + args, err := GetShardRowIDArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs, args) + } +} + func TestGetDropForeignKeyArgs(t *testing.T) { inArgs := &DropForeignKeyArgs{ FkName: model.NewCIStr("fk-name"), @@ -571,11 +623,32 @@ func TestGetDropForeignKeyArgs(t *testing.T) { j2 := &Job{} require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionDropForeignKey))) args, err := GetDropForeignKeyArgs(j2) - require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionDropColumn))) require.NoError(t, err) require.Equal(t, inArgs, args) } } + +func TestGetAlterTTLInfoArgs(t *testing.T) { + ttlEanble := true + ttlCronJobSchedule := "ttl-schedule" + inArgs := &AlterTTLInfoArgs{ + TTLInfo: &TTLInfo{ + ColumnName: model.NewCIStr("column_name"), + IntervalExprStr: "1", + IntervalTimeUnit: 10010, + }, + TTLEnable: &ttlEanble, + TTLCronJobSchedule: &ttlCronJobSchedule, + } + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionAlterTTLInfo))) + args, err := GetAlterTTLInfoArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs, args) + } +} + func TestAddCheckConstraintArgs(t *testing.T) { Constraint := &ConstraintInfo{ @@ -614,6 +687,72 @@ func TestCheckConstraintArgs(t *testing.T) { } } +func TestLockTableArgs(t *testing.T) { + inArgs := &LockTablesArgs{ + LockTables: []TableLockTpInfo{{1, 1, model.TableLockNone}}, + UnlockTables: []TableLockTpInfo{{2, 2, model.TableLockNone}}, + IndexOfLock: 13, + IndexOfUnlock: 24, + } + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + for _, tp := range []ActionType{ActionLockTable, ActionUnlockTable} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, tp))) + + args, err := GetLockTablesArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs.LockTables, args.LockTables) + require.Equal(t, inArgs.UnlockTables, args.UnlockTables) + require.Equal(t, inArgs.IndexOfLock, args.IndexOfLock) + require.Equal(t, inArgs.IndexOfUnlock, args.IndexOfUnlock) + } + } +} + +func TestRepairTableArgs(t *testing.T) { + inArgs := &RepairTableArgs{&TableInfo{ID: 1, Name: model.NewCIStr("t")}} + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionRepairTable))) + + args, err := GetRepairTableArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs.TableInfo, args.TableInfo) + } +} + +func TestRecoverArgs(t *testing.T) { + recoverInfo := &RecoverTableInfo{ + SchemaID: 1, + DropJobID: 2, + TableInfo: &TableInfo{ + ID: 100, + Name: model.NewCIStr("table"), + }, + OldSchemaName: "old", + OldTableName: "table", + } + + inArgs := &RecoverArgs{ + RecoverInfo: &RecoverSchemaInfo{ + RecoverTableInfos: []*RecoverTableInfo{recoverInfo}, + }, + CheckFlag: 2, + } + + for _, v := range []JobVersion{JobVersion1, JobVersion2} { + for _, tp := range []ActionType{ActionRecoverTable, ActionRecoverSchema} { + j2 := &Job{} + require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, tp))) + + args, err := GetRecoverArgs(j2) + require.NoError(t, err) + require.Equal(t, inArgs.CheckFlag, args.CheckFlag) + require.Equal(t, inArgs.RecoverInfo, args.RecoverInfo) + } + } +} + func TestPlacementPolicyArgs(t *testing.T) { inArgs := &PlacementPolicyArgs{ Policy: &PolicyInfo{ID: 1, Name: model.NewCIStr("policy"), State: StateDeleteOnly}, diff --git a/pkg/planner/core/binary_plan_test.go b/pkg/planner/core/binary_plan_test.go index 71d778d634d5f..21ec04ec7e868 100644 --- a/pkg/planner/core/binary_plan_test.go +++ b/pkg/planner/core/binary_plan_test.go @@ -143,7 +143,6 @@ func TestTooLongBinaryPlan(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists th") - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;") tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i") @@ -205,7 +204,6 @@ func TestLongBinaryPlan(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists th") - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 1000;") tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i") diff --git a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json index 8bdc0d82ce056..d00d6941014cb 100644 --- a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json @@ -406,11 +406,11 @@ "└─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 4439.11 mpp[tiflash] test.t.a, Column#5", " └─Projection 4439.11 mpp[tiflash] Column#5, test.t.a", - " └─HashAgg 4439.11 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#14)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─HashAgg 4439.11 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#8)->Column#5, funcs:firstrow(test.t.a)->test.t.a", " └─ExchangeReceiver 4439.11 mpp[tiflash] ", " └─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", - " └─HashAgg 4439.11 mpp[tiflash] group by:Column#17, Column#18, funcs:sum(Column#16)->Column#14", - " └─Projection 5548.89 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#16, test.t.a->Column#17, test.t.c->Column#18", + " └─HashAgg 4439.11 mpp[tiflash] group by:Column#11, Column#12, funcs:sum(Column#10)->Column#8", + " └─Projection 5548.89 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#10, test.t.a->Column#11, test.t.c->Column#12", " └─Selection 5548.89 mpp[tiflash] or(lt(test.t.b, 2), gt(test.t.a, 2))", " └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo" ], diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 918e632026d99..c098a8320b796 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -1377,6 +1377,10 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte for _, candidate := range candidates { path := candidate.path if path.PartialIndexPaths != nil { + // prefer tiflash, while current table path is tikv, skip it. + if ds.PreferStoreType&h.PreferTiFlash != 0 && path.StoreType == kv.TiKV { + continue + } idxMergeTask, err := convertToIndexMergeScan(ds, prop, candidate, opt) if err != nil { return nil, 0, err @@ -1512,9 +1516,11 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte } } if path.IsTablePath() { + // prefer tiflash, while current table path is tikv, skip it. if ds.PreferStoreType&h.PreferTiFlash != 0 && path.StoreType == kv.TiKV { continue } + // prefer tikv, while current table path is tiflash, skip it. if ds.PreferStoreType&h.PreferTiKV != 0 && path.StoreType == kv.TiFlash { continue } diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index 97856482388a2..0fca55f71a337 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -427,7 +427,6 @@ func BenchmarkEncodePlan(b *testing.B) { tk := testkit.NewTestKit(b, store) tk.MustExec("use test") tk.MustExec("drop table if exists th") - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;") tk.MustExec("set @@tidb_slow_log_threshold=200000") @@ -451,7 +450,6 @@ func BenchmarkEncodeFlatPlan(b *testing.B) { tk := testkit.NewTestKit(b, store) tk.MustExec("use test") tk.MustExec("drop table if exists th") - tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;") tk.MustExec("set @@tidb_slow_log_threshold=200000") diff --git a/pkg/planner/core/tests/partition/BUILD.bazel b/pkg/planner/core/tests/partition/BUILD.bazel new file mode 100644 index 0000000000000..d491c473419e4 --- /dev/null +++ b/pkg/planner/core/tests/partition/BUILD.bazel @@ -0,0 +1,25 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "partition_test", + timeout = "short", + srcs = ["bench_test.go"], + flaky = True, + deps = [ + "//pkg/autoid_service", + "//pkg/config", + "//pkg/domain", + "//pkg/expression", + "//pkg/kv", + "//pkg/session", + "//pkg/session/types", + "//pkg/store/mockstore", + "//pkg/util/benchdaily", + "//pkg/util/chunk", + "//pkg/util/logutil", + "//pkg/util/sqlexec", + "@com_github_pingcap_log//:log", + "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", + ], +) diff --git a/pkg/planner/core/tests/partition/bench_test.go b/pkg/planner/core/tests/partition/bench_test.go new file mode 100644 index 0000000000000..6fee1ad3f629e --- /dev/null +++ b/pkg/planner/core/tests/partition/bench_test.go @@ -0,0 +1,1717 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package session + +import ( + "context" + "strconv" + "strings" + "testing" + "time" + + "github.com/pingcap/log" + _ "github.com/pingcap/tidb/pkg/autoid_service" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" + "github.com/pingcap/tidb/pkg/store/mockstore" + "github.com/pingcap/tidb/pkg/util/benchdaily" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/sqlexec" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +var ( + pointQuery = "select * from t where id = 1" + pointQueryPrepared = "select * from t where id = ?" + expectedPointPlan = "Point_Get" + // IN (...) uses TryFastPlan, which does not use Plan Cache + batchPointQuery = "select * from t where id = 1 or id = 5000 or id = 2 or id = 100000" + batchPointQueryPrepared = "select * from t where id IN (?,?,?)" + expectedBatchPointPlan = "Batch_Point_Get" + expectedIndexPlan = "IndexLookUp" + expectedTableScanPlan = "TableReader" + partitionByHash = `partition by hash(id) partitions 7` + partitionByHashExpr = `partition by hash(floor(id*0.5)) partitions 7` + partitionByKey = `partition by key(id) partitions 7` + partitionByRange = `partition by range(id) (partition p0 values less than (10), partition p1 values less than (1000), partition p3 values less than (100000), partition pMax values less than (maxvalue))` + partitionByRangeExpr = `partition by range(floor(id*0.5)) (partition p0 values less than (10), partition p1 values less than (1000), partition p3 values less than (100000), partition pMax values less than (maxvalue))` + partitionByRangeColumns = `partition by range columns (id) (partition p0 values less than (10), partition p1 values less than (1000), partition p3 values less than (100000), partition pMax values less than (maxvalue))` + pointArgs = 1 + batchArgs = []any{2, 10000, 1} + partitionByRangePrep = "partition by range (id) (partition p0 values less than (10), partition p1 values less than (63), partition p3 values less than (100), partition pMax values less than (maxvalue))" + partitionByRangeExprPrep = "partition by range (floor(id*0.5)*2) (partition p0 values less than (10), partition p1 values less than (63), partition p3 values less than (100), partition pMax values less than (maxvalue))" + partitionByRangeColumnsPrep = "partition by range columns (id) (partition p0 values less than (10), partition p1 values less than (63), partition p3 values less than (100), partition pMax values less than (maxvalue))" +) + +type accessType int + +const ( + pointGet accessType = iota + indexLookup + tableScan +) + +func prepareBenchSession() (sessiontypes.Session, *domain.Domain, kv.Storage) { + config.UpdateGlobal(func(cfg *config.Config) { + cfg.Instance.EnableSlowLog.Store(false) + }) + + store, err := mockstore.NewMockStore() + if err != nil { + logutil.BgLogger().Fatal(err.Error()) + } + dom, err := session.BootstrapSession(store) + if err != nil { + logutil.BgLogger().Fatal(err.Error()) + } + // TODO: use environment variable "log_level" if set? + log.SetLevel(zapcore.ErrorLevel) + se, err := session.CreateSession4Test(store) + if err != nil { + logutil.BgLogger().Fatal(err.Error()) + } + mustExecute(se, "use test") + return se, dom, store +} + +func mustExecute(s sessiontypes.Session, sql string, args ...any) { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBootstrap) + _, err := s.ExecuteInternal(ctx, sql, args...) + defer cancel() + if err != nil { + logutil.BgLogger().Fatal("mustExecute error", zap.String("sql", sql), zap.Error(err), zap.Stack("stack")) + } +} + +func drainRecordSet(ctx context.Context, rs sqlexec.RecordSet, alloc chunk.Allocator) ([]chunk.Row, error) { + var rows []chunk.Row + var req *chunk.Chunk + req = rs.NewChunk(alloc) + for { + err := rs.Next(ctx, req) + if err != nil || req.NumRows() == 0 { + return rows, err + } + iter := chunk.NewIterator4Chunk(req) + for r := iter.Begin(); r != iter.End(); r = iter.Next() { + rows = append(rows, r) + } + req = chunk.Renew(req, 1024) + } +} + +func runPointSelect(b *testing.B, se sessiontypes.Session, query, expectedPlan string, enablePlanCache bool) { + ctx := context.Background() + alloc := chunk.NewAllocator() + if enablePlanCache { + mustExecute(se, "set tidb_enable_non_prepared_plan_cache = 1") + // Make sure to flush the plan cache, so previous benchmark can not be hit. + mustExecute(se, "set tidb_session_plan_cache_size = 0") + mustExecute(se, "create table tTemp (id int)") + mustExecute(se, "insert into tTemp values (1)") + rs, err := se.Execute(ctx, "select * from tTemp where id = 1 or id = 9") + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + alloc.Reset() + rs, err = se.Execute(ctx, "select * from tTemp where id = 1 or id IN (2,5)") + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + alloc.Reset() + mustExecute(se, "drop table tTemp") + mustExecute(se, "set tidb_session_plan_cache_size = default") + } else { + mustExecute(se, "set tidb_enable_non_prepared_plan_cache = 0") + } + rs, err := se.Execute(ctx, "explain "+query) + if err != nil { + b.Fatal(err) + } + // Note: [Batch]PointGet don't use the non-prepared plan cache, + // since it is already using the FastPlan! + expectHits := enablePlanCache && (expectedPlan != "Point_Get" && expectedPlan != "Batch_Point_Get") + checkHits := true + resStrings, err := session.ResultSetToStringSlice(ctx, se, rs[0]) + if err != nil { + logutil.BgLogger().Error("error in ResultSetToStrinSlice", zap.String("query", query), zap.Error(err)) + } + if !strings.HasPrefix(resStrings[0][0], expectedPlan) { + logutil.BgLogger().Error("expected other query plan", zap.String("query", query), zap.String("expectedPlan", expectedPlan), zap.Any("explain", resStrings)) + checkHits = false + } + hits := 0 + + b.ResetTimer() + for i := 0; i < b.N; i++ { + rs, err = se.Execute(ctx, query) + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + if checkHits && se.GetSessionVars().FoundInPlanCache { + hits++ + } + alloc.Reset() + } + b.StopTimer() + if !expectHits && checkHits && hits > 0 { + logutil.BgLogger().Error("Not expected Plan Cache to be used with PointGet", zap.Int("hits", hits), zap.Int("b.N", b.N)) + } + if expectHits && checkHits && hits == 0 && b.N > 5 { + logutil.BgLogger().Error("expected Plan Cache to be used with PointSelect", zap.Int("hits", hits), zap.Int("b.N", b.N)) + } +} + +func preparePointGet(se sessiontypes.Session, partitionBy string) { + mustExecute(se, `drop table if exists t`) + mustExecute(se, "CREATE TABLE t (id int primary key, d varchar(255), key (d)) "+partitionBy) + mustExecute(se, `insert into t (id) values (1), (8), (5000), (10000), (100000)`) + mustExecute(se, "analyze table t") +} + +func insert1kRows(se sessiontypes.Session) { + // Create 1k unique index entries, so it is cheaper with index lookup instead of table scan + // Range using: 10, 5000, 10000, 1000000 + mustExecute(se, `insert into t values (1,1), (5000,5000), (10000,10000), (99900,99900)`) + mustExecute(se, `insert into t select id + 1, d from t `) // 8 + mustExecute(se, `insert into t select id + 2, d from t `) // 16 + mustExecute(se, `insert into t select id + 4, d from t `) // 32 + mustExecute(se, `insert into t select id + 8, d from t `) // 64 + mustExecute(se, `insert into t select id + 16, d from t `) // 128 + mustExecute(se, `insert into t select id + 32, d from t `) // 256 + mustExecute(se, `insert into t select id + 64, d from t `) // 512 + mustExecute(se, `insert into t select id + 128, d from t `) // 1024 + mustExecute(se, "analyze table t") +} + +func prepareIndexLookup(se sessiontypes.Session, partitionBy string) { + mustExecute(se, `drop table if exists t`) + mustExecute(se, "CREATE TABLE t (id int, d varchar(255), key idx_id (id), key(d)) "+partitionBy) + insert1kRows(se) +} + +func prepareTableScan(se sessiontypes.Session, partitionBy string) { + mustExecute(se, `drop table if exists t`) + mustExecute(se, "CREATE TABLE t (id int, d varchar(255), key(d)) "+partitionBy) + insert1kRows(se) +} + +func benchmarkPointGetPlanCache(b *testing.B, partitionBy string) { + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + preparePointGet(se, partitionBy) + pointQuery := "select * from t where id = 1" + expectedPointPlan := "Point_Get" + b.Run("PointGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, true) + }) + b.Run("PointGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, false) + }) + // IN (...) uses TryFastPlan, which does not use Plan Cache + batchPointQuery := "select * from t where id = 1 or id = 5000 or id = 2 or id = 100000" + expectedPlan := "Batch_Point_Get" + if partitionBy != "" { + // Batch_Point_Get is not yet enabled for partitioned tables! + expectedPlan = "TableReader" + } + b.Run("BatchPointGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPlan, true) + }) + b.Run("BatchPointGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPlan, false) + }) + + // Additional tests for IndexScan + prepareIndexLookup(se, partitionBy) + expectedPointPlan = "IndexLookUp" + b.Run("IndexGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, true) + }) + b.Run("IndexGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, false) + }) + b.Run("BatchIndexGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPointPlan, true) + }) + b.Run("BatchIndexGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPointPlan, false) + }) + + // Additional tests for TableScan + mustExecute(se, "alter table t drop index idx_id") + mustExecute(se, "analyze table t") + expectedPointPlan = "TableReader" + b.Run("TableGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, true) + }) + b.Run("TableGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, pointQuery, expectedPointPlan, false) + }) + b.Run("BatchTableGetPlanCacheOn", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPointPlan, true) + }) + b.Run("BatchTableGetPlanCacheOff", func(b *testing.B) { + runPointSelect(b, se, batchPointQuery, expectedPointPlan, false) + }) +} + +func runBenchmark(b *testing.B, partitionBy, query, expectedPointPlan string, access accessType, enablePlanCache bool) { + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + switch access { + case pointGet: + preparePointGet(se, partitionBy) + case indexLookup: + prepareIndexLookup(se, partitionBy) + case tableScan: + prepareTableScan(se, partitionBy) + } + runPointSelect(b, se, query, expectedPointPlan, enablePlanCache) +} + +func BenchmarkNonPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkNonPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkNonPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedBatchPointPlan, pointGet, true) +} + +func BenchmarkNonPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedBatchPointPlan, pointGet, false) +} + +func BenchmarkNonPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkNonPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkNonPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkNonPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkNonPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkNonPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, "", pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkNonPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkNonPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, "", batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkNonPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, "") +} + +func BenchmarkHashPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkHashPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkHashPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkHashPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkHashPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkHashPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkHashPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkHashPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkHashPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkHashPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkHashPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkHashPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHash, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkHashPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByHash) +} + +func BenchmarkHashExprPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkHashExprPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkHashExprPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkHashExprPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkHashExprPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkHashExprPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkHashExprPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkHashExprPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkHashExprPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkHashExprPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkHashExprPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkHashExprPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByHashExpr, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkHashExprPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByHashExpr) +} + +func BenchmarkKeyPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkKeyPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkKeyPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkKeyPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkKeyPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkKeyPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkKeyPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkKeyPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkKeyPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkKeyPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkKeyPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkKeyPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByKey, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkKeyPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByKey) +} + +func getListPartitionDef(expr string, useColumns bool) string { + partitionBy := `partition by list` + if useColumns { + partitionBy += ` columns` + } + partitionBy += `(` + expr + `) (` + ranges := []int{1, 5000, 10000, 99900} + for partID, i := range ranges { + vals := 256 + partVals := make([]string, 0, vals) + for j := 0; j < vals; j++ { + partVals = append(partVals, strconv.Itoa(i+j)) + } + if expr != "" && i == 1 { + // for floor(id*0.5)*2 + partVals = append(partVals, "0") + } + if partID > 0 { + partitionBy += "," + } + partitionBy += "partition p" + strconv.Itoa(partID) + " values in (" + strings.Join(partVals, ",") + ")" + } + partitionBy += ")" + return partitionBy +} + +func BenchmarkListPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkListPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkListPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkListPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkListPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkListPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkListPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", false), batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, getListPartitionDef("id", false)) +} + +func BenchmarkListExprPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkListExprPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkListExprPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkListExprPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkListExprPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListExprPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkListExprPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListExprPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkListExprPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListExprPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListExprPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListExprPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListExprPartition(b *testing.B) { + partitionBy := getListPartitionDef("floor(id*0.5)*2", false) + benchmarkPointGetPlanCache(b, partitionBy) +} + +func BenchmarkListColumnsPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkListColumnsPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkListColumnsPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkListColumnsPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkListColumnsPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListColumnsPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkListColumnsPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkListColumnsPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkListColumnsPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListColumnsPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListColumnsPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkListColumnsPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, getListPartitionDef("id", true), batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkListColumnsPartition(b *testing.B) { + partitionBy := getListPartitionDef("id", true) + benchmarkPointGetPlanCache(b, partitionBy) +} + +func BenchmarkRangePartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkRangePartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkRangePartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkRangePartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkRangePartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangePartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkRangePartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangePartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkRangePartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangePartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangePartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangePartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRange, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangePartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByRange) +} + +func BenchmarkRangeExprPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkRangeExprPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkRangeExprPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkRangeExprPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkRangeExprPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangeExprPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkRangeExprPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangeExprPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkRangeExprPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangeExprPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangeExprPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangeExprPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeExpr, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangeExprPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByRangeExpr) +} + +func BenchmarkRangeColumnsPartitionPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedPointPlan, pointGet, true) +} + +func BenchmarkRangeColumnsPartitionPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedPointPlan, pointGet, false) +} + +func BenchmarkRangeColumnsPartitionBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedTableScanPlan, pointGet, true) +} + +func BenchmarkRangeColumnsPartitionBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedTableScanPlan, pointGet, false) +} + +func BenchmarkRangeColumnsPartitionIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangeColumnsPartitionIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedIndexPlan, indexLookup, false) +} + +func BenchmarkRangeColumnsPartitionBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedIndexPlan, indexLookup, true) +} + +func BenchmarkRangeColumnsPartitionBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedIndexPlan, indexLookup, false) +} +func BenchmarkRangeColumnsPartitionTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangeColumnsPartitionTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, pointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangeColumnsPartitionBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedTableScanPlan, tableScan, true) +} + +func BenchmarkRangeColumnsPartitionBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmark(b, partitionByRangeColumns, batchPointQuery, expectedTableScanPlan, tableScan, false) +} + +func BenchmarkRangeColumnsPartition(b *testing.B) { + benchmarkPointGetPlanCache(b, partitionByRangeColumns) +} + +// TODO: Add benchmarks for {RANGE|LIST} COLUMNS, multi columns!!! + +func runPreparedPointSelect(b *testing.B, se sessiontypes.Session, query string, enablePlanCache bool, args ...any) { + ctx := context.Background() + if enablePlanCache { + mustExecute(se, "set tidb_enable_prepared_plan_cache = 1") + } else { + mustExecute(se, "set tidb_enable_prepared_plan_cache = 0") + } + // TODO: add check for actual EXPLAIN, so the plan is what is expected + // see runPointSelect + stmtID, _, _, err := se.PrepareStmt(query) + if err != nil { + b.Fatal(err) + } + + hits := 0 + params := expression.Args2Expressions4Test(args...) + alloc := chunk.NewAllocator() + b.ResetTimer() + for i := 0; i < b.N; i++ { + rs, err := se.ExecutePreparedStmt(ctx, stmtID, params) + if err != nil { + b.Fatal(err) + } + if se.GetSessionVars().FoundInPlanCache { + hits++ + } + _, err = drainRecordSet(ctx, rs, alloc) + if err != nil { + b.Fatal(err) + } + if enablePlanCache && i > 0 { + if se.GetSessionVars().FoundInPlanCache { + hits++ + } else { + warns := se.GetSessionVars().StmtCtx.GetWarnings() + logutil.BgLogger().Warn("No plan cache hit", zap.Any("warns", warns)) + } + } + alloc.Reset() + } + b.StopTimer() + if enablePlanCache && hits < b.N/2 { + logutil.BgLogger().Error("Plan cache was not used enough", zap.Int("hits", hits), zap.Int("b.N", b.N)) + } +} + +func benchPreparedPointGet(b *testing.B, partitionBy string) { + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + + preparePointGet(se, partitionBy) + pointQuery := "select * from t where id = ?" + pointArgs := 1 + b.Run("PointGetPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, false, pointArgs) + }) + b.Run("PointGetPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, true, pointArgs) + }) + batchPointQuery := "select * from t where id IN (?,?,?)" + batchArgs := []any{2, 10000, 1} + b.Run("BatchPointGetPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, false, batchArgs...) + }) + b.Run("BatchPointGetPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, true, batchArgs...) + }) + + // Additional ones for testing Index + prepareIndexLookup(se, partitionBy) + + b.Run("IndexPointPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, false, pointArgs) + }) + b.Run("IndexPointPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, true, pointArgs) + }) + b.Run("IndexBatchPointPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, false, batchArgs...) + }) + b.Run("IndexBatchPointPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, true, batchArgs...) + }) + // Additional ones for testing TableScan + + mustExecute(se, "alter table t drop index idx_id") + mustExecute(se, "analyze table t") + + b.Run("TableScanPointPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, false, pointArgs) + }) + b.Run("TableScanPointPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, pointQuery, true, pointArgs) + }) + b.Run("TableScanBatchPlanCacheOff", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, false, batchArgs...) + }) + b.Run("TableScanBatchPlanCacheOn", func(b *testing.B) { + runPreparedPointSelect(b, se, batchPointQuery, true, batchArgs...) + }) +} + +func runBenchmarkPrepared(b *testing.B, partitionBy, query string, access accessType, enablePlanCache bool, qArgs ...any) { + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + switch access { + case pointGet: + preparePointGet(se, partitionBy) + case indexLookup: + prepareIndexLookup(se, partitionBy) + case tableScan: + prepareTableScan(se, partitionBy) + } + runPreparedPointSelect(b, se, query, enablePlanCache, qArgs...) +} + +func BenchmarkNonPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkNonPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkNonPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkNonPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkNonPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkNonPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkNonPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkNonPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkNonPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkNonPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkNonPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkNonPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, "", batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkNonPartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, "") +} + +func BenchmarkHashPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkHashPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkHashPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkHashPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkHashPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkHashPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkHashPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkHashPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkHashPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkHashPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkHashPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkHashPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHash, batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkHashPartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, partitionByHash) +} + +func BenchmarkHashExprPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkHashExprPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkHashExprPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkHashExprPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkHashExprPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkHashExprPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkHashExprPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByHashExpr, batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkHashExprPartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, partitionByHashExpr) +} + +func BenchmarkListPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkListPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkListPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkListPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkListPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkListPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkListPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkListPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkListPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkListPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkListPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkListPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", false), batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkListPartitionPrepared(b *testing.B) { + partitionBy := getListPartitionDef("id", false) + benchPreparedPointGet(b, partitionBy) +} + +func BenchmarkListExprPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkListExprPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkListExprPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkListExprPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkListExprPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkListExprPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkListExprPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkListExprPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkListExprPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkListExprPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkListExprPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkListExprPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("floor(id*0.5)*2", false), batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkListExprPartitionPrepared(b *testing.B) { + partitionBy := getListPartitionDef("floor(id*0.5)*2", false) + benchPreparedPointGet(b, partitionBy) +} + +func BenchmarkListColumnsPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkListColumnsPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkListColumnsPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkListColumnsPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkListColumnsPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkListColumnsPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkListColumnsPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, getListPartitionDef("id", true), batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkListColumnPartitionPrepared(b *testing.B) { + partitionBy := getListPartitionDef("id", true) + benchPreparedPointGet(b, partitionBy) +} + +func BenchmarkRangePartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkRangePartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkRangePartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkRangePartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkRangePartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkRangePartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkRangePartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkRangePartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkRangePartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkRangePartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkRangePartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkRangePartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangePrep, batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkRangePartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, partitionByRangePrep) +} + +func BenchmarkRangeExprPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkRangeExprPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkRangeExprPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkRangeExprPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkRangeExprPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkRangeExprPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkRangeExprPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeExprPrep, batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkRangeExprPartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, partitionByRangeExprPrep) +} + +func BenchmarkRangeColumnsPartitionPreparedPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, pointGet, true, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, pointGet, false, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchPointGetPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, pointGet, true, batchArgs...) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchPointGetPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, pointGet, false, batchArgs...) +} + +func BenchmarkRangeColumnsPartitionPreparedIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, indexLookup, true, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, indexLookup, false, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchIndexLookupPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, indexLookup, true, batchArgs...) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchIndexLookupPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, indexLookup, false, batchArgs...) +} +func BenchmarkRangeColumnsPartitionPreparedTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, tableScan, true, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, pointQueryPrepared, tableScan, false, pointArgs) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchTableScanPlanCacheOn(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, tableScan, true, batchArgs...) +} + +func BenchmarkRangeColumnsPartitionPreparedBatchTableScanPlanCacheOff(b *testing.B) { + runBenchmarkPrepared(b, partitionByRangeColumnsPrep, batchPointQueryPrepared, tableScan, false, batchArgs...) +} + +func BenchmarkRangeColumnPartitionPrepared(b *testing.B) { + benchPreparedPointGet(b, partitionByRangeColumnsPrep) +} + +// TODO: Add benchmarks for {RANGE|LIST} COLUMNS, both single and multi columns!!! + +func BenchmarkHashPartitionMultiPointSelect(b *testing.B) { + ctx := context.Background() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + + alloc := chunk.NewAllocator() + mustExecute(se, `create table t (id int primary key, dt datetime) partition by hash(id) partitions 64`) + b.ResetTimer() + for i := 0; i < b.N; i++ { + rs, err := se.Execute(ctx, "select * from t where id = 2330") + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + rs, err = se.Execute(ctx, "select * from t where id = 1233 or id = 1512") + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + rs, err = se.Execute(ctx, "select * from t where id in (117, 1233, 15678)") + if err != nil { + b.Fatal(err) + } + _, err = drainRecordSet(ctx, rs[0], alloc) + if err != nil { + b.Fatal(err) + } + alloc.Reset() + } + b.StopTimer() +} + +func TestBenchDaily(t *testing.T) { + benchdaily.Run( + BenchmarkNonPartitionPointGetPlanCacheOn, + //BenchmarkNonPartitionPointGetPlanCacheOff, + BenchmarkNonPartitionBatchPointGetPlanCacheOn, + //BenchmarkNonPartitionBatchPointGetPlanCacheOff, + BenchmarkNonPartitionIndexLookupPlanCacheOn, + //BenchmarkNonPartitionIndexLookupPlanCacheOff, + BenchmarkNonPartitionBatchIndexLookupPlanCacheOn, + //BenchmarkNonPartitionBatchIndexLookupPlanCacheOff, + //BenchmarkNonPartitionTableScanPlanCacheOn, + //BenchmarkNonPartitionTableScanPlanCacheOff, + //BenchmarkNonPartitionBatchTableScanPlanCacheOn, + //BenchmarkNonPartitionBatchTableScanPlanCacheOff, + BenchmarkHashPartitionPointGetPlanCacheOn, + //BenchmarkHashPartitionPointGetPlanCacheOff, + BenchmarkHashPartitionBatchPointGetPlanCacheOn, + //BenchmarkHashPartitionBatchPointGetPlanCacheOff, + BenchmarkHashPartitionIndexLookupPlanCacheOn, + //BenchmarkHashPartitionIndexLookupPlanCacheOff, + BenchmarkHashPartitionBatchIndexLookupPlanCacheOn, + //BenchmarkHashPartitionBatchIndexLookupPlanCacheOff, + //BenchmarkHashPartitionTableScanPlanCacheOn, + //BenchmarkHashPartitionTableScanPlanCacheOff, + //BenchmarkHashPartitionBatchTableScanPlanCacheOn, + //BenchmarkHashPartitionBatchTableScanPlanCacheOff, + /* + BenchmarkHashExprPartitionPointGetPlanCacheOn, + BenchmarkHashExprPartitionPointGetPlanCacheOff, + BenchmarkHashExprPartitionBatchPointGetPlanCacheOn, + BenchmarkHashExprPartitionBatchPointGetPlanCacheOff, + BenchmarkHashExprPartitionIndexLookupPlanCacheOn, + BenchmarkHashExprPartitionIndexLookupPlanCacheOff, + BenchmarkHashExprPartitionBatchIndexLookupPlanCacheOn, + BenchmarkHashExprPartitionBatchIndexLookupPlanCacheOff, + BenchmarkHashExprPartitionTableScanPlanCacheOn, + BenchmarkHashExprPartitionTableScanPlanCacheOff, + BenchmarkHashExprPartitionBatchTableScanPlanCacheOn, + BenchmarkHashExprPartitionBatchTableScanPlanCacheOff, + BenchmarkKeyPartitionPointGetPlanCacheOn, + BenchmarkKeyPartitionPointGetPlanCacheOff, + BenchmarkKeyPartitionBatchPointGetPlanCacheOn, + BenchmarkKeyPartitionBatchPointGetPlanCacheOff, + BenchmarkKeyPartitionIndexLookupPlanCacheOn, + BenchmarkKeyPartitionIndexLookupPlanCacheOff, + BenchmarkKeyPartitionBatchIndexLookupPlanCacheOn, + BenchmarkKeyPartitionBatchIndexLookupPlanCacheOff, + BenchmarkKeyPartitionTableScanPlanCacheOn, + BenchmarkKeyPartitionTableScanPlanCacheOff, + BenchmarkKeyPartitionBatchTableScanPlanCacheOn, + BenchmarkKeyPartitionBatchTableScanPlanCacheOff, + BenchmarkListPartitionPointGetPlanCacheOn, + BenchmarkListPartitionPointGetPlanCacheOff, + BenchmarkListPartitionBatchPointGetPlanCacheOn, + BenchmarkListPartitionBatchPointGetPlanCacheOff, + BenchmarkListPartitionIndexLookupPlanCacheOn, + BenchmarkListPartitionIndexLookupPlanCacheOff, + BenchmarkListPartitionBatchIndexLookupPlanCacheOn, + BenchmarkListPartitionBatchIndexLookupPlanCacheOff, + BenchmarkListPartitionTableScanPlanCacheOn, + BenchmarkListPartitionTableScanPlanCacheOff, + BenchmarkListPartitionBatchTableScanPlanCacheOn, + BenchmarkListPartitionBatchTableScanPlanCacheOff, + BenchmarkListExprPartitionPointGetPlanCacheOn, + BenchmarkListExprPartitionPointGetPlanCacheOff, + BenchmarkListExprPartitionBatchPointGetPlanCacheOn, + BenchmarkListExprPartitionBatchPointGetPlanCacheOff, + BenchmarkListExprPartitionIndexLookupPlanCacheOn, + BenchmarkListExprPartitionIndexLookupPlanCacheOff, + BenchmarkListExprPartitionBatchIndexLookupPlanCacheOn, + BenchmarkListExprPartitionBatchIndexLookupPlanCacheOff, + BenchmarkListExprPartitionTableScanPlanCacheOn, + BenchmarkListExprPartitionTableScanPlanCacheOff, + BenchmarkListExprPartitionBatchTableScanPlanCacheOn, + BenchmarkListExprPartitionBatchTableScanPlanCacheOff, + BenchmarkListColumnsPartitionPointGetPlanCacheOn, + BenchmarkListColumnsPartitionPointGetPlanCacheOff, + BenchmarkListColumnsPartitionBatchPointGetPlanCacheOn, + BenchmarkListColumnsPartitionBatchPointGetPlanCacheOff, + BenchmarkListColumnsPartitionIndexLookupPlanCacheOn, + BenchmarkListColumnsPartitionIndexLookupPlanCacheOff, + BenchmarkListColumnsPartitionBatchIndexLookupPlanCacheOn, + BenchmarkListColumnsPartitionBatchIndexLookupPlanCacheOff, + BenchmarkListColumnsPartitionTableScanPlanCacheOn, + BenchmarkListColumnsPartitionTableScanPlanCacheOff, + BenchmarkListColumnsPartitionBatchTableScanPlanCacheOn, + BenchmarkListColumnsPartitionBatchTableScanPlanCacheOff, + */ + BenchmarkRangePartitionPointGetPlanCacheOn, + //BenchmarkRangePartitionPointGetPlanCacheOff, + BenchmarkRangePartitionBatchPointGetPlanCacheOn, + //BenchmarkRangePartitionBatchPointGetPlanCacheOff, + BenchmarkRangePartitionIndexLookupPlanCacheOn, + //BenchmarkRangePartitionIndexLookupPlanCacheOff, + BenchmarkRangePartitionBatchIndexLookupPlanCacheOn, + //BenchmarkRangePartitionBatchIndexLookupPlanCacheOff, + //BenchmarkRangePartitionTableScanPlanCacheOn, + //BenchmarkRangePartitionTableScanPlanCacheOff, + //BenchmarkRangePartitionBatchTableScanPlanCacheOn, + //BenchmarkRangePartitionBatchTableScanPlanCacheOff, + /* + BenchmarkRangeExprPartitionPointGetPlanCacheOn, + BenchmarkRangeExprPartitionPointGetPlanCacheOff, + BenchmarkRangeExprPartitionBatchPointGetPlanCacheOn, + BenchmarkRangeExprPartitionBatchPointGetPlanCacheOff, + BenchmarkRangeExprPartitionIndexLookupPlanCacheOn, + BenchmarkRangeExprPartitionIndexLookupPlanCacheOff, + BenchmarkRangeExprPartitionBatchIndexLookupPlanCacheOn, + BenchmarkRangeExprPartitionBatchIndexLookupPlanCacheOff, + BenchmarkRangeExprPartitionTableScanPlanCacheOn, + BenchmarkRangeExprPartitionTableScanPlanCacheOff, + BenchmarkRangeExprPartitionBatchTableScanPlanCacheOn, + BenchmarkRangeExprPartitionBatchTableScanPlanCacheOff, + BenchmarkRangeColumnsPartitionPointGetPlanCacheOn, + BenchmarkRangeColumnsPartitionPointGetPlanCacheOff, + BenchmarkRangeColumnsPartitionBatchPointGetPlanCacheOn, + BenchmarkRangeColumnsPartitionBatchPointGetPlanCacheOff, + BenchmarkRangeColumnsPartitionIndexLookupPlanCacheOn, + BenchmarkRangeColumnsPartitionIndexLookupPlanCacheOff, + BenchmarkRangeColumnsPartitionBatchIndexLookupPlanCacheOn, + BenchmarkRangeColumnsPartitionBatchIndexLookupPlanCacheOff, + BenchmarkRangeColumnsPartitionTableScanPlanCacheOn, + BenchmarkRangeColumnsPartitionTableScanPlanCacheOff, + BenchmarkRangeColumnsPartitionBatchTableScanPlanCacheOn, + BenchmarkRangeColumnsPartitionBatchTableScanPlanCacheOff, + BenchmarkNonPartitionPreparedPointGetPlanCacheOn, + BenchmarkNonPartitionPreparedPointGetPlanCacheOff, + BenchmarkNonPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkNonPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkNonPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkNonPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkNonPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkNonPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkNonPartitionPreparedTableScanPlanCacheOn, + BenchmarkNonPartitionPreparedTableScanPlanCacheOff, + BenchmarkNonPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkNonPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkHashPartitionPreparedPointGetPlanCacheOn, + BenchmarkHashPartitionPreparedPointGetPlanCacheOff, + BenchmarkHashPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkHashPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkHashPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkHashPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkHashPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkHashPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkHashPartitionPreparedTableScanPlanCacheOn, + BenchmarkHashPartitionPreparedTableScanPlanCacheOff, + BenchmarkHashPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkHashPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkHashExprPartitionPreparedPointGetPlanCacheOn, + BenchmarkHashExprPartitionPreparedPointGetPlanCacheOff, + BenchmarkHashExprPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkHashExprPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkHashExprPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkHashExprPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkHashExprPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkHashExprPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkHashExprPartitionPreparedTableScanPlanCacheOn, + BenchmarkHashExprPartitionPreparedTableScanPlanCacheOff, + BenchmarkHashExprPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkHashExprPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkListPartitionPreparedPointGetPlanCacheOn, + BenchmarkListPartitionPreparedPointGetPlanCacheOff, + BenchmarkListPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkListPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkListPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkListPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkListPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkListPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkListPartitionPreparedTableScanPlanCacheOn, + BenchmarkListPartitionPreparedTableScanPlanCacheOff, + BenchmarkListPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkListPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkListExprPartitionPreparedPointGetPlanCacheOn, + BenchmarkListExprPartitionPreparedPointGetPlanCacheOff, + BenchmarkListExprPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkListExprPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkListExprPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkListExprPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkListExprPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkListExprPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkListExprPartitionPreparedTableScanPlanCacheOn, + BenchmarkListExprPartitionPreparedTableScanPlanCacheOff, + BenchmarkListExprPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkListExprPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkListColumnsPartitionPreparedPointGetPlanCacheOn, + BenchmarkListColumnsPartitionPreparedPointGetPlanCacheOff, + BenchmarkListColumnsPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkListColumnsPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkListColumnsPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkListColumnsPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkListColumnsPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkListColumnsPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkListColumnsPartitionPreparedTableScanPlanCacheOn, + BenchmarkListColumnsPartitionPreparedTableScanPlanCacheOff, + BenchmarkListColumnsPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkListColumnsPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkRangePartitionPreparedPointGetPlanCacheOn, + BenchmarkRangePartitionPreparedPointGetPlanCacheOff, + BenchmarkRangePartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkRangePartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkRangePartitionPreparedIndexLookupPlanCacheOn, + BenchmarkRangePartitionPreparedIndexLookupPlanCacheOff, + BenchmarkRangePartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkRangePartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkRangePartitionPreparedTableScanPlanCacheOn, + BenchmarkRangePartitionPreparedTableScanPlanCacheOff, + BenchmarkRangePartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkRangePartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkRangeExprPartitionPreparedPointGetPlanCacheOn, + BenchmarkRangeExprPartitionPreparedPointGetPlanCacheOff, + BenchmarkRangeExprPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkRangeExprPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkRangeExprPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkRangeExprPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkRangeExprPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkRangeExprPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkRangeExprPartitionPreparedTableScanPlanCacheOn, + BenchmarkRangeExprPartitionPreparedTableScanPlanCacheOff, + BenchmarkRangeExprPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkRangeExprPartitionPreparedBatchTableScanPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedPointGetPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedPointGetPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedBatchPointGetPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedBatchPointGetPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedIndexLookupPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedIndexLookupPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedBatchIndexLookupPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedBatchIndexLookupPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedTableScanPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedTableScanPlanCacheOff, + BenchmarkRangeColumnsPartitionPreparedBatchTableScanPlanCacheOn, + BenchmarkRangeColumnsPartitionPreparedBatchTableScanPlanCacheOff, + */ + ) +} diff --git a/pkg/privilege/privileges/ldap/ldap_common_test.go b/pkg/privilege/privileges/ldap/ldap_common_test.go index 2d84982d7ffa0..e5e1d11f69a32 100644 --- a/pkg/privilege/privileges/ldap/ldap_common_test.go +++ b/pkg/privilege/privileges/ldap/ldap_common_test.go @@ -22,6 +22,7 @@ import ( "fmt" "math/rand" "net" + "strconv" "sync" "testing" "time" @@ -52,7 +53,7 @@ func TestConnectThrough636(t *testing.T) { startListen := make(chan struct{}) // this test only tests whether the LDAP with LTS enabled will fallback from StartTLS - randomTLSServicePort := rand.Int()%10000 + 10000 + var randomTLSServiceAddress string serverWg := &sync.WaitGroup{} serverWg.Add(1) go func() { @@ -64,8 +65,11 @@ func TestConnectThrough636(t *testing.T) { tlsConfig := &tls.Config{ Certificates: []tls.Certificate{cert}, } - ln, err = tls.Listen("tcp", fmt.Sprintf("localhost:%d", randomTLSServicePort), tlsConfig) + ln, err = tls.Listen("tcp", ":0", tlsConfig) + require.NoError(t, err) + + randomTLSServiceAddress = ln.Addr().String() startListen <- struct{}{} for { @@ -100,7 +104,11 @@ func TestConnectThrough636(t *testing.T) { impl := &ldapAuthImpl{} impl.SetEnableTLS(true) impl.SetLDAPServerHost("localhost") - impl.SetLDAPServerPort(randomTLSServicePort) + _, port, err := net.SplitHostPort(randomTLSServiceAddress) + require.NoError(t, err) + p, err := strconv.Atoi(port) + require.NoError(t, err) + impl.SetLDAPServerPort(p) impl.caPool = x509.NewCertPool() require.True(t, impl.caPool.AppendCertsFromPEM(tlsCAStr)) diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index cbc7150e85c30..98137f1bf7ce1 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -1106,9 +1106,6 @@ type SessionVars struct { // OptimizerEnableNAAJ enables TiDB to use null-aware anti join. OptimizerEnableNAAJ bool - // EnableTablePartition enables table partition feature. - EnableTablePartition string - // EnableCascadesPlanner enables the cascades planner. EnableCascadesPlanner bool diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index cc1005a50fa25..323c69eb44830 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -2075,9 +2075,11 @@ var defaultSysVars = []*SysVar{ s.SetEnableIndexMerge(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, SetSession: func(s *SessionVars, val string) error { - s.EnableTablePartition = val - return nil + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { + if s == Off { + vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases")) + } + return On, nil }}, // Keeping tidb_enable_list_partition here, to give errors if setting it to anything other than ON {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableListTablePartition, Value: On, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue, _ string, _ ScopeFlag) (string, error) { diff --git a/pkg/sessionctx/variable/varsutil_test.go b/pkg/sessionctx/variable/varsutil_test.go index e997e1555f461..0e0610a3cb2ec 100644 --- a/pkg/sessionctx/variable/varsutil_test.go +++ b/pkg/sessionctx/variable/varsutil_test.go @@ -258,14 +258,6 @@ func TestVarsutil(t *testing.T) { require.Equal(t, "3", val) require.Equal(t, int64(3), v.RetryLimit) - require.Equal(t, "", v.EnableTablePartition) - err = v.SetSystemVar(TiDBEnableTablePartition, "on") - require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBEnableTablePartition) - require.NoError(t, err) - require.Equal(t, "ON", val) - require.Equal(t, "ON", v.EnableTablePartition) - require.Equal(t, DefTiDBOptJoinReorderThreshold, v.TiDBOptJoinReorderThreshold) err = v.SetSystemVar(TiDBOptJoinReorderThreshold, "5") require.NoError(t, err) @@ -493,10 +485,6 @@ func TestValidate(t *testing.T) { {SecureAuth, "3", true}, {MyISAMUseMmap, "ON", false}, {MyISAMUseMmap, "OFF", false}, - {TiDBEnableTablePartition, "ON", false}, - {TiDBEnableTablePartition, "OFF", false}, - {TiDBEnableTablePartition, "AUTO", false}, - {TiDBEnableTablePartition, "UN", true}, {TiDBOptCorrelationExpFactor, "a", true}, {TiDBOptCorrelationExpFactor, "-10", false}, {TiDBOptCorrelationThreshold, "a", true}, diff --git a/pkg/statistics/handle/storage/gc_test.go b/pkg/statistics/handle/storage/gc_test.go index 020ebdad84737..42dd3d39a1137 100644 --- a/pkg/statistics/handle/storage/gc_test.go +++ b/pkg/statistics/handle/storage/gc_test.go @@ -63,7 +63,6 @@ func TestGCPartition(t *testing.T) { testKit.MustExec("set @@tidb_analyze_version = 1") testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") - testKit.MustExec("set @@session.tidb_enable_table_partition=1") testKit.MustExec(`create table t (a bigint(64), b bigint(64), index idx(a, b)) partition by range (a) ( partition p0 values less than (3), diff --git a/pkg/table/tables/test/partition/partition_test.go b/pkg/table/tables/test/partition/partition_test.go index 99634fae16dc8..3e1155a0a32fc 100644 --- a/pkg/table/tables/test/partition/partition_test.go +++ b/pkg/table/tables/test/partition/partition_test.go @@ -154,8 +154,6 @@ func TestHashPartitionAddRecord(t *testing.T) { require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), "drop table if exists t1;") require.NoError(t, err) - _, err = tk.Session().Execute(context.Background(), "set @@session.tidb_enable_table_partition = '1';") - require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY HASH (id) partitions 4;`) require.NoError(t, err) tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) diff --git a/pkg/util/dbterror/ddl_terror.go b/pkg/util/dbterror/ddl_terror.go index 931c99a7171a0..0c5dd5fecedd8 100644 --- a/pkg/util/dbterror/ddl_terror.go +++ b/pkg/util/dbterror/ddl_terror.go @@ -168,8 +168,6 @@ var ( ErrFunctionalIndexRowValueIsNotAllowed = ClassDDL.NewStd(mysql.ErrFunctionalIndexRowValueIsNotAllowed) // ErrUnsupportedCreatePartition returns for does not support create partitions. ErrUnsupportedCreatePartition = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "partition type, treat as normal table"), nil)) - // ErrTablePartitionDisabled returns for table partition is disabled. - ErrTablePartitionDisabled = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Partitions are ignored because Table Partition is disabled, please set 'tidb_enable_table_partition' if you need to need to enable it", nil)) // ErrUnsupportedIndexType returns for unsupported index type. ErrUnsupportedIndexType = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "index type"), nil)) // ErrWindowInvalidWindowFuncUse returns for invalid window function use. diff --git a/tests/integrationtest/r/ddl/db_partition.result b/tests/integrationtest/r/ddl/db_partition.result index d78ae88d8e494..d85be09d2b988 100644 --- a/tests/integrationtest/r/ddl/db_partition.result +++ b/tests/integrationtest/r/ddl/db_partition.result @@ -1,64 +1,3 @@ -set @@session.tidb_enable_table_partition = 'ON'; -drop table if exists t; -create table t (id int) partition by list (id) ( -partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -Table Create Table -t CREATE TABLE `t` ( - `id` int(11) DEFAULT NULL -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin -PARTITION BY LIST (`id`) -(PARTITION `p0` VALUES IN (1,2), - PARTITION `p1` VALUES IN (3,4)) -alter table t add partition ( -partition p4 values in (7), -partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 'OFF'; -drop table if exists t; -create table t (id int) partition by list (id) ( -partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -Table Create Table -t CREATE TABLE `t` ( - `id` int(11) DEFAULT NULL -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin -alter table t add partition ( -partition p4 values in (7), -partition p5 values in (8,9)); -Error 1505 (HY000): Partition management on a not partitioned table is not possible -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 0; -drop table if exists t; -create table t (id int) partition by list (id) ( -partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -Table Create Table -t CREATE TABLE `t` ( - `id` int(11) DEFAULT NULL -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin -alter table t add partition ( -partition p4 values in (7), -partition p5 values in (8,9)); -Error 1505 (HY000): Partition management on a not partitioned table is not possible -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 'ON'; -drop table if exists t; -create table t (id int) partition by list (id) ( -partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -Table Create Table -t CREATE TABLE `t` ( - `id` int(11) DEFAULT NULL -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin -PARTITION BY LIST (`id`) -(PARTITION `p0` VALUES IN (1,2), - PARTITION `p1` VALUES IN (3,4)) -alter table t add partition ( -partition p4 values in (7), -partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = default; drop table if exists t; create table t (a varchar(255) charset utf8mb4 collate utf8mb4_bin) partition by range columns (a) (partition p0A values less than ("A"), @@ -1840,7 +1779,6 @@ t CREATE TABLE `t` ( `store_id` int(11) DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY KEY (`hired`) PARTITIONS 11 -set @@session.tidb_enable_table_partition = 1; drop database if exists test_db_with_partition_const; create database test_db_with_partition_const; use test_db_with_partition_const; @@ -1909,7 +1847,6 @@ partition by range(day(time_recorded)) ( partition p0 values less than (1) ); Error 1486 (HY000): Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed -set @@session.tidb_enable_table_partition = default; drop table if exists test_1465; create table test_1465 (a int) partition by range(a) ( @@ -3100,7 +3037,6 @@ Error 1526 (HY000): Table has no partition for value from column_list insert into t values ('2023-08-31', 'c', "FAIL"); alter table rcp EXCHANGE PARTITION p20230829 WITH TABLE t; Error 1737 (HY000): Found a row that does not match the partition -set @@session.tidb_enable_table_partition = 1; drop table if exists t1, t2, t3, t4, t5; create table t1 ( time_recorded datetime ) partition by range(TO_DAYS(time_recorded)) ( @@ -3118,7 +3054,6 @@ create table t5 ( time_recorded timestamp ) partition by range(unix_timestamp(time_recorded)) ( partition p1 values less than (1559192604) ); -set @@session.tidb_enable_table_partition = default; set @@tidb_enable_exchange_partition=1; drop table if exists pt1; create table pt1(a int, b int, c int) PARTITION BY hash (a) partitions 1; diff --git a/tests/integrationtest/r/executor/executor.result b/tests/integrationtest/r/executor/executor.result index 40aecdb0bb4af..d77fc2b8d3260 100644 --- a/tests/integrationtest/r/executor/executor.result +++ b/tests/integrationtest/r/executor/executor.result @@ -2474,7 +2474,6 @@ select a from (select 1 a), (select 2 a); Error 1052 (23000): Column 'a' in field list is ambiguous set sql_mode = default; drop table if exists th; -set @@session.tidb_enable_table_partition = '1'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8); @@ -2504,7 +2503,6 @@ select * from th where a=5; a b 5 5 drop table if exists th; -set @@session.tidb_enable_table_partition = default; drop table if exists view_t; create table view_t (a int,b int); insert into view_t values(1,2); diff --git a/tests/integrationtest/r/executor/partition/write.result b/tests/integrationtest/r/executor/partition/write.result index 4202344c8a1a6..805995e69c696 100644 --- a/tests/integrationtest/r/executor/partition/write.result +++ b/tests/integrationtest/r/executor/partition/write.result @@ -718,7 +718,6 @@ affected rows: 1 info: drop table t1; set tidb_opt_fix_control=default; -set @@session.tidb_enable_table_partition = '1'; drop table if exists replace_test; create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) partition by hash(id) partitions 4; @@ -807,7 +806,6 @@ replace into tIssue989(a, b) values (111, 2); select * from tIssue989; a b 111 2 -set @@session.tidb_enable_table_partition = default; drop table if exists insert_update_ignore_test; create table insert_update_ignore_test (a int) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200)); insert ignore into insert_update_ignore_test values(1000); diff --git a/tests/integrationtest/r/select.result b/tests/integrationtest/r/select.result index edf2817208647..1ad0adbb023af 100644 --- a/tests/integrationtest/r/select.result +++ b/tests/integrationtest/r/select.result @@ -356,7 +356,6 @@ id estRows task access object operator info Projection_3 1.00 root sysdate()->Column#1, sleep(1)->Column#2, sysdate()->Column#3 └─TableDual_4 1.00 root rows:1 drop table if exists th; -set @@session.tidb_enable_table_partition = '1'; set @@session.tidb_partition_prune_mode = 'static'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); diff --git a/tests/integrationtest/r/session/variable.result b/tests/integrationtest/r/session/variable.result index 17d55640abc50..b13376a27b8eb 100644 --- a/tests/integrationtest/r/session/variable.result +++ b/tests/integrationtest/r/session/variable.result @@ -80,13 +80,16 @@ id DROP TABLE t_sel_in_share; set @@tidb_enable_noop_functions = default; set tidb_enable_table_partition=off; +show warnings; +Level Code Message +Warning 1105 tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases +set global tidb_enable_table_partition=off; +show warnings; +Level Code Message +Warning 1105 tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases show variables like 'tidb_enable_table_partition'; Variable_name Value -tidb_enable_table_partition OFF -set global tidb_enable_table_partition = on; -show variables like 'tidb_enable_table_partition'; -Variable_name Value -tidb_enable_table_partition OFF +tidb_enable_table_partition ON show global variables like 'tidb_enable_table_partition'; Variable_name Value tidb_enable_table_partition ON @@ -111,10 +114,6 @@ set tidb_enable_list_partition=1; show variables like 'tidb_enable_list_partition'; Variable_name Value tidb_enable_list_partition ON -show variables like 'tidb_enable_table_partition'; -Variable_name Value -tidb_enable_table_partition ON -set global tidb_enable_table_partition = default; set @@foreign_key_checks=0; CREATE TABLE address ( id bigint(20) NOT NULL AUTO_INCREMENT, diff --git a/tests/integrationtest/t/ddl/db_partition.test b/tests/integrationtest/t/ddl/db_partition.test index f6053a7fe98d0..d23b9ddf6edd3 100644 --- a/tests/integrationtest/t/ddl/db_partition.test +++ b/tests/integrationtest/t/ddl/db_partition.test @@ -1,44 +1,3 @@ -# TestDisableTablePartition -set @@session.tidb_enable_table_partition = 'ON'; -drop table if exists t; -create table t (id int) partition by list (id) ( - partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -alter table t add partition ( - partition p4 values in (7), - partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 'OFF'; -drop table if exists t; -create table t (id int) partition by list (id) ( - partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; --- error 1505 -alter table t add partition ( - partition p4 values in (7), - partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 0; -drop table if exists t; -create table t (id int) partition by list (id) ( - partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; --- error 1505 -alter table t add partition ( - partition p4 values in (7), - partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = 'ON'; -drop table if exists t; -create table t (id int) partition by list (id) ( - partition p0 values in (1,2),partition p1 values in (3,4)); -show create table t; -alter table t add partition ( - partition p4 values in (7), - partition p5 values in (8,9)); -insert into t values (1),(3),(7),(9); -set @@session.tidb_enable_table_partition = default; - # TestPartitionRangeColumnsCollate drop table if exists t; create table t (a varchar(255) charset utf8mb4 collate utf8mb4_bin) partition by range columns (a) @@ -1214,7 +1173,6 @@ alter table t coalesce partition 2; show create table t; # TestConstAndTimezoneDepent -set @@session.tidb_enable_table_partition = 1; drop database if exists test_db_with_partition_const; create database test_db_with_partition_const; use test_db_with_partition_const; @@ -1283,7 +1241,6 @@ create table t2332 ( time_recorded timestamp ) partition by range(day(time_recorded)) ( partition p0 values less than (1) ); -set @@session.tidb_enable_table_partition = default; # TestUnsupportedPartitionManagementDDLs drop table if exists test_1465; @@ -2013,7 +1970,6 @@ alter table rcp EXCHANGE PARTITION p20230829 WITH TABLE t; # TODO: add test not in first partition (both last without maxvalue and also not last with/without maxvalue) # TestConstAndTimezoneDepent2 -set @@session.tidb_enable_table_partition = 1; drop table if exists t1, t2, t3, t4, t5; create table t1 ( time_recorded datetime ) partition by range(TO_DAYS(time_recorded)) ( @@ -2031,7 +1987,6 @@ create table t5 ( time_recorded timestamp ) partition by range(unix_timestamp(time_recorded)) ( partition p1 values less than (1559192604) ); -set @@session.tidb_enable_table_partition = default; # TestExchangePartitionExpressIndex diff --git a/tests/integrationtest/t/executor/executor.test b/tests/integrationtest/t/executor/executor.test index 55ef4a299075c..dc453b948e829 100644 --- a/tests/integrationtest/t/executor/executor.test +++ b/tests/integrationtest/t/executor/executor.test @@ -1424,7 +1424,6 @@ set sql_mode = default; # TestSelectHashPartitionTable drop table if exists th; -set @@session.tidb_enable_table_partition = '1'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8); @@ -1432,7 +1431,6 @@ select b from th order by a; select * from th where a=-2; select * from th where a=5; drop table if exists th; -set @@session.tidb_enable_table_partition = default; # TestSelectView drop table if exists view_t; diff --git a/tests/integrationtest/t/executor/partition/write.test b/tests/integrationtest/t/executor/partition/write.test index 0a269ed4e399e..87313a9103500 100644 --- a/tests/integrationtest/t/executor/partition/write.test +++ b/tests/integrationtest/t/executor/partition/write.test @@ -505,7 +505,6 @@ set tidb_opt_fix_control=default; # TestHashPartitionedTableReplace -set @@session.tidb_enable_table_partition = '1'; drop table if exists replace_test; create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) partition by hash(id) partitions 4; @@ -596,8 +595,6 @@ insert into tIssue989 (a, b) values (1, 2); replace into tIssue989(a, b) values (111, 2); select * from tIssue989; -set @@session.tidb_enable_table_partition = default; - ## test partition insert/update ignore to invalid partition drop table if exists insert_update_ignore_test; create table insert_update_ignore_test (a int) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200)); diff --git a/tests/integrationtest/t/select.test b/tests/integrationtest/t/select.test index 2dadb4434b497..dcbb5d0ea7420 100644 --- a/tests/integrationtest/t/select.test +++ b/tests/integrationtest/t/select.test @@ -181,7 +181,6 @@ desc select sysdate(), sleep(1), sysdate(); # test select partition table drop table if exists th; -set @@session.tidb_enable_table_partition = '1'; set @@session.tidb_partition_prune_mode = 'static'; create table th (a int, b int) partition by hash(a) partitions 3; insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8); @@ -310,4 +309,4 @@ SELECT * FROM t1 STRAIGHT_JOIN t2 USING(id) STRAIGHT_JOIN t4 USING(id); SELECT * FROM t1 STRAIGHT_JOIN t2 USING(i); --error 1054 -SELECT * FROM t1 STRAIGHT_JOIN t3 USING(id); \ No newline at end of file +SELECT * FROM t1 STRAIGHT_JOIN t3 USING(id); diff --git a/tests/integrationtest/t/session/variable.test b/tests/integrationtest/t/session/variable.test index d3fcb4205a6d4..353fe4c32eefe 100644 --- a/tests/integrationtest/t/session/variable.test +++ b/tests/integrationtest/t/session/variable.test @@ -68,8 +68,9 @@ set @@tidb_enable_noop_functions = default; # TestEnablePartition set tidb_enable_table_partition=off; -show variables like 'tidb_enable_table_partition'; -set global tidb_enable_table_partition = on; +show warnings; +set global tidb_enable_table_partition=off; +show warnings; show variables like 'tidb_enable_table_partition'; show global variables like 'tidb_enable_table_partition'; set tidb_enable_list_partition=on; @@ -84,9 +85,7 @@ set tidb_enable_list_partition=1; show variables like 'tidb_enable_list_partition'; connect (conn1, localhost, root,, session__variable); connection conn1; -show variables like 'tidb_enable_table_partition'; disconnect conn1; -set global tidb_enable_table_partition = default; # TestIgnoreForeignKey set @@foreign_key_checks=0;