From 52e9f138d9583685b50808f18267d2fd82de0838 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 29 Aug 2021 09:39:27 -0400 Subject: [PATCH 01/45] refine sync dml --- go.mod | 1 + syncer/job.go | 2 +- syncer/job_test.go | 4 +- syncer/syncer.go | 249 +++++++++++++++--------------------------- syncer/syncer_test.go | 5 +- 5 files changed, 96 insertions(+), 165 deletions(-) diff --git a/go.mod b/go.mod index ccfba6847b..1852d869b6 100644 --- a/go.mod +++ b/go.mod @@ -20,6 +20,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.14.3 github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d github.com/labstack/echo/v4 v4.4.0 + github.com/pingcap/br v5.2.0-alpha.0.20210714104733-65ae7dd3a2f2+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/dumpling v0.0.0-20210806043112-4e012e54ce89 github.com/pingcap/errors v0.11.5-0.20210513014640-40f9a1999b3b diff --git a/syncer/job.go b/syncer/job.go index 598439a327..fb0f5d383e 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -171,7 +171,7 @@ func newFlushJob() *job { // put queues into bucket to monitor them. func queueBucketName(queueID int) string { - return fmt.Sprintf("q_%d", queueID%defaultBucketCount) + return fmt.Sprintf("q_%d", queueID) } func dmlWorkerLagKey(queueID int) string { diff --git a/syncer/job_test.go b/syncer/job_test.go index a65da4b2f0..388b4769a3 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -116,8 +116,8 @@ func (t *testJobSuite) TestQueueBucketName(c *C) { c.Assert(name, Equals, "q_0") name = queueBucketName(8) - c.Assert(name, Equals, "q_0") + c.Assert(name, Equals, "q_8") name = queueBucketName(9) - c.Assert(name, Equals, "q_1") + c.Assert(name, Equals, "q_9") } diff --git a/syncer/syncer.go b/syncer/syncer.go index 5797ca6eaf..6a2e9d743f 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -43,6 +43,9 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" + "golang.org/x/sync/errgroup" + + brutils "github.com/pingcap/br/pkg/utils" "github.com/pingcap/dm/dm/config" common2 "github.com/pingcap/dm/dm/ctl/common" @@ -90,8 +93,8 @@ var ( maxPauseOrStopWaitTime = 10 * time.Second - adminQueueName = "admin queue" - defaultBucketCount = 8 + adminQueueName = "ddl queue" + dmlQueueName = "dml queue" ) // BinlogType represents binlog sync type. @@ -145,10 +148,10 @@ type Syncer struct { ddlDB *conn.BaseDB ddlDBConn *dbconn.DBConn - jobs []chan *job + dmlJobCh chan *job + ddlJobCh chan *job jobsClosed atomic.Bool jobsChanLock sync.Mutex - queueBucketMapping []string waitXIDJob atomic.Int64 isTransactionEnd bool waitTransactionLock sync.Mutex @@ -274,12 +277,10 @@ func (s *Syncer) GetSecondsBehindMaster() int64 { return s.secondsBehindMaster.Load() } -func (s *Syncer) newJobChans(count int) { +func (s *Syncer) newJobChans() { s.closeJobChans() - s.jobs = make([]chan *job, 0, count) - for i := 0; i < count; i++ { - s.jobs = append(s.jobs, make(chan *job, s.cfg.QueueSize)) - } + s.dmlJobCh = make(chan *job, s.cfg.QueueSize*s.cfg.WorkerCount) + s.ddlJobCh = make(chan *job, 10) s.jobsClosed.Store(false) } @@ -289,9 +290,8 @@ func (s *Syncer) closeJobChans() { if s.jobsClosed.Load() { return } - for _, ch := range s.jobs { - close(ch) - } + close(s.dmlJobCh) + close(s.ddlJobCh) s.jobsClosed.Store(true) } @@ -557,7 +557,7 @@ func (s *Syncer) reset() { s.streamerController.Close(s.tctx) } // create new job chans - s.newJobChans(s.cfg.WorkerCount + 1) + s.newJobChans() s.execError.Store(nil) s.setErrLocation(nil, nil, false) @@ -884,16 +884,8 @@ func (s *Syncer) updateReplicationLag(job *job, lagKey string) { s.secondsBehindMaster.Store(lag) } -func (s *Syncer) checkWait(job *job) bool { - if job.tp == ddl { - return true - } - - if s.checkpoint.CheckGlobalPoint() { - return true - } - - return false +func (s *Syncer) checkFlush() bool { + return s.checkpoint.CheckGlobalPoint() } func (s *Syncer) saveTablePoint(db, table string, location binlog.Location) { @@ -979,50 +971,46 @@ func (s *Syncer) addJob(job *job) error { s.updateReplicationLag(job, skipLagKey) case flush: s.addCount(false, adminQueueName, job.tp, 1, job.targetSchema, job.targetTable) - // ugly code addJob and sync, refine it later - s.jobWg.Add(s.cfg.WorkerCount) - for i := 0; i < s.cfg.WorkerCount; i++ { - startTime := time.Now() - s.jobs[i] <- job - // flush for every DML queue - metrics.AddJobDurationHistogram.WithLabelValues("flush", s.cfg.Name, s.queueBucketMapping[i], s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - } + s.jobWg.Add(1) + s.dmlJobCh <- job + startTime := time.Now() + metrics.AddJobDurationHistogram.WithLabelValues("flush", s.cfg.Name, dmlQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) s.jobWg.Wait() s.addCount(true, adminQueueName, job.tp, 1, job.targetSchema, job.targetTable) return s.flushCheckPoints() case ddl: - s.jobWg.Wait() s.addCount(false, adminQueueName, job.tp, 1, job.targetSchema, job.targetTable) s.jobWg.Add(1) - queueBucket = s.cfg.WorkerCount startTime := time.Now() - s.jobs[queueBucket] <- job + s.ddlJobCh <- job metrics.AddJobDurationHistogram.WithLabelValues("ddl", s.cfg.Name, adminQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) + s.jobWg.Wait() case insert, update, del: - s.jobWg.Add(1) queueBucket = int(utils.GenHashKey(job.key)) % s.cfg.WorkerCount - s.addCount(false, s.queueBucketMapping[queueBucket], job.tp, 1, job.targetSchema, job.targetTable) + s.addCount(false, dmlQueueName, job.tp, 1, job.targetSchema, job.targetTable) startTime := time.Now() s.tctx.L().Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", job.key)) - s.jobs[queueBucket] <- job + s.dmlJobCh <- job s.isTransactionEnd = false failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { s.tctx.L().Info("receive dml job", zap.Any("dml job", job)) time.Sleep(100 * time.Millisecond) }) - metrics.AddJobDurationHistogram.WithLabelValues(job.tp.String(), s.cfg.Name, s.queueBucketMapping[queueBucket], s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) + metrics.AddJobDurationHistogram.WithLabelValues(job.tp.String(), s.cfg.Name, dmlQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) } // nolint:ifshort - wait := s.checkWait(job) + flush := s.checkFlush() failpoint.Inject("flushFirstJob", func() { if waitJobsDone { s.tctx.L().Info("trigger flushFirstJob") waitJobsDone = false - wait = true + flush = true } }) - if wait { + if flush { + s.jobWg.Add(1) + s.dmlJobCh <- newFlushJob() s.jobWg.Wait() s.c.reset() } @@ -1069,7 +1057,7 @@ func (s *Syncer) addJob(job *job) error { } } - if wait { + if flush || job.tp == ddl { // interrupted after save checkpoint and before flush checkpoint. failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err := handleFlushCheckpointStage(4, val.(int), "before flush checkpoint") @@ -1285,31 +1273,11 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. } } -// DML synced in batch by one worker. -func (s *Syncer) syncDML( - tctx *tcontext.Context, queueBucket string, db *dbconn.DBConn, jobChan chan *job, workerLagKey string) { - defer s.wg.Done() - - idx := 0 - count := s.cfg.Batch - jobs := make([]*job, 0, count) - // db_schema->db_table->opType - tpCnt := make(map[string]map[string]map[opType]int64) - queueID := strings.TrimPrefix(workerLagKey, "lag_") - - // clearF is used to reset job queue. - clearF := func() { - for i := 0; i < idx; i++ { - s.jobWg.Done() - } - idx = 0 - jobs = jobs[0:0] - // clear tpCnt map - tpCnt = make(map[string]map[string]map[opType]int64) - } - +func (s *Syncer) executeDML(tctx *tcontext.Context, jobs []*job) func(uint64) error { // successF is used to calculate lag metric and q/tps. - successF := func() { + successF := func(queueID int, jobs []*job) { + queueBucket := queueBucketName(queueID) + workerLagKey := dmlWorkerLagKey(queueID) if len(jobs) > 0 { // NOTE: we can use the first job of job queue to calculate lag because when this job committed, // every event before this job's event in this queue has already commit. @@ -1328,35 +1296,43 @@ func (s *Syncer) syncDML( } else { s.updateReplicationLag(nil, workerLagKey) } - // calculate qps - for dbSchema, tableM := range tpCnt { - for dbTable, tpM := range tableM { - for tpName, cnt := range tpM { - s.addCount(true, queueBucket, tpName, cnt, dbSchema, dbTable) - } - } + + for _, sqlJob := range jobs { + s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetSchema, sqlJob.targetTable) } metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket).Observe(float64(len(jobs))) } - fatalF := func(affected int, err error) { + fatalF := func(job *job, err error) { s.execError.Store(err) if !utils.IsContextCanceledError(err) { - err = s.handleEventError(err, jobs[affected].startLocation, jobs[affected].currentLocation, false, "") + err = s.handleEventError(err, job.startLocation, job.currentLocation, false, "") s.runFatalChan <- unit.NewProcessError(err) } - clearF() } - executeSQLs := func() (int, error) { + executeSQLs := func(workerID uint64) (err error) { + var ( + queueID = int(workerID - 1) + affect int + db = s.toDBConns[queueID] + ) + + defer func() { + if err == nil { + successF(queueID, jobs) + } else { + fatalF(jobs[affect], err) + } + }() + if len(jobs) == 0 { - return 0, nil + return nil } - failpoint.Inject("failSecondJob", func() { if failExecuteSQL && failOnce.CAS(false, true) { s.tctx.L().Info("trigger failSecondJob") - failpoint.Return(0, terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock")) + failpoint.Return(terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock")) } }) @@ -1373,93 +1349,55 @@ func (s *Syncer) syncDML( // use background context to execute sqls as much as possible ctctx, cancel := tctx.WithTimeout(maxDMLExecutionDuration) defer cancel() - affect, err := db.ExecuteSQL(ctctx, queries, args...) + affect, err = db.ExecuteSQL(ctctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { s.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") } }) - return affect, err + return err } + return executeSQLs +} - var err error - var affect int - tickerInterval := waitTime - failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { - t := val.(int) - tickerInterval = time.Duration(t) * time.Second - tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) - }) - timer := time.NewTimer(tickerInterval) - defer timer.Stop() +// DML synced in batch by one worker. +func (s *Syncer) syncDML(tctx *tcontext.Context) { + defer s.wg.Done() - for { - // resets the time interval for each loop to prevent a certain amount of time being spent on the previous ticker - // execution to `executeSQLs` resulting in the next ticker not waiting for the full waitTime. - if !timer.Stop() { - select { - case <-timer.C: - default: - } - } - timer.Reset(tickerInterval) - failpoint.Inject("noJobInQueueLog", func() { - tctx.L().Debug("timer Reset", - zap.String("workerLagKey", workerLagKey), - zap.Duration("tickerInterval", tickerInterval), - zap.Int64("current ts", time.Now().Unix())) - }) + allJobs := make([]*job, 0, s.cfg.Batch) + dmlWorkerPool := brutils.NewWorkerPool(uint(s.cfg.WorkerCount), "dml_worker") + eg := new(errgroup.Group) + for { select { - case sqlJob, ok := <-jobChan: - metrics.QueueSizeGauge.WithLabelValues(s.cfg.Name, queueID, s.cfg.SourceID).Set(float64(len(jobChan))) + case sqlJob, ok := <-s.dmlJobCh: if !ok { - if len(jobs) > 0 { - tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) + if len(allJobs) > 0 { + tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) } return } - idx++ if sqlJob.tp != flush && len(sqlJob.sql) > 0 { - jobs = append(jobs, sqlJob) - if _, ok := tpCnt[sqlJob.targetSchema]; !ok { - tpCnt[sqlJob.targetSchema] = make(map[string]map[opType]int64) - } - if _, ok := tpCnt[sqlJob.targetSchema][sqlJob.targetTable]; !ok { - tpCnt[sqlJob.targetSchema][sqlJob.targetTable] = make(map[opType]int64) - } - tpCnt[sqlJob.targetSchema][sqlJob.targetTable][sqlJob.tp]++ + allJobs = append(allJobs, sqlJob) } - if idx >= count || sqlJob.tp == flush { - affect, err = executeSQLs() - if err != nil { - fatalF(affect, err) - continue - } - successF() - clearF() + if len(allJobs) < s.cfg.Batch && sqlJob.tp != flush { + continue } - case <-timer.C: - if len(jobs) > 0 { - failpoint.Inject("syncDMLTicker", func() { - tctx.L().Info("job queue not full, executeSQLs by ticker") - }) - affect, err = executeSQLs() - if err != nil { - fatalF(affect, err) - continue + + dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, allJobs)) + if sqlJob.tp == flush { + if err := eg.Wait(); err != nil { + tctx.L().Error("flush dml jobs", log.ShortError(err)) } - successF() - clearF() - } else { - failpoint.Inject("noJobInQueueLog", func() { - tctx.L().Debug("no job in queue, update lag to zero", zap.String( - "workerLagKey", workerLagKey), zap.Int64("current ts", time.Now().Unix())) - }) - // update lag metric even if there is no job in the queue - s.updateReplicationLag(nil, workerLagKey) + s.jobWg.Done() + } + allJobs = allJobs[0:0] + case <-time.After(waitTime): + if len(allJobs) > 0 && dmlWorkerPool.HasWorker() { + dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, allJobs)) + allJobs = allJobs[0:0] } } } @@ -1612,27 +1550,20 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } } - s.queueBucketMapping = make([]string, 0, s.cfg.WorkerCount+1) // before starting syncDML, we should initialise the workerLagMap to prevent data races. // for example, while thread 1 is setting the key of s.workerLagMap // thread-2 might be calling s.updateReplicationLag( to get the key from s.workerLagMap) for i := 0; i < s.cfg.WorkerCount; i++ { s.workerLagMap[dmlWorkerLagKey(i)] = atomic.NewInt64(0) } - for i := 0; i < s.cfg.WorkerCount; i++ { - s.wg.Add(1) - name := queueBucketName(i) - s.queueBucketMapping = append(s.queueBucketMapping, name) - workerLagKey := dmlWorkerLagKey(i) - go func(i int, name, workerLagKey string) { - s.syncDML(tctx, name, s.toDBConns[i], s.jobs[i], workerLagKey) - }(i, name, workerLagKey) - } + s.wg.Add(1) + go func() { + s.syncDML(tctx) + }() - s.queueBucketMapping = append(s.queueBucketMapping, adminQueueName) s.wg.Add(1) go func() { - s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.jobs[s.cfg.WorkerCount]) + s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.ddlJobCh) }() s.wg.Add(1) diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index ea9aaf700c..5086ea2314 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -987,14 +987,13 @@ func (s *testSyncerSuite) TestCasuality(c *C) { s.cfg.WorkerCount = 1 syncer := NewSyncer(s.cfg, nil) - syncer.jobs = []chan *job{make(chan *job, 1)} - syncer.queueBucketMapping = []string{"queue_0", adminQueueName} + syncer.dmlJobCh = make(chan *job, 1) var wg sync.WaitGroup wg.Add(1) go func() { defer wg.Done() - job := <-syncer.jobs[0] + job := <-syncer.dmlJobCh c.Assert(job.tp, Equals, flush) syncer.jobWg.Done() }() From 96008ac4066ec208cc505dc467a492c2064a5a7a Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 00:59:13 +0800 Subject: [PATCH 02/45] fix bug --- syncer/syncer.go | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 6a2e9d743f..63f8e300c6 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -956,7 +956,7 @@ func (s *Syncer) addJob(job *job) error { } }) - if waitXIDStatus(s.waitXIDJob.Load()) == waitComplete { + if waitXIDStatus(s.waitXIDJob.Load()) == waitComplete && job.tp != flush { s.tctx.L().Info("All jobs is completed before syncer close, the coming job will be reject", zap.Any("job", job)) return nil } @@ -1425,7 +1425,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { s.waitTransactionLock.Lock() if s.isTransactionEnd { s.waitXIDJob.Store(int64(waitComplete)) - s.jobWg.Wait() tctx.L().Info("the last job is transaction end, done directly") runCancel() s.waitTransactionLock.Unlock() @@ -1439,7 +1438,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Info("received syncer's done") case <-time.After(maxPauseOrStopWaitTime): tctx.L().Info("wait transaction end timeout") - s.jobWg.Wait() runCancel() } } @@ -1598,7 +1596,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { err = terror.ErrSyncerUnitPanic.Generate(err1) } - s.jobWg.Wait() var ( err2 error exitSafeModeLoc binlog.Location @@ -1609,15 +1606,17 @@ func (s *Syncer) Run(ctx context.Context) (err error) { exitSafeModeLoc = savedGlobalLastLocation.Clone() } s.checkpoint.SaveSafeModeExitPoint(&exitSafeModeLoc) - if err2 = s.execError.Load(); err2 != nil && (terror.ErrDBExecuteFailed.Equal(err2) || terror.ErrDBUnExpect.Equal(err2)) { - err2 = s.checkpoint.FlushSafeModeExitPoint(s.tctx) - } else { - err2 = s.flushCheckPoints() + + // flush all jobs before exit + if err2 = s.flushJobs(); err2 != nil { + tctx.L().Warn("failed to flush jobs when exit task", zap.Error(err2)) } - if err2 != nil { - tctx.L().Warn("failed to flush checkpoints when exit task", zap.Error(err2)) - } else { - tctx.L().Info("flush checkpoints when exit task") + + // if any execute error, flush safemode exit point + if err2 = s.execError.Load(); err2 != nil && (terror.ErrDBExecuteFailed.Equal(err2) || terror.ErrDBUnExpect.Equal(err2)) { + if err2 = s.checkpoint.FlushSafeModeExitPoint(s.tctx); err2 != nil { + tctx.L().Warn("failed to flush safe mode checkpoints when exit task", zap.Error(err2)) + } } }() From b54a927c11ad7e26ca6537b4af78a7248fc997ff Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 29 Aug 2021 23:11:38 -0400 Subject: [PATCH 03/45] fix data race --- syncer/syncer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 63f8e300c6..2ffb201cc2 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1393,7 +1393,7 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { } s.jobWg.Done() } - allJobs = allJobs[0:0] + allJobs = make([]*job, 0, s.cfg.Batch) case <-time.After(waitTime): if len(allJobs) > 0 && dmlWorkerPool.HasWorker() { dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, allJobs)) From 53de6ceacafdff92211b4cedb94322816422ca88 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 29 Aug 2021 23:28:28 -0400 Subject: [PATCH 04/45] fix data race --- syncer/syncer.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 2ffb201cc2..70e2287c44 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1386,7 +1386,8 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { continue } - dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, allJobs)) + jobs := allJobs + dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, jobs)) if sqlJob.tp == flush { if err := eg.Wait(); err != nil { tctx.L().Error("flush dml jobs", log.ShortError(err)) From 22dad39f35de80ffc89d60dc215ee3411176d8ca Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 00:56:35 -0400 Subject: [PATCH 05/45] fix test --- syncer/syncer.go | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 70e2287c44..ee86e9d02d 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1369,6 +1369,12 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { dmlWorkerPool := brutils.NewWorkerPool(uint(s.cfg.WorkerCount), "dml_worker") eg := new(errgroup.Group) + failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { + t := val.(int) + waitTime = time.Duration(t) * time.Second + tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) + }) + for { select { case sqlJob, ok := <-s.dmlJobCh: @@ -1396,9 +1402,19 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { } allJobs = make([]*job, 0, s.cfg.Batch) case <-time.After(waitTime): - if len(allJobs) > 0 && dmlWorkerPool.HasWorker() { - dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, allJobs)) - allJobs = allJobs[0:0] + if len(allJobs) > 0 { + if dmlWorkerPool.HasWorker() { + jobs := allJobs + dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, jobs)) + allJobs = make([]*job, 0, s.cfg.Batch) + } + } else { + // waiting #2060 + failpoint.Inject("noJobInQueueLog", func() { + tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) + }) + // update lag metric even if there is no job in the queue + // s.updateReplicationLag(nil, workerLagKey) } } } From 9606c90939032b2dc2bc3541826f5ab9d4a9a369 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 01:24:24 -0400 Subject: [PATCH 06/45] update --- .golangci.yml | 7 ++ go.mod | 30 +++-- go.sum | 306 ++++++++++++++++++++++++++++++++++++++--------- syncer/syncer.go | 2 +- 4 files changed, 270 insertions(+), 75 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index d91ec8cfba..31cb46f582 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -114,6 +114,13 @@ issues: - EXC0002 # golint - EXC0003 + exclude-rules: + # TODO Remove it. + # Exclude some staticcheck messages + - linters: + - staticcheck + text: "SA1019:" + # Fix found issues (if it's supported by the linter) fix: false diff --git a/go.mod b/go.mod index 1852d869b6..c6a37e51ad 100644 --- a/go.mod +++ b/go.mod @@ -11,23 +11,22 @@ require ( github.com/dustin/go-humanize v1.0.0 github.com/getkin/kin-openapi v0.66.0 github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929 - github.com/go-sql-driver/mysql v1.5.0 + github.com/go-sql-driver/mysql v1.6.0 github.com/gogo/gateway v1.1.0 github.com/gogo/protobuf v1.3.2 - github.com/golang/mock v1.4.4 - github.com/golang/protobuf v1.3.4 - github.com/google/uuid v1.1.1 + github.com/golang/mock v1.6.0 + github.com/golang/protobuf v1.5.2 + github.com/google/uuid v1.1.2 github.com/grpc-ecosystem/grpc-gateway v1.14.3 github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d github.com/labstack/echo/v4 v4.4.0 - github.com/pingcap/br v5.2.0-alpha.0.20210714104733-65ae7dd3a2f2+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 - github.com/pingcap/dumpling v0.0.0-20210806043112-4e012e54ce89 + github.com/pingcap/dumpling v0.0.0-20210827102405-445b35d0f4b7 github.com/pingcap/errors v0.11.5-0.20210513014640-40f9a1999b3b github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd - github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 - github.com/pingcap/parser v0.0.0-20210728071818-915a01041dbb - github.com/pingcap/tidb v1.1.0-beta.0.20210730074105-0ed15511f8d0 + github.com/pingcap/log v0.0.0-20210818144256-6455d4a4c6f9 + github.com/pingcap/parser v0.0.0-20210823033705-7a7940986a30 + github.com/pingcap/tidb v1.1.0-beta.0.20210829194212-04e144aeb58a github.com/pingcap/tidb-tools v5.2.0-alpha.0.20210727084616-915b22e4d42c+incompatible github.com/prometheus/client_golang v1.5.1 github.com/rakyll/statik v0.1.6 @@ -40,18 +39,17 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285 github.com/tidwall/gjson v1.6.5 github.com/tidwall/sjson v1.1.2 - github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d + github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d github.com/unrolled/render v1.0.1 go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b go.uber.org/atomic v1.9.0 - go.uber.org/zap v1.18.1 - golang.org/x/exp v0.0.0-20200513190911-00229845015e // indirect - golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4 + go.uber.org/zap v1.19.0 + golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c - golang.org/x/sys v0.0.0-20210510120138-977fb7262007 + golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba - google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63 - google.golang.org/grpc v1.27.1 + google.golang.org/genproto v0.0.0-20200305110556-506484158171 + google.golang.org/grpc v1.29.1 gopkg.in/yaml.v2 v2.4.0 ) diff --git a/go.sum b/go.sum index d5cc09b18e..39f767393e 100644 --- a/go.sum +++ b/go.sum @@ -7,8 +7,9 @@ cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTj cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0 h1:MZQCQQaRwOrAcuKjiHWHrgKykt4fZyuwF2dtiG3fGW8= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0 h1:3ithwDMr7/3vpAMXiH+ZQnYbuIsh+OPhUPMFC9enmn0= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0 h1:xE3CPsOgttP4ACBePh79zTKALtXwn/Edhcr16R5hMWU= @@ -25,31 +26,43 @@ cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0Zeo cloud.google.com/go/storage v1.6.0 h1:UDpwYIwla4jHGzZJaEJYx1tOejbgSoNqsAfHAUYe2r8= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= +github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= github.com/DATA-DOG/go-sqlmock v1.5.0 h1:Shsta01QNfFxHCfpW6YH2STWB0MudeXXEWMr20OEh60= github.com/DATA-DOG/go-sqlmock v1.5.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/HdrHistogram/hdrhistogram-go v0.9.0 h1:dpujRju0R4M/QZzcnR1LH1qm+TVG3UzkWdp5tH1WMcg= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/HdrHistogram/hdrhistogram-go v0.9.0/go.mod h1:nxrse8/Tzg2tg3DZcZjm6qEclQKK70g0KxO61gFFZD4= +github.com/HdrHistogram/hdrhistogram-go v1.1.0 h1:6dpdDPTRoo78HxAJ6T1HfMiKSnqhgRRqzCuPshRkQ7I= +github.com/HdrHistogram/hdrhistogram-go v1.1.0/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk= github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= +github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= +github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= +github.com/ReneKroon/ttlcache/v2 v2.3.0/go.mod h1:zbo6Pv/28e21Z8CzzqgYRArQYGYtjONRxaAKGxzQvG4= +github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/VividCortex/ewma v1.1.1 h1:MnEK4VOv6n0RSY4vtRe3h11qjxL3+t0B8yOL8iMXdcM= github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= github.com/VividCortex/mysqlerr v0.0.0-20200629151747-c28746d985dd/go.mod h1:f3HiCrHjHBdcm6E83vGaXh1KomZMA2P6aeo3hKx/wg0= github.com/Xeoncross/go-aesctr-with-hmac v0.0.0-20200623134604-12b17a7ff502/go.mod h1:pmnBM9bxWSiHvC/gSWunUIyDvGn33EkP2CUjxFKtTTM= +github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT0m65DJ+Wo= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= github.com/apache/thrift v0.0.0-20181112125854-24918abba929/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714 h1:Jz3KVLYY5+JO7rDiX0sAuRGtuv2vG01r17Y9nLMWNUw= @@ -60,6 +73,7 @@ github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5 github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go v1.35.3 h1:r0puXncSaAfRt7Btml2swUo74Kao+vKhO3VLjwDjK54= github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= +github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -82,8 +96,10 @@ github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+ github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chaos-mesh/go-sqlsmith v0.0.0-20210730091118-5201af078b27 h1:ApShE13PMBO32hnXDgqCGziFeVdIT3NPNTkpot/8zq4= github.com/chaos-mesh/go-sqlsmith v0.0.0-20210730091118-5201af078b27/go.mod h1:47upat8Bv8e4ceTZRrDe9Mv0qacY9w8alFJd9Zb5TKY= -github.com/cheggaaa/pb/v3 v3.0.4 h1:QZEPYOj2ix6d5oEg63fbHmpolrnNiwjUsk+h74Yt4bM= github.com/cheggaaa/pb/v3 v3.0.4/go.mod h1:7rgWxLrAUcFMkvJuv09+DYi7mMUYi8nO9iOWcvGJPfw= +github.com/cheggaaa/pb/v3 v3.0.8 h1:bC8oemdChbke2FHIIGy9mn4DPJ2caZYQnfbRqwmdCoA= +github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= +github.com/cheynewallace/tabby v1.1.1/go.mod h1:Pba/6cUL8uYqvOc9RkyvFbHGrQ9wShyrn6/S/1OYVys= github.com/chzyer/logex v1.1.10 h1:Swpa1K6QvQznwJRcfTfQJmTE72DqScAa40E+fbHEXEE= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e h1:fY5BOSpyZCqRo5OhCuC+XN+r/bBCmeuuJtjz+bCNIf8= @@ -91,9 +107,22 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1 h1:q763qf9huN11kDQavWsoZXJNW3xEE4JJyHa5Q25/sd8= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/codahale/hdrhistogram v0.9.0/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/cockroachdb/datadriven v1.0.0 h1:uhZrAfEayBecH2w2tZmhe20HJ7hDvrrA4x2Bg9YdZKM= +github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= +github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= +github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= +github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5/go.mod h1:JXfQr3d+XO4bL1pxGwKKo09xylQSdZ/mpZ9b2wfVcPs= +github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= +github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= +github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64 h1:W1SHiII3e0jVwvaQFglwu3kS9NLxOeTpvik7MbKCyuQ= github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= @@ -136,6 +165,7 @@ github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/deepmap/oapi-codegen v1.8.2 h1:SegyeYGcdi0jLLrpbCMoJxnUUn8GBXHsvr4rbzjuhfU= github.com/deepmap/oapi-codegen v1.8.2/go.mod h1:YLgSKSDv/bZQB7N4ws6luhozi3cEdRktEqrX88CvjIw= github.com/denisenkom/go-mssqldb v0.0.0-20191124224453-732737034ffd/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= +github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/ristretto v0.0.1 h1:cJwdnj42uV8Jg4+KLrYovLiCgIfz9wtWm6E6KA+1tLs= github.com/dgraph-io/ristretto v0.0.1/go.mod h1:T40EBc7CJke8TkpiYfGGKAeFjSaxuFXhuXRyumBd6RE= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= @@ -153,21 +183,34 @@ github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaI github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0= +github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= +github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= +github.com/felixge/httpsnoop v1.0.1 h1:lvB5Jl89CsZtGIWuTcDM1E/vkVs49/Ml7JJe07l8SPQ= +github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= +github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= +github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsouza/fake-gcs-server v1.17.0/go.mod h1:D1rTE4YCyHFNa99oyJJ5HyclvN/0uQR+pM/VdlL83bw= github.com/fsouza/fake-gcs-server v1.19.0 h1:XyaGOlqo+R5sjT03x2ymk0xepaQlgwhRLTT2IopW0zA= github.com/fsouza/fake-gcs-server v1.19.0/go.mod h1:JtXHY/QzHhtyIxsNfIuQ+XgHtRb5B/w8nqbL5O8zqo0= +github.com/fzipp/gocyclo v0.3.1/go.mod h1:DJHO6AUmbdqj2ET4Z9iArSuwWgYDRryYt2wASxc7x3E= +github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= github.com/getkin/kin-openapi v0.61.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= github.com/getkin/kin-openapi v0.66.0 h1:GNEnjUYK5j5TR8CrXBQ5Nao9kVd7kPpoTr0Y0q4eaSA= github.com/getkin/kin-openapi v0.66.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= +github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= @@ -177,9 +220,11 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= +github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= github.com/go-chi/chi v4.0.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= github.com/go-chi/chi/v5 v5.0.0/go.mod h1:BBug9lr0cqtdAhsu6R4AAdvufI0/XBzAQSsUqJpoZOs= github.com/go-echarts/go-echarts v1.0.0/go.mod h1:qbmyAb/Rl1f2w7wKba1D4LoNq4U164yO4/wedFbcWyo= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -188,6 +233,7 @@ github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2 github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= +github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929 h1:cjv3hcFlmma66+fYTvhHt/sbwZWWJs09iv2ipVRIr0I= github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= @@ -210,15 +256,20 @@ github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= +github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.3.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= +github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= +github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= +github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= +github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/goccy/go-graphviz v0.0.5/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= github.com/gogo/gateway v1.1.0 h1:u0SuhL9+Il+UbjM9VIE3ntfRujKbvVpFvNB4HbjeVQ0= github.com/gogo/gateway v1.1.0/go.mod h1:S7rR8FRQyG3QFESeSv4l2WnsyzlCLG0CzBbUUo/mbic= +github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -227,9 +278,9 @@ github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zV github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -241,22 +292,33 @@ github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4 h1:87PNWwrRvUSnqS4dlcBU/ftvOIBep4sYuBLlh6rX2wk= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf h1:gFVkHXmVAhEbxZVDln5V9GKrLaluNoFHDbrZwAWZgws= github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= +github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= +github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -265,8 +327,11 @@ github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6 h1:BKbKCqvP6I+rmFHt06ZmyQtvB8xAkWdhFyr0ZUNZcxQ= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= @@ -274,18 +339,23 @@ github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OI github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9 h1:K+lX49/3eURCE1IjlaZN//u6c+9nfDAMnyQ9E2dsJbY= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9 h1:2tft2559dNwKl2znYB58oVTql0grRB+Ml3LWIBbc4WM= +github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/gorilla/handlers v1.4.2 h1:0QniY0USkHQ1RGCLfKxeNHK9bkDHGRYGNDFBCS+YARg= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= +github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= +github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= @@ -307,22 +377,32 @@ github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtg github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/hypnoglow/gormzap v0.3.0/go.mod h1:5Wom8B7Jl2oK0Im9hs6KQ+Kl92w4Y7gKCrj66rhyvw0= github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 h1:VHgatEHNcBFEB7inlalqfNqw65aNkM1lGX2yt3NmbS8= github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334/go.mod h1:SK73tn/9oHe+/Y0h39VT4UCxmurVJkR5NA7kMEAOgSE= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= +github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= +github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= +github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jedib0t/go-pretty/v6 v6.2.2/go.mod h1:+nE9fyyHGil+PuISTCrp7avEdo6bqoMwqZnuiK2r2a0= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= +github.com/jinzhu/now v1.1.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= +github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= @@ -344,23 +424,35 @@ github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/u github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= +github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= +github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= +github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d h1:cVtBfNW5XTHiKQe7jDaDBSh/EVM4XLPutLAGboIXuM0= github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d/go.mod h1:P2viExyCEfeWGU259JnaQ34Inuec4R38JCyBx2edgD0= +github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= +github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= +github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= +github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.10.5 h1:7q6vHIqubShURwQz8cQK6yIe/xC3IF0Vm7TGfqjewrc= github.com/klauspost/compress v1.10.5/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/compress v1.11.7 h1:0hzRabrMN4tSTvMfnL3SCv1ZGeAP23ynzodBgaHeMeg= +github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/cpuid v1.2.1 h1:vJi+O/nMdFt0vqm8NZBI6wzALWdA2X+egi0ogNyrC/w= github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -371,6 +463,7 @@ github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4FW1e6jwpg= github.com/labstack/echo/v4 v4.4.0 h1:rblX1cN6T4LvUW9ZKMPZ17uPl/Dc8igP7ZmjGHZoj4A= github.com/labstack/echo/v4 v4.4.0/go.mod h1:PvmtTvhVqKDzDQy4d3bWzPjZLzom4iQbAZy2sgZ/qI8= @@ -401,15 +494,23 @@ github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHX github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.10 h1:CoZ3S2P7pvtP45xOtBw+/mDL2z0RKI576gSkzRRpdGg= +github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.10/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= +github.com/mattn/go-runewidth v0.0.12 h1:Y41i/hVW3Pgwr8gV+J23B9YEY0zxjptBuCWEaxmAOow= +github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= +github.com/mattn/go-sqlite3 v1.14.5/go.mod h1:WVKg1VTActs4Qso6iwGbiFih2UIHo0ENGwNd0Lj+XmI= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= +github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= +github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= +github.com/minio/sio v0.3.0/go.mod h1:8b0yPp2avGThviy/+OCJBI6OMpvxoUuiLvE6F1lebhw= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -420,7 +521,11 @@ github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= +github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= +github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/ncw/directio v1.0.4 h1:CojwI07mCEmRkajgx42Pf8jyCwTs1ji9/Ij9/PJG12k= github.com/ncw/directio v1.0.4/go.mod h1:CKGdcN7StAaqjT7Qack3lAXeX4pjnyc46YeqZH1yWVY= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5/go.mod h1:jpp1/29i3P1S/RLdc7JQKbRpFeM1dOBd8T9ki5s+AY8= @@ -431,19 +536,27 @@ github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= +github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.13.0 h1:M76yO2HkZASFjXL0HSoZJ1AYEmQxNJmY41Jx1zNUq1Y= +github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= @@ -451,19 +564,18 @@ github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCr github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= -github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= -github.com/pingcap/br v5.1.0-alpha.0.20210601094737-6cb0c4abc210+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= -github.com/pingcap/br v5.2.0-alpha.0.20210714104733-65ae7dd3a2f2+incompatible h1:KIh0fJHRB44kRPccXl7V0f5mtzXmR+0mM8vNciWst8g= +github.com/pingcap/badger v1.5.1-0.20210828064554-21c3176422cb h1:JiVgYjg2k1iDfRm1SeI+S34mtvOkqmdhPdJje34LsWw= +github.com/pingcap/badger v1.5.1-0.20210828064554-21c3176422cb/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= github.com/pingcap/br v5.2.0-alpha.0.20210714104733-65ae7dd3a2f2+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 h1:R8gStypOBmpnHEx1qi//SaqxJVI4inOqljg/Aj5/390= github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/dumpling v0.0.0-20210806043112-4e012e54ce89 h1:9YPEDjHfXt6M2ctq6QWD6Er9dRytyUE5RYX8ta5FHZY= -github.com/pingcap/dumpling v0.0.0-20210806043112-4e012e54ce89/go.mod h1:RRwOgPlm8tt3OaiG4hUgjpw1zCfnBETHu+vW67CsJ+k= +github.com/pingcap/dumpling v0.0.0-20210827102405-445b35d0f4b7 h1:s4hUNlRZKC69uUcmy32tK8l/eiZ6zYctDj8WQYAX4WQ= +github.com/pingcap/dumpling v0.0.0-20210827102405-445b35d0f4b7/go.mod h1:N5h2P9lA4wRLj9IX6enQfItwXfeeLG2QwjNv24bsicA= +github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= @@ -485,45 +597,50 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210722091755-91a52cd9e8db h1:PSW6P83KZi5WopPBiecU286PWMSl2rvxCBZT94iBX+I= github.com/pingcap/kvproto v0.0.0-20210722091755-91a52cd9e8db/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210806074406-317f69fb54b4 h1:4EUpHzPFHwleKkVALyMqQbQcNziPZvU+vhUT9Wzj93E= +github.com/pingcap/kvproto v0.0.0-20210806074406-317f69fb54b4/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 h1:k2BbABz9+TNpYRwsCCFS8pEEnFVOdbgEjL/kTlLuzZQ= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= +github.com/pingcap/log v0.0.0-20210818144256-6455d4a4c6f9 h1:6t7vOzOGF3/iz+wpcwu8N/+aoWTOMq2xc+Y0pYMJOhU= +github.com/pingcap/log v0.0.0-20210818144256-6455d4a4c6f9/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/parser v0.0.0-20210415081931-48e7f467fd74/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= -github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= -github.com/pingcap/parser v0.0.0-20210728071818-915a01041dbb h1:TX35KERCC2YumPK1MzUmqbgJSxbzjrM9CmfmFJTJNws= github.com/pingcap/parser v0.0.0-20210728071818-915a01041dbb/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= +github.com/pingcap/parser v0.0.0-20210823033705-7a7940986a30 h1:UPeHfUVa9AanMJJ1Sp6+WRWYazL4qHCDcYwP33L5ZUw= +github.com/pingcap/parser v0.0.0-20210823033705-7a7940986a30/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= -github.com/pingcap/tidb v1.1.0-beta.0.20210517044538-8ad868f801fc/go.mod h1:MTGiqcSY5WEi5LzxVW1dZ2BiMiALuVvHQ6zuDHQyL70= -github.com/pingcap/tidb v1.1.0-beta.0.20210730074105-0ed15511f8d0 h1:QkfWLth/N96KTcyPfkvVsr8XnK9YYgJrt4RmU6al2Tc= +github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 h1:7rvAtZe/ZUzOKzgriNPQoBNvleJXBk4z7L3Z47+tS98= +github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= github.com/pingcap/tidb v1.1.0-beta.0.20210730074105-0ed15511f8d0/go.mod h1:vvO2Sq9M60PKpJjroZtq72hMjc4ulDn6hlGedRKCeDY= +github.com/pingcap/tidb v1.1.0-beta.0.20210827045605-d4cadb53ef12/go.mod h1:l9fxVqsmjYB+Y8u/VjmiAtDkcX7vl+v5eMdUmcQKgdA= +github.com/pingcap/tidb v1.1.0-beta.0.20210829194212-04e144aeb58a h1:2gh5KjGJhaOiRF5JNmAvN2hq7KXkJS6DEqd+Vr+aBVs= +github.com/pingcap/tidb v1.1.0-beta.0.20210829194212-04e144aeb58a/go.mod h1:a0NVnhRRxnIB3EguntFC/3Ed7gggERWuTGUd5OwjA9s= github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= -github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v5.2.0-alpha.0.20210727084616-915b22e4d42c+incompatible h1:3UusiiE+cbOYzjMT3oxk2gN3lUT0ccjMiul51EEvZZc= github.com/pingcap/tidb-tools v5.2.0-alpha.0.20210727084616-915b22e4d42c+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pingcap/tipb v0.0.0-20210708040514-0f154bb0dc0f h1:q6WgGOeY+hbkvtKLyi6nAew7Ptl5vXyeI61VJuJdXnQ= github.com/pingcap/tipb v0.0.0-20210708040514-0f154bb0dc0f/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= github.com/prometheus/client_golang v1.5.1 h1:bdHYieyGlH+6OLEk2YQha8THib30KP0/yD0YH9m6xcA= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= @@ -536,12 +653,14 @@ github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7q github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1 h1:KOMtN28tlbam3/7ZKEYKHhKoJZYYj3gMH4uc62x7X7U= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= @@ -560,10 +679,12 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b h1:gQZ0qzfKHQIybLANtM3mBXNUtOfsCFXeTsnBqCsx1KM= github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= +github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -585,8 +706,11 @@ github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07 h1:oI+RNwuC9jF2g github.com/siddontang/go-log v0.0.0-20180807004314-8d05993dda07/go.mod h1:yFdBgwXP24JziuRl2NMUahT7nGLNOKi1SIiFxMttVD4= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= +github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -627,6 +751,7 @@ github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJ github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285 h1:uSDYjYejelKyceA6DiCsngFof9jAyeaSyX9XC5a1a7Q= github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= +github.com/thoas/go-funk v0.8.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= @@ -641,10 +766,12 @@ github.com/tidwall/pretty v1.0.2 h1:Z7S3cePv9Jwm1KwS0513MRaoUe3S01WPbLNV40pwWZU= github.com/tidwall/pretty v1.0.2/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tidwall/sjson v1.1.2 h1:NC5okI+tQ8OG/oyzchvwXXxRxCV/FVdhODbPKkQ25jQ= github.com/tidwall/sjson v1.1.2/go.mod h1:SEzaDwxiPzKzNfUEO4HbYF/m4UCSJDsGgNqsS1LvdoY= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210727120905-55155ad2e543 h1:Yi/Xn7NdbxicB/4Ve3myyqtEkqiVaAjWYUDTTETBdFg= github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210727120905-55155ad2e543/go.mod h1:LP8O6zZXAWKU781u1nt/v9nC0hxPipPxOxSoZT9Goqg= -github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d h1:K0XnvsnT6ofLDuM8Rt3PuFQO4p8bNraeHYstspD316g= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210824090536-16d902a3c7e5 h1:7CJYiW8gKiI3IQOQSAZyqZq0GxB+bmrnZgk9QNZ1cPo= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210824090536-16d902a3c7e5/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o= github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw= +github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d h1:AFm1Dzw+QRUevWRfrFp45CPPkuK/zdSWcfxI10z+WVE= +github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA= github.com/tklauser/go-sysconf v0.3.4 h1:HT8SVixZd3IzLdfs/xlpq0jeSfTX57g1v6wB1EuzV7M= github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= github.com/tklauser/numcpus v0.2.1 h1:ct88eFm+Q7m2ZfXJdan1xYoXKlmwsfP+k88q05KvlZc= @@ -658,8 +785,9 @@ github.com/uber-go/atomic v1.4.0 h1:yOuPqEq4ovnhEjpHmfFwsqBXDYbQeT6Nb0bwD6XnD5o= github.com/uber-go/atomic v1.4.0/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= -github.com/uber/jaeger-lib v2.4.0+incompatible h1:fY7QsGQWiCt8pajv4r7JEvmATdCVaWxXbjwyYwsNaLQ= github.com/uber/jaeger-lib v2.4.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= +github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= +github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.5-pre/go.mod h1:FwP/aQVg39TXzItUBMwnWp9T9gPQnXw4Poh4/oBQZ/0= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= @@ -672,16 +800,23 @@ github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0ob github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/urfave/negroni v1.0.0 h1:kIimOitoypq34K7TG7DUaJ9kq/N4Ofuwi1sjz0KipXc= +github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= +github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/fasttemplate v1.2.1 h1:TVEnxayobAdVkhQfrfes2IzOB6o+z4roRkPF52WA1u4= github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= +github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/msgpack/v5 v5.0.0-beta.1/go.mod h1:xlngVLeyQ/Qi05oQxhQ+oTuqa03RjMwMfk/7/TCs+QI= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= +github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xitongsys/parquet-go v1.5.1/go.mod h1:xUxwM8ELydxh4edHGegYq1pA8NnMKDx0K/GyB0o2bww= @@ -691,7 +826,11 @@ github.com/xitongsys/parquet-go v1.6.0/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXv github.com/xitongsys/parquet-go-source v0.0.0-20190524061010-2b72cbee77d5/go.mod h1:xxCx7Wpym/3QCo6JhujJX51dzSXrwmb0oH6FQb39SEA= github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0/go.mod h1:HYhIKsdns7xz80OgkbgJYrtQY7FjHWHKH6cvN7+czGE= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= +github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= +github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= +github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= @@ -706,8 +845,9 @@ go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3C go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.5 h1:dntmOdLpSpHlVqbW5Eay97DelsZHe+55D+xC6i0dDS0= +go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -716,7 +856,6 @@ go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.8.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q= go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= @@ -737,16 +876,19 @@ go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= -go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +go.uber.org/zap v1.19.0 h1:mZQZefskPPCMIBCSEH0v2/iUqqLrYtaeqwD6FUGUnFE= +go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -755,7 +897,10 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2 h1:It14KIkyBFYkHkwZ7k45minvA9aorojkyjGk9KJ5B/w= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= @@ -765,8 +910,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20200513190911-00229845015e h1:rMqLP+9XLy+LdbCXHjJHAmTfXCr93W7oruWA6Hq1Alc= golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= +golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/image v0.0.0-20200119044424-58c23975cae1/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -779,6 +924,8 @@ golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 h1:VLliZ0d+/avPrXXH+OakdXhpJuEoBZuwh1m2j7U6Iug= golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= @@ -800,6 +947,7 @@ golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -818,15 +966,16 @@ golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4 h1:4nGaVu0QrbjT/AK2PRLuQfQuh6DJve+pELhqTdAj3x0= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420 h1:a8jGStKg0XqKDlKqjLrXn0ioF5MH36pT7Z0BRTqLhbk= +golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -842,6 +991,7 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -862,12 +1012,16 @@ golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191128015809-6d18c012aee9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -877,19 +1031,21 @@ golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007 h1:gG67DSER+11cZvqIMb8S8bt0vZtiN6xWYARwirrOSfE= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c h1:F1jZWGFhYfh0Ci55sIpILtKKK8p3i2/krTr0H1rg74I= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -898,8 +1054,9 @@ golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3 golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -908,16 +1065,22 @@ golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524210228-3d17549cdc6b/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190606050223-4d9ae51c2468/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190611222205-d73e1c7e250b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= @@ -948,18 +1111,26 @@ golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.4 h1:cVngSRcfgyZCzys3KYOpCFa+4dqX/Oub9tAq00ttGVs= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= +golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= +gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= @@ -969,6 +1140,7 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.22.0 h1:J1Pl9P2lnmYFSJvgs70DKELqHNh8CNWXPbud4njEE2s= google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= @@ -977,6 +1149,7 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5 h1:tycE03LOZYQNhDpS27tcQdAzLCVMaj7QT2SXxebnpCM= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -995,9 +1168,11 @@ google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvx google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63 h1:YzfoEYWbODU5Fbt37+h7X16BWQbad7Q4S6gclTKFXM8= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171 h1:xes2Q2k+d/+YNXVw0FpZkIDJiaux4OVrRKXRAzH6A0U= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= @@ -1009,19 +1184,29 @@ google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRn google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1 h1:zvIju4sqAGvwKspUQOhwnpcqSbzi7/H6QomNNjTL4sk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= @@ -1031,6 +1216,7 @@ gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eR gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= gopkg.in/jcmturner/gokrb5.v7 v7.3.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= +gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= @@ -1048,22 +1234,26 @@ gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gorm.io/driver/mysql v1.0.6/go.mod h1:KdrTanmfLPPyAOeYGyG+UpDys7/7eeWT1zCq+oekYnU= +gorm.io/driver/sqlite v1.1.4/go.mod h1:mJCeTFr7+crvS+TRnWc5Z3UvwxUN1BGBLMrf5LA9DYw= +gorm.io/gorm v1.20.7/go.mod h1:0HFTzE/SqkGTzK6TlDPPQbAYCluiVvhzoA1+aVyzenw= +gorm.io/gorm v1.21.9/go.mod h1:F+OptMscr0P2F2qU97WT1WimdH9GaQPoDW7AYd5i2Y0= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= honnef.co/go/tools v0.2.0 h1:ws8AfbgTX3oIczLPNPCu5166oBg9ST2vNs0rcht+mDE= honnef.co/go/tools v0.2.0/go.mod h1:lPVVZ2BS5TfnjLyizF7o7hv7j9/L+8cZY2hLyjP9cGY= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= +moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= diff --git a/syncer/syncer.go b/syncer/syncer.go index ee86e9d02d..a6f3a5c5a0 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -45,7 +45,7 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" - brutils "github.com/pingcap/br/pkg/utils" + brutils "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/dm/dm/config" common2 "github.com/pingcap/dm/dm/ctl/common" From 57145f9e76a5f24fb18b98c9b2d5eaf42b14babf Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 01:25:45 -0400 Subject: [PATCH 07/45] update --- syncer/syncer.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index a6f3a5c5a0..d79a38b0a3 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -45,8 +45,6 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" - brutils "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/dm/dm/config" common2 "github.com/pingcap/dm/dm/ctl/common" "github.com/pingcap/dm/dm/pb" @@ -73,6 +71,7 @@ import ( onlineddl "github.com/pingcap/dm/syncer/online-ddl-tools" sm "github.com/pingcap/dm/syncer/safe-mode" "github.com/pingcap/dm/syncer/shardddl" + brutils "github.com/pingcap/tidb/br/pkg/utils" ) var ( From 90f17bc159eca77461ee8e1f51630cc90d1e5b01 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 02:56:10 -0400 Subject: [PATCH 08/45] fix test --- dm/ctl/common/util.go | 2 +- dm/worker/status.go | 2 +- syncer/metrics/metrics.go | 2 +- syncer/syncer.go | 3 ++- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dm/ctl/common/util.go b/dm/ctl/common/util.go index 0fa4522018..e98f624a38 100644 --- a/dm/ctl/common/util.go +++ b/dm/ctl/common/util.go @@ -25,7 +25,7 @@ import ( "sync" "time" - "github.com/golang/protobuf/jsonpb" + "github.com/gogo/protobuf/jsonpb" "github.com/golang/protobuf/proto" "github.com/pingcap/errors" "github.com/pingcap/failpoint" diff --git a/dm/worker/status.go b/dm/worker/status.go index fd1175a63c..d624962ed7 100644 --- a/dm/worker/status.go +++ b/dm/worker/status.go @@ -18,7 +18,7 @@ import ( "fmt" "sort" - "github.com/golang/protobuf/jsonpb" + "github.com/gogo/protobuf/jsonpb" "go.uber.org/zap" "github.com/pingcap/dm/dm/pb" diff --git a/syncer/metrics/metrics.go b/syncer/metrics/metrics.go index 07f26e286d..c0c14a1352 100644 --- a/syncer/metrics/metrics.go +++ b/syncer/metrics/metrics.go @@ -127,7 +127,7 @@ var ( Subsystem: "syncer", Name: "queue_size", Help: "remain size of the DML queue", - }, []string{"task", "queue_id", "source_id"}) + }, []string{"task", "source_id"}) BinlogPosGauge = metricsproxy.NewGaugeVec( prometheus.GaugeOpts{ diff --git a/syncer/syncer.go b/syncer/syncer.go index d79a38b0a3..0b2a7275ba 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" toolutils "github.com/pingcap/tidb-tools/pkg/utils" + brutils "github.com/pingcap/tidb/br/pkg/utils" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" @@ -71,7 +72,6 @@ import ( onlineddl "github.com/pingcap/dm/syncer/online-ddl-tools" sm "github.com/pingcap/dm/syncer/safe-mode" "github.com/pingcap/dm/syncer/shardddl" - brutils "github.com/pingcap/tidb/br/pkg/utils" ) var ( @@ -1377,6 +1377,7 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { for { select { case sqlJob, ok := <-s.dmlJobCh: + metrics.QueueSizeGauge.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Set(float64(len(s.dmlJobCh))) if !ok { if len(allJobs) > 0 { tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) From 34d49898b9e56c39901c9e912b3917b397517137 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 03:33:16 -0400 Subject: [PATCH 09/45] fix test --- tests/shardddl4_1/run.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/shardddl4_1/run.sh b/tests/shardddl4_1/run.sh index 2a10b644ea..fc39a3df22 100644 --- a/tests/shardddl4_1/run.sh +++ b/tests/shardddl4_1/run.sh @@ -509,10 +509,14 @@ function DM_145_CASE { run_sql_source2 "insert into ${shardddl1}.${tb2} values(300),(301),(302),(303),(304),(305);" run_sql_source1 "alter table ${shardddl1}.${tb1} engine=innodb;" + run_sql_source2 "alter table ${shardddl1}.${tb1} engine=innodb;" + run_sql_source2 "alter table ${shardddl1}.${tb2} engine=innodb;" - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status test" \ - "This type of ALTER TABLE is currently unsupported" 1 + run_sql_source1 "insert into ${shardddl1}.${tb1} values(400),(401),(402),(403),(404),(405);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(500),(501),(502),(503),(504),(505);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(600),(601),(602),(603),(604),(605);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml } # Defragment. @@ -820,7 +824,7 @@ function DM_152 { function run() { init_cluster init_database - start=131 + start=145 end=152 for i in $(seq -f "%03g" ${start} ${end}); do DM_${i} From 687d4e00542a8668c7305b9642397734532f4445 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 05:00:48 -0400 Subject: [PATCH 10/45] fix ut --- syncer/syncer_test.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index 5086ea2314..44b7a360a3 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -1211,6 +1211,10 @@ func (s *testSyncerSuite) TestRun(c *C) { ddl, "ALTER TABLE `test_1`.`t_3` ADD PRIMARY KEY(`id`, `name`)", nil, + }, { + flush, + "", + nil, }, } @@ -1271,6 +1275,10 @@ func (s *testSyncerSuite) TestRun(c *C) { del, "DELETE FROM `test_1`.`t_2` WHERE `id` = ? LIMIT 1", []interface{}{int32(3)}, + }, { + flush, + "", + nil, }, } @@ -1429,6 +1437,10 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { update, "UPDATE `test_1`.`t_1` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1", []interface{}{int32(1), "b", int32(2)}, + }, { + flush, + "", + nil, }, } From 0ae0f3304e1ea56671c9ad1d36cdbba67bc52a0d Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 30 Aug 2021 05:29:24 -0400 Subject: [PATCH 11/45] fix it --- syncer/syncer.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/syncer/syncer.go b/syncer/syncer.go index 0b2a7275ba..733004f1e9 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1403,6 +1403,9 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { allJobs = make([]*job, 0, s.cfg.Batch) case <-time.After(waitTime): if len(allJobs) > 0 { + failpoint.Inject("syncDMLTicker", func() { + tctx.L().Info("job queue not full, executeSQLs by ticker") + }) if dmlWorkerPool.HasWorker() { jobs := allJobs dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, jobs)) From 31639992db9c247943ec9e0e634597d67d3c94bf Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 31 Aug 2021 06:16:20 -0400 Subject: [PATCH 12/45] save work --- syncer/dml.go | 164 +++++++++++++++++++++++++++++++++++++++++++++ syncer/job.go | 5 +- syncer/job_test.go | 2 +- syncer/syncer.go | 80 ++++------------------ 4 files changed, 180 insertions(+), 71 deletions(-) diff --git a/syncer/dml.go b/syncer/dml.go index 106404c02b..c9309f1965 100644 --- a/syncer/dml.go +++ b/syncer/dml.go @@ -18,16 +18,22 @@ import ( "fmt" "strconv" "strings" + "sync" + "time" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/types" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/expression" "go.uber.org/zap" + "golang.org/x/sync/errgroup" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/utils" + "github.com/pingcap/dm/syncer/metrics" ) // genDMLParam stores pruned columns, data as well as the original columns, data, index. @@ -619,3 +625,161 @@ func pruneGeneratedColumnDML(ti *model.TableInfo, data [][]interface{}) ([]*mode } return cols, rows, nil } + +func (s *Syncer) compact(jobCh chan *job) (chan *job, chan *job) { + compactedCh := make(chan *job, s.cfg.QueueSize) + nonCompactedCh := make(chan *job, s.cfg.QueueSize) + go func() { + defer func() { + close(compactedCh) + close(nonCompactedCh) + }() + + for j := range jobCh { + // TODO: add compact logic + nonCompactedCh <- j + } + }() + return compactedCh, nonCompactedCh +} + +func (s *Syncer) resolveConflict(nonCompactedCh <-chan *job) []chan *job { + nonConflictChs := make([]chan *job, s.cfg.WorkerCount) + for i := range nonConflictChs { + nonConflictChs[i] = make(chan *job, s.cfg.QueueSize) + } + go func() { + defer func() { + for i := range nonConflictChs { + close(nonConflictChs[i]) + } + }() + + for j := range nonCompactedCh { + startTime := time.Now() + queueBucket := int(utils.GenHashKey(j.key)) % s.cfg.WorkerCount + if !s.cfg.DisableCausality && j.tp != flush { + if s.c.detectConflict(j.keys) { + s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) + nonConflictChs[queueBucket] <- newFlushJob() + s.c.reset() + } + // no error because we have called detectConfilict + // nolint:errcheck + s.c.add(j.keys) + j.key = s.c.get(j.key) + s.tctx.L().Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) + } + metrics.ConflictDetectDurationHistogram.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) + nonConflictChs[queueBucket] <- j + } + }() + return nonConflictChs +} + +func (s *Syncer) mergeDMLValues(compactedCh chan *job, nonConflictChs ...chan *job) (chan *job, []chan *job) { + var wg sync.WaitGroup + + compactedMergedCh := make(chan *job) + nonConflictMergedCh := make([]chan *job, len(nonConflictChs)) + + for i := 0; i < len(nonConflictMergedCh); i++ { + nonConflictMergedCh[i] = make(chan *job) + } + + mergeCompactedFunc := func(inCh chan *job, outCh chan *job) { + defer wg.Done() + for j := range inCh { + // TODO: add logic for merge + outCh <- j + } + } + mergeNonConflictFunc := func(inCh chan *job, outCh chan *job) { + defer wg.Done() + for j := range inCh { + // TODO: add logic for merge + outCh <- j + } + } + + wg.Add(len(nonConflictChs) + 1) + + go mergeCompactedFunc(compactedCh, compactedMergedCh) + + for i, nonConflictCh := range nonConflictChs { + go mergeNonConflictFunc(nonConflictCh, nonConflictMergedCh[i]) + } + + go func() { + wg.Wait() + close(compactedMergedCh) + for _, i := range nonConflictMergedCh { + close(i) + } + }() + return compactedMergedCh, nonConflictMergedCh +} + +func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs []chan *job) { + var wg sync.WaitGroup + eg := new(errgroup.Group) + + executeJobs := func(jobCh chan *job) { + defer wg.Done() + + allJobs := make([]*job, 0, s.cfg.Batch) + for { + select { + case j, ok := <-jobCh: + if !ok { + if len(allJobs) > 0 { + s.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) + } + return + } + if j.tp != flush && len(j.sql) > 0 { + allJobs = append(allJobs, j) + } + + if len(allJobs) < s.cfg.Batch && j.tp != flush { + continue + } + + jobs := allJobs + s.dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(jobs)) + if j.tp == flush { + if err := eg.Wait(); err != nil { + s.tctx.L().Error("flush dml jobs", log.ShortError(err)) + } + s.jobWg.Done() + } + allJobs = make([]*job, 0, s.cfg.Batch) + case <-time.After(waitTime): + if len(allJobs) > 0 { + failpoint.Inject("syncDMLTicker", func() { + s.tctx.L().Info("job queue not full, executeSQLs by ticker") + }) + if s.dmlWorkerPool.HasWorker() { + jobs := allJobs + s.dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(jobs)) + allJobs = make([]*job, 0, s.cfg.Batch) + } + } else { + // waiting #2060 + failpoint.Inject("noJobInQueueLog", func() { + s.tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) + }) + // update lag metric even if there is no job in the queue + // s.updateReplicationLag(nil, workerLagKey) + } + } + } + } + + go executeJobs(compactedMergedCh) + + for _, nonConflictMergedCh := range nonConflictMergedChs { + go executeJobs(nonConflictMergedCh) + } + wg.Wait() +} diff --git a/syncer/job.go b/syncer/job.go index fb0f5d383e..2e183f22e5 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -70,6 +70,7 @@ type job struct { sql string args []interface{} key string + keys []string retry bool location binlog.Location // location of last received (ROTATE / QUERY / XID) event, for global/table checkpoint startLocation binlog.Location // start location of the sql in binlog, for handle_error @@ -87,7 +88,7 @@ func (j *job) String() string { } func newDMLJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, - key string, location, startLocation, cmdLocation binlog.Location, eventHeader *replication.EventHeader) *job { + keys []string, location, startLocation, cmdLocation binlog.Location, eventHeader *replication.EventHeader) *job { return &job{ tp: tp, sourceTbl: map[string][]string{sourceSchema: {sourceTable}}, @@ -95,7 +96,7 @@ func newDMLJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, targetTable: targetTable, sql: sql, args: args, - key: key, + keys: keys, retry: true, location: location, diff --git a/syncer/job_test.go b/syncer/job_test.go index 388b4769a3..3b4f0cc0b2 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -89,7 +89,7 @@ func (t *testJobSuite) TestJob(c *C) { jobStr string }{ { - newDMLJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, "1", location, location, location, ec.header), + newDMLJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, []string{"1"}, location, location, location, ec.header), "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(ddlInfo, []string{"create database test"}, binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), nil, "create database test", ec.header), diff --git a/syncer/syncer.go b/syncer/syncer.go index 733004f1e9..dcb549e258 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -44,7 +44,6 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" - "golang.org/x/sync/errgroup" "github.com/pingcap/dm/dm/config" common2 "github.com/pingcap/dm/dm/ctl/common" @@ -229,6 +228,8 @@ type Syncer struct { secondsBehindMaster atomic.Int64 // current task delay second behind upstream workerLagMap map[string]*atomic.Int64 // worker's sync lag key:WorkerLagKey val: lag lastCheckpointFlushedTime time.Time + + dmlWorkerPool *brutils.WorkerPool } // NewSyncer creates a new Syncer. @@ -268,6 +269,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.recordedActiveRelayLog = false syncer.workerLagMap = make(map[string]*atomic.Int64, cfg.WorkerCount+2) // map size = WorkerCount + ddlkey + skipkey syncer.lastCheckpointFlushedTime = time.Time{} + syncer.dmlWorkerPool = brutils.NewWorkerPool(uint(cfg.WorkerCount), "dml_worker") return syncer } @@ -1272,7 +1274,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. } } -func (s *Syncer) executeDML(tctx *tcontext.Context, jobs []*job) func(uint64) error { +func (s *Syncer) executeDML(jobs []*job) func(uint64) error { // successF is used to calculate lag metric and q/tps. successF := func(queueID int, jobs []*job) { queueBucket := queueBucketName(queueID) @@ -1346,7 +1348,7 @@ func (s *Syncer) executeDML(tctx *tcontext.Context, jobs []*job) func(uint64) er time.Sleep(time.Duration(t) * time.Second) }) // use background context to execute sqls as much as possible - ctctx, cancel := tctx.WithTimeout(maxDMLExecutionDuration) + ctctx, cancel := s.tctx.WithTimeout(maxDMLExecutionDuration) defer cancel() affect, err = db.ExecuteSQL(ctctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { @@ -1364,63 +1366,16 @@ func (s *Syncer) executeDML(tctx *tcontext.Context, jobs []*job) func(uint64) er func (s *Syncer) syncDML(tctx *tcontext.Context) { defer s.wg.Done() - allJobs := make([]*job, 0, s.cfg.Batch) - dmlWorkerPool := brutils.NewWorkerPool(uint(s.cfg.WorkerCount), "dml_worker") - eg := new(errgroup.Group) - failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { t := val.(int) waitTime = time.Duration(t) * time.Second tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) }) - for { - select { - case sqlJob, ok := <-s.dmlJobCh: - metrics.QueueSizeGauge.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Set(float64(len(s.dmlJobCh))) - if !ok { - if len(allJobs) > 0 { - tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) - } - return - } - if sqlJob.tp != flush && len(sqlJob.sql) > 0 { - allJobs = append(allJobs, sqlJob) - } - - if len(allJobs) < s.cfg.Batch && sqlJob.tp != flush { - continue - } - - jobs := allJobs - dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, jobs)) - if sqlJob.tp == flush { - if err := eg.Wait(); err != nil { - tctx.L().Error("flush dml jobs", log.ShortError(err)) - } - s.jobWg.Done() - } - allJobs = make([]*job, 0, s.cfg.Batch) - case <-time.After(waitTime): - if len(allJobs) > 0 { - failpoint.Inject("syncDMLTicker", func() { - tctx.L().Info("job queue not full, executeSQLs by ticker") - }) - if dmlWorkerPool.HasWorker() { - jobs := allJobs - dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(tctx, jobs)) - allJobs = make([]*job, 0, s.cfg.Batch) - } - } else { - // waiting #2060 - failpoint.Inject("noJobInQueueLog", func() { - tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) - }) - // update lag metric even if there is no job in the queue - // s.updateReplicationLag(nil, workerLagKey) - } - } - } + compactedCh, nonCompactedCh := s.compact(s.dmlJobCh) + nonConflictChs := s.resolveConflict(nonCompactedCh) + compactedMergedCh, nonConflictMergedCh := s.mergeDMLValues(compactedCh, nonConflictChs...) + s.dmlWorkers(compactedMergedCh, nonConflictMergedCh) } // Run starts running for sync, we should guarantee it can rerun when paused. @@ -2208,7 +2163,9 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err if keys != nil { key = keys[i] } - err = s.commitJob(jobType, originSchema, originTable, schemaName, tableName, sqls[i], arg, key, &ec) + + job := newDMLJob(jobType, originSchema, originTable, schemaName, tableName, sqls[i], arg, key, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, ec.header) + err = s.addJobFunc(job) if err != nil { return err } @@ -2799,19 +2756,6 @@ func (s *Syncer) trackDDL(usedSchema string, sql string, tableNames [][]*filter. return nil } -func (s *Syncer) commitJob(tp opType, sourceSchema, sourceTable, targetSchema, targetTable, sql string, args []interface{}, keys []string, ec *eventContext) error { - startTime := time.Now() - key, err := s.resolveCasuality(keys) - if err != nil { - return terror.ErrSyncerUnitResolveCasualityFail.Generate(err) - } - s.tctx.L().Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) - metrics.ConflictDetectDurationHistogram.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - - job := newDMLJob(tp, sourceSchema, sourceTable, targetSchema, targetTable, sql, args, key, *ec.lastLocation, *ec.startLocation, *ec.currentLocation, ec.header) - return s.addJobFunc(job) -} - func (s *Syncer) resolveCasuality(keys []string) (string, error) { if s.cfg.DisableCausality { if len(keys) > 0 { From 6f1fe9160f92497acf8e471a9d0cf1ec56f1ca80 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 31 Aug 2021 09:30:30 -0400 Subject: [PATCH 13/45] fix wait --- syncer/dml.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/syncer/dml.go b/syncer/dml.go index c9309f1965..7a89324c69 100644 --- a/syncer/dml.go +++ b/syncer/dml.go @@ -703,9 +703,7 @@ func (s *Syncer) mergeDMLValues(compactedCh chan *job, nonConflictChs ...chan *j } wg.Add(len(nonConflictChs) + 1) - go mergeCompactedFunc(compactedCh, compactedMergedCh) - for i, nonConflictCh := range nonConflictChs { go mergeNonConflictFunc(nonConflictCh, nonConflictMergedCh[i]) } @@ -776,8 +774,8 @@ func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs [] } } + wg.Add(1 + len(nonConflictMergedChs)) go executeJobs(compactedMergedCh) - for _, nonConflictMergedCh := range nonConflictMergedChs { go executeJobs(nonConflictMergedCh) } From 4851cd40bd546bc11f5dd2a0b6d6a5954139b4a9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 31 Aug 2021 23:58:10 -0400 Subject: [PATCH 14/45] fix --- syncer/dml.go | 97 +++++++++++++++++++++++++++++++++++++----------- syncer/job.go | 10 +++++ syncer/syncer.go | 34 +++++++++++------ 3 files changed, 108 insertions(+), 33 deletions(-) diff --git a/syncer/dml.go b/syncer/dml.go index 7a89324c69..4a7cc8e8ab 100644 --- a/syncer/dml.go +++ b/syncer/dml.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/expression" "go.uber.org/zap" - "golang.org/x/sync/errgroup" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" @@ -637,6 +636,9 @@ func (s *Syncer) compact(jobCh chan *job) (chan *job, chan *job) { for j := range jobCh { // TODO: add compact logic + if j.tp == flush { + compactedCh <- j + } nonCompactedCh <- j } }() @@ -657,11 +659,14 @@ func (s *Syncer) resolveConflict(nonCompactedCh <-chan *job) []chan *job { for j := range nonCompactedCh { startTime := time.Now() - queueBucket := int(utils.GenHashKey(j.key)) % s.cfg.WorkerCount if !s.cfg.DisableCausality && j.tp != flush { if s.c.detectConflict(j.keys) { s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) - nonConflictChs[queueBucket] <- newFlushJob() + s.conflictJobWg.Add(s.cfg.WorkerCount) + // TODO: only flush for conflict channel + for _, nonConflictCh := range nonConflictChs { + nonConflictCh <- newConflictJob() + } s.c.reset() } // no error because we have called detectConfilict @@ -671,7 +676,15 @@ func (s *Syncer) resolveConflict(nonCompactedCh <-chan *job) []chan *job { s.tctx.L().Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) } metrics.ConflictDetectDurationHistogram.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - nonConflictChs[queueBucket] <- j + + if j.tp == flush { + for _, nonConflictCh := range nonConflictChs { + nonConflictCh <- j + } + } else { + queueBucket := int(utils.GenHashKey(j.key)) % s.cfg.WorkerCount + nonConflictChs[queueBucket] <- j + } } }() return nonConflictChs @@ -718,11 +731,28 @@ func (s *Syncer) mergeDMLValues(compactedCh chan *job, nonConflictChs ...chan *j return compactedMergedCh, nonConflictMergedCh } -func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs []chan *job) { +// waitTimeout waits for the waitgroup for the specified max timeout. +// Returns true if waiting timed out. +func waitTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { + c := make(chan struct{}) + go func() { + defer close(c) + wg.Wait() + }() + select { + case <-c: + return false + case <-time.After(timeout): + return true + } +} + +func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs []chan *job) (int, chan *job) { var wg sync.WaitGroup - eg := new(errgroup.Group) + executeWg := make([]sync.WaitGroup, len(nonConflictMergedChs)+1) + flushCh := make(chan *job, len(nonConflictMergedChs)+1) - executeJobs := func(jobCh chan *job) { + executeJobs := func(jobCh chan *job, workerWg *sync.WaitGroup, needWait bool) { defer wg.Done() allJobs := make([]*job, 0, s.cfg.Batch) @@ -739,29 +769,46 @@ func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs [] allJobs = append(allJobs, j) } - if len(allJobs) < s.cfg.Batch && j.tp != flush { + if len(allJobs) < s.cfg.Batch && j.tp != flush && j.tp != conflict { continue } + if needWait { + workerWg.Wait() + s.conflictJobWg.Wait() + } + jobs := allJobs - s.dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(jobs)) + workerWg.Add(1) + + if j.tp == conflict { + s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { + workerWg.Done() + s.conflictJobWg.Done() + })) + } else { + s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { workerWg.Done() })) + } + + // TODO: waiting for async flush if j.tp == flush { - if err := eg.Wait(); err != nil { - s.tctx.L().Error("flush dml jobs", log.ShortError(err)) - } - s.jobWg.Done() + workerWg.Wait() + flushCh <- j } allJobs = make([]*job, 0, s.cfg.Batch) case <-time.After(waitTime): if len(allJobs) > 0 { + if needWait && waitTimeout(workerWg, time.Millisecond) { + continue + } + failpoint.Inject("syncDMLTicker", func() { s.tctx.L().Info("job queue not full, executeSQLs by ticker") }) - if s.dmlWorkerPool.HasWorker() { - jobs := allJobs - s.dmlWorkerPool.ApplyWithIDInErrorGroup(eg, s.executeDML(jobs)) - allJobs = make([]*job, 0, s.cfg.Batch) - } + jobs := allJobs + workerWg.Add(1) + s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { workerWg.Done() })) + allJobs = make([]*job, 0, s.cfg.Batch) } else { // waiting #2060 failpoint.Inject("noJobInQueueLog", func() { @@ -775,9 +822,15 @@ func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs [] } wg.Add(1 + len(nonConflictMergedChs)) - go executeJobs(compactedMergedCh) - for _, nonConflictMergedCh := range nonConflictMergedChs { - go executeJobs(nonConflictMergedCh) + go executeJobs(compactedMergedCh, &executeWg[len(nonConflictMergedChs)], true) + for i, nonConflictMergedCh := range nonConflictMergedChs { + go executeJobs(nonConflictMergedCh, &executeWg[i], true) } - wg.Wait() + + go func() { + wg.Wait() + close(flushCh) + }() + + return len(nonConflictMergedChs) + 1, flushCh } diff --git a/syncer/job.go b/syncer/job.go index 2e183f22e5..cac449eefb 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -34,6 +34,7 @@ const ( flush skip // used by Syncer.recordSkipSQLsLocation to record global location, but not execute SQL rotate + conflict ) func (t opType) String() string { @@ -54,6 +55,8 @@ func (t opType) String() string { return "skip" case rotate: return "rotate" + case conflict: + return "conflict" } return "" @@ -170,6 +173,13 @@ func newFlushJob() *job { } } +func newConflictJob() *job { + return &job{ + tp: conflict, + jobAddTime: time.Now(), + } +} + // put queues into bucket to monitor them. func queueBucketName(queueID int) string { return fmt.Sprintf("q_%d", queueID) diff --git a/syncer/syncer.go b/syncer/syncer.go index dcb549e258..42bcb43ce1 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -134,8 +134,9 @@ type Syncer struct { streamerController *StreamerController enableRelay bool - wg sync.WaitGroup - jobWg sync.WaitGroup + wg sync.WaitGroup + jobWg sync.WaitGroup + conflictJobWg sync.WaitGroup schemaTracker *schema.Tracker @@ -1274,7 +1275,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. } } -func (s *Syncer) executeDML(jobs []*job) func(uint64) error { +func (s *Syncer) executeDML(jobs []*job, clearF func()) func(uint64) { // successF is used to calculate lag metric and q/tps. successF := func(queueID int, jobs []*job) { queueBucket := queueBucketName(queueID) @@ -1312,11 +1313,12 @@ func (s *Syncer) executeDML(jobs []*job) func(uint64) error { } } - executeSQLs := func(workerID uint64) (err error) { + executeSQLs := func(workerID uint64) { var ( queueID = int(workerID - 1) affect int db = s.toDBConns[queueID] + err error ) defer func() { @@ -1325,15 +1327,17 @@ func (s *Syncer) executeDML(jobs []*job) func(uint64) error { } else { fatalF(jobs[affect], err) } + clearF() }() if len(jobs) == 0 { - return nil + return } failpoint.Inject("failSecondJob", func() { if failExecuteSQL && failOnce.CAS(false, true) { s.tctx.L().Info("trigger failSecondJob") - failpoint.Return(terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock")) + err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") + failpoint.Return() } }) @@ -1357,25 +1361,33 @@ func (s *Syncer) executeDML(jobs []*job) func(uint64) error { affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") } }) - return err } return executeSQLs } // DML synced in batch by one worker. -func (s *Syncer) syncDML(tctx *tcontext.Context) { +func (s *Syncer) syncDML() { defer s.wg.Done() failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { t := val.(int) waitTime = time.Duration(t) * time.Second - tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) + s.tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) }) compactedCh, nonCompactedCh := s.compact(s.dmlJobCh) nonConflictChs := s.resolveConflict(nonCompactedCh) compactedMergedCh, nonConflictMergedCh := s.mergeDMLValues(compactedCh, nonConflictChs...) - s.dmlWorkers(compactedMergedCh, nonConflictMergedCh) + flushNum, flushCh := s.dmlWorkers(compactedMergedCh, nonConflictMergedCh) + + num := 0 + for range flushCh { + num++ + if num == flushNum { + s.jobWg.Done() + num = 0 + } + } } // Run starts running for sync, we should guarantee it can rerun when paused. @@ -1531,7 +1543,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } s.wg.Add(1) go func() { - s.syncDML(tctx) + s.syncDML() }() s.wg.Add(1) From 30be49c0b8bf0b62965fe3aa867edc9d10a74941 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 6 Sep 2021 09:59:02 -0400 Subject: [PATCH 15/45] update causality and dml worker --- syncer/causality.go | 68 ++++++++--- syncer/causality_test.go | 76 +++++++++++- syncer/dml.go | 215 --------------------------------- syncer/dml_worker.go | 249 +++++++++++++++++++++++++++++++++++++++ syncer/job.go | 2 +- syncer/job_test.go | 2 +- syncer/syncer.go | 152 ++---------------------- syncer/syncer_test.go | 111 ++++++----------- 8 files changed, 418 insertions(+), 457 deletions(-) create mode 100644 syncer/dml_worker.go diff --git a/syncer/causality.go b/syncer/causality.go index e0c5e750b5..4f3dd06e68 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -14,31 +14,43 @@ package syncer import ( - "github.com/pingcap/dm/pkg/terror" + "time" + + "github.com/pingcap/dm/syncer/metrics" ) -// causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. +// Causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. // causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. // if some conflicts exist in more than one groups, then syncer waits all SQLs that are grouped be executed and reset causality. // this mechanism meets quiescent consistency to ensure correctness. -type causality struct { - relations map[string]string +type Causality struct { + relations map[string]string + CausalityCh chan *job + in chan *job + + // for metrics + task string + source string } -func newCausality() *causality { - return &causality{ - relations: make(map[string]string), +// RunCausality creates and runs causality. +func RunCausality(chanSize int, task, source string, in chan *job) *Causality { + causality := &Causality{ + relations: make(map[string]string), + CausalityCh: make(chan *job, chanSize), + in: in, + task: task, + source: source, } + go causality.run() + return causality } -func (c *causality) add(keys []string) error { +func (c *Causality) add(keys []string) { if len(keys) == 0 { - return nil + return } - if c.detectConflict(keys) { - return terror.ErrSyncUnitCausalityConflict.Generate() - } // find causal key selectedRelation := keys[0] var nonExistKeys []string @@ -53,19 +65,43 @@ func (c *causality) add(keys []string) error { for _, key := range nonExistKeys { c.relations[key] = selectedRelation } - return nil } -func (c *causality) get(key string) string { +func (c *Causality) run() { + defer c.close() + + for j := range c.in { + startTime := time.Now() + if j.tp != flush { + if c.detectConflict(j.keys) { + // s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) + c.CausalityCh <- newCausalityJob() + c.reset() + } + c.add(j.keys) + j.key = c.get(j.key) + // s.tctx.L().Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) + } + metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) + + c.CausalityCh <- j + } +} + +func (c *Causality) close() { + close(c.CausalityCh) +} + +func (c *Causality) get(key string) string { return c.relations[key] } -func (c *causality) reset() { +func (c *Causality) reset() { c.relations = make(map[string]string) } // detectConflict detects whether there is a conflict. -func (c *causality) detectConflict(keys []string) bool { +func (c *Causality) detectConflict(keys []string) bool { if len(keys) == 0 { return false } diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 2b4c41f7e0..316f987a83 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -14,25 +14,91 @@ package syncer import ( + "time" + . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/mock" + + "github.com/pingcap/dm/pkg/binlog" + "github.com/pingcap/dm/pkg/utils" ) -func (s *testSyncerSuite) TestCausality(c *C) { - ca := newCausality() +func (s *testSyncerSuite) TestDetectConflict(c *C) { + ca := &Causality{ + relations: make(map[string]string), + } caseData := []string{"test_1", "test_2", "test_3"} excepted := map[string]string{ "test_1": "test_1", "test_2": "test_1", "test_3": "test_1", } - c.Assert(ca.add(caseData), IsNil) + c.Assert(ca.detectConflict(caseData), IsFalse) + ca.add(caseData) c.Assert(ca.relations, DeepEquals, excepted) - c.Assert(ca.add([]string{"test_4"}), IsNil) + c.Assert(ca.detectConflict([]string{"test_4"}), IsFalse) + ca.add([]string{"test_4"}) excepted["test_4"] = "test_4" c.Assert(ca.relations, DeepEquals, excepted) conflictData := []string{"test_4", "test_3"} c.Assert(ca.detectConflict(conflictData), IsTrue) - c.Assert(ca.add(conflictData), NotNil) ca.reset() c.Assert(ca.relations, HasLen, 0) } + +func (s *testSyncerSuite) TestCasuality(c *C) { + p := parser.New() + se := mock.NewContext() + schema := "create table tb(a int primary key, b int unique);" + ti, err := createTableInfo(p, se, int64(0), schema) + c.Assert(err, IsNil) + + location := binlog.NewLocation("") + jobCh := make(chan *job, 10) + causality := RunCausality(1024, "task", "source", jobCh) + testCases := []struct { + op opType + vals [][]interface{} + }{ + { + op: insert, + vals: [][]interface{}{{1, 2}}, + }, + { + op: insert, + vals: [][]interface{}{{2, 3}}, + }, + { + op: update, + vals: [][]interface{}{{2, 3}, {4, 3}}, + }, + { + op: del, + vals: [][]interface{}{{1, 2}}, + }, + { + op: insert, + vals: [][]interface{}{{1, 3}}, + }, + } + results := []opType{insert, insert, update, del, conflict, insert} + + for _, tc := range testCases { + var keys []string + for _, val := range tc.vals { + keys = append(keys, genMultipleKeys(ti, val, "tb")...) + } + job := newDMLJob(tc.op, "", "", "", "", "", nil, keys, location, location, location, nil) + jobCh <- job + } + + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return len(causality.CausalityCh) == len(results) + }), IsTrue) + + for _, op := range results { + job := <-causality.CausalityCh + c.Assert(job.tp, Equals, op) + } +} diff --git a/syncer/dml.go b/syncer/dml.go index 4a7cc8e8ab..106404c02b 100644 --- a/syncer/dml.go +++ b/syncer/dml.go @@ -18,10 +18,7 @@ import ( "fmt" "strconv" "strings" - "sync" - "time" - "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/types" @@ -31,8 +28,6 @@ import ( "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" - "github.com/pingcap/dm/pkg/utils" - "github.com/pingcap/dm/syncer/metrics" ) // genDMLParam stores pruned columns, data as well as the original columns, data, index. @@ -624,213 +619,3 @@ func pruneGeneratedColumnDML(ti *model.TableInfo, data [][]interface{}) ([]*mode } return cols, rows, nil } - -func (s *Syncer) compact(jobCh chan *job) (chan *job, chan *job) { - compactedCh := make(chan *job, s.cfg.QueueSize) - nonCompactedCh := make(chan *job, s.cfg.QueueSize) - go func() { - defer func() { - close(compactedCh) - close(nonCompactedCh) - }() - - for j := range jobCh { - // TODO: add compact logic - if j.tp == flush { - compactedCh <- j - } - nonCompactedCh <- j - } - }() - return compactedCh, nonCompactedCh -} - -func (s *Syncer) resolveConflict(nonCompactedCh <-chan *job) []chan *job { - nonConflictChs := make([]chan *job, s.cfg.WorkerCount) - for i := range nonConflictChs { - nonConflictChs[i] = make(chan *job, s.cfg.QueueSize) - } - go func() { - defer func() { - for i := range nonConflictChs { - close(nonConflictChs[i]) - } - }() - - for j := range nonCompactedCh { - startTime := time.Now() - if !s.cfg.DisableCausality && j.tp != flush { - if s.c.detectConflict(j.keys) { - s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) - s.conflictJobWg.Add(s.cfg.WorkerCount) - // TODO: only flush for conflict channel - for _, nonConflictCh := range nonConflictChs { - nonConflictCh <- newConflictJob() - } - s.c.reset() - } - // no error because we have called detectConfilict - // nolint:errcheck - s.c.add(j.keys) - j.key = s.c.get(j.key) - s.tctx.L().Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) - } - metrics.ConflictDetectDurationHistogram.WithLabelValues(s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - - if j.tp == flush { - for _, nonConflictCh := range nonConflictChs { - nonConflictCh <- j - } - } else { - queueBucket := int(utils.GenHashKey(j.key)) % s.cfg.WorkerCount - nonConflictChs[queueBucket] <- j - } - } - }() - return nonConflictChs -} - -func (s *Syncer) mergeDMLValues(compactedCh chan *job, nonConflictChs ...chan *job) (chan *job, []chan *job) { - var wg sync.WaitGroup - - compactedMergedCh := make(chan *job) - nonConflictMergedCh := make([]chan *job, len(nonConflictChs)) - - for i := 0; i < len(nonConflictMergedCh); i++ { - nonConflictMergedCh[i] = make(chan *job) - } - - mergeCompactedFunc := func(inCh chan *job, outCh chan *job) { - defer wg.Done() - for j := range inCh { - // TODO: add logic for merge - outCh <- j - } - } - mergeNonConflictFunc := func(inCh chan *job, outCh chan *job) { - defer wg.Done() - for j := range inCh { - // TODO: add logic for merge - outCh <- j - } - } - - wg.Add(len(nonConflictChs) + 1) - go mergeCompactedFunc(compactedCh, compactedMergedCh) - for i, nonConflictCh := range nonConflictChs { - go mergeNonConflictFunc(nonConflictCh, nonConflictMergedCh[i]) - } - - go func() { - wg.Wait() - close(compactedMergedCh) - for _, i := range nonConflictMergedCh { - close(i) - } - }() - return compactedMergedCh, nonConflictMergedCh -} - -// waitTimeout waits for the waitgroup for the specified max timeout. -// Returns true if waiting timed out. -func waitTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { - c := make(chan struct{}) - go func() { - defer close(c) - wg.Wait() - }() - select { - case <-c: - return false - case <-time.After(timeout): - return true - } -} - -func (s *Syncer) dmlWorkers(compactedMergedCh chan *job, nonConflictMergedChs []chan *job) (int, chan *job) { - var wg sync.WaitGroup - executeWg := make([]sync.WaitGroup, len(nonConflictMergedChs)+1) - flushCh := make(chan *job, len(nonConflictMergedChs)+1) - - executeJobs := func(jobCh chan *job, workerWg *sync.WaitGroup, needWait bool) { - defer wg.Done() - - allJobs := make([]*job, 0, s.cfg.Batch) - for { - select { - case j, ok := <-jobCh: - if !ok { - if len(allJobs) > 0 { - s.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) - } - return - } - if j.tp != flush && len(j.sql) > 0 { - allJobs = append(allJobs, j) - } - - if len(allJobs) < s.cfg.Batch && j.tp != flush && j.tp != conflict { - continue - } - - if needWait { - workerWg.Wait() - s.conflictJobWg.Wait() - } - - jobs := allJobs - workerWg.Add(1) - - if j.tp == conflict { - s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { - workerWg.Done() - s.conflictJobWg.Done() - })) - } else { - s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { workerWg.Done() })) - } - - // TODO: waiting for async flush - if j.tp == flush { - workerWg.Wait() - flushCh <- j - } - allJobs = make([]*job, 0, s.cfg.Batch) - case <-time.After(waitTime): - if len(allJobs) > 0 { - if needWait && waitTimeout(workerWg, time.Millisecond) { - continue - } - - failpoint.Inject("syncDMLTicker", func() { - s.tctx.L().Info("job queue not full, executeSQLs by ticker") - }) - jobs := allJobs - workerWg.Add(1) - s.dmlWorkerPool.ApplyWithID(s.executeDML(jobs, func() { workerWg.Done() })) - allJobs = make([]*job, 0, s.cfg.Batch) - } else { - // waiting #2060 - failpoint.Inject("noJobInQueueLog", func() { - s.tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) - }) - // update lag metric even if there is no job in the queue - // s.updateReplicationLag(nil, workerLagKey) - } - } - } - } - - wg.Add(1 + len(nonConflictMergedChs)) - go executeJobs(compactedMergedCh, &executeWg[len(nonConflictMergedChs)], true) - for i, nonConflictMergedCh := range nonConflictMergedChs { - go executeJobs(nonConflictMergedCh, &executeWg[i], true) - } - - go func() { - wg.Wait() - close(flushCh) - }() - - return len(nonConflictMergedChs) + 1, flushCh -} diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go new file mode 100644 index 0000000000..5730877a05 --- /dev/null +++ b/syncer/dml_worker.go @@ -0,0 +1,249 @@ +// Copyright 2021 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "go.uber.org/zap" + + "github.com/pingcap/dm/dm/unit" + "github.com/pingcap/dm/pkg/terror" + "github.com/pingcap/dm/pkg/utils" + "github.com/pingcap/dm/syncer/metrics" +) + +// RunDMLWorker runs dml workers. +func (s *Syncer) RunDMLWorker(compactedCh chan map[opType][]*job, pullCh chan struct{}, causalityCh chan *job) { + go s.runCompactedDMLWorker(compactedCh, pullCh) + go s.runCausalityDMLWorker(causalityCh) +} + +func (s *Syncer) successFunc(queueID int, jobs []*job) { + queueBucket := queueBucketName(queueID) + if len(jobs) > 0 { + // NOTE: we can use the first job of job queue to calculate lag because when this job committed, + // every event before this job's event in this queue has already commit. + // and we can use this job to maintain the oldest binlog event ts among all workers. + j := jobs[0] + switch j.tp { + case ddl: + metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDDLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) + case insert, update, del: + metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDMLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) + // metric only increases by 1 because dm batches sql jobs in a single transaction. + metrics.FinishedTransactionTotal.WithLabelValues(s.cfg.SourceID, s.cfg.WorkerName, s.cfg.SourceID).Inc() + } + } + + for _, sqlJob := range jobs { + s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetSchema, sqlJob.targetTable) + } + s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) + metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket).Observe(float64(len(jobs))) +} + +func (s *Syncer) fatalFunc(job *job, err error) { + s.execError.Store(err) + if !utils.IsContextCanceledError(err) { + err = s.handleEventError(err, job.startLocation, job.currentLocation, false, "") + s.runFatalChan <- unit.NewProcessError(err) + } +} + +func (s *Syncer) executeBatchJobs(jobs []*job, clearFunc func()) func(uint64) { + executeJobs := func(workerID uint64) { + var ( + queueID = int(workerID - 1) + affect int + db = s.toDBConns[queueID] + err error + ) + + defer func() { + if err == nil { + s.successFunc(queueID, jobs) + } else { + s.fatalFunc(jobs[affect], err) + } + clearFunc() + }() + + if len(jobs) == 0 { + return + } + failpoint.Inject("failSecondJob", func() { + if failExecuteSQL && failOnce.CAS(false, true) { + // s.tctx.L().Info("trigger failSecondJob") + err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") + failpoint.Return() + } + }) + + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + for _, j := range jobs { + queries = append(queries, j.sql) + args = append(args, j.args) + } + failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { + t := v.(int) + time.Sleep(time.Duration(t) * time.Second) + }) + // use background context to execute sqls as much as possible + ctctx, cancel := s.tctx.WithTimeout(maxDMLExecutionDuration) + defer cancel() + affect, err = db.ExecuteSQL(ctctx, queries, args...) + failpoint.Inject("SafeModeExit", func(val failpoint.Value) { + if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { + s.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) + affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") + } + }) + } + return executeJobs +} + +// func (s *Syncer) mergeValues(jobs []*job) []*job { +// results := make([]*job, 0, len(jobs)) +// curTable := "" +// curTp := null +// for _, j := range jobs { +// if j.tp == update && !j.UpdateKeys() { +// j.tp = replace +// } +// if curTable != j.TableName() || curTp != j.tp || j.tp == update { +// curTable = j.TableName +// curTp = j.tp +// j.sql = genSQL() +// results = append(results, j) +// continue +// } +// j.sql = appendValue() +// j.args = appendArgs() +// } +// return results +// } + +func (s *Syncer) executeCompactedJobs(jobsMap map[opType][]*job) { + var wg sync.WaitGroup + opOrder := []opType{del, insert, update} + for _, op := range opOrder { + jobs := jobsMap[op] + for i := 0; i < len(jobs); i += s.cfg.Batch { + j := i + s.cfg.Batch + if j >= len(jobs) { + j = len(jobs) + } + batchJobs := jobs[i:j] + wg.Add(1) + s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) + } + wg.Wait() + } +} + +func (s *Syncer) executeCausalityJobs(jobCh chan *job) { + jobs := make([]*job, 0, s.cfg.Batch) + var wg sync.WaitGroup + for { + select { + case j, ok := <-jobCh: + if !ok { + // if len(jobs) > 0 { + // // s.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) + // } + return + } + if j.tp != flush && j.tp != conflict && len(j.sql) > 0 { + jobs = append(jobs, j) + if len(jobs) < s.cfg.Batch { + continue + } + } + + // wait for previous jobs executed + wg.Wait() + // wait for previous causality jobs + s.causalityWg.Wait() + + batchJobs := jobs + wg.Add(1) + + if j.tp == conflict { + s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { + wg.Done() + s.causalityWg.Done() + })) + } else { + s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) + } + + // TODO: waiting for async flush + if j.tp == flush { + wg.Wait() + // flushCh <- j + } + jobs = make([]*job, 0, s.cfg.Batch) + case <-time.After(waitTime): + if len(jobs) > 0 { + // failpoint.Inject("syncDMLTicker", func() { + // s.tctx.L().Info("job queue not full, executeSQLs by ticker") + // }) + batchJobs := jobs + wg.Add(1) + s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) + jobs = make([]*job, 0, s.cfg.Batch) + } + // // waiting #2060 + // failpoint.Inject("noJobInQueueLog", func() { + // s.tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) + // }) + // // update lag metric even if there is no job in the queue + // // s.updateReplicationLag(nil, workerLagKey) + } + } +} + +func (s *Syncer) runCompactedDMLWorker(compactedCh chan map[opType][]*job, pullCh chan struct{}) { + for jobsMap := range compactedCh { + if _, ok := jobsMap[flush]; ok { + } else { + s.executeCompactedJobs(jobsMap) + } + pullCh <- struct{}{} + } +} + +func (s *Syncer) runCausalityDMLWorker(causalityCh chan *job) { + causalityJobChs := make([]chan *job, s.cfg.WorkerCount) + + for i := 0; i < s.cfg.WorkerCount; i++ { + causalityJobChs[i] = make(chan *job, s.cfg.QueueSize) + go s.executeCausalityJobs(causalityJobChs[i]) + } + + for j := range causalityCh { + if j.tp == flush || j.tp == conflict { + for _, causalityJobCh := range causalityJobChs { + causalityJobCh <- j + } + } else { + causalityJobChs[int(utils.GenHashKey(j.key))%s.cfg.WorkerCount] <- j + } + } +} diff --git a/syncer/job.go b/syncer/job.go index b992b52e0c..0a4ffa389d 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -173,7 +173,7 @@ func newFlushJob() *job { } } -func newConflictJob() *job { +func newCausalityJob() *job { return &job{ tp: conflict, jobAddTime: time.Now(), diff --git a/syncer/job_test.go b/syncer/job_test.go index 3b4f0cc0b2..6742268e6a 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -90,7 +90,7 @@ func (t *testJobSuite) TestJob(c *C) { }{ { newDMLJob(insert, "test", "t1", "test", "t1", "insert into test.t1 values(?)", []interface{}{1}, []string{"1"}, location, location, location, ec.header), - "tp: insert, sql: insert into test.t1 values(?), args: [1], key: 1, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + "tp: insert, sql: insert into test.t1 values(?), args: [1], key: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(ddlInfo, []string{"create database test"}, binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation(""), nil, "create database test", ec.header), "tp: ddl, sql: , args: [], key: , ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", diff --git a/syncer/syncer.go b/syncer/syncer.go index 65478cd46c..7094cd4fb6 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -40,11 +40,12 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" toolutils "github.com/pingcap/tidb-tools/pkg/utils" - brutils "github.com/pingcap/tidb/br/pkg/utils" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" + brutils "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/dm/dm/config" common2 "github.com/pingcap/dm/dm/ctl/common" "github.com/pingcap/dm/dm/pb" @@ -135,9 +136,9 @@ type Syncer struct { streamerController *StreamerController enableRelay bool - wg sync.WaitGroup - jobWg sync.WaitGroup - conflictJobWg sync.WaitGroup + wg sync.WaitGroup + jobWg sync.WaitGroup + causalityWg sync.WaitGroup schemaTracker *schema.Tracker @@ -156,8 +157,6 @@ type Syncer struct { isTransactionEnd bool waitTransactionLock sync.Mutex - c *causality - tableRouter *router.Table binlogFilter *bf.BinlogEvent columnMapping *cm.Mapping @@ -231,7 +230,7 @@ type Syncer struct { workerJobTSArray []*atomic.Int64 // worker's sync job TS array, note that idx=0 is skip idx and idx=1 is ddl idx,sql worker job idx=(queue id + 2) lastCheckpointFlushedTime time.Time - dmlWorkerPool *brutils.WorkerPool + connectionPool *brutils.WorkerPool } // NewSyncer creates a new Syncer. @@ -251,7 +250,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.c = newCausality() syncer.done = nil syncer.setTimezone() syncer.addJobFunc = syncer.addJob @@ -274,7 +272,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.workerJobTSArray[i] = atomic.NewInt64(0) } syncer.lastCheckpointFlushedTime = time.Time{} - syncer.dmlWorkerPool = brutils.NewWorkerPool(uint(cfg.WorkerCount), "dml_worker") + syncer.connectionPool = brutils.NewWorkerPool(uint(cfg.WorkerCount), "connection_pool") return syncer } @@ -1022,7 +1020,6 @@ func (s *Syncer) addJob(job *job) error { s.jobWg.Add(1) s.dmlJobCh <- newFlushJob() s.jobWg.Wait() - s.c.reset() } if s.execError.Load() != nil { @@ -1290,99 +1287,6 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. } } -func (s *Syncer) executeDML(jobs []*job, clearF func()) func(uint64) { - // successF is used to calculate lag metric and q/tps. - successF := func(queueID int, jobs []*job) { - queueBucket := queueBucketName(queueID) - if len(jobs) > 0 { - // NOTE: we can use the first job of job queue to calculate lag because when this job committed, - // every event before this job's event in this queue has already commit. - // and we can use this job to maintain the oldest binlog event ts among all workers. - j := jobs[0] - switch j.tp { - case ddl: - metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDDLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) - case insert, update, del: - metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDMLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) - // metric only increases by 1 because dm batches sql jobs in a single transaction. - metrics.FinishedTransactionTotal.WithLabelValues(s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Inc() - } - } - - for _, sqlJob := range jobs { - s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetSchema, sqlJob.targetTable) - } - // reset job TS when this batch is finished. - s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) - metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket).Observe(float64(len(jobs))) - } - - fatalF := func(job *job, err error) { - s.execError.Store(err) - if !utils.IsContextCanceledError(err) { - err = s.handleEventError(err, job.startLocation, job.currentLocation, false, "") - s.runFatalChan <- unit.NewProcessError(err) - } - } - - executeSQLs := func(workerID uint64) { - var ( - queueID = int(workerID - 1) - affect int - db = s.toDBConns[queueID] - err error - ) - - defer func() { - if err == nil { - successF(queueID, jobs) - } else { - fatalF(jobs[affect], err) - } - clearF() - }() - - if len(jobs) == 0 { - return - } - failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { - t := v.(int) // sleep time - s.tctx.L().Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) - time.Sleep(time.Second * time.Duration(t)) - }) - - failpoint.Inject("failSecondJob", func() { - if failExecuteSQL && failOnce.CAS(false, true) { - s.tctx.L().Info("trigger failSecondJob") - err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") - failpoint.Return() - } - }) - - queries := make([]string, 0, len(jobs)) - args := make([][]interface{}, 0, len(jobs)) - for _, j := range jobs { - queries = append(queries, j.sql) - args = append(args, j.args) - } - failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { - t := v.(int) - time.Sleep(time.Duration(t) * time.Second) - }) - // use background context to execute sqls as much as possible - ctctx, cancel := s.tctx.WithTimeout(maxDMLExecutionDuration) - defer cancel() - affect, err = db.ExecuteSQL(ctctx, queries, args...) - failpoint.Inject("SafeModeExit", func(val failpoint.Value) { - if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { - s.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) - affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") - } - }) - } - return executeSQLs -} - // DML synced in batch by one worker. func (s *Syncer) syncDML() { defer s.wg.Done() @@ -1393,19 +1297,10 @@ func (s *Syncer) syncDML() { s.tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) }) - compactedCh, nonCompactedCh := s.compact(s.dmlJobCh) - nonConflictChs := s.resolveConflict(nonCompactedCh) - compactedMergedCh, nonConflictMergedCh := s.mergeDMLValues(compactedCh, nonConflictChs...) - flushNum, flushCh := s.dmlWorkers(compactedMergedCh, nonConflictMergedCh) - - num := 0 - for range flushCh { - num++ - if num == flushNum { - s.jobWg.Done() - num = 0 - } - } + chanSize := s.cfg.QueueSize * s.cfg.WorkerCount + // compactor := RunCompactor(chanSize, chanSize, s.dmlJobCh) + causality := RunCausality(chanSize, s.cfg.Name, s.cfg.SourceID, nil) + s.RunDMLWorker(nil, nil, causality.CausalityCh) } // Run starts running for sync, we should guarantee it can rerun when paused. @@ -2795,31 +2690,6 @@ func (s *Syncer) trackDDL(usedSchema string, sql string, tableNames [][]*filter. return nil } -func (s *Syncer) resolveCasuality(keys []string) (string, error) { - if s.cfg.DisableCausality { - if len(keys) > 0 { - return keys[0], nil - } - return "", nil - } - - if s.c.detectConflict(keys) { - s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", keys)) - if err := s.flushJobs(); err != nil { - return "", err - } - s.c.reset() - } - if err := s.c.add(keys); err != nil { - return "", err - } - var key string - if len(keys) > 0 { - key = keys[0] - } - return s.c.get(key), nil -} - func (s *Syncer) genRouter() error { s.tableRouter, _ = router.NewTableRouter(s.cfg.CaseSensitive, []*router.TableRule{}) for _, rule := range s.cfg.RouteRules { diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index 44b7a360a3..93fb0d260d 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/util/mock" "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/dm/pb" @@ -284,7 +283,9 @@ func (s *testSyncerSuite) TestSelectDB(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.baList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BAList) c.Assert(err, IsNil) err = syncer.genRouter() @@ -393,7 +394,9 @@ func (s *testSyncerSuite) TestSelectTable(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.baList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BAList) c.Assert(err, IsNil) c.Assert(syncer.genRouter(), IsNil) @@ -431,7 +434,9 @@ func (s *testSyncerSuite) TestIgnoreDB(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.baList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BAList) c.Assert(err, IsNil) c.Assert(syncer.genRouter(), IsNil) @@ -526,7 +531,9 @@ func (s *testSyncerSuite) TestIgnoreTable(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.baList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BAList) c.Assert(err, IsNil) c.Assert(syncer.genRouter(), IsNil) @@ -620,7 +627,9 @@ func (s *testSyncerSuite) TestSkipDML(c *C) { p, err := s.mockParser(db, mock) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) c.Assert(syncer.genRouter(), IsNil) syncer.binlogFilter, err = bf.NewBinlogEvent(false, s.cfg.FilterRules) @@ -878,7 +887,9 @@ func (s *testSyncerSuite) TestGeneratedColumn(c *C) { c.Assert(err, IsNil) } - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) // use upstream dbConn as mock downstream dbConn, err := s.db.Conn(context.Background()) c.Assert(err, IsNil) @@ -963,73 +974,13 @@ func (s *testSyncerSuite) TestGeneratedColumn(c *C) { } func (s *testSyncerSuite) TestcheckpointID(c *C) { - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) checkpointID := syncer.checkpointID() c.Assert(checkpointID, Equals, "101") } -func (s *testSyncerSuite) TestCasuality(c *C) { - p := parser.New() - se := mock.NewContext() - schema := "create table tb(a int primary key, b int unique);" - ti, err := createTableInfo(p, se, int64(0), schema) - c.Assert(err, IsNil) - insertValues := [][]interface{}{ - {1, 2}, - {3, 4}, - {4, 1}, - {1, 4}, // this insert conflict with the first one - } - keys := make([][]string, len(insertValues)) - for i := range insertValues { - keys[i] = genMultipleKeys(ti, insertValues[i], "tb") - } - - s.cfg.WorkerCount = 1 - syncer := NewSyncer(s.cfg, nil) - syncer.dmlJobCh = make(chan *job, 1) - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - job := <-syncer.dmlJobCh - c.Assert(job.tp, Equals, flush) - syncer.jobWg.Done() - }() - - // no conflict - key1, err := syncer.resolveCasuality(keys[0]) - c.Assert(err, IsNil) - c.Assert(key1, Equals, keys[0][0]) - - key2, err := syncer.resolveCasuality(keys[1]) - c.Assert(err, IsNil) - c.Assert(key2, Equals, keys[1][0]) - - key3, err := syncer.resolveCasuality(keys[2]) - c.Assert(err, IsNil) - c.Assert(key3, Equals, keys[2][0]) - - // will detect casuality and add a flush job - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - dbConn, err := db.Conn(context.Background()) - c.Assert(err, IsNil) - - syncer.setupMockCheckpoint(c, dbConn, mock) - mock.ExpectBegin() - mock.ExpectExec(".*INSERT INTO .* VALUES.* ON DUPLICATE KEY UPDATE.*").WillReturnResult(sqlmock.NewResult(0, 1)) - mock.ExpectCommit() - key4, err := syncer.resolveCasuality(keys[3]) - c.Assert(err, IsNil) - c.Assert(key4, Equals, keys[3][0]) - if err := mock.ExpectationsWereMet(); err != nil { - c.Errorf("checkpoint db unfulfilled expectations: %s", err) - } - wg.Wait() -} - // TODO: add `TestSharding` later. func (s *testSyncerSuite) TestRun(c *C) { @@ -1075,7 +1026,9 @@ func (s *testSyncerSuite) TestRun(c *C) { s.cfg.MaxRetry = 1 s.cfg.DisableCausality = false - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.cfg.CheckpointFlushInterval = 30 syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db, func() {})} syncer.toDBConns = []*dbconn.DBConn{ @@ -1173,10 +1126,6 @@ func (s *testSyncerSuite) TestRun(c *C) { del, "DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", []interface{}{int64(580981944116838401)}, - }, { - flush, - "", - nil, }, { // in the first minute, safe mode is true, will split update to delete + replace update, @@ -1325,7 +1274,9 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { }, } - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db, func() {})} syncer.toDBConns = []*dbconn.DBConn{ {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, @@ -1510,7 +1461,9 @@ func (s *testSyncerSuite) TestTrackDDL(c *C) { checkPointDBConn, err := checkPointDB.Conn(context.Background()) c.Assert(err, IsNil) - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) syncer.toDBConns = []*dbconn.DBConn{ {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, @@ -1967,7 +1920,9 @@ func (s *testSyncerSuite) TestTimezone(c *C) { } for _, testCase := range testCases { - syncer := NewSyncer(s.cfg, nil) + cfg, err := s.cfg.Clone() + c.Assert(err, IsNil) + syncer := NewSyncer(cfg, nil) c.Assert(syncer.genRouter(), IsNil) s.resetBinlogSyncer(c) From 2ca637c4d6b2ba7029792ad98bf7d1e61ef50d4b Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 7 Sep 2021 05:27:49 -0400 Subject: [PATCH 16/45] refine --- syncer/causality.go | 57 ++++++--- syncer/causality_test.go | 9 +- syncer/dml_worker.go | 255 +++++++++++++++++++++------------------ syncer/job.go | 2 +- syncer/syncer.go | 88 +++++++++----- 5 files changed, 242 insertions(+), 169 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 4f3dd06e68..f6ac2bfcbb 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -14,8 +14,12 @@ package syncer import ( + "sync" "time" + "go.uber.org/zap" + + "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/syncer/metrics" ) @@ -25,27 +29,46 @@ import ( // this mechanism meets quiescent consistency to ensure correctness. type Causality struct { relations map[string]string - CausalityCh chan *job + causalityCh chan *job in chan *job + logger log.Logger + chanSize int // for metrics task string source string } -// RunCausality creates and runs causality. -func RunCausality(chanSize int, task, source string, in chan *job) *Causality { +func newCausality(chanSize int, task, source string, pLogger *log.Logger) *Causality { causality := &Causality{ - relations: make(map[string]string), - CausalityCh: make(chan *job, chanSize), - in: in, - task: task, - source: source, + relations: make(map[string]string), + task: task, + chanSize: chanSize, + source: source, + logger: pLogger.WithFields(zap.String("component", "causality")), } - go causality.run() return causality } +func (c *Causality) run(in chan *job) chan *job { + c.in = in + c.causalityCh = make(chan *job, c.chanSize) + + var wg sync.WaitGroup + + wg.Add(1) + go func() { + defer wg.Done() + c.runCausality() + }() + + go func() { + defer c.close() + wg.Wait() + }() + return c.causalityCh +} + func (c *Causality) add(keys []string) { if len(keys) == 0 { return @@ -67,29 +90,29 @@ func (c *Causality) add(keys []string) { } } -func (c *Causality) run() { - defer c.close() - +func (c *Causality) runCausality() { for j := range c.in { startTime := time.Now() if j.tp != flush { if c.detectConflict(j.keys) { - // s.tctx.L().Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) - c.CausalityCh <- newCausalityJob() + c.logger.Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) + c.causalityCh <- newCausalityJob() c.reset() } c.add(j.keys) j.key = c.get(j.key) - // s.tctx.L().Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) + c.logger.Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) + } else { + c.reset() } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) - c.CausalityCh <- j + c.causalityCh <- j } } func (c *Causality) close() { - close(c.CausalityCh) + close(c.causalityCh) } func (c *Causality) get(key string) string { diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 316f987a83..77d2527300 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/dm/pkg/binlog" + "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/utils" ) @@ -56,7 +57,9 @@ func (s *testSyncerSuite) TestCasuality(c *C) { location := binlog.NewLocation("") jobCh := make(chan *job, 10) - causality := RunCausality(1024, "task", "source", jobCh) + logger := log.L() + causality := newCausality(1024, "task", "source", &logger) + causalityCh := causality.run(jobCh) testCases := []struct { op opType vals [][]interface{} @@ -94,11 +97,11 @@ func (s *testSyncerSuite) TestCasuality(c *C) { } c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return len(causality.CausalityCh) == len(results) + return len(causality.causalityCh) == len(results) }), IsTrue) for _, op := range results { - job := <-causality.CausalityCh + job := <-causalityCh c.Assert(job.tp, Equals, op) } } diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 5730877a05..cea806dbea 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -21,64 +21,102 @@ import ( "github.com/pingcap/failpoint" "go.uber.org/zap" - "github.com/pingcap/dm/dm/unit" + brutils "github.com/pingcap/tidb/br/pkg/utils" + + tcontext "github.com/pingcap/dm/pkg/context" + "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/terror" "github.com/pingcap/dm/pkg/utils" + "github.com/pingcap/dm/syncer/dbconn" "github.com/pingcap/dm/syncer/metrics" ) -// RunDMLWorker runs dml workers. -func (s *Syncer) RunDMLWorker(compactedCh chan map[opType][]*job, pullCh chan struct{}, causalityCh chan *job) { - go s.runCompactedDMLWorker(compactedCh, pullCh) - go s.runCausalityDMLWorker(causalityCh) +// DMLWorker is used to sync dml. +type DMLWorker struct { + batch int + workerCount int + queueSize int + toDBConns []*dbconn.DBConn + tctx *tcontext.Context + causalityWg sync.WaitGroup + connectionPool *brutils.WorkerPool + logger log.Logger + + // for metrics + task string + source string + worker string + + // callback func + successFunc func(int, []*job) + fatalFunc func(*job, error) + lagFunc func(*job, int) + addCountFunc func(bool, string, opType, int64, string, string) + + // channel + causalityCh chan *job + flushCh chan *job } -func (s *Syncer) successFunc(queueID int, jobs []*job) { - queueBucket := queueBucketName(queueID) - if len(jobs) > 0 { - // NOTE: we can use the first job of job queue to calculate lag because when this job committed, - // every event before this job's event in this queue has already commit. - // and we can use this job to maintain the oldest binlog event ts among all workers. - j := jobs[0] - switch j.tp { - case ddl: - metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDDLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) - case insert, update, del: - metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDMLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) - // metric only increases by 1 because dm batches sql jobs in a single transaction. - metrics.FinishedTransactionTotal.WithLabelValues(s.cfg.SourceID, s.cfg.WorkerName, s.cfg.SourceID).Inc() - } +func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, source, worker string, + successFunc func(int, []*job), fatalFunc func(*job, error), lagFunc func(*job, int), addCountFunc func(bool, string, opType, int64, string, string), +) *DMLWorker { + return &DMLWorker{ + batch: batch, + workerCount: workerCount, + queueSize: queueSize, + task: task, + source: source, + worker: worker, + connectionPool: brutils.NewWorkerPool(uint(workerCount), "dml_connection_pool"), + logger: pLogger.WithFields(zap.String("component", "causality")), + successFunc: successFunc, + fatalFunc: fatalFunc, + lagFunc: lagFunc, + addCountFunc: addCountFunc, } +} - for _, sqlJob := range jobs { - s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetSchema, sqlJob.targetTable) - } - s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) - metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket).Observe(float64(len(jobs))) +// Run runs dml workers. +func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, causalityCh chan *job) (int, chan *job) { + w.tctx = tctx + w.toDBConns = toDBConns + w.causalityCh = causalityCh + w.flushCh = make(chan *job) + + var wg sync.WaitGroup + + wg.Add(1) + go func() { + defer wg.Done() + w.runCausalityDMLWorker(causalityCh) + }() + + go func() { + defer w.close() + wg.Wait() + }() + + return w.workerCount, w.flushCh } -func (s *Syncer) fatalFunc(job *job, err error) { - s.execError.Store(err) - if !utils.IsContextCanceledError(err) { - err = s.handleEventError(err, job.startLocation, job.currentLocation, false, "") - s.runFatalChan <- unit.NewProcessError(err) - } +func (w *DMLWorker) close() { + close(w.flushCh) } -func (s *Syncer) executeBatchJobs(jobs []*job, clearFunc func()) func(uint64) { - executeJobs := func(workerID uint64) { +func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func() { + executeJobs := func() { var ( - queueID = int(workerID - 1) - affect int - db = s.toDBConns[queueID] - err error + affect int + db = w.toDBConns[queueID] + err error ) defer func() { if err == nil { - s.successFunc(queueID, jobs) + w.successFunc(queueID, jobs) } else { - s.fatalFunc(jobs[affect], err) + w.fatalFunc(jobs[affect], err) } clearFunc() }() @@ -86,6 +124,12 @@ func (s *Syncer) executeBatchJobs(jobs []*job, clearFunc func()) func(uint64) { if len(jobs) == 0 { return } + failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { + t := v.(int) // sleep time + w.logger.Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) + time.Sleep(time.Second * time.Duration(t)) + }) + failpoint.Inject("failSecondJob", func() { if failExecuteSQL && failOnce.CAS(false, true) { // s.tctx.L().Info("trigger failSecondJob") @@ -105,12 +149,12 @@ func (s *Syncer) executeBatchJobs(jobs []*job, clearFunc func()) func(uint64) { time.Sleep(time.Duration(t) * time.Second) }) // use background context to execute sqls as much as possible - ctctx, cancel := s.tctx.WithTimeout(maxDMLExecutionDuration) + ctctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) defer cancel() affect, err = db.ExecuteSQL(ctctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { - s.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) + w.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") } }) @@ -118,60 +162,26 @@ func (s *Syncer) executeBatchJobs(jobs []*job, clearFunc func()) func(uint64) { return executeJobs } -// func (s *Syncer) mergeValues(jobs []*job) []*job { -// results := make([]*job, 0, len(jobs)) -// curTable := "" -// curTp := null -// for _, j := range jobs { -// if j.tp == update && !j.UpdateKeys() { -// j.tp = replace -// } -// if curTable != j.TableName() || curTp != j.tp || j.tp == update { -// curTable = j.TableName -// curTp = j.tp -// j.sql = genSQL() -// results = append(results, j) -// continue -// } -// j.sql = appendValue() -// j.args = appendArgs() -// } -// return results -// } - -func (s *Syncer) executeCompactedJobs(jobsMap map[opType][]*job) { - var wg sync.WaitGroup - opOrder := []opType{del, insert, update} - for _, op := range opOrder { - jobs := jobsMap[op] - for i := 0; i < len(jobs); i += s.cfg.Batch { - j := i + s.cfg.Batch - if j >= len(jobs) { - j = len(jobs) - } - batchJobs := jobs[i:j] - wg.Add(1) - s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) - } - wg.Wait() - } -} - -func (s *Syncer) executeCausalityJobs(jobCh chan *job) { - jobs := make([]*job, 0, s.cfg.Batch) +func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { + jobs := make([]*job, 0, w.batch) + workerJobIdx := dmlWorkerJobIdx(queueID) var wg sync.WaitGroup for { select { case j, ok := <-jobCh: if !ok { - // if len(jobs) > 0 { - // // s.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", allJobs)) - // } + if len(jobs) > 0 { + w.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) + } return } if j.tp != flush && j.tp != conflict && len(j.sql) > 0 { + if len(jobs) == 0 { + // set job TS when received first job of this batch. + w.lagFunc(j, workerJobIdx) + } jobs = append(jobs, j) - if len(jobs) < s.cfg.Batch { + if len(jobs) < w.batch { continue } } @@ -179,71 +189,80 @@ func (s *Syncer) executeCausalityJobs(jobCh chan *job) { // wait for previous jobs executed wg.Wait() // wait for previous causality jobs - s.causalityWg.Wait() + w.causalityWg.Wait() batchJobs := jobs wg.Add(1) if j.tp == conflict { - s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { + w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() - s.causalityWg.Done() + w.causalityWg.Done() })) } else { - s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) + w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) } // TODO: waiting for async flush if j.tp == flush { wg.Wait() - // flushCh <- j + w.flushCh <- j } - jobs = make([]*job, 0, s.cfg.Batch) + jobs = make([]*job, 0, w.batch) case <-time.After(waitTime): if len(jobs) > 0 { - // failpoint.Inject("syncDMLTicker", func() { - // s.tctx.L().Info("job queue not full, executeSQLs by ticker") - // }) + failpoint.Inject("syncDMLTicker", func() { + w.tctx.L().Info("job queue not full, executeSQLs by ticker") + }) batchJobs := jobs wg.Add(1) - s.connectionPool.ApplyWithID(s.executeBatchJobs(batchJobs, func() { wg.Done() })) - jobs = make([]*job, 0, s.cfg.Batch) + w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) + jobs = make([]*job, 0, w.batch) + } else { + failpoint.Inject("noJobInQueueLog", func() { + w.tctx.L().Debug("no job in queue, update lag to zero", zap.Int( + "workerJobIdx", workerJobIdx), zap.Int64("current ts", time.Now().Unix())) + }) + w.lagFunc(nil, workerJobIdx) } - // // waiting #2060 - // failpoint.Inject("noJobInQueueLog", func() { - // s.tctx.L().Debug("no job in queue, update lag to zero", zap.Int64("current ts", time.Now().Unix())) - // }) - // // update lag metric even if there is no job in the queue - // // s.updateReplicationLag(nil, workerLagKey) } } } -func (s *Syncer) runCompactedDMLWorker(compactedCh chan map[opType][]*job, pullCh chan struct{}) { - for jobsMap := range compactedCh { - if _, ok := jobsMap[flush]; ok { - } else { - s.executeCompactedJobs(jobsMap) - } - pullCh <- struct{}{} +func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { + causalityJobChs := make([]chan *job, w.workerCount) + + for i := 0; i < w.workerCount; i++ { + causalityJobChs[i] = make(chan *job, w.queueSize) + go w.executeCausalityJobs(i, causalityJobChs[i]) } -} -func (s *Syncer) runCausalityDMLWorker(causalityCh chan *job) { - causalityJobChs := make([]chan *job, s.cfg.WorkerCount) + defer func() { + for i := 0; i < w.workerCount; i++ { + close(causalityJobChs[i]) + } + }() - for i := 0; i < s.cfg.WorkerCount; i++ { - causalityJobChs[i] = make(chan *job, s.cfg.QueueSize) - go s.executeCausalityJobs(causalityJobChs[i]) + queueBucketMapping := make([]string, w.workerCount) + for i := 0; i < w.workerCount; i++ { + queueBucketMapping[i] = queueBucketName(i) } for j := range causalityCh { if j.tp == flush || j.tp == conflict { - for _, causalityJobCh := range causalityJobChs { + // flush for every DML queue + for i, causalityJobCh := range causalityJobChs { + startTime := time.Now() causalityJobCh <- j + metrics.AddJobDurationHistogram.WithLabelValues("flush", w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) } } else { - causalityJobChs[int(utils.GenHashKey(j.key))%s.cfg.WorkerCount] <- j + queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount + w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetSchema, j.targetTable) + startTime := time.Now() + w.tctx.L().Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.key)) + causalityJobChs[queueBucket] <- j + metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) } } } diff --git a/syncer/job.go b/syncer/job.go index 0a4ffa389d..46b49b8a1d 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -182,7 +182,7 @@ func newCausalityJob() *job { // put queues into bucket to monitor them. func queueBucketName(queueID int) string { - return fmt.Sprintf("q_%d", queueID) + return fmt.Sprintf("q_%d", queueID%defaultBucketCount) } func dmlWorkerJobIdx(queueID int) int { diff --git a/syncer/syncer.go b/syncer/syncer.go index de23ad92d7..e1bec34d78 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" toolutils "github.com/pingcap/tidb-tools/pkg/utils" - brutils "github.com/pingcap/tidb/br/pkg/utils" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" @@ -91,8 +90,8 @@ var ( maxPauseOrStopWaitTime = 10 * time.Second - adminQueueName = "ddl queue" - dmlQueueName = "dml queue" + adminQueueName = "admin queue" + defaultBucketCount = 8 ) // BinlogType represents binlog sync type. @@ -135,9 +134,8 @@ type Syncer struct { streamerController *StreamerController enableRelay bool - wg sync.WaitGroup - jobWg sync.WaitGroup - causalityWg sync.WaitGroup + wg sync.WaitGroup + jobWg sync.WaitGroup schemaTracker *schema.Tracker @@ -156,6 +154,9 @@ type Syncer struct { isTransactionEnd bool waitTransactionLock sync.Mutex + causality *Causality + dmlWorker *DMLWorker + tableRouter *router.Table binlogFilter *bf.BinlogEvent columnMapping *cm.Mapping @@ -228,8 +229,6 @@ type Syncer struct { secondsBehindMaster atomic.Int64 // current task delay second behind upstream workerJobTSArray []*atomic.Int64 // worker's sync job TS array, note that idx=0 is skip idx and idx=1 is ddl idx,sql worker job idx=(queue id + 2) lastCheckpointFlushedTime time.Time - - connectionPool *brutils.WorkerPool } // NewSyncer creates a new Syncer. @@ -249,6 +248,8 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) + syncer.causality = newCausality(cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) + syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() syncer.addJobFunc = syncer.addJob @@ -271,7 +272,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.workerJobTSArray[i] = atomic.NewInt64(0) } syncer.lastCheckpointFlushedTime = time.Time{} - syncer.connectionPool = brutils.NewWorkerPool(uint(cfg.WorkerCount), "connection_pool") return syncer } @@ -283,7 +283,7 @@ func (s *Syncer) GetSecondsBehindMaster() int64 { func (s *Syncer) newJobChans() { s.closeJobChans() s.dmlJobCh = make(chan *job, s.cfg.QueueSize*s.cfg.WorkerCount) - s.ddlJobCh = make(chan *job, 10) + s.ddlJobCh = make(chan *job, s.cfg.QueueSize) s.jobsClosed.Store(false) } @@ -980,8 +980,6 @@ func (s *Syncer) addJob(job *job) error { s.addCount(false, adminQueueName, job.tp, 1, job.targetSchema, job.targetTable) s.jobWg.Add(1) s.dmlJobCh <- job - startTime := time.Now() - metrics.AddJobDurationHistogram.WithLabelValues("flush", s.cfg.Name, dmlQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) s.jobWg.Wait() s.addCount(true, adminQueueName, job.tp, 1, job.targetSchema, job.targetTable) return s.flushCheckPoints() @@ -994,9 +992,6 @@ func (s *Syncer) addJob(job *job) error { metrics.AddJobDurationHistogram.WithLabelValues("ddl", s.cfg.Name, adminQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) s.jobWg.Wait() case insert, update, del: - queueBucket = int(utils.GenHashKey(job.key)) % s.cfg.WorkerCount - s.addCount(false, dmlQueueName, job.tp, 1, job.targetSchema, job.targetTable) - startTime := time.Now() s.tctx.L().Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", job.key)) s.dmlJobCh <- job s.isTransactionEnd = false @@ -1004,7 +999,6 @@ func (s *Syncer) addJob(job *job) error { s.tctx.L().Info("receive dml job", zap.Any("dml job", job)) time.Sleep(100 * time.Millisecond) }) - metrics.AddJobDurationHistogram.WithLabelValues(job.tp.String(), s.cfg.Name, dmlQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) } // nolint:ifshort @@ -1287,8 +1281,40 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. } } +func (s *Syncer) successFunc(queueID int, jobs []*job) { + queueBucket := queueBucketName(queueID) + if len(jobs) > 0 { + // NOTE: we can use the first job of job queue to calculate lag because when this job committed, + // every event before this job's event in this queue has already commit. + // and we can use this job to maintain the oldest binlog event ts among all workers. + j := jobs[0] + switch j.tp { + case ddl: + metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDDLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) + case insert, update, del: + metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDMLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) + // metric only increases by 1 because dm batches sql jobs in a single transaction. + metrics.FinishedTransactionTotal.WithLabelValues(s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Inc() + } + } + + for _, sqlJob := range jobs { + s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetSchema, sqlJob.targetTable) + } + s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) + metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket).Observe(float64(len(jobs))) +} + +func (s *Syncer) fatalFunc(job *job, err error) { + s.execError.Store(err) + if !utils.IsContextCanceledError(err) { + err = s.handleEventError(err, job.startLocation, job.currentLocation, false, "") + s.runFatalChan <- unit.NewProcessError(err) + } +} + // DML synced in batch by one worker. -func (s *Syncer) syncDML() { +func (s *Syncer) syncDML(tctx *tcontext.Context) { defer s.wg.Done() failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { @@ -1297,10 +1323,18 @@ func (s *Syncer) syncDML() { s.tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) }) - chanSize := s.cfg.QueueSize * s.cfg.WorkerCount - // compactor := RunCompactor(chanSize, chanSize, s.dmlJobCh) - causality := RunCausality(chanSize, s.cfg.Name, s.cfg.SourceID, nil) - s.RunDMLWorker(nil, nil, causality.CausalityCh) + // TODO: add compactor + causalityCh := s.causality.run(s.dmlJobCh) + flushCount, flushCh := s.dmlWorker.run(tctx, s.toDBConns, causalityCh) + + counter := 0 + for range flushCh { + counter++ + if counter == flushCount { + counter = 0 + s.jobWg.Done() + } + } } // Run starts running for sync, we should guarantee it can rerun when paused. @@ -1449,19 +1483,13 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } s.wg.Add(1) - go func() { - s.syncDML() - }() + go s.syncDML(tctx) s.wg.Add(1) - go func() { - s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.ddlJobCh) - }() + go s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.ddlJobCh) s.wg.Add(1) - go func() { - s.printStatus(runCtx) - }() + go s.printStatus(runCtx) s.wg.Add(1) go func() { From 9b5c1fe794074d0fbb3e6cab73ebdf0bb2ad5da9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 7 Sep 2021 07:38:27 -0400 Subject: [PATCH 17/45] fix ut --- syncer/job_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/syncer/job_test.go b/syncer/job_test.go index 6742268e6a..dba5c99cb3 100644 --- a/syncer/job_test.go +++ b/syncer/job_test.go @@ -116,8 +116,8 @@ func (t *testJobSuite) TestQueueBucketName(c *C) { c.Assert(name, Equals, "q_0") name = queueBucketName(8) - c.Assert(name, Equals, "q_8") + c.Assert(name, Equals, "q_0") name = queueBucketName(9) - c.Assert(name, Equals, "q_9") + c.Assert(name, Equals, "q_1") } From cb7a29808b3cb1cdf4236c421a13322930ddb403 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Tue, 7 Sep 2021 22:40:31 -0400 Subject: [PATCH 18/45] fix causality --- syncer/causality.go | 7 ++++++- syncer/dml_worker.go | 5 ++++- syncer/syncer.go | 2 -- 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index f6ac2bfcbb..2f1bf5b080 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -100,7 +100,12 @@ func (c *Causality) runCausality() { c.reset() } c.add(j.keys) - j.key = c.get(j.key) + + var key string + if len(j.keys) > 0 { + key = j.keys[0] + } + j.key = c.get(key) c.logger.Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) } else { c.reset() diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index cea806dbea..3f4c16864c 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -190,7 +190,6 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { wg.Wait() // wait for previous causality jobs w.causalityWg.Wait() - batchJobs := jobs wg.Add(1) @@ -214,6 +213,10 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { failpoint.Inject("syncDMLTicker", func() { w.tctx.L().Info("job queue not full, executeSQLs by ticker") }) + // wait for previous jobs executed + wg.Wait() + // wait for previous causality jobs + w.causalityWg.Wait() batchJobs := jobs wg.Add(1) w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) diff --git a/syncer/syncer.go b/syncer/syncer.go index f727022222..c2589e436c 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -962,7 +962,6 @@ func (s *Syncer) addJob(job *job) error { s.tctx.L().Info("All jobs is completed before syncer close, the coming job will be reject", zap.Any("job", job)) return nil } - var queueBucket int switch job.tp { case xid: s.waitXIDJob.CAS(int64(waiting), int64(waitComplete)) @@ -987,7 +986,6 @@ func (s *Syncer) addJob(job *job) error { metrics.AddJobDurationHistogram.WithLabelValues("ddl", s.cfg.Name, adminQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) s.jobWg.Wait() case insert, update, del: - s.tctx.L().Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", job.key)) s.dmlJobCh <- job s.isTransactionEnd = false failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { From 1401f4d7746d4a778ebf9e75bf4d139d5cdeac45 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 00:10:18 -0400 Subject: [PATCH 19/45] refine queue size metrics --- syncer/causality.go | 2 ++ syncer/dml_worker.go | 2 ++ syncer/metrics/metrics.go | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/syncer/causality.go b/syncer/causality.go index 2f1bf5b080..8f250577fa 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -92,6 +92,8 @@ func (c *Causality) add(keys []string) { func (c *Causality) runCausality() { for j := range c.in { + metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_queue", "").Set(float64(len(c.in))) + startTime := time.Now() if j.tp != flush { if c.detectConflict(j.keys) { diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 3f4c16864c..64602fd5f4 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -166,9 +166,11 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { jobs := make([]*job, 0, w.batch) workerJobIdx := dmlWorkerJobIdx(queueID) var wg sync.WaitGroup + queueBucket := queueBucketName(queueID) for { select { case j, ok := <-jobCh: + metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, "").Set(float64(len(jobCh))) if !ok { if len(jobs) > 0 { w.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) diff --git a/syncer/metrics/metrics.go b/syncer/metrics/metrics.go index c0c14a1352..07f26e286d 100644 --- a/syncer/metrics/metrics.go +++ b/syncer/metrics/metrics.go @@ -127,7 +127,7 @@ var ( Subsystem: "syncer", Name: "queue_size", Help: "remain size of the DML queue", - }, []string{"task", "source_id"}) + }, []string{"task", "queue_id", "source_id"}) BinlogPosGauge = metricsproxy.NewGaugeVec( prometheus.GaugeOpts{ From 5da87a19992d41296940aefd1ef9e5cd6b22b420 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 02:57:32 -0400 Subject: [PATCH 20/45] fix --- syncer/causality.go | 7 ++--- syncer/causality_test.go | 2 +- syncer/dml_worker.go | 55 +++++++++++++++++++++++++--------------- syncer/syncer.go | 4 ++- tests/shardddl1/run.sh | 21 +++++++++++++-- 5 files changed, 61 insertions(+), 28 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 8f250577fa..d6eed184cd 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -25,7 +25,7 @@ import ( // Causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. // causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. -// if some conflicts exist in more than one groups, then syncer waits all SQLs that are grouped be executed and reset causality. +// if some conflicts exist in more than one groups, causality generate a conflict job and reset. // this mechanism meets quiescent consistency to ensure correctness. type Causality struct { relations map[string]string @@ -92,12 +92,13 @@ func (c *Causality) add(keys []string) { func (c *Causality) runCausality() { for j := range c.in { - metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_queue", "").Set(float64(len(c.in))) + metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_queue", c.source).Set(float64(len(c.in))) startTime := time.Now() if j.tp != flush { + // detectConflict before add if c.detectConflict(j.keys) { - c.logger.Debug("meet causality key, will generate a flush job and wait all sqls executed", zap.Strings("keys", j.keys)) + c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", j.keys)) c.causalityCh <- newCausalityJob() c.reset() } diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 77d2527300..9f79494620 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -74,7 +74,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { }, { op: update, - vals: [][]interface{}{{2, 3}, {4, 3}}, + vals: [][]interface{}{{2, 3}, {3, 4}}, }, { op: del, diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 64602fd5f4..d80c2335f1 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -19,9 +19,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "go.uber.org/zap" - brutils "github.com/pingcap/tidb/br/pkg/utils" + "go.uber.org/zap" tcontext "github.com/pingcap/dm/pkg/context" "github.com/pingcap/dm/pkg/log" @@ -48,6 +47,7 @@ type DMLWorker struct { worker string // callback func + // TODO: refine callback func successFunc func(int, []*job) fatalFunc func(*job, error) lagFunc func(*job, int) @@ -59,7 +59,10 @@ type DMLWorker struct { } func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, source, worker string, - successFunc func(int, []*job), fatalFunc func(*job, error), lagFunc func(*job, int), addCountFunc func(bool, string, opType, int64, string, string), + successFunc func(int, []*job), + fatalFunc func(*job, error), + lagFunc func(*job, int), + addCountFunc func(bool, string, opType, int64, string, string), ) *DMLWorker { return &DMLWorker{ batch: batch, @@ -77,7 +80,7 @@ func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, } } -// Run runs dml workers. +// run runs dml workers, return worker count and flush job channel. func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, causalityCh chan *job) (int, chan *job) { w.tctx = tctx w.toDBConns = toDBConns @@ -104,11 +107,12 @@ func (w *DMLWorker) close() { close(w.flushCh) } -func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func() { - executeJobs := func() { +// executeBatchJobs execute jobs with batch size. +func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func(uint64) { + executeJobs := func(id uint64) { var ( affect int - db = w.toDBConns[queueID] + db = w.toDBConns[int(id)-1] err error ) @@ -132,7 +136,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) failpoint.Inject("failSecondJob", func() { if failExecuteSQL && failOnce.CAS(false, true) { - // s.tctx.L().Info("trigger failSecondJob") + w.logger.Info("trigger failSecondJob") err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") failpoint.Return() } @@ -154,7 +158,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) affect, err = db.ExecuteSQL(ctctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { - w.tctx.L().Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) + w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") } }) @@ -162,6 +166,8 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) return executeJobs } +// executeCausalityJobs execute jobs in same queueBucket +// All the jobs received should be executed consecutively. func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { jobs := make([]*job, 0, w.batch) workerJobIdx := dmlWorkerJobIdx(queueID) @@ -170,10 +176,10 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { for { select { case j, ok := <-jobCh: - metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, "").Set(float64(len(jobCh))) + metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, w.source).Set(float64(len(jobCh))) if !ok { if len(jobs) > 0 { - w.tctx.L().Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) + w.logger.Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) } return } @@ -190,21 +196,23 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { // wait for previous jobs executed wg.Wait() - // wait for previous causality jobs - w.causalityWg.Wait() batchJobs := jobs wg.Add(1) if j.tp == conflict { - w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { + w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() w.causalityWg.Done() })) } else { - w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) + w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) + } + + // wait for all conflict jobs done + if j.tp == conflict { + w.causalityWg.Wait() } - // TODO: waiting for async flush if j.tp == flush { wg.Wait() w.flushCh <- j @@ -213,7 +221,7 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { case <-time.After(waitTime): if len(jobs) > 0 { failpoint.Inject("syncDMLTicker", func() { - w.tctx.L().Info("job queue not full, executeSQLs by ticker") + w.logger.Info("job queue not full, executeSQLs by ticker") }) // wait for previous jobs executed wg.Wait() @@ -221,11 +229,11 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { w.causalityWg.Wait() batchJobs := jobs wg.Add(1) - w.connectionPool.Apply(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) + w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) jobs = make([]*job, 0, w.batch) } else { failpoint.Inject("noJobInQueueLog", func() { - w.tctx.L().Debug("no job in queue, update lag to zero", zap.Int( + w.logger.Debug("no job in queue, update lag to zero", zap.Int( "workerJobIdx", workerJobIdx), zap.Int64("current ts", time.Now().Unix())) }) w.lagFunc(nil, workerJobIdx) @@ -234,6 +242,7 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { } } +// runCausalityDMLWorker distribute jobs by queueBucket. func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { causalityJobChs := make([]chan *job, w.workerCount) @@ -255,17 +264,21 @@ func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { for j := range causalityCh { if j.tp == flush || j.tp == conflict { + if j.tp == conflict { + w.causalityWg.Add(w.workerCount) + } // flush for every DML queue for i, causalityJobCh := range causalityJobChs { + w.addCountFunc(false, queueBucketMapping[i], j.tp, 1, j.targetSchema, j.targetTable) startTime := time.Now() causalityJobCh <- j - metrics.AddJobDurationHistogram.WithLabelValues("flush", w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) + metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) } } else { queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetSchema, j.targetTable) startTime := time.Now() - w.tctx.L().Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.key)) + w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.key)) causalityJobChs[queueBucket] <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) } diff --git a/syncer/syncer.go b/syncer/syncer.go index 4851ebac6a..a1ab1254a1 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -832,7 +832,7 @@ func (s *Syncer) addCount(isFinished bool, queueBucket string, tp opType, n int6 m = metrics.FinishedJobsTotal } switch tp { - case insert, update, del, ddl, flush: + case insert, update, del, ddl, flush, conflict: m.WithLabelValues(tp.String(), s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetSchema, targetTable).Add(float64(n)) case skip, xid: // ignore skip/xid jobs @@ -1320,6 +1320,8 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { causalityCh := s.causality.run(s.dmlJobCh) flushCount, flushCh := s.dmlWorker.run(tctx, s.toDBConns, causalityCh) + // wait all worker flushed + // use counter is enough since we only add new flush job after previous flush job done counter := 0 for range flushCh { counter++ diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index b698317886..d77fa2da11 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -427,7 +427,7 @@ function DM_UpdateBARule() { run_case UpdateBARule "double-source-optimistic" "init_table 111 121 211 221" "clean_table" "optimistic" } -function DM_ADD_DROP_COLUMNS_CASE { +function DM_ADD_DROP_COLUMNS_CASE() { # add cols run_sql_source1 "alter table ${shardddl1}.${tb1} add column col1 int, add column col2 int, add column col3 int;" run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,now(),1,1,1);" @@ -500,7 +500,7 @@ function DM_ADD_DROP_COLUMNS() { "clean_table" "optimistic" } -function DM_COLUMN_INDEX_CASE { +function DM_COLUMN_INDEX_CASE() { # add col and index run_sql_source1 "alter table ${shardddl1}.${tb1} add column col3 int, add index idx_col1(col1);" run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,1,1,1);" @@ -573,6 +573,22 @@ function DM_COLUMN_INDEX() { "clean_table" "optimistic" } +function DM_CAUSALITY_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,2)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(2,3)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=3, b=4 where b=3" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=1" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,3)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_CAUSALITY() { + run_case CAUSALITY "single-source-no-sharding" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int unique);\"" \ + "clean_table" "" +} + function run() { init_cluster init_database @@ -584,6 +600,7 @@ function run() { DM_RestartMaster DM_ADD_DROP_COLUMNS DM_COLUMN_INDEX + DM_CAUSALITY start=1 end=5 for i in $(seq -f "%03g" ${start} ${end}); do From 06dda5df5ecb96fdb9fcd38a31c0c1e31014929e Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 03:38:26 -0400 Subject: [PATCH 21/45] debug ci --- syncer/dml_worker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index d80c2335f1..2718e914c2 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -200,9 +200,11 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { wg.Add(1) if j.tp == conflict { + w.logger.Debug("receive conflict job", zap.String("queue bucket", queueBucket)) w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() w.causalityWg.Done() + w.logger.Debug("done conflict job", zap.String("queue bucket", queueBucket)) })) } else { w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) From f31328ee10ca10be2780ce422f5e6bce1c198e9d Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 05:17:11 -0400 Subject: [PATCH 22/45] fix --- syncer/dml_worker.go | 12 ++------ tests/shardddl1/run.sh | 65 +++++++++++++++++++++++++++++++++--------- 2 files changed, 55 insertions(+), 22 deletions(-) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 2718e914c2..2e4d321753 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -200,21 +200,14 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { wg.Add(1) if j.tp == conflict { - w.logger.Debug("receive conflict job", zap.String("queue bucket", queueBucket)) w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() w.causalityWg.Done() - w.logger.Debug("done conflict job", zap.String("queue bucket", queueBucket)) })) } else { w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) } - // wait for all conflict jobs done - if j.tp == conflict { - w.causalityWg.Wait() - } - if j.tp == flush { wg.Wait() w.flushCh <- j @@ -227,8 +220,6 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { }) // wait for previous jobs executed wg.Wait() - // wait for previous causality jobs - w.causalityWg.Wait() batchJobs := jobs wg.Add(1) w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) @@ -276,6 +267,9 @@ func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { causalityJobCh <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) } + if j.tp == conflict { + w.causalityWg.Wait() + } } else { queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetSchema, j.targetTable) diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index d77fa2da11..2ed494771b 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -574,6 +574,8 @@ function DM_COLUMN_INDEX() { } function DM_CAUSALITY_CASE() { + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,2)" run_sql_source1 "insert into ${shardddl1}.${tb1} values(2,3)" run_sql_source1 "update ${shardddl1}.${tb1} set a=3, b=4 where b=3" @@ -581,6 +583,56 @@ function DM_CAUSALITY_CASE() { run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,3)" check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log + run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,12)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(12,13)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=13, b=14 where b=13" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=11" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,13)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,22)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(22,23)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=23, b=24 where b=23" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=21" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,23)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log + run_sql_source1 "insert into ${shardddl1}.${tb1} values(31,32)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(32,33)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=33, b=34 where b=33" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=31" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(31,33)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log + run_sql_source1 "insert into ${shardddl1}.${tb1} values(41,42)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(42,43)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=43, b=44 where b=43" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=41" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(41,43)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log + truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log + run_sql_source1 "insert into ${shardddl1}.${tb1} values(51,52)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(52,53)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=53, b=54 where b=53" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=51" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(51,53)" + check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml } function DM_CAUSALITY() { @@ -593,20 +645,7 @@ function run() { init_cluster init_database - DM_UpdateBARule - DM_RENAME_TABLE - DM_RENAME_COLUMN_OPTIMISTIC - DM_RemoveLock - DM_RestartMaster - DM_ADD_DROP_COLUMNS - DM_COLUMN_INDEX DM_CAUSALITY - start=1 - end=5 - for i in $(seq -f "%03g" ${start} ${end}); do - DM_${i} - sleep 1 - done } cleanup_data $shardddl From f02faeb37f9524a95faebdfb510afb3e8ae59b91 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 05:25:58 -0400 Subject: [PATCH 23/45] revert --- tests/shardddl1/run.sh | 51 ------------------------------------------ 1 file changed, 51 deletions(-) diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 2ed494771b..0018d78ace 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -574,63 +574,12 @@ function DM_COLUMN_INDEX() { } function DM_CAUSALITY_CASE() { - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,2)" run_sql_source1 "insert into ${shardddl1}.${tb1} values(2,3)" run_sql_source1 "update ${shardddl1}.${tb1} set a=3, b=4 where b=3" run_sql_source1 "delete from ${shardddl1}.${tb1} where a=1" run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,3)" - check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log - run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,12)" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(12,13)" - run_sql_source1 "update ${shardddl1}.${tb1} set a=13, b=14 where b=13" - run_sql_source1 "delete from ${shardddl1}.${tb1} where a=11" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,13)" - check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log - run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,22)" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(22,23)" - run_sql_source1 "update ${shardddl1}.${tb1} set a=23, b=24 where b=23" - run_sql_source1 "delete from ${shardddl1}.${tb1} where a=21" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,23)" - check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log - run_sql_source1 "insert into ${shardddl1}.${tb1} values(31,32)" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(32,33)" - run_sql_source1 "update ${shardddl1}.${tb1} set a=33, b=34 where b=33" - run_sql_source1 "delete from ${shardddl1}.${tb1} where a=31" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(31,33)" - check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log - run_sql_source1 "insert into ${shardddl1}.${tb1} values(41,42)" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(42,43)" - run_sql_source1 "update ${shardddl1}.${tb1} set a=43, b=44 where b=43" - run_sql_source1 "delete from ${shardddl1}.${tb1} where a=41" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(41,43)" - check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - truncate -s 0 $WORK_DIR/worker1/log/dm-worker.log - truncate -s 0 $WORK_DIR/worker2/log/dm-worker.log - run_sql_source1 "insert into ${shardddl1}.${tb1} values(51,52)" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(52,53)" - run_sql_source1 "update ${shardddl1}.${tb1} set a=53, b=54 where b=53" - run_sql_source1 "delete from ${shardddl1}.${tb1} where a=51" - run_sql_source1 "insert into ${shardddl1}.${tb1} values(51,53)" check_log_contain_with_retry "meet causality key, will generate a conflict job to flush all sqls" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log check_sync_diff $WORK_DIR $cur/conf/diff_config.toml } From 7c44aeca6cfc4f9bc24e7b845a7ac11b67622e12 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 05:46:14 -0400 Subject: [PATCH 24/45] add test back --- tests/shardddl1/run.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/shardddl1/run.sh b/tests/shardddl1/run.sh index 0018d78ace..f0c370b387 100644 --- a/tests/shardddl1/run.sh +++ b/tests/shardddl1/run.sh @@ -595,6 +595,19 @@ function run() { init_database DM_CAUSALITY + DM_UpdateBARule + DM_RENAME_TABLE + DM_RENAME_COLUMN_OPTIMISTIC + DM_RemoveLock + DM_RestartMaster + DM_ADD_DROP_COLUMNS + DM_COLUMN_INDEX + start=1 + end=5 + for i in $(seq -f "%03g" ${start} ${end}); do + DM_${i} + sleep 1 + done } cleanup_data $shardddl From 22c9374ec1c766bc67d132897ca12ff9c77442b6 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Wed, 8 Sep 2021 06:31:36 -0400 Subject: [PATCH 25/45] update metrics --- syncer/causality.go | 2 +- syncer/dml_worker.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/syncer/causality.go b/syncer/causality.go index d6eed184cd..fc4bbfb4e3 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -92,7 +92,7 @@ func (c *Causality) add(keys []string) { func (c *Causality) runCausality() { for j := range c.in { - metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_queue", c.source).Set(float64(len(c.in))) + metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.in))) startTime := time.Now() if j.tp != flush { diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 2e4d321753..ef60124d98 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -256,6 +256,7 @@ func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { } for j := range causalityCh { + metrics.QueueSizeGauge.WithLabelValues(w.task, "causality_output", w.source).Set(float64(len(causalityCh))) if j.tp == flush || j.tp == conflict { if j.tp == conflict { w.causalityWg.Add(w.workerCount) From 4eaba28af6d301fcd6f14b15cbe89490f66c5135 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 04:53:34 -0400 Subject: [PATCH 26/45] add more comment --- syncer/causality.go | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index fc4bbfb4e3..37c15f7d74 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -39,6 +39,7 @@ type Causality struct { source string } +// newCausality creates a causality instance. func newCausality(chanSize int, task, source string, pLogger *log.Logger) *Causality { causality := &Causality{ relations: make(map[string]string), @@ -50,6 +51,7 @@ func newCausality(chanSize int, task, source string, pLogger *log.Logger) *Causa return causality } +// run runs a causality instance. func (c *Causality) run(in chan *job) chan *job { c.in = in c.causalityCh = make(chan *job, c.chanSize) @@ -69,6 +71,7 @@ func (c *Causality) run(in chan *job) chan *job { return c.causalityCh } +// add adds keys relation. func (c *Causality) add(keys []string) { if len(keys) == 0 { return @@ -90,26 +93,29 @@ func (c *Causality) add(keys []string) { } } +// runCausality receives dml jobs and returns causality jobs +// When meet conflict, returns a conflict job. func (c *Causality) runCausality() { for j := range c.in { metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.in))) startTime := time.Now() if j.tp != flush { + keys := j.keys // detectConflict before add - if c.detectConflict(j.keys) { - c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", j.keys)) + if c.detectConflict(keys) { + c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) c.causalityCh <- newCausalityJob() c.reset() } - c.add(j.keys) + c.add(keys) var key string - if len(j.keys) > 0 { - key = j.keys[0] + if len(keys) > 0 { + key = keys[0] } j.key = c.get(key) - c.logger.Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) + c.logger.Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) } else { c.reset() } @@ -119,14 +125,17 @@ func (c *Causality) runCausality() { } } +// close closes output channel. func (c *Causality) close() { close(c.causalityCh) } +// get gets relation for a key. func (c *Causality) get(key string) string { return c.relations[key] } +// reset resets relations. func (c *Causality) reset() { c.relations = make(map[string]string) } From ddd4f64bc026707e0f4e3c2927659432b37aaf4f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 04:57:10 -0400 Subject: [PATCH 27/45] reorder --- syncer/causality.go | 44 ++++++++++++++++++++++---------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 37c15f7d74..3bfbdc2e18 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -71,28 +71,6 @@ func (c *Causality) run(in chan *job) chan *job { return c.causalityCh } -// add adds keys relation. -func (c *Causality) add(keys []string) { - if len(keys) == 0 { - return - } - - // find causal key - selectedRelation := keys[0] - var nonExistKeys []string - for _, key := range keys { - if val, ok := c.relations[key]; ok { - selectedRelation = val - } else { - nonExistKeys = append(nonExistKeys, key) - } - } - // set causal relations for those non-exist keys - for _, key := range nonExistKeys { - c.relations[key] = selectedRelation - } -} - // runCausality receives dml jobs and returns causality jobs // When meet conflict, returns a conflict job. func (c *Causality) runCausality() { @@ -125,6 +103,28 @@ func (c *Causality) runCausality() { } } +// add adds keys relation. +func (c *Causality) add(keys []string) { + if len(keys) == 0 { + return + } + + // find causal key + selectedRelation := keys[0] + var nonExistKeys []string + for _, key := range keys { + if val, ok := c.relations[key]; ok { + selectedRelation = val + } else { + nonExistKeys = append(nonExistKeys, key) + } + } + // set causal relations for those non-exist keys + for _, key := range nonExistKeys { + c.relations[key] = selectedRelation + } +} + // close closes output channel. func (c *Causality) close() { close(c.causalityCh) From 17e1e151a7d4755367afd6fea0953a7fc221d2f6 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 05:11:49 -0400 Subject: [PATCH 28/45] review causality --- syncer/causality.go | 58 ++++++++++++++++++++++------------------ syncer/causality_test.go | 2 +- syncer/syncer.go | 2 +- 3 files changed, 34 insertions(+), 28 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 3bfbdc2e18..81309cb774 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -28,11 +28,12 @@ import ( // if some conflicts exist in more than one groups, causality generate a conflict job and reset. // this mechanism meets quiescent consistency to ensure correctness. type Causality struct { - relations map[string]string - causalityCh chan *job - in chan *job - logger log.Logger - chanSize int + relations map[string]string + causalityCh chan *job + in chan *job + logger log.Logger + chanSize int + disableCausality bool // for metrics task string @@ -40,13 +41,14 @@ type Causality struct { } // newCausality creates a causality instance. -func newCausality(chanSize int, task, source string, pLogger *log.Logger) *Causality { +func newCausality(disableCausality bool, chanSize int, task, source string, pLogger *log.Logger) *Causality { causality := &Causality{ - relations: make(map[string]string), - task: task, - chanSize: chanSize, - source: source, - logger: pLogger.WithFields(zap.String("component", "causality")), + relations: make(map[string]string), + disableCausality: disableCausality, + task: task, + chanSize: chanSize, + source: source, + logger: pLogger.WithFields(zap.String("component", "causality")), } return causality } @@ -80,21 +82,25 @@ func (c *Causality) runCausality() { startTime := time.Now() if j.tp != flush { keys := j.keys - // detectConflict before add - if c.detectConflict(keys) { - c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) - c.causalityCh <- newCausalityJob() - c.reset() - } - c.add(keys) - var key string if len(keys) > 0 { key = keys[0] } - j.key = c.get(key) + + if c.disableCausality { + j.key = key + } else { + // detectConflict before add + if c.detectConflict(keys) { + c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) + c.causalityCh <- newCausalityJob() + c.reset() + } + c.add(keys) + j.key = c.get(key) + } c.logger.Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) - } else { + } else if !c.disableCausality { c.reset() } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) @@ -103,6 +109,11 @@ func (c *Causality) runCausality() { } } +// close closes output channel. +func (c *Causality) close() { + close(c.causalityCh) +} + // add adds keys relation. func (c *Causality) add(keys []string) { if len(keys) == 0 { @@ -125,11 +136,6 @@ func (c *Causality) add(keys []string) { } } -// close closes output channel. -func (c *Causality) close() { - close(c.causalityCh) -} - // get gets relation for a key. func (c *Causality) get(key string) string { return c.relations[key] diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 2b31ca2aae..792e823d58 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -58,7 +58,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { jobCh := make(chan *job, 10) logger := log.L() - causality := newCausality(1024, "task", "source", &logger) + causality := newCausality(false, 1024, "task", "source", &logger) causalityCh := causality.run(jobCh) testCases := []struct { op opType diff --git a/syncer/syncer.go b/syncer/syncer.go index 4714c77fc6..9600bff244 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -242,7 +242,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.causality = newCausality(cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) + syncer.causality = newCausality(false, cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() From 50542763782b094fbfeff5666a8ec8c6b3529251 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 05:12:46 -0400 Subject: [PATCH 29/45] fix --- syncer/syncer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/syncer/syncer.go b/syncer/syncer.go index 9600bff244..a7fc6cf130 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -242,7 +242,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.causality = newCausality(false, cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) + syncer.causality = newCausality(cfg.DisableCausality, cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() From f61e2386097f3b0ab8528e1947d4ed79b6163b04 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 05:28:58 -0400 Subject: [PATCH 30/45] update --- syncer/causality.go | 2 +- syncer/dml_worker.go | 195 ++++++++++++++++++++++--------------------- 2 files changed, 103 insertions(+), 94 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 81309cb774..574a57d9aa 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -109,7 +109,7 @@ func (c *Causality) runCausality() { } } -// close closes output channel. +// close closes outer channel. func (c *Causality) close() { close(c.causalityCh) } diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 84835d1407..85f03ec359 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -59,6 +59,7 @@ type DMLWorker struct { flushCh chan *job } +// newDMLWorker creates new DML Worker. func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, source, worker string, successFunc func(int, []*job), fatalFunc func(*job, error), @@ -73,7 +74,7 @@ func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, source: source, worker: worker, connectionPool: brutils.NewWorkerPool(uint(workerCount), "dml_connection_pool"), - logger: pLogger.WithFields(zap.String("component", "causality")), + logger: pLogger.WithFields(zap.String("component", "dml_worker")), successFunc: successFunc, fatalFunc: fatalFunc, lagFunc: lagFunc, @@ -81,7 +82,11 @@ func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, } } -// run runs dml workers, return worker count and flush job channel. +// run runs dml workers, return all workers count and flush job channel. +// |‾ causality worker ‾| +// causality -|- causality worker |=> connection pool +// |_ causality worker _| +// . func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, causalityCh chan *job) (int, chan *job) { w.tctx = tctx w.toDBConns = toDBConns @@ -90,10 +95,13 @@ func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, caus var wg sync.WaitGroup + // flushCount is as many as workerCount + flushCount := w.workerCount + wg.Add(1) go func() { defer wg.Done() - w.runCausalityDMLWorker(causalityCh) + w.runCausalityDMLWorker() }() go func() { @@ -101,70 +109,59 @@ func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, caus wg.Wait() }() - return w.workerCount, w.flushCh + return flushCount, w.flushCh } +// close closes outer channel. func (w *DMLWorker) close() { close(w.flushCh) } -// executeBatchJobs execute jobs with batch size. -func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func(uint64) { - executeJobs := func(id uint64) { - var ( - affect int - db = w.toDBConns[int(id)-1] - err error - ) +// runCausalityDMLWorker distribute jobs by queueBucket. +func (w *DMLWorker) runCausalityDMLWorker() { + causalityJobChs := make([]chan *job, w.workerCount) - defer func() { - if err == nil { - w.successFunc(queueID, jobs) - } else { - w.fatalFunc(jobs[affect], err) - } - clearFunc() - }() + for i := 0; i < w.workerCount; i++ { + causalityJobChs[i] = make(chan *job, w.queueSize) + go w.executeCausalityJobs(i, causalityJobChs[i]) + } - if len(jobs) == 0 { - return + defer func() { + for i := 0; i < w.workerCount; i++ { + close(causalityJobChs[i]) } - failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { - t := v.(int) // sleep time - w.logger.Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) - time.Sleep(time.Second * time.Duration(t)) - }) + }() - failpoint.Inject("failSecondJob", func() { - if failExecuteSQL && failOnce.CAS(false, true) { - w.logger.Info("trigger failSecondJob") - err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") - failpoint.Return() - } - }) + queueBucketMapping := make([]string, w.workerCount) + for i := 0; i < w.workerCount; i++ { + queueBucketMapping[i] = queueBucketName(i) + } - queries := make([]string, 0, len(jobs)) - args := make([][]interface{}, 0, len(jobs)) - for _, j := range jobs { - queries = append(queries, j.sql) - args = append(args, j.args) - } - failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { - t := v.(int) - time.Sleep(time.Duration(t) * time.Second) - }) - // use background context to execute sqls as much as possible - ctctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) - defer cancel() - affect, err = db.ExecuteSQL(ctctx, queries, args...) - failpoint.Inject("SafeModeExit", func(val failpoint.Value) { - if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { - w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) - affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") + for j := range w.causalityCh { + metrics.QueueSizeGauge.WithLabelValues(w.task, "causality_output", w.source).Set(float64(len(w.causalityCh))) + if j.tp == flush || j.tp == conflict { + if j.tp == conflict { + w.causalityWg.Add(w.workerCount) } - }) + // flush for every DML queue + for i, causalityJobCh := range causalityJobChs { + w.addCountFunc(false, queueBucketMapping[i], j.tp, 1, j.targetTable) + startTime := time.Now() + causalityJobCh <- j + metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) + } + if j.tp == conflict { + w.causalityWg.Wait() + } + } else { + queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount + w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetTable) + startTime := time.Now() + w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.key)) + causalityJobChs[queueBucket] <- j + metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) + } } - return executeJobs } // executeCausalityJobs execute jobs in same queueBucket @@ -184,7 +181,7 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { } return } - if j.tp != flush && j.tp != conflict && len(j.sql) > 0 { + if j.tp != flush && j.tp != conflict { if len(jobs) == 0 { // set job TS when received first job of this batch. w.lagFunc(j, workerJobIdx) @@ -236,49 +233,61 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { } } -// runCausalityDMLWorker distribute jobs by queueBucket. -func (w *DMLWorker) runCausalityDMLWorker(causalityCh chan *job) { - causalityJobChs := make([]chan *job, w.workerCount) +// executeBatchJobs execute jobs with batch size. +func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func(uint64) { + executeJobs := func(id uint64) { + var ( + affect int + db = w.toDBConns[int(id)-1] + err error + ) - for i := 0; i < w.workerCount; i++ { - causalityJobChs[i] = make(chan *job, w.queueSize) - go w.executeCausalityJobs(i, causalityJobChs[i]) - } + defer func() { + if err == nil { + w.successFunc(queueID, jobs) + } else { + w.fatalFunc(jobs[affect], err) + } + clearFunc() + }() - defer func() { - for i := 0; i < w.workerCount; i++ { - close(causalityJobChs[i]) + if len(jobs) == 0 { + return } - }() - - queueBucketMapping := make([]string, w.workerCount) - for i := 0; i < w.workerCount; i++ { - queueBucketMapping[i] = queueBucketName(i) - } + failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { + t := v.(int) // sleep time + w.logger.Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) + time.Sleep(time.Second * time.Duration(t)) + }) - for j := range causalityCh { - metrics.QueueSizeGauge.WithLabelValues(w.task, "causality_output", w.source).Set(float64(len(causalityCh))) - if j.tp == flush || j.tp == conflict { - if j.tp == conflict { - w.causalityWg.Add(w.workerCount) - } - // flush for every DML queue - for i, causalityJobCh := range causalityJobChs { - w.addCountFunc(false, queueBucketMapping[i], j.tp, 1, j.targetTable) - startTime := time.Now() - causalityJobCh <- j - metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) - } - if j.tp == conflict { - w.causalityWg.Wait() + failpoint.Inject("failSecondJob", func() { + if failExecuteSQL && failOnce.CAS(false, true) { + w.logger.Info("trigger failSecondJob") + err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") + failpoint.Return() } - } else { - queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount - w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetTable) - startTime := time.Now() - w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.key)) - causalityJobChs[queueBucket] <- j - metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) + }) + + var queries []string + var args [][]interface{} + for _, j := range jobs { + queries = append(queries, j.sql) + args = append(args, j.args) } + failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { + t := v.(int) + time.Sleep(time.Duration(t) * time.Second) + }) + // use background context to execute sqls as much as possible + ctctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) + defer cancel() + affect, err = db.ExecuteSQL(ctctx, queries, args...) + failpoint.Inject("SafeModeExit", func(val failpoint.Value) { + if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { + w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) + affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") + } + }) } + return executeJobs } From 1f80f274f869a78b734cdd7f8f450e0b1d6d3e1a Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 06:19:10 -0400 Subject: [PATCH 31/45] review dml_worker --- syncer/dml_worker.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 85f03ec359..18b34659bc 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -83,9 +83,9 @@ func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, } // run runs dml workers, return all workers count and flush job channel. -// |‾ causality worker ‾| -// causality -|- causality worker |=> connection pool -// |_ causality worker _| +// |‾ causality worker ‾| +// causality -|- causality worker |=> connection pool +// |_ causality worker _| // . func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, causalityCh chan *job) (int, chan *job) { w.tctx = tctx @@ -141,17 +141,18 @@ func (w *DMLWorker) runCausalityDMLWorker() { metrics.QueueSizeGauge.WithLabelValues(w.task, "causality_output", w.source).Set(float64(len(w.causalityCh))) if j.tp == flush || j.tp == conflict { if j.tp == conflict { + w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) w.causalityWg.Add(w.workerCount) } // flush for every DML queue for i, causalityJobCh := range causalityJobChs { - w.addCountFunc(false, queueBucketMapping[i], j.tp, 1, j.targetTable) startTime := time.Now() causalityJobCh <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) } if j.tp == conflict { w.causalityWg.Wait() + w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) } } else { queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount @@ -174,13 +175,14 @@ func (w *DMLWorker) executeCausalityJobs(queueID int, jobCh chan *job) { for { select { case j, ok := <-jobCh: - metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, w.source).Set(float64(len(jobCh))) if !ok { if len(jobs) > 0 { w.logger.Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) } return } + metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, w.source).Set(float64(len(jobCh))) + if j.tp != flush && j.tp != conflict { if len(jobs) == 0 { // set job TS when received first job of this batch. From 26c7ec80f17390a9656ab6abe86cbbf5a0c19bde Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 27 Sep 2021 23:03:54 -0400 Subject: [PATCH 32/45] update channel size --- syncer/causality.go | 2 +- syncer/dml_worker.go | 8 ++++---- syncer/job.go | 7 ++++--- syncer/syncer.go | 6 +++--- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 574a57d9aa..38367275a3 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -93,7 +93,7 @@ func (c *Causality) runCausality() { // detectConflict before add if c.detectConflict(keys) { c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) - c.causalityCh <- newCausalityJob() + c.causalityCh <- newConflictJob() c.reset() } c.add(keys) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 18b34659bc..9f5500f5ae 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -35,7 +35,7 @@ import ( type DMLWorker struct { batch int workerCount int - queueSize int + chanSize int toDBConns []*dbconn.DBConn tctx *tcontext.Context causalityWg sync.WaitGroup @@ -60,7 +60,7 @@ type DMLWorker struct { } // newDMLWorker creates new DML Worker. -func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, source, worker string, +func newDMLWorker(batch, workerCount, chanSize int, pLogger *log.Logger, task, source, worker string, successFunc func(int, []*job), fatalFunc func(*job, error), lagFunc func(*job, int), @@ -69,7 +69,7 @@ func newDMLWorker(batch, workerCount, queueSize int, pLogger *log.Logger, task, return &DMLWorker{ batch: batch, workerCount: workerCount, - queueSize: queueSize, + chanSize: chanSize, task: task, source: source, worker: worker, @@ -122,7 +122,7 @@ func (w *DMLWorker) runCausalityDMLWorker() { causalityJobChs := make([]chan *job, w.workerCount) for i := 0; i < w.workerCount; i++ { - causalityJobChs[i] = make(chan *job, w.queueSize) + causalityJobChs[i] = make(chan *job, w.chanSize) go w.executeCausalityJobs(i, causalityJobChs[i]) } diff --git a/syncer/job.go b/syncer/job.go index 8e124064c7..e85ae2bd55 100644 --- a/syncer/job.go +++ b/syncer/job.go @@ -172,10 +172,11 @@ func newFlushJob() *job { } } -func newCausalityJob() *job { +func newConflictJob() *job { return &job{ - tp: conflict, - jobAddTime: time.Now(), + tp: conflict, + targetTable: &filter.Table{}, + jobAddTime: time.Now(), } } diff --git a/syncer/syncer.go b/syncer/syncer.go index a7fc6cf130..0ec1e41233 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -242,7 +242,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.causality = newCausality(cfg.DisableCausality, cfg.WorkerCount*cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) + syncer.causality = newCausality(cfg.DisableCausality, cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() @@ -271,7 +271,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { func (s *Syncer) newJobChans() { s.closeJobChans() - s.dmlJobCh = make(chan *job, s.cfg.QueueSize*s.cfg.WorkerCount) + s.dmlJobCh = make(chan *job, s.cfg.QueueSize) s.ddlJobCh = make(chan *job, s.cfg.QueueSize) s.jobsClosed.Store(false) } @@ -1266,7 +1266,7 @@ func (s *Syncer) fatalFunc(job *job, err error) { } } -// DML synced in batch by one worker. +// DML synced with causality. func (s *Syncer) syncDML(tctx *tcontext.Context) { defer s.wg.Done() From 13bed05337a221013884caeef4ffbbf05562a1c6 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 30 Sep 2021 04:00:16 -0400 Subject: [PATCH 33/45] address comment --- dm/config/task.go | 8 +++- dm/config/task_test.go | 1 - syncer/causality.go | 86 +++++++++++++++++++--------------------- syncer/causality_test.go | 6 +-- syncer/dml_worker.go | 4 +- syncer/syncer.go | 4 +- syncer/syncer_test.go | 1 - 7 files changed, 53 insertions(+), 57 deletions(-) diff --git a/dm/config/task.go b/dm/config/task.go index 5e89b2f24c..8c4a41b309 100644 --- a/dm/config/task.go +++ b/dm/config/task.go @@ -243,8 +243,9 @@ type SyncerConfig struct { MaxRetry int `yaml:"max-retry" toml:"max-retry" json:"max-retry"` // refine following configs to top level configs? - AutoFixGTID bool `yaml:"auto-fix-gtid" toml:"auto-fix-gtid" json:"auto-fix-gtid"` - EnableGTID bool `yaml:"enable-gtid" toml:"enable-gtid" json:"enable-gtid"` + AutoFixGTID bool `yaml:"auto-fix-gtid" toml:"auto-fix-gtid" json:"auto-fix-gtid"` + EnableGTID bool `yaml:"enable-gtid" toml:"enable-gtid" json:"enable-gtid"` + // deprecated DisableCausality bool `yaml:"disable-detect" toml:"disable-detect" json:"disable-detect"` SafeMode bool `yaml:"safe-mode" toml:"safe-mode" json:"safe-mode"` // deprecated, use `ansi-quotes` in top level config instead @@ -620,6 +621,9 @@ func (c *TaskConfig) adjust() error { if inst.Syncer.EnableANSIQuotes { log.L().Warn("DM could discover proper ANSI_QUOTES, `enable-ansi-quotes` is no longer take effect") } + if inst.Syncer.DisableCausality { + log.L().Warn("`disable-causality` is no longer take effect") + } for _, name := range inst.ExpressionFilters { if _, ok := c.ExprFilter[name]; !ok { diff --git a/dm/config/task_test.go b/dm/config/task_test.go index fae3ee4b94..83bf15ffb0 100644 --- a/dm/config/task_test.go +++ b/dm/config/task_test.go @@ -700,7 +700,6 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { MaxRetry: 10, AutoFixGTID: true, EnableGTID: true, - DisableCausality: false, SafeMode: true, }, CleanDumpFile: true, diff --git a/syncer/causality.go b/syncer/causality.go index 38367275a3..58a8e3ab77 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -23,17 +23,16 @@ import ( "github.com/pingcap/dm/syncer/metrics" ) -// Causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. +// causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. // causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. // if some conflicts exist in more than one groups, causality generate a conflict job and reset. // this mechanism meets quiescent consistency to ensure correctness. -type Causality struct { - relations map[string]string - causalityCh chan *job - in chan *job - logger log.Logger - chanSize int - disableCausality bool +type causality struct { + relations map[string]string + outCh chan *job + inCh chan *job + logger log.Logger + chanSize int // for metrics task string @@ -41,81 +40,76 @@ type Causality struct { } // newCausality creates a causality instance. -func newCausality(disableCausality bool, chanSize int, task, source string, pLogger *log.Logger) *Causality { - causality := &Causality{ - relations: make(map[string]string), - disableCausality: disableCausality, - task: task, - chanSize: chanSize, - source: source, - logger: pLogger.WithFields(zap.String("component", "causality")), +func newCausality(chanSize int, task, source string, pLogger *log.Logger) *causality { + causality := &causality{ + relations: make(map[string]string), + task: task, + chanSize: chanSize, + source: source, + logger: pLogger.WithFields(zap.String("component", "causality")), } return causality } // run runs a causality instance. -func (c *Causality) run(in chan *job) chan *job { - c.in = in - c.causalityCh = make(chan *job, c.chanSize) +func (c *causality) run(inCh chan *job) chan *job { + c.inCh = inCh + c.outCh = make(chan *job, c.chanSize) var wg sync.WaitGroup wg.Add(1) go func() { defer wg.Done() - c.runCausality() + c.runcausality() }() go func() { defer c.close() wg.Wait() }() - return c.causalityCh + return c.outCh } -// runCausality receives dml jobs and returns causality jobs +// runcausality receives dml jobs and returns causality jobs // When meet conflict, returns a conflict job. -func (c *Causality) runCausality() { - for j := range c.in { - metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.in))) +func (c *causality) runcausality() { + for j := range c.outCh { + metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.outCh))) startTime := time.Now() - if j.tp != flush { + if j.tp == flush { + c.reset() + } else { keys := j.keys var key string if len(keys) > 0 { key = keys[0] } - if c.disableCausality { - j.key = key - } else { - // detectConflict before add - if c.detectConflict(keys) { - c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) - c.causalityCh <- newConflictJob() - c.reset() - } - c.add(keys) - j.key = c.get(key) + // detectConflict before add + if c.detectConflict(keys) { + c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) + c.outCh <- newConflictJob() + c.reset() } + c.add(keys) + j.key = c.get(key) c.logger.Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) - } else if !c.disableCausality { - c.reset() } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) - c.causalityCh <- j + c.outCh <- j } } // close closes outer channel. -func (c *Causality) close() { - close(c.causalityCh) +func (c *causality) close() { + close(c.outCh) } // add adds keys relation. -func (c *Causality) add(keys []string) { +func (c *causality) add(keys []string) { if len(keys) == 0 { return } @@ -137,17 +131,17 @@ func (c *Causality) add(keys []string) { } // get gets relation for a key. -func (c *Causality) get(key string) string { +func (c *causality) get(key string) string { return c.relations[key] } // reset resets relations. -func (c *Causality) reset() { +func (c *causality) reset() { c.relations = make(map[string]string) } // detectConflict detects whether there is a conflict. -func (c *Causality) detectConflict(keys []string) bool { +func (c *causality) detectConflict(keys []string) bool { if len(keys) == 0 { return false } diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 792e823d58..8ee356de34 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -27,7 +27,7 @@ import ( ) func (s *testSyncerSuite) TestDetectConflict(c *C) { - ca := &Causality{ + ca := &causality{ relations: make(map[string]string), } caseData := []string{"test_1", "test_2", "test_3"} @@ -58,7 +58,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { jobCh := make(chan *job, 10) logger := log.L() - causality := newCausality(false, 1024, "task", "source", &logger) + causality := newCausality(1024, "task", "source", &logger) causalityCh := causality.run(jobCh) testCases := []struct { op opType @@ -100,7 +100,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { } c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return len(causality.causalityCh) == len(results) + return len(causality.outCh) == len(results) }), IsTrue) for _, op := range results { diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 9f5500f5ae..fb6436f0ae 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -281,9 +281,9 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) time.Sleep(time.Duration(t) * time.Second) }) // use background context to execute sqls as much as possible - ctctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) + ctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) defer cancel() - affect, err = db.ExecuteSQL(ctctx, queries, args...) + affect, err = db.ExecuteSQL(ctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) diff --git a/syncer/syncer.go b/syncer/syncer.go index 0ec1e41233..1df1685f66 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -152,7 +152,7 @@ type Syncer struct { isTransactionEnd bool waitTransactionLock sync.Mutex - causality *Causality + causality *causality dmlWorker *DMLWorker tableRouter *router.Table @@ -242,7 +242,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.causality = newCausality(cfg.DisableCausality, cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) + syncer.causality = newCausality(cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index cef72c2b76..fc1032c34f 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -760,7 +760,6 @@ func (s *testSyncerSuite) TestRun(c *C) { s.cfg.Batch = 1000 s.cfg.WorkerCount = 2 s.cfg.MaxRetry = 1 - s.cfg.DisableCausality = false cfg, err := s.cfg.Clone() c.Assert(err, IsNil) From dd10d2a3d2f57eb9dbacc637d01106bdc6d24347 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Thu, 30 Sep 2021 04:33:23 -0400 Subject: [PATCH 34/45] fix typo --- syncer/causality.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 58a8e3ab77..e597e8feaf 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -74,8 +74,8 @@ func (c *causality) run(inCh chan *job) chan *job { // runcausality receives dml jobs and returns causality jobs // When meet conflict, returns a conflict job. func (c *causality) runcausality() { - for j := range c.outCh { - metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.outCh))) + for j := range c.inCh { + metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.inCh))) startTime := time.Now() if j.tp == flush { From 68be61e2458b1be0703148cf015d33d618510962 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 8 Oct 2021 02:07:38 -0400 Subject: [PATCH 35/45] wrap causality --- syncer/causality.go | 23 +++++++---------- syncer/causality_test.go | 18 ++++++++++--- syncer/dml_worker.go | 56 +++++++++++++++++----------------------- syncer/syncer.go | 13 +++------- 4 files changed, 52 insertions(+), 58 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index e597e8feaf..c4d1ce6d18 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -32,30 +32,28 @@ type causality struct { outCh chan *job inCh chan *job logger log.Logger - chanSize int // for metrics task string source string } -// newCausality creates a causality instance. -func newCausality(chanSize int, task, source string, pLogger *log.Logger) *causality { +// causalityWrap creates and runs a causality instance. +func causalityWrap(inCh chan *job, syncer *Syncer) chan *job { causality := &causality{ relations: make(map[string]string), - task: task, - chanSize: chanSize, - source: source, - logger: pLogger.WithFields(zap.String("component", "causality")), + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + logger: syncer.tctx.Logger.WithFields(zap.String("component", "causality")), + inCh: inCh, + outCh: make(chan *job, syncer.cfg.QueueSize), } - return causality + causality.run() + return causality.outCh } // run runs a causality instance. -func (c *causality) run(inCh chan *job) chan *job { - c.inCh = inCh - c.outCh = make(chan *job, c.chanSize) - +func (c *causality) run() { var wg sync.WaitGroup wg.Add(1) @@ -68,7 +66,6 @@ func (c *causality) run(inCh chan *job) chan *job { defer c.close() wg.Wait() }() - return c.outCh } // runcausality receives dml jobs and returns causality jobs diff --git a/syncer/causality_test.go b/syncer/causality_test.go index 8ee356de34..dc5f223e39 100644 --- a/syncer/causality_test.go +++ b/syncer/causality_test.go @@ -21,7 +21,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/dm/dm/config" "github.com/pingcap/dm/pkg/binlog" + tcontext "github.com/pingcap/dm/pkg/context" "github.com/pingcap/dm/pkg/log" "github.com/pingcap/dm/pkg/utils" ) @@ -57,9 +59,17 @@ func (s *testSyncerSuite) TestCasuality(c *C) { c.Assert(err, IsNil) jobCh := make(chan *job, 10) - logger := log.L() - causality := newCausality(1024, "task", "source", &logger) - causalityCh := causality.run(jobCh) + syncer := &Syncer{ + cfg: &config.SubTaskConfig{ + SyncerConfig: config.SyncerConfig{ + QueueSize: 1024, + }, + Name: "task", + SourceID: "source", + }, + tctx: tcontext.Background().WithLogger(log.L()), + } + causalityCh := causalityWrap(jobCh, syncer) testCases := []struct { op opType vals [][]interface{} @@ -100,7 +110,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { } c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return len(causality.outCh) == len(results) + return len(causalityCh) == len(results) }), IsTrue) for _, op := range results { diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index fb6436f0ae..833df44977 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -59,45 +59,39 @@ type DMLWorker struct { flushCh chan *job } -// newDMLWorker creates new DML Worker. -func newDMLWorker(batch, workerCount, chanSize int, pLogger *log.Logger, task, source, worker string, - successFunc func(int, []*job), - fatalFunc func(*job, error), - lagFunc func(*job, int), - addCountFunc func(bool, string, opType, int64, *filter.Table), -) *DMLWorker { - return &DMLWorker{ - batch: batch, - workerCount: workerCount, - chanSize: chanSize, - task: task, - source: source, - worker: worker, - connectionPool: brutils.NewWorkerPool(uint(workerCount), "dml_connection_pool"), - logger: pLogger.WithFields(zap.String("component", "dml_worker")), - successFunc: successFunc, - fatalFunc: fatalFunc, - lagFunc: lagFunc, - addCountFunc: addCountFunc, +// dmlWorkerWrap creates and runs a dmlWorker instance and return all workers count and flush job channel. +func dmlWorkerWrap(causalityCh chan *job, syncer *Syncer) (int, chan *job) { + dmlWorker := &DMLWorker{ + batch: syncer.cfg.Batch, + workerCount: syncer.cfg.WorkerCount, + chanSize: syncer.cfg.QueueSize, + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + worker: syncer.cfg.WorkerName, + connectionPool: brutils.NewWorkerPool(uint(syncer.cfg.WorkerCount), "dml_connection_pool"), + logger: syncer.tctx.Logger.WithFields(zap.String("component", "dml_worker")), + successFunc: syncer.successFunc, + fatalFunc: syncer.fatalFunc, + lagFunc: syncer.updateReplicationJobTS, + addCountFunc: syncer.addCount, + tctx: syncer.tctx, + toDBConns: syncer.toDBConns, + causalityCh: causalityCh, + flushCh: make(chan *job), } + dmlWorker.run() + // flushCount is as many as workerCount + return dmlWorker.workerCount, dmlWorker.flushCh } -// run runs dml workers, return all workers count and flush job channel. +// run runs dml workers // |‾ causality worker ‾| // causality -|- causality worker |=> connection pool // |_ causality worker _| // . -func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, causalityCh chan *job) (int, chan *job) { - w.tctx = tctx - w.toDBConns = toDBConns - w.causalityCh = causalityCh - w.flushCh = make(chan *job) - +func (w *DMLWorker) run() { var wg sync.WaitGroup - // flushCount is as many as workerCount - flushCount := w.workerCount - wg.Add(1) go func() { defer wg.Done() @@ -108,8 +102,6 @@ func (w *DMLWorker) run(tctx *tcontext.Context, toDBConns []*dbconn.DBConn, caus defer w.close() wg.Wait() }() - - return flushCount, w.flushCh } // close closes outer channel. diff --git a/syncer/syncer.go b/syncer/syncer.go index 1df1685f66..58ba8cf410 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -152,9 +152,6 @@ type Syncer struct { isTransactionEnd bool waitTransactionLock sync.Mutex - causality *causality - dmlWorker *DMLWorker - tableRouter *router.Table binlogFilter *bf.BinlogEvent columnMapping *cm.Mapping @@ -242,8 +239,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client) *Syncer { syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.causality = newCausality(cfg.QueueSize, cfg.Name, cfg.SourceID, &logger) - syncer.dmlWorker = newDMLWorker(cfg.Batch, cfg.WorkerCount, cfg.QueueSize, &logger, cfg.Name, cfg.SourceID, cfg.WorkerName, syncer.successFunc, syncer.fatalFunc, syncer.updateReplicationJobTS, syncer.addCount) syncer.done = nil syncer.setTimezone() syncer.addJobFunc = syncer.addJob @@ -1267,7 +1262,7 @@ func (s *Syncer) fatalFunc(job *job, err error) { } // DML synced with causality. -func (s *Syncer) syncDML(tctx *tcontext.Context) { +func (s *Syncer) syncDML() { defer s.wg.Done() failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { @@ -1277,8 +1272,8 @@ func (s *Syncer) syncDML(tctx *tcontext.Context) { }) // TODO: add compactor - causalityCh := s.causality.run(s.dmlJobCh) - flushCount, flushCh := s.dmlWorker.run(tctx, s.toDBConns, causalityCh) + causalityCh := causalityWrap(s.dmlJobCh, s) + flushCount, flushCh := dmlWorkerWrap(causalityCh, s) // wait all worker flushed // use counter is enough since we only add new flush job after previous flush job done @@ -1438,7 +1433,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } s.wg.Add(1) - go s.syncDML(tctx) + go s.syncDML() s.wg.Add(1) go s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.ddlJobCh) From e5b8f7d9290a701025c8d0a5ff0db2e4a275bab3 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Fri, 8 Oct 2021 22:21:13 -0400 Subject: [PATCH 36/45] address comment --- .golangci.yml | 2 +- syncer/causality.go | 36 ++++++++++-------------------------- 2 files changed, 11 insertions(+), 27 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index 6be8f699da..7385bd6320 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -120,7 +120,7 @@ issues: text: "SA1019:" # Fix found issues (if it's supported by the linter) - fix: true + fix: false run: # timeout for analysis, e.g. 30s, 5m, default is 1m diff --git a/syncer/causality.go b/syncer/causality.go index c4d1ce6d18..2a60130b4a 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -14,7 +14,6 @@ package syncer import ( - "sync" "time" "go.uber.org/zap" @@ -48,28 +47,17 @@ func causalityWrap(inCh chan *job, syncer *Syncer) chan *job { inCh: inCh, outCh: make(chan *job, syncer.cfg.QueueSize), } - causality.run() - return causality.outCh -} -// run runs a causality instance. -func (c *causality) run() { - var wg sync.WaitGroup - - wg.Add(1) go func() { - defer wg.Done() - c.runcausality() + causality.runcausality() + causality.close() }() - go func() { - defer c.close() - wg.Wait() - }() + return causality.outCh } -// runcausality receives dml jobs and returns causality jobs -// When meet conflict, returns a conflict job. +// runcausality receives dml jobs and send causality jobs by adding causality key. +// When meet conflict, sends a conflict job. func (c *causality) runcausality() { for j := range c.inCh { metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.inCh))) @@ -90,8 +78,7 @@ func (c *causality) runcausality() { c.outCh <- newConflictJob() c.reset() } - c.add(keys) - j.key = c.get(key) + j.key = c.add(keys) c.logger.Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) @@ -105,10 +92,10 @@ func (c *causality) close() { close(c.outCh) } -// add adds keys relation. -func (c *causality) add(keys []string) { +// add adds keys relation and return the relation. The keys must `detectConflict` first to ensure correctness. +func (c *causality) add(keys []string) string { if len(keys) == 0 { - return + return "" } // find causal key @@ -125,11 +112,8 @@ func (c *causality) add(keys []string) { for _, key := range nonExistKeys { c.relations[key] = selectedRelation } -} -// get gets relation for a key. -func (c *causality) get(key string) string { - return c.relations[key] + return selectedRelation } // reset resets relations. From 6a379153626a14be7c2b5f4bae0ed935093cdc7d Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sat, 9 Oct 2021 01:06:06 -0400 Subject: [PATCH 37/45] address comment --- syncer/causality.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 2a60130b4a..28b0f59273 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -49,16 +49,16 @@ func causalityWrap(inCh chan *job, syncer *Syncer) chan *job { } go func() { - causality.runcausality() + causality.runCausality() causality.close() }() return causality.outCh } -// runcausality receives dml jobs and send causality jobs by adding causality key. +// runCausality receives dml jobs and send causality jobs by adding causality key. // When meet conflict, sends a conflict job. -func (c *causality) runcausality() { +func (c *causality) runCausality() { for j := range c.inCh { metrics.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.inCh))) From e9dbabfa51deac694a4825dc027f7183963c3c9b Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 17 Oct 2021 10:08:59 -0400 Subject: [PATCH 38/45] update causality --- syncer/causality.go | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/syncer/causality.go b/syncer/causality.go index 0732d73a73..b3594b9bca 100644 --- a/syncer/causality.go +++ b/syncer/causality.go @@ -66,20 +66,14 @@ func (c *causality) run() { if j.tp == flush { c.reset() } else { - keys := j.keys - var key string - if len(keys) > 0 { - key = keys[0] - } - // detectConflict before add - if c.detectConflict(keys) { - c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) + if c.detectConflict(j.keys) { + c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", j.keys)) c.outCh <- newConflictJob() c.reset() } - j.key = c.add(keys) - c.logger.Debug("key for keys", zap.String("key", key), zap.Strings("keys", keys)) + j.key = c.add(j.keys) + c.logger.Debug("key for keys", zap.String("key", j.key), zap.Strings("keys", j.keys)) } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) From a679f695a8b0a090242e466c75febef195ac0c0f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Sun, 17 Oct 2021 10:50:22 -0400 Subject: [PATCH 39/45] remove waittime --- syncer/dml_worker.go | 234 ++++++++++++++++++------------------------ syncer/syncer.go | 7 -- tests/metrics/run.sh | 18 +--- tests/only_dml/run.sh | 6 +- 4 files changed, 106 insertions(+), 159 deletions(-) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 00b117e75d..d8b629c21e 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/filter" - brutils "github.com/pingcap/tidb/br/pkg/utils" "go.uber.org/zap" tcontext "github.com/pingcap/dm/pkg/context" @@ -33,14 +32,13 @@ import ( // DMLWorker is used to sync dml. type DMLWorker struct { - batch int - workerCount int - chanSize int - toDBConns []*dbconn.DBConn - tctx *tcontext.Context - causalityWg sync.WaitGroup - connectionPool *brutils.WorkerPool - logger log.Logger + batch int + workerCount int + chanSize int + toDBConns []*dbconn.DBConn + tctx *tcontext.Context + causalityWg sync.WaitGroup + logger log.Logger // for metrics task string @@ -62,22 +60,21 @@ type DMLWorker struct { // dmlWorkerWrap creates and runs a dmlWorker instance and return all workers count and flush job channel. func dmlWorkerWrap(inCh chan *job, syncer *Syncer) (int, chan *job) { dmlWorker := &DMLWorker{ - batch: syncer.cfg.Batch, - workerCount: syncer.cfg.WorkerCount, - chanSize: syncer.cfg.QueueSize, - task: syncer.cfg.Name, - source: syncer.cfg.SourceID, - worker: syncer.cfg.WorkerName, - connectionPool: brutils.NewWorkerPool(uint(syncer.cfg.WorkerCount), "dml_connection_pool"), - logger: syncer.tctx.Logger.WithFields(zap.String("component", "dml_worker")), - successFunc: syncer.successFunc, - fatalFunc: syncer.fatalFunc, - lagFunc: syncer.updateReplicationJobTS, - addCountFunc: syncer.addCount, - tctx: syncer.tctx, - toDBConns: syncer.toDBConns, - inCh: inCh, - flushCh: make(chan *job), + batch: syncer.cfg.Batch, + workerCount: syncer.cfg.WorkerCount, + chanSize: syncer.cfg.QueueSize, + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + worker: syncer.cfg.WorkerName, + logger: syncer.tctx.Logger.WithFields(zap.String("component", "dml_worker")), + successFunc: syncer.successFunc, + fatalFunc: syncer.fatalFunc, + lagFunc: syncer.updateReplicationJobTS, + addCountFunc: syncer.addCount, + tctx: syncer.tctx, + toDBConns: syncer.toDBConns, + inCh: inCh, + flushCh: make(chan *job), } go func() { @@ -140,131 +137,104 @@ func (w *DMLWorker) run() { } } -// executeCausalityJobs execute jobs in same queueBucket +// executeJobs execute jobs in same queueBucket // All the jobs received should be executed consecutively. func (w *DMLWorker) executeJobs(queueID int, jobCh chan *job) { jobs := make([]*job, 0, w.batch) workerJobIdx := dmlWorkerJobIdx(queueID) - var wg sync.WaitGroup queueBucket := queueBucketName(queueID) - for { - select { - case j, ok := <-jobCh: - if !ok { - if len(jobs) > 0 { - w.logger.Warn("have unexecuted jobs when close job chan!", zap.Any("rest job", jobs)) - } - return - } - metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, w.source).Set(float64(len(jobCh))) + for j := range jobCh { + metrics.QueueSizeGauge.WithLabelValues(w.task, queueBucket, w.source).Set(float64(len(jobCh))) - if j.tp != flush && j.tp != conflict { - if len(jobs) == 0 { - // set job TS when received first job of this batch. - w.lagFunc(j, workerJobIdx) - } - jobs = append(jobs, j) - if len(jobs) < w.batch { - continue - } + if j.tp != flush && j.tp != conflict { + if len(jobs) == 0 { + // set job TS when received first job of this batch. + w.lagFunc(j, workerJobIdx) } - - // wait for previous jobs executed - wg.Wait() - batchJobs := jobs - wg.Add(1) - - if j.tp == conflict { - w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { - wg.Done() - w.causalityWg.Done() - })) - } else { - w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) + jobs = append(jobs, j) + if len(jobs) < w.batch && len(jobCh) > 0 { + continue } + } - if j.tp == flush { - wg.Wait() - w.flushCh <- j - } - jobs = make([]*job, 0, w.batch) - case <-time.After(waitTime): - if len(jobs) > 0 { - failpoint.Inject("syncDMLTicker", func() { - w.logger.Info("job queue not full, executeSQLs by ticker") - }) - // wait for previous jobs executed - wg.Wait() - batchJobs := jobs - wg.Add(1) - w.connectionPool.ApplyWithID(w.executeBatchJobs(queueID, batchJobs, func() { wg.Done() })) - jobs = make([]*job, 0, w.batch) - } else { - failpoint.Inject("noJobInQueueLog", func() { - w.logger.Debug("no job in queue, update lag to zero", zap.Int( - "workerJobIdx", workerJobIdx), zap.Int64("current ts", time.Now().Unix())) - }) - w.lagFunc(nil, workerJobIdx) + failpoint.Inject("syncDMLBatchNotFull", func() { + if len(jobCh) == 0 && len(jobs) < w.batch { + w.logger.Info("execute not full job queue") } + }) + + batchJobs := jobs + w.executeBatchJobs(queueID, batchJobs) + if j.tp == conflict { + w.causalityWg.Done() + } + + if j.tp == flush { + w.flushCh <- j + } + jobs = jobs[0:0] + + if len(jobCh) == 0 { + failpoint.Inject("noJobInQueueLog", func() { + w.logger.Debug("no job in queue, update lag to zero", zap.Int( + "workerJobIdx", workerJobIdx), zap.Int64("current ts", time.Now().Unix())) + }) + w.lagFunc(nil, workerJobIdx) } } } // executeBatchJobs execute jobs with batch size. -func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job, clearFunc func()) func(uint64) { - executeJobs := func(id uint64) { - var ( - affect int - db = w.toDBConns[int(id)-1] - err error - ) - - defer func() { - if err == nil { - w.successFunc(queueID, jobs) - } else { - w.fatalFunc(jobs[affect], err) - } - clearFunc() - }() +func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { + var ( + affect int + db = w.toDBConns[queueID] + err error + ) - if len(jobs) == 0 { - return + defer func() { + if err == nil { + w.successFunc(queueID, jobs) + } else { + w.fatalFunc(jobs[affect], err) } - failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { - t := v.(int) // sleep time - w.logger.Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) - time.Sleep(time.Second * time.Duration(t)) - }) - - failpoint.Inject("failSecondJob", func() { - if failExecuteSQL && failOnce.CAS(false, true) { - w.logger.Info("trigger failSecondJob") - err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") - failpoint.Return() - } - }) + }() - var queries []string - var args [][]interface{} - for _, j := range jobs { - queries = append(queries, j.sql) - args = append(args, j.args) + if len(jobs) == 0 { + return + } + failpoint.Inject("BlockExecuteSQLs", func(v failpoint.Value) { + t := v.(int) // sleep time + w.logger.Info("BlockExecuteSQLs", zap.Any("job", jobs[0]), zap.Int("sleep time", t)) + time.Sleep(time.Second * time.Duration(t)) + }) + + failpoint.Inject("failSecondJob", func() { + if failExecuteSQL && failOnce.CAS(false, true) { + w.logger.Info("trigger failSecondJob") + err = terror.ErrDBExecuteFailed.Delegate(errors.New("failSecondJob"), "mock") + failpoint.Return() } - failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { - t := v.(int) - time.Sleep(time.Duration(t) * time.Second) - }) - // use background context to execute sqls as much as possible - ctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) - defer cancel() - affect, err = db.ExecuteSQL(ctx, queries, args...) - failpoint.Inject("SafeModeExit", func(val failpoint.Value) { - if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { - w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) - affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") - } - }) + }) + + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + for _, j := range jobs { + queries = append(queries, j.sql) + args = append(args, j.args) } - return executeJobs + failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { + t := v.(int) + time.Sleep(time.Duration(t) * time.Second) + }) + // use background context to execute sqls as much as possible + ctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) + defer cancel() + affect, err = db.ExecuteSQL(ctx, queries, args...) + failpoint.Inject("SafeModeExit", func(val failpoint.Value) { + if intVal, ok := val.(int); ok && intVal == 4 && len(jobs) > 0 { + w.logger.Warn("fail to exec DML", zap.String("failpoint", "SafeModeExit")) + affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") + } + }) } diff --git a/syncer/syncer.go b/syncer/syncer.go index 58ba8cf410..88a376ce76 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -75,7 +75,6 @@ var ( maxRetryCount = 100 retryTimeout = 3 * time.Second - waitTime = 10 * time.Millisecond // MaxDDLConnectionTimeoutMinute also used by SubTask.ExecuteDDL. MaxDDLConnectionTimeoutMinute = 5 @@ -1265,12 +1264,6 @@ func (s *Syncer) fatalFunc(job *job, err error) { func (s *Syncer) syncDML() { defer s.wg.Done() - failpoint.Inject("changeTickerInterval", func(val failpoint.Value) { - t := val.(int) - waitTime = time.Duration(t) * time.Second - s.tctx.L().Info("changeTickerInterval", zap.Int("current ticker interval second", t)) - }) - // TODO: add compactor causalityCh := causalityWrap(s.dmlJobCh, s) flushCount, flushCh := dmlWorkerWrap(causalityCh, s) diff --git a/tests/metrics/run.sh b/tests/metrics/run.sh index 3140957567..67b4bfa2c5 100755 --- a/tests/metrics/run.sh +++ b/tests/metrics/run.sh @@ -17,10 +17,8 @@ function run() { check_dashboard_datasource - # add changeTickerInterval to keep metric from updating to zero too quickly when there is no work in the queue. inject_points=( "github.com/pingcap/dm/syncer/BlockDDLJob=return(1)" - "github.com/pingcap/dm/syncer/changeTickerInterval=return(10)" "github.com/pingcap/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 beacuse we inject BlockDDLJob(ddl) to sleep(1) ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -78,7 +76,6 @@ function run() { rm -rf $WORK_DIR/worker2/log/dm-worker.log # clean up the old log inject_points=( "github.com/pingcap/dm/syncer/BlockExecuteSQLs=return(2)" - "github.com/pingcap/dm/syncer/changeTickerInterval=return(10)" "github.com/pingcap/dm/syncer/ShowLagInLog=return(2)" # test lag metric >= 2 beacuse we inject BlockExecuteSQLs to sleep(2) although skip lag is 0 (locally), but we use that lag of all dml/skip lag, so lag still >= 2 ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -138,31 +135,18 @@ function run() { wait_pattern_exit dm-worker1.toml inject_points=( - "github.com/pingcap/dm/syncer/changeTickerInterval=return(5)" "github.com/pingcap/dm/syncer/noJobInQueueLog=return()" "github.com/pingcap/dm/syncer/IgnoreSomeTypeEvent=return(\"HeartbeatEvent\")" ) - # Since the following test needs to ensure that the dml queue is empty for a long time, - # it needs to ignore upstream heartbeat events to ensure that flushjobs are not triggered export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" - # First set the ticker interval to 5s -> expect the execSQL interval to be greater than 5s - # At 5s, the first no job log will appear in the log - # At 6s, the ticker has already waited 1s and the ticker goes to 1/5th of the way - # At 6s, a dml job is added to job chan and the ticker is reset - # At 11s the ticker write the log of the second no job - # Check that the interval between the two ticker logs is > 5s - rm -rf $WORK_DIR/worker1/log/dm-worker.log # clean up the old log run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - check_log_contain_with_retry 'no job in queue, update lag to zero' $WORK_DIR/worker1/log/dm-worker.log echo "make a dml job" run_sql_source1 "insert into metrics.t1 (id, name, ts) values (1004, 'zmj4', '2022-05-11 12:01:05')" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - echo "sleep 6s" - sleep 6 - check_ticker_interval $WORK_DIR/worker1/log/dm-worker.log 5 + check_log_contain_with_retry 'no job in queue, update lag to zero' $WORK_DIR/worker1/log/dm-worker.log run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ "\"result\": true" 3 diff --git a/tests/only_dml/run.sh b/tests/only_dml/run.sh index 88049f0298..85a6174fbd 100755 --- a/tests/only_dml/run.sh +++ b/tests/only_dml/run.sh @@ -42,7 +42,7 @@ function insert_data() { } function run() { - export GO_FAILPOINTS="github.com/pingcap/dm/pkg/streamer/SetHeartbeatInterval=return(1);github.com/pingcap/dm/syncer/syncDMLTicker=return(true)" + export GO_FAILPOINTS="github.com/pingcap/dm/pkg/streamer/SetHeartbeatInterval=return(1);github.com/pingcap/dm/syncer/syncDMLBatchNotFull=return(true)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 1 row affected' @@ -129,8 +129,8 @@ function run() { done kill $pid - check_log_contain_with_retry 'job queue not full, executeSQLs by ticker' $WORK_DIR/worker1/log/dm-worker.log - check_log_contain_with_retry 'job queue not full, executeSQLs by ticker' $WORK_DIR/worker2/log/dm-worker.log + check_log_contain_with_retry 'execute not full job queue' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'execute not full job queue' $WORK_DIR/worker2/log/dm-worker.log check_sync_diff $WORK_DIR $cur/conf/diff_config.toml export GO_FAILPOINTS="" } From 1bc65b91e40bd157b2911856a7f1fa8e796774c0 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 02:35:53 -0400 Subject: [PATCH 40/45] remove flush count --- syncer/dml_worker.go | 20 +++++++++----------- syncer/syncer.go | 11 ++--------- 2 files changed, 11 insertions(+), 20 deletions(-) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index d8b629c21e..3dc183e753 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -37,7 +37,7 @@ type DMLWorker struct { chanSize int toDBConns []*dbconn.DBConn tctx *tcontext.Context - causalityWg sync.WaitGroup + wg sync.WaitGroup logger log.Logger // for metrics @@ -58,7 +58,7 @@ type DMLWorker struct { } // dmlWorkerWrap creates and runs a dmlWorker instance and return all workers count and flush job channel. -func dmlWorkerWrap(inCh chan *job, syncer *Syncer) (int, chan *job) { +func dmlWorkerWrap(inCh chan *job, syncer *Syncer) chan *job { dmlWorker := &DMLWorker{ batch: syncer.cfg.Batch, workerCount: syncer.cfg.WorkerCount, @@ -81,7 +81,7 @@ func dmlWorkerWrap(inCh chan *job, syncer *Syncer) (int, chan *job) { dmlWorker.run() dmlWorker.close() }() - return dmlWorker.workerCount, dmlWorker.flushCh + return dmlWorker.flushCh } // close closes outer channel. @@ -114,17 +114,19 @@ func (w *DMLWorker) run() { if j.tp == flush || j.tp == conflict { if j.tp == conflict { w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) - w.causalityWg.Add(w.workerCount) } + w.wg.Add(w.workerCount) // flush for every DML queue for i, jobCh := range jobChs { startTime := time.Now() jobCh <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[i], w.source).Observe(time.Since(startTime).Seconds()) } + w.wg.Wait() if j.tp == conflict { - w.causalityWg.Wait() w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) + } else { + w.flushCh <- j } } else { queueBucket := int(utils.GenHashKey(j.key)) % w.workerCount @@ -165,15 +167,11 @@ func (w *DMLWorker) executeJobs(queueID int, jobCh chan *job) { batchJobs := jobs w.executeBatchJobs(queueID, batchJobs) - if j.tp == conflict { - w.causalityWg.Done() + if j.tp == conflict || j.tp == flush { + w.wg.Done() } - if j.tp == flush { - w.flushCh <- j - } jobs = jobs[0:0] - if len(jobCh) == 0 { failpoint.Inject("noJobInQueueLog", func() { w.logger.Debug("no job in queue, update lag to zero", zap.Int( diff --git a/syncer/syncer.go b/syncer/syncer.go index 88a376ce76..fc786367eb 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -1266,17 +1266,10 @@ func (s *Syncer) syncDML() { // TODO: add compactor causalityCh := causalityWrap(s.dmlJobCh, s) - flushCount, flushCh := dmlWorkerWrap(causalityCh, s) + flushCh := dmlWorkerWrap(causalityCh, s) - // wait all worker flushed - // use counter is enough since we only add new flush job after previous flush job done - counter := 0 for range flushCh { - counter++ - if counter == flushCount { - counter = 0 - s.jobWg.Done() - } + s.jobWg.Done() } } From 6164cf1de0fe24a346fc78dff484c6476989afe9 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 05:07:53 -0400 Subject: [PATCH 41/45] fix ci --- tests/dmctl_basic/conf/get_task.yaml | 1 + tests/import_v10x/conf/task.yaml | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/dmctl_basic/conf/get_task.yaml b/tests/dmctl_basic/conf/get_task.yaml index 20664bf4de..bb86061a95 100644 --- a/tests/dmctl_basic/conf/get_task.yaml +++ b/tests/dmctl_basic/conf/get_task.yaml @@ -152,6 +152,7 @@ syncers: max-retry: 0 auto-fix-gtid: false enable-gtid: false + disable-detect: false safe-mode: false enable-ansi-quotes: false clean-dump-file: true diff --git a/tests/import_v10x/conf/task.yaml b/tests/import_v10x/conf/task.yaml index 76f707e037..0676560aed 100644 --- a/tests/import_v10x/conf/task.yaml +++ b/tests/import_v10x/conf/task.yaml @@ -96,6 +96,7 @@ syncers: max-retry: 0 auto-fix-gtid: false enable-gtid: false + disable-detect: false safe-mode: false enable-ansi-quotes: false sync-02: @@ -107,6 +108,7 @@ syncers: max-retry: 0 auto-fix-gtid: false enable-gtid: true + disable-detect: false safe-mode: false enable-ansi-quotes: false clean-dump-file: false From c944bde8549c57ea6b413553b3ba1d4f66608b23 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 07:18:27 -0400 Subject: [PATCH 42/45] fix ci --- tests/new_relay/configs/tasks/test.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/new_relay/configs/tasks/test.yaml b/tests/new_relay/configs/tasks/test.yaml index b1e3dd7c2d..17b0698cfc 100644 --- a/tests/new_relay/configs/tasks/test.yaml +++ b/tests/new_relay/configs/tasks/test.yaml @@ -53,6 +53,7 @@ syncers: max-retry: 0 auto-fix-gtid: false enable-gtid: false + disable-detect: false safe-mode: false enable-ansi-quotes: false clean-dump-file: false From b5f504badb9449d566d97fd069cbaa45e8163a48 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 22:57:01 -0400 Subject: [PATCH 43/45] address comment --- syncer/dml_worker.go | 7 +++---- syncer/syncer.go | 13 +++++++------ tests/new_relay/configs/tasks/test.yaml | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/syncer/dml_worker.go b/syncer/dml_worker.go index 3dc183e753..08a8882775 100644 --- a/syncer/dml_worker.go +++ b/syncer/dml_worker.go @@ -37,7 +37,7 @@ type DMLWorker struct { chanSize int toDBConns []*dbconn.DBConn tctx *tcontext.Context - wg sync.WaitGroup + wg sync.WaitGroup // counts conflict/flush jobs in all DML job channels. logger log.Logger // for metrics @@ -57,7 +57,7 @@ type DMLWorker struct { flushCh chan *job } -// dmlWorkerWrap creates and runs a dmlWorker instance and return all workers count and flush job channel. +// dmlWorkerWrap creates and runs a dmlWorker instance and returns flush job channel. func dmlWorkerWrap(inCh chan *job, syncer *Syncer) chan *job { dmlWorker := &DMLWorker{ batch: syncer.cfg.Batch, @@ -165,8 +165,7 @@ func (w *DMLWorker) executeJobs(queueID int, jobCh chan *job) { } }) - batchJobs := jobs - w.executeBatchJobs(queueID, batchJobs) + w.executeBatchJobs(queueID, jobs) if j.tp == conflict || j.tp == flush { w.wg.Done() } diff --git a/syncer/syncer.go b/syncer/syncer.go index fc786367eb..8478820a52 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -131,8 +131,8 @@ type Syncer struct { streamerController *StreamerController enableRelay bool - wg sync.WaitGroup - jobWg sync.WaitGroup + wg sync.WaitGroup // counts goroutines + jobWg sync.WaitGroup // counts ddl/flush job in-flight in s.dmlJobCh and s.ddlJobCh schemaTracker *schema.Tracker @@ -949,15 +949,15 @@ func (s *Syncer) addJob(job *job) error { } // nolint:ifshort - flush := s.checkFlush() + needFlush := s.checkFlush() failpoint.Inject("flushFirstJob", func() { if waitJobsDone { s.tctx.L().Info("trigger flushFirstJob") waitJobsDone = false - flush = true + needFlush = true } }) - if flush { + if needFlush { s.jobWg.Add(1) s.dmlJobCh <- newFlushJob() s.jobWg.Wait() @@ -1005,7 +1005,7 @@ func (s *Syncer) addJob(job *job) error { } } - if flush || job.tp == ddl { + if needFlush || job.tp == ddl { // interrupted after save checkpoint and before flush checkpoint. failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err := handleFlushCheckpointStage(4, val.(int), "before flush checkpoint") @@ -2927,6 +2927,7 @@ func (s *Syncer) recordSkipSQLsLocation(ec *eventContext) error { return s.addJobFunc(job) } +// flushJobs add a flush job and wait for all jobs finished. func (s *Syncer) flushJobs() error { s.tctx.L().Info("flush all jobs", zap.Stringer("global checkpoint", s.checkpoint)) job := newFlushJob() diff --git a/tests/new_relay/configs/tasks/test.yaml b/tests/new_relay/configs/tasks/test.yaml index 17b0698cfc..42f0d2ec8b 100644 --- a/tests/new_relay/configs/tasks/test.yaml +++ b/tests/new_relay/configs/tasks/test.yaml @@ -74,5 +74,5 @@ mysql-instances: loader: null loader-thread: 0 syncer-config-name: sync-01 - syncer: null +syncer: null syncer-thread: 0 From c3c7d2a3c69023c4d3bdcd08d214ecd5013df94f Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 23:08:31 -0400 Subject: [PATCH 44/45] fix fmt --- tests/new_relay/configs/tasks/test.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/new_relay/configs/tasks/test.yaml b/tests/new_relay/configs/tasks/test.yaml index 42f0d2ec8b..17b0698cfc 100644 --- a/tests/new_relay/configs/tasks/test.yaml +++ b/tests/new_relay/configs/tasks/test.yaml @@ -74,5 +74,5 @@ mysql-instances: loader: null loader-thread: 0 syncer-config-name: sync-01 -syncer: null + syncer: null syncer-thread: 0 From 87ba1fbcaf018242c3268a2725d421ec76d60288 Mon Sep 17 00:00:00 2001 From: gmhdbjd Date: Mon, 18 Oct 2021 23:32:50 -0400 Subject: [PATCH 45/45] address comment --- Makefile | 2 -- 1 file changed, 2 deletions(-) diff --git a/Makefile b/Makefile index 8304807646..6770b11a00 100644 --- a/Makefile +++ b/Makefile @@ -146,8 +146,6 @@ fmt: tools_setup tools/bin/shfmt -l -w -d "tests/" ; \ echo "gofumports"; \ tools/bin/gofumports -w -d -local $(PACKAGE_NAME) $(PACKAGE_DIRECTORIES) 2>&1 | awk "{print} END{if(NR>0) {exit 1}}" ;\ - echo "golangci-lint"; \ - tools/bin/golangci-lint run --config=$(CURDIR)/.golangci.yml --issues-exit-code=1 $(PACKAGE_DIRECTORIES) ;\ fi lint: tools_setup