Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ddl: combine table/partition/db id allocation with job id #54669

Merged
merged 9 commits into from
Jul 17, 2024
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions br/pkg/glue/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ type Session interface {
ExecuteInternal(ctx context.Context, sql string, args ...any) error
CreateDatabase(ctx context.Context, schema *model.DBInfo) error
CreateTable(ctx context.Context, dbName model.CIStr, table *model.TableInfo,
cs ...ddl.CreateTableWithInfoConfigurier) error
cs ...ddl.CreateTableOption) error
CreatePlacementPolicy(ctx context.Context, policy *model.PolicyInfo) error
Close()
GetGlobalVariable(name string) (string, error)
Expand All @@ -54,7 +54,7 @@ type Session interface {
// BatchCreateTableSession is an interface to batch create table parallelly
type BatchCreateTableSession interface {
CreateTables(ctx context.Context, tables map[string][]*model.TableInfo,
cs ...ddl.CreateTableWithInfoConfigurier) error
cs ...ddl.CreateTableOption) error
}

// Progress is an interface recording the current execution progress.
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/gluetidb/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,13 +212,13 @@ func (gs *tidbSession) CreatePlacementPolicy(ctx context.Context, policy *model.

// CreateTables implements glue.BatchCreateTableSession.
func (gs *tidbSession) CreateTables(_ context.Context,
tables map[string][]*model.TableInfo, cs ...ddl.CreateTableWithInfoConfigurier) error {
tables map[string][]*model.TableInfo, cs ...ddl.CreateTableOption) error {
return errors.Trace(executor.BRIECreateTables(gs.se, tables, brComment, cs...))
}

// CreateTable implements glue.Session.
func (gs *tidbSession) CreateTable(_ context.Context, dbName model.CIStr,
table *model.TableInfo, cs ...ddl.CreateTableWithInfoConfigurier) error {
table *model.TableInfo, cs ...ddl.CreateTableOption) error {
return errors.Trace(executor.BRIECreateTable(gs.se, dbName, table, brComment, cs...))
}

Expand Down
4 changes: 2 additions & 2 deletions br/pkg/gluetidb/mock/mock.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,14 +79,14 @@ func (*mockSession) CreatePlacementPolicy(_ context.Context, _ *model.PolicyInfo

// CreateTables implements glue.BatchCreateTableSession.
func (*mockSession) CreateTables(_ context.Context, _ map[string][]*model.TableInfo,
_ ...ddl.CreateTableWithInfoConfigurier) error {
_ ...ddl.CreateTableOption) error {
log.Fatal("unimplemented CreateDatabase for mock session")
return nil
}

// CreateTable implements glue.Session.
func (*mockSession) CreateTable(_ context.Context, _ model.CIStr,
_ *model.TableInfo, _ ...ddl.CreateTableWithInfoConfigurier) error {
_ *model.TableInfo, _ ...ddl.CreateTableOption) error {
log.Fatal("unimplemented CreateDatabase for mock session")
return nil
}
Expand Down
41 changes: 26 additions & 15 deletions br/pkg/restore/internal/prealloc_db/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,26 +280,29 @@ func (db *DB) CreateTablePostRestore(ctx context.Context, table *metautil.Table,
return nil
}

func (db *DB) tableIDAllocFilter() ddl.AllocTableIDIf {
return func(ti *model.TableInfo) bool {
if db.preallocedIDs == nil {
return true
}
prealloced := db.preallocedIDs.PreallocedFor(ti)
if prealloced {
log.Info("reusing table ID", zap.Stringer("table", ti.Name))
}
return !prealloced
func (db *DB) canReuseTableID(ti *model.TableInfo) bool {
if db.preallocedIDs == nil {
return false
}
prealloced := db.preallocedIDs.PreallocedFor(ti)
if prealloced {
log.Info("reusing table ID", zap.Stringer("table", ti.Name))
}
return prealloced
}

// CreateTables execute a internal CREATE TABLES.
func (db *DB) CreateTables(ctx context.Context, tables []*metautil.Table,
ddlTables map[restore.UniqueTableName]bool, supportPolicy bool, policyMap *sync.Map) error {
if batchSession, ok := db.se.(glue.BatchCreateTableSession); ok {
m := map[string][]*model.TableInfo{}
idReusableTbls := map[string][]*model.TableInfo{}
idNonReusableTbls := map[string][]*model.TableInfo{}
for _, table := range tables {
m[table.DB.Name.L] = append(m[table.DB.Name.L], table.Info)
if db.canReuseTableID(table.Info) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It splits the batched tables into idResuableTbls and idNonResuableTbls. I prefer to splits the unbatched tables before here

if rc.batchDdlSize > minBatchDdlSize && len(rc.dbPool) > 0 {

idReusableTbls[table.DB.Name.L] = append(idReusableTbls[table.DB.Name.L], table.Info)
} else {
idNonReusableTbls[table.DB.Name.L] = append(idNonReusableTbls[table.DB.Name.L], table.Info)
}
if !supportPolicy {
log.Info("set placementPolicyRef to nil when target tidb not support policy",
zap.Stringer("table", table.Info.Name), zap.Stringer("db", table.DB.Name))
Expand All @@ -314,8 +317,15 @@ func (db *DB) CreateTables(ctx context.Context, tables []*metautil.Table,
ttlInfo.Enable = false
}
}
if err := batchSession.CreateTables(ctx, m, db.tableIDAllocFilter()); err != nil {
return err
if len(idReusableTbls) > 0 {
if err := batchSession.CreateTables(ctx, idReusableTbls, ddl.WithIDAllocated(true)); err != nil {
return err
}
}
if len(idNonReusableTbls) > 0 {
if err := batchSession.CreateTables(ctx, idNonReusableTbls); err != nil {
return err
}
}

for _, table := range tables {
Expand Down Expand Up @@ -345,7 +355,8 @@ func (db *DB) CreateTable(ctx context.Context, table *metautil.Table,
ttlInfo.Enable = false
}

err := db.se.CreateTable(ctx, table.DB.Name, table.Info, db.tableIDAllocFilter())
reuseID := db.canReuseTableID(table.Info)
err := db.se.CreateTable(ctx, table.DB.Name, table.Info, ddl.WithIDAllocated(reuseID))
if err != nil {
log.Error("create table failed",
zap.Stringer("db", table.DB.Name),
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/task/restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ func TestPreCheckTableClusterIndex(t *testing.T) {
Collate: "utf8mb4_bin",
},
}
err = se.CreateTable(ctx, tables[i].DB.Name, tables[i].Info, ddl.OnExistIgnore)
err = se.CreateTable(ctx, tables[i].DB.Name, tables[i].Info, ddl.WithOnExist(ddl.OnExistIgnore))
require.NoError(t, err)
}

Expand Down
14 changes: 5 additions & 9 deletions pkg/ddl/db_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,9 +322,7 @@ func TestCreateTableWithInfo(t *testing.T) {
Name: model.NewCIStr("t"),
}}

require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), info, ddl.OnExistError, ddl.AllocTableIDIf(func(ti *model.TableInfo) bool {
return false
})))
require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError), ddl.WithIDAllocated(true)))
tk.MustQuery("select tidb_table_id from information_schema.tables where table_name = 't'").Check(testkit.Rows("42042"))
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers)

Expand All @@ -342,9 +340,7 @@ func TestCreateTableWithInfo(t *testing.T) {
Name: model.NewCIStr("tt"),
}}
tk.Session().SetValue(sessionctx.QueryString, "skip")
require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), info, ddl.OnExistError, ddl.AllocTableIDIf(func(ti *model.TableInfo) bool {
return true
})))
require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError)))
idGen, ok := tk.MustQuery("select tidb_table_id from information_schema.tables where table_name = 'tt'").Rows()[0][0].(string)
require.True(t, ok)
idGenNum, err := strconv.ParseInt(idGen, 10, 64)
Expand Down Expand Up @@ -374,7 +370,7 @@ func TestBatchCreateTable(t *testing.T) {

// correct name
tk.Session().SetValue(sessionctx.QueryString, "skip")
err := d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), infos, ddl.OnExistError)
err := d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError))
require.NoError(t, err)

tk.MustQuery("show tables like '%tables_%'").Check(testkit.Rows("tables_1", "tables_2", "tables_3"))
Expand All @@ -389,7 +385,7 @@ func TestBatchCreateTable(t *testing.T) {
// duplicated name
infos[1].Name = model.NewCIStr("tables_1")
tk.Session().SetValue(sessionctx.QueryString, "skip")
err = d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), infos, ddl.OnExistError)
err = d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError))
require.True(t, terror.ErrorEqual(err, infoschema.ErrTableExists))

newinfo := &model.TableInfo{
Expand Down Expand Up @@ -418,7 +414,7 @@ func TestBatchCreateTable(t *testing.T) {

tk.Session().SetValue(sessionctx.QueryString, "skip")
tk.Session().SetValue(sessionctx.QueryString, "skip")
err = d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), []*model.TableInfo{newinfo}, ddl.OnExistError)
err = d.BatchCreateTableWithInfo(tk.Session(), model.NewCIStr("test"), []*model.TableInfo{newinfo}, ddl.WithOnExist(ddl.OnExistError))
require.NoError(t, err)
}

Expand Down
117 changes: 67 additions & 50 deletions pkg/ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,45 +104,44 @@ const (
// OnExist specifies what to do when a new object has a name collision.
type OnExist uint8

// AllocTableIDIf specifies whether to retain the old table ID.
// If this returns "false", then we would assume the table ID has been
// allocated before calling `CreateTableWithInfo` family.
type AllocTableIDIf func(*model.TableInfo) bool

// CreateTableWithInfoConfig is the configuration of `CreateTableWithInfo`.
type CreateTableWithInfoConfig struct {
OnExist OnExist
ShouldAllocTableID AllocTableIDIf
// CreateTableConfig is the configuration of `CreateTableWithInfo`.
type CreateTableConfig struct {
OnExist OnExist
// IDAllocated indicates whether the job has allocated all IDs for tables affected
// in the job, if true, DDL will not allocate IDs for them again, it's only used
// by BR now. By reusing IDs BR can save a lot of works such as rewriting table
// IDs in backed up KVs.
IDAllocated bool
}

// CreateTableWithInfoConfigurier is the "diff" which can be applied to the
// CreateTableWithInfoConfig, currently implementations are "OnExist" and "AllocTableIDIf".
type CreateTableWithInfoConfigurier interface {
// Apply the change over the config.
Apply(*CreateTableWithInfoConfig)
}
// CreateTableOption is the option for creating table.
type CreateTableOption func(*CreateTableConfig)

// GetCreateTableWithInfoConfig applies the series of configurier from default config
// GetCreateTableConfig applies the series of config options from default config
// and returns the final config.
func GetCreateTableWithInfoConfig(cs []CreateTableWithInfoConfigurier) CreateTableWithInfoConfig {
config := CreateTableWithInfoConfig{}
func GetCreateTableConfig(cs []CreateTableOption) CreateTableConfig {
cfg := CreateTableConfig{}
for _, c := range cs {
c.Apply(&config)
}
if config.ShouldAllocTableID == nil {
config.ShouldAllocTableID = func(*model.TableInfo) bool { return true }
c(&cfg)
}
return config
return cfg
}

// Apply implements Configurier.
func (o OnExist) Apply(c *CreateTableWithInfoConfig) {
c.OnExist = o
// WithOnExist applies the OnExist option.
func WithOnExist(o OnExist) CreateTableOption {
return func(cfg *CreateTableConfig) {
cfg.OnExist = o
}
}

// Apply implements Configurier.
func (a AllocTableIDIf) Apply(c *CreateTableWithInfoConfig) {
c.ShouldAllocTableID = a
// WithIDAllocated applies the IDAllocated option.
// WARNING!!!: if idAllocated == true, DDL will NOT allocate IDs by itself. That
// means if the caller can not promise ID is unique, then we got inconsistency.
// This option is only exposed to be used by BR.
func WithIDAllocated(idAllocated bool) CreateTableOption {
return func(cfg *CreateTableConfig) {
cfg.IDAllocated = idAllocated
}
}

const (
Expand Down Expand Up @@ -216,13 +215,13 @@ type DDL interface {
schema model.CIStr,
info *model.TableInfo,
involvingRef []model.InvolvingSchemaInfo,
cs ...CreateTableWithInfoConfigurier) error
cs ...CreateTableOption) error

// BatchCreateTableWithInfo is like CreateTableWithInfo, but can handle multiple tables.
BatchCreateTableWithInfo(ctx sessionctx.Context,
schema model.CIStr,
info []*model.TableInfo,
cs ...CreateTableWithInfoConfigurier) error
cs ...CreateTableOption) error

// CreatePlacementPolicyWithInfo creates a placement policy
//
Expand Down Expand Up @@ -265,15 +264,21 @@ type DDL interface {
DoDDLJob(ctx sessionctx.Context, job *model.Job) error
}

type limitJobTask struct {
job *model.Job
// JobWrapper is used to wrap a job and some other information.
// exported for testing.
type JobWrapper struct {
*model.Job
// IDAllocated see config of same name in CreateTableConfig.
// exported for test.
IDAllocated bool
// when we combine multiple jobs into one task,
// append the errChs to this slice.
errChs []chan error
cacheErr error
}

func (t *limitJobTask) NotifyError(err error) {
// NotifyError notifies the error to all error channels.
func (t *JobWrapper) NotifyError(err error) {
for _, errCh := range t.errChs {
errCh <- err
}
Expand All @@ -283,9 +288,9 @@ func (t *limitJobTask) NotifyError(err error) {
type ddl struct {
m sync.RWMutex
wg tidbutil.WaitGroupWrapper // It's only used to deal with data race in restart_test.
limitJobCh chan *limitJobTask
limitJobCh chan *JobWrapper
// limitJobChV2 is used to limit the number of jobs being executed in local worker.
limitJobChV2 chan *limitJobTask
limitJobChV2 chan *JobWrapper

*ddlCtx
sessPool *sess.Pool
Expand All @@ -297,7 +302,7 @@ type ddl struct {
ddlJobNotifyCh chan struct{}

// localJobCh is used to delivery job in local TiDB nodes.
localJobCh chan *limitJobTask
localJobCh chan *JobWrapper
// globalIDLocal locks global id to reduce write conflict.
globalIDLock sync.Mutex
}
Expand Down Expand Up @@ -756,11 +761,11 @@ func newDDL(ctx context.Context, options ...Option) *ddl {

d := &ddl{
ddlCtx: ddlCtx,
limitJobCh: make(chan *limitJobTask, batchAddingJobs),
limitJobChV2: make(chan *limitJobTask, batchAddingJobs),
limitJobCh: make(chan *JobWrapper, batchAddingJobs),
limitJobChV2: make(chan *JobWrapper, batchAddingJobs),
enableTiFlashPoll: atomicutil.NewBool(true),
ddlJobNotifyCh: make(chan struct{}, 100),
localJobCh: make(chan *limitJobTask, 1),
localJobCh: make(chan *JobWrapper, 1),
}

taskexecutor.RegisterTaskType(proto.Backfill,
Expand Down Expand Up @@ -1152,8 +1157,8 @@ func setDDLJobMode(job *model.Job) {
job.LocalMode = false
}

func (d *ddl) deliverJobTask(task *limitJobTask) {
if task.job.LocalMode {
func (d *ddl) deliverJobTask(task *JobWrapper) {
if task.LocalMode {
d.limitJobChV2 <- task
} else {
d.limitJobCh <- task
Expand All @@ -1165,6 +1170,14 @@ func (d *ddl) deliverJobTask(task *limitJobTask) {
// - context.Cancel: job has been sent to worker, but not found in history DDL job before cancel
// - other: found in history DDL job and return that job error
func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error {
return d.doDDLJobWrapper(ctx, &JobWrapper{
Job: job,
errChs: []chan error{make(chan error)},
})
}

func (d *ddl) doDDLJobWrapper(ctx sessionctx.Context, jobW *JobWrapper) error {
job := jobW.Job
job.TraceInfo = &model.TraceInfo{
ConnectionID: ctx.GetSessionVars().ConnectionID,
SessionAlias: ctx.GetSessionVars().SessionAlias,
Expand All @@ -1177,23 +1190,27 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error {
// Get a global job ID and put the DDL job in the queue.
setDDLJobQuery(ctx, job)
setDDLJobMode(job)
task := &limitJobTask{job, []chan error{make(chan error)}, nil}
d.deliverJobTask(task)
d.deliverJobTask(jobW)

failpoint.Inject("mockParallelSameDDLJobTwice", func(val failpoint.Value) {
if val.(bool) {
<-task.errChs[0]
<-jobW.errChs[0]
// The same job will be put to the DDL queue twice.
job = job.Clone()
task1 := &limitJobTask{job, []chan error{make(chan error)}, nil}
d.deliverJobTask(task1)
newJobW := &JobWrapper{
Job: job,
IDAllocated: jobW.IDAllocated,
errChs: []chan error{make(chan error)},
}
d.deliverJobTask(newJobW)
// The second job result is used for test.
task = task1
jobW = newJobW
}
})

// worker should restart to continue handling tasks in limitJobCh, and send back through task.err
err := <-task.errChs[0]
// worker should restart to continue handling tasks in limitJobCh, and send back through jobW.err
err := <-jobW.errChs[0]
// job.ID must be allocated after previous channel receive returns nil.
defer d.delJobDoneCh(job.ID)
if err != nil {
// The transaction of enqueuing job is failed.
Expand Down
Loading
Loading