diff --git a/pkg/ddl/ddl_test.go b/pkg/ddl/ddl_test.go index 4fd597a1bb099..edf9970d415c4 100644 --- a/pkg/ddl/ddl_test.go +++ b/pkg/ddl/ddl_test.go @@ -207,84 +207,6 @@ func TestIgnorableSpec(t *testing.T) { } } -func TestBuildJobDependence(t *testing.T) { - store := createMockStore(t) - defer func() { - require.NoError(t, store.Close()) - }() - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - // Add some non-add-index jobs. - job1 := &model.Job{ID: 1, TableID: 1, Version: model.JobVersion1, Type: model.ActionAddColumn} - job2 := &model.Job{ID: 2, TableID: 1, Version: model.JobVersion1, Type: model.ActionCreateTable} - job3 := &model.Job{ID: 3, TableID: 2, Version: model.JobVersion1, Type: model.ActionDropColumn} - job6 := &model.Job{ID: 6, TableID: 1, Version: model.JobVersion1, Type: model.ActionDropTable} - job7 := &model.Job{ID: 7, TableID: 2, Version: model.JobVersion1, Type: model.ActionModifyColumn} - job9 := &model.Job{ID: 9, SchemaID: 111, Version: model.JobVersion1, Type: model.ActionDropSchema} - job11 := &model.Job{ID: 11, TableID: 2, Version: model.JobVersion1, Type: model.ActionRenameTable} - job11.FillArgs(&model.RenameTableArgs{ - OldSchemaID: 111, - NewTableName: pmodel.NewCIStr("new table name"), - OldSchemaName: pmodel.NewCIStr("old db name"), - }) - err := kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - require.NoError(t, m.EnQueueDDLJob(job1)) - require.NoError(t, m.EnQueueDDLJob(job2)) - require.NoError(t, m.EnQueueDDLJob(job3)) - require.NoError(t, m.EnQueueDDLJob(job6)) - require.NoError(t, m.EnQueueDDLJob(job7)) - require.NoError(t, m.EnQueueDDLJob(job9)) - require.NoError(t, m.EnQueueDDLJob(job11)) - return nil - }) - require.NoError(t, err) - job4 := &model.Job{ID: 4, TableID: 1, Type: model.ActionAddIndex} - err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - err := buildJobDependence(m, job4) - require.NoError(t, err) - require.Equal(t, job4.DependencyID, int64(2)) - return nil - }) - require.NoError(t, err) - job5 := &model.Job{ID: 5, TableID: 2, Type: model.ActionAddIndex} - err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - err := buildJobDependence(m, job5) - require.NoError(t, err) - require.Equal(t, job5.DependencyID, int64(3)) - return nil - }) - require.NoError(t, err) - job8 := &model.Job{ID: 8, TableID: 3, Type: model.ActionAddIndex} - err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - err := buildJobDependence(m, job8) - require.NoError(t, err) - require.Equal(t, job8.DependencyID, int64(0)) - return nil - }) - require.NoError(t, err) - job10 := &model.Job{ID: 10, SchemaID: 111, TableID: 3, Type: model.ActionAddIndex} - err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - err := buildJobDependence(m, job10) - require.NoError(t, err) - require.Equal(t, job10.DependencyID, int64(9)) - return nil - }) - require.NoError(t, err) - job12 := &model.Job{ID: 12, SchemaID: 112, TableID: 2, Type: model.ActionAddIndex} - err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMutator(txn) - err := buildJobDependence(m, job12) - require.NoError(t, err) - require.Equal(t, job12.DependencyID, int64(11)) - return nil - }) - require.NoError(t, err) -} - func TestError(t *testing.T) { kvErrs := []*terror.Error{ dbterror.ErrDDLJobNotFound, diff --git a/pkg/ddl/job_submitter.go b/pkg/ddl/job_submitter.go index 52592956fee18..04c5e6455f127 100644 --- a/pkg/ddl/job_submitter.go +++ b/pkg/ddl/job_submitter.go @@ -104,22 +104,16 @@ func (s *JobSubmitter) addBatchDDLJobs(jobWs []*JobWrapper) { err error newWs []*JobWrapper ) - // DDLForce2Queue is a flag to tell DDL worker to always push the job to the DDL queue. - toTable := !variable.DDLForce2Queue.Load() fastCreate := variable.EnableFastCreateTable.Load() - if toTable { - if fastCreate { - newWs, err = mergeCreateTableJobs(jobWs) - if err != nil { - logutil.DDLLogger().Warn("failed to merge create table jobs", zap.Error(err)) - } else { - jobWs = newWs - } + if fastCreate { + newWs, err = mergeCreateTableJobs(jobWs) + if err != nil { + logutil.DDLLogger().Warn("failed to merge create table jobs", zap.Error(err)) + } else { + jobWs = newWs } - err = s.addBatchDDLJobs2Table(jobWs) - } else { - err = s.addBatchDDLJobs2Queue(jobWs) } + err = s.addBatchDDLJobs2Table(jobWs) var jobs string for _, jobW := range jobWs { if err == nil { @@ -139,7 +133,6 @@ func (s *JobSubmitter) addBatchDDLJobs(jobWs []*JobWrapper) { logutil.DDLLogger().Info("add DDL jobs", zap.Int("batch count", len(jobWs)), zap.String("jobs", jobs), - zap.Bool("table", toTable), zap.Bool("fast_create", fastCreate)) } @@ -302,12 +295,6 @@ func (s *JobSubmitter) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { } startTS = txn.StartTS() - if variable.DDLForce2Queue.Load() { - if err := s.checkFlashbackJobInQueue(t); err != nil { - return err - } - } - return nil }) if err != nil { @@ -355,64 +342,6 @@ func (s *JobSubmitter) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { return nil } -func (s *JobSubmitter) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - return kv.RunInNewTxn(ctx, s.store, true, func(_ context.Context, txn kv.Transaction) error { - t := meta.NewMutator(txn) - - for _, jobW := range jobWs { - intest.Assert(jobW.Version != 0, "Job version should not be zero") - } - - count := getRequiredGIDCount(jobWs) - ids, err := t.GenGlobalIDs(count) - if err != nil { - return errors.Trace(err) - } - assignGIDsForJobs(jobWs, ids) - - if err := s.checkFlashbackJobInQueue(t); err != nil { - return errors.Trace(err) - } - - for _, jobW := range jobWs { - jobW.FillArgsWithSubJobs() - job := jobW.Job - job.StartTS = txn.StartTS() - setJobStateToQueueing(job) - if err = buildJobDependence(t, job); err != nil { - return errors.Trace(err) - } - jobListKey := meta.DefaultJobListKey - if job.MayNeedReorg() { - jobListKey = meta.AddIndexJobListKey - } - if err = t.EnQueueDDLJob(job, jobListKey); err != nil { - return errors.Trace(err) - } - } - failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr")) - } - }) - return nil - }) -} - -func (*JobSubmitter) checkFlashbackJobInQueue(t *meta.Mutator) error { - jobs, err := t.GetAllDDLJobsInQueue(meta.DefaultJobListKey) - if err != nil { - return errors.Trace(err) - } - for _, job := range jobs { - if job.Type == model.ActionFlashbackCluster { - return errors.Errorf("Can't add ddl job, have flashback cluster job") - } - } - return nil -} - // GenGIDAndInsertJobsWithRetry generate job related global ID and inserts DDL jobs to the DDL job // table with retry. job id allocation and job insertion are in the same transaction, // as we want to make sure DDL jobs are inserted in id order, then we can query from @@ -767,39 +696,6 @@ func setJobStateToQueueing(job *model.Job) { job.State = model.JobStateQueueing } -// buildJobDependence sets the curjob's dependency-ID. -// The dependency-job's ID must less than the current job's ID, and we need the largest one in the list. -func buildJobDependence(t *meta.Mutator, curJob *model.Job) error { - // Jobs in the same queue are ordered. If we want to find a job's dependency-job, we need to look for - // it from the other queue. So if the job is "ActionAddIndex" job, we need find its dependency-job from DefaultJobList. - jobListKey := meta.DefaultJobListKey - if !curJob.MayNeedReorg() { - jobListKey = meta.AddIndexJobListKey - } - jobs, err := t.GetAllDDLJobsInQueue(jobListKey) - if err != nil { - return errors.Trace(err) - } - - for _, job := range jobs { - if curJob.ID < job.ID { - continue - } - isDependent, err := curJob.IsDependentOn(job) - if err != nil { - return errors.Trace(err) - } - if isDependent { - logutil.DDLLogger().Info("current DDL job depends on other job", - zap.Stringer("currentJob", curJob), - zap.Stringer("dependentJob", job)) - curJob.DependencyID = job.ID - break - } - } - return nil -} - func (s *JobSubmitter) notifyNewJobSubmitted() { if s.ownerManager.IsOwner() { asyncNotify(s.ddlJobNotifyCh) diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index 4e5318be18e0c..d0ad455e8e37e 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -178,9 +178,8 @@ type Option func(m *Mutator) // Mutator is for handling meta information in a transaction. type Mutator struct { - txn *structure.TxStructure - StartTS uint64 // StartTS is the txn's start TS. - jobListKey JobListKeyType + txn *structure.TxStructure + StartTS uint64 // StartTS is the txn's start TS. } var _ Reader = (*Mutator)(nil) @@ -192,8 +191,7 @@ func NewMutator(txn kv.Transaction, options ...Option) *Mutator { txn.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) t := structure.NewStructure(txn, txn, mMetaPrefix) m := &Mutator{txn: t, - StartTS: txn.StartTS(), - jobListKey: DefaultJobListKey, + StartTS: txn.StartTS(), } for _, opt := range options { opt(m) @@ -1446,33 +1444,6 @@ var ( mDDLJobHistoryKey = []byte("DDLJobHistory") ) -var ( - // DefaultJobListKey keeps all actions of DDL jobs except "add index". - // this and below list are always appended, so the order is the same as the - // job's creation order. - DefaultJobListKey JobListKeyType = mDDLJobListKey - // AddIndexJobListKey only keeps the action of adding index. - AddIndexJobListKey JobListKeyType = mDDLJobAddIdxList -) - -func (m *Mutator) enQueueDDLJob(key []byte, job *model.Job) error { - b, err := job.Encode(true) - if err == nil { - err = m.txn.RPush(key, b) - } - return errors.Trace(err) -} - -// EnQueueDDLJob adds a DDL job to the list. -func (m *Mutator) EnQueueDDLJob(job *model.Job, jobListKeys ...JobListKeyType) error { - listKey := m.jobListKey - if len(jobListKeys) != 0 { - listKey = jobListKeys[0] - } - - return m.enQueueDDLJob(listKey, job) -} - // JobListKeyType is a key type of the DDL job queue. type JobListKeyType []byte @@ -1495,34 +1466,6 @@ func (m *Mutator) getDDLJob(key []byte, index int64) (*model.Job, error) { return job, errors.Trace(err) } -// GetAllDDLJobsInQueue gets all DDL Jobs in the current queue. -// The length of jobListKeys can only be 1 or 0. -// If its length is 1, we need to replace m.jobListKey with jobListKeys[0]. -// Otherwise, we use m.jobListKey directly. -func (m *Mutator) GetAllDDLJobsInQueue(jobListKeys ...JobListKeyType) ([]*model.Job, error) { - listKey := m.jobListKey - if len(jobListKeys) != 0 { - listKey = jobListKeys[0] - } - - values, err := m.txn.LGetAll(listKey) - if err != nil || values == nil { - return nil, errors.Trace(err) - } - - jobs := make([]*model.Job, 0, len(values)) - for _, val := range values { - job := &model.Job{} - err = job.Decode(val) - if err != nil { - return nil, errors.Trace(err) - } - jobs = append(jobs, job) - } - - return jobs, nil -} - func (*Mutator) jobIDKey(id int64) []byte { b := make([]byte, 8) binary.BigEndian.PutUint64(b, uint64(id)) diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index d75a2931ba0eb..c87f8a094a886 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -590,99 +590,6 @@ func (job *Job) String() string { return ret } -func (job *Job) hasDependentSchema(other *Job) (bool, error) { - if other.Type == ActionDropSchema || other.Type == ActionCreateSchema { - if other.SchemaID == job.SchemaID { - return true, nil - } - if job.Type == ActionRenameTable { - args, err := GetRenameTableArgs(job) - if err != nil { - return false, errors.Trace(err) - } - - if other.SchemaID == args.OldSchemaID { - return true, nil - } - } - if job.Type == ActionExchangeTablePartition { - args, err := GetExchangeTablePartitionArgs(job) - if err != nil { - return false, errors.Trace(err) - } - if other.SchemaID == args.PTSchemaID { - return true, nil - } - } - } - return false, nil -} - -func (job *Job) hasDependentTableForExchangePartition(other *Job) (bool, error) { - if job.Type == ActionExchangeTablePartition { - // TODO this code seems buggy, we haven't encode Args into RawArgs yet, so cannot decode. - // but it's very old code for previous job queue, will be removed later anyway. - args, err := GetExchangeTablePartitionArgs(job) - if err != nil { - return false, errors.Trace(err) - } - if args.PTTableID == other.TableID || args.PartitionID == other.TableID { - return true, nil - } - - if other.Type == ActionExchangeTablePartition { - otherArgs, err := GetExchangeTablePartitionArgs(other) - if err != nil { - return false, errors.Trace(err) - } - if job.TableID == other.TableID || job.TableID == otherArgs.PTTableID || job.TableID == otherArgs.PartitionID { - return true, nil - } - if args.PTTableID == other.TableID || args.PTTableID == otherArgs.PTTableID || args.PTTableID == otherArgs.PartitionID { - return true, nil - } - if args.PartitionID == other.TableID || args.PartitionID == otherArgs.PTTableID || args.PartitionID == otherArgs.PartitionID { - return true, nil - } - } - } - return false, nil -} - -// IsDependentOn returns whether the job depends on "other". -// How to check the job depends on "other"? -// 1. The two jobs handle the same database when one of the two jobs is an ActionDropSchema or ActionCreateSchema type. -// 2. Or the two jobs handle the same table. -// 3. Or other job is flashback cluster. -func (job *Job) IsDependentOn(other *Job) (bool, error) { - if other.Type == ActionFlashbackCluster { - return true, nil - } - - isDependent, err := job.hasDependentSchema(other) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - isDependent, err = other.hasDependentSchema(job) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - - // TODO: If a job is ActionRenameTable, we need to check table name. - if other.TableID == job.TableID { - return true, nil - } - isDependent, err = job.hasDependentTableForExchangePartition(other) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - isDependent, err = other.hasDependentTableForExchangePartition(job) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - return false, nil -} - // IsFinished returns whether job is finished or not. // If the job state is Done or Cancelled, it is finished. func (job *Job) IsFinished() bool { diff --git a/pkg/meta/model/job_test.go b/pkg/meta/model/job_test.go index a38d44b9f7bbc..a0eed61aba9d3 100644 --- a/pkg/meta/model/job_test.go +++ b/pkg/meta/model/job_test.go @@ -66,172 +66,6 @@ func TestJobCodec(t *testing.T) { job.FillArgs(&RenameTableArgs{OldSchemaID: 2, NewTableName: model.NewCIStr("table1")}) job.BinlogInfo.AddDBInfo(123, &DBInfo{ID: 1, Name: model.NewCIStr("test_history_db")}) job.BinlogInfo.AddTableInfo(123, &TableInfo{ID: 1, Name: model.NewCIStr("test_history_tbl")}) - job1 := &Job{ - Version: GetJobVerInUse(), - ID: 2, - TableID: 2, - SchemaID: 1, - Type: ActionRenameTable, - BinlogInfo: &HistoryInfo{}, - } - job1.FillArgs(&RenameTableArgs{OldSchemaID: 3, NewTableName: model.NewCIStr("new_table_name")}) - job2 := &Job{ - Version: JobVersion1, - ID: 3, - TableID: 3, - SchemaID: 3, - Type: ActionCreateSchema, - BinlogInfo: &HistoryInfo{}, - } - job3 := &Job{ - Version: JobVersion1, - ID: 4, - TableID: 4, - SchemaID: 4, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job3.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 6, PTSchemaID: 3, PTTableID: 5, PartitionName: "pt", WithValidation: true}) - job4 := &Job{ - Version: JobVersion1, - ID: 5, - TableID: 5, - SchemaID: 3, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job4.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 6, PTSchemaID: 4, PTTableID: 2, PartitionName: "pt", WithValidation: true}) - job5 := &Job{ - Version: JobVersion1, - ID: 6, - TableID: 6, - SchemaID: 6, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job5.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 2, PTSchemaID: 6, PTTableID: 5, PartitionName: "pt", WithValidation: true}) - job6 := &Job{ - Version: JobVersion1, - ID: 7, - TableID: 7, - SchemaID: 7, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job6.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 6, PTSchemaID: 4, PTTableID: 2, PartitionName: "pt", WithValidation: true}) - job7 := &Job{ - Version: JobVersion1, - ID: 8, - TableID: 8, - SchemaID: 8, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job7.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 8, PTSchemaID: 4, PTTableID: 6, PartitionName: "pt", WithValidation: true}) - job8 := &Job{ - Version: JobVersion1, - ID: 9, - TableID: 9, - SchemaID: 9, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job8.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 8, PTSchemaID: 9, PTTableID: 9, PartitionName: "pt", WithValidation: true}) - job9 := &Job{ - Version: JobVersion1, - ID: 10, - TableID: 10, - SchemaID: 10, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job9.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 10, PTSchemaID: 10, PTTableID: 8, PartitionName: "pt", WithValidation: true}) - job10 := &Job{ - Version: JobVersion1, - ID: 11, - TableID: 11, - SchemaID: 11, - Type: ActionDropSchema, - BinlogInfo: &HistoryInfo{}, - } - job11 := &Job{ - Version: JobVersion1, - ID: 12, - TableID: 12, - SchemaID: 11, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job11.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 10, PTSchemaID: 10, PTTableID: 8, PartitionName: "pt", WithValidation: true}) - job12 := &Job{ - Version: JobVersion1, - ID: 13, - TableID: 13, - SchemaID: 11, - Type: ActionDropTable, - BinlogInfo: &HistoryInfo{}, - } - job13 := &Job{ - Version: JobVersion1, - ID: 14, - TableID: 12, - SchemaID: 14, - Type: ActionDropTable, - BinlogInfo: &HistoryInfo{}, - } - job14 := &Job{ - Version: JobVersion1, - ID: 15, - TableID: 15, - SchemaID: 15, - Type: ActionExchangeTablePartition, - BinlogInfo: &HistoryInfo{}, - } - job14.FillArgs(&ExchangeTablePartitionArgs{PartitionID: 16, PTSchemaID: 17, PTTableID: 12, PartitionName: "pt", WithValidation: true}) - job15 := &Job{ - Version: JobVersion1, - ID: 16, - Type: ActionFlashbackCluster, - BinlogInfo: &HistoryInfo{}, - } - - for _, j := range []*Job{job1, job2, job3, job4, job5, job6, job7, job8, job9, job10, job11, job12, job13, job14, job15} { - _, err := j.Encode(true) - require.NoError(t, err) - } - - cases := []struct { - name string - left, right *Job - dependent bool - }{ - {"same table id", job, job1, true}, - {"related to same schema", job2, job1, true}, - {"test ActionCreateSchema and ActionExchangeTablePartition is dependent.", job3, job2, true}, - {"test random and ActionExchangeTablePartition is dependent because TableID is same.", job4, job, true}, - {"test ActionExchangeTablePartition and ActionExchangeTablePartition is dependent.", job5, job4, true}, - // this case is invalid, actually. - {"PT partition ID same as other job's NT table ID", job6, job5, true}, - // invalid too - {"PT table ID same as other job's partition ID", job7, job6, true}, - // invalid too - {"2 PT table has same partition ID", job8, job7, true}, - // invalid too - {"PT table ID same as other job's partition ID", job9, job8, true}, - {"test ActionDropSchema and ActionExchangeTablePartition is dependent.", job11, job10, true}, - {"test ActionDropTable and ActionExchangeTablePartition is dependent.", job11, job12, false}, - {"NT table ID same as other job's table ID", job11, job13, true}, - {"test ActionDropTable and ActionExchangeTablePartition is dependent.", job13, job14, true}, - {"test ActionFlashbackCluster with other ddl jobs are dependent.", job, job15, true}, - } - - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - isDependent, err := c.left.IsDependentOn(c.right) - require.NoError(t, err) - require.Equal(t, c.dependent, isDependent) - }) - } require.Equal(t, false, job.IsCancelled()) b, err := job.Encode(false) diff --git a/pkg/meta/reader.go b/pkg/meta/reader.go index f8d52761d03e1..c377916b3fe12 100644 --- a/pkg/meta/reader.go +++ b/pkg/meta/reader.go @@ -32,7 +32,6 @@ type Reader interface { GetAllNameToIDAndTheMustLoadedTableInfo(dbID int64) (map[string]int64, []*model.TableInfo, error) GetMetadataLock() (enable bool, isNull bool, err error) - GetAllDDLJobsInQueue(jobListKeys ...JobListKeyType) ([]*model.Job, error) GetHistoryDDLJob(id int64) (*model.Job, error) GetHistoryDDLCount() (uint64, error) GetLastHistoryDDLJobsIterator() (LastJobIterator, error) diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 0d5c5d295abcc..7dd4ed0381a1b 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -1534,7 +1534,6 @@ func upgrade(s sessiontypes.Session) { upgradeToVer99After(s) } - variable.DDLForce2Queue.Store(false) updateBootstrapVer(s) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) _, err = s.ExecuteInternal(ctx, "COMMIT") diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 6d3faf5902fb8..4a5d54ffca1fa 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -1621,7 +1621,6 @@ var ( InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) - DDLForce2Queue = atomic.NewBool(false) EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) EnableMDL = atomic.NewBool(false) AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize)