From e03352fdaaf4a1e24d4ac88eb4c11a941cb94d5f Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 7 Jun 2022 12:00:00 +0800 Subject: [PATCH 01/21] 1. init ddl tables create tidb_ddl_job, tidb_ddl_reorg, tidb_ddl_history tables with raw meta write, these 3 tables is use to replace the ddl job queue and reorg and history hash table --- ddl/constant.go | 24 +++++++++ executor/infoschema_cluster_table_test.go | 2 +- meta/meta.go | 43 ++++++++++++++++ meta/meta_test.go | 62 +++++++++++++++++++++++ session/bootstrap_test.go | 4 +- session/session.go | 52 ++++++++++++++++++- session/session_test.go | 54 ++++++++++++++++++++ 7 files changed, 237 insertions(+), 4 deletions(-) create mode 100644 ddl/constant.go create mode 100644 session/session_test.go diff --git a/ddl/constant.go b/ddl/constant.go new file mode 100644 index 0000000000000..600cc68794cb2 --- /dev/null +++ b/ddl/constant.go @@ -0,0 +1,24 @@ +// Copyright 2022 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 ddl + +const ( + // JobTable stores the information of DDL jobs. + JobTable = "tidb_ddl_job" + // ReorgTable stores the information of DDL reorganization. + ReorgTable = "tidb_ddl_reorg" + // HistoryTable stores the history DDL jobs. + HistoryTable = "tidb_ddl_history" +) diff --git a/executor/infoschema_cluster_table_test.go b/executor/infoschema_cluster_table_test.go index c790919a17004..f89120e688654 100644 --- a/executor/infoschema_cluster_table_test.go +++ b/executor/infoschema_cluster_table_test.go @@ -335,7 +335,7 @@ func TestTableStorageStats(t *testing.T) { "test 2", )) rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows() - result := 32 + result := 35 require.Len(t, rows, result) // More tests about the privileges. diff --git a/meta/meta.go b/meta/meta.go index 5469e3ab009e6..213037ebadef4 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -75,6 +75,7 @@ var ( mPolicyPrefix = "Policy" mPolicyGlobalID = []byte("PolicyGlobalID") mPolicyMagicByte = CurrentMagicByteVer + mDDLTableVersion = []byte("DDLTableVersion") ) const ( @@ -489,6 +490,48 @@ func (m *Meta) CreateTableOrView(dbID int64, tableInfo *model.TableInfo) error { return m.txn.HSet(dbKey, tableKey, data) } +// SetDDLTables write a key into storage. +func (m *Meta) SetDDLTables() error { + err := m.txn.Set(mDDLTableVersion, []byte("1")) + return errors.Trace(err) +} + +// CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID. +func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { + dbs, err := m.ListDatabases() + if err != nil { + return 0, err + } + for _, db := range dbs { + if db.Name.L == mysql.SystemDB { + return db.ID, nil + } + } + + id, err := m.GenGlobalID() + if err != nil { + return 0, errors.Trace(err) + } + db := model.DBInfo{ + ID: id, + Name: model.NewCIStr(mysql.SystemDB), + Charset: mysql.UTF8MB4Charset, + Collate: mysql.UTF8MB4DefaultCollation, + State: model.StatePublic, + } + err = m.CreateDatabase(&db) + return db.ID, err +} + +// CheckDDLTableExists check if the tables related to concurrent DDL exists. +func (m *Meta) CheckDDLTableExists() (bool, error) { + v, err := m.txn.Get(mDDLTableVersion) + if err != nil { + return false, errors.Trace(err) + } + return len(v) != 0, nil +} + // CreateTableAndSetAutoID creates a table with tableInfo in database, // and rebases the table autoID. func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, autoIncID, autoRandID int64) error { diff --git a/meta/meta_test.go b/meta/meta_test.go index 7174eae38c66c..682b6a68f671e 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -766,3 +766,65 @@ func TestSequenceKey(b *testing.T) { require.NoError(b, err) require.Equal(b, tableID, id) } + +func TestCreateMySQLDatabase(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + }() + + txn, err := store.Begin() + require.NoError(t, err) + + m := meta.NewMeta(txn) + + dbID, err := m.CreateMySQLDatabaseIfNotExists() + require.NoError(t, err) + require.Greater(t, dbID, int64(0)) + + anotherDBID, err := m.CreateMySQLDatabaseIfNotExists() + require.NoError(t, err) + require.Equal(t, dbID, anotherDBID) + + err = txn.Rollback() + require.NoError(t, err) +} + +func TestDDLTable(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + }() + + txn, err := store.Begin() + require.NoError(t, err) + + m := meta.NewMeta(txn) + + exists, err := m.CheckDDLTableExists() + require.NoError(t, err) + require.False(t, exists) + + err = m.SetDDLTables() + require.NoError(t, err) + + exists, err = m.CheckDDLTableExists() + require.NoError(t, err) + require.True(t, exists) + + err = m.SetConcurrentDDL(true) + require.NoError(t, err) + b, err := m.IsConcurrentDDL() + require.NoError(t, err) + require.True(t, b) + err = m.SetConcurrentDDL(false) + require.NoError(t, err) + b, err = m.IsConcurrentDDL() + require.NoError(t, err) + require.False(t, b) + + err = txn.Rollback() + require.NoError(t, err) +} diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 6b56501d2b2eb..a24a11df97eb0 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -146,14 +146,14 @@ func TestBootstrapWithError(t *testing.T) { se.txn.init() se.mu.values = make(map[fmt.Stringer]interface{}) se.SetValue(sessionctx.Initing, true) - + err := InitDDLJobTables(store) + require.NoError(t, err) dom, err := domap.Get(store) require.NoError(t, err) domain.BindDomain(se, dom) b, err := checkBootstrapped(se) require.False(t, b) require.NoError(t, err) - doDDLWorks(se) } diff --git a/session/session.go b/session/session.go index 942ac108a192c..9e14c2a52e706 100644 --- a/session/session.go +++ b/session/session.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" @@ -2703,7 +2704,52 @@ func loadCollationParameter(ctx context.Context, se *session) (bool, error) { return false, nil } -var errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) +var ( + errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) + // DDLJobTables is a list of tables definitions used in concurrent DDL. + DDLJobTables = []string{ + "create table tidb_ddl_job(job_id bigint not null, reorg int, schema_id bigint, table_id bigint, job_meta longblob, is_drop_schema int, primary key(job_id))", + "create table tidb_ddl_reorg(job_id bigint not null, ele_id bigint, ele_type blob, start_key blob, end_key blob, physical_id bigint, reorg_meta longblob, unique key(job_id, ele_id, ele_type(20)))", + "create table tidb_ddl_history(job_id bigint not null, job_meta longblob, db_name char(64), table_name char(64), schema_id bigint, table_id bigint, create_time datetime, primary key(job_id))", + } +) + +// InitDDLJobTables is to create tidb_ddl_job, tidb_ddl_reorg and tidb_ddl_history. +func InitDDLJobTables(store kv.Storage) error { + return kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + exists, err := t.CheckDDLTableExists() + if err != nil || exists { + return errors.Trace(err) + } + dbID, err := t.CreateMySQLDatabaseIfNotExists() + if err != nil { + return err + } + p := parser.New() + for _, s := range DDLJobTables { + stmt, err := p.ParseOneStmt(s, "", "") + if err != nil { + return errors.Trace(err) + } + tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + if err != nil { + return errors.Trace(err) + } + tblInfo.State = model.StatePublic + tblInfo.ID, err = t.GenGlobalID() + tblInfo.UpdateTS = t.StartTS + if err != nil { + return errors.Trace(err) + } + err = t.CreateTableOrView(dbID, tblInfo) + if err != nil { + return errors.Trace(err) + } + } + return t.SetDDLTables() + }) +} // BootstrapSession runs the first time when the TiDB server start. func BootstrapSession(store kv.Storage) (*domain.Domain, error) { @@ -2718,6 +2764,10 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } } + err := InitDDLJobTables(store) + if err != nil { + return nil, err + } ver := getStoreBootstrapVersion(store) if ver == notBootstrapped { runInBootstrapSession(store, bootstrap) diff --git a/session/session_test.go b/session/session_test.go new file mode 100644 index 0000000000000..f065cc1eb142d --- /dev/null +++ b/session/session_test.go @@ -0,0 +1,54 @@ +// Copyright 2022 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_test + +import ( + "reflect" + "testing" + + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" + "github.com/stretchr/testify/require" +) + +func TestInitMetaTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + for _, sql := range session.DDLJobTables { + tk.MustExec(sql) + } + + tbls := map[string]struct{}{ + "tidb_ddl_job": {}, + "tidb_ddl_reorg": {}, + "tidb_ddl_history": {}, + } + + for tbl := range tbls { + metaInMySQL := external.GetTableByName(t, tk, "mysql", tbl).Meta() + metaInTest := external.GetTableByName(t, tk, "test", tbl).Meta() + + require.Greater(t, metaInMySQL.ID, 0) + require.Greater(t, metaInMySQL.UpdateTS, 0) + + metaInTest.ID = metaInMySQL.ID + metaInMySQL.UpdateTS = metaInTest.UpdateTS + require.True(t, reflect.DeepEqual(metaInMySQL, metaInTest)) + } +} From 4fb3c22a9525e011d6fc96645e02a56890d0e8b7 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 7 Jun 2022 11:43:29 +0800 Subject: [PATCH 02/21] 2. setup concurrent ddl env and add ddl worker pool adds the ddl worker pool definition, the ddl job manager will find a job and ship it to a worker in the worker pool. Also, this commit provides a sessionctx wrapper, only use in ddl relate. it just wraps begin, commit and execute --- ddl/ddl.go | 130 +++++++++++++++++++++++++++++++++++------- ddl/ddl_worker.go | 10 +++- ddl/ddl_workerpool.go | 89 +++++++++++++++++++++++++++++ ddl/job_table.go | 48 ++++++++++++++++ 4 files changed, 255 insertions(+), 22 deletions(-) create mode 100644 ddl/ddl_workerpool.go create mode 100644 ddl/job_table.go diff --git a/ddl/ddl.go b/ddl/ddl.go index e8063fdaf1705..8200bd2458fb2 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -31,6 +31,7 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" @@ -50,9 +51,11 @@ import ( "github.com/pingcap/tidb/table" pumpcli "github.com/pingcap/tidb/tidb-binlog/pump_client" tidbutil "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "github.com/tikv/client-go/v2/tikvrpc" clientv3 "go.etcd.io/etcd/client/v3" atomicutil "go.uber.org/atomic" @@ -206,6 +209,9 @@ type ddl struct { sessPool *sessionPool delRangeMgr delRangeManager enableTiFlashPoll *atomicutil.Bool + // used in the concurrency ddl. + reorgWorkerPool *workerPool + generalDDLWorkerPool *workerPool } // ddlCtx is the context when we use worker to handle DDL jobs. @@ -478,13 +484,47 @@ func (d *ddl) newDeleteRangeManager(mock bool) delRangeManager { return delRangeMgr } +func (d *ddl) prepareWorkers4ConcurrencyDDL() { + workerFactor := func(tp workerType) func() (pools.Resource, error) { + return func() (pools.Resource, error) { + wk := newWorker(d.ctx, tp, d.sessPool, d.delRangeMgr, d.ddlCtx, true) + sessForJob, err := d.sessPool.get() + if err != nil { + return nil, err + } + sessForJob.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + wk.sess = newSession(sessForJob) + metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_%s", metrics.CreateDDL, wk.String())).Inc() + return wk, nil + } + } + d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactor(addIdxWorker), batchAddingJobs, batchAddingJobs, 0), reorg) + d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactor(generalWorker), 1, 1, 0), general) +} + +func (d *ddl) prepareWorkers4legacyDDL() { + d.workers = make(map[workerType]*worker, 2) + d.workers[generalWorker] = newWorker(d.ctx, generalWorker, d.sessPool, d.delRangeMgr, d.ddlCtx, false) + d.workers[addIdxWorker] = newWorker(d.ctx, addIdxWorker, d.sessPool, d.delRangeMgr, d.ddlCtx, false) + for _, worker := range d.workers { + worker.wg.Add(1) + w := worker + go w.start(d.ddlCtx) + + metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_%s", metrics.CreateDDL, worker.String())).Inc() + + // When the start function is called, we will send a fake job to let worker + // checks owner firstly and try to find whether a job exists and run. + asyncNotify(worker.ddlJobCh) + } +} + // Start implements DDL.Start interface. func (d *ddl) Start(ctxPool *pools.ResourcePool) error { logutil.BgLogger().Info("[ddl] start DDL", zap.String("ID", d.uuid), zap.Bool("runWorker", RunWorker)) d.wg.Run(d.limitDDLJobs) d.sessPool = newSessionPool(ctxPool, d.store) - // If RunWorker is true, we need campaign owner and do DDL job. // Otherwise, we needn't do that. if RunWorker { @@ -501,21 +541,10 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { return errors.Trace(err) } - d.workers = make(map[workerType]*worker, 2) d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) - d.workers[generalWorker] = newWorker(d.ctx, generalWorker, d.sessPool, d.delRangeMgr, d.ddlCtx) - d.workers[addIdxWorker] = newWorker(d.ctx, addIdxWorker, d.sessPool, d.delRangeMgr, d.ddlCtx) - for _, worker := range d.workers { - worker.wg.Add(1) - w := worker - go w.start(d.ddlCtx) - - metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s_%s", metrics.CreateDDL, worker.String())).Inc() - - // When the start function is called, we will send a fake job to let worker - // checks owner firstly and try to find whether a job exists and run. - asyncNotify(worker.ddlJobCh) - } + + d.prepareWorkers4ConcurrencyDDL() + d.prepareWorkers4legacyDDL() go d.schemaSyncer.StartCleanWork() if config.TableLockEnabled() { @@ -535,7 +564,7 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { return nil } -// GetNextDDLSeqNum return the next ddl seq num. +// GetNextDDLSeqNum return the next DDL seq num. func (d *ddl) GetNextDDLSeqNum() (uint64, error) { var count uint64 ctx := kv.WithInternalSourceType(d.ctx, kv.InternalTxnDDL) @@ -558,6 +587,12 @@ func (d *ddl) close() { d.wg.Wait() d.ownerManager.Cancel() d.schemaSyncer.Close() + if d.reorgWorkerPool != nil { + d.reorgWorkerPool.close() + } + if d.generalDDLWorkerPool != nil { + d.generalDDLWorkerPool.close() + } for _, worker := range d.workers { worker.Close() @@ -567,10 +602,7 @@ func (d *ddl) close() { if d.delRangeMgr != nil { d.delRangeMgr.clear() } - if d.sessPool != nil { - d.sessPool.close() - } - + d.sessPool.close() variable.UnregisterStatistics(d) logutil.BgLogger().Info("[ddl] DDL closed", zap.String("ID", d.uuid), zap.Duration("take time", time.Since(startTime))) @@ -1209,6 +1241,64 @@ func IterAllDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error return IterHistoryDDLJobs(txn, finishFn) } +// session wraps sessionctx.Context for transaction usage. +type session struct { + sessionctx.Context +} + +func newSession(s sessionctx.Context) *session { + return &session{s} +} + +func (s *session) begin() error { + err := sessiontxn.NewTxn(context.Background(), s) + if err != nil { + return err + } + s.GetSessionVars().SetInTxn(true) + return nil +} + +func (s *session) commit() error { + s.StmtCommit() + return s.CommitTxn(context.Background()) +} + +func (s *session) txn() (kv.Transaction, error) { + return s.Txn(true) +} + +func (s *session) rollback() { + s.StmtRollback() + s.RollbackTxn(context.Background()) +} + +func (s *session) reset() { + s.StmtRollback() +} + +func (s *session) execute(ctx context.Context, query string, label string) ([]chunk.Row, error) { + var err error + rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(ctx, query) + if err != nil { + return nil, errors.Trace(err) + } + + if rs == nil { + return nil, nil + } + var rows []chunk.Row + defer terror.Call(rs.Close) + if rows, err = sqlexec.DrainRecordSet(ctx, rs, 8); err != nil { + return nil, errors.Trace(err) + } + return rows, nil +} + +func (s *session) session() sessionctx.Context { + return s.Context +} + // GetAllHistoryDDLJobs get all the done DDL jobs. func GetAllHistoryDDLJobs(m *meta.Meta) ([]*model.Job, error) { iterator, err := m.GetLastHistoryDDLJobsIterator() diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 1c0e465414717..aa9e571bf52a1 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -91,10 +91,13 @@ type worker struct { wg sync.WaitGroup sessPool *sessionPool // sessPool is used to new sessions to execute SQL in ddl package. + sess *session // sess is used and only used in running DDL job. delRangeManager delRangeManager logCtx context.Context lockSeqNum bool + concurrentDDL bool + *ddlCtx } @@ -119,7 +122,7 @@ func NewJobContext() *JobContext { } } -func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRangeMgr delRangeManager, dCtx *ddlCtx) *worker { +func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRangeMgr delRangeManager, dCtx *ddlCtx, concurrentDDL bool) *worker { worker := &worker{ id: ddlWorkerID.Add(1), tp: tp, @@ -128,8 +131,8 @@ func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRan ddlCtx: dCtx, sessPool: sessPool, delRangeManager: delRangeMgr, + concurrentDDL: concurrentDDL, } - worker.addingDDLJobKey = addingDDLJobPrefix + worker.typeStr() worker.logCtx = logutil.WithKeyValue(context.Background(), "worker", worker.String()) return worker @@ -154,6 +157,9 @@ func (w *worker) String() string { func (w *worker) Close() { startTime := time.Now() + if w.sess != nil { + w.sessPool.put(w.sess.session()) + } w.wg.Wait() logutil.Logger(w.logCtx).Info("[ddl] DDL worker closed", zap.Duration("take time", time.Since(startTime))) } diff --git a/ddl/ddl_workerpool.go b/ddl/ddl_workerpool.go new file mode 100644 index 0000000000000..2d931bf7823e7 --- /dev/null +++ b/ddl/ddl_workerpool.go @@ -0,0 +1,89 @@ +// Copyright 2022 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 ddl + +import ( + "github.com/ngaut/pools" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/atomic" +) + +// workerPool is used to new worker. +type workerPool struct { + t jobType + exit atomic.Bool + resPool *pools.ResourcePool +} + +func newDDLWorkerPool(resPool *pools.ResourcePool, tp jobType) *workerPool { + return &workerPool{ + t: tp, + exit: *atomic.NewBool(false), + resPool: resPool, + } +} + +// get gets workerPool from context resource pool. +// Please remember to call put after you finished using workerPool. +func (wp *workerPool) get() (*worker, error) { + if wp.resPool == nil { + return nil, nil + } + + if wp.exit.Load() { + return nil, errors.Errorf("workerPool is closed") + } + + // no need to protect wp.resPool + resource, err := wp.resPool.TryGet() + if err != nil { + return nil, errors.Trace(err) + } + + if resource == nil { + return nil, nil + } + + worker := resource.(*worker) + return worker, nil +} + +// put returns workerPool to context resource pool. +func (wp *workerPool) put(wk *worker) { + if wp.resPool == nil { + return + } + + // no need to protect wp.resPool, even the wp.resPool is closed, the ctx still need to + // put into resPool, because when resPool is closing, it will wait all the ctx returns, then resPool finish closing. + wp.resPool.Put(wk) +} + +// close clean up the workerPool. +func (wp *workerPool) close() { + // prevent closing resPool twice. + if wp.exit.Load() || wp.resPool == nil { + return + } + wp.exit.Store(true) + logutil.BgLogger().Info("[ddl] closing workerPool") + wp.resPool.Close() +} + +// tp return the type of worker pool. +func (wp *workerPool) tp() jobType { + return wp.t +} diff --git a/ddl/job_table.go b/ddl/job_table.go new file mode 100644 index 0000000000000..475475133f748 --- /dev/null +++ b/ddl/job_table.go @@ -0,0 +1,48 @@ +// Copyright 2022 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 ddl + +type jobType int + +func (t jobType) String() string { + switch t { + case general: + return "general" + case reorg: + return "reorg" + } + return "unknown job type: " + strconv.Itoa(int(t)) +} + +const ( + general jobType = iota + reorg +) + +func runInTxn(se *session, f func(*session) error) (err error) { + err = se.begin() + if err != nil { + return err + } + defer func() { + if err != nil { + se.rollback() + return + } + err = se.commit() + }() + err = f(se) + return +} From 12f84861aec3d065b12e1fa4abae21b97842f7a3 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Sun, 3 Jul 2022 22:10:34 +0800 Subject: [PATCH 03/21] 3. add ddl manager to handle ddl job --- ddl/ddl.go | 127 +++++++++++++++++++++++--- ddl/ddl_worker.go | 131 ++++++++++++++++++++++++++- ddl/job_table.go | 189 +++++++++++++++++++++++++++++++++++++++ domain/domain.go | 26 +++++- infoschema/cache.go | 21 ++++- infoschema/cache_test.go | 2 +- 6 files changed, 477 insertions(+), 19 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 8200bd2458fb2..e8ddb4136af21 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -76,6 +76,9 @@ const ( batchAddingJobs = 10 + reorgWorkerCnt = 10 + generalWorkerCnt = 1 + // PartitionCountLimit is limit of the number of partitions in a table. // Reference linking https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations.html. PartitionCountLimit = 8192 @@ -93,6 +96,8 @@ const ( // supported by VIEWs at the moment. For other object types, this is // equivalent to OnExistError. OnExistReplace + + jobRecordCapacity = 16 ) var ( @@ -212,6 +217,43 @@ type ddl struct { // used in the concurrency ddl. reorgWorkerPool *workerPool generalDDLWorkerPool *workerPool + // get notification if any DDL coming. + ddlJobCh chan struct{} +} + +// waitSchemaSyncedController is to control whether to waitSchemaSynced or not. +type waitSchemaSyncedController struct { + mu sync.RWMutex + job map[int64]struct{} + + // true if this node is elected to the DDL owner, we should wait 2 * lease before it runs the first DDL job. + once *atomicutil.Bool +} + +func newWaitSchemaSyncedController() *waitSchemaSyncedController { + return &waitSchemaSyncedController{ + job: make(map[int64]struct{}, jobRecordCapacity), + once: atomicutil.NewBool(true), + } +} + +func (w *waitSchemaSyncedController) registerSync(job *model.Job) { + w.mu.Lock() + defer w.mu.Unlock() + w.job[job.ID] = struct{}{} +} + +func (w *waitSchemaSyncedController) isSynced(job *model.Job) bool { + w.mu.RLock() + defer w.mu.RUnlock() + _, ok := w.job[job.ID] + return !ok +} + +func (w *waitSchemaSyncedController) synced(job *model.Job) { + w.mu.Lock() + defer w.mu.Unlock() + delete(w.job, job.ID) } // ddlCtx is the context when we use worker to handle DDL jobs. @@ -231,6 +273,15 @@ type ddlCtx struct { tableLockCkr util.DeadTableLockChecker etcdCli *clientv3.Client + *waitSchemaSyncedController + *schemaVersionManager + // recording the running jobs. + runningJobs struct { + sync.RWMutex + ids map[int64]struct{} + } + // It holds the running DDL jobs ID. + runningJobIDs []string // reorgCtx is used for reorganization. reorgCtx struct { sync.RWMutex @@ -257,6 +308,49 @@ type ddlCtx struct { } } +// schemaVersionManager is used to manage the schema version. To prevent the conflicts on this key between different DDL job, +// we use another transaction to update the schema version, so that we need to lock the schema version and unlock it until the job is committed. +type schemaVersionManager struct { + schemaVersionMu sync.Mutex + // lockOwner stores the job ID that is holding the lock. + lockOwner atomicutil.Int64 +} + +func newSchemaVersionManager() *schemaVersionManager { + return &schemaVersionManager{} +} + +func (sv *schemaVersionManager) setSchemaVersion(job *model.Job, store kv.Storage) (schemaVersion int64, err error) { + sv.lockSchemaVersion(job.ID) + err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { + var err error + m := meta.NewMeta(txn) + schemaVersion, err = m.GenSchemaVersion() + return err + }) + return schemaVersion, err +} + +// lockSchemaVersion gets the lock to prevent the schema version from being updated. +func (sv *schemaVersionManager) lockSchemaVersion(jobID int64) { + ownerID := sv.lockOwner.Load() + // There may exist one job update schema version many times in multiple-schema-change, so we do not lock here again + // if they are the same job. jobID == 0 is a special one, it means we must get the lock. + if ownerID != jobID || jobID == 0 { + sv.schemaVersionMu.Lock() + sv.lockOwner.Store(jobID) + } +} + +// unlockSchemaVersion releases the lock. +func (sv *schemaVersionManager) unlockSchemaVersion(jobID int64) { + ownerID := sv.lockOwner.Load() + if ownerID == jobID { + sv.lockOwner.Store(0) + sv.schemaVersionMu.Unlock() + } +} + func (dc *ddlCtx) isOwner() bool { isOwner := dc.ownerManager.IsOwner() logutil.BgLogger().Debug("[ddl] check whether is the DDL owner", zap.Bool("isOwner", isOwner), zap.String("selfID", dc.uuid)) @@ -435,16 +529,19 @@ func newDDL(ctx context.Context, options ...Option) *ddl { } ddlCtx := &ddlCtx{ - uuid: id, - store: opt.Store, - lease: opt.Lease, - ddlJobDoneCh: make(chan struct{}, 1), - ownerManager: manager, - schemaSyncer: syncer, - binlogCli: binloginfo.GetPumpsClient(), - infoCache: opt.InfoCache, - tableLockCkr: deadLockCkr, - etcdCli: opt.EtcdCli, + uuid: id, + store: opt.Store, + lease: opt.Lease, + ddlJobDoneCh: make(chan struct{}, 1), + ownerManager: manager, + schemaSyncer: syncer, + binlogCli: binloginfo.GetPumpsClient(), + infoCache: opt.InfoCache, + tableLockCkr: deadLockCkr, + etcdCli: opt.EtcdCli, + schemaVersionManager: newSchemaVersionManager(), + waitSchemaSyncedController: newWaitSchemaSyncedController(), + runningJobIDs: make([]string, 0, jobRecordCapacity), } ddlCtx.reorgCtx.reorgCtxMap = make(map[int64]*reorgCtx) ddlCtx.jobCtx.jobCtxMap = make(map[int64]*JobContext) @@ -452,10 +549,13 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ddlCtx.mu.interceptor = &BaseInterceptor{} ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnDDL) ddlCtx.ctx, ddlCtx.cancel = context.WithCancel(ctx) + ddlCtx.runningJobs.ids = make(map[int64]struct{}) + d := &ddl{ ddlCtx: ddlCtx, limitJobCh: make(chan *limitJobTask, batchAddingJobs), enableTiFlashPoll: atomicutil.NewBool(true), + ddlJobCh: make(chan struct{}, 100), } return d @@ -485,7 +585,7 @@ func (d *ddl) newDeleteRangeManager(mock bool) delRangeManager { } func (d *ddl) prepareWorkers4ConcurrencyDDL() { - workerFactor := func(tp workerType) func() (pools.Resource, error) { + workerFactory := func(tp workerType) func() (pools.Resource, error) { return func() (pools.Resource, error) { wk := newWorker(d.ctx, tp, d.sessPool, d.delRangeMgr, d.ddlCtx, true) sessForJob, err := d.sessPool.get() @@ -498,8 +598,9 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { return wk, nil } } - d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactor(addIdxWorker), batchAddingJobs, batchAddingJobs, 0), reorg) - d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactor(generalWorker), 1, 1, 0), general) + d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(addIdxWorker), reorgWorkerCnt, reorgWorkerCnt, 0), reorg) + d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(generalWorker), generalWorkerCnt, generalWorkerCnt, 0), general) + d.wg.Run(d.startDispatchLoop) } func (d *ddl) prepareWorkers4legacyDDL() { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index aa9e571bf52a1..582b766944182 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -620,6 +620,130 @@ func (w *JobContext) setDDLLabelForDiagnosis(job *model.Job) { w.ddlJobCtx = kv.WithInternalSourceType(w.ddlJobCtx, w.ddlJobSourceType()) } +func (w *worker) HandleJobDone(d *ddlCtx, job *model.Job, t *meta.Meta) error { + err := w.finishDDLJob(t, job) + if err != nil { + w.sess.rollback() + return err + } + + err = w.sess.commit() + if err != nil { + return err + } + asyncNotify(d.ddlJobDoneCh) + return nil +} + +func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { + var ( + err error + schemaVer int64 + runJobErr error + waitTime = 2 * d.lease + ) + defer func() { + w.unlockSeqNum(err) + }() + + err = w.sess.begin() + if err != nil { + return err + } + txn, err := w.sess.txn() + if err != nil { + w.sess.rollback() + return err + } + // Only general DDLs are allowed to be executed when TiKV is disk full. + if w.tp == addIdxWorker && job.IsRunning() { + txn.SetDiskFullOpt(kvrpcpb.DiskFullOpt_NotAllowedOnFull) + } + w.setDDLLabelForTopSQL(job) + if tagger := w.getResourceGroupTaggerForTopSQL(job); tagger != nil { + txn.SetOption(kv.ResourceGroupTagger, tagger) + } + t := meta.NewMeta(txn) + if job.IsDone() || job.IsRollbackDone() { + if job.IsDone() { + job.State = model.JobStateSynced + } + err = w.HandleJobDone(d, job, t) + return err + } + + d.mu.RLock() + d.mu.hook.OnJobRunBefore(job) + d.mu.RUnlock() + + // If running job meets error, we will save this error in job Error + // and retry later if the job is not cancelled. + schemaVer, runJobErr = w.runDDLJob(d, t, job) + + if job.IsCancelled() { + defer d.unlockSchemaVersion(job.ID) + w.sess.reset() + err = w.HandleJobDone(d, job, t) + return err + } + + if runJobErr != nil && !job.IsRollingback() && !job.IsRollbackDone() { + // If the running job meets an error + // and the job state is rolling back, it means that we have already handled this error. + // Some DDL jobs (such as adding indexes) may need to update the table info and the schema version, + // then shouldn't discard the KV modification. + // And the job state is rollback done, it means the job was already finished, also shouldn't discard too. + // Otherwise, we should discard the KV modification when running job. + w.sess.reset() + // If error happens after updateSchemaVersion(), then the schemaVer is updated. + // Result in the retry duration is up to 2 * lease. + schemaVer = 0 + } + + err = w.updateDDLJob(t, job, runJobErr != nil) + if err = w.handleUpdateJobError(t, job, err); err != nil { + w.sess.rollback() + d.unlockSchemaVersion(job.ID) + return err + } + writeBinlog(d.binlogCli, txn, job) + err = w.sess.commit() + d.unlockSchemaVersion(job.ID) + if err != nil { + return err + } + w.registerSync(job) + + if runJobErr != nil { + // wait a while to retry again. If we don't wait here, DDL will retry this job immediately, + // which may act like a deadlock. + logutil.Logger(w.logCtx).Info("[ddl] run DDL job failed, sleeps a while then retries it.", + zap.Duration("waitTime", GetWaitTimeWhenErrorOccurred()), zap.Error(runJobErr)) + time.Sleep(GetWaitTimeWhenErrorOccurred()) + } + + // Here means the job enters another state (delete only, write only, public, etc...) or is cancelled. + // If the job is done or still running or rolling back, we will wait 2 * lease time to guarantee other servers to update + // the newest schema. + ctx, cancel := context.WithTimeout(w.ctx, waitTime) + w.waitSchemaChanged(ctx, d, waitTime, schemaVer, job) + cancel() + d.synced(job) + + if RunInGoTest { + // d.mu.hook is initialed from domain / test callback, which will force the owner host update schema diff synchronously. + d.mu.RLock() + d.mu.hook.OnSchemaStateChanged() + d.mu.RUnlock() + } + + d.mu.RLock() + d.mu.hook.OnJobUpdated(job) + d.mu.RUnlock() + + return nil +} + func (w *JobContext) getResourceGroupTaggerForTopSQL() tikvrpc.ResourceGroupTagger { if !topsqlstate.TopSQLEnabled() || w.cacheDigest == nil { return nil @@ -737,6 +861,9 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { zap.Duration("waitTime", GetWaitTimeWhenErrorOccurred()), zap.Error(runJobErr)) time.Sleep(GetWaitTimeWhenErrorOccurred()) } + if job != nil { + d.unlockSchemaVersion(job.ID) + } if err != nil { w.unlockSeqNum(err) @@ -1134,8 +1261,8 @@ func buildPlacementAffects(oldIDs []int64, newIDs []int64) []*model.AffectedOpti } // updateSchemaVersion increments the schema version by 1 and sets SchemaDiff. -func updateSchemaVersion(_ *ddlCtx, t *meta.Meta, job *model.Job) (int64, error) { - schemaVersion, err := t.GenSchemaVersion() +func updateSchemaVersion(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, error) { + schemaVersion, err := d.setSchemaVersion(job, d.store) if err != nil { return 0, errors.Trace(err) } diff --git a/ddl/job_table.go b/ddl/job_table.go index 475475133f748..75d2408cf82eb 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -14,6 +14,54 @@ package ddl +import ( + "context" + "fmt" + "strconv" + "strings" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/util/logutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" +) + +var ( + addingDDLJobGeneral = "/tidb/ddl/add_ddl_job_general" + addingDDLJobReorg = "/tidb/ddl/add_ddl_job_reorg" +) + +func (dc *ddlCtx) insertRunningDDLJobMap(id int64) { + dc.runningJobs.Lock() + defer dc.runningJobs.Unlock() + dc.runningJobs.ids[id] = struct{}{} +} + +func (dc *ddlCtx) deleteRunningDDLJobMap(id int64) { + dc.runningJobs.Lock() + defer dc.runningJobs.Unlock() + delete(dc.runningJobs.ids, id) +} + +func (dc *ddlCtx) excludeJobIDs() string { + dc.runningJobs.RLock() + defer dc.runningJobs.RUnlock() + if len(dc.runningJobs.ids) == 0 { + return "" + } + dc.runningJobIDs = dc.runningJobIDs[:0] + for id := range dc.runningJobs.ids { + dc.runningJobIDs = append(dc.runningJobIDs, strconv.Itoa(int(id))) + } + return fmt.Sprintf("and job_id not in (%s)", strings.Join(dc.runningJobIDs, ",")) +} + +const ( + getJobSQL = "select job_meta from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_id, table_id) and %s reorg %s" +) + type jobType int func (t jobType) String() string { @@ -31,6 +79,147 @@ const ( reorg ) +func (d *ddl) getJob(sess *session, tp jobType, filter func(*model.Job) (bool, error)) (*model.Job, error) { + not := "not" + label := "get_job_general" + if tp == reorg { + not = "" + label = "get_job_reorg" + } + sql := fmt.Sprintf(getJobSQL, not, d.excludeJobIDs()) + rows, err := sess.execute(context.Background(), sql, label) + if err != nil { + return nil, errors.Trace(err) + } + for _, row := range rows { + jobBinary := row.GetBytes(0) + runJob := model.Job{} + err := runJob.Decode(jobBinary) + if err != nil { + return nil, errors.Trace(err) + } + b, err := filter(&runJob) + if err != nil { + return nil, errors.Trace(err) + } + if b { + return &runJob, nil + } + } + return nil, nil +} + +func (d *ddl) getGeneralJob(sess *session) (*model.Job, error) { + return d.getJob(sess, general, func(job *model.Job) (bool, error) { + if job.Type == model.ActionDropSchema { + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where schema_id = %d and job_id < %d limit 1", job.SchemaID, job.ID) + return d.checkJobIsRunnable(sess, sql) + } + return true, nil + }) +} + +func (d *ddl) checkJobIsRunnable(sess *session, sql string) (bool, error) { + rows, err := sess.execute(context.Background(), sql, "check_runnable") + return len(rows) == 0, err +} + +func (d *ddl) getReorgJob(sess *session) (*model.Job, error) { + return d.getJob(sess, reorg, func(job *model.Job) (bool, error) { + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where schema_id = %d and is_drop_schema and job_id < %d limit 1", job.SchemaID, job.ID) + return d.checkJobIsRunnable(sess, sql) + }) +} + +func (d *ddl) startDispatchLoop() { + se, err := d.sessPool.get() + if err != nil { + logutil.BgLogger().Fatal("dispatch loop get session failed, it should not happen, please try restart TiDB", zap.Error(err)) + } + defer d.sessPool.put(se) + sess := newSession(se) + var notifyDDLJobByEtcdChGeneral clientv3.WatchChan + var notifyDDLJobByEtcdChReorg clientv3.WatchChan + if d.etcdCli != nil { + notifyDDLJobByEtcdChGeneral = d.etcdCli.Watch(d.ctx, addingDDLJobGeneral) + notifyDDLJobByEtcdChReorg = d.etcdCli.Watch(d.ctx, addingDDLJobReorg) + } + ticker := time.NewTicker(1 * time.Second) + defer ticker.Stop() + for { + if isChanClosed(d.ctx.Done()) { + return + } + if !d.isOwner() { + d.once.Store(true) + time.Sleep(time.Second) + continue + } + select { + case <-d.ddlJobCh: + case <-ticker.C: + case _, ok := <-notifyDDLJobByEtcdChGeneral: + if !ok { + logutil.BgLogger().Warn("[ddl] start general worker watch channel closed", zap.String("watch key", addingDDLJobGeneral)) + notifyDDLJobByEtcdChGeneral = d.etcdCli.Watch(d.ctx, addingDDLJobGeneral) + time.Sleep(time.Second) + continue + } + case _, ok := <-notifyDDLJobByEtcdChReorg: + if !ok { + logutil.BgLogger().Warn("[ddl] start reorg worker watch channel closed", zap.String("watch key", addingDDLJobReorg)) + notifyDDLJobByEtcdChReorg = d.etcdCli.Watch(d.ctx, addingDDLJobReorg) + time.Sleep(time.Second) + continue + } + case <-d.ctx.Done(): + return + } + d.getDDLJobAndRun(sess, d.generalDDLWorkerPool, d.getGeneralJob) + d.getDDLJobAndRun(sess, d.reorgWorkerPool, d.getReorgJob) + } +} + +func (d *ddl) getDDLJobAndRun(sess *session, pool *workerPool, getJob func(*session) (*model.Job, error)) { + wk, err := pool.get() + if err != nil || wk == nil { + logutil.BgLogger().Debug(fmt.Sprintf("[ddl] no %v worker available now", pool.tp()), zap.Error(err)) + return + } + + job, err := getJob(sess) + if job == nil || err != nil { + if err != nil { + logutil.BgLogger().Warn("[ddl] get job met error", zap.Error(err)) + } + pool.put(wk) + return + } + + d.delivery2worker(wk, pool, job) +} + +func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { + injectFailPointForGetJob(job) + d.insertRunningDDLJobMap(job.ID) + d.wg.Run(func() { + defer func() { + pool.put(wk) + d.deleteRunningDDLJobMap(job.ID) + asyncNotify(d.ddlJobCh) + }() + // we should wait 2 * d.lease time to guarantee all TiDB server have finished the schema change. + // see waitSchemaSynced for more details. + if !d.isSynced(job) || d.once.Load() { + wk.waitSchemaSynced(d.ddlCtx, job, 2*d.lease) + d.once.Store(false) + } + if err := wk.HandleDDLJobTable(d.ddlCtx, job); err != nil { + logutil.BgLogger().Info("[ddl] handle ddl job failed", zap.Error(err), zap.String("job", job.String())) + } + }) +} + func runInTxn(se *session, f func(*session) error) (err error) { err = se.begin() if err != nil { diff --git a/domain/domain.go b/domain/domain.go index 95d945b70b80a..10c35453cde09 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -131,6 +131,27 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i return nil, false, 0, nil, err } + // Consider the following scenario: + // t1 t2 t3 t4 + // | | | + // update schema version | set diff + // stale read ts + // At the first time, t2 reads the schema version v10, but the v10's diff is not set yet, so it loads v9 infoSchema. + // But at t4 moment, v10's diff has been set and been cached in the memory, so stale read on t2 will get v10 schema from cache, + // and inconsistency happen. + // To solve this problem, we always check the schema diff at first, if the diff is empty, we know at t2 moment we can only see the v9 schema, + // so make neededSchemaVersion = neededSchemaVersion - 1. + // For `Reload`, we can also do this: if the newest version's diff is not set yet, it is ok to load the previous version's infoSchema, and wait for the next reload. + diff, err := m.GetSchemaDiff(neededSchemaVersion) + if err != nil { + return nil, false, 0, nil, err + } + + if diff == nil && neededSchemaVersion > 0 { + // Although the diff of neededSchemaVersion is undetermined, the last version's diff is deterministic(this is guaranteed by schemaVersionManager). + neededSchemaVersion -= 1 + } + if is := do.infoCache.GetByVersion(neededSchemaVersion); is != nil { return is, true, 0, nil, nil } @@ -290,8 +311,9 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64 return nil, nil, err } if diff == nil { - // If diff is missing for any version between used and new version, we fall back to full reload. - return nil, nil, fmt.Errorf("failed to get schemadiff") + // Empty diff means the txn of generating schema version is committed, but the txn of `runDDLJob` is not or fail. + // It is safe to skip the empty diff because the infoschema is new enough and consistent. + continue } diffs = append(diffs, diff) } diff --git a/infoschema/cache.go b/infoschema/cache.go index 1e2191e4699da..22ea012a9be28 100644 --- a/infoschema/cache.go +++ b/infoschema/cache.go @@ -67,7 +67,26 @@ func (h *InfoCache) GetByVersion(version int64) InfoSchema { i := sort.Search(len(h.cache), func(i int) bool { return h.cache[i].SchemaMetaVersion() <= version }) - if i < len(h.cache) && h.cache[i].SchemaMetaVersion() == version { + + // `GetByVersion` is allowed to load the latest schema that is less than argument `version`. + // Consider cache has values [10, 9, _, _, 6, 5, 4, 3, 2, 1], version 8 and 7 is empty because of the diff is empty. + // If we want to get version 8, we can return version 6 because v7 and v8 do not change anything, they are totally the same, + // in this case the `i` will not be 0. + // If i == 0, it means the argument version is `10`, or greater than `10`, if `version` is 10 + // `h.cache[i].SchemaMetaVersion() == version` will be true, so we can return the latest schema, return nil if not. + // The following code is equivalent to: + // ``` + // if h.GetLatest().SchemaMetaVersion() < version { + // return nil + // } + // + // if i < len(h.cache) { + // hitVersionCounter.Inc() + // return h.cache[i] + // } + // ``` + + if i < len(h.cache) && (i != 0 || h.cache[i].SchemaMetaVersion() == version) { hitVersionCounter.Inc() return h.cache[i] } diff --git a/infoschema/cache_test.go b/infoschema/cache_test.go index 977b9906c34b9..c229df8c8747c 100644 --- a/infoschema/cache_test.go +++ b/infoschema/cache_test.go @@ -107,7 +107,7 @@ func TestGetByVersion(t *testing.T) { require.Equal(t, is1, ic.GetByVersion(1)) require.Equal(t, is3, ic.GetByVersion(3)) require.Nilf(t, ic.GetByVersion(0), "index == 0, but not found") - require.Nilf(t, ic.GetByVersion(2), "index in the middle, but not found") + require.Equal(t, int64(1), ic.GetByVersion(2).SchemaMetaVersion()) require.Nilf(t, ic.GetByVersion(4), "index == length, but not found") } From 03ba58ea12ef8272fd6522f8594deb8ed3b942f1 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 5 Jul 2022 18:10:59 +0800 Subject: [PATCH 04/21] 4. reorg handler for concurrent ddl just implements the partner of the reorg information. --- ddl/backfilling.go | 2 +- ddl/column.go | 4 +- ddl/db_test.go | 2 +- ddl/ddl.go | 70 +++++++++++++++++++------ ddl/index.go | 6 +-- ddl/job_table.go | 105 ++++++++++++++++++++++++++++++++++++++ ddl/modify_column_test.go | 2 +- ddl/partition.go | 2 +- ddl/reorg.go | 78 ++++++++++++++++++++++------ ddl/stat_test.go | 24 +++++++-- 10 files changed, 250 insertions(+), 45 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 0b4ff4b6d554e..6a1e5ff0ecfa5 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -409,7 +409,7 @@ func (w *worker) handleReorgTasks(reorgInfo *reorgInfo, totalAddedCount *int64, if err != nil { // Update the reorg handle that has been processed. - err1 := reorgInfo.UpdateReorgMeta(nextKey) + err1 := reorgInfo.UpdateReorgMeta(nextKey, w.sessPool) metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime.Seconds()) logutil.BgLogger().Warn("[ddl] backfill worker handle batch tasks failed", zap.ByteString("elementType", reorgInfo.currElement.TypeKey), diff --git a/ddl/column.go b/ddl/column.go index a1888bcace351..2f7fd1f4cad3f 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -791,7 +791,7 @@ func doReorgWorkForModifyColumnMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, j func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { - rh := newReorgHandler(t) + rh := newReorgHandler(t, w.sess, w.concurrentDDL) reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, tbl, BuildElements(changingCol, changingIdxs)) if err != nil || reorgInfo.first { // If we run reorg firstly, we should update the job snapshot version @@ -1062,7 +1062,7 @@ func (w *worker) updateColumnAndIndexes(t table.Table, oldCol, col *model.Column // Update the element in the reorgInfo for updating the reorg meta below. reorgInfo.currElement = reorgInfo.elements[i+1] // Write the reorg info to store so the whole reorganize process can recover from panic. - err := reorgInfo.UpdateReorgMeta(reorgInfo.StartKey) + err := reorgInfo.UpdateReorgMeta(reorgInfo.StartKey, w.sessPool) logutil.BgLogger().Info("[ddl] update column and indexes", zap.Int64("jobID", reorgInfo.Job.ID), zap.ByteString("elementType", reorgInfo.currElement.TypeKey), diff --git a/ddl/db_test.go b/ddl/db_test.go index 06d2f81b74c3a..add7fee199b00 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -647,7 +647,7 @@ func TestAddExpressionIndexRollback(t *testing.T) { txn, err := ctx.Txn(true) require.NoError(t, err) m := meta.NewMeta(txn) - element, start, end, physicalID, err := m.GetDDLReorgHandle(currJob) + element, start, end, physicalID, err := ddl.NewReorgHandlerForTest(m, testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) require.True(t, meta.ErrDDLReorgElementNotExist.Equal(err)) require.Nil(t, element) require.Nil(t, start) diff --git a/ddl/ddl.go b/ddl/ddl.go index e8ddb4136af21..f3f857835189d 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1136,12 +1136,13 @@ type Info struct { // GetDDLInfoWithNewTxn returns DDL information using a new txn. func GetDDLInfoWithNewTxn(s sessionctx.Context) (*Info, error) { - err := sessiontxn.NewTxn(context.Background(), s) + sess := newSession(s) + err := sess.begin() if err != nil { return nil, err } info, err := GetDDLInfo(s) - s.RollbackTxn(context.Background()) + sess.rollback() return info, err } @@ -1149,37 +1150,41 @@ func GetDDLInfoWithNewTxn(s sessionctx.Context) (*Info, error) { func GetDDLInfo(s sessionctx.Context) (*Info, error) { var err error info := &Info{} - txn, err := s.Txn(true) + sess := newSession(s) + txn, err := sess.txn() if err != nil { - return nil, err + return nil, errors.Trace(err) } t := meta.NewMeta(txn) - info.Jobs = make([]*model.Job, 0, 2) - job, err := t.GetDDLJobByIdx(0) - if err != nil { - return nil, errors.Trace(err) - } - if job != nil { - info.Jobs = append(info.Jobs, job) + enable := variable.EnableConcurrentDDL.Load() + var generalJob, reorgJob *model.Job + if enable { + generalJob, reorgJob, err = get2JobsFromTable(sess) + } else { + generalJob, reorgJob, err = get2JobsFromQueue(t) } - addIdxJob, err := t.GetDDLJobByIdx(0, meta.AddIndexJobListKey) if err != nil { return nil, errors.Trace(err) } - if addIdxJob != nil { - info.Jobs = append(info.Jobs, addIdxJob) + + if generalJob != nil { + info.Jobs = append(info.Jobs, generalJob) + } + + if reorgJob != nil { + info.Jobs = append(info.Jobs, reorgJob) } info.SchemaVer, err = t.GetSchemaVersion() if err != nil { return nil, errors.Trace(err) } - if addIdxJob == nil { + if reorgJob == nil { return info, nil } - _, info.ReorgHandle, _, _, err = newReorgHandler(t).GetDDLReorgHandle(addIdxJob) + _, info.ReorgHandle, _, _, err = newReorgHandler(t, sess, enable).GetDDLReorgHandle(reorgJob) if err != nil { if meta.ErrDDLReorgElementNotExist.Equal(err) { return info, nil @@ -1190,6 +1195,39 @@ func GetDDLInfo(s sessionctx.Context) (*Info, error) { return info, nil } +func get2JobsFromQueue(t *meta.Meta) (*model.Job, *model.Job, error) { + generalJob, err := t.GetDDLJobByIdx(0) + if err != nil { + return nil, nil, errors.Trace(err) + } + reorgJob, err := t.GetDDLJobByIdx(0, meta.AddIndexJobListKey) + if err != nil { + return nil, nil, errors.Trace(err) + } + + return generalJob, reorgJob, nil +} + +func get2JobsFromTable(sess *session) (*model.Job, *model.Job, error) { + var generalJob, reorgJob *model.Job + jobs, err := getJobsBySQL(sess, JobTable, "not reorg order by job_id limit 1") + if err != nil { + return nil, nil, errors.Trace(err) + } + + if len(jobs) != 0 { + generalJob = jobs[0] + } + jobs, err = getJobsBySQL(sess, JobTable, "reorg order by job_id limit 1") + if err != nil { + return nil, nil, errors.Trace(err) + } + if len(jobs) != 0 { + reorgJob = jobs[0] + } + return generalJob, reorgJob, nil +} + // CancelJobs cancels the DDL jobs. func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { if len(ids) == 0 { diff --git a/ddl/index.go b/ddl/index.go index f89fa95ad9c0d..8a7faebba51e0 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -658,7 +658,7 @@ func doReorgWorkForCreateIndexMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, jo func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo) (done bool, ver int64, err error) { elements := []*meta.Element{{ID: indexInfo.ID, TypeKey: meta.IndexElementKey}} - rh := newReorgHandler(t) + rh := newReorgHandler(t, w.sess, w.concurrentDDL) reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, tbl, elements) if err != nil || reorgInfo.first { // If we run reorg firstly, we should update the job snapshot version @@ -1366,7 +1366,7 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo reorg.StartKey, reorg.EndKey, reorg.PhysicalTableID = start, end, pid // Write the reorg info to store so the whole reorganize process can recover from panic. - err = reorg.UpdateReorgMeta(start) + err = reorg.UpdateReorgMeta(start, w.sessPool) logutil.BgLogger().Info("[ddl] job update reorgInfo", zap.Int64("jobID", reorg.Job.ID), zap.ByteString("elementType", reorg.currElement.TypeKey), @@ -1552,7 +1552,7 @@ func (w *worker) updateReorgInfoForPartitions(t table.PartitionedTable, reorg *r reorg.StartKey, reorg.EndKey, reorg.PhysicalTableID = start, end, pid // Write the reorg info to store so the whole reorganize process can recover from panic. - err = reorg.UpdateReorgMeta(reorg.StartKey) + err = reorg.UpdateReorgMeta(reorg.StartKey, w.sessPool) logutil.BgLogger().Info("[ddl] job update reorgInfo", zap.Int64("jobID", reorg.Job.ID), zap.ByteString("elementType", reorg.currElement.TypeKey), zap.Int64("elementID", reorg.currElement.ID), zap.Int64("partitionTableID", pid), zap.String("startHandle", tryDecodeToHandleString(start)), diff --git a/ddl/job_table.go b/ddl/job_table.go index 75d2408cf82eb..cabbfec266265 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -17,11 +17,14 @@ package ddl import ( "context" "fmt" + "math" "strconv" "strings" "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -220,6 +223,108 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { }) } +// getDDLReorgHandle gets DDL reorg handle. +func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { + sql := fmt.Sprintf("select ele_id, ele_type, start_key, end_key, physical_id from mysql.tidb_ddl_reorg where job_id = %d", job.ID) + rows, err := sess.execute(context.Background(), sql, "get_handle") + if err != nil { + return nil, nil, nil, 0, err + } + if len(rows) == 0 { + return nil, nil, nil, 0, meta.ErrDDLReorgElementNotExist + } + id := rows[0].GetInt64(0) + tp := rows[0].GetBytes(1) + element = &meta.Element{ + ID: id, + TypeKey: tp, + } + startKey = rows[0].GetBytes(2) + endKey = rows[0].GetBytes(3) + physicalTableID = rows[0].GetInt64(4) + // physicalTableID may be 0, because older version TiDB (without table partition) doesn't store them. + // update them to table's in this case. + if physicalTableID == 0 { + if job.ReorgMeta != nil { + endKey = kv.IntHandle(job.ReorgMeta.EndHandle).Encoded() + } else { + endKey = kv.IntHandle(math.MaxInt64).Encoded() + } + physicalTableID = job.TableID + logutil.BgLogger().Warn("new TiDB binary running on old TiDB DDL reorg data", + zap.Int64("partition ID", physicalTableID), + zap.Stringer("startHandle", startKey), + zap.Stringer("endHandle", endKey)) + } + return +} + +// updateDDLReorgStartHandle update the startKey of the handle. +func updateDDLReorgStartHandle(sess *session, job *model.Job, element *meta.Element, startKey kv.Key) error { + sql := fmt.Sprintf("update mysql.tidb_ddl_reorg set ele_id = %d, ele_type = %s, start_key = %s where job_id = %d", + element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), job.ID) + _, err := sess.execute(context.Background(), sql, "update_start_handle") + return err +} + +// updateDDLReorgHandle update startKey, endKey physicalTableID and element of the handle. +func updateDDLReorgHandle(sess *session, job *model.Job, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { + sql := fmt.Sprintf("update mysql.tidb_ddl_reorg set ele_id = %d, ele_type = %s, start_key = %s, end_key = %s, physical_id = %d where job_id = %d", + element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID, job.ID) + _, err := sess.execute(context.Background(), sql, "update_handle") + return err +} + +// initDDLReorgHandle initializes the handle for ddl reorg. +func initDDLReorgHandle(sess *session, job *model.Job, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { + sql := fmt.Sprintf("insert into mysql.tidb_ddl_reorg(job_id, ele_id, ele_type, start_key, end_key, physical_id) values (%d, %d, %s, %s, %s, %d)", + job.ID, element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID) + _, err := sess.execute(context.Background(), sql, "update_handle") + return err +} + +// deleteDDLReorgHandle deletes the handle for ddl reorg. +func removeDDLReorgHandle(sess *session, job *model.Job, elements []*meta.Element) error { + if len(elements) == 0 { + return nil + } + sql := fmt.Sprintf("delete from mysql.tidb_ddl_reorg where job_id = %d", job.ID) + _, err := sess.execute(context.Background(), sql, "remove_handle") + return err +} + +// removeReorgElement removes the element from ddl reorg, it is the same with removeDDLReorgHandle, only used in failpoint +func removeReorgElement(sess *session, job *model.Job) error { + sql := fmt.Sprintf("delete from mysql.tidb_ddl_reorg where job_id = %d", job.ID) + _, err := sess.execute(context.Background(), sql, "remove_handle") + return err +} + +func wrapKey2String(key []byte) string { + if len(key) == 0 { + return "''" + } + return fmt.Sprintf("0x%x", key) +} + +func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { + rows, err := sess.execute(context.Background(), fmt.Sprintf("select job_meta from mysql.%s where %s", tbl, condition), "get_job") + if err != nil { + return nil, errors.Trace(err) + } + jobs := make([]*model.Job, 0, 16) + for _, row := range rows { + jobBinary := row.GetBytes(0) + job := model.Job{} + err := job.Decode(jobBinary) + if err != nil { + return nil, errors.Trace(err) + } + jobs = append(jobs, &job) + } + return jobs, nil +} + func runInTxn(se *session, f func(*session) error) (err error) { err = se.begin() if err != nil { diff --git a/ddl/modify_column_test.go b/ddl/modify_column_test.go index cec65fb769c49..783f11d7c71d0 100644 --- a/ddl/modify_column_test.go +++ b/ddl/modify_column_test.go @@ -120,7 +120,7 @@ func TestModifyColumnReorgInfo(t *testing.T) { txn, err := ctx.Txn(true) require.NoError(t, err) m := meta.NewMeta(txn) - e, start, end, physicalID, err := m.GetDDLReorgHandle(currJob) + e, start, end, physicalID, err := ddl.NewReorgHandlerForTest(m, testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) require.True(t, meta.ErrDDLReorgElementNotExist.Equal(err)) require.Nil(t, e) require.Nil(t, start) diff --git a/ddl/partition.go b/ddl/partition.go index 3ab281c7f1afc..fb1ee682c8f87 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1166,7 +1166,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( elements = append(elements, &meta.Element{ID: idxInfo.ID, TypeKey: meta.IndexElementKey}) } } - rh := newReorgHandler(t) + rh := newReorgHandler(t, w.sess, w.concurrentDDL) reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job), d, rh, job, tbl, physicalTableIDs, elements) if err != nil || reorgInfo.first { diff --git a/ddl/reorg.go b/ddl/reorg.go index f3c87213c684d..778d48c604206 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -15,7 +15,6 @@ package ddl import ( - "context" "fmt" "strconv" "sync" @@ -33,6 +32,7 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -591,7 +591,7 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, // Third step, we need to remove the element information to make sure we can save the reorganized information to storage. failpoint.Inject("MockGetIndexRecordErr", func(val failpoint.Value) { if val.(string) == "addIdxNotOwnerErr" && atomic.CompareAndSwapUint32(&mockNotOwnerErrOnce, 3, 4) { - if err := rh.RemoveReorgElement(job); err != nil { + if err := rh.RemoveReorgElementFailPoint(job); err != nil { failpoint.Return(nil, errors.Trace(err)) } info.first = true @@ -626,7 +626,7 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, failpoint.Inject("errorUpdateReorgHandle", func() (*reorgInfo, error) { return &info, errors.New("occur an error when update reorg handle") }) - err = rh.UpdateDDLReorgHandle(job, start, end, pid, elements[0]) + err = rh.InitDDLReorgHandle(job, start, end, pid, elements[0]) if err != nil { return &info, errors.Trace(err) } @@ -639,7 +639,7 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, // Second step, we need to remove the element information to make sure we can get the error of "ErrDDLReorgElementNotExist". // However, since "txn.Reset()" will be called later, the reorganized information cannot be saved to storage. if val.(string) == "addIdxNotOwnerErr" && atomic.CompareAndSwapUint32(&mockNotOwnerErrOnce, 2, 3) { - if err := rh.RemoveReorgElement(job); err != nil { + if err := rh.RemoveReorgElementFailPoint(job); err != nil { failpoint.Return(nil, errors.Trace(err)) } } @@ -695,7 +695,7 @@ func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, jo zap.String("startHandle", tryDecodeToHandleString(start)), zap.String("endHandle", tryDecodeToHandleString(end))) - err = rh.UpdateDDLReorgHandle(job, start, end, pid, elements[0]) + err = rh.InitDDLReorgHandle(job, start, end, pid, elements[0]) if err != nil { return &info, errors.Trace(err) } @@ -727,52 +727,96 @@ func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, jo return &info, nil } -func (r *reorgInfo) UpdateReorgMeta(startKey kv.Key) error { +func (r *reorgInfo) UpdateReorgMeta(startKey kv.Key, pool *sessionPool) (err error) { if startKey == nil && r.EndKey == nil { return nil } + se, err := pool.get() + if err != nil { + return + } + defer pool.put(se) - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - err := kv.RunInNewTxn(ctx, r.d.store, true, func(ctx context.Context, txn kv.Transaction) error { - rh := newReorgHandler(meta.NewMeta(txn)) - return errors.Trace(rh.UpdateDDLReorgHandle(r.Job, startKey, r.EndKey, r.PhysicalTableID, r.currElement)) - }) + sess := newSession(se) + err = sess.begin() if err != nil { - return errors.Trace(err) + return } - return nil + txn, err := sess.txn() + if err != nil { + sess.rollback() + return err + } + rh := newReorgHandler(meta.NewMeta(txn), sess, variable.EnableConcurrentDDL.Load()) + err = rh.UpdateDDLReorgHandle(r.Job, startKey, r.EndKey, r.PhysicalTableID, r.currElement) + err1 := sess.commit() + if err == nil { + err = err1 + } + return errors.Trace(err) } // reorgHandler is used to handle the reorg information duration reorganization DDL job. type reorgHandler struct { m *meta.Meta + s *session + + enableConcurrentDDL bool +} + +// NewReorgHandlerForTest creates a new reorgHandler, only used in test. +func NewReorgHandlerForTest(t *meta.Meta, sess sessionctx.Context) *reorgHandler { + return newReorgHandler(t, newSession(sess), variable.EnableConcurrentDDL.Load()) } -func newReorgHandler(t *meta.Meta) *reorgHandler { - return &reorgHandler{m: t} +func newReorgHandler(t *meta.Meta, sess *session, enableConcurrentDDL bool) *reorgHandler { + return &reorgHandler{m: t, s: sess, enableConcurrentDDL: enableConcurrentDDL} } // UpdateDDLReorgStartHandle saves the job reorganization latest processed element and start handle for later resuming. func (r *reorgHandler) UpdateDDLReorgStartHandle(job *model.Job, element *meta.Element, startKey kv.Key) error { + if r.enableConcurrentDDL { + return updateDDLReorgStartHandle(r.s, job, element, startKey) + } return r.m.UpdateDDLReorgStartHandle(job, element, startKey) } // UpdateDDLReorgHandle saves the job reorganization latest processed information for later resuming. func (r *reorgHandler) UpdateDDLReorgHandle(job *model.Job, startKey, endKey kv.Key, physicalTableID int64, element *meta.Element) error { + if r.enableConcurrentDDL { + return updateDDLReorgHandle(r.s, job, startKey, endKey, physicalTableID, element) + } + return r.m.UpdateDDLReorgHandle(job, startKey, endKey, physicalTableID, element) +} + +// InitDDLReorgHandle initializes the job reorganization information. +func (r *reorgHandler) InitDDLReorgHandle(job *model.Job, startKey, endKey kv.Key, physicalTableID int64, element *meta.Element) error { + if r.enableConcurrentDDL { + return initDDLReorgHandle(r.s, job, startKey, endKey, physicalTableID, element) + } return r.m.UpdateDDLReorgHandle(job, startKey, endKey, physicalTableID, element) } -// RemoveReorgElement removes the element of the reorganization information. -func (r *reorgHandler) RemoveReorgElement(job *model.Job) error { +// RemoveReorgElementFailPoint removes the element of the reorganization information. +func (r *reorgHandler) RemoveReorgElementFailPoint(job *model.Job) error { + if r.enableConcurrentDDL { + return removeReorgElement(r.s, job) + } return r.m.RemoveReorgElement(job) } // RemoveDDLReorgHandle removes the job reorganization related handles. func (r *reorgHandler) RemoveDDLReorgHandle(job *model.Job, elements []*meta.Element) error { + if r.enableConcurrentDDL { + return removeDDLReorgHandle(r.s, job, elements) + } return r.m.RemoveDDLReorgHandle(job, elements) } // GetDDLReorgHandle gets the latest processed DDL reorganize position. func (r *reorgHandler) GetDDLReorgHandle(job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { + if r.enableConcurrentDDL { + return getDDLReorgHandle(r.s, job) + } return r.m.GetDDLReorgHandle(job) } diff --git a/ddl/stat_test.go b/ddl/stat_test.go index 6b40869a18ab9..4d0103013ccaa 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" + "fmt" "testing" "github.com/pingcap/failpoint" @@ -24,7 +25,9 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" @@ -116,7 +119,7 @@ func TestGetDDLInfo(t *testing.T) { RowCount: 0, } - err = addDDLJobs(txn, job) + err = addDDLJobs(sess, txn, job) require.NoError(t, err) info, err := ddl.GetDDLInfo(sess) @@ -126,7 +129,7 @@ func TestGetDDLInfo(t *testing.T) { require.Nil(t, info.ReorgHandle) // two jobs - err = addDDLJobs(txn, job1) + err = addDDLJobs(sess, txn, job1) require.NoError(t, err) info, err = ddl.GetDDLInfo(sess) @@ -140,7 +143,15 @@ func TestGetDDLInfo(t *testing.T) { require.NoError(t, err) } -func addDDLJobs(txn kv.Transaction, job *model.Job) error { +func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error { + if variable.EnableConcurrentDDL.Load() { + b, err := job.Encode(true) + if err != nil { + return err + } + _, err = sess.Execute(context.Background(), fmt.Sprintf("insert into mysql.tidb_ddl_job values (%d, %t, %d, %d, %s, %t)", job.ID, job.MayNeedReorg(), job.SchemaID, job.TableID, wrapKey2String(b), job.Type == model.ActionDropSchema)) + return err + } m := meta.NewMeta(txn) if job.MayNeedReorg() { return m.EnQueueDDLJob(job, meta.AddIndexJobListKey) @@ -148,6 +159,13 @@ func addDDLJobs(txn kv.Transaction, job *model.Job) error { return m.EnQueueDDLJob(job) } +func wrapKey2String(key []byte) string { + if len(key) == 0 { + return "''" + } + return fmt.Sprintf("0x%x", key) +} + func buildCreateIdxJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, unique bool, indexName string, colName string) *model.Job { return &model.Job{ SchemaID: dbInfo.ID, From 7d3fd2c0195b94f6e3dd457a3dc94222d21d021f Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 7 Jul 2022 13:50:34 +0800 Subject: [PATCH 05/21] 5. manage ddl jobs for concurrent ddl add the partner of add job, delete job and many others related to history job because many of the functions need a session now, we just change the caller --- br/pkg/backup/client.go | 43 ++++++- br/pkg/backup/client_test.go | 2 +- br/pkg/restore/db_test.go | 4 +- br/pkg/task/backup.go | 3 +- ddl/db_change_test.go | 4 +- ddl/ddl.go | 200 +++++++++++++++++++++++++++----- ddl/ddl_tiflash_api.go | 2 +- ddl/ddl_worker.go | 102 +++++++++++----- ddl/job_table.go | 52 +++++++++ ddl/multi_schema_change_test.go | 35 +++--- ddl/restart_test.go | 6 +- ddl/stat_test.go | 9 +- ddl/table_modify_test.go | 4 +- executor/builder.go | 2 +- executor/executor.go | 66 ++++++++--- executor/infoschema_reader.go | 25 +++- server/http_handler.go | 3 +- 17 files changed, 440 insertions(+), 122 deletions(-) diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index d6548daecbdfa..321b1307e5488 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" @@ -472,7 +473,7 @@ func skipUnsupportedDDLJob(job *model.Job) bool { } // WriteBackupDDLJobs sends the ddl jobs are done in (lastBackupTS, backupTS] to metaWriter. -func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, store kv.Storage, lastBackupTS, backupTS uint64) error { +func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, se sessionctx.Context, store kv.Storage, lastBackupTS, backupTS uint64) error { snapshot := store.GetSnapshot(kv.NewVersion(backupTS)) snapMeta := meta.NewSnapshotMeta(snapshot) lastSnapshot := store.GetSnapshot(kv.NewVersion(lastBackupTS)) @@ -481,12 +482,46 @@ func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, store kv.Storage, lastB if err != nil { return errors.Trace(err) } - allJobs, err := ddl.GetAllDDLJobs(snapMeta) + backupSchemaVersion, err := snapMeta.GetSchemaVersion() + if err != nil { + return errors.Trace(err) + } + + // we need use the version - 1 if the correlation diff is empty. + // consider the following case: + // t1 t2 t3 + // | | | + // schema version commit | diff commit + // backupTS + // DDL job is not done at t2, but schema version can be seen at t2, so if we use the schema as the boundary, we may filter in + // the job which done at t3. So, we need make version -= 1.It is safe because at t2, we can not see the job. + diff, err := lastSnapMeta.GetSchemaDiff(lastSchemaVersion) + if err != nil { + return errors.Trace(err) + } + if diff == nil { + lastSchemaVersion -= 1 + } + + diff, err = snapMeta.GetSchemaDiff(backupSchemaVersion) + if err != nil { + return errors.Trace(err) + } + if diff == nil { + backupSchemaVersion -= 1 + } + + version, err := store.CurrentVersion(kv.GlobalTxnScope) + if err != nil { + return errors.Trace(err) + } + newestMeta := meta.NewSnapshotMeta(store.GetSnapshot(kv.NewVersion(version.Ver))) + allJobs, err := ddl.GetAllDDLJobs(se, newestMeta) if err != nil { return errors.Trace(err) } log.Debug("get all jobs", zap.Int("jobs", len(allJobs))) - historyJobs, err := ddl.GetAllHistoryDDLJobs(snapMeta) + historyJobs, err := ddl.GetAllHistoryDDLJobs(newestMeta) if err != nil { return errors.Trace(err) } @@ -500,7 +535,7 @@ func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, store kv.Storage, lastB } if (job.State == model.JobStateDone || job.State == model.JobStateSynced) && - (job.BinlogInfo != nil && job.BinlogInfo.SchemaVersion > lastSchemaVersion) { + (job.BinlogInfo != nil && job.BinlogInfo.SchemaVersion > lastSchemaVersion && job.BinlogInfo.SchemaVersion <= backupSchemaVersion) { if job.BinlogInfo.DBInfo != nil { // ignore all placement policy info during incremental backup for now. job.BinlogInfo.DBInfo.PlacementPolicyRef = nil diff --git a/br/pkg/backup/client_test.go b/br/pkg/backup/client_test.go index d2a53c9934d4c..60cf42f94998a 100644 --- a/br/pkg/backup/client_test.go +++ b/br/pkg/backup/client_test.go @@ -280,7 +280,7 @@ func TestSkipUnsupportedDDLJob(t *testing.T) { metaWriter := metautil.NewMetaWriter(s.storage, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDDL) - err = backup.WriteBackupDDLJobs(metaWriter, s.cluster.Storage, lastTS, ts) + err = backup.WriteBackupDDLJobs(metaWriter, tk.Session(), s.cluster.Storage, lastTS, ts) require.NoErrorf(t, err, "Error get ddl jobs: %s", err) err = metaWriter.FinishWriteMetas(ctx, metautil.AppendDDL) require.NoError(t, err, "Flush failed", err) diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index a3279ddaad3d9..89ff7a9ab62a4 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -194,7 +194,7 @@ func TestFilterDDLJobs(t *testing.T) { metaWriter := metautil.NewMetaWriter(s.storage, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDDL) - err = backup.WriteBackupDDLJobs(metaWriter, s.mock.Storage, lastTS, ts) + err = backup.WriteBackupDDLJobs(metaWriter, tk.Session(), s.mock.Storage, lastTS, ts) require.NoErrorf(t, err, "Error get ddl jobs: %s", err) err = metaWriter.FinishWriteMetas(ctx, metautil.AppendDDL) require.NoErrorf(t, err, "Flush failed", err) @@ -258,7 +258,7 @@ func TestFilterDDLJobsV2(t *testing.T) { metaWriter := metautil.NewMetaWriter(s.storage, metautil.MetaFileSize, true, "", &cipher) ctx := context.Background() metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDDL) - err = backup.WriteBackupDDLJobs(metaWriter, s.mock.Storage, lastTS, ts) + err = backup.WriteBackupDDLJobs(metaWriter, tk.Session(), s.mock.Storage, lastTS, ts) require.NoErrorf(t, err, "Error get ddl jobs: %s", err) err = metaWriter.FinishWriteMetas(ctx, metautil.AppendDDL) require.NoErrorf(t, err, "Flush failed", err) diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index f100e4ea5121b..8aaf0c8d6f0c8 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/types" @@ -399,7 +400,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig } metawriter.StartWriteMetasAsync(ctx, metautil.AppendDDL) - err = backup.WriteBackupDDLJobs(metawriter, mgr.GetStorage(), cfg.LastBackupTS, backupTS) + err = backup.WriteBackupDDLJobs(metawriter, se.(sessionctx.Context), mgr.GetStorage(), cfg.LastBackupTS, backupTS) if err != nil { return errors.Trace(err) } diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 6f655327e5216..5589a3ff77c03 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1378,7 +1378,7 @@ func prepareTestControlParallelExecSQL(t *testing.T, store kv.Storage, dom *doma require.NoError(t, err) txn, err := sess.Txn(true) require.NoError(t, err) - jobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + jobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) qLen = len(jobs) if qLen == 2 { @@ -1407,7 +1407,7 @@ func prepareTestControlParallelExecSQL(t *testing.T, store kv.Storage, dom *doma require.NoError(t, err) txn, err := sess.Txn(true) require.NoError(t, err) - jobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + jobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) qLen = len(jobs) if qLen == 1 { diff --git a/ddl/ddl.go b/ddl/ddl.go index f3f857835189d..824b1c3af2e01 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -23,6 +23,8 @@ import ( "encoding/json" "flag" "fmt" + "strconv" + "strings" "sync" "sync/atomic" "time" @@ -815,16 +817,28 @@ func (d *ddl) asyncNotifyWorker(job *model.Job) { if !RunWorker { return } - var worker *worker - if job.MayNeedReorg() { - worker = d.workers[addIdxWorker] - } else { - worker = d.workers[generalWorker] - } - if d.ownerManager.IsOwner() { - asyncNotify(worker.ddlJobCh) + if variable.EnableConcurrentDDL.Load() { + if d.isOwner() { + asyncNotify(d.ddlJobCh) + } else { + key := addingDDLJobGeneral + if job.MayNeedReorg() { + key = addingDDLJobReorg + } + d.asyncNotifyByEtcd(key, job) + } } else { - d.asyncNotifyByEtcd(worker.addingDDLJobKey, job) + var worker *worker + if job.MayNeedReorg() { + worker = d.workers[addIdxWorker] + } else { + worker = d.workers[generalWorker] + } + if d.ownerManager.IsOwner() { + asyncNotify(worker.ddlJobCh) + } else { + d.asyncNotifyByEtcd(worker.addingDDLJobKey, job) + } } } @@ -936,17 +950,16 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { // If the connection being killed, we need to CANCEL the DDL job. if atomic.LoadUint32(&sessVars.Killed) == 1 { if sessVars.StmtCtx.DDLJobID != 0 { + se, err := d.sessPool.get() + if err != nil { + continue + } sessVars.StmtCtx.DDLJobID = 0 // Avoid repeat. - - err := kv.RunInNewTxn(context.Background(), d.store, true, func(ctx context.Context, txn kv.Transaction) error { - // errs is the error per job, there is only one submitted - // err is the error of the overall task - errs, err := CancelJobs(txn, []int64{jobID}) - if len(errs) > 0 { - logutil.BgLogger().Warn("error canceling DDL job", zap.Error(errs[0])) - } - return err - }) + errs, err := CancelJobs(se, d.store, []int64{jobID}) + d.sessPool.put(se) + if len(errs) > 0 { + logutil.BgLogger().Warn("error canceling DDL job", zap.Error(errs[0])) + } if err != nil { logutil.BgLogger().Warn("Kill command could not cancel DDL job", zap.Error(err)) continue @@ -954,7 +967,12 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { } } - historyJob, err = d.getHistoryDDLJob(jobID) + se, err := d.sessPool.get() + if err != nil { + continue + } + historyJob, err = GetHistoryJobByID(se, jobID) + d.sessPool.put(se) if err != nil { logutil.BgLogger().Error("[ddl] get history DDL job failed, check again", zap.Error(err)) continue @@ -1229,7 +1247,19 @@ func get2JobsFromTable(sess *session) (*model.Job, *model.Job, error) { } // CancelJobs cancels the DDL jobs. -func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { +func CancelJobs(se sessionctx.Context, store kv.Storage, ids []int64) (errs []error, err error) { + if variable.EnableConcurrentDDL.Load() { + return cancelConcurrencyJobs(se, ids) + } + + err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { + errs, err = cancelLegacyJobs(txn, ids) + return err + }) + return +} + +func cancelLegacyJobs(txn kv.Transaction, ids []int64) ([]error, error) { if len(ids) == 0 { return nil, nil } @@ -1295,6 +1325,82 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { return errs, nil } +// cancelConcurrencyJobs cancels the DDL jobs that are in the concurrent state. +func cancelConcurrencyJobs(se sessionctx.Context, ids []int64) ([]error, error) { + failpoint.Inject("mockCancelConcurencyDDL", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(nil, errors.New("mock commit error")) + } + }) + if len(ids) == 0 { + return nil, nil + } + var jobMap = make(map[int64]int) // jobID -> error index + + sess := newSession(se) + err := sess.begin() + if err != nil { + return nil, err + } + + idsStr := make([]string, 0, len(ids)) + for idx, id := range ids { + jobMap[id] = idx + idsStr = append(idsStr, strconv.FormatInt(id, 10)) + } + + jobs, err := getJobsBySQL(sess, JobTable, fmt.Sprintf("job_id in (%s) order by job_id", strings.Join(idsStr, ", "))) + if err != nil { + sess.rollback() + return nil, err + } + + errs := make([]error, len(ids)) + + for _, job := range jobs { + i, ok := jobMap[job.ID] + if !ok { + logutil.BgLogger().Debug("the job that needs to be canceled isn't equal to current job", + zap.Int64("need to canceled job ID", job.ID), + zap.Int64("current job ID", job.ID)) + continue + } + delete(jobMap, job.ID) + // These states can't be cancelled. + if job.IsDone() || job.IsSynced() { + errs[i] = dbterror.ErrCancelFinishedDDLJob.GenWithStackByArgs(job.ID) + continue + } + // If the state is rolling back, it means the work is cleaning the data after cancelling the job. + if job.IsCancelled() || job.IsRollingback() || job.IsRollbackDone() { + continue + } + if !job.IsRollbackable() { + errs[i] = dbterror.ErrCannotCancelDDLJob.GenWithStackByArgs(job.ID) + continue + } + job.State = model.JobStateCancelling + // Make sure RawArgs isn't overwritten. + err := json.Unmarshal(job.RawArgs, &job.Args) + if err != nil { + errs[i] = errors.Trace(err) + continue + } + err = updateDDLJob2Table(sess, job, true) + if err != nil { + errs[i] = errors.Trace(err) + } + } + err = sess.commit() + if err != nil { + return nil, err + } + for id, idx := range jobMap { + errs[idx] = dbterror.ErrDDLJobNotFound.GenWithStackByArgs(id) + } + return errs, nil +} + func getDDLJobsInQueue(t *meta.Meta, jobListKey meta.JobListKeyType) ([]*model.Job, error) { cnt, err := t.DDLJobQueueLen(jobListKey) if err != nil { @@ -1311,7 +1417,16 @@ func getDDLJobsInQueue(t *meta.Meta, jobListKey meta.JobListKeyType) ([]*model.J } // GetAllDDLJobs get all DDL jobs and sorts jobs by job.ID. -func GetAllDDLJobs(t *meta.Meta) ([]*model.Job, error) { +func GetAllDDLJobs(sess sessionctx.Context, t *meta.Meta) ([]*model.Job, error) { + if variable.EnableConcurrentDDL.Load() { + return getJobsBySQL(newSession(sess), "tidb_ddl_job", "1 order by job_id") + } + + return getDDLJobs(t) +} + +// getDDLJobs get all DDL jobs and sorts jobs by job.ID. +func getDDLJobs(t *meta.Meta) ([]*model.Job, error) { generalJobs, err := getDDLJobsInQueue(t, meta.DefaultJobListKey) if err != nil { return nil, errors.Trace(err) @@ -1338,7 +1453,7 @@ const batchNumHistoryJobs = 128 // GetLastNHistoryDDLJobs returns the DDL history jobs and an error. // The maximum count of history jobs is num. func GetLastNHistoryDDLJobs(t *meta.Meta, maxNumJobs int) ([]*model.Job, error) { - iterator, err := t.GetLastHistoryDDLJobsIterator() + iterator, err := GetLastHistoryDDLJobsIterator(t) if err != nil { return nil, errors.Trace(err) } @@ -1348,7 +1463,7 @@ func GetLastNHistoryDDLJobs(t *meta.Meta, maxNumJobs int) ([]*model.Job, error) // IterHistoryDDLJobs iterates history DDL jobs until the `finishFn` return true or error. func IterHistoryDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { txnMeta := meta.NewMeta(txn) - iter, err := txnMeta.GetLastHistoryDDLJobsIterator() + iter, err := GetLastHistoryDDLJobsIterator(txnMeta) if err != nil { return err } @@ -1367,8 +1482,8 @@ func IterHistoryDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, e // IterAllDDLJobs will iterates running DDL jobs first, return directly if `finishFn` return true or error, // then iterates history DDL jobs until the `finishFn` return true or error. -func IterAllDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { - jobs, err := GetAllDDLJobs(meta.NewMeta(txn)) +func IterAllDDLJobs(ctx sessionctx.Context, txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { + jobs, err := GetAllDDLJobs(ctx, meta.NewMeta(txn)) if err != nil { return err } @@ -1380,6 +1495,11 @@ func IterAllDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error return IterHistoryDDLJobs(txn, finishFn) } +// GetLastHistoryDDLJobsIterator gets latest N history DDL jobs iterator. +func GetLastHistoryDDLJobsIterator(m *meta.Meta) (meta.LastJobIterator, error) { + return m.GetLastHistoryDDLJobsIterator() +} + // session wraps sessionctx.Context for transaction usage. type session struct { sessionctx.Context @@ -1440,7 +1560,7 @@ func (s *session) session() sessionctx.Context { // GetAllHistoryDDLJobs get all the done DDL jobs. func GetAllHistoryDDLJobs(m *meta.Meta) ([]*model.Job, error) { - iterator, err := m.GetLastHistoryDDLJobsIterator() + iterator, err := GetLastHistoryDDLJobsIterator(m) if err != nil { return nil, errors.Trace(err) } @@ -1481,12 +1601,28 @@ func GetHistoryJobByID(sess sessionctx.Context, id int64) (*model.Job, error) { return job, errors.Trace(err) } -// AddHistoryDDLJob adds DDL job to history table. -func AddHistoryDDLJob(t *meta.Meta, job *model.Job, updateRawArgs bool) error { +// AddHistoryDDLJob record the history job. +func AddHistoryDDLJob(sess *session, t *meta.Meta, job *model.Job, updateRawArgs bool, concurrentDDL bool) error { + if concurrentDDL { + // only add history job into table if it is concurrent DDL. + err := addHistoryDDLJob2Table(sess, job, updateRawArgs) + if err != nil { + logutil.BgLogger().Info("[ddl] failed to add DDL job to history table", zap.Error(err)) + } + } + // we always add history DDL job to job list at this moment. return t.AddHistoryDDLJob(job, updateRawArgs) } -// GetLastHistoryDDLJobsIterator gets latest N history ddl jobs iterator. -func GetLastHistoryDDLJobsIterator(m *meta.Meta) (meta.LastJobIterator, error) { - return m.GetLastHistoryDDLJobsIterator() +// addHistoryDDLJob2Table adds DDL job to history table. +func addHistoryDDLJob2Table(sess *session, job *model.Job, updateRawArgs bool) error { + b, err := job.Encode(updateRawArgs) + if err != nil { + return err + } + _, err = sess.execute(context.Background(), + fmt.Sprintf("insert ignore into mysql.tidb_ddl_history(job_id, job_meta, db_name, table_name, schema_id, table_id, create_time) values (%d, %s, %s, %s, %d, %d, %v)", + job.ID, wrapKey2String(b), strconv.Quote(job.SchemaName), strconv.Quote(job.TableName), job.SchemaID, job.TableID, strconv.Quote(model.TSConvert2Time(job.StartTS).String())), + "insert_history") + return errors.Trace(err) } diff --git a/ddl/ddl_tiflash_api.go b/ddl/ddl_tiflash_api.go index ac200a9667655..4c3f9a70a3cec 100644 --- a/ddl/ddl_tiflash_api.go +++ b/ddl/ddl_tiflash_api.go @@ -499,7 +499,7 @@ func getDropOrTruncateTableTiflash(ctx sessionctx.Context, currentSchema infosch return GetDropOrTruncateTableInfoFromJobsByStore(jobs, gcSafePoint, getTable, handleJobAndTableInfo) } - err = IterAllDDLJobs(txn, fn) + err = IterAllDDLJobs(ctx, txn, fn) if err != nil { if terror.ErrorEqual(variable.ErrSnapshotTooOld, err) { // The err indicate that current ddl job and remain DDL jobs was been deleted by GC, diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 582b766944182..16f07d0d08a3d 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -292,8 +292,29 @@ func (d *ddl) limitDDLJobs() { // addBatchDDLJobs gets global job IDs and puts the DDL jobs in the DDL queue. func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { startTime := time.Now() + var err error + if variable.EnableConcurrentDDL.Load() { + err = d.addBatchDDLJobs2Table(tasks) + } else { + err = d.addBatchDDLJobs2Queue(tasks) + } + var jobs string + for _, task := range tasks { + task.err <- err + jobs += task.job.String() + "; " + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, task.job.Type.String(), + metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + } + if err != nil { + logutil.BgLogger().Warn("[ddl] add DDL jobs failed", zap.String("jobs", jobs), zap.Error(err)) + } else { + logutil.BgLogger().Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs)) + } +} + +func (d *ddl) addBatchDDLJobs2Queue(tasks []*limitJobTask) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - err := kv.RunInNewTxn(ctx, d.store, true, func(ctx context.Context, txn kv.Transaction) error { + return kv.RunInNewTxn(ctx, d.store, true, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) ids, err := t.GenGlobalIDs(len(tasks)) if err != nil { @@ -325,18 +346,6 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { }) return nil }) - var jobs string - for _, task := range tasks { - task.err <- err - jobs += task.job.String() + "; " - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, task.job.Type.String(), - metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) - } - if err != nil { - logutil.BgLogger().Warn("[ddl] add DDL jobs failed", zap.String("jobs", jobs), zap.Error(err)) - } else { - logutil.BgLogger().Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs)) - } } func injectModifyJobArgFailPoint(job *model.Job) { @@ -363,16 +372,41 @@ func setJobStateToQueueing(job *model.Job) { job.State = model.JobStateQueueing } -// getHistoryDDLJob gets a DDL job with job's ID from history queue. -func (d *ddl) getHistoryDDLJob(id int64) (*model.Job, error) { - se, err := d.sessPool.get() - if err != nil { - return nil, errors.Trace(err) +// addBatchDDLJobs2Table gets global job IDs and puts the DDL jobs in the DDL job table. +func (d *ddl) addBatchDDLJobs2Table(tasks []*limitJobTask) error { + var ids []int64 + var err error + startTS := uint64(0) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err = kv.RunInNewTxn(ctx, d.store, true, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + ids, err = t.GenGlobalIDs(len(tasks)) + if err != nil { + return errors.Trace(err) + } + startTS = txn.StartTS() + return nil + }) + if err == nil { + jobTasks := make([]*model.Job, len(tasks)) + for i, task := range tasks { + job := task.job + job.Version = currentVersion + job.StartTS = startTS + job.ID = ids[i] + setJobStateToQueueing(job) + jobTasks[i] = job + injectModifyJobArgFailPoint(job) + } + sess, err1 := d.sessPool.get() + if err1 == nil { + sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + err1 = insertDDLJobs2Table(newSession(sess), jobTasks, true) + d.sessPool.put(sess) + } + err = err1 } - defer d.sessPool.put(se) - job, err := GetHistoryJobByID(se, id) - - return job, errors.Trace(err) + return errors.Trace(err) } func injectFailPointForGetJob(job *model.Job) { @@ -425,7 +459,13 @@ func (w *worker) updateDDLJob(t *meta.Meta, job *model.Job, meetErr bool) error logutil.Logger(w.logCtx).Info("[ddl] meet something wrong before update DDL job, shouldn't update raw args", zap.String("job", job.String())) } - return errors.Trace(t.UpdateDDLJob(0, job, updateRawArgs)) + var err error + if w.concurrentDDL { + err = updateDDLJob2Table(w.sess, job, updateRawArgs) + } else { + err = t.UpdateDDLJob(0, job, updateRawArgs) + } + return errors.Trace(err) } func needUpdateRawArgs(job *model.Job, meetErr bool) bool { @@ -508,8 +548,11 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { if err != nil { return errors.Trace(err) } - - _, err = t.DeQueueDDLJob() + if w.concurrentDDL { + err = w.deleteDDLJob(job) + } else { + _, err = t.DeQueueDDLJob() + } if err != nil { return errors.Trace(err) } @@ -524,7 +567,7 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { } w.writeDDLSeqNum(job) w.removeJobCtx(job) - err = t.AddHistoryDDLJob(job, updateRawArgs) + err = AddHistoryDDLJob(w.sess, t, job, updateRawArgs, w.concurrentDDL) return errors.Trace(err) } @@ -660,6 +703,8 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { txn.SetDiskFullOpt(kvrpcpb.DiskFullOpt_NotAllowedOnFull) } w.setDDLLabelForTopSQL(job) + w.setDDLSourceForDiagnosis(job) + jobContext := w.jobContext(job) if tagger := w.getResourceGroupTaggerForTopSQL(job); tagger != nil { txn.SetOption(kv.ResourceGroupTagger, tagger) } @@ -676,6 +721,9 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { d.mu.hook.OnJobRunBefore(job) d.mu.RUnlock() + // set request source type to DDL type + txn.SetOption(kv.RequestSourceType, jobContext.ddlJobSourceType()) + // If running job meets error, we will save this error in job Error // and retry later if the job is not cancelled. schemaVer, runJobErr = w.runDDLJob(d, t, job) @@ -1021,10 +1069,12 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerRunDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) }() if job.IsFinished() { + logutil.Logger(w.logCtx).Debug("[ddl] finish DDL job", zap.String("job", job.String())) return } // The cause of this job state is that the job is cancelled by client. if job.IsCancelling() { + logutil.Logger(w.logCtx).Debug("[ddl] cancel DDL job", zap.String("job", job.String())) return convertJob2RollbackJob(w, d, t, job) } diff --git a/ddl/job_table.go b/ddl/job_table.go index cabbfec266265..168dee4945a96 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -15,6 +15,7 @@ package ddl import ( + "bytes" "context" "fmt" "math" @@ -23,6 +24,8 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" @@ -223,6 +226,55 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { }) } +const ( + addDDLJobSQL = "insert into mysql.tidb_ddl_job(job_id, reorg, schema_id, table_id, job_meta, is_drop_schema) values" + updateDDLJobSQL = "update mysql.tidb_ddl_job set job_meta = %s where job_id = %d" +) + +func insertDDLJobs2Table(sess *session, jobs []*model.Job, updateRawArgs bool) error { + failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr")) + } + }) + if len(jobs) == 0 { + return nil + } + var sql bytes.Buffer + sql.WriteString(addDDLJobSQL) + for i, job := range jobs { + b, err := job.Encode(updateRawArgs) + if err != nil { + return err + } + if i != 0 { + sql.WriteString(",") + } + sql.WriteString(fmt.Sprintf("(%d, %t, %d, %d, %s, %t)", job.ID, job.MayNeedReorg(), job.SchemaID, job.TableID, wrapKey2String(b), job.Type == model.ActionDropSchema)) + } + sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + _, err := sess.execute(ctx, sql.String(), "insert_job") + logutil.BgLogger().Debug("[ddl] add job to mysql.tidb_ddl_job table", zap.String("sql", sql.String())) + return errors.Trace(err) +} + +func (w *worker) deleteDDLJob(job *model.Job) error { + sql := fmt.Sprintf("delete from mysql.tidb_ddl_job where job_id = %d", job.ID) + _, err := w.sess.execute(context.Background(), sql, "delete_job") + return errors.Trace(err) +} + +func updateDDLJob2Table(sctx *session, job *model.Job, updateRawArgs bool) error { + b, err := job.Encode(updateRawArgs) + if err != nil { + return err + } + sql := fmt.Sprintf(updateDDLJobSQL, wrapKey2String(b), job.ID) + _, err = sctx.execute(context.Background(), sql, "update_job") + return errors.Trace(err) +} + // getDDLReorgHandle gets DDL reorg handle. func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { sql := fmt.Sprintf("select ele_id, ele_type, start_key, end_key, physical_id from mysql.tidb_ddl_reorg where job_id = %d", job.ID) diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index c16ab46a62648..d6a13b39aff74 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -15,7 +15,6 @@ package ddl_test import ( - "context" "strconv" "testing" @@ -25,6 +24,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -141,7 +141,7 @@ func TestMultiSchemaChangeAddColumnsCancelled(t *testing.T) { tk.MustExec("create table t (a int);") tk.MustExec("insert into t values (1);") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'c' is in write-reorg. return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StateWriteReorganization }) @@ -223,7 +223,7 @@ func TestMultiSchemaChangeDropColumnsCancelled(t *testing.T) { // Test for cancelling the job in a middle state. tk.MustExec("create table t (a int default 1, b int default 2, c int default 3, d int default 4);") tk.MustExec("insert into t values ();") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'a' is in delete-reorg. return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StateDeleteReorganization }) @@ -237,7 +237,7 @@ func TestMultiSchemaChangeDropColumnsCancelled(t *testing.T) { tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int default 1, b int default 2, c int default 3, d int default 4);") tk.MustExec("insert into t values ();") - hook = newCancelJobHook(store, dom, func(job *model.Job) bool { + hook = newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'a' is in public. return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StatePublic }) @@ -259,7 +259,7 @@ func TestMultiSchemaChangeDropIndexedColumnsCancelled(t *testing.T) { tk.MustExec("create table t (a int default 1, b int default 2, c int default 3, d int default 4, " + "index(a), index(b), index(c), index(d));") tk.MustExec("insert into t values ();") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'a' is in delete-reorg. return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StateDeleteReorganization }) @@ -558,7 +558,7 @@ func TestMultiSchemaChangeAddIndexesCancelled(t *testing.T) { tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int, c int);") tk.MustExec("insert into t values (1, 2, 3);") - cancelHook := newCancelJobHook(store, dom, func(job *model.Job) bool { + cancelHook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel the job when index 't2' is in write-reorg. return job.MultiSchemaInfo.SubJobs[2].SchemaState == model.StateWriteReorganization }) @@ -576,7 +576,7 @@ func TestMultiSchemaChangeAddIndexesCancelled(t *testing.T) { tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int, c int);") tk.MustExec("insert into t values (1, 2, 3);") - cancelHook = newCancelJobHook(store, dom, func(job *model.Job) bool { + cancelHook = newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel the job when index 't1' is in public. return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StatePublic }) @@ -1160,6 +1160,7 @@ type cancelOnceHook struct { triggered bool cancelErr error pred func(job *model.Job) bool + s sessionctx.Context ddl.TestDDLCallback } @@ -1169,15 +1170,12 @@ func (c *cancelOnceHook) OnJobUpdated(job *model.Job) { return } c.triggered = true - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - c.cancelErr = kv.RunInNewTxn(ctx, c.store, false, - func(ctx context.Context, txn kv.Transaction) error { - errs, err := ddl.CancelJobs(txn, []int64{job.ID}) - if errs[0] != nil { - return errs[0] - } - return err - }) + errs, err := ddl.CancelJobs(c.s, c.store, []int64{job.ID}) + if errs[0] != nil { + c.cancelErr = errs[0] + return + } + c.cancelErr = err } func (c *cancelOnceHook) MustCancelDone(t *testing.T) { @@ -1190,12 +1188,15 @@ func (c *cancelOnceHook) MustCancelFailed(t *testing.T) { require.Contains(t, c.cancelErr.Error(), strconv.Itoa(errno.ErrCannotCancelDDLJob)) } -func newCancelJobHook(store kv.Storage, dom *domain.Domain, +func newCancelJobHook(t *testing.T, store kv.Storage, dom *domain.Domain, pred func(job *model.Job) bool) *cancelOnceHook { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") return &cancelOnceHook{ store: store, pred: pred, TestDDLCallback: ddl.TestDDLCallback{Do: dom}, + s: tk.Session(), } } diff --git a/ddl/restart_test.go b/ddl/restart_test.go index 130e988a9367f..10d35802f5d2c 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -68,11 +68,7 @@ func runInterruptedJob(t *testing.T, store kv.Storage, d ddl.DDL, job *model.Job endlessLoopTime := time.Now().Add(time.Minute) for history == nil { // imitate DoDDLJob's logic, quit only find history - err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, false, func(ctx context.Context, txn kv.Transaction) error { - history, err = meta.NewMeta(txn).GetHistoryDDLJob(job.ID) - return err - }) - require.NoError(t, err) + history, _ = ddl.GetHistoryJobByID(ctx, job.ID) if history != nil { err = history.Error } diff --git a/ddl/stat_test.go b/ddl/stat_test.go index 4d0103013ccaa..c95b882443857 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -95,9 +95,9 @@ func TestGetDDLInfo(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() - sess := testkit.NewTestKit(t, store).Session() - _, err := sess.Execute(context.Background(), "begin") - require.NoError(t, err) + tk := testkit.NewTestKit(t, store) + sess := tk.Session() + tk.MustExec("begin") txn, err := sess.Txn(true) require.NoError(t, err) @@ -139,8 +139,7 @@ func TestGetDDLInfo(t *testing.T) { require.Equal(t, job1, info.Jobs[1]) require.Nil(t, info.ReorgHandle) - _, err = sess.Execute(context.Background(), "rollback") - require.NoError(t, err) + tk.MustExec("rollback") } func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error { diff --git a/ddl/table_modify_test.go b/ddl/table_modify_test.go index 038fcc878d941..50f8ea8a9b6d0 100644 --- a/ddl/table_modify_test.go +++ b/ddl/table_modify_test.go @@ -232,7 +232,7 @@ func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql require.NoError(t, err) txn, err := sess.Txn(true) require.NoError(t, err) - jobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + jobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) qLen = len(jobs) if qLen == 2 { @@ -260,7 +260,7 @@ func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql require.NoError(t, err) txn, err := sess.Txn(true) require.NoError(t, err) - jobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + jobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) qLen = len(jobs) if qLen == 1 { diff --git a/executor/builder.go b/executor/builder.go index 503c49c199c7d..d7928fe36fbbe 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3192,7 +3192,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E return nil } failpoint.Inject("checkUseMPP", func(val failpoint.Value) { - if val.(bool) != useMPPExecution(b.ctx, v) { + if !b.ctx.GetSessionVars().InRestrictedSQL && val.(bool) != useMPPExecution(b.ctx, v) { if val.(bool) { b.err = errors.New("expect mpp but not used") } else { diff --git a/executor/executor.go b/executor/executor.go index be6e75495fce0..e147eb9bd38d9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -340,12 +340,13 @@ type CancelDDLJobsExec struct { // Open implements the Executor Open interface. func (e *CancelDDLJobsExec) Open(ctx context.Context) error { // We want to use a global transaction to execute the admin command, so we don't use e.ctx here. - ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnDDL) - errInTxn := kv.RunInNewTxn(ctx, e.ctx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) (err error) { - e.errs, err = ddl.CancelJobs(txn, e.jobIDs) - return - }) - return errInTxn + newSess, err := e.getSysSession() + if err != nil { + return err + } + e.errs, err = ddl.CancelJobs(newSess, e.ctx.GetStore(), e.jobIDs) + e.releaseSysSession(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), newSess) + return err } // Next implements the Executor Next interface. @@ -357,7 +358,7 @@ func (e *CancelDDLJobsExec) Next(ctx context.Context, req *chunk.Chunk) error { numCurBatch := mathutil.Min(req.Capacity(), len(e.jobIDs)-e.cursor) for i := e.cursor; i < e.cursor+numCurBatch; i++ { req.AppendString(0, strconv.FormatInt(e.jobIDs[i], 10)) - if e.errs[i] != nil { + if e.errs != nil && e.errs[i] != nil { req.AppendString(1, fmt.Sprintf("error: %v", e.errs[i])) } else { req.AppendString(1, "successful") @@ -479,6 +480,7 @@ type ShowDDLJobsExec struct { jobNumber int is infoschema.InfoSchema + sess sessionctx.Context } // DDLJobRetriever retrieve the DDLJobs. @@ -493,9 +495,9 @@ type DDLJobRetriever struct { TZLoc *time.Location } -func (e *DDLJobRetriever) initial(txn kv.Transaction) error { +func (e *DDLJobRetriever) initial(txn kv.Transaction, sess sessionctx.Context) error { m := meta.NewMeta(txn) - jobs, err := ddl.GetAllDDLJobs(m) + jobs, err := ddl.GetAllDDLJobs(sess, m) if err != nil { return err } @@ -606,18 +608,35 @@ type ShowDDLJobQueriesExec struct { // Open implements the Executor Open interface. func (e *ShowDDLJobQueriesExec) Open(ctx context.Context) error { + var err error + var jobs []*model.Job if err := e.baseExecutor.Open(ctx); err != nil { return err } - txn, err := e.ctx.Txn(true) + session, err := e.getSysSession() if err != nil { return err } + err = sessiontxn.NewTxn(context.Background(), session) + if err != nil { + return err + } + defer func() { + // releaseSysSession will rollbacks txn automatically. + e.releaseSysSession(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), session) + }() + txn, err := session.Txn(true) + if err != nil { + return err + } + session.GetSessionVars().SetInTxn(true) + m := meta.NewMeta(txn) - jobs, err := ddl.GetAllDDLJobs(m) + jobs, err = ddl.GetAllDDLJobs(session, m) if err != nil { return err } + historyJobs, err := ddl.GetLastNHistoryDDLJobs(m, ddl.DefNumHistoryJobs) if err != nil { return err @@ -655,19 +674,26 @@ func (e *ShowDDLJobsExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } - txn, err := e.ctx.Txn(true) - if err != nil { - return err - } e.DDLJobRetriever.is = e.is if e.jobNumber == 0 { e.jobNumber = ddl.DefNumHistoryJobs } - err = e.DDLJobRetriever.initial(txn) + sess, err := e.getSysSession() if err != nil { return err } - return nil + e.sess = sess + err = sessiontxn.NewTxn(context.Background(), sess) + if err != nil { + return err + } + txn, err := sess.Txn(true) + if err != nil { + return err + } + sess.GetSessionVars().SetInTxn(true) + err = e.DDLJobRetriever.initial(txn, sess) + return err } // Next implements the Executor Next interface. @@ -706,6 +732,12 @@ func (e *ShowDDLJobsExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } +// Close implements the Executor Close interface. +func (e *ShowDDLJobsExec) Close() error { + e.releaseSysSession(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), e.sess) + return e.baseExecutor.Close() +} + func getSchemaName(is infoschema.InfoSchema, id int64) string { var schemaName string DBInfo, ok := is.SchemaByID(id) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index fc7d7d2310a35..b31a684ec30cb 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1271,6 +1271,7 @@ type DDLJobsReaderExec struct { cacheJobs []*model.Job is infoschema.InfoSchema + sess sessionctx.Context } // Open implements the Executor Next interface. @@ -1278,13 +1279,23 @@ func (e *DDLJobsReaderExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } - txn, err := e.ctx.Txn(true) + e.DDLJobRetriever.is = e.is + e.activeRoles = e.ctx.GetSessionVars().ActiveRoles + sess, err := e.getSysSession() if err != nil { return err } - e.DDLJobRetriever.is = e.is - e.activeRoles = e.ctx.GetSessionVars().ActiveRoles - err = e.DDLJobRetriever.initial(txn) + e.sess = sess + err = sessiontxn.NewTxn(context.Background(), sess) + if err != nil { + return err + } + txn, err := sess.Txn(true) + if err != nil { + return err + } + sess.GetSessionVars().SetInTxn(true) + err = e.DDLJobRetriever.initial(txn, sess) if err != nil { return err } @@ -1335,6 +1346,12 @@ func (e *DDLJobsReaderExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } +// Close implements the Executor Close interface. +func (e *DDLJobsReaderExec) Close() error { + e.releaseSysSession(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), e.sess) + return e.baseExecutor.Close() +} + func (e *memtableRetriever) setDataFromEngines() { var rows [][]types.Datum rows = append(rows, diff --git a/server/http_handler.go b/server/http_handler.go index 331f0206ef6dd..6c15a3c6b3ff9 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -917,8 +917,7 @@ func (h flashReplicaHandler) getDropOrTruncateTableTiflash(currentSchema infosch fn := func(jobs []*model.Job) (bool, error) { return executor.GetDropOrTruncateTableInfoFromJobs(jobs, gcSafePoint, dom, handleJobAndTableInfo) } - - err = ddl.IterAllDDLJobs(txn, fn) + err = ddl.IterAllDDLJobs(s, txn, fn) if err != nil { if terror.ErrorEqual(variable.ErrSnapshotTooOld, err) { // The err indicate that current ddl job and remain DDL jobs was been deleted by GC, From e541bbb3ae99e9998a26082a88f8b5b0398e5705 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 8 Jun 2022 17:49:37 +0800 Subject: [PATCH 06/21] 6. add metrics for concurrent ddl add metrics --- ddl/column.go | 4 +- ddl/ddl.go | 4 + ddl/index.go | 2 +- ddl/job_table.go | 3 + ddl/reorg.go | 8 +- metrics/ddl.go | 38 +++++++- metrics/grafana/tidb.json | 200 +++++++++++++++++++++++++++++++++++++- metrics/metrics.go | 2 + 8 files changed, 250 insertions(+), 11 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 2f7fd1f4cad3f..879c3b87c4314 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -701,7 +701,7 @@ func (w *worker) doModifyColumnTypeWithData( // Make sure job args change after `updateVersionAndTableInfoWithCheck`, otherwise, the job args will // be updated in `updateDDLJob` even if it meets an error in `updateVersionAndTableInfoWithCheck`. job.SchemaState = model.StateDeleteOnly - metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn).Set(0) + metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, job.SchemaName, tblInfo.Name.String()).Set(0) job.Args = append(job.Args, changingCol, changingIdxs, rmIdxIDs) case model.StateDeleteOnly: // Column from null to not null. @@ -1103,7 +1103,7 @@ func newUpdateColumnWorker(sessCtx sessionctx.Context, id int, t table.PhysicalT backfillWorker: newBackfillWorker(sessCtx, id, t, reorgInfo), oldColInfo: oldCol, newColInfo: newCol, - metricCounter: metrics.BackfillTotalCounter.WithLabelValues("update_col_rate"), + metricCounter: metrics.BackfillTotalCounter.WithLabelValues(metrics.GenerateReorgLabel("update_col_rate", reorgInfo.SchemaName, t.Meta().Name.String())), rowDecoder: rowDecoder, rowMap: make(map[int64]types.Datum, len(decodeColMap)), sqlMode: reorgInfo.ReorgMeta.SQLMode, diff --git a/ddl/ddl.go b/ddl/ddl.go index 824b1c3af2e01..f199fdd40c930 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1537,7 +1537,11 @@ func (s *session) reset() { } func (s *session) execute(ctx context.Context, query string, label string) ([]chunk.Row, error) { + startTime := time.Now() var err error + defer func() { + metrics.DDLJobTableDuration.WithLabelValues(label + "-" + metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + }() rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(ctx, query) if err != nil { return nil, errors.Trace(err) diff --git a/ddl/index.go b/ddl/index.go index 8a7faebba51e0..b1f9132bc3ba9 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -577,7 +577,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo return ver, err } job.SchemaState = model.StateDeleteOnly - metrics.GetBackfillProgressByLabel(metrics.LblAddIndex).Set(0) + metrics.GetBackfillProgressByLabel(metrics.LblAddIndex, job.SchemaName, tblInfo.Name.String()).Set(0) case model.StateDeleteOnly: // delete only -> write only indexInfo.State = model.StateWriteOnly diff --git a/ddl/job_table.go b/ddl/job_table.go index 168dee4945a96..8fc4e60b2556b 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -209,10 +210,12 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { injectFailPointForGetJob(job) d.insertRunningDDLJobMap(job.ID) d.wg.Run(func() { + metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Inc() defer func() { pool.put(wk) d.deleteRunningDDLJobMap(job.ID) asyncNotify(d.ddlJobCh) + metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Dec() }() // we should wait 2 * d.lease time to guarantee all TiDB server have finished the schema change. // see waitSchemaSynced for more details. diff --git a/ddl/reorg.go b/ddl/reorg.go index 778d48c604206..a4d0dcf033f2f 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -253,9 +253,9 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo switch reorgInfo.Type { case model.ActionAddIndex, model.ActionAddPrimaryKey: - metrics.GetBackfillProgressByLabel(metrics.LblAddIndex).Set(100) + metrics.GetBackfillProgressByLabel(metrics.LblAddIndex, job.SchemaName, tblInfo.Name.String()).Set(0) case model.ActionModifyColumn: - metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn).Set(100) + metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, job.SchemaName, tblInfo.Name.String()).Set(0) } if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { logutil.BgLogger().Warn("[ddl] run reorg job done, removeDDLReorgHandle failed", zap.Error(err1)) @@ -321,9 +321,9 @@ func updateBackfillProgress(w *worker, reorgInfo *reorgInfo, tblInfo *model.Tabl } switch reorgInfo.Type { case model.ActionAddIndex, model.ActionAddPrimaryKey: - metrics.GetBackfillProgressByLabel(metrics.LblAddIndex).Set(progress * 100) + metrics.GetBackfillProgressByLabel(metrics.LblAddIndex, reorgInfo.SchemaName, tblInfo.Name.String()).Set(progress * 100) case model.ActionModifyColumn: - metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn).Set(progress * 100) + metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, reorgInfo.SchemaName, tblInfo.Name.String()).Set(progress * 100) } } diff --git a/metrics/ddl.go b/metrics/ddl.go index 47e043519c9c5..5c3595231d143 100644 --- a/metrics/ddl.go +++ b/metrics/ddl.go @@ -14,7 +14,11 @@ package metrics -import "github.com/prometheus/client_golang/prometheus" +import ( + "strings" + + "github.com/prometheus/client_golang/prometheus" +) // Metrics for the DDL package. var ( @@ -123,6 +127,22 @@ var ( Name: "backfill_percentage_progress", Help: "Percentage progress of backfill", }, []string{LblType}) + + DDLJobTableDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "ddl", + Name: "job_table_duration_seconds", + Help: "Bucketed histogram of processing time (s) of the 3 DDL job tables", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s + }, []string{LblType}) + + DDLRunningJobCount = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "ddl", + Name: "running_job_count", + Help: "Running DDL jobs count", + }, []string{LblType}) ) // Label constants. @@ -133,7 +153,19 @@ const ( LblModifyColumn = "modify_column" ) +// GenerateReorgLabel returns the label with schema name and table name. +func GenerateReorgLabel(label string, schemaName string, tableName string) string { + var stringBuilder strings.Builder + stringBuilder.Grow(len(label) + len(schemaName) + len(tableName) + 2) + stringBuilder.WriteString(label) + stringBuilder.WriteString("_") + stringBuilder.WriteString(schemaName) + stringBuilder.WriteString("_") + stringBuilder.WriteString(tableName) + return stringBuilder.String() +} + // GetBackfillProgressByLabel returns the Gauge showing the percentage progress for the given type label. -func GetBackfillProgressByLabel(lbl string) prometheus.Gauge { - return BackfillProgressGauge.WithLabelValues(lbl) +func GetBackfillProgressByLabel(label string, schemaName string, tableName string) prometheus.Gauge { + return BackfillProgressGauge.WithLabelValues(GenerateReorgLabel(label, schemaName, tableName)) } diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index ad32c8280230b..63bb2b81471fa 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -10622,6 +10622,204 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "DDL job table duration", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 40 + }, + "hiddenSeries": false, + "id": 250, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(tidb_ddl_job_table_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", }[1m])) by (le, type))", + "format": "time_series", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-95", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "DDL job table duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:629", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:630", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "current count of the running DDL jobs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 40 + }, + "hiddenSeries": false, + "id": 251, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_ddl_running_job_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{ type }}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "DDL Running Job Cnt", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -12049,7 +12247,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_ddl_backfill_percentage_progress{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"add_index\"}", + "expr": "tidb_ddl_backfill_percentage_progress{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-{{type}}", diff --git a/metrics/metrics.go b/metrics/metrics.go index 8818c96b44801..ad962c975dae3 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -100,6 +100,8 @@ func RegisterMetrics() { prometheus.MustRegister(BackfillTotalCounter) prometheus.MustRegister(BackfillProgressGauge) prometheus.MustRegister(DDLWorkerHistogram) + prometheus.MustRegister(DDLJobTableDuration) + prometheus.MustRegister(DDLRunningJobCount) prometheus.MustRegister(DeploySyncerHistogram) prometheus.MustRegister(DistSQLPartialCountHistogram) prometheus.MustRegister(DistSQLCoprCacheCounter) From 322507318c983ac13649344a268612f4723fa14a Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 30 Jun 2022 12:34:49 +0800 Subject: [PATCH 07/21] 7. support multiple tables --- ddl/callback.go | 14 ++ ddl/callback_test.go | 20 +++ ddl/ddl.go | 11 +- ddl/ddl_api.go | 2 + ddl/ddl_worker.go | 6 + ddl/job_table.go | 70 +++++++++- ddl/job_table_test.go | 174 +++++++++++++++++++++++++ ddl/stat_test.go | 4 +- ddl/table_test.go | 1 + planner/core/exhaust_physical_plans.go | 6 +- session/session.go | 4 +- 11 files changed, 296 insertions(+), 16 deletions(-) create mode 100644 ddl/job_table_test.go diff --git a/ddl/callback.go b/ddl/callback.go index e2cf24df3553d..84b22cdfed944 100644 --- a/ddl/callback.go +++ b/ddl/callback.go @@ -55,6 +55,10 @@ type Callback interface { OnJobUpdated(job *model.Job) // OnWatched is called after watching owner is completed. OnWatched(ctx context.Context) + // OnGetJobBefore is called before getting job. + OnGetJobBefore(jobType string) + // OnGetJobAfter is called after getting job. + OnGetJobAfter(jobType string, job *model.Job) } // BaseCallback implements Callback.OnChanged interface. @@ -86,6 +90,16 @@ func (*BaseCallback) OnWatched(ctx context.Context) { // Nothing to do. } +// OnGetJobBefore implements Callback.OnGetJobBefore interface. +func (c *BaseCallback) OnGetJobBefore(jobType string) { + // Nothing to do. +} + +// OnGetJobAfter implements Callback.OnGetJobAfter interface. +func (c *BaseCallback) OnGetJobAfter(jobType string, job *model.Job) { + // Nothing to do. +} + // DomainReloader is used to avoid import loop. type DomainReloader interface { Reload() error diff --git a/ddl/callback_test.go b/ddl/callback_test.go index c8975da252cad..85e26209fff7c 100644 --- a/ddl/callback_test.go +++ b/ddl/callback_test.go @@ -52,6 +52,8 @@ type TestDDLCallback struct { onJobUpdated func(*model.Job) OnJobUpdatedExported func(*model.Job) onWatched func(ctx context.Context) + OnGetJobBeforeExported func(string) + OnGetJobAfterExported func(string, *model.Job) } // OnChanged mock the same behavior with the main DDL hook. @@ -118,6 +120,24 @@ func (tc *TestDDLCallback) OnWatched(ctx context.Context) { tc.BaseCallback.OnWatched(ctx) } +// OnGetJobBefore implements Callback.OnGetJobBefore interface. +func (tc *TestDDLCallback) OnGetJobBefore(jobType string) { + if tc.OnGetJobBeforeExported != nil { + tc.OnGetJobBeforeExported(jobType) + return + } + tc.BaseCallback.OnGetJobBefore(jobType) +} + +// OnGetJobAfter implements Callback.OnGetJobAfter interface. +func (tc *TestDDLCallback) OnGetJobAfter(jobType string, job *model.Job) { + if tc.OnGetJobAfterExported != nil { + tc.OnGetJobAfterExported(jobType, job) + return + } + tc.BaseCallback.OnGetJobAfter(jobType, job) +} + func TestCallback(t *testing.T) { cb := &BaseCallback{} require.Nil(t, cb.OnChanged(nil)) diff --git a/ddl/ddl.go b/ddl/ddl.go index f199fdd40c930..e1669067ba367 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1419,7 +1419,7 @@ func getDDLJobsInQueue(t *meta.Meta, jobListKey meta.JobListKeyType) ([]*model.J // GetAllDDLJobs get all DDL jobs and sorts jobs by job.ID. func GetAllDDLJobs(sess sessionctx.Context, t *meta.Meta) ([]*model.Job, error) { if variable.EnableConcurrentDDL.Load() { - return getJobsBySQL(newSession(sess), "tidb_ddl_job", "1 order by job_id") + return getJobsBySQL(newSession(sess), JobTable, "1 order by job_id") } return getDDLJobs(t) @@ -1542,7 +1542,7 @@ func (s *session) execute(ctx context.Context, query string, label string) ([]ch defer func() { metrics.DDLJobTableDuration.WithLabelValues(label + "-" + metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() - rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(ctx, query) + rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(kv.WithInternalSourceType(ctx, kv.InternalTxnDDL), query) if err != nil { return nil, errors.Trace(err) } @@ -1625,8 +1625,11 @@ func addHistoryDDLJob2Table(sess *session, job *model.Job, updateRawArgs bool) e return err } _, err = sess.execute(context.Background(), - fmt.Sprintf("insert ignore into mysql.tidb_ddl_history(job_id, job_meta, db_name, table_name, schema_id, table_id, create_time) values (%d, %s, %s, %s, %d, %d, %v)", - job.ID, wrapKey2String(b), strconv.Quote(job.SchemaName), strconv.Quote(job.TableName), job.SchemaID, job.TableID, strconv.Quote(model.TSConvert2Time(job.StartTS).String())), + fmt.Sprintf("insert ignore into mysql.tidb_ddl_history(job_id, job_meta, db_name, table_name, schema_ids, table_ids, create_time) values (%d, %s, %s, %s, %s, %s, %v)", + job.ID, wrapKey2String(b), strconv.Quote(job.SchemaName), strconv.Quote(job.TableName), + strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), + strconv.Quote(strconv.FormatInt(job.TableID, 10)), + strconv.Quote(model.TSConvert2Time(job.StartTS).String())), "insert_history") return errors.Trace(err) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index dd2a63215d2ee..e8c257ec10393 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3958,6 +3958,7 @@ func (d *ddl) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, sp Type: model.ActionExchangeTablePartition, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{defID, ptSchema.ID, ptMeta.ID, partName, spec.WithValidation}, + CtxVars: []interface{}{[]int64{ntSchema.ID, ptSchema.ID}, []int64{ntMeta.ID, ptMeta.ID}}, } err = d.DoDDLJob(ctx, job) @@ -5515,6 +5516,7 @@ func (d *ddl) renameTables(ctx sessionctx.Context, oldIdents, newIdents []ast.Id Type: model.ActionRenameTables, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{oldSchemaIDs, newSchemaIDs, tableNames, tableIDs, oldSchemaNames, oldTableNames}, + CtxVars: []interface{}{append(oldSchemaIDs, newSchemaIDs...), tableIDs}, } err = d.DoDDLJob(ctx, job) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 16f07d0d08a3d..c581932549ae7 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -17,6 +17,7 @@ package ddl import ( "context" "fmt" + "math/rand" "strconv" "sync" "sync/atomic" @@ -693,6 +694,11 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { if err != nil { return err } + failpoint.Inject("mockRunJobTime", func(val failpoint.Value) { + if val.(bool) { + time.Sleep(time.Duration(rand.Intn(5)) * time.Second) // #nosec G404 + } + }) txn, err := w.sess.txn() if err != nil { w.sess.rollback() diff --git a/ddl/job_table.go b/ddl/job_table.go index 8fc4e60b2556b..df4f401e030cc 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" + "golang.org/x/exp/slices" ) var ( @@ -66,7 +67,7 @@ func (dc *ddlCtx) excludeJobIDs() string { } const ( - getJobSQL = "select job_meta from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_id, table_id) and %s reorg %s" + getJobSQL = "select job_meta, processing from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids) and %s reorg %s order by processing desc, job_id" ) type jobType int @@ -105,11 +106,18 @@ func (d *ddl) getJob(sess *session, tp jobType, filter func(*model.Job) (bool, e if err != nil { return nil, errors.Trace(err) } + if row.GetInt64(1) == 1 { + return &runJob, nil + } b, err := filter(&runJob) if err != nil { return nil, errors.Trace(err) } if b { + if err := d.markJobProcessing(sess, &runJob); err != nil { + logutil.BgLogger().Warn("[ddl] handle ddl job failed: mark job is processing meet error", zap.Error(err), zap.String("job", runJob.String())) + return nil, errors.Trace(err) + } return &runJob, nil } } @@ -119,10 +127,11 @@ func (d *ddl) getJob(sess *session, tp jobType, filter func(*model.Job) (bool, e func (d *ddl) getGeneralJob(sess *session) (*model.Job, error) { return d.getJob(sess, general, func(job *model.Job) (bool, error) { if job.Type == model.ActionDropSchema { - sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where schema_id = %d and job_id < %d limit 1", job.SchemaID, job.ID) + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where find_in_set(%s, schema_ids) != 0 and processing limit 1", strconv.Quote(strconv.FormatInt(job.SchemaID, 10))) return d.checkJobIsRunnable(sess, sql) } - return true, nil + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job t1, (select table_ids from mysql.tidb_ddl_job where job_id = %d) t2 where processing and find_in_set(t1.table_ids, t2.table_ids) != 0", job.ID) + return d.checkJobIsRunnable(sess, sql) }) } @@ -133,7 +142,8 @@ func (d *ddl) checkJobIsRunnable(sess *session, sql string) (bool, error) { func (d *ddl) getReorgJob(sess *session) (*model.Job, error) { return d.getJob(sess, reorg, func(job *model.Job) (bool, error) { - sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where schema_id = %d and is_drop_schema and job_id < %d limit 1", job.SchemaID, job.ID) + sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where (find_in_set(%s, schema_ids) != 0 and type = %d and processing) or (find_in_set(%s, table_ids) != 0 and processing) limit 1", + strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), model.ActionDropSchema, strconv.Quote(strconv.FormatInt(job.TableID, 10))) return d.checkJobIsRunnable(sess, sql) }) } @@ -194,6 +204,10 @@ func (d *ddl) getDDLJobAndRun(sess *session, pool *workerPool, getJob func(*sess return } + d.mu.RLock() + d.mu.hook.OnGetJobBefore(pool.tp().String()) + d.mu.RUnlock() + job, err := getJob(sess) if job == nil || err != nil { if err != nil { @@ -202,6 +216,9 @@ func (d *ddl) getDDLJobAndRun(sess *session, pool *workerPool, getJob func(*sess pool.put(wk) return } + d.mu.RLock() + d.mu.hook.OnGetJobAfter(pool.tp().String(), job) + d.mu.RUnlock() d.delivery2worker(wk, pool, job) } @@ -229,8 +246,14 @@ func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) { }) } +func (d *ddl) markJobProcessing(sess *session, job *model.Job) error { + sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + _, err := sess.execute(context.Background(), fmt.Sprintf("update mysql.tidb_ddl_job set processing = 1 where job_id = %d", job.ID), "mark_job_processing") + return errors.Trace(err) +} + const ( - addDDLJobSQL = "insert into mysql.tidb_ddl_job(job_id, reorg, schema_id, table_id, job_meta, is_drop_schema) values" + addDDLJobSQL = "insert into mysql.tidb_ddl_job(job_id, reorg, schema_ids, table_ids, job_meta, type, processing) values" updateDDLJobSQL = "update mysql.tidb_ddl_job set job_meta = %s where job_id = %d" ) @@ -253,7 +276,7 @@ func insertDDLJobs2Table(sess *session, jobs []*model.Job, updateRawArgs bool) e if i != 0 { sql.WriteString(",") } - sql.WriteString(fmt.Sprintf("(%d, %t, %d, %d, %s, %t)", job.ID, job.MayNeedReorg(), job.SchemaID, job.TableID, wrapKey2String(b), job.Type == model.ActionDropSchema)) + sql.WriteString(fmt.Sprintf("(%d, %t, %s, %s, %s, %d, %t)", job.ID, job.MayNeedReorg(), strconv.Quote(job2SchemaIDs(job)), strconv.Quote(job2TableIDs(job)), wrapKey2String(b), job.Type, !job.NotStarted())) } sess.SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) @@ -262,6 +285,41 @@ func insertDDLJobs2Table(sess *session, jobs []*model.Job, updateRawArgs bool) e return errors.Trace(err) } +func job2SchemaIDs(job *model.Job) string { + return job2UniqueIDs(job, true) +} + +func job2TableIDs(job *model.Job) string { + return job2UniqueIDs(job, false) +} + +func job2UniqueIDs(job *model.Job, schema bool) string { + switch job.Type { + case model.ActionExchangeTablePartition, model.ActionRenameTables: + var ids []int64 + if schema { + ids = job.CtxVars[0].([]int64) + } else { + ids = job.CtxVars[1].([]int64) + } + set := make(map[int64]struct{}, len(ids)) + for _, id := range ids { + set[id] = struct{}{} + } + + s := make([]string, 0, len(set)) + for id := range set { + s = append(s, strconv.FormatInt(id, 10)) + } + slices.Sort(s) + return strings.Join(s, ",") + } + if schema { + return strconv.FormatInt(job.SchemaID, 10) + } + return strconv.FormatInt(job.TableID, 10) +} + func (w *worker) deleteDDLJob(job *model.Job) error { sql := fmt.Sprintf("delete from mysql.tidb_ddl_job where job_id = %d", job.ID) _, err := w.sess.execute(context.Background(), sql, "delete_job") diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go new file mode 100644 index 0000000000000..905efb17c1f16 --- /dev/null +++ b/ddl/job_table_test.go @@ -0,0 +1,174 @@ +// Copyright 2022 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 ddl_test + +import ( + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util" + "github.com/stretchr/testify/require" + "golang.org/x/exp/slices" +) + +func TestDDLScheduling(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE e (id INT NOT NULL) PARTITION BY RANGE (id) (PARTITION p1 VALUES LESS THAN (50), PARTITION p2 VALUES LESS THAN (100));") + tk.MustExec("CREATE TABLE e2 (id INT NOT NULL);") + tk.MustExec("CREATE TABLE e3 (id INT NOT NULL);") + + d := dom.DDL() + + once := true + + ddlJobs := []string{ + "alter table e2 add index idx(id)", + "alter table e2 add index idx1(id)", + "alter table e2 add index idx2(id)", + "create table e5 (id int)", + "ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2;", + "alter table e add index idx(id)", + "alter table e add partition (partition p3 values less than (150))", + "create table e4 (id int)", + "alter table e3 add index idx1(id)", + "ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e3;", + } + + hook := &ddl.TestDDLCallback{} + var wg util.WaitGroupWrapper + wg.Add(1) + hook.OnGetJobBeforeExported = func(jobType string) { + if once { + once = false + for i, job := range ddlJobs { + wg.Run(func() { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_exchange_partition=1") + recordSet, _ := tk.Exec(job) + if recordSet != nil { + require.NoError(t, recordSet.Close()) + } + }) + for { + time.Sleep(time.Millisecond * 100) + jobs, err := ddl.GetAllDDLJobs(testkit.NewTestKit(t, store).Session(), nil) + require.NoError(t, err) + if len(jobs) == i+1 { + break + } + } + } + wg.Done() + } + } + + record := make([]int64, 0, 16) + hook.OnGetJobAfterExported = func(jobType string, job *model.Job) { + // record the job schedule order + record = append(record, job.ID) + } + + err := failpoint.Enable("github.com/pingcap/tidb/ddl/mockRunJobTime", `return(true)`) + require.NoError(t, err) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/ddl/mockRunJobTime") + require.NoError(t, err) + }() + + d.SetHook(hook) + wg.Wait() + + // sort all the job id. + ids := make(map[int64]struct{}, 16) + for _, id := range record { + ids[id] = struct{}{} + } + + sortedIDs := make([]int64, 0, 16) + for id := range ids { + sortedIDs = append(sortedIDs, id) + } + slices.Sort(sortedIDs) + + // map the job id to the DDL sequence. + // sortedIDs may looks like [30, 32, 34, 36, ...], it is the same order with the job in `ddlJobs`, 30 is the first job in `ddlJobs`, 32 is second... + // record may looks like [30, 30, 32, 32, 34, 32, 36, 34, ...] + // and the we map the record to the DDL sequence, [0, 0, 1, 1, 2, 1, 3, 2, ...] + for i := range record { + idx, b := slices.BinarySearch(sortedIDs, record[i]) + require.True(t, b) + record[i] = int64(idx) + } + + check(t, record, 0, 1, 2) + check(t, record, 0, 4) + check(t, record, 1, 4) + check(t, record, 2, 4) + check(t, record, 4, 5) + check(t, record, 4, 6) + check(t, record, 4, 9) + check(t, record, 5, 6) + check(t, record, 5, 9) + check(t, record, 6, 9) + check(t, record, 8, 9) +} + +// check will check if there are any cross between ids. +// e.g. if ids is [1, 2] this function checks all `1` is before or after than `2` in record. +func check(t *testing.T, record []int64, ids ...int64) { + // have return true if there are any `i` is before `j`, false if there are any `j` is before `i`. + have := func(i, j int64) bool { + for _, id := range record { + if id == i { + return true + } + if id == j { + return false + } + } + require.FailNow(t, "should not reach here") + return false + } + + // all checks if all `i` is before `j`. + all := func(i, j int64) { + meet := false + for _, id := range record { + if id == j { + meet = true + } + require.False(t, meet && id == i) + } + } + + for i := 0; i < len(ids)-1; i++ { + for j := i + 1; j < len(ids); j++ { + if have(ids[i], ids[j]) { + all(ids[i], ids[j]) + } else { + all(ids[j], ids[i]) + } + } + } +} diff --git a/ddl/stat_test.go b/ddl/stat_test.go index c95b882443857..d71d010e0d6ad 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -17,6 +17,7 @@ package ddl_test import ( "context" "fmt" + "strconv" "testing" "github.com/pingcap/failpoint" @@ -148,7 +149,8 @@ func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error if err != nil { return err } - _, err = sess.Execute(context.Background(), fmt.Sprintf("insert into mysql.tidb_ddl_job values (%d, %t, %d, %d, %s, %t)", job.ID, job.MayNeedReorg(), job.SchemaID, job.TableID, wrapKey2String(b), job.Type == model.ActionDropSchema)) + _, err = sess.Execute(context.Background(), fmt.Sprintf("insert into mysql.tidb_ddl_job(job_id, reorg, schema_ids, table_ids, job_meta, type, processing) values (%d, %t, %s, %s, %s, %d, %t)", + job.ID, job.MayNeedReorg(), strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), strconv.Quote(strconv.FormatInt(job.TableID, 10)), wrapKey2String(b), job.Type, false)) return err } m := meta.NewMeta(txn) diff --git a/ddl/table_test.go b/ddl/table_test.go index 0a1dff690f5b2..83a7c49ce5553 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -63,6 +63,7 @@ func testRenameTables(t *testing.T, ctx sessionctx.Context, d ddl.DDL, oldSchema Type: model.ActionRenameTables, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{oldSchemaIDs, newSchemaIDs, newTableNames, oldTableIDs, oldSchemaNames, oldTableNames}, + CtxVars: []interface{}{append(oldSchemaIDs, newSchemaIDs...), oldTableIDs}, } ctx.SetValue(sessionctx.QueryString, "skip") require.NoError(t, d.DoDDLJob(ctx, job)) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index be439dbd946df..ff5265e2f8da7 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -822,7 +822,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( } joins = make([]PhysicalPlan, 0, 3) failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) { + if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { failpoint.Return(p.constructIndexHashJoin(prop, outerIdx, innerTask, nil, keyOff2IdxOff, path, lastColMng)) } }) @@ -857,7 +857,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( } innerTask := p.constructInnerIndexScanTask(ds, helper.chosenPath, helper.chosenRanges.Range(), helper.chosenRemained, outerJoinKeys, us, rangeInfo, false, false, avgInnerRowCnt, maxOneRow) failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) { + if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { failpoint.Return(p.constructIndexHashJoin(prop, outerIdx, innerTask, helper.chosenRanges, keyOff2IdxOff, helper.chosenPath, helper.lastColManager)) } }) @@ -1785,7 +1785,7 @@ func (p *LogicalJoin) shouldUseMPPBCJ() bool { // If the hint is not figured, we will pick all candidates. func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) { + if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { indexJoins, _ := p.tryToGetIndexJoin(prop) failpoint.Return(indexJoins, true, nil) } diff --git a/session/session.go b/session/session.go index 9e14c2a52e706..5560789d0b45a 100644 --- a/session/session.go +++ b/session/session.go @@ -2708,9 +2708,9 @@ var ( errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) // DDLJobTables is a list of tables definitions used in concurrent DDL. DDLJobTables = []string{ - "create table tidb_ddl_job(job_id bigint not null, reorg int, schema_id bigint, table_id bigint, job_meta longblob, is_drop_schema int, primary key(job_id))", + "create table tidb_ddl_job(job_id bigint not null, reorg int, schema_ids text(65535), table_ids text(65535), job_meta longblob, type int, processing int, primary key(job_id))", "create table tidb_ddl_reorg(job_id bigint not null, ele_id bigint, ele_type blob, start_key blob, end_key blob, physical_id bigint, reorg_meta longblob, unique key(job_id, ele_id, ele_type(20)))", - "create table tidb_ddl_history(job_id bigint not null, job_meta longblob, db_name char(64), table_name char(64), schema_id bigint, table_id bigint, create_time datetime, primary key(job_id))", + "create table tidb_ddl_history(job_id bigint not null, job_meta longblob, db_name char(64), table_name char(64), schema_ids text(65535), table_ids text(65535), create_time datetime, primary key(job_id))", } ) From f91dedbf5dc35d9288ee558dd6dba373db82bdbf Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 7 Jul 2022 13:58:54 +0800 Subject: [PATCH 08/21] 8. fix test --- ddl/db_test.go | 2 + ddl/ddl.go | 10 ++++ ddl/ddl_api_test.go | 25 +++++----- ddl/ddl_test.go | 13 +++++ ddl/ddl_worker_test.go | 69 ++++++++++++++------------- ddl/index.go | 2 +- ddl/job_table_test.go | 24 +++++++--- ddl/multi_schema_change_test.go | 16 +++---- ddl/restart_test.go | 3 +- ddl/stat_test.go | 2 +- domain/domain_test.go | 2 + executor/executor_test.go | 2 +- expression/integration_serial_test.go | 1 + server/tidb_test.go | 18 +++---- session/bootstrap_test.go | 3 +- session/session_test.go | 4 +- tests/realtikvtest/testkit.go | 2 +- 17 files changed, 120 insertions(+), 78 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index add7fee199b00..67d97721441ca 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1274,6 +1274,8 @@ func TestCancelJobWriteConflict(t *testing.T) { stmt := fmt.Sprintf("admin cancel ddl jobs %d", job.ID) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn", `return("no_retry")`)) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn")) }() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/mockCancelConcurencyDDL", `return(true)`)) + defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/mockCancelConcurencyDDL")) }() rs, cancelErr = tk2.Session().Execute(context.Background(), stmt) } } diff --git a/ddl/ddl.go b/ddl/ddl.go index e1669067ba367..a65dadd266340 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -602,6 +602,11 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { } d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(addIdxWorker), reorgWorkerCnt, reorgWorkerCnt, 0), reorg) d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(generalWorker), generalWorkerCnt, generalWorkerCnt, 0), general) + failpoint.Inject("NoDDLDispatchLoop", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return() + } + }) d.wg.Run(d.startDispatchLoop) } @@ -1605,6 +1610,11 @@ func GetHistoryJobByID(sess sessionctx.Context, id int64) (*model.Job, error) { return job, errors.Trace(err) } +// AddHistoryDDLJobForTest used for test. +func AddHistoryDDLJobForTest(sess sessionctx.Context, t *meta.Meta, job *model.Job, updateRawArgs bool) error { + return AddHistoryDDLJob(newSession(sess), t, job, updateRawArgs, variable.EnableConcurrentDDL.Load()) +} + // AddHistoryDDLJob record the history job. func AddHistoryDDLJob(sess *session, t *meta.Meta, job *model.Job, updateRawArgs bool, concurrentDDL bool) error { if concurrentDDL { diff --git a/ddl/ddl_api_test.go b/ddl/ddl_api_test.go index 90f0291ae2b35..b23692445659e 100644 --- a/ddl/ddl_api_test.go +++ b/ddl/ddl_api_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" "golang.org/x/exp/slices" @@ -47,15 +48,15 @@ func TestGetDDLJobs(t *testing.T) { SchemaID: 1, Type: model.ActionCreateTable, } - err := addDDLJobs(txn, jobs[i]) + err := addDDLJobs(sess, txn, jobs[i]) require.NoError(t, err) - currJobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + currJobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) require.Len(t, currJobs, i+1) currJobs2 = currJobs2[:0] - err = ddl.IterAllDDLJobs(txn, func(jobs []*model.Job) (b bool, e error) { + err = ddl.IterAllDDLJobs(sess, txn, func(jobs []*model.Job) (b bool, e error) { for _, job := range jobs { if job.NotStarted() { currJobs2 = append(currJobs2, job) @@ -69,7 +70,7 @@ func TestGetDDLJobs(t *testing.T) { require.Len(t, currJobs2, i+1) } - currJobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + currJobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) for i, job := range jobs { @@ -95,15 +96,15 @@ func TestGetDDLJobsIsSort(t *testing.T) { require.NoError(t, err) // insert 5 drop table jobs to DefaultJobListKey queue - enQueueDDLJobs(t, txn, model.ActionDropTable, 10, 15) + enQueueDDLJobs(t, sess, txn, model.ActionDropTable, 10, 15) // insert 5 create table jobs to DefaultJobListKey queue - enQueueDDLJobs(t, txn, model.ActionCreateTable, 0, 5) + enQueueDDLJobs(t, sess, txn, model.ActionCreateTable, 0, 5) // insert add index jobs to AddIndexJobListKey queue - enQueueDDLJobs(t, txn, model.ActionAddIndex, 5, 10) + enQueueDDLJobs(t, sess, txn, model.ActionAddIndex, 5, 10) - currJobs, err := ddl.GetAllDDLJobs(meta.NewMeta(txn)) + currJobs, err := ddl.GetAllDDLJobs(sess, meta.NewMeta(txn)) require.NoError(t, err) require.Len(t, currJobs, 15) @@ -124,8 +125,8 @@ func TestGetHistoryDDLJobs(t *testing.T) { err := kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, false, func(ctx context.Context, txn kv.Transaction) error { return meta.NewMeta(txn).ClearAllHistoryJob() }) - require.NoError(t, err) + testkit.NewTestKit(t, store).MustExec("delete from mysql.tidb_ddl_history") tk := testkit.NewTestKit(t, store) sess := tk.Session() @@ -143,7 +144,7 @@ func TestGetHistoryDDLJobs(t *testing.T) { SchemaID: 1, Type: model.ActionCreateTable, } - err = ddl.AddHistoryDDLJob(m, jobs[i], true) + err = ddl.AddHistoryDDLJobForTest(sess, m, jobs[i], true) require.NoError(t, err) historyJobs, err := ddl.GetLastNHistoryDDLJobs(m, ddl.DefNumHistoryJobs) @@ -204,14 +205,14 @@ func TestIsJobRollbackable(t *testing.T) { } } -func enQueueDDLJobs(t *testing.T, txn kv.Transaction, jobType model.ActionType, start, end int) { +func enQueueDDLJobs(t *testing.T, sess session.Session, txn kv.Transaction, jobType model.ActionType, start, end int) { for i := start; i < end; i++ { job := &model.Job{ ID: int64(i), SchemaID: 1, Type: jobType, } - err := addDDLJobs(txn, job) + err := addDDLJobs(sess, txn, job) require.NoError(t, err) } } diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 54eb13b955629..b0fedd991639b 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -273,6 +274,9 @@ func TestNotifyDDLJob(t *testing.T) { require.NoError(t, store.Close()) }() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NoDDLDispatchLoop", `return(true)`)) + defer require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/NoDDLDispatchLoop")) + getFirstNotificationAfterStartDDL := func(d *ddl) { select { case <-d.workers[addIdxWorker].ddlJobCh: @@ -284,6 +288,11 @@ func TestNotifyDDLJob(t *testing.T) { default: // The notification may be received by the worker. } + + select { + case <-d.ddlJobCh: + default: + } } d, err := testNewDDLAndStart( @@ -314,6 +323,7 @@ func TestNotifyDDLJob(t *testing.T) { d.asyncNotifyWorker(job) select { case <-d.workers[generalWorker].ddlJobCh: + case <-d.ddlJobCh: default: require.FailNow(t, "do not get the general job notification") } @@ -323,6 +333,7 @@ func TestNotifyDDLJob(t *testing.T) { d.asyncNotifyWorker(job) select { case <-d.workers[addIdxWorker].ddlJobCh: + case <-d.ddlJobCh: default: require.FailNow(t, "do not get the add index job notification") } @@ -354,6 +365,8 @@ func TestNotifyDDLJob(t *testing.T) { require.FailNow(t, "should not get the add index job notification") case <-d1.workers[generalWorker].ddlJobCh: require.FailNow(t, "should not get the general job notification") + case <-d1.ddlJobCh: + require.FailNow(t, "should not get the job notification") default: } } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 6c21be950aead..01a11dd1da9eb 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -15,7 +15,6 @@ package ddl_test import ( - "context" "strconv" "sync" "testing" @@ -24,7 +23,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" @@ -33,7 +31,7 @@ import ( "github.com/stretchr/testify/require" ) -const testLease = 5 * time.Millisecond +const testLease = 5 * time.Second func TestCheckOwner(t *testing.T) { _, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease) @@ -102,30 +100,29 @@ func TestParallelDDL(t *testing.T) { tk.MustExec("create table test_parallel_ddl_2.t3(c1 int, c2 int, c3 int, c4 int)") // set hook to execute jobs after all jobs are in queue. - jobCnt := int64(11) + jobCnt := 11 tc := &ddl.TestDDLCallback{Do: dom} once := sync.Once{} var checkErr error tc.OnJobRunBeforeExported = func(job *model.Job) { // TODO: extract a unified function for other tests. once.Do(func() { - qLen1 := int64(0) - qLen2 := int64(0) - var err error for { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - checkErr = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { - m := meta.NewMeta(txn) - qLen1, err = m.DDLJobQueueLen() - if err != nil { - return err + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("begin") + txn, err := tk1.Session().Txn(true) + require.NoError(t, err) + jobs, err := ddl.GetAllDDLJobs(tk1.Session(), meta.NewMeta(txn)) + require.NoError(t, err) + tk1.MustExec("rollback") + var qLen1, qLen2 int + for _, job := range jobs { + if !job.MayNeedReorg() { + qLen1++ + } else { + qLen2++ } - qLen2, err = m.DDLJobQueueLen(meta.AddIndexJobListKey) - if err != nil { - return err - } - return nil - }) + } if checkErr != nil { break } @@ -139,6 +136,25 @@ func TestParallelDDL(t *testing.T) { } }) } + + once1 := sync.Once{} + tc.OnGetJobBeforeExported = func(string) { + once1.Do(func() { + for { + tk := testkit.NewTestKit(t, store) + tk.MustExec("begin") + txn, err := tk.Session().Txn(true) + require.NoError(t, err) + jobs, err := ddl.GetAllDDLJobs(tk.Session(), meta.NewMeta(txn)) + require.NoError(t, err) + tk.MustExec("rollback") + if len(jobs) == jobCnt { + break + } + time.Sleep(time.Millisecond * 20) + } + }) + } dom.DDL().SetHook(tc) /* @@ -232,8 +248,7 @@ func TestParallelDDL(t *testing.T) { time.Sleep(5 * time.Millisecond) wg.Run(func() { tk := testkit.NewTestKit(t, store) - err := tk.ExecToErr("alter table test_parallel_ddl_2.t3 add index db3_idx1(c2)") - require.Error(t, err) + _ = tk.ExecToErr("alter table test_parallel_ddl_2.t3 add index db3_idx1(c2)") rs := tk.MustQuery("select json_extract(@@tidb_last_ddl_info, '$.seq_num')") seqIDs[10], _ = strconv.Atoi(rs.Rows()[0][0].(string)) }) @@ -253,18 +268,6 @@ func TestParallelDDL(t *testing.T) { require.Less(t, seqIDs[6], seqIDs[7]) require.Less(t, seqIDs[7], seqIDs[9]) require.Less(t, seqIDs[9], seqIDs[10]) - - // General job order. - require.Less(t, seqIDs[1], seqIDs[3]) - require.Less(t, seqIDs[3], seqIDs[4]) - require.Less(t, seqIDs[4], seqIDs[6]) - require.Less(t, seqIDs[6], seqIDs[7]) - require.Less(t, seqIDs[7], seqIDs[9]) - - // Reorg job order. - require.Less(t, seqIDs[2], seqIDs[5]) - require.Less(t, seqIDs[5], seqIDs[8]) - require.Less(t, seqIDs[8], seqIDs[10]) } func TestJobNeedGC(t *testing.T) { diff --git a/ddl/index.go b/ddl/index.go index b1f9132bc3ba9..0a3e9fe1b7814 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1352,7 +1352,7 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo ts := oracle.GoTimeToTS(time.Now()) s := reorg.d.store.(tikv.Storage) s.UpdateSPCache(ts, time.Now()) - time.Sleep(time.Millisecond * 3) + time.Sleep(time.Second * 3) } }) currentVer, err := getValidCurrentVersion(reorg.d.store) diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index 905efb17c1f16..c47dc5cc4a7c5 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -15,19 +15,33 @@ package ddl_test import ( + "sync" "testing" "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" "golang.org/x/exp/slices" ) -func TestDDLScheduling(t *testing.T) { +func TestDDLSchedulingMultiTimes(t *testing.T) { + if !variable.EnableConcurrentDDL.Load() { + t.Skipf("test requires concurrent ddl") + } + for i := 0; i < 3; i++ { + testDDLScheduling(t) + } +} + +// testDDLScheduling tests the DDL scheduling. See Concurrent DDL RFC for the rules of DDL scheduling. +// This test checks the chosen job records to see if there are wrong scheduling, if job A and job B cannot run concurrently, +// then the all the record of job A must before or after job B, no cross record between these 2 jobs should be in between. +func testDDLScheduling(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -39,8 +53,6 @@ func TestDDLScheduling(t *testing.T) { d := dom.DDL() - once := true - ddlJobs := []string{ "alter table e2 add index idx(id)", "alter table e2 add index idx1(id)", @@ -57,9 +69,9 @@ func TestDDLScheduling(t *testing.T) { hook := &ddl.TestDDLCallback{} var wg util.WaitGroupWrapper wg.Add(1) + var once sync.Once hook.OnGetJobBeforeExported = func(jobType string) { - if once { - once = false + once.Do(func() { for i, job := range ddlJobs { wg.Run(func() { tk := testkit.NewTestKit(t, store) @@ -80,7 +92,7 @@ func TestDDLScheduling(t *testing.T) { } } wg.Done() - } + }) } record := make([]int64, 0, 16) diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index d6a13b39aff74..4c65431a426e8 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -361,7 +361,7 @@ func TestMultiSchemaChangeRenameColumns(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int default 1, b int default 2)") tk.MustExec("insert into t values ()") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'c' is in write-reorg. return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization }) @@ -430,7 +430,7 @@ func TestMultiSchemaChangeAlterColumns(t *testing.T) { // Test cancel job with alter columns tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int default 1, b int default 2)") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'a' is in write-reorg. return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization }) @@ -498,7 +498,7 @@ func TestMultiSchemaChangeChangeColumns(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int default 1, b int default 2)") tk.MustExec("insert into t values ()") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'c' is in write-reorg. return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization }) @@ -627,7 +627,7 @@ func TestMultiSchemaChangeDropIndexesCancelled(t *testing.T) { // Test for cancelling the job in a middle state. tk.MustExec("create table t (a int, b int, index(a), unique index(b), index idx(a, b));") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StateDeleteOnly }) dom.DDL().SetHook(hook) @@ -641,7 +641,7 @@ func TestMultiSchemaChangeDropIndexesCancelled(t *testing.T) { // Test for cancelling the job in none state. tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int, index(a), unique index(b), index idx(a, b));") - hook = newCancelJobHook(store, dom, func(job *model.Job) bool { + hook = newCancelJobHook(t, store, dom, func(job *model.Job) bool { return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StatePublic }) dom.DDL().SetHook(hook) @@ -737,7 +737,7 @@ func TestMultiSchemaChangeRenameIndexes(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int default 1, b int default 2, index t(a))") tk.MustExec("insert into t values ()") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { // Cancel job when the column 'c' is in write-reorg. return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization }) @@ -887,7 +887,7 @@ func TestMultiSchemaChangeModifyColumnsCancelled(t *testing.T) { // Test for cancelling the job in a middle state. tk.MustExec("create table t (a int, b int, c int, index i1(a), unique index i2(b), index i3(a, b));") tk.MustExec("insert into t values (1, 2, 3);") - hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + hook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { return job.MultiSchemaInfo.SubJobs[2].SchemaState == model.StateWriteReorganization }) dom.DDL().SetHook(hook) @@ -986,7 +986,7 @@ func TestMultiSchemaChangeMixCancelled(t *testing.T) { tk.MustExec("create table t (a int, b int, c int, index i1(c), index i2(c));") tk.MustExec("insert into t values (1, 2, 3);") origin := dom.DDL().GetHook() - cancelHook := newCancelJobHook(store, dom, func(job *model.Job) bool { + cancelHook := newCancelJobHook(t, store, dom, func(job *model.Job) bool { return job.MultiSchemaInfo != nil && len(job.MultiSchemaInfo.SubJobs) > 8 && job.MultiSchemaInfo.SubJobs[8].SchemaState == model.StateWriteReorganization diff --git a/ddl/restart_test.go b/ddl/restart_test.go index 10d35802f5d2c..293ce76d50642 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" @@ -68,7 +67,7 @@ func runInterruptedJob(t *testing.T, store kv.Storage, d ddl.DDL, job *model.Job endlessLoopTime := time.Now().Add(time.Minute) for history == nil { // imitate DoDDLJob's logic, quit only find history - history, _ = ddl.GetHistoryJobByID(ctx, job.ID) + history, _ = ddl.GetHistoryJobByID(testkit.NewTestKit(t, store).Session(), job.ID) if history != nil { err = history.Error } diff --git a/ddl/stat_test.go b/ddl/stat_test.go index d71d010e0d6ad..3c298ea4ba435 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -149,7 +149,7 @@ func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error if err != nil { return err } - _, err = sess.Execute(context.Background(), fmt.Sprintf("insert into mysql.tidb_ddl_job(job_id, reorg, schema_ids, table_ids, job_meta, type, processing) values (%d, %t, %s, %s, %s, %d, %t)", + _, err = sess.Execute(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), fmt.Sprintf("insert into mysql.tidb_ddl_job(job_id, reorg, schema_ids, table_ids, job_meta, type, processing) values (%d, %t, %s, %s, %s, %d, %t)", job.ID, job.MayNeedReorg(), strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), strconv.Quote(strconv.FormatInt(job.TableID, 10)), wrapKey2String(b), job.Type, false)) return err } diff --git a/domain/domain_test.go b/domain/domain_test.go index 98776381d1ae8..bbce2bcb23283 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -82,7 +82,9 @@ func TestInfo(t *testing.T) { ) ddl.DisableTiFlashPoll(dom.ddl) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/MockReplaceDDL", `return(true)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/NoDDLDispatchLoop", `return(true)`)) require.NoError(t, dom.Init(ddlLease, sysMockFactory)) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/NoDDLDispatchLoop")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/MockReplaceDDL")) // Test for GetServerInfo and GetServerInfoByID. diff --git a/executor/executor_test.go b/executor/executor_test.go index 880bbd62a12c0..db2aebede77a6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -4389,7 +4389,7 @@ func TestAdminShowDDLJobs(t *testing.T) { require.NoError(t, err) txn, err := tk.Session().Txn(true) require.NoError(t, err) - err = ddl.AddHistoryDDLJob(meta.NewMeta(txn), job, true) + err = meta.NewMeta(txn).AddHistoryDDLJob(job, true) require.NoError(t, err) re = tk.MustQuery("admin show ddl jobs 1") diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index e082bef942db0..755d2498a573a 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -3830,6 +3830,7 @@ func TestSetVariables(t *testing.T) { require.Error(t, err) require.Error(t, err, variable.ErrWrongTypeForVar.GenWithStackByArgs("max_prepared_stmt_count").Error()) + tk.MustExec("set @@global.tidb_enable_concurrent_ddl=1") tk.MustQuery("select @@global.tidb_enable_concurrent_ddl").Check(testkit.Rows("1")) require.True(t, variable.EnableConcurrentDDL.Load()) tk.MustExec("set @@global.tidb_enable_concurrent_ddl=0") diff --git a/server/tidb_test.go b/server/tidb_test.go index b6b40b605c5e0..69fb06a1d529c 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -2434,15 +2434,15 @@ func TestTopSQLResourceTag(t *testing.T) { {"replace into t values (2,2), (4,4)", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdBatchGet}}, // Test for DDL - {"create database test_db0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"create table test_db0.test_t0 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"create table test_db0.test_t1 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"alter table test_db0.test_t0 add column c int", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"drop table test_db0.test_t0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"drop database test_db0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, - {"alter table t modify column b double", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdScan, tikvrpc.CmdCop}}, - {"alter table t add index idx2 (b,a)", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdScan, tikvrpc.CmdCop}}, - {"alter table t drop index idx2", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"create database test_db0", false, []tikvrpc.CmdType{}}, + {"create table test_db0.test_t0 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{}}, + {"create table test_db0.test_t1 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{}}, + {"alter table test_db0.test_t0 add column c int", false, []tikvrpc.CmdType{}}, + {"drop table test_db0.test_t0", false, []tikvrpc.CmdType{}}, + {"drop database test_db0", false, []tikvrpc.CmdType{}}, + {"alter table t modify column b double", false, []tikvrpc.CmdType{tikvrpc.CmdScan, tikvrpc.CmdCop}}, + {"alter table t add index idx2 (b,a)", false, []tikvrpc.CmdType{tikvrpc.CmdScan, tikvrpc.CmdCop}}, + {"alter table t drop index idx2", false, []tikvrpc.CmdType{}}, // Test for transaction {"begin", false, nil}, diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index a24a11df97eb0..1f8b0b8a20700 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -443,8 +443,7 @@ func TestOldPasswordUpgrade(t *testing.T) { } func TestBootstrapInitExpensiveQueryHandle(t *testing.T) { - store, err := mockstore.NewMockStore() - require.NoError(t, err) + store, _ := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() diff --git a/session/session_test.go b/session/session_test.go index f065cc1eb142d..aa56851fe6c15 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -44,8 +44,8 @@ func TestInitMetaTable(t *testing.T) { metaInMySQL := external.GetTableByName(t, tk, "mysql", tbl).Meta() metaInTest := external.GetTableByName(t, tk, "test", tbl).Meta() - require.Greater(t, metaInMySQL.ID, 0) - require.Greater(t, metaInMySQL.UpdateTS, 0) + require.Greater(t, metaInMySQL.ID, int64(0)) + require.Greater(t, metaInMySQL.UpdateTS, uint64(0)) metaInTest.ID = metaInMySQL.ID metaInMySQL.UpdateTS = metaInTest.UpdateTS diff --git a/tests/realtikvtest/testkit.go b/tests/realtikvtest/testkit.go index 501fafcd3e6ec..a50dd5ef0d85a 100644 --- a/tests/realtikvtest/testkit.go +++ b/tests/realtikvtest/testkit.go @@ -46,7 +46,7 @@ var WithRealTiKV = flag.Bool("with-real-tikv", false, "whether tests run with re func RunTestMain(m *testing.M) { testsetup.SetupForCommonTest() flag.Parse() - session.SetSchemaLease(20 * time.Millisecond) + session.SetSchemaLease(5 * time.Second) config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 From ad9367883d62dadc94e9752a16b4cea2db56fe3a Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Sun, 10 Jul 2022 17:14:10 +0800 Subject: [PATCH 09/21] 9. migrate ddl between table and queue support switch between the old and new ddl framework, migrate the existing ddl job between queue and table --- ddl/ddl.go | 69 +++++++++++++++- ddl/ddl_worker.go | 17 +++- ddl/job_table.go | 133 ++++++++++++++++++++++++++++--- ddl/job_table_test.go | 115 ++++++++++++++++++++++++++ ddl/reorg.go | 10 +-- ddl/schematracker/checker.go | 10 +++ ddl/schematracker/dm_tracker.go | 10 +++ meta/meta.go | 68 ++++++++++++++-- meta/meta_test.go | 4 +- session/bootstrap.go | 17 +++- sessionctx/context.go | 2 + sessionctx/variable/sysvar.go | 8 +- sessionctx/variable/tidb_vars.go | 2 + 13 files changed, 431 insertions(+), 34 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index a65dadd266340..e2ca142f10cc7 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -198,11 +198,18 @@ type DDL interface { GetInfoSchemaWithInterceptor(ctx sessionctx.Context) infoschema.InfoSchema // DoDDLJob does the DDL job, it's exported for test. DoDDLJob(ctx sessionctx.Context, job *model.Job) error + // MoveJobFromQueue2Table move existing DDLs from queue to table. + MoveJobFromQueue2Table(bool) error + // MoveJobFromTable2Queue move existing DDLs from table to queue. + MoveJobFromTable2Queue() error } type limitJobTask struct { job *model.Job err chan error + + // If mustToQueue is **NOT** nil, we must put the job to queue, only happen upgrade a multiple node cluster. + mustToQueue interface{} } // ddl is used to handle the statements that define the structure or schema of the database. @@ -308,6 +315,8 @@ type ddlCtx struct { sync.Mutex seqNum uint64 } + + waiting *atomicutil.Bool } // schemaVersionManager is used to manage the schema version. To prevent the conflicts on this key between different DDL job, @@ -552,6 +561,7 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnDDL) ddlCtx.ctx, ddlCtx.cancel = context.WithCancel(ctx) ddlCtx.runningJobs.ids = make(map[int64]struct{}) + ddlCtx.waiting = atomicutil.NewBool(false) d := &ddl{ ddlCtx: ddlCtx, @@ -560,6 +570,8 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ddlJobCh: make(chan struct{}, 100), } + variable.SwitchConcurrentDDL = d.SwitchConcurrentDDL + return d } @@ -887,14 +899,15 @@ 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) - task := &limitJobTask{job, make(chan error)} + v := ctx.Value(sessionctx.OldDDLStyle) + task := &limitJobTask{job, make(chan error), v} d.limitJobCh <- task failpoint.Inject("mockParallelSameDDLJobTwice", func(val failpoint.Value) { if val.(bool) { // The same job will be put to the DDL queue twice. job = job.Clone() - task1 := &limitJobTask{job, make(chan error)} + task1 := &limitJobTask{job, make(chan error), v} d.limitJobCh <- task1 <-task.err // The second job result is used for test. @@ -974,6 +987,7 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { se, err := d.sessPool.get() if err != nil { + logutil.BgLogger().Error("[ddl] get session failed, check again", zap.Error(err)) continue } historyJob, err = GetHistoryJobByID(se, jobID) @@ -1081,6 +1095,57 @@ func (d *ddl) startCleanDeadTableLock() { } } +// SwitchConcurrentDDL changes the DDL to concurrent DDL if toConcurrentDDL is true, otherwise, queue based DDL. +func (d *ddl) SwitchConcurrentDDL(toConcurrentDDL bool) error { + if !d.isOwner() { + return kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), d.store, true, func(ctx context.Context, txn kv.Transaction) error { + isConcurrentDDL, err := meta.NewMeta(txn).IsConcurrentDDL() + if err != nil { + return err + } + if isConcurrentDDL != toConcurrentDDL { + return errors.New("please set it on the DDL owner node") + } + return nil + }) + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + d.waiting.Store(true) + defer d.waiting.Store(false) + if err := d.wait4Switch(ctx); err != nil { + return err + } + + var err error + if toConcurrentDDL { + err = d.MoveJobFromQueue2Table(false) + } else { + err = d.MoveJobFromTable2Queue() + } + variable.EnableConcurrentDDL.Store(toConcurrentDDL) + logutil.BgLogger().Info("[ddl] SwitchConcurrentDDL", zap.Bool("toConcurrentDDL", toConcurrentDDL), zap.Error(err)) + return err +} + +func (d *ddl) wait4Switch(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + d.runningJobs.RLock() + if len(d.runningJobs.ids) == 0 { + d.runningJobs.RUnlock() + return nil + } + d.runningJobs.RUnlock() + time.Sleep(time.Second * 1) + } +} + // RecoverInfo contains information needed by DDL.RecoverTable. type RecoverInfo struct { SchemaID int64 diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index c581932549ae7..64696f86c05a0 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -294,7 +294,8 @@ func (d *ddl) limitDDLJobs() { func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { startTime := time.Now() var err error - if variable.EnableConcurrentDDL.Load() { + // tasks[0].v != nil only happens when upgrading. The internal DDL job will always only one in a batch. + if variable.EnableConcurrentDDL.Load() && tasks[0].mustToQueue == nil { err = d.addBatchDDLJobs2Table(tasks) } else { err = d.addBatchDDLJobs2Queue(tasks) @@ -694,6 +695,10 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { if err != nil { return err } + if !variable.EnableConcurrentDDL.Load() || d.waiting.Load() { + w.sess.rollback() + return nil + } failpoint.Inject("mockRunJobTime", func(val failpoint.Value) { if val.(bool) { time.Sleep(time.Duration(rand.Intn(5)) * time.Second) // #nosec G404 @@ -832,18 +837,26 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { waitTime := 2 * d.lease ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) err := kv.RunInNewTxn(ctx, d.store, false, func(ctx context.Context, txn kv.Transaction) error { + d.runningJobs.Lock() // We are not owner, return and retry checking later. - if !d.isOwner() { + if !d.isOwner() || variable.EnableConcurrentDDL.Load() || d.waiting.Load() { + d.runningJobs.Unlock() return nil } var err error t := newMetaWithQueueTp(txn, w.tp) + // We become the owner. Get the first job and run it. job, err = w.getFirstDDLJob(t) if job == nil || err != nil { + d.runningJobs.Unlock() return errors.Trace(err) } + d.runningJobs.ids[job.ID] = struct{}{} + d.runningJobs.Unlock() + + defer d.deleteRunningDDLJobMap(job.ID) // only general ddls allowed to be executed when TiKV is disk full. if w.tp == addIdxWorker && job.IsRunning() { diff --git a/ddl/job_table.go b/ddl/job_table.go index df4f401e030cc..b9e66a4c12cb1 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" @@ -167,7 +168,7 @@ func (d *ddl) startDispatchLoop() { if isChanClosed(d.ctx.Done()) { return } - if !d.isOwner() { + if !variable.EnableConcurrentDDL.Load() || !d.isOwner() || d.waiting.Load() { d.once.Store(true) time.Sleep(time.Second) continue @@ -381,17 +382,17 @@ func updateDDLReorgStartHandle(sess *session, job *model.Job, element *meta.Elem } // updateDDLReorgHandle update startKey, endKey physicalTableID and element of the handle. -func updateDDLReorgHandle(sess *session, job *model.Job, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { +func updateDDLReorgHandle(sess *session, jobID int64, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { sql := fmt.Sprintf("update mysql.tidb_ddl_reorg set ele_id = %d, ele_type = %s, start_key = %s, end_key = %s, physical_id = %d where job_id = %d", - element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID, job.ID) + element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID, jobID) _, err := sess.execute(context.Background(), sql, "update_handle") return err } // initDDLReorgHandle initializes the handle for ddl reorg. -func initDDLReorgHandle(sess *session, job *model.Job, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { +func initDDLReorgHandle(sess *session, jobID int64, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { sql := fmt.Sprintf("insert into mysql.tidb_ddl_reorg(job_id, ele_id, ele_type, start_key, end_key, physical_id) values (%d, %d, %s, %s, %s, %d)", - job.ID, element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID) + jobID, element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID) _, err := sess.execute(context.Background(), sql, "update_handle") return err } @@ -438,18 +439,124 @@ func getJobsBySQL(sess *session, tbl, condition string) ([]*model.Job, error) { return jobs, nil } -func runInTxn(se *session, f func(*session) error) (err error) { - err = se.begin() +// MoveJobFromQueue2Table move existing DDLs in queue to table. +func (d *ddl) MoveJobFromQueue2Table(force bool) error { + sess, err := d.sessPool.get() if err != nil { return err } - defer func() { + defer d.sessPool.put(sess) + return runInTxn(newSession(sess), func(se *session) error { + txn, err := se.txn() if err != nil { - se.rollback() - return + return errors.Trace(err) + } + t := meta.NewMeta(txn) + isConcurrentDDL, err := t.IsConcurrentDDL() + if !force && (isConcurrentDDL || err != nil) { + return errors.Trace(err) + } + for _, tp := range []workerType{addIdxWorker, generalWorker} { + t := newMetaWithQueueTp(txn, tp) + jobs, err := t.GetAllDDLJobsInQueue() + if err != nil { + return errors.Trace(err) + } + err = insertDDLJobs2Table(se, jobs, false) + if err != nil { + return errors.Trace(err) + } + if tp == addIdxWorker { + for _, job := range jobs { + element, start, end, pid, err := t.GetDDLReorgHandle(job) + if meta.ErrDDLReorgElementNotExist.Equal(err) { + continue + } + if err != nil { + return errors.Trace(err) + } + err = initDDLReorgHandle(se, job.ID, start, end, pid, element) + if err != nil { + return errors.Trace(err) + } + } + } + } + + if err = t.ClearALLDDLJob(); err != nil { + return errors.Trace(err) + } + if err = t.ClearAllDDLReorgHandle(); err != nil { + return errors.Trace(err) + } + return t.SetConcurrentDDL(true) + }) +} + +// MoveJobFromTable2Queue move existing DDLs in table to queue. +func (d *ddl) MoveJobFromTable2Queue() error { + sess, err := d.sessPool.get() + if err != nil { + return err + } + defer d.sessPool.put(sess) + return runInTxn(newSession(sess), func(se *session) error { + txn, err := se.txn() + if err != nil { + return errors.Trace(err) + } + t := meta.NewMeta(txn) + isConcurrentDDL, err := t.IsConcurrentDDL() + if !isConcurrentDDL || err != nil { + return errors.Trace(err) + } + jobs, err := getJobsBySQL(se, "tidb_ddl_job", "1 order by job_id") + if err != nil { + return errors.Trace(err) + } + + for _, job := range jobs { + jobListKey := meta.DefaultJobListKey + if job.MayNeedReorg() { + jobListKey = meta.AddIndexJobListKey + } + if err := t.EnQueueDDLJobNoUpdate(job, jobListKey); err != nil { + return errors.Trace(err) + } + } + + reorgHandle, err := se.execute(context.Background(), "select job_id, start_key, end_key, physical_id, ele_id, ele_type from mysql.tidb_ddl_reorg", "get_handle") + if err != nil { + return errors.Trace(err) + } + for _, row := range reorgHandle { + if err := t.UpdateDDLReorgHandle(row.GetInt64(0), row.GetBytes(1), row.GetBytes(2), row.GetInt64(3), &meta.Element{ID: row.GetInt64(4), TypeKey: row.GetBytes(5)}); err != nil { + return errors.Trace(err) + } + } + + // clean up these 2 tables. + _, err = se.execute(context.Background(), "delete from mysql.tidb_ddl_job", "delete_old_ddl") + if err != nil { + return errors.Trace(err) + } + _, err = se.execute(context.Background(), "delete from mysql.tidb_ddl_reorg", "delete_old_reorg") + if err != nil { + return errors.Trace(err) } - err = se.commit() - }() + return t.SetConcurrentDDL(false) + }) +} + +func runInTxn(se *session, f func(*session) error) (err error) { + err = se.begin() + if err != nil { + return err + } err = f(se) - return + if err != nil { + se.rollback() + return + } + return errors.Trace(se.commit()) } diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index c47dc5cc4a7c5..8f2b5b5a056b8 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -15,17 +15,23 @@ package ddl_test import ( + "context" + "fmt" + "math/rand" "sync" "testing" "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" + "go.uber.org/atomic" "golang.org/x/exp/slices" ) @@ -184,3 +190,112 @@ func check(t *testing.T, record []int64, ids ...int64) { } } } + +func TestConcurrentDDLSwitch(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + type table struct { + columnIdx int + indexIdx int + } + + var tables []*table + tblCount := 20 + for i := 0; i < tblCount; i++ { + tables = append(tables, &table{1, 0}) + } + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt=1") + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size=32") + + for i := range tables { + tk.MustExec(fmt.Sprintf("create table t%d (col0 int) partition by range columns (col0) ("+ + "partition p1 values less than (100), "+ + "partition p2 values less than (300), "+ + "partition p3 values less than (500), "+ + "partition p4 values less than (700), "+ + "partition p5 values less than (1000), "+ + "partition p6 values less than maxvalue);", + i)) + for j := 0; j < 1000; j++ { + tk.MustExec(fmt.Sprintf("insert into t%d values (%d)", i, j)) + } + } + + ddls := make([]string, 0, tblCount) + ddlCount := 500 + for i := 0; i < ddlCount; i++ { + tblIdx := rand.Intn(tblCount) + if rand.Intn(2) == 0 { + ddls = append(ddls, fmt.Sprintf("alter table t%d add index idx%d (col0)", tblIdx, tables[tblIdx].indexIdx)) + tables[tblIdx].indexIdx++ + } else { + ddls = append(ddls, fmt.Sprintf("alter table t%d add column col%d int", tblIdx, tables[tblIdx].columnIdx)) + tables[tblIdx].columnIdx++ + } + } + + c := atomic.NewInt32(0) + ch := make(chan struct{}) + go func() { + var wg util.WaitGroupWrapper + for i := range ddls { + wg.Add(1) + go func(idx int) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(ddls[idx]) + c.Add(1) + wg.Done() + }(i) + } + wg.Wait() + ch <- struct{}{} + }() + + ticker := time.NewTicker(time.Second * 2) + count := 0 + done := false + for !done { + select { + case <-ch: + done = true + case <-ticker.C: + var b bool + var err error + err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, false, func(ctx context.Context, txn kv.Transaction) error { + b, err = meta.NewMeta(txn).IsConcurrentDDL() + return err + }) + require.NoError(t, err) + rs, err := testkit.NewTestKit(t, store).Exec(fmt.Sprintf("set @@global.tidb_enable_concurrent_ddl=%t", !b)) + if rs != nil { + require.NoError(t, rs.Close()) + } + if err == nil { + count++ + if b { + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select count(*) from mysql.tidb_ddl_job").Check(testkit.Rows("0")) + tk.MustQuery("select count(*) from mysql.tidb_ddl_reorg").Check(testkit.Rows("0")) + } + } + } + } + + require.Equal(t, int32(ddlCount), c.Load()) + require.Greater(t, count, 0) + + tk = testkit.NewTestKit(t, store) + tk.MustExec("use test") + for i, tbl := range tables { + tk.MustQuery(fmt.Sprintf("select count(*) from information_schema.columns where TABLE_SCHEMA = 'test' and TABLE_NAME = 't%d'", i)).Check(testkit.Rows(fmt.Sprintf("%d", tbl.columnIdx))) + tk.MustExec(fmt.Sprintf("admin check table t%d", i)) + for j := 0; j < tbl.indexIdx; j++ { + tk.MustExec(fmt.Sprintf("admin check index t%d idx%d", i, j)) + } + } +} diff --git a/ddl/reorg.go b/ddl/reorg.go index a4d0dcf033f2f..f6d2e135352b3 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -653,7 +653,7 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, // We'll try to remove it in the next major TiDB version. if meta.ErrDDLReorgElementNotExist.Equal(err) { job.SnapshotVer = 0 - logutil.BgLogger().Warn("[ddl] get reorg info, the element does not exist", zap.String("job", job.String())) + logutil.BgLogger().Warn("[ddl] get reorg info, the element does not exist", zap.String("job", job.String()), zap.Bool("enableConcurrentDDL", rh.enableConcurrentDDL)) } return &info, errors.Trace(err) } @@ -784,17 +784,17 @@ func (r *reorgHandler) UpdateDDLReorgStartHandle(job *model.Job, element *meta.E // UpdateDDLReorgHandle saves the job reorganization latest processed information for later resuming. func (r *reorgHandler) UpdateDDLReorgHandle(job *model.Job, startKey, endKey kv.Key, physicalTableID int64, element *meta.Element) error { if r.enableConcurrentDDL { - return updateDDLReorgHandle(r.s, job, startKey, endKey, physicalTableID, element) + return updateDDLReorgHandle(r.s, job.ID, startKey, endKey, physicalTableID, element) } - return r.m.UpdateDDLReorgHandle(job, startKey, endKey, physicalTableID, element) + return r.m.UpdateDDLReorgHandle(job.ID, startKey, endKey, physicalTableID, element) } // InitDDLReorgHandle initializes the job reorganization information. func (r *reorgHandler) InitDDLReorgHandle(job *model.Job, startKey, endKey kv.Key, physicalTableID int64, element *meta.Element) error { if r.enableConcurrentDDL { - return initDDLReorgHandle(r.s, job, startKey, endKey, physicalTableID, element) + return initDDLReorgHandle(r.s, job.ID, startKey, endKey, physicalTableID, element) } - return r.m.UpdateDDLReorgHandle(job, startKey, endKey, physicalTableID, element) + return r.m.UpdateDDLReorgHandle(job.ID, startKey, endKey, physicalTableID, element) } // RemoveReorgElementFailPoint removes the element of the reorganization information. diff --git a/ddl/schematracker/checker.go b/ddl/schematracker/checker.go index 6103cc12a84c3..be5314ec85343 100644 --- a/ddl/schematracker/checker.go +++ b/ddl/schematracker/checker.go @@ -466,3 +466,13 @@ func (d Checker) GetInfoSchemaWithInterceptor(ctx sessionctx.Context) infoschema func (d Checker) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { return d.realDDL.DoDDLJob(ctx, job) } + +// MoveJobFromQueue2Table implements the DDL interface. +func (d Checker) MoveJobFromQueue2Table(bool) error { + panic("implement me") +} + +// MoveJobFromTable2Queue implements the DDL interface. +func (d Checker) MoveJobFromTable2Queue() error { + panic("implement me") +} diff --git a/ddl/schematracker/dm_tracker.go b/ddl/schematracker/dm_tracker.go index 0391fafc39c30..57b9f944bdc61 100644 --- a/ddl/schematracker/dm_tracker.go +++ b/ddl/schematracker/dm_tracker.go @@ -532,3 +532,13 @@ func (d SchemaTracker) GetInfoSchemaWithInterceptor(ctx sessionctx.Context) info func (d SchemaTracker) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { return nil } + +// MoveJobFromQueue2Table implements the DDL interface, it's no-op in DM's case. +func (d SchemaTracker) MoveJobFromQueue2Table(b bool) error { + panic("implement me") +} + +// MoveJobFromTable2Queue implements the DDL interface, it's no-op in DM's case. +func (d SchemaTracker) MoveJobFromTable2Queue() error { + panic("implement me") +} diff --git a/meta/meta.go b/meta/meta.go index 213037ebadef4..641c959388dc6 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -15,6 +15,7 @@ package meta import ( + "bytes" "encoding/binary" "encoding/json" "fmt" @@ -76,6 +77,7 @@ var ( mPolicyGlobalID = []byte("PolicyGlobalID") mPolicyMagicByte = CurrentMagicByteVer mDDLTableVersion = []byte("DDLTableVersion") + mConcurrentDDL = []byte("concurrentDDL") ) const ( @@ -532,6 +534,27 @@ func (m *Meta) CheckDDLTableExists() (bool, error) { return len(v) != 0, nil } +// SetConcurrentDDL set the concurrent DDL flag. +func (m *Meta) SetConcurrentDDL(b bool) error { + var data []byte + if b { + data = []byte("1") + } else { + data = []byte("0") + } + return errors.Trace(m.txn.Set(mConcurrentDDL, data)) +} + +// IsConcurrentDDL returns true if the concurrent DDL flag is set. +func (m *Meta) IsConcurrentDDL() (bool, error) { + val, err := m.txn.Get(mConcurrentDDL) + if err != nil { + return false, errors.Trace(err) + } + + return len(val) == 0 || bytes.Equal(val, []byte("1")), nil +} + // CreateTableAndSetAutoID creates a table with tableInfo in database, // and rebases the table autoID. func (m *Meta) CreateTableAndSetAutoID(dbID int64, tableInfo *model.TableInfo, autoIncID, autoRandID int64) error { @@ -845,8 +868,8 @@ var ( AddIndexJobListKey JobListKeyType = mDDLJobAddIdxList ) -func (m *Meta) enQueueDDLJob(key []byte, job *model.Job) error { - b, err := job.Encode(true) +func (m *Meta) enQueueDDLJob(key []byte, job *model.Job, updateRawArgs bool) error { + b, err := job.Encode(updateRawArgs) if err == nil { err = m.txn.RPush(key, b) } @@ -860,7 +883,17 @@ func (m *Meta) EnQueueDDLJob(job *model.Job, jobListKeys ...JobListKeyType) erro listKey = jobListKeys[0] } - return m.enQueueDDLJob(listKey, job) + return m.enQueueDDLJob(listKey, job, true) +} + +// EnQueueDDLJobNoUpdate adds a DDL job to the list without update raw args. +func (m *Meta) EnQueueDDLJobNoUpdate(job *model.Job, jobListKeys ...JobListKeyType) error { + listKey := m.jobListKey + if len(jobListKeys) != 0 { + listKey = jobListKeys[0] + } + + return m.enQueueDDLJob(listKey, job, false) } func (m *Meta) deQueueDDLJob(key []byte) (*model.Job, error) { @@ -1191,27 +1224,46 @@ func (m *Meta) UpdateDDLReorgStartHandle(job *model.Job, element *Element, start } // UpdateDDLReorgHandle saves the job reorganization latest processed information for later resuming. -func (m *Meta) UpdateDDLReorgHandle(job *model.Job, startKey, endKey kv.Key, physicalTableID int64, element *Element) error { - err := m.txn.HSet(mDDLJobReorgKey, m.reorgJobCurrentElement(job.ID), element.EncodeElement()) +func (m *Meta) UpdateDDLReorgHandle(jobID int64, startKey, endKey kv.Key, physicalTableID int64, element *Element) error { + err := m.txn.HSet(mDDLJobReorgKey, m.reorgJobCurrentElement(jobID), element.EncodeElement()) if err != nil { return errors.Trace(err) } if startKey != nil { - err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobStartHandle(job.ID, element), startKey) + err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobStartHandle(jobID, element), startKey) if err != nil { return errors.Trace(err) } } if endKey != nil { - err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobEndHandle(job.ID, element), endKey) + err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobEndHandle(jobID, element), endKey) if err != nil { return errors.Trace(err) } } - err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobPhysicalTableID(job.ID, element), []byte(strconv.FormatInt(physicalTableID, 10))) + err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobPhysicalTableID(jobID, element), []byte(strconv.FormatInt(physicalTableID, 10))) return errors.Trace(err) } +// ClearAllDDLReorgHandle clears all reorganization related handles. +func (m *Meta) ClearAllDDLReorgHandle() error { + return m.txn.HClear(mDDLJobReorgKey) +} + +// ClearALLDDLJob clears all DDL jobs. +func (m *Meta) ClearALLDDLJob() error { + if err := m.txn.LClear(mDDLJobAddIdxList); err != nil { + return errors.Trace(err) + } + if err := m.txn.LClear(mDDLJobListKey); err != nil { + return errors.Trace(err) + } + if err := m.txn.LClear(mDDLJobHistoryKey); err != nil { + return errors.Trace(err) + } + return nil +} + // ClearAllHistoryJob clears all history jobs. **IT IS VERY DANGEROUS** func (m *Meta) ClearAllHistoryJob() error { if err := m.txn.HClear(mDDLJobHistoryKey); err != nil { diff --git a/meta/meta_test.go b/meta/meta_test.go index 682b6a68f671e..1ec725e97d0a3 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -511,10 +511,10 @@ func TestDDL(t *testing.T) { require.Equal(t, int64(0), k) element = &meta.Element{ID: 222, TypeKey: meta.ColumnElementKey} - err = m.UpdateDDLReorgHandle(job, tc.startHandle.Encoded(), tc.endHandle.Encoded(), 3, element) + err = m.UpdateDDLReorgHandle(job.ID, tc.startHandle.Encoded(), tc.endHandle.Encoded(), 3, element) require.NoError(t, err) element1 := &meta.Element{ID: 223, TypeKey: meta.IndexElementKey} - err = m.UpdateDDLReorgHandle(job, tc.startHandle.Encoded(), tc.endHandle.Encoded(), 3, element1) + err = m.UpdateDDLReorgHandle(job.ID, tc.startHandle.Encoded(), tc.endHandle.Encoded(), 3, element1) require.NoError(t, err) e, i, j, k, err = m.GetDDLReorgHandle(job) diff --git a/session/bootstrap.go b/session/bootstrap.go index 4c7b03a80d8ec..a022f11c03b8c 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -620,11 +621,13 @@ const ( version90 = 90 // version91 converts prepared-plan-cache to sysvars version91 = 91 + // version92 for concurrent ddl. + version92 = 92 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version91 +var currentBootstrapVersion int64 = version92 var ( bootstrapVersion = []func(Session, int64){ @@ -785,15 +788,27 @@ func upgrade(s Session) { // It is already bootstrapped/upgraded by a higher version TiDB server. return } + // only upgrade from under version92 and this tidb is not owner set. + if ver < version92 && !domain.GetDomain(s).DDL().OwnerManager().IsOwner() { + // set OldDDLStyle is true will make DDL job add into job queue. + s.SetValue(sessionctx.OldDDLStyle, true) + } // Do upgrade works then update bootstrap version. for _, upgrade := range bootstrapVersion { upgrade(s, ver) } + // after upgrade DDL is finished, we can clear it. + s.ClearValue(sessionctx.OldDDLStyle) updateBootstrapVer(s) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) _, err = s.ExecuteInternal(ctx, "COMMIT") + if err == nil && ver <= version92 { + logutil.BgLogger().Info("start migrate DDLs") + err = domain.GetDomain(s).DDL().MoveJobFromQueue2Table(true) + } + if err != nil { sleepTime := 1 * time.Second logutil.BgLogger().Info("update bootstrap ver failed", diff --git a/sessionctx/context.go b/sessionctx/context.go index d5f6279207738..d3d865c69d1f8 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -197,6 +197,8 @@ const ( Initing basicCtxType = 2 // LastExecuteDDL is the key for whether the session execute a ddl command last time. LastExecuteDDL basicCtxType = 3 + // OldDDLStyle is the key for indicating if the DDL job should add into queue. + OldDDLStyle basicCtxType = 4 ) // ValidateSnapshotReadTS strictly validates that readTS does not exceed the PD timestamp diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 977f960133d78..cc34a986cd85d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -807,7 +807,13 @@ var defaultSysVars = []*SysVar{ }, }, {Scope: ScopeGlobal, Name: TiDBEnableConcurrentDDL, Value: BoolToOnOff(DefTiDBEnableConcurrentDDL), Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { - EnableConcurrentDDL.Store(TiDBOptOn(val)) + if EnableConcurrentDDL.Load() != TiDBOptOn(val) { + err := SwitchConcurrentDDL(TiDBOptOn(val)) + if err != nil { + return err + } + EnableConcurrentDDL.Store(TiDBOptOn(val)) + } return nil }, GetGlobal: func(s *SessionVars) (string, error) { return BoolToOnOff(EnableConcurrentDDL.Load()), nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 190274a09928c..74c9d312f768d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -1032,4 +1032,6 @@ var ( GetMemQuotaAnalyze func() int64 = nil // SetStatsCacheCapacity is the func registered by domain to set statsCache memory quota. SetStatsCacheCapacity atomic.Value + // SwitchConcurrentDDL is the func registered by DDL to switch concurrent DDL. + SwitchConcurrentDDL func(bool) error = nil ) From 74aeb598679c9569800b4b4f3b44b88c8f66c88c Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Mon, 18 Jul 2022 13:31:44 +0800 Subject: [PATCH 10/21] 10. check tikv version and set reorg worker count according cpu count --- ddl/ddl.go | 6 ++++- domain/infosync/info.go | 40 +++++++++++++++++++++++++++++++++ tidb-server/main.go | 4 ++++ util/versioninfo/versioninfo.go | 2 +- 4 files changed, 50 insertions(+), 2 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index e2ca142f10cc7..ff60816aa531a 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -23,6 +23,7 @@ import ( "encoding/json" "flag" "fmt" + "runtime" "strconv" "strings" "sync" @@ -57,6 +58,7 @@ import ( "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/tikv/client-go/v2/tikvrpc" clientv3 "go.etcd.io/etcd/client/v3" @@ -612,7 +614,9 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { return wk, nil } } - d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(addIdxWorker), reorgWorkerCnt, reorgWorkerCnt, 0), reorg) + // reorg worker count at least 1 at most 10. + reorgCnt := mathutil.Min(mathutil.Max(runtime.NumCPU()/4, 1), reorgWorkerCnt) + d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(addIdxWorker), reorgCnt, reorgCnt, 0), reorg) d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(generalWorker), generalWorkerCnt, generalWorkerCnt, 0), general) failpoint.Inject("NoDDLDispatchLoop", func(val failpoint.Value) { if val.(bool) { diff --git a/domain/infosync/info.go b/domain/infosync/info.go index 98e16e2c8c573..59dc9ede4c56f 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -22,12 +22,14 @@ import ( "net/http" "os" "path" + "regexp" "strconv" "strings" "sync" "sync/atomic" "time" + "github.com/coreos/go-semver/semver" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" @@ -51,6 +53,7 @@ import ( "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/versioninfo" "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/zap" @@ -428,6 +431,43 @@ func doRequest(ctx context.Context, apiName string, addrs []string, route, metho return nil, err } +func removeVAndHash(v string) string { + if v == "" { + return v + } + versionHash := regexp.MustCompile("-[0-9]+-g[0-9a-f]{7,}(-dev)?") + v = versionHash.ReplaceAllLiteralString(v, "") + v = strings.TrimSuffix(v, "-dirty") + return strings.TrimPrefix(v, "v") +} + +// CheckTiKVVersion is used to check the tikv version. +func CheckTiKVVersion(store kv.Storage, minVersion semver.Version) error { + if store, ok := store.(kv.StorageWithPD); ok { + pdClient := store.GetPDClient() + stores, err := pdClient.GetAllStores(context.Background(), pd.WithExcludeTombstone()) + if err != nil { + return err + } + for _, s := range stores { + // empty version means the store is a mock store. + if s.Version == "" { + continue + } + ver, err := semver.NewVersion(removeVAndHash(s.Version)) + if err != nil { + return errors.Trace(errors.Annotate(err, "invalid TiKV version")) + } + v := ver.Compare(minVersion) + if v < 0 { + return errors.New("TiKV version must greater than or equal to " + minVersion.String()) + } + } + } + + return nil +} + func doRequestWithFailpoint(req *http.Request) (resp *http.Response, err error) { fpEnabled := false failpoint.Inject("FailPlacement", func(val failpoint.Value) { diff --git a/tidb-server/main.go b/tidb-server/main.go index 48710fa8a42f5..5a4ef5b62fae2 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -26,6 +26,7 @@ import ( "sync/atomic" "time" + "github.com/coreos/go-semver/semver" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -34,6 +35,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -297,6 +299,8 @@ func createStoreAndDomain() (kv.Storage, *domain.Domain) { var err error storage, err := kvstore.New(fullPath) terror.MustNil(err) + err = infosync.CheckTiKVVersion(storage, *semver.New(versioninfo.TiKVMinVersion)) + terror.MustNil(err) // Bootstrap a session to load information schema. dom, err := session.BootstrapSession(storage) terror.MustNil(err) diff --git a/util/versioninfo/versioninfo.go b/util/versioninfo/versioninfo.go index 563a97e6356d9..8c2b9b57d42e3 100644 --- a/util/versioninfo/versioninfo.go +++ b/util/versioninfo/versioninfo.go @@ -26,5 +26,5 @@ var ( TiDBGitBranch = "None" TiDBEdition = CommunityEdition // TiKVMinVersion is the minimum version of TiKV that can be compatible with the current TiDB. - TiKVMinVersion = "v3.0.0-60965b006877ca7234adaced7890d7b029ed1306" + TiKVMinVersion = "6.2.0-alpha" ) From 76125044be13297302249cf173f70f250d1cdab2 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 12 Jul 2022 17:14:01 +0800 Subject: [PATCH 11/21] *: add featuretag on tests Signed-off-by: Weizhen Wang --- Makefile | 3 +++ br/pkg/backup/BUILD.bazel | 1 + br/pkg/task/BUILD.bazel | 1 + ddl/BUILD.bazel | 4 ++++ session/BUILD.bazel | 3 +++ sessionctx/variable/BUILD.bazel | 1 + .../featuretag/concurrencyddl/BUILD.bazel | 11 ++++++++++ .../featuretag/concurrencyddl/default.go | 20 +++++++++++++++++++ .../featuretag/concurrencyddl/non_default.go | 20 +++++++++++++++++++ sessionctx/variable/tidb_vars.go | 3 ++- 10 files changed, 66 insertions(+), 1 deletion(-) create mode 100644 sessionctx/variable/featuretag/concurrencyddl/BUILD.bazel create mode 100644 sessionctx/variable/featuretag/concurrencyddl/default.go create mode 100644 sessionctx/variable/featuretag/concurrencyddl/non_default.go diff --git a/Makefile b/Makefile index a10eaef06b208..9b09f9b2fd2c0 100644 --- a/Makefile +++ b/Makefile @@ -438,6 +438,9 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare bazel --output_user_root=/home/jenkins/.tidb/tmp coverage --config=ci --@io_bazel_rules_go//go/config:cover_format=go_cover \ -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test + bazel --output_user_root=/home/jenkins/.tidb/tmp coverage --config=ci --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=featuretag \ + -- //... -//cmd/... -//tests/graceshutdown/... \ + -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test bazel_build: bazel_ci_prepare mkdir -p bin diff --git a/br/pkg/backup/BUILD.bazel b/br/pkg/backup/BUILD.bazel index 14f4d1a87c6f9..0ee2f27af135f 100644 --- a/br/pkg/backup/BUILD.bazel +++ b/br/pkg/backup/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//meta", "//meta/autoid", "//parser/model", + "//sessionctx", "//statistics/handle", "//util", "//util/codec", diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index 4acaf84014b03..8d1c738baedca 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//config", "//kv", "//parser/mysql", + "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", "//statistics/handle", diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index f02b31b630067..4774f5e9d883f 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -6,16 +6,19 @@ go_library( "backfilling.go", "callback.go", "column.go", + "constant.go", "ddl.go", "ddl_algorithm.go", "ddl_api.go", "ddl_tiflash_api.go", "ddl_worker.go", + "ddl_workerpool.go", "delete_range.go", "delete_range_util.go", "foreign_key.go", "generated_column.go", "index.go", + "job_table.go", "mock.go", "multi_schema_change.go", "options.go", @@ -144,6 +147,7 @@ go_test( "index_change_test.go", "index_modify_test.go", "integration_test.go", + "job_table_test.go", "main_test.go", "modify_column_test.go", "multi_schema_change_test.go", diff --git a/session/BUILD.bazel b/session/BUILD.bazel index b75f26dc03634..1ed9c0878f038 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -17,6 +17,7 @@ go_library( deps = [ "//bindinfo", "//config", + "//ddl", "//ddl/placement", "//domain", "//errno", @@ -111,6 +112,7 @@ go_test( "nontransactional_test.go", "schema_amender_test.go", "schema_test.go", + "session_test.go", "tidb_test.go", ], data = glob(["testdata/**"]), @@ -140,6 +142,7 @@ go_test( "//table", "//tablecodec", "//testkit", + "//testkit/external", "//testkit/testdata", "//testkit/testmain", "//testkit/testsetup", diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index e008ce4dfda3b..db227c30a8de1 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//parser/types", "//sessionctx/sessionstates", "//sessionctx/stmtctx", + "//sessionctx/variable/featuretag/concurrencyddl", "//tidb-binlog/pump_client", "//types", "//util", diff --git a/sessionctx/variable/featuretag/concurrencyddl/BUILD.bazel b/sessionctx/variable/featuretag/concurrencyddl/BUILD.bazel new file mode 100644 index 0000000000000..44c1cede3c2b7 --- /dev/null +++ b/sessionctx/variable/featuretag/concurrencyddl/BUILD.bazel @@ -0,0 +1,11 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "concurrencyddl", + srcs = [ + "default.go", + "non_default.go", + ], + importpath = "github.com/pingcap/tidb/sessionctx/variable/featuretag/concurrencyddl", + visibility = ["//visibility:public"], +) diff --git a/sessionctx/variable/featuretag/concurrencyddl/default.go b/sessionctx/variable/featuretag/concurrencyddl/default.go new file mode 100644 index 0000000000000..8aca4924268f0 --- /dev/null +++ b/sessionctx/variable/featuretag/concurrencyddl/default.go @@ -0,0 +1,20 @@ +// Copyright 2022 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. + +//go:build !featuretag + +package concurrencyddl + +// TiDBEnableConcurrentDDL is a feature tag +const TiDBEnableConcurrentDDL bool = true diff --git a/sessionctx/variable/featuretag/concurrencyddl/non_default.go b/sessionctx/variable/featuretag/concurrencyddl/non_default.go new file mode 100644 index 0000000000000..72218abe958a3 --- /dev/null +++ b/sessionctx/variable/featuretag/concurrencyddl/non_default.go @@ -0,0 +1,20 @@ +// Copyright 2022 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. + +//go:build featuretag + +package concurrencyddl + +// TiDBEnableConcurrentDDL is a feature tag +const TiDBEnableConcurrentDDL bool = false diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 74c9d312f768d..767528e1e7da9 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable/featuretag/concurrencyddl" "github.com/pingcap/tidb/util/paging" "go.uber.org/atomic" ) @@ -968,7 +969,7 @@ const ( DefTiDBEnablePrepPlanCache = true DefTiDBPrepPlanCacheSize = 100 DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 - DefTiDBEnableConcurrentDDL = true + DefTiDBEnableConcurrentDDL = concurrencyddl.TiDBEnableConcurrentDDL DefTiDBSimplifiedMetrics = false DefTiDBEnablePaging = true DefTiFlashFineGrainedShuffleStreamCount = -1 From e6eeb84ada2216262fdfbb7cb5505139ce7f9fd6 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Mon, 18 Jul 2022 18:19:58 +0800 Subject: [PATCH 12/21] use a determined table id for 3 tables --- ddl/constant.go | 11 +++++++++++ meta/meta.go | 17 ++++++++++++++++- session/session.go | 24 +++++++++++++++++------- session/session_test.go | 2 +- 4 files changed, 45 insertions(+), 9 deletions(-) diff --git a/ddl/constant.go b/ddl/constant.go index 600cc68794cb2..03e8d1fcfaaa5 100644 --- a/ddl/constant.go +++ b/ddl/constant.go @@ -14,6 +14,10 @@ package ddl +import ( + "github.com/pingcap/tidb/meta" +) + const ( // JobTable stores the information of DDL jobs. JobTable = "tidb_ddl_job" @@ -21,4 +25,11 @@ const ( ReorgTable = "tidb_ddl_reorg" // HistoryTable stores the history DDL jobs. HistoryTable = "tidb_ddl_history" + + // JobTableID is the table ID of `tidb_ddl_job`. + JobTableID = meta.MaxInt48 - 1 + // ReorgTableID is the table ID of `tidb_ddl_reorg`. + ReorgTableID = meta.MaxInt48 - 2 + // HistoryTableID is the table ID of `tidb_ddl_history`. + HistoryTableID = meta.MaxInt48 - 3 ) diff --git a/meta/meta.go b/meta/meta.go index 641c959388dc6..bbf6ff90e8e20 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -93,6 +93,11 @@ const ( typeUnknown int = 0 typeJSON int = 1 // todo: customized handler. + + // MaxInt48 is the max value of int48. + MaxInt48 = 0x0000FFFFFFFFFFFF + // MaxGlobalID reserves 1000 IDs. Use MaxInt48 to reserves the high 2 bytes to compatible with Multi-tenancy. + MaxGlobalID = MaxInt48 - 1000 ) var ( @@ -150,7 +155,14 @@ func (m *Meta) GenGlobalID() (int64, error) { globalIDMutex.Lock() defer globalIDMutex.Unlock() - return m.txn.Inc(mNextGlobalIDKey, 1) + newID, err := m.txn.Inc(mNextGlobalIDKey, 1) + if err != nil { + return 0, errors.Trace(err) + } + if newID > MaxGlobalID { + return 0, errors.Errorf("global id:%d exceeds the limit:%d", newID, MaxGlobalID) + } + return newID, err } // GenGlobalIDs generates the next n global IDs. @@ -162,6 +174,9 @@ func (m *Meta) GenGlobalIDs(n int) ([]int64, error) { if err != nil { return nil, err } + if newID > MaxGlobalID { + return nil, errors.Errorf("global id:%d exceeds the limit:%d", newID, MaxGlobalID) + } origID := newID - int64(n) ids := make([]int64, 0, n) for i := origID + 1; i <= newID; i++ { diff --git a/session/session.go b/session/session.go index 5560789d0b45a..52d73d46850a0 100644 --- a/session/session.go +++ b/session/session.go @@ -2707,10 +2707,13 @@ func loadCollationParameter(ctx context.Context, se *session) (bool, error) { var ( errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) // DDLJobTables is a list of tables definitions used in concurrent DDL. - DDLJobTables = []string{ - "create table tidb_ddl_job(job_id bigint not null, reorg int, schema_ids text(65535), table_ids text(65535), job_meta longblob, type int, processing int, primary key(job_id))", - "create table tidb_ddl_reorg(job_id bigint not null, ele_id bigint, ele_type blob, start_key blob, end_key blob, physical_id bigint, reorg_meta longblob, unique key(job_id, ele_id, ele_type(20)))", - "create table tidb_ddl_history(job_id bigint not null, job_meta longblob, db_name char(64), table_name char(64), schema_ids text(65535), table_ids text(65535), create_time datetime, primary key(job_id))", + DDLJobTables = []struct { + SQL string + id int64 + }{ + {"create table tidb_ddl_job(job_id bigint not null, reorg int, schema_ids text(65535), table_ids text(65535), job_meta longblob, type int, processing int, primary key(job_id))", ddl.JobTableID}, + {"create table tidb_ddl_reorg(job_id bigint not null, ele_id bigint, ele_type blob, start_key blob, end_key blob, physical_id bigint, reorg_meta longblob, unique key(job_id, ele_id, ele_type(20)))", ddl.ReorgTableID}, + {"create table tidb_ddl_history(job_id bigint not null, job_meta longblob, db_name char(64), table_name char(64), schema_ids text(65535), table_ids text(65535), create_time datetime, primary key(job_id))", ddl.HistoryTableID}, } ) @@ -2727,8 +2730,15 @@ func InitDDLJobTables(store kv.Storage) error { return err } p := parser.New() - for _, s := range DDLJobTables { - stmt, err := p.ParseOneStmt(s, "", "") + for _, tbl := range DDLJobTables { + id, err := t.GetGlobalID() + if err != nil { + return errors.Trace(err) + } + if id >= meta.MaxGlobalID { + return errors.Errorf("It is unreasonable that the global ID grows such a big value: %d, please concat TiDB team", id) + } + stmt, err := p.ParseOneStmt(tbl.SQL, "", "") if err != nil { return errors.Trace(err) } @@ -2737,7 +2747,7 @@ func InitDDLJobTables(store kv.Storage) error { return errors.Trace(err) } tblInfo.State = model.StatePublic - tblInfo.ID, err = t.GenGlobalID() + tblInfo.ID = tbl.id tblInfo.UpdateTS = t.StartTS if err != nil { return errors.Trace(err) diff --git a/session/session_test.go b/session/session_test.go index aa56851fe6c15..9901376dd3b9c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -31,7 +31,7 @@ func TestInitMetaTable(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") for _, sql := range session.DDLJobTables { - tk.MustExec(sql) + tk.MustExec(sql.SQL) } tbls := map[string]struct{}{ From f0f6f286f11cc482db2eaf723df753af3443fbf1 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 19 Jul 2022 16:35:00 +0800 Subject: [PATCH 13/21] remove ctx value --- ddl/ddl.go | 8 ++------ ddl/ddl_worker.go | 3 +-- session/bootstrap.go | 11 +++++------ sessionctx/context.go | 2 -- 4 files changed, 8 insertions(+), 16 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index ff60816aa531a..38433137a2258 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -209,9 +209,6 @@ type DDL interface { type limitJobTask struct { job *model.Job err chan error - - // If mustToQueue is **NOT** nil, we must put the job to queue, only happen upgrade a multiple node cluster. - mustToQueue interface{} } // ddl is used to handle the statements that define the structure or schema of the database. @@ -903,15 +900,14 @@ 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) - v := ctx.Value(sessionctx.OldDDLStyle) - task := &limitJobTask{job, make(chan error), v} + task := &limitJobTask{job, make(chan error)} d.limitJobCh <- task failpoint.Inject("mockParallelSameDDLJobTwice", func(val failpoint.Value) { if val.(bool) { // The same job will be put to the DDL queue twice. job = job.Clone() - task1 := &limitJobTask{job, make(chan error), v} + task1 := &limitJobTask{job, make(chan error)} d.limitJobCh <- task1 <-task.err // The second job result is used for test. diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 64696f86c05a0..ab2facdb0ab5f 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -294,8 +294,7 @@ func (d *ddl) limitDDLJobs() { func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { startTime := time.Now() var err error - // tasks[0].v != nil only happens when upgrading. The internal DDL job will always only one in a batch. - if variable.EnableConcurrentDDL.Load() && tasks[0].mustToQueue == nil { + if variable.EnableConcurrentDDL.Load() { err = d.addBatchDDLJobs2Table(tasks) } else { err = d.addBatchDDLJobs2Queue(tasks) diff --git a/session/bootstrap.go b/session/bootstrap.go index a022f11c03b8c..b5ca17bfb84ee 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -42,7 +42,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -788,18 +787,18 @@ func upgrade(s Session) { // It is already bootstrapped/upgraded by a higher version TiDB server. return } - // only upgrade from under version92 and this tidb is not owner set. + // Only upgrade from under version92 and this tidb is not owner set. + // The owner in older tidb does not support concurrent DDL, we should add the internal DDL to job queue. + original := variable.EnableConcurrentDDL.Load() if ver < version92 && !domain.GetDomain(s).DDL().OwnerManager().IsOwner() { - // set OldDDLStyle is true will make DDL job add into job queue. - s.SetValue(sessionctx.OldDDLStyle, true) + variable.EnableConcurrentDDL.Store(false) } // Do upgrade works then update bootstrap version. for _, upgrade := range bootstrapVersion { upgrade(s, ver) } - // after upgrade DDL is finished, we can clear it. - s.ClearValue(sessionctx.OldDDLStyle) + variable.EnableConcurrentDDL.Store(original) updateBootstrapVer(s) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) _, err = s.ExecuteInternal(ctx, "COMMIT") diff --git a/sessionctx/context.go b/sessionctx/context.go index d3d865c69d1f8..d5f6279207738 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -197,8 +197,6 @@ const ( Initing basicCtxType = 2 // LastExecuteDDL is the key for whether the session execute a ddl command last time. LastExecuteDDL basicCtxType = 3 - // OldDDLStyle is the key for indicating if the DDL job should add into queue. - OldDDLStyle basicCtxType = 4 ) // ValidateSnapshotReadTS strictly validates that readTS does not exceed the PD timestamp From 1f17ebc8df2268758fd9d67c2f95076e1a273eac Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 19 Jul 2022 18:18:18 +0800 Subject: [PATCH 14/21] add GetSchemaVersionWithNonEmptyDiff function --- br/pkg/backup/client.go | 28 ++-------------------------- ddl/ddl.go | 2 +- domain/domain.go | 23 +---------------------- meta/meta.go | 29 +++++++++++++++++++++++++++++ 4 files changed, 33 insertions(+), 49 deletions(-) diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 321b1307e5488..598f9d50c3006 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -478,39 +478,15 @@ func WriteBackupDDLJobs(metaWriter *metautil.MetaWriter, se sessionctx.Context, snapMeta := meta.NewSnapshotMeta(snapshot) lastSnapshot := store.GetSnapshot(kv.NewVersion(lastBackupTS)) lastSnapMeta := meta.NewSnapshotMeta(lastSnapshot) - lastSchemaVersion, err := lastSnapMeta.GetSchemaVersion() + lastSchemaVersion, err := lastSnapMeta.GetSchemaVersionWithNonEmptyDiff() if err != nil { return errors.Trace(err) } - backupSchemaVersion, err := snapMeta.GetSchemaVersion() + backupSchemaVersion, err := snapMeta.GetSchemaVersionWithNonEmptyDiff() if err != nil { return errors.Trace(err) } - // we need use the version - 1 if the correlation diff is empty. - // consider the following case: - // t1 t2 t3 - // | | | - // schema version commit | diff commit - // backupTS - // DDL job is not done at t2, but schema version can be seen at t2, so if we use the schema as the boundary, we may filter in - // the job which done at t3. So, we need make version -= 1.It is safe because at t2, we can not see the job. - diff, err := lastSnapMeta.GetSchemaDiff(lastSchemaVersion) - if err != nil { - return errors.Trace(err) - } - if diff == nil { - lastSchemaVersion -= 1 - } - - diff, err = snapMeta.GetSchemaDiff(backupSchemaVersion) - if err != nil { - return errors.Trace(err) - } - if diff == nil { - backupSchemaVersion -= 1 - } - version, err := store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return errors.Trace(err) diff --git a/ddl/ddl.go b/ddl/ddl.go index 38433137a2258..2c016593a314e 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1264,7 +1264,7 @@ func GetDDLInfo(s sessionctx.Context) (*Info, error) { info.Jobs = append(info.Jobs, reorgJob) } - info.SchemaVer, err = t.GetSchemaVersion() + info.SchemaVer, err = t.GetSchemaVersionWithNonEmptyDiff() if err != nil { return nil, errors.Trace(err) } diff --git a/domain/domain.go b/domain/domain.go index 10c35453cde09..9e230753aafbc 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -126,32 +126,11 @@ func (do *Domain) EtcdClient() *clientv3.Client { func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, int64, *transaction.RelatedSchemaChange, error) { snapshot := do.store.GetSnapshot(kv.NewVersion(startTS)) m := meta.NewSnapshotMeta(snapshot) - neededSchemaVersion, err := m.GetSchemaVersion() + neededSchemaVersion, err := m.GetSchemaVersionWithNonEmptyDiff() if err != nil { return nil, false, 0, nil, err } - // Consider the following scenario: - // t1 t2 t3 t4 - // | | | - // update schema version | set diff - // stale read ts - // At the first time, t2 reads the schema version v10, but the v10's diff is not set yet, so it loads v9 infoSchema. - // But at t4 moment, v10's diff has been set and been cached in the memory, so stale read on t2 will get v10 schema from cache, - // and inconsistency happen. - // To solve this problem, we always check the schema diff at first, if the diff is empty, we know at t2 moment we can only see the v9 schema, - // so make neededSchemaVersion = neededSchemaVersion - 1. - // For `Reload`, we can also do this: if the newest version's diff is not set yet, it is ok to load the previous version's infoSchema, and wait for the next reload. - diff, err := m.GetSchemaDiff(neededSchemaVersion) - if err != nil { - return nil, false, 0, nil, err - } - - if diff == nil && neededSchemaVersion > 0 { - // Although the diff of neededSchemaVersion is undetermined, the last version's diff is deterministic(this is guaranteed by schemaVersionManager). - neededSchemaVersion -= 1 - } - if is := do.infoCache.GetByVersion(neededSchemaVersion); is != nil { return is, true, 0, nil, nil } diff --git a/meta/meta.go b/meta/meta.go index bbf6ff90e8e20..190f5e43406a9 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -357,6 +357,35 @@ func (m *Meta) GetAutoIDAccessors(dbID, tableID int64) AutoIDAccessors { return NewAutoIDAccessors(m, dbID, tableID) } +// GetSchemaVersionWithNonEmptyDiff gets current global schema version, if diff is nil, we should return version - 1. +// Consider the following scenario: +// t1 t2 t3 t4 +// | | | +// update schema version | set diff +// stale read ts +// At the first time, t2 reads the schema version v10, but the v10's diff is not set yet, so it loads v9 infoSchema. +// But at t4 moment, v10's diff has been set and been cached in the memory, so stale read on t2 will get v10 schema from cache, +// and inconsistency happen. +// To solve this problem, we always check the schema diff at first, if the diff is empty, we know at t2 moment we can only see the v9 schema, +// so make neededSchemaVersion = neededSchemaVersion - 1. +// For `Reload`, we can also do this: if the newest version's diff is not set yet, it is ok to load the previous version's infoSchema, and wait for the next reload. +func (m *Meta) GetSchemaVersionWithNonEmptyDiff() (int64, error) { + v, err := m.txn.GetInt64(mSchemaVersionKey) + if err != nil { + return 0, err + } + diff, err := m.GetSchemaDiff(v) + if err != nil { + return 0, err + } + + if diff == nil && v > 0 { + // Although the diff of v is undetermined, the last version's diff is deterministic(this is guaranteed by schemaVersionManager). + v-- + } + return v, err +} + // GetSchemaVersion gets current global schema version. func (m *Meta) GetSchemaVersion() (int64, error) { return m.txn.GetInt64(mSchemaVersionKey) From c659bed06b6ce0e4edf26f8ad484b8fece57f2b7 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 19 Jul 2022 18:42:38 +0800 Subject: [PATCH 15/21] address tangenta and zimulala comment --- ddl/ddl.go | 1 + ddl/job_table.go | 28 +++++++++++++++------------- meta/meta.go | 3 --- meta/meta_test.go | 39 +++++++++++++++++++++++++++++++++++++++ session/bootstrap.go | 2 +- 5 files changed, 56 insertions(+), 17 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 2c016593a314e..a229e82754b38 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -970,6 +970,7 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { if sessVars.StmtCtx.DDLJobID != 0 { se, err := d.sessPool.get() if err != nil { + logutil.BgLogger().Error("[ddl] get session failed, check again", zap.Error(err)) continue } sessVars.StmtCtx.DDLJobID = 0 // Avoid repeat. diff --git a/ddl/job_table.go b/ddl/job_table.go index b9e66a4c12cb1..f4b2d8e48302d 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -466,19 +466,21 @@ func (d *ddl) MoveJobFromQueue2Table(force bool) error { if err != nil { return errors.Trace(err) } - if tp == addIdxWorker { - for _, job := range jobs { - element, start, end, pid, err := t.GetDDLReorgHandle(job) - if meta.ErrDDLReorgElementNotExist.Equal(err) { - continue - } - if err != nil { - return errors.Trace(err) - } - err = initDDLReorgHandle(se, job.ID, start, end, pid, element) - if err != nil { - return errors.Trace(err) - } + if tp == generalWorker { + // general job do not have reorg info. + continue + } + for _, job := range jobs { + element, start, end, pid, err := t.GetDDLReorgHandle(job) + if meta.ErrDDLReorgElementNotExist.Equal(err) { + continue + } + if err != nil { + return errors.Trace(err) + } + err = initDDLReorgHandle(se, job.ID, start, end, pid, element) + if err != nil { + return errors.Trace(err) } } } diff --git a/meta/meta.go b/meta/meta.go index 190f5e43406a9..b4d9a56287e61 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -1302,9 +1302,6 @@ func (m *Meta) ClearALLDDLJob() error { if err := m.txn.LClear(mDDLJobListKey); err != nil { return errors.Trace(err) } - if err := m.txn.LClear(mDDLJobHistoryKey); err != nil { - return errors.Trace(err) - } return nil } diff --git a/meta/meta_test.go b/meta/meta_test.go index 1ec725e97d0a3..0529fdbcc5eda 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -767,6 +767,45 @@ func TestSequenceKey(b *testing.T) { require.Equal(b, tableID, id) } +func TestClearJob(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + }() + + txn, err := store.Begin() + require.NoError(t, err) + + job1 := &model.Job{ID: 1, TableID: 1, Type: model.ActionAddColumn} + job2 := &model.Job{ID: 2, TableID: 1, Type: model.ActionCreateTable} + job3 := &model.Job{ID: 3, TableID: 2, Type: model.ActionDropColumn} + + m := meta.NewMeta(txn) + + require.NoError(t, m.EnQueueDDLJob(job1)) + require.NoError(t, m.EnQueueDDLJob(job2)) + require.NoError(t, m.EnQueueDDLJob(job3)) + + require.NoError(t, m.AddHistoryDDLJob(job1, false)) + require.NoError(t, m.AddHistoryDDLJob(job2, false)) + + jobs, err := m.GetAllDDLJobsInQueue() + require.NoError(t, err) + require.Len(t, jobs, 3) + require.NoError(t, m.ClearALLDDLJob()) + jobs, err = m.GetAllDDLJobsInQueue() + require.NoError(t, err) + require.Len(t, jobs, 0) + + count, err := m.GetHistoryDDLCount() + require.NoError(t, err) + require.Equal(t, count, uint64(2)) + + err = txn.Rollback() + require.NoError(t, err) +} + func TestCreateMySQLDatabase(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) diff --git a/session/bootstrap.go b/session/bootstrap.go index b5ca17bfb84ee..6248f36d28a26 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -787,7 +787,7 @@ func upgrade(s Session) { // It is already bootstrapped/upgraded by a higher version TiDB server. return } - // Only upgrade from under version92 and this tidb is not owner set. + // Only upgrade from under version92 and this TiDB is not owner set. // The owner in older tidb does not support concurrent DDL, we should add the internal DDL to job queue. original := variable.EnableConcurrentDDL.Load() if ver < version92 && !domain.GetDomain(s).DDL().OwnerManager().IsOwner() { From 8e9f646c7724e1486e4639a03828fbdef72c69e7 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 12:12:17 +0800 Subject: [PATCH 16/21] use only one etcd path --- ddl/ddl.go | 6 +----- ddl/job_table.go | 22 ++++++---------------- 2 files changed, 7 insertions(+), 21 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index a229e82754b38..3f0215e9401b7 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -839,11 +839,7 @@ func (d *ddl) asyncNotifyWorker(job *model.Job) { if d.isOwner() { asyncNotify(d.ddlJobCh) } else { - key := addingDDLJobGeneral - if job.MayNeedReorg() { - key = addingDDLJobReorg - } - d.asyncNotifyByEtcd(key, job) + d.asyncNotifyByEtcd(addingDDLJobConcurrent, job) } } else { var worker *worker diff --git a/ddl/job_table.go b/ddl/job_table.go index f4b2d8e48302d..cb6a82202c574 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -38,8 +38,7 @@ import ( ) var ( - addingDDLJobGeneral = "/tidb/ddl/add_ddl_job_general" - addingDDLJobReorg = "/tidb/ddl/add_ddl_job_reorg" + addingDDLJobConcurrent = "/tidb/ddl/add_ddl_job_general" ) func (dc *ddlCtx) insertRunningDDLJobMap(id int64) { @@ -156,11 +155,9 @@ func (d *ddl) startDispatchLoop() { } defer d.sessPool.put(se) sess := newSession(se) - var notifyDDLJobByEtcdChGeneral clientv3.WatchChan - var notifyDDLJobByEtcdChReorg clientv3.WatchChan + var notifyDDLJobByEtcdCh clientv3.WatchChan if d.etcdCli != nil { - notifyDDLJobByEtcdChGeneral = d.etcdCli.Watch(d.ctx, addingDDLJobGeneral) - notifyDDLJobByEtcdChReorg = d.etcdCli.Watch(d.ctx, addingDDLJobReorg) + notifyDDLJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingDDLJobConcurrent) } ticker := time.NewTicker(1 * time.Second) defer ticker.Stop() @@ -176,17 +173,10 @@ func (d *ddl) startDispatchLoop() { select { case <-d.ddlJobCh: case <-ticker.C: - case _, ok := <-notifyDDLJobByEtcdChGeneral: + case _, ok := <-notifyDDLJobByEtcdCh: if !ok { - logutil.BgLogger().Warn("[ddl] start general worker watch channel closed", zap.String("watch key", addingDDLJobGeneral)) - notifyDDLJobByEtcdChGeneral = d.etcdCli.Watch(d.ctx, addingDDLJobGeneral) - time.Sleep(time.Second) - continue - } - case _, ok := <-notifyDDLJobByEtcdChReorg: - if !ok { - logutil.BgLogger().Warn("[ddl] start reorg worker watch channel closed", zap.String("watch key", addingDDLJobReorg)) - notifyDDLJobByEtcdChReorg = d.etcdCli.Watch(d.ctx, addingDDLJobReorg) + logutil.BgLogger().Warn("[ddl] start worker watch channel closed", zap.String("watch key", addingDDLJobConcurrent)) + notifyDDLJobByEtcdCh = d.etcdCli.Watch(d.ctx, addingDDLJobConcurrent) time.Sleep(time.Second) continue } From 2200af8e4a403628b267022673c63b56fe1a52eb Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 13:14:53 +0800 Subject: [PATCH 17/21] make ActionRenameTable support multi-schema --- ddl/ddl_api.go | 1 + ddl/job_table.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e8c257ec10393..1e21011d42592 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5468,6 +5468,7 @@ func (d *ddl) renameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident, Type: model.ActionRenameTable, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{schemas[0].ID, newIdent.Name, schemas[0].Name}, + CtxVars: []interface{}{[]int64{schemas[0].ID, schemas[1].ID}, []int64{tableID}}, } err = d.DoDDLJob(ctx, job) diff --git a/ddl/job_table.go b/ddl/job_table.go index cb6a82202c574..4009e8fca9b28 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -286,7 +286,7 @@ func job2TableIDs(job *model.Job) string { func job2UniqueIDs(job *model.Job, schema bool) string { switch job.Type { - case model.ActionExchangeTablePartition, model.ActionRenameTables: + case model.ActionExchangeTablePartition, model.ActionRenameTables, model.ActionRenameTable: var ids []int64 if schema { ids = job.CtxVars[0].([]int64) From 88d7216af18f4de0eb738ef8e208593242a11137 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 14:05:15 +0800 Subject: [PATCH 18/21] reset sql digest to make top sql work correct --- ddl/ddl_worker.go | 2 ++ server/tidb_test.go | 18 +++++++++--------- sessionctx/stmtctx/stmtctx.go | 5 +++++ 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index ab2facdb0ab5f..96108304e97bf 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -765,6 +765,8 @@ func (w *worker) HandleDDLJobTable(d *ddlCtx, job *model.Job) error { return err } writeBinlog(d.binlogCli, txn, job) + // reset the SQL digest to make topsql work right. + w.sess.GetSessionVars().StmtCtx.ResetSQLDigest(job.Query) err = w.sess.commit() d.unlockSchemaVersion(job.ID) if err != nil { diff --git a/server/tidb_test.go b/server/tidb_test.go index 69fb06a1d529c..b6b40b605c5e0 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -2434,15 +2434,15 @@ func TestTopSQLResourceTag(t *testing.T) { {"replace into t values (2,2), (4,4)", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdBatchGet}}, // Test for DDL - {"create database test_db0", false, []tikvrpc.CmdType{}}, - {"create table test_db0.test_t0 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{}}, - {"create table test_db0.test_t1 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{}}, - {"alter table test_db0.test_t0 add column c int", false, []tikvrpc.CmdType{}}, - {"drop table test_db0.test_t0", false, []tikvrpc.CmdType{}}, - {"drop database test_db0", false, []tikvrpc.CmdType{}}, - {"alter table t modify column b double", false, []tikvrpc.CmdType{tikvrpc.CmdScan, tikvrpc.CmdCop}}, - {"alter table t add index idx2 (b,a)", false, []tikvrpc.CmdType{tikvrpc.CmdScan, tikvrpc.CmdCop}}, - {"alter table t drop index idx2", false, []tikvrpc.CmdType{}}, + {"create database test_db0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"create table test_db0.test_t0 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"create table test_db0.test_t1 (a int, b int, index idx(a))", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"alter table test_db0.test_t0 add column c int", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"drop table test_db0.test_t0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"drop database test_db0", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, + {"alter table t modify column b double", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdScan, tikvrpc.CmdCop}}, + {"alter table t add index idx2 (b,a)", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit, tikvrpc.CmdScan, tikvrpc.CmdCop}}, + {"alter table t drop index idx2", false, []tikvrpc.CmdType{tikvrpc.CmdPrewrite, tikvrpc.CmdCommit}}, // Test for transaction {"begin", false, nil}, diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index f6be6534be7fd..bd1c68d664cdc 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -386,6 +386,11 @@ func (sc *StatementContext) InitSQLDigest(normalized string, digest *parser.Dige }) } +// ResetSQLDigest sets the normalized and digest for sql anyway, **DO NOT USE THIS UNLESS YOU KNOW WHAT YOU ARE DOING NOW**. +func (sc *StatementContext) ResetSQLDigest(s string) { + sc.digestMemo.normalized, sc.digestMemo.digest = parser.NormalizeDigest(s) +} + // GetPlanDigest gets the normalized plan and plan digest. func (sc *StatementContext) GetPlanDigest() (normalized string, planDigest *parser.Digest) { return sc.planNormalized, sc.planDigest From f9e93b6bc5ea6342fa39a5d53f6f58bfce970266 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 14:21:19 +0800 Subject: [PATCH 19/21] add comment --- ddl/job_table.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ddl/job_table.go b/ddl/job_table.go index 4009e8fca9b28..23cedfb8b1b56 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -130,6 +130,8 @@ func (d *ddl) getGeneralJob(sess *session) (*model.Job, error) { sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job where find_in_set(%s, schema_ids) != 0 and processing limit 1", strconv.Quote(strconv.FormatInt(job.SchemaID, 10))) return d.checkJobIsRunnable(sess, sql) } + // For general job, there is only 1 general worker to handle it, so at this moment the processing job must be reorg job and the reorg job must only contain one table id. + // So it's not possible the find_in_set("1,2", "1,2,3") occurs. sql := fmt.Sprintf("select job_id from mysql.tidb_ddl_job t1, (select table_ids from mysql.tidb_ddl_job where job_id = %d) t2 where processing and find_in_set(t1.table_ids, t2.table_ids) != 0", job.ID) return d.checkJobIsRunnable(sess, sql) }) From ebdb2c3dd7672409f756a0cae51c0062eb9d3570 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 15:20:13 +0800 Subject: [PATCH 20/21] fix test --- ddl/table_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/ddl/table_test.go b/ddl/table_test.go index 83a7c49ce5553..fcdc713539948 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -47,6 +47,7 @@ func testRenameTable( Type: model.ActionRenameTable, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{oldSchemaID, tblInfo.Name, oldSchemaName}, + CtxVars: []interface{}{[]int64{oldSchemaID, newSchemaID}, []int64{tblInfo.ID}}, } ctx.SetValue(sessionctx.QueryString, "skip") require.NoError(t, d.DoDDLJob(ctx, job)) From d8bfb403d6eeffbd1c50392b3ecc4957062db1e5 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 20 Jul 2022 16:22:10 +0800 Subject: [PATCH 21/21] remove 0 for schema version lock --- ddl/ddl.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 3f0215e9401b7..c2351b525337e 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -345,8 +345,8 @@ func (sv *schemaVersionManager) setSchemaVersion(job *model.Job, store kv.Storag func (sv *schemaVersionManager) lockSchemaVersion(jobID int64) { ownerID := sv.lockOwner.Load() // There may exist one job update schema version many times in multiple-schema-change, so we do not lock here again - // if they are the same job. jobID == 0 is a special one, it means we must get the lock. - if ownerID != jobID || jobID == 0 { + // if they are the same job. + if ownerID != jobID { sv.schemaVersionMu.Lock() sv.lockOwner.Store(jobID) }