From 45b8a5c893df8dd65452acb9a14599ae7bab39b5 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 17 Jan 2022 15:40:12 +0800 Subject: [PATCH 01/72] reduce go routine size --- cdc/sorter/unified/merger.go | 116 ++++++++++++--------------- cdc/sorter/unified/unified_sorter.go | 23 ++---- 2 files changed, 58 insertions(+), 81 deletions(-) diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index b3fe8cab196..7bcd103b84c 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" - "golang.org/x/sync/errgroup" ) // TODO refactor this into a struct Merger. @@ -48,7 +47,7 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch metricSorterMergerStartTsGauge := sorterMergerStartTsGauge.WithLabelValues(captureAddr, changefeedID) metricSorterMergeCountHistogram := sorterMergeCountHistogram.WithLabelValues(captureAddr, changefeedID) - lastResolvedTs := make([]uint64, numSorters) + lastResolvedArrTs := make([]uint64, numSorters) minResolvedTs := uint64(0) var workingSet map[*flushTask]struct{} pendingSet := &sync.Map{} @@ -407,82 +406,67 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch resolvedTsNotifier := ¬ify.Notifier{} defer resolvedTsNotifier.Close() - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - for { - var task *flushTask - select { - case <-ctx.Done(): - return ctx.Err() - case task = <-in: - } - - if task == nil { - tableID, tableName := util.TableIDFromCtx(ctx) - log.Debug("Merger input channel closed, exiting", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName)) - return nil - } - - if !task.isEmpty { - pendingSet.Store(task, nil) - } // otherwise it is an empty flush + resolvedTsReceiver, err := resolvedTsNotifier.NewReceiver(time.Second * 1) + if err != nil { + if cerrors.ErrOperateOnClosedNotifier.Equal(err) { + // This won't happen unless `resolvedTsNotifier` has been closed, which is + // impossible at this point. + log.Panic("unexpected error", zap.Error(err)) + } + return errors.Trace(err) + } - if lastResolvedTs[task.heapSorterID] < task.maxResolvedTs { - lastResolvedTs[task.heapSorterID] = task.maxResolvedTs - } + defer resolvedTsReceiver.Stop() - minTemp := uint64(math.MaxUint64) - for _, ts := range lastResolvedTs { - if minTemp > ts { - minTemp = ts + var lastResolvedTs uint64 + for { + var task *flushTask + select { + case <-ctx.Done(): + return ctx.Err() + case <-resolvedTsReceiver.C: + curResolvedTs := atomic.LoadUint64(&minResolvedTs) + if curResolvedTs > lastResolvedTs { + err := onMinResolvedTsUpdate(curResolvedTs) + if err != nil { + return errors.Trace(err) } + } else if curResolvedTs < lastResolvedTs { + log.Panic("resolved-ts regressed in sorter", + zap.Uint64("curResolved-ts", curResolvedTs), + zap.Uint64("lastResolved-ts", lastResolvedTs)) } - - if minTemp > minResolvedTs { - atomic.StoreUint64(&minResolvedTs, minTemp) - resolvedTsNotifier.Notify() - } + case task = <-in: } - }) - - errg.Go(func() error { - resolvedTsReceiver, err := resolvedTsNotifier.NewReceiver(time.Second * 1) - if err != nil { - if cerrors.ErrOperateOnClosedNotifier.Equal(err) { - // This won't happen unless `resolvedTsNotifier` has been closed, which is - // impossible at this point. - log.Panic("unexpected error", zap.Error(err)) - } - return errors.Trace(err) + if task == nil { + tableID, tableName := util.TableIDFromCtx(ctx) + log.Debug("Merger input channel closed, exiting", + zap.Int64("tableID", tableID), + zap.String("tableName", tableName)) + return nil } - defer resolvedTsReceiver.Stop() + if !task.isEmpty { + pendingSet.Store(task, nil) + } // otherwise it is an empty flush - var lastResolvedTs uint64 - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-resolvedTsReceiver.C: - curResolvedTs := atomic.LoadUint64(&minResolvedTs) - if curResolvedTs > lastResolvedTs { - err := onMinResolvedTsUpdate(curResolvedTs) - if err != nil { - return errors.Trace(err) - } - } else if curResolvedTs < lastResolvedTs { - log.Panic("resolved-ts regressed in sorter", - zap.Uint64("curResolved-ts", curResolvedTs), - zap.Uint64("lastResolved-ts", lastResolvedTs)) - } + if lastResolvedArrTs[task.heapSorterID] < task.maxResolvedTs { + lastResolvedArrTs[task.heapSorterID] = task.maxResolvedTs + } + + minTemp := uint64(math.MaxUint64) + for _, ts := range lastResolvedArrTs { + if minTemp > ts { + minTemp = ts } } - }) - return errg.Wait() + if minTemp > minResolvedTs { + atomic.StoreUint64(&minResolvedTs, minTemp) + resolvedTsNotifier.Notify() + } + } } func mergerCleanUp(in <-chan *flushTask) { diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index b0061fb9f8f..8190a214f41 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -150,6 +150,10 @@ func (s *Sorter) Run(ctx context.Context) error { } } + errg.Go(func() error { + return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) + }) + errg.Go(func() error { defer func() { // cancelling the heapSorters from the outside @@ -161,20 +165,6 @@ func (s *Sorter) Run(ctx context.Context) error { failpoint.Inject("InjectHeapSorterExitDelay", func() {}) }() - select { - case <-subctx.Done(): - startCancel() - return errors.Trace(subctx.Err()) - case err := <-heapSorterErrCh: - return errors.Trace(err) - } - }) - - errg.Go(func() error { - return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) - }) - - errg.Go(func() error { captureAddr := util.CaptureAddrFromCtx(ctx) changefeedID := util.ChangefeedIDFromCtx(ctx) @@ -187,7 +177,10 @@ func (s *Sorter) Run(ctx context.Context) error { for { select { case <-subctx.Done(): - return subctx.Err() + startCancel() + return errors.Trace(subctx.Err()) + case err := <-heapSorterErrCh: + return errors.Trace(err) case event := <-s.inputCh: if event.RawKV != nil && event.RawKV.OpType == model.OpTypeResolved { // broadcast resolved events From e734e65a4b3ad482f8d348be0c1511778f63c372 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 17 Jan 2022 18:10:53 +0800 Subject: [PATCH 02/72] reduce go routine size --- cdc/sorter/unified/merger.go | 45 ++++++++++++++++++------------------ 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index 7bcd103b84c..df0ac08553e 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -438,33 +438,34 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch zap.Uint64("lastResolved-ts", lastResolvedTs)) } case task = <-in: - } - if task == nil { - tableID, tableName := util.TableIDFromCtx(ctx) - log.Debug("Merger input channel closed, exiting", - zap.Int64("tableID", tableID), - zap.String("tableName", tableName)) - return nil - } - if !task.isEmpty { - pendingSet.Store(task, nil) - } // otherwise it is an empty flush + if task == nil { + tableID, tableName := util.TableIDFromCtx(ctx) + log.Debug("Merger input channel closed, exiting", + zap.Int64("tableID", tableID), + zap.String("tableName", tableName)) + return nil + } - if lastResolvedArrTs[task.heapSorterID] < task.maxResolvedTs { - lastResolvedArrTs[task.heapSorterID] = task.maxResolvedTs - } + if !task.isEmpty { + pendingSet.Store(task, nil) + } // otherwise it is an empty flush - minTemp := uint64(math.MaxUint64) - for _, ts := range lastResolvedArrTs { - if minTemp > ts { - minTemp = ts + if lastResolvedArrTs[task.heapSorterID] < task.maxResolvedTs { + lastResolvedArrTs[task.heapSorterID] = task.maxResolvedTs } - } - if minTemp > minResolvedTs { - atomic.StoreUint64(&minResolvedTs, minTemp) - resolvedTsNotifier.Notify() + minTemp := uint64(math.MaxUint64) + for _, ts := range lastResolvedArrTs { + if minTemp > ts { + minTemp = ts + } + } + + if minTemp > minResolvedTs { + atomic.StoreUint64(&minResolvedTs, minTemp) + resolvedTsNotifier.Notify() + } } } } From 5c209ff46e43d21f4301b3b2f6fac12e97d6d77f Mon Sep 17 00:00:00 2001 From: sdojjy Date: Tue, 18 Jan 2022 10:13:59 +0800 Subject: [PATCH 03/72] reduce go routine size --- cdc/kv/client.go | 23 +++++++---------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 79be03f7a41..c6dd6a1ebe4 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -508,11 +508,18 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { return s.requestRegionToStore(ctx, g) }) + tableID, tableName := util.TableIDFromCtx(ctx) + cfID := util.ChangefeedIDFromCtx(ctx) g.Go(func() error { + timer := time.NewTimer(defaultCheckRegionRateLimitInterval) + defer timer.Stop() for { select { case <-ctx.Done(): return ctx.Err() + case <-timer.C: + s.handleRateLimit(ctx) + timer.Reset(defaultCheckRegionRateLimitInterval) case task := <-s.requestRangeCh: s.rangeChSizeGauge.Dec() // divideAndSendEventFeedToRegions could be block for some time, @@ -526,22 +533,6 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { g.Go(func() error { return s.divideAndSendEventFeedToRegions(ctx, task.span, task.ts) }) - } - } - }) - - tableID, tableName := util.TableIDFromCtx(ctx) - cfID := util.ChangefeedIDFromCtx(ctx) - g.Go(func() error { - timer := time.NewTimer(defaultCheckRegionRateLimitInterval) - defer timer.Stop() - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-timer.C: - s.handleRateLimit(ctx) - timer.Reset(defaultCheckRegionRateLimitInterval) case errInfo := <-s.errCh: s.errChSizeGauge.Dec() allowed := s.checkRateLimit(errInfo.singleRegionInfo.verID.GetID()) From a6b74d6c48e45d3df17264cb9bfaccea478ce8a3 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Tue, 18 Jan 2022 12:58:38 +0800 Subject: [PATCH 04/72] reduce go routine size --- cdc/puller/puller.go | 155 +++++++++++++++++++++---------------------- 1 file changed, 76 insertions(+), 79 deletions(-) diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index 56be744b400..d2345c8fd22 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -140,92 +140,89 @@ func (p *pullerImpl) Run(ctx context.Context) error { }() lastResolvedTs := p.checkpointTs - g.Go(func() error { - metricsTicker := time.NewTicker(15 * time.Second) - defer metricsTicker.Stop() - output := func(raw *model.RawKVEntry) error { - // even after https://github.com/pingcap/tiflow/pull/2038, kv client - // could still miss region change notification, which leads to resolved - // ts update missing in puller, however resolved ts fallback here can - // be ignored since no late data is received and the guarantee of - // resolved ts is not broken. - if raw.CRTs < p.resolvedTs || (raw.CRTs == p.resolvedTs && raw.OpType != model.OpTypeResolved) { - log.Warn("The CRTs is fallen back in puller", - zap.Reflect("row", raw), - zap.Uint64("CRTs", raw.CRTs), - zap.Uint64("resolvedTs", p.resolvedTs), - zap.Int64("tableID", tableID)) - return nil - } - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case p.outputCh <- raw: - } + metricsTicker := time.NewTicker(15 * time.Second) + defer metricsTicker.Stop() + output := func(raw *model.RawKVEntry) error { + // even after https://github.com/pingcap/tiflow/pull/2038, kv client + // could still miss region change notification, which leads to resolved + // ts update missing in puller, however resolved ts fallback here can + // be ignored since no late data is received and the guarantee of + // resolved ts is not broken. + if raw.CRTs < p.resolvedTs || (raw.CRTs == p.resolvedTs && raw.OpType != model.OpTypeResolved) { + log.Warn("The CRTs is fallen back in puller", + zap.Reflect("row", raw), + zap.Uint64("CRTs", raw.CRTs), + zap.Uint64("resolvedTs", p.resolvedTs), + zap.Int64("tableID", tableID)) return nil } + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case p.outputCh <- raw: + } + return nil + } - start := time.Now() - initialized := false - for { - var e model.RegionFeedEvent - select { - case <-metricsTicker.C: - metricEventChanSize.Observe(float64(len(eventCh))) - metricOutputChanSize.Observe(float64(len(p.outputCh))) - metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) - continue - case e = <-eventCh: - case <-ctx.Done(): - return errors.Trace(ctx.Err()) + start := time.Now() + initialized := false + for { + var e model.RegionFeedEvent + select { + case <-metricsTicker.C: + metricEventChanSize.Observe(float64(len(eventCh))) + metricOutputChanSize.Observe(float64(len(p.outputCh))) + metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) + continue + case e = <-eventCh: + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + } + if e.Val != nil { + metricTxnCollectCounterKv.Inc() + if err := output(e.Val); err != nil { + return errors.Trace(err) } - if e.Val != nil { - metricTxnCollectCounterKv.Inc() - if err := output(e.Val); err != nil { - return errors.Trace(err) - } - } else if e.Resolved != nil { - metricTxnCollectCounterResolved.Inc() - if !regionspan.IsSubSpan(e.Resolved.Span, p.spans...) { - log.Panic("the resolved span is not in the total span", - zap.Reflect("resolved", e.Resolved), - zap.Int64("tableID", tableID), - zap.Reflect("spans", p.spans), - ) - } - // Forward is called in a single thread - p.tsTracker.Forward(e.Resolved.Span, e.Resolved.ResolvedTs) - resolvedTs := p.tsTracker.Frontier() - if resolvedTs > 0 && !initialized { - // Advancing to a non-zero value means the puller level - // resolved ts is initialized. - atomic.StoreInt64(&p.initialized, 1) - initialized = true - - spans := make([]string, 0, len(p.spans)) - for i := range p.spans { - spans = append(spans, p.spans[i].String()) - } - log.Info("puller is initialized", - zap.Duration("duration", time.Since(start)), - zap.String("changefeed", changefeedID), - zap.Int64("tableID", tableID), - zap.Strings("spans", spans), - zap.Uint64("resolvedTs", resolvedTs)) - } - if !initialized || resolvedTs == lastResolvedTs { - continue - } - lastResolvedTs = resolvedTs - err := output(&model.RawKVEntry{CRTs: resolvedTs, OpType: model.OpTypeResolved, RegionID: e.RegionID}) - if err != nil { - return errors.Trace(err) + } else if e.Resolved != nil { + metricTxnCollectCounterResolved.Inc() + if !regionspan.IsSubSpan(e.Resolved.Span, p.spans...) { + log.Panic("the resolved span is not in the total span", + zap.Reflect("resolved", e.Resolved), + zap.Int64("tableID", tableID), + zap.Reflect("spans", p.spans), + ) + } + // Forward is called in a single thread + p.tsTracker.Forward(e.Resolved.Span, e.Resolved.ResolvedTs) + resolvedTs := p.tsTracker.Frontier() + if resolvedTs > 0 && !initialized { + // Advancing to a non-zero value means the puller level + // resolved ts is initialized. + atomic.StoreInt64(&p.initialized, 1) + initialized = true + + spans := make([]string, 0, len(p.spans)) + for i := range p.spans { + spans = append(spans, p.spans[i].String()) } - atomic.StoreUint64(&p.resolvedTs, resolvedTs) + log.Info("puller is initialized", + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", changefeedID), + zap.Int64("tableID", tableID), + zap.Strings("spans", spans), + zap.Uint64("resolvedTs", resolvedTs)) } + if !initialized || resolvedTs == lastResolvedTs { + continue + } + lastResolvedTs = resolvedTs + err := output(&model.RawKVEntry{CRTs: resolvedTs, OpType: model.OpTypeResolved, RegionID: e.RegionID}) + if err != nil { + return errors.Trace(err) + } + atomic.StoreUint64(&p.resolvedTs, resolvedTs) } - }) - return g.Wait() + } } func (p *pullerImpl) GetResolvedTs() uint64 { From 1fe5ab537f35a6277fe78aea935e9fde45a329f6 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Wed, 19 Jan 2022 14:31:01 +0800 Subject: [PATCH 05/72] address comments --- cdc/kv/client.go | 11 ++++++----- cdc/kv/region_worker.go | 9 +++++---- cdc/sorter/unified/unified_sorter.go | 8 +++++--- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index c6dd6a1ebe4..c77fc9bebd9 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -498,6 +498,8 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { log.Debug("event feed started", zap.Stringer("span", s.totalSpan), zap.Uint64("ts", ts)) + ctx, cancel := context.WithCancel(ctx) + defer cancel() g, ctx := errgroup.WithContext(ctx) g.Go(func() error { @@ -564,13 +566,12 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { } }) - g.Go(func() error { - return s.regionRouter.Run(ctx) - }) - s.requestRangeCh <- rangeRequestTask{span: s.totalSpan, ts: ts} s.rangeChSizeGauge.Inc() - + err := s.regionRouter.Run(ctx) + if err != nil { + return err + } return g.Wait() } diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index 9f288418183..95f2ead526e 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -599,15 +599,16 @@ func (w *regionWorker) run(parentCtx context.Context) error { wg.Go(func() error { return w.eventHandler(ctx) }) - wg.Go(func() error { - return w.collectWorkpoolError(ctx) - }) - err := wg.Wait() + err := w.collectWorkpoolError(ctx) // ErrRegionWorkerExit means the region worker exits normally, but we don't // need to terminate the other goroutines in errgroup if cerror.ErrRegionWorkerExit.Equal(err) { return nil } + err = wg.Wait() + if cerror.ErrRegionWorkerExit.Equal(err) { + return nil + } return err } diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index 8190a214f41..4cabfc8caa6 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -154,7 +154,7 @@ func (s *Sorter) Run(ctx context.Context) error { return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) }) - errg.Go(func() error { + err := func() error { defer func() { // cancelling the heapSorters from the outside for _, hs := range heapSorters { @@ -224,8 +224,10 @@ func (s *Sorter) Run(ctx context.Context) error { } } } - }) - + }() + if err != nil { + return printError(err) + } return printError(errg.Wait()) } From 162401b92fbb23e105c40d57df8be50163d42bee Mon Sep 17 00:00:00 2001 From: sdojjy Date: Thu, 20 Jan 2022 11:35:17 +0800 Subject: [PATCH 06/72] remove notifier --- cdc/sorter/unified/merger.go | 54 +++++++++++++++++++----------------- 1 file changed, 28 insertions(+), 26 deletions(-) diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index df0ac08553e..37d10d2848c 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sorter" cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/notify" "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" @@ -404,38 +403,38 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch return nil } - resolvedTsNotifier := ¬ify.Notifier{} - defer resolvedTsNotifier.Close() + resolvedTsTicker := time.NewTicker(time.Second * 1) + resolvedTsTickCh := make(chan struct{}) - resolvedTsReceiver, err := resolvedTsNotifier.NewReceiver(time.Second * 1) - if err != nil { - if cerrors.ErrOperateOnClosedNotifier.Equal(err) { - // This won't happen unless `resolvedTsNotifier` has been closed, which is - // impossible at this point. - log.Panic("unexpected error", zap.Error(err)) - } - return errors.Trace(err) - } - - defer resolvedTsReceiver.Stop() + defer resolvedTsTicker.Stop() var lastResolvedTs uint64 + resolvedTsTickFunc := func() error { + curResolvedTs := atomic.LoadUint64(&minResolvedTs) + if curResolvedTs > lastResolvedTs { + err := onMinResolvedTsUpdate(curResolvedTs) + if err != nil { + return errors.Trace(err) + } + } else if curResolvedTs < lastResolvedTs { + log.Panic("resolved-ts regressed in sorter", + zap.Uint64("curResolved-ts", curResolvedTs), + zap.Uint64("lastResolved-ts", lastResolvedTs)) + } + return nil + } for { var task *flushTask select { case <-ctx.Done(): return ctx.Err() - case <-resolvedTsReceiver.C: - curResolvedTs := atomic.LoadUint64(&minResolvedTs) - if curResolvedTs > lastResolvedTs { - err := onMinResolvedTsUpdate(curResolvedTs) - if err != nil { - return errors.Trace(err) - } - } else if curResolvedTs < lastResolvedTs { - log.Panic("resolved-ts regressed in sorter", - zap.Uint64("curResolved-ts", curResolvedTs), - zap.Uint64("lastResolved-ts", lastResolvedTs)) + case <-resolvedTsTicker.C: + if err := resolvedTsTickFunc(); err != nil { + return err + } + case <-resolvedTsTickCh: + if err := resolvedTsTickFunc(); err != nil { + return err } case task = <-in: @@ -464,7 +463,10 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch if minTemp > minResolvedTs { atomic.StoreUint64(&minResolvedTs, minTemp) - resolvedTsNotifier.Notify() + select { + case resolvedTsTickCh <- struct{}{}: + default: + } } } } From 239a21fe146a2b2950dad06e7d4e201791d4d030 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Jan 2022 11:19:47 +0800 Subject: [PATCH 07/72] build(deps): bump nanoid from 3.1.30 to 3.2.0 in /dm/ui (#4437) ref pingcap/tiflow#4418 --- dm/ui/yarn.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dm/ui/yarn.lock b/dm/ui/yarn.lock index 55fc76b6156..e2d36417eab 100644 --- a/dm/ui/yarn.lock +++ b/dm/ui/yarn.lock @@ -3133,9 +3133,9 @@ mute-stream@0.0.8: integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA== nanoid@^3.1.22, nanoid@^3.1.30: - version "3.1.30" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.1.30.tgz#63f93cc548d2a113dc5dfbc63bfa09e2b9b64362" - integrity sha512-zJpuPDwOv8D2zq2WRoMe1HsfZthVewpel9CAvTfc/2mBD1uUT/agc5f7GHGWXlYkFvi1mVxe4IjvP2HNrop7nQ== + version "3.2.0" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.2.0.tgz#62667522da6673971cca916a6d3eff3f415ff80c" + integrity sha512-fmsZYa9lpn69Ad5eDn7FMcnnSR+8R34W9qJEijxYhTbfOWzr22n1QxCMzXLK+ODyW2973V3Fux959iQoUxzUIA== natural-compare@^1.4.0: version "1.4.0" From e5ee9da0f20c56de86645363369fcc267776c90d Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 24 Jan 2022 11:52:40 +0800 Subject: [PATCH 08/72] wait merger --- cdc/sorter/unified/unified_sorter.go | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index a1ca7abcd71..5048e3aadcf 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -125,6 +125,11 @@ func (s *Sorter) Run(ctx context.Context) error { sorterConfig := config.GetGlobalServerConfig().Sorter numConcurrentHeaps := sorterConfig.NumConcurrentWorker + ctx, cancel := context.WithCancel(ctx) + var cancelOnce sync.Once + defer func() { + cancelOnce.Do(func() { cancel() }) + }() errg, subctx := errgroup.WithContext(ctx) heapSorterCollectCh := make(chan *flushTask, heapCollectChSize) // mergerCleanUp will consumer the remaining elements in heapSorterCollectCh to prevent any FD leak. @@ -151,7 +156,11 @@ func (s *Sorter) Run(ctx context.Context) error { } errg.Go(func() error { - return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) + err := printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) + if err != nil { + cancelOnce.Do(func() { cancel() }) + } + return err }) err := func() error { @@ -226,9 +235,10 @@ func (s *Sorter) Run(ctx context.Context) error { } }() if err != nil { + cancelOnce.Do(func() { cancel() }) return printError(err) } - return printError(errg.Wait()) + return errg.Wait() } // AddEntry implements the EventSorter interface From 9dfefbbdc358e8fec10e9aa7e4a4619ba0ceec04 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 24 Jan 2022 11:54:13 +0800 Subject: [PATCH 09/72] wait merger --- cdc/sorter/unified/backend_pool.go | 2 +- pkg/version/version.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go index 03e93ddf89f..703882ab103 100644 --- a/cdc/sorter/unified/backend_pool.go +++ b/cdc/sorter/unified/backend_pool.go @@ -385,7 +385,7 @@ func (p *backEndPool) cleanUpStaleFiles() error { // checkDataDirSatisfied check if the data-dir meet the requirement during server running // the caller should guarantee that dir exist func checkDataDirSatisfied() error { - const dataDirAvailLowThreshold = 1 // percentage + const dataDirAvailLowThreshold = 10 // percentage conf := config.GetGlobalServerConfig() diskInfo, err := fsutil.GetDiskInfo(conf.DataDir) diff --git a/pkg/version/version.go b/pkg/version/version.go index 0892b80dd73..ba4d2f78e8a 100644 --- a/pkg/version/version.go +++ b/pkg/version/version.go @@ -24,7 +24,7 @@ import ( // Version information. var ( - ReleaseVersion = "v5.3.0" + ReleaseVersion = "None" BuildTS = "None" GitHash = "None" GitBranch = "None" From 0236882e2134fd6cf9329ae9b78462a416a978f8 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 24 Jan 2022 12:29:47 +0800 Subject: [PATCH 10/72] pipeline(ticdc): reduce go routine size (#4334) ref pingcap/tiflow#4351 --- cdc/processor/pipeline/sorter.go | 12 ++++-------- cdc/processor/processor.go | 12 +----------- cdc/puller/puller.go | 19 +++++++------------ cdc/sorter/unified/unified_sorter.go | 3 ++- pkg/util/cancel_monitor.go | 19 ++++++++++--------- 5 files changed, 24 insertions(+), 41 deletions(-) diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index 3032c41fcaa..0368d27a758 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -140,14 +140,6 @@ func (n *sorterNode) StartActorNode(ctx pipeline.NodeContext, isTableActorMode b ctx.Throw(errors.Trace(eventSorter.Run(stdCtx))) return nil }) - n.eg.Go(func() error { - // Since the flowController is implemented by `Cond`, it is not cancelable - // by a context. We need to listen on cancellation and aborts the flowController - // manually. - <-stdCtx.Done() - n.flowController.Abort() - return nil - }) n.eg.Go(func() error { lastSentResolvedTs := uint64(0) lastSendResolvedTsTime := time.Now() // the time at which we last sent a resolved-ts. @@ -314,6 +306,10 @@ func (n *sorterNode) Destroy(ctx pipeline.NodeContext) error { log.Warn("schedule table cleanup task failed", zap.Error(err)) } } + // Since the flowController is implemented by `Cond`, it is not cancelable by a context + // the flowController will be blocked in a background goroutine, + // We need to abort the flowController manually in the nodeRunner + n.flowController.Abort() return n.eg.Wait() } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index e4f9251acd3..b66f7831089 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -363,6 +363,7 @@ func (p *processor) tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR p.handleWorkload() } p.doGCSchemaStorage(ctx) + p.metricSyncTableNumGauge.Set(float64(len(p.tables))) if p.newSchedulerEnabled { if err := p.agent.Tick(ctx); err != nil { @@ -974,17 +975,6 @@ func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID mode sink, p.changefeed.Info.GetTargetTs(), ) - p.wg.Add(1) - p.metricSyncTableNumGauge.Inc() - go func() { - table.Wait() - p.wg.Done() - p.metricSyncTableNumGauge.Dec() - log.Debug("Table pipeline exited", zap.Int64("tableID", tableID), - cdcContext.ZapFieldChangefeed(ctx), - zap.String("name", table.Name()), - zap.Any("replicaInfo", replicaInfo)) - }() if p.redoManager.Enabled() { p.redoManager.AddTable(tableID, replicaInfo.StartTs) diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index a04831a0eba..ed698dec057 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -139,21 +139,11 @@ func (p *pullerImpl) Run(ctx context.Context) error { txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "kv") txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "resolved") }() - g.Go(func() error { - for { - select { - case <-ctx.Done(): - return nil - case <-time.After(15 * time.Second): - metricEventChanSize.Observe(float64(len(eventCh))) - metricOutputChanSize.Observe(float64(len(p.outputCh))) - metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) - } - } - }) lastResolvedTs := p.checkpointTs g.Go(func() error { + metricsTicker := time.NewTicker(15 * time.Second) + defer metricsTicker.Stop() output := func(raw *model.RawKVEntry) error { // even after https://github.com/pingcap/tiflow/pull/2038, kv client // could still miss region change notification, which leads to resolved @@ -181,6 +171,11 @@ func (p *pullerImpl) Run(ctx context.Context) error { for { var e model.RegionFeedEvent select { + case <-metricsTicker.C: + metricEventChanSize.Observe(float64(len(eventCh))) + metricOutputChanSize.Observe(float64(len(p.outputCh))) + metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) + continue case e = <-eventCh: case <-ctx.Done(): return errors.Trace(ctx.Err()) diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index e8c1770a8f9..42a0394e0ec 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -114,7 +114,7 @@ func (s *Sorter) Run(ctx context.Context) error { defer close(s.closeCh) - finish := util.MonitorCancelLatency(ctx, "Unified Sorter") + finish, startCancel := util.MonitorCancelLatency(ctx, "Unified Sorter") defer finish() ctx = context.WithValue(ctx, ctxKey{}, s) @@ -163,6 +163,7 @@ func (s *Sorter) Run(ctx context.Context) error { select { case <-subctx.Done(): + startCancel() return errors.Trace(subctx.Err()) case err := <-heapSorterErrCh: return errors.Trace(err) diff --git a/pkg/util/cancel_monitor.go b/pkg/util/cancel_monitor.go index fb4953bc9cb..3e22ce4d288 100644 --- a/pkg/util/cancel_monitor.go +++ b/pkg/util/cancel_monitor.go @@ -21,12 +21,14 @@ import ( "go.uber.org/zap" ) -// MonitorCancelLatency monitors the latency from ctx being cancelled and the returned function being called -func MonitorCancelLatency(ctx context.Context, identifier string) func() { +// MonitorCancelLatency monitors the latency from ctx being cancelled +// the first returned function should be called when the cancellation is done +// the second returned function should be called to mark the cancellation is started, it will start a +// background go routine to monitor the latency util finish is called or cancellation is done +func MonitorCancelLatency(ctx context.Context, identifier string) (func(), func()) { finishedCh := make(chan struct{}) - go func() { - select { - case <-ctx.Done(): + start := func() { + go func() { log.Debug("MonitorCancelLatency: Cancelled", zap.String("identifier", identifier)) ticker := time.NewTicker(time.Second) defer ticker.Stop() @@ -43,10 +45,9 @@ func MonitorCancelLatency(ctx context.Context, identifier string) func() { zap.Int("duration", elapsed), zap.Error(ctx.Err())) } } - case <-finishedCh: - } - }() + }() + } return func() { close(finishedCh) - } + }, start } From 3e43ac02e0e86310c711c81a388cb48e241a699b Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Mon, 24 Jan 2022 15:00:07 +0800 Subject: [PATCH 11/72] mq (ticdc): Make mq sink flush resolvedTs async (#4356) ref pingcap/tiflow#4222 --- cdc/sink/mq.go | 102 ++++++++++++++++++++++++++++++++++------------ cdc/sink/mysql.go | 3 ++ 2 files changed, 79 insertions(+), 26 deletions(-) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index afca2bc9a7d..898cc45a204 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -17,6 +17,7 @@ import ( "context" "net/url" "strings" + "sync" "sync/atomic" "time" @@ -39,11 +40,21 @@ import ( type mqEvent struct { row *model.RowChangedEvent - resolvedTs uint64 + resolvedTs model.Ts +} + +type resolvedTsEvent struct { + tableID model.TableID + resolvedTs model.Ts } const ( - defaultPartitionInputChSize = 12800 + // Depend on this size, every `partitionInputCh` will take + // approximately 16.3 KiB memory. + defaultPartitionInputChSize = 1024 + // Depend on this size, `resolvedBuffer` will take + // approximately 2 KiB memory. + defaultResolvedTsEventBufferSize = 128 // -1 means broadcast to all partitions, it's the default for the default open protocol. defaultDDLDispatchPartition = -1 ) @@ -55,12 +66,13 @@ type mqSink struct { filter *filter.Filter protocol config.Protocol - partitionNum int32 - partitionInput []chan mqEvent - partitionResolvedTs []uint64 - tableCheckpointTs map[model.TableID]uint64 - resolvedNotifier *notify.Notifier - resolvedReceiver *notify.Receiver + partitionNum int32 + partitionInput []chan mqEvent + partitionResolvedTs []uint64 + tableCheckpointTsMap sync.Map + resolvedBuffer chan resolvedTsEvent + resolvedNotifier *notify.Notifier + resolvedReceiver *notify.Receiver statistics *Statistics } @@ -110,9 +122,10 @@ func newMqSink( partitionNum: partitionNum, partitionInput: partitionInput, partitionResolvedTs: make([]uint64, partitionNum), - tableCheckpointTs: make(map[model.TableID]uint64), - resolvedNotifier: notifier, - resolvedReceiver: resolvedReceiver, + + resolvedBuffer: make(chan resolvedTsEvent, defaultResolvedTsEventBufferSize), + resolvedNotifier: notifier, + resolvedReceiver: resolvedReceiver, statistics: NewStatistics(ctx, "MQ", opts), } @@ -164,19 +177,59 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha return nil } +// FlushRowChangedEvents is thread-safety func (k *mqSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { - if checkpointTs, ok := k.tableCheckpointTs[tableID]; ok && resolvedTs <= checkpointTs { + var checkpointTs uint64 + v, ok := k.tableCheckpointTsMap.Load(tableID) + if ok { + checkpointTs = v.(uint64) + } + if resolvedTs <= checkpointTs { return checkpointTs, nil } + select { + case <-ctx.Done(): + return 0, ctx.Err() + case k.resolvedBuffer <- resolvedTsEvent{ + tableID: tableID, + resolvedTs: resolvedTs, + }: + } + k.statistics.PrintStatus(ctx) + return checkpointTs, nil +} + +// bgFlushTs flush resolvedTs to workers and flush the mqProducer +func (k *mqSink) bgFlushTs(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case msg := <-k.resolvedBuffer: + resolvedTs := msg.resolvedTs + err := k.flushTsToWorker(ctx, resolvedTs) + if err != nil { + return errors.Trace(err) + } + err = k.mqProducer.Flush(ctx) + if err != nil { + return errors.Trace(err) + } + // Since CDC does not guarantee exactly once semantic, it won't cause any problem + // here even if the table was moved or removed. + // ref: https://github.com/pingcap/tiflow/pull/4356#discussion_r787405134 + k.tableCheckpointTsMap.Store(msg.tableID, resolvedTs) + } + } +} +func (k *mqSink) flushTsToWorker(ctx context.Context, resolvedTs model.Ts) error { + // flush resolvedTs to all partition workers for i := 0; i < int(k.partitionNum); i++ { select { case <-ctx.Done(): - return 0, ctx.Err() - case k.partitionInput[i] <- struct { - row *model.RowChangedEvent - resolvedTs uint64 - }{resolvedTs: resolvedTs}: + return errors.Trace(ctx.Err()) + case k.partitionInput[i] <- mqEvent{resolvedTs: resolvedTs}: } } @@ -185,23 +238,16 @@ flushLoop: for { select { case <-ctx.Done(): - return 0, ctx.Err() + return errors.Trace(ctx.Err()) case <-k.resolvedReceiver.C: for i := 0; i < int(k.partitionNum); i++ { if resolvedTs > atomic.LoadUint64(&k.partitionResolvedTs[i]) { continue flushLoop } } - break flushLoop + return nil } } - err := k.mqProducer.Flush(ctx) - if err != nil { - return 0, errors.Trace(err) - } - k.tableCheckpointTs[tableID] = resolvedTs - k.statistics.PrintStatus(ctx) - return resolvedTs, nil } func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -273,6 +319,9 @@ func (k *mqSink) Barrier(cxt context.Context, tableID model.TableID) error { func (k *mqSink) run(ctx context.Context) error { defer k.resolvedReceiver.Stop() wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { + return k.bgFlushTs(ctx) + }) for i := int32(0); i < k.partitionNum; i++ { partition := i wg.Go(func() error { @@ -331,6 +380,7 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { continue case e = <-input: } + // flush resolvedTs event if e.row == nil { if e.resolvedTs != 0 { op, err := encoder.AppendResolvedEvent(e.resolvedTs) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 081ad8bd2c6..d2946f85197 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -224,6 +224,9 @@ func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row // FlushRowChangedEvents will flush all received events, we don't allow mysql // sink to receive events before resolving func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { + // Since CDC does not guarantee exactly once semantic, it won't cause any problem + // here even if the table was moved or removed. + // ref: https://github.com/pingcap/tiflow/pull/4356#discussion_r787405134 v, ok := s.tableMaxResolvedTs.Load(tableID) if !ok || v.(uint64) < resolvedTs { s.tableMaxResolvedTs.Store(tableID, resolvedTs) From 39aaf995aa7ff5e6820c3726e64b3a536dd65f50 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Mon, 24 Jan 2022 15:02:07 +0800 Subject: [PATCH 12/72] mq (ticdc): Make mq sink flush resolvedTs async (#4356) ref pingcap/tiflow#4222 From a39d7b7928fe4c6fc80ea12198795e46f42047c8 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 24 Jan 2022 16:37:35 +0800 Subject: [PATCH 13/72] use ticker instead of timer --- cdc/kv/client.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 6aa0f5c6cb6..8169fdcf4fa 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -528,15 +528,14 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { tableID, tableName := util.TableIDFromCtx(ctx) cfID := util.ChangefeedIDFromCtx(ctx) g.Go(func() error { - timer := time.NewTimer(defaultCheckRegionRateLimitInterval) - defer timer.Stop() + checkRegionRateLimitTicker := time.NewTicker(defaultCheckRegionRateLimitInterval) + defer checkRegionRateLimitTicker.Stop() for { select { case <-ctx.Done(): return ctx.Err() - case <-timer.C: + case <-checkRegionRateLimitTicker.C: s.handleRateLimit(ctx) - timer.Reset(defaultCheckRegionRateLimitInterval) case task := <-s.requestRangeCh: s.rangeChSizeGauge.Dec() // divideAndSendEventFeedToRegions could be block for some time, From 2b93ae46e1490fb1be5b40e2f430622b846b8b5c Mon Sep 17 00:00:00 2001 From: sdojjy Date: Mon, 24 Jan 2022 16:58:15 +0800 Subject: [PATCH 14/72] use once --- cdc/kv/region_worker.go | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index 67afd10c888..a8bab8c4379 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -600,25 +600,33 @@ func (w *regionWorker) run(parentCtx context.Context) error { } }() w.parentCtx = parentCtx - wg, ctx := errgroup.WithContext(parentCtx) + ctx, cancel := context.WithCancel(parentCtx) + once := sync.Once{} + wg, ctx := errgroup.WithContext(ctx) w.initMetrics(ctx) w.initPoolHandles(w.concurrent) wg.Go(func() error { - return w.checkErrorReconnect(w.resolveLock(ctx)) + return w.handleError(cancel, w.checkErrorReconnect(w.resolveLock(ctx)), &once) }) wg.Go(func() error { - return w.eventHandler(ctx) + return w.handleError(cancel, w.eventHandler(ctx), &once) }) err := w.collectWorkpoolError(ctx) + if err != nil { + return w.handleError(cancel, err, &once) + } + return wg.Wait() +} + +func (w *regionWorker) handleError(cancel context.CancelFunc, err error, once *sync.Once) error { + once.Do(func() { + cancel() + }) // ErrRegionWorkerExit means the region worker exits normally, but we don't // need to terminate the other goroutines in errgroup if cerror.ErrRegionWorkerExit.Equal(err) { return nil } - err = wg.Wait() - if cerror.ErrRegionWorkerExit.Equal(err) { - return nil - } return err } From b196894a27e448b5e3ef26d76bf308437ea62ee8 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Mon, 24 Jan 2022 19:20:10 +0800 Subject: [PATCH 15/72] openAPI (ticdc): add some unit tests and migrate related test to testify (#4387) ref pingcap/tiflow#2411 --- cdc/api/middleware.go | 72 ++++++ cdc/api/open.go | 115 +++++----- cdc/api/open_test.go | 450 ++++++++++++++++++++++++++++++++++++++ cdc/api/owner.go | 17 +- cdc/api/owner_test.go | 110 +++------- cdc/api/router.go | 73 ------- cdc/api/router_test.go | 60 ----- cdc/api/util_test.go | 1 + cdc/api/validator_test.go | 1 + cdc/capture/capture.go | 8 +- cdc/cdc_test.go | 22 -- cdc/http.go | 115 ++++------ cdc/http_test.go | 236 ++++++-------------- cdc/server.go | 10 +- cdc/server_test.go | 236 ++++++++++++++++---- 15 files changed, 940 insertions(+), 586 deletions(-) create mode 100644 cdc/api/middleware.go create mode 100644 cdc/api/open_test.go delete mode 100644 cdc/api/router.go delete mode 100644 cdc/api/router_test.go delete mode 100644 cdc/cdc_test.go diff --git a/cdc/api/middleware.go b/cdc/api/middleware.go new file mode 100644 index 00000000000..0abe7c38cf0 --- /dev/null +++ b/cdc/api/middleware.go @@ -0,0 +1,72 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package api + +import ( + "net/http" + "time" + + "github.com/gin-gonic/gin" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "go.uber.org/zap" +) + +func logMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + start := time.Now() + path := c.Request.URL.Path + query := c.Request.URL.RawQuery + c.Next() + + cost := time.Since(start) + + err := c.Errors.Last() + var stdErr error + if err != nil { + stdErr = err.Err + } + + log.Info(path, + zap.Int("status", c.Writer.Status()), + zap.String("method", c.Request.Method), + zap.String("path", path), + zap.String("query", query), + zap.String("ip", c.ClientIP()), + zap.String("user-agent", c.Request.UserAgent()), + zap.Error(stdErr), + zap.Duration("duration", cost), + ) + } +} + +func errorHandleMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + c.Next() + // because we will return immediately after an error occurs in http_handler + // there wil be only one error in c.Errors + lastError := c.Errors.Last() + if lastError != nil { + err := lastError.Err + // put the error into response + if IsHTTPBadRequestError(err) { + c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + } else { + c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + } + c.Abort() + return + } + } +} diff --git a/cdc/api/open.go b/cdc/api/open.go index 80041ffeca6..14f0b29c403 100644 --- a/cdc/api/open.go +++ b/cdc/api/open.go @@ -49,41 +49,62 @@ const ( // openAPI provides capture APIs. type openAPI struct { capture *capture.Capture + // use for unit test only + testStatusProvider owner.StatusProvider } -// RegisterOpoenAPIRoutes registers routes for OpenAPI -func RegisterOpoenAPIRoutes(router *gin.Engine, capture *capture.Capture) { - openAPI := openAPI{capture: capture} +func NewOpenAPI(c *capture.Capture) openAPI { + return openAPI{capture: c} +} + +// NewOpenAPI4Test return a openAPI for test +func NewOpenAPI4Test(c *capture.Capture, p owner.StatusProvider) openAPI { + return openAPI{capture: c, testStatusProvider: p} +} + +func (h *openAPI) statusProvider() owner.StatusProvider { + if h.testStatusProvider != nil { + return h.testStatusProvider + } + return h.capture.StatusProvider() +} + +// RegisterOpenAPIRoutes registers routes for OpenAPI +func RegisterOpenAPIRoutes(router *gin.Engine, api openAPI) { + v1 := router.Group("/api/v1") + + v1.Use(logMiddleware()) + v1.Use(errorHandleMiddleware()) // common API - router.GET("/api/v1/status", openAPI.ServerStatus) - router.GET("/api/v1/health", openAPI.Health) - router.POST("/api/v1/log", SetLogLevel) + v1.GET("/status", api.ServerStatus) + v1.GET("/health", api.Health) + v1.POST("/log", SetLogLevel) // changefeed API - changefeedGroup := router.Group("/api/v1/changefeeds") - changefeedGroup.GET("", openAPI.ListChangefeed) - changefeedGroup.GET("/:changefeed_id", openAPI.GetChangefeed) - changefeedGroup.POST("", openAPI.CreateChangefeed) - changefeedGroup.PUT("/:changefeed_id", openAPI.UpdateChangefeed) - changefeedGroup.POST("/:changefeed_id/pause", openAPI.PauseChangefeed) - changefeedGroup.POST("/:changefeed_id/resume", openAPI.ResumeChangefeed) - changefeedGroup.DELETE("/:changefeed_id", openAPI.RemoveChangefeed) - changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", openAPI.RebalanceTable) - changefeedGroup.POST("/:changefeed_id/tables/move_table", openAPI.MoveTable) + changefeedGroup := v1.Group("/changefeeds") + changefeedGroup.GET("", api.ListChangefeed) + changefeedGroup.GET("/:changefeed_id", api.GetChangefeed) + changefeedGroup.POST("", api.CreateChangefeed) + changefeedGroup.PUT("/:changefeed_id", api.UpdateChangefeed) + changefeedGroup.POST("/:changefeed_id/pause", api.PauseChangefeed) + changefeedGroup.POST("/:changefeed_id/resume", api.ResumeChangefeed) + changefeedGroup.DELETE("/:changefeed_id", api.RemoveChangefeed) + changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", api.RebalanceTable) + changefeedGroup.POST("/:changefeed_id/tables/move_table", api.MoveTable) // owner API - ownerGroup := router.Group("/api/v1/owner") - ownerGroup.POST("/resign", openAPI.ResignOwner) + ownerGroup := v1.Group("/owner") + ownerGroup.POST("/resign", api.ResignOwner) // processor API - processorGroup := router.Group("/api/v1/processors") - processorGroup.GET("", openAPI.ListProcessor) - processorGroup.GET("/:changefeed_id/:capture_id", openAPI.GetProcessor) + processorGroup := v1.Group("/processors") + processorGroup.GET("", api.ListProcessor) + processorGroup.GET("/:changefeed_id/:capture_id", api.GetProcessor) // capture API - captureGroup := router.Group("/api/v1/captures") - captureGroup.GET("", openAPI.ListCapture) + captureGroup := v1.Group("/captures") + captureGroup.GET("", api.ListCapture) } // ListChangefeed lists all changgefeeds in cdc cluster @@ -101,17 +122,17 @@ func (h *openAPI) ListChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() state := c.Query(apiOpVarChangefeedState) // get all changefeed status - statuses, err := statusProvider.GetAllChangeFeedStatuses(ctx) + statuses, err := h.statusProvider().GetAllChangeFeedStatuses(ctx) if err != nil { _ = c.Error(err) return } // get all changefeed infos - infos, err := statusProvider.GetAllChangeFeedInfo(ctx) + infos, err := h.statusProvider().GetAllChangeFeedInfo(ctx) if err != nil { // this call will return a parsedError generated by the error we passed in // so it is no need to check the parsedError @@ -166,7 +187,6 @@ func (h *openAPI) GetChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -175,19 +195,19 @@ func (h *openAPI) GetChangefeed(c *gin.Context) { return } - info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) + info, err := h.statusProvider().GetChangeFeedInfo(ctx, changefeedID) if err != nil { _ = c.Error(err) return } - status, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + status, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return } - processorInfos, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) + processorInfos, err := h.statusProvider().GetAllTaskStatuses(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -279,7 +299,7 @@ func (h *openAPI) PauseChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -288,7 +308,7 @@ func (h *openAPI) PauseChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -322,7 +342,7 @@ func (h *openAPI) ResumeChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -330,7 +350,7 @@ func (h *openAPI) ResumeChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -370,7 +390,7 @@ func (h *openAPI) UpdateChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -378,7 +398,7 @@ func (h *openAPI) UpdateChangefeed(c *gin.Context) { _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } - info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) + info, err := h.statusProvider().GetChangeFeedInfo(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -426,7 +446,7 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -434,7 +454,7 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -468,7 +488,7 @@ func (h *openAPI) RebalanceTable(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -477,7 +497,7 @@ func (h *openAPI) RebalanceTable(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -508,7 +528,7 @@ func (h *openAPI) MoveTable(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -516,7 +536,7 @@ func (h *openAPI) MoveTable(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -582,7 +602,6 @@ func (h *openAPI) GetProcessor(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() @@ -598,7 +617,7 @@ func (h *openAPI) GetProcessor(c *gin.Context) { return } - statuses, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) + statuses, err := h.statusProvider().GetAllTaskStatuses(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -609,7 +628,7 @@ func (h *openAPI) GetProcessor(c *gin.Context) { return } - positions, err := statusProvider.GetTaskPositions(ctx, changefeedID) + positions, err := h.statusProvider().GetTaskPositions(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -648,10 +667,9 @@ func (h *openAPI) ListProcessor(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() - infos, err := statusProvider.GetProcessors(ctx) + infos, err := h.statusProvider().GetProcessors(ctx) if err != nil { _ = c.Error(err) return @@ -678,10 +696,9 @@ func (h *openAPI) ListCapture(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() - captureInfos, err := statusProvider.GetCaptures(ctx) + captureInfos, err := h.statusProvider().GetCaptures(ctx) if err != nil { _ = c.Error(err) return diff --git a/cdc/api/open_test.go b/cdc/api/open_test.go new file mode 100644 index 00000000000..48a1fb53d4d --- /dev/null +++ b/cdc/api/open_test.go @@ -0,0 +1,450 @@ +// 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 api + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "net/http" + "net/http/httptest" + "testing" + + "github.com/gin-gonic/gin" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +const ( + changeFeedID = "test-changeFeed" + captureID = "test-capture" + nonExistChangefeedID = "non-exist-changefeed" +) + +type mockStatusProvider struct { + mock.Mock +} + +type testCase struct { + url string + method string +} + +func (p *mockStatusProvider) GetAllChangeFeedStatuses(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedStatus, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.ChangeFeedID]*model.ChangeFeedStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetChangeFeedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedStatus, error) { + args := p.Called(ctx, changefeedID) + log.Info("err", zap.Error(args.Error(1))) + return args.Get(0).(*model.ChangeFeedStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetAllChangeFeedInfo(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedInfo, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.ChangeFeedID]*model.ChangeFeedInfo), args.Error(1) +} + +func (p *mockStatusProvider) GetChangeFeedInfo(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedInfo, error) { + args := p.Called(ctx) + return args.Get(0).(*model.ChangeFeedInfo), args.Error(1) +} + +func (p *mockStatusProvider) GetAllTaskStatuses(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskStatus, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.CaptureID]*model.TaskStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetTaskPositions(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskPosition, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.CaptureID]*model.TaskPosition), args.Error(1) +} + +func (p *mockStatusProvider) GetProcessors(ctx context.Context) ([]*model.ProcInfoSnap, error) { + args := p.Called(ctx) + return args.Get(0).([]*model.ProcInfoSnap), args.Error(1) +} + +func (p *mockStatusProvider) GetCaptures(ctx context.Context) ([]*model.CaptureInfo, error) { + args := p.Called(ctx) + return args.Get(0).([]*model.CaptureInfo), args.Error(1) +} + +func newRouter(p *mockStatusProvider) *gin.Engine { + c := capture.NewCapture4Test(true) + router := gin.New() + RegisterOpenAPIRoutes(router, NewOpenAPI4Test(c, p)) + return router +} + +func newStatusProvider() *mockStatusProvider { + statusProvider := &mockStatusProvider{} + statusProvider.On("GetChangeFeedStatus", mock.Anything, changeFeedID). + Return(&model.ChangeFeedStatus{CheckpointTs: 1}, nil) + + statusProvider.On("GetChangeFeedStatus", mock.Anything, nonExistChangefeedID). + Return(new(model.ChangeFeedStatus), + cerror.ErrChangeFeedNotExists.GenWithStackByArgs(nonExistChangefeedID)) + + statusProvider.On("GetAllTaskStatuses", mock.Anything). + Return(map[model.CaptureID]*model.TaskStatus{captureID: {}}, nil) + + statusProvider.On("GetTaskPositions", mock.Anything). + Return(map[model.CaptureID]*model.TaskPosition{captureID: {Error: &model.RunningError{Message: "test"}}}, nil) + + statusProvider.On("GetAllChangeFeedStatuses", mock.Anything). + Return(map[model.ChangeFeedID]*model.ChangeFeedStatus{ + changeFeedID + "1": {CheckpointTs: 1}, + changeFeedID + "2": {CheckpointTs: 2}, + }, nil) + + statusProvider.On("GetAllChangeFeedInfo", mock.Anything). + Return(map[model.ChangeFeedID]*model.ChangeFeedInfo{ + changeFeedID + "1": {State: model.StateNormal}, + changeFeedID + "2": {State: model.StateStopped}, + }, nil) + + statusProvider.On("GetAllTaskStatuses", mock.Anything). + Return(map[model.CaptureID]*model.TaskStatus{captureID: {}}, nil) + + statusProvider.On("GetChangeFeedInfo", mock.Anything). + Return(&model.ChangeFeedInfo{State: model.StateNormal}, nil) + + statusProvider.On("GetProcessors", mock.Anything). + Return([]*model.ProcInfoSnap{{CfID: changeFeedID, CaptureID: captureID}}, nil) + + statusProvider.On("GetCaptures", mock.Anything). + Return([]*model.CaptureInfo{{ID: captureID}}, nil) + + return statusProvider +} + +func TestListChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + + // test list changefeed succeeded + api := testCase{url: "/api/v1/changefeeds", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.ChangefeedCommonInfo + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, 2, len(resp)) + + // test list changefeed with specific state + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds?state=%s", "stopped"), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + resp = []model.ChangefeedCommonInfo{} + err = json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, 1, len(resp)) + require.Equal(t, model.StateStopped, resp[0].FeedState) + require.Equal(t, uint64(0x2), resp[0].CheckpointTSO) +} + +func TestGetChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + + // test get changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp model.ChangefeedDetail + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, model.StateNormal, resp.FeedState) + + // test get changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestPauseChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test pause changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test pause changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestResumeChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test resume changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test resume changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestRemoveChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test remove changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "DELETE"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test remove changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), method: "DELETE"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestRebalanceTable(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test rebalance table succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test rebalance table failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestMoveTable(t *testing.T) { + t.Parallel() + + data := struct { + CaptureID string `json:"capture_id"` + TableID int64 `json:"table_id"` + }{captureID, 1} + b, err := json.Marshal(&data) + require.Nil(t, err) + body := bytes.NewReader(b) + + router := newRouter(newStatusProvider()) + // test move table succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test move table failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestResignOwner(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test resign owner succeeded + api := testCase{url: "/api/v1/owner/resign", method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) +} + +func TestGetProcessor(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test get processor succeeded + api := testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, captureID), method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + processorDetail := &model.ProcessorDetail{} + err := json.NewDecoder(w.Body).Decode(processorDetail) + require.Nil(t, err) + require.Equal(t, "test", processorDetail.Error.Message) + + // test get processor fail due to capture ID error + api = testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, "non-exist-capture"), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + httpError := &model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(httpError) + require.Nil(t, err) + require.Contains(t, httpError.Error, "capture not exists, key: non-exist-capture") +} + +func TestListProcessor(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test list processor succeeded + api := testCase{url: "/api/v1/processors", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.ProcessorCommonInfo + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, changeFeedID, resp[0].CfID) +} + +func TestListCapture(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test list processor succeeded + api := testCase{url: "/api/v1/captures", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.Capture + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, captureID, resp[0].ID) +} + +func TestServerStatus(t *testing.T) { + t.Parallel() + // capture is owner + ownerRouter := newRouter(newStatusProvider()) + api := testCase{url: "/api/v1/status", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + ownerRouter.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp model.ServerStatus + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, "capture-for-test", resp.ID) + require.True(t, resp.IsOwner) + + // capture is not owner + c := capture.NewCapture4Test(false) + r := gin.New() + RegisterOpenAPIRoutes(r, NewOpenAPI4Test(c, nil)) + api = testCase{url: "/api/v1/status", method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + r.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + resp = model.ServerStatus{} + err = json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.False(t, resp.IsOwner) +} + +func TestSetLogLevel(t *testing.T) { + t.Parallel() + + // test set log level succeeded + data := struct { + Level string `json:"log_level"` + }{"warn"} + router := newRouter(newStatusProvider()) + api := testCase{url: "/api/v1/log", method: "POST"} + w := httptest.NewRecorder() + b, err := json.Marshal(&data) + require.Nil(t, err) + body := bytes.NewReader(b) + req, _ := http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + + // test set log level failed + data = struct { + Level string `json:"log_level"` + }{"foo"} + api = testCase{url: "/api/v1/log", method: "POST"} + w = httptest.NewRecorder() + b, err = json.Marshal(&data) + require.Nil(t, err) + body = bytes.NewReader(b) + req, _ = http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + httpError := &model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(httpError) + require.Nil(t, err) + require.Contains(t, httpError.Error, "fail to change log level: foo") +} + +// TODO: finished these test cases after we decouple those APIs from etcdClient. +func TestCreateChangefeed(t *testing.T) {} +func TestUpdateChangefeed(t *testing.T) {} +func TestHealth(t *testing.T) {} diff --git a/cdc/api/owner.go b/cdc/api/owner.go index d6f0726cdad..a8b565c4a1a 100644 --- a/cdc/api/owner.go +++ b/cdc/api/owner.go @@ -82,11 +82,16 @@ type ownerAPI struct { // RegisterOwnerAPIRoutes registers routes for owner APIs. func RegisterOwnerAPIRoutes(router *gin.Engine, capture *capture.Capture) { ownerAPI := ownerAPI{capture: capture} - router.POST("/capture/owner/resign", gin.WrapF(ownerAPI.handleResignOwner)) - router.POST("/capture/owner/admin", gin.WrapF(ownerAPI.handleChangefeedAdmin)) - router.POST("/capture/owner/rebalance_trigger", gin.WrapF(ownerAPI.handleRebalanceTrigger)) - router.POST("/capture/owner/move_table", gin.WrapF(ownerAPI.handleMoveTable)) - router.POST("/capture/owner/changefeed/query", gin.WrapF(ownerAPI.handleChangefeedQuery)) + owner := router.Group("/capture/owner") + + owner.Use(errorHandleMiddleware()) + owner.Use(logMiddleware()) + + owner.POST("/resign", gin.WrapF(ownerAPI.handleResignOwner)) + owner.POST("/admin", gin.WrapF(ownerAPI.handleChangefeedAdmin)) + owner.POST("/rebalance_trigger", gin.WrapF(ownerAPI.handleRebalanceTrigger)) + owner.POST("/move_table", gin.WrapF(ownerAPI.handleMoveTable)) + owner.POST("/changefeed/query", gin.WrapF(ownerAPI.handleChangefeedQuery)) } func handleOwnerResp(w http.ResponseWriter, err error) { @@ -269,7 +274,7 @@ func (h *ownerAPI) handleChangefeedQuery(w http.ResponseWriter, req *http.Reques writeData(w, resp) } -func handleAdminLogLevel(w http.ResponseWriter, r *http.Request) { +func HandleAdminLogLevel(w http.ResponseWriter, r *http.Request) { var level string data, err := io.ReadAll(r.Body) r.Body.Close() diff --git a/cdc/api/owner_test.go b/cdc/api/owner_test.go index 7c19890d191..bd09d0ec7cc 100644 --- a/cdc/api/owner_test.go +++ b/cdc/api/owner_test.go @@ -14,124 +14,64 @@ package api import ( - "bytes" "fmt" "io" "net/http" "net/http/httptest" "net/url" + "testing" "github.com/gin-gonic/gin" - "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3/concurrency" ) -type httpStatusSuite struct{} - -var _ = check.Suite(&httpStatusSuite{}) - -func (s *httpStatusSuite) TestHTTPStatus(c *check.C) { - defer testleak.AfterTest(c)() - +func TestHTTPStatus(t *testing.T) { + t.Parallel() router := gin.New() - RegisterRoutes(router, capture.NewCapture4Test(), nil) - + RegisterOwnerAPIRoutes(router, nil) ts := httptest.NewServer(router) defer ts.Close() addr := ts.URL - testPprof(c, addr) - testReisgnOwner(c, addr) - testHandleChangefeedAdmin(c, addr) - testHandleRebalance(c, addr) - testHandleMoveTable(c, addr) - testHandleChangefeedQuery(c, addr) - testHandleFailpoint(c, addr) + testReisgnOwner(t, addr) + testHandleChangefeedAdmin(t, addr) + testHandleRebalance(t, addr) + testHandleMoveTable(t, addr) + testHandleChangefeedQuery(t, addr) } -func testPprof(c *check.C, addr string) { - testValidPprof := func(uri string) { - resp, err := http.Get(uri) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.Equals, 200) - _, err = io.ReadAll(resp.Body) - c.Assert(err, check.IsNil) - } - testValidPprof(fmt.Sprintf("%s/debug/pprof", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/cmdline", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/mutex", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/heap?debug=1", addr)) -} - -func testReisgnOwner(c *check.C, addr string) { +func testReisgnOwner(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/resign", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleChangefeedAdmin(c *check.C, addr string) { +func testHandleChangefeedAdmin(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/admin", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleRebalance(c *check.C, addr string) { +func testHandleRebalance(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/rebalance_trigger", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleMoveTable(c *check.C, addr string) { +func testHandleMoveTable(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/move_table", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleChangefeedQuery(c *check.C, addr string) { +func testHandleChangefeedQuery(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/changefeed/query", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testRequestNonOwnerFailed(c *check.C, uri string) { +func testRequestNonOwnerFailed(t *testing.T, uri string) { resp, err := http.PostForm(uri, url.Values{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) data, err := io.ReadAll(resp.Body) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.Equals, http.StatusBadRequest) - c.Assert(string(data), check.Equals, concurrency.ErrElectionNotLeader.Error()) -} - -func testHandleFailpoint(c *check.C, addr string) { - fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" - uri := fmt.Sprintf("%s/debug/fail/%s", addr, fp) - body := bytes.NewReader([]byte("return(true)")) - req, err := http.NewRequest("PUT", uri, body) - c.Assert(err, check.IsNil) - - resp, err := http.DefaultClient.Do(req) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer resp.Body.Close() - c.Assert(resp.StatusCode, check.GreaterEqual, 200) - c.Assert(resp.StatusCode, check.Less, 300) - - failpointHit := false - failpoint.Inject("TestHandleFailpoint", func() { - failpointHit = true - }) - c.Assert(failpointHit, check.IsTrue) - - req, err = http.NewRequest("DELETE", uri, body) - c.Assert(err, check.IsNil) - resp, err = http.DefaultClient.Do(req) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.GreaterEqual, 200) - c.Assert(resp.StatusCode, check.Less, 300) - - failpointHit = false - failpoint.Inject("TestHandleFailpoint", func() { - failpointHit = true - }) - c.Assert(failpointHit, check.IsFalse) + require.Equal(t, http.StatusBadRequest, resp.StatusCode) + require.Equal(t, concurrency.ErrElectionNotLeader.Error(), string(data)) } diff --git a/cdc/api/router.go b/cdc/api/router.go deleted file mode 100644 index c12e5d356bf..00000000000 --- a/cdc/api/router.go +++ /dev/null @@ -1,73 +0,0 @@ -// 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 api - -import ( - "net/http" - "net/http/pprof" - - "github.com/gin-gonic/gin" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/pkg/util" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promhttp" - swaggerFiles "github.com/swaggo/files" - ginSwagger "github.com/swaggo/gin-swagger" - - // use for OpenAPI online docs - _ "github.com/pingcap/tiflow/docs/swagger" -) - -// RegisterRoutes create a router for OpenAPI -func RegisterRoutes( - router *gin.Engine, - capture *capture.Capture, - registry prometheus.Gatherer, -) { - // online docs - router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) - - // Open API - RegisterOpoenAPIRoutes(router, capture) - - // Owner API - RegisterOwnerAPIRoutes(router, capture) - - // Status API - RegisterStatusAPIRoutes(router, capture) - - // Log API - router.POST("/admin/log", gin.WrapF(handleAdminLogLevel)) - - // pprof debug API - pprofGroup := router.Group("/debug/pprof/") - pprofGroup.GET("", gin.WrapF(pprof.Index)) - pprofGroup.GET("/:any", gin.WrapF(pprof.Index)) - pprofGroup.GET("/cmdline", gin.WrapF(pprof.Cmdline)) - pprofGroup.GET("/profile", gin.WrapF(pprof.Profile)) - pprofGroup.GET("/symbol", gin.WrapF(pprof.Symbol)) - pprofGroup.GET("/trace", gin.WrapF(pprof.Trace)) - pprofGroup.GET("/threadcreate", gin.WrapF(pprof.Handler("threadcreate").ServeHTTP)) - - // Failpoint API - if util.FailpointBuild { - // `http.StripPrefix` is needed because `failpoint.HttpHandler` assumes that it handles the prefix `/`. - router.Any("/debug/fail/*any", gin.WrapH(http.StripPrefix("/debug/fail", &failpoint.HttpHandler{}))) - } - - // Promtheus metrics API - prometheus.DefaultGatherer = registry - router.Any("/metrics", gin.WrapH(promhttp.Handler())) -} diff --git a/cdc/api/router_test.go b/cdc/api/router_test.go deleted file mode 100644 index 76abe25f46b..00000000000 --- a/cdc/api/router_test.go +++ /dev/null @@ -1,60 +0,0 @@ -// 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 api - -import ( - "fmt" - "net/http" - "net/http/httptest" - "testing" - - "github.com/gin-gonic/gin" - "github.com/stretchr/testify/require" -) - -func TestPProfPath(t *testing.T) { - t.Parallel() - - router := gin.New() - RegisterRoutes(router, nil, nil) - - apis := []*testCase{ - {"/debug/pprof/", http.MethodGet}, - {"/debug/pprof/cmdline", http.MethodGet}, - {"/debug/pprof/symbol", http.MethodGet}, - // these two apis make will make ut slow - //{"/debug/pprof/profile", http.MethodGet}, - //{"/debug/pprof/trace", http.MethodGet}, - {"/debug/pprof/threadcreate", http.MethodGet}, - {"/debug/pprof/allocs", http.MethodGet}, - {"/debug/pprof/block", http.MethodGet}, - {"/debug/pprof/goroutine?debug=1", http.MethodGet}, - {"/debug/pprof/mutex?debug=1", http.MethodGet}, - } - for _, api := range apis { - w := httptest.NewRecorder() - req, _ := http.NewRequest(api.method, api.url, nil) - router.ServeHTTP(w, req) - require.Equal(t, 200, w.Code, api.String()) - } -} - -type testCase struct { - url string - method string -} - -func (a *testCase) String() string { - return fmt.Sprintf("%s:%s", a.method, a.url) -} diff --git a/cdc/api/util_test.go b/cdc/api/util_test.go index 75a2adbffe6..f9b8f36d5d5 100644 --- a/cdc/api/util_test.go +++ b/cdc/api/util_test.go @@ -22,6 +22,7 @@ import ( ) func TestIsHTTPBadRequestError(t *testing.T) { + t.Parallel() err := cerror.ErrAPIInvalidParam.GenWithStack("aa") require.Equal(t, true, IsHTTPBadRequestError(err)) err = cerror.ErrAPIInvalidParam.Wrap(errors.New("aa")) diff --git a/cdc/api/validator_test.go b/cdc/api/validator_test.go index f7d0dea59b1..967e7b11cdb 100644 --- a/cdc/api/validator_test.go +++ b/cdc/api/validator_test.go @@ -23,6 +23,7 @@ import ( ) func TestVerifyUpdateChangefeedConfig(t *testing.T) { + t.Parallel() ctx := context.Background() ctx, cancel := context.WithCancel(ctx) defer cancel() diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 97b17835f2d..b2f8fae43aa 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -107,10 +107,14 @@ func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *etcd.C } } -func NewCapture4Test() *Capture { - return &Capture{ +func NewCapture4Test(isOwner bool) *Capture { + res := &Capture{ info: &model.CaptureInfo{ID: "capture-for-test", AdvertiseAddr: "127.0.0.1", Version: "test"}, } + if isOwner { + res.owner = &owner.Owner{} + } + return res } func (c *Capture) reset(ctx context.Context) error { diff --git a/cdc/cdc_test.go b/cdc/cdc_test.go deleted file mode 100644 index 9dd368511d5..00000000000 --- a/cdc/cdc_test.go +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "testing" - - "github.com/pingcap/check" -) - -func TestSuite(t *testing.T) { check.TestingT(t) } diff --git a/cdc/http.go b/cdc/http.go index df1faa2411d..acdda995acf 100644 --- a/cdc/http.go +++ b/cdc/http.go @@ -14,90 +14,61 @@ package cdc import ( - "context" "net/http" - "strings" - "time" + "net/http/pprof" "github.com/gin-gonic/gin" - "github.com/pingcap/log" + "github.com/pingcap/failpoint" "github.com/pingcap/tiflow/cdc/api" - "github.com/pingcap/tiflow/cdc/model" - "go.uber.org/zap" -) + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/pkg/util" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + swaggerFiles "github.com/swaggo/files" + ginSwagger "github.com/swaggo/gin-swagger" -// timeoutMiddleware wraps the request context with a timeout -func timeoutMiddleware(timeout time.Duration) gin.HandlerFunc { - return func(c *gin.Context) { - // wrap the request context with a timeout - ctx, cancel := context.WithTimeout(c.Request.Context(), timeout) + // use for OpenAPI online docs + _ "github.com/pingcap/tiflow/docs/swagger" +) - defer func() { - // check if context timeout was reached - if ctx.Err() == context.DeadlineExceeded { +// RegisterRoutes create a router for OpenAPI +func RegisterRoutes( + router *gin.Engine, + capture *capture.Capture, + registry prometheus.Gatherer, +) { + // online docs + router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) - // write response and abort the request - c.Writer.WriteHeader(http.StatusGatewayTimeout) - c.Abort() - } + // Open API + api.RegisterOpenAPIRoutes(router, api.NewOpenAPI(capture)) - // cancel to clear resources after finished - cancel() - }() + // Owner API + api.RegisterOwnerAPIRoutes(router, capture) - // replace request with context wrapped request - c.Request = c.Request.WithContext(ctx) - c.Next() - } -} + // Status API + api.RegisterStatusAPIRoutes(router, capture) -func logMiddleware() gin.HandlerFunc { - return func(c *gin.Context) { - start := time.Now() - path := c.Request.URL.Path - query := c.Request.URL.RawQuery - c.Next() + // Log API + router.POST("/admin/log", gin.WrapF(api.HandleAdminLogLevel)) - cost := time.Since(start) + // pprof debug API + pprofGroup := router.Group("/debug/pprof/") + pprofGroup.GET("", gin.WrapF(pprof.Index)) + pprofGroup.GET("/:any", gin.WrapF(pprof.Index)) + pprofGroup.GET("/cmdline", gin.WrapF(pprof.Cmdline)) + pprofGroup.GET("/profile", gin.WrapF(pprof.Profile)) + pprofGroup.GET("/symbol", gin.WrapF(pprof.Symbol)) + pprofGroup.GET("/trace", gin.WrapF(pprof.Trace)) + pprofGroup.GET("/threadcreate", gin.WrapF(pprof.Handler("threadcreate").ServeHTTP)) - err := c.Errors.Last() - var stdErr error - if err != nil { - stdErr = err.Err - } - // Do not log metrics related requests when there is no error - if strings.Contains(path, "/metrics") && err == nil { - return - } - log.Info(path, - zap.Int("status", c.Writer.Status()), - zap.String("method", c.Request.Method), - zap.String("path", path), - zap.String("query", query), - zap.String("ip", c.ClientIP()), - zap.String("user-agent", c.Request.UserAgent()), - zap.Error(stdErr), - zap.Duration("duration", cost), - ) + // Failpoint API + if util.FailpointBuild { + // `http.StripPrefix` is needed because `failpoint.HttpHandler` assumes that it handles the prefix `/`. + router.Any("/debug/fail/*any", gin.WrapH(http.StripPrefix("/debug/fail", &failpoint.HttpHandler{}))) } -} -func errorHandleMiddleware() gin.HandlerFunc { - return func(c *gin.Context) { - c.Next() - // because we will return immediately after an error occurs in http_handler - // there wil be only one error in c.Errors - lastError := c.Errors.Last() - if lastError != nil { - err := lastError.Err - // put the error into response - if api.IsHTTPBadRequestError(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } else { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) - } - c.Abort() - return - } - } + // Promtheus metrics API + prometheus.DefaultGatherer = registry + router.Any("/metrics", gin.WrapH(promhttp.Handler())) } diff --git a/cdc/http_test.go b/cdc/http_test.go index a59719e468f..39398a82639 100644 --- a/cdc/http_test.go +++ b/cdc/http_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// 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. @@ -14,182 +14,80 @@ package cdc import ( - "context" - "crypto/tls" - "encoding/json" + "bytes" "fmt" "net/http" - "strings" - "sync" - "time" + "net/http/httptest" + "testing" - "github.com/pingcap/check" - "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/failpoint" + + "github.com/gin-gonic/gin" "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/retry" - security2 "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util/testleak" - "github.com/tikv/pd/pkg/tempurl" + "github.com/stretchr/testify/require" ) -type httpStatusSuite struct{} - -var _ = check.Suite(&httpStatusSuite{}) - -const retryTime = 20 - -func (s *httpStatusSuite) TestServerTLSWithoutCommonName(c *check.C) { - defer testleak.AfterTest(c) - addr := tempurl.Alloc()[len("http://"):] - // Do not specify common name - security, err := security2.NewCredential4Test("") - c.Assert(err, check.IsNil) - conf := config.GetDefaultServerConfig() - conf.Addr = addr - conf.AdvertiseAddr = addr - conf.Security = &security - config.StoreGlobalServerConfig(conf) - - server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test() - c.Assert(err, check.IsNil) - err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) - c.Assert(err, check.IsNil) - defer func() { - c.Assert(server.statusServer.Close(), check.IsNil) - }() - - statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := server.tcpServer.Run(ctx) - c.Check(err, check.ErrorMatches, ".*ErrTCPServerClosed.*") - }() - - // test cli sends request without a cert will success - err = retry.Do(ctx, func() error { - tr := &http.Transport{ - TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, - } - cli := &http.Client{Transport: tr} - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) - - // test cli sends request with a cert will success - err = retry.Do(ctx, func() error { - tlsConfig, err := security.ToTLSConfigWithVerify() - if err != nil { - c.Assert(err, check.IsNil) - } - cli := httputil.NewClient(tlsConfig) - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) - - cancel() - wg.Wait() +type testCase struct { + url string + method string } -func (s *httpStatusSuite) TestServerTLSWithCommonName(c *check.C) { - defer testleak.AfterTest(c) - addr := tempurl.Alloc()[len("http://"):] - // specify a common name - security, err := security2.NewCredential4Test("test") - c.Assert(err, check.IsNil) - conf := config.GetDefaultServerConfig() - conf.Addr = addr - conf.AdvertiseAddr = addr - conf.Security = &security - config.StoreGlobalServerConfig(conf) - - server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test() - c.Assert(err, check.IsNil) - err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) - c.Assert(err, check.IsNil) - defer func() { - c.Assert(server.statusServer.Close(), check.IsNil) - }() - - statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := server.tcpServer.Run(ctx) - c.Check(err, check.ErrorMatches, ".*ErrTCPServerClosed.*") - }() +func (a *testCase) String() string { + return fmt.Sprintf("%s:%s", a.method, a.url) +} - // test cli sends request without a cert will fail - err = retry.Do(ctx, func() error { - tr := &http.Transport{ - TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, - } - cli := &http.Client{Transport: tr} - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(strings.Contains(err.Error(), "remote error: tls: bad certificate"), check.IsTrue) +func TestPProfPath(t *testing.T) { + router := gin.New() + RegisterRoutes(router, capture.NewCapture4Test(false), nil) + + apis := []*testCase{ + {"/debug/pprof/", http.MethodGet}, + {"/debug/pprof/cmdline", http.MethodGet}, + {"/debug/pprof/symbol", http.MethodGet}, + // these two apis make will make ut slow + //{"/debug/pprof/profile", http.MethodGet}, + //{"/debug/pprof/trace", http.MethodGet}, + {"/debug/pprof/threadcreate", http.MethodGet}, + {"/debug/pprof/allocs", http.MethodGet}, + {"/debug/pprof/block", http.MethodGet}, + {"/debug/pprof/goroutine?debug=1", http.MethodGet}, + {"/debug/pprof/mutex?debug=1", http.MethodGet}, + } + for _, api := range apis { + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code, api.String()) + } +} - // test cli sends request with a cert will success - err = retry.Do(ctx, func() error { - tlsConfig, err := security.ToTLSConfigWithVerify() - if err != nil { - c.Assert(err, check.IsNil) - } - cli := httputil.NewClient(tlsConfig) - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) +func TestHandleFailpoint(t *testing.T) { + router := gin.New() + RegisterRoutes(router, capture.NewCapture4Test(false), nil) + fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" + uri := fmt.Sprintf("/debug/fail/%s", fp) + body := bytes.NewReader([]byte("return(true)")) + req, err := http.NewRequest("PUT", uri, body) + require.Nil(t, err) + w := httptest.NewRecorder() + router.ServeHTTP(w, req) + require.True(t, w.Code >= 200 && w.Code <= 300) + + failpointHit := false + failpoint.Inject("TestHandleFailpoint", func() { + failpointHit = true + }) + require.True(t, failpointHit) + + req, err = http.NewRequest("DELETE", uri, body) + require.Nil(t, err) + w = httptest.NewRecorder() + router.ServeHTTP(w, req) + require.True(t, w.Code >= 200 && w.Code <= 300) + + failpointHit = false + failpoint.Inject("TestHandleFailpoint", func() { + failpointHit = true + }) + require.False(t, failpointHit) } diff --git a/cdc/server.go b/cdc/server.go index 86d8529fbfb..cf3ab6f651c 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" tidbkv "github.com/pingcap/tidb/kv" - "github.com/pingcap/tiflow/cdc/api" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/sorter/unified" @@ -218,16 +217,9 @@ func (s *Server) startStatusHTTP(lis net.Listener) error { // discard gin log output gin.DefaultWriter = io.Discard - router := gin.New() - - router.Use(logMiddleware()) - // request will timeout after 10 second - router.Use(timeoutMiddleware(time.Second * 10)) - router.Use(errorHandleMiddleware()) - // Register APIs. - api.RegisterRoutes(router, s.capture, registry) + RegisterRoutes(router, s.capture, registry) // No need to configure TLS because it is already handled by `s.tcpServer`. s.statusServer = &http.Server{Handler: router} diff --git a/cdc/server_test.go b/cdc/server_test.go index 567eb4c71f7..b6eccf3c74b 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -15,27 +15,36 @@ package cdc import ( "context" + "crypto/tls" + "encoding/json" + "fmt" + "net/http" "net/url" "os" "os/user" "path/filepath" "runtime" + "sync" "testing" "time" - "github.com/pingcap/check" + "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/retry" + security2 "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/tempurl" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) -type serverSuite struct { +type testServer struct { server *Server e *embed.Etcd clientURL *url.URL @@ -44,19 +53,20 @@ type serverSuite struct { errg *errgroup.Group } -func (s *serverSuite) SetUpTest(c *check.C) { +func newServer(t *testing.T) *testServer { var err error - dir := c.MkDir() + dir := t.TempDir() + s := &testServer{} s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdEndpoints := []string{ "http://" + s.clientURL.Host, "http://invalid-pd-host:2379", } server, err := NewServer(pdEndpoints) - c.Assert(err, check.IsNil) - c.Assert(server, check.NotNil) + require.Nil(t, err) + require.NotNil(t, server) s.server = server s.ctx, s.cancel = context.WithCancel(context.Background()) @@ -65,78 +75,75 @@ func (s *serverSuite) SetUpTest(c *check.C) { Context: s.ctx, DialTimeout: 5 * time.Second, }) - c.Assert(err, check.IsNil) + require.Nil(t, err) etcdClient := etcd.NewCDCEtcdClient(s.ctx, client) s.server.etcdClient = &etcdClient - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) + s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { t.Log(e) }) + return s } -func (s *serverSuite) TearDownTest(c *check.C) { +func (s *testServer) close(t *testing.T) { s.server.Close() s.e.Close() s.cancel() err := s.errg.Wait() if err != nil { - c.Errorf("Error group error: %s", err) + t.Errorf("Error group error: %s", err) } } -var _ = check.Suite(&serverSuite{}) - -func (s *serverSuite) TestEtcdHealthChecker(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestServerBasic(t *testing.T) { + t.Parallel() + s := newServer(t) + defer s.close(t) + testEtcdHealthChecker(t, s) + testSetUpDataDir(t, s) +} +func testEtcdHealthChecker(t *testing.T, s *testServer) { s.errg.Go(func() error { err := s.server.etcdHealthChecker(s.ctx) - c.Assert(err, check.Equals, context.Canceled) + require.Equal(t, context.Canceled, err) return nil }) // longer than one check tick 3s time.Sleep(time.Second * 4) - s.cancel() } -func (s *serverSuite) TestSetUpDataDir(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func testSetUpDataDir(t *testing.T, s *testServer) { conf := config.GetGlobalServerConfig() // DataDir is not set, and no changefeed exist, use the default conf.DataDir = "" err := s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(conf.DataDir, check.Equals, defaultDataDir) - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(defaultDataDir, config.DefaultSortDir)) + require.Nil(t, err) + require.Equal(t, defaultDataDir, conf.DataDir) + require.Equal(t, filepath.Join(defaultDataDir, config.DefaultSortDir), conf.Sorter.SortDir) // DataDir is not set, but has existed changefeed, use the one with the largest available space conf.DataDir = "" - dir := c.MkDir() + dir := t.TempDir() err = s.server.etcdClient.SaveChangeFeedInfo(s.ctx, &model.ChangeFeedInfo{SortDir: dir}, "a") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = s.server.etcdClient.SaveChangeFeedInfo(s.ctx, &model.ChangeFeedInfo{}, "b") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) - c.Assert(conf.DataDir, check.Equals, dir) - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(dir, config.DefaultSortDir)) + require.Equal(t, dir, conf.DataDir) + require.Equal(t, filepath.Join(dir, config.DefaultSortDir), conf.Sorter.SortDir) - conf.DataDir = c.MkDir() + conf.DataDir = t.TempDir() // DataDir has been set, just use it err = s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(conf.DataDir, check.Not(check.Equals), "") - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(conf.DataDir, config.DefaultSortDir)) - - s.cancel() + require.Nil(t, err) + require.NotEqual(t, "", conf.DataDir) + require.Equal(t, filepath.Join(conf.DataDir, config.DefaultSortDir), conf.Sorter.SortDir) } func TestCheckDir(t *testing.T) { - t.Parallel() me, err := user.Current() require.Nil(t, err) if me.Name == "root" || runtime.GOOS == "windows" { @@ -166,3 +173,154 @@ func TestCheckDir(t *testing.T) { _, err = checkDir(file) require.Error(t, err) } + +const retryTime = 20 + +func TestServerTLSWithoutCommonName(t *testing.T) { + addr := tempurl.Alloc()[len("http://"):] + // Do not specify common name + security, err := security2.NewCredential4Test("") + require.Nil(t, err) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test(false) + require.Nil(t, err) + err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) + require.Nil(t, err) + defer func() { + require.Nil(t, server.statusServer.Close()) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := server.tcpServer.Run(ctx) + require.Contains(t, err.Error(), "ErrTCPServerClosed") + }() + + // test cli sends request without a cert will success + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + require.Nil(t, err) + + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) + + cancel() + wg.Wait() +} + +func TestServerTLSWithCommonName(t *testing.T) { + addr := tempurl.Alloc()[len("http://"):] + // specify a common name + security, err := security2.NewCredential4Test("test") + require.Nil(t, err) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test(false) + require.Nil(t, err) + err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) + require.Nil(t, err) + defer func() { + require.Nil(t, server.statusServer.Close()) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := server.tcpServer.Run(ctx) + require.Contains(t, err.Error(), "ErrTCPServerClosed") + }() + + // test cli sends request without a cert will fail + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Contains(t, err.Error(), "remote error: tls: bad certificate") + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + require.Nil(t, err) + + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) +} From 7a227b421dbfcdafee02148e787138798edadf31 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Mon, 24 Jan 2022 23:42:13 +0800 Subject: [PATCH 16/72] tests(ticdc): disable hibernate-regions-compatible to fix tiflash tests (#4460) ref pingcap/tiflow#4461 --- tests/integration_tests/_utils/start_tidb_cluster_impl | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 82d36de3a5a..6d121376b50 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -169,6 +169,10 @@ max-open-files = 4096 [raftstore] # true (default value) for high reliability, this can prevent data loss when power failure. sync-log = false +[cdc] +# tiflash proxy does not support this feature yet. +# TODO: enable hibernate-regions-compatible +hibernate-regions-compatible = false EOF # tidb server config file From 9d0ec397e7d03a4953ae7d82ef515e7b712b9c3b Mon Sep 17 00:00:00 2001 From: Ehco Date: Tue, 25 Jan 2022 10:00:12 +0800 Subject: [PATCH 17/72] grafana(dm): fix grafana expr (#4405) close pingcap/tiflow#4404 --- dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json | 8 ++++---- dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json b/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json index a7c0d6006cf..d05a6730bc0 100644 --- a/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json +++ b/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json @@ -3059,7 +3059,7 @@ "steppedLine": false, "targets": [ { - "expr": "dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"master\"} - ON( task,source_id) dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"syncer\"}", + "expr": "dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"master\"} - ON(task,source_id) dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"syncer\"}", "format": "time_series", "hide": false, "interval": "$interval", @@ -3716,12 +3716,12 @@ "steppedLine": false, "targets": [ { - "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"} - ON(instance, task) dm_syncer_binlog_file{instance=~\"$instance\", task=~\"$task\", node=\"syncer\"}", + "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"} - ON(instance) group_right dm_syncer_binlog_file{instance=~\"$instance\", task=~\"$task\", node=\"syncer\"}", "format": "time_series", "hide": false, "interval": "$interval", "intervalFactor": 2, - "legendFormat": "{{task}} - {{instance}}", + "legendFormat": "{{instance}} - {{task}}", "refId": "B" } ], @@ -5088,7 +5088,7 @@ "steppedLine": false, "targets": [ { - "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"master\"} - ON(instance, job) dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"}", + "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"master\"} - ON(instance) dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"}", "format": "time_series", "hide": false, "interval": "$interval", diff --git a/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json b/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json index 7b8bf19e037..e29a8005e40 100644 --- a/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json +++ b/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json @@ -450,7 +450,7 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.90, sum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 90", @@ -458,7 +458,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.95,\nsum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.95,\nsum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 95", @@ -466,7 +466,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.99,\nsum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.99,\nsum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 99", From 43a1d78f94db503ef991919ea9a6f4609c07bf11 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 25 Jan 2022 10:56:11 +0800 Subject: [PATCH 18/72] consumer(ticdc): update the logic of consumer. (#4129) close pingcap/tiflow#4128 --- cmd/kafka-consumer/main.go | 104 +++++++++++++++++++++---------------- 1 file changed, 60 insertions(+), 44 deletions(-) diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index ab71f256bb9..659e106323f 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -80,16 +80,16 @@ func init() { File: logPath, }) if err != nil { - log.Fatal("init logger failed", zap.Error(err)) + log.Panic("init logger failed", zap.Error(err)) } upstreamURI, err := url.Parse(upstreamURIStr) if err != nil { - log.Fatal("invalid upstream-uri", zap.Error(err)) + log.Panic("invalid upstream-uri", zap.Error(err)) } scheme := strings.ToLower(upstreamURI.Scheme) if scheme != "kafka" { - log.Fatal("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`", + log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`", zap.String("upstreamURI", upstreamURIStr)) } s := upstreamURI.Query().Get("version") @@ -107,20 +107,20 @@ func init() { config, err := newSaramaConfig() if err != nil { - log.Fatal("Error creating sarama config", zap.Error(err)) + log.Panic("Error creating sarama config", zap.Error(err)) } s = upstreamURI.Query().Get("partition-num") if s == "" { partition, err := getPartitionNum(kafkaAddrs, kafkaTopic, config) if err != nil { - log.Fatal("can not get partition number", zap.String("topic", kafkaTopic), zap.Error(err)) + log.Panic("can not get partition number", zap.String("topic", kafkaTopic), zap.Error(err)) } kafkaPartitionNum = partition } else { c, err := strconv.ParseInt(s, 10, 32) if err != nil { - log.Fatal("invalid partition-num of upstream-uri") + log.Panic("invalid partition-num of upstream-uri") } kafkaPartitionNum = int32(c) } @@ -129,7 +129,7 @@ func init() { if s != "" { c, err := strconv.Atoi(s) if err != nil { - log.Fatal("invalid max-message-bytes of upstream-uri") + log.Panic("invalid max-message-bytes of upstream-uri") } log.Info("Setting max-message-bytes", zap.Int("max-message-bytes", c)) kafkaMaxMessageBytes = c @@ -139,7 +139,7 @@ func init() { if s != "" { c, err := strconv.Atoi(s) if err != nil { - log.Fatal("invalid max-batch-size of upstream-uri") + log.Panic("invalid max-batch-size of upstream-uri") } log.Info("Setting max-batch-size", zap.Int("max-batch-size", c)) kafkaMaxBatchSize = c @@ -228,24 +228,24 @@ func main() { */ config, err := newSaramaConfig() if err != nil { - log.Fatal("Error creating sarama config", zap.Error(err)) + log.Panic("Error creating sarama config", zap.Error(err)) } err = waitTopicCreated(kafkaAddrs, kafkaTopic, config) if err != nil { - log.Fatal("wait topic created failed", zap.Error(err)) + log.Panic("wait topic created failed", zap.Error(err)) } /** * Setup a new Sarama consumer group */ consumer, err := NewConsumer(context.TODO()) if err != nil { - log.Fatal("Error creating consumer", zap.Error(err)) + log.Panic("Error creating consumer", zap.Error(err)) } ctx, cancel := context.WithCancel(context.Background()) client, err := sarama.NewConsumerGroup(kafkaAddrs, kafkaGroupID, config) if err != nil { - log.Fatal("Error creating consumer group client", zap.Error(err)) + log.Panic("Error creating consumer group client", zap.Error(err)) } wg := &sync.WaitGroup{} @@ -257,7 +257,7 @@ func main() { // server-side rebalance happens, the consumer session will need to be // recreated to get the new claims if err := client.Consume(ctx, strings.Split(kafkaTopic, ","), consumer); err != nil { - log.Fatal("Error from consumer: %v", zap.Error(err)) + log.Panic("Error from consumer: %v", zap.Error(err)) } // check if context was cancelled, signaling that the consumer should stop if ctx.Err() != nil { @@ -269,7 +269,7 @@ func main() { go func() { if err := consumer.Run(ctx); err != nil { - log.Fatal("Error running consumer: %v", zap.Error(err)) + log.Panic("Error running consumer: %v", zap.Error(err)) } }() @@ -287,7 +287,7 @@ func main() { cancel() wg.Wait() if err = client.Close(); err != nil { - log.Fatal("Error closing client", zap.Error(err)) + log.Panic("Error closing client", zap.Error(err)) } } @@ -384,9 +384,9 @@ func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim saram if sink == nil { panic("sink should initialized") } -ClaimMessages: + for message := range claim.Messages() { - log.Info("Message claimed", zap.Int32("partition", message.Partition), zap.ByteString("key", message.Key), zap.ByteString("value", message.Value)) + log.Debug("Message claimed", zap.Int32("partition", message.Partition), zap.ByteString("key", message.Key), zap.ByteString("value", message.Value)) batchDecoder, err := codec.NewJSONEventBatchDecoder(message.Key, message.Value) if err != nil { return errors.Trace(err) @@ -396,7 +396,7 @@ ClaimMessages: for { tp, hasNext, err := batchDecoder.HasNext() if err != nil { - log.Fatal("decode message key failed", zap.Error(err)) + log.Panic("decode message key failed", zap.Error(err)) } if !hasNext { break @@ -405,7 +405,7 @@ ClaimMessages: counter++ // If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning. if len(message.Key)+len(message.Value) > kafkaMaxMessageBytes && counter > 1 { - log.Fatal("kafka max-messages-bytes exceeded", zap.Int("max-message-bytes", kafkaMaxMessageBytes), + log.Panic("kafka max-messages-bytes exceeded", zap.Int("max-message-bytes", kafkaMaxMessageBytes), zap.Int("receviedBytes", len(message.Key)+len(message.Value))) } @@ -413,21 +413,22 @@ ClaimMessages: case model.MqMessageTypeDDL: ddl, err := batchDecoder.NextDDLEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } c.appendDDL(ddl) case model.MqMessageTypeRow: row, err := batchDecoder.NextRowChangedEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) if row.CommitTs <= globalResolvedTs || row.CommitTs <= sink.resolvedTs { - log.Debug("filter fallback row", zap.ByteString("row", message.Key), + log.Debug("RowChangedEvent fallback row, ignore it", + zap.Uint64("commitTs", row.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs), zap.Uint64("sinkResolvedTs", sink.resolvedTs), - zap.Int32("partition", partition)) - break ClaimMessages + zap.Int32("partition", partition), + zap.ByteString("row", message.Key)) } // FIXME: hack to set start-ts in row changed event, as start-ts // is not contained in TiCDC open protocol @@ -440,9 +441,8 @@ ClaimMessages: c.fakeTableIDGenerator.generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) err = sink.EmitRowChangedEvents(ctx, row) if err != nil { - log.Fatal("emit row changed event failed", zap.Error(err)) + log.Panic("emit row changed event failed", zap.Error(err)) } - log.Info("Emit RowChangedEvent", zap.Any("row", row)) lastCommitTs, ok := sink.tablesMap.Load(row.Table.TableID) if !ok || lastCommitTs.(uint64) < row.CommitTs { sink.tablesMap.Store(row.Table.TableID, row.CommitTs) @@ -450,21 +450,29 @@ ClaimMessages: case model.MqMessageTypeResolved: ts, err := batchDecoder.NextResolvedEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } resolvedTs := atomic.LoadUint64(&sink.resolvedTs) - if resolvedTs < ts { + // `resolvedTs` should be monotonically increasing, it's allowed to receive redandunt one. + if ts < resolvedTs { + log.Panic("partition resolved ts fallback", + zap.Uint64("ts", ts), + zap.Uint64("resolvedTs", resolvedTs), + zap.Int32("partition", partition)) + } else if ts > resolvedTs { log.Debug("update sink resolved ts", zap.Uint64("ts", ts), zap.Int32("partition", partition)) atomic.StoreUint64(&sink.resolvedTs, ts) + } else { + log.Info("redundant sink resolved ts", zap.Uint64("ts", ts), zap.Int32("partition", partition)) } } session.MarkMessage(message, "") } if counter > kafkaMaxBatchSize { - log.Fatal("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", kafkaMaxBatchSize), + log.Panic("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", kafkaMaxBatchSize), zap.Int("actual-batch-size", counter)) } } @@ -479,8 +487,11 @@ func (c *Consumer) appendDDL(ddl *model.DDLEvent) { return } globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) - if ddl.CommitTs <= globalResolvedTs { - log.Error("unexpected ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) + if ddl.CommitTs < globalResolvedTs { + log.Panic("unexpected ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) + } + if ddl.CommitTs == globalResolvedTs { + log.Warn("receive redundant ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) return } c.ddlList = append(c.ddlList, ddl) @@ -521,14 +532,15 @@ func (c *Consumer) forEachSink(fn func(sink *partitionSink) error) error { // Run runs the Consumer func (c *Consumer) Run(ctx context.Context) error { var lastGlobalResolvedTs uint64 + ticker := time.NewTicker(100 * time.Millisecond) + defer ticker.Stop() for { select { case <-ctx.Done(): return ctx.Err() - default: + case <-ticker.C: } - time.Sleep(100 * time.Millisecond) - // handle ddl + // initialize the `globalResolvedTs` as min of all partition's `ResolvedTs` globalResolvedTs := uint64(math.MaxUint64) err := c.forEachSink(func(sink *partitionSink) error { resolvedTs := atomic.LoadUint64(&sink.resolvedTs) @@ -540,6 +552,7 @@ func (c *Consumer) Run(ctx context.Context) error { if err != nil { return errors.Trace(err) } + // handle ddl todoDDL := c.getFrontDDL() if todoDDL != nil && globalResolvedTs >= todoDDL.CommitTs { // flush DMLs @@ -562,18 +575,21 @@ func (c *Consumer) Run(ctx context.Context) error { if todoDDL != nil && todoDDL.CommitTs < globalResolvedTs { globalResolvedTs = todoDDL.CommitTs } - if lastGlobalResolvedTs == globalResolvedTs { - continue + if lastGlobalResolvedTs > globalResolvedTs { + log.Panic("global ResolvedTs fallback") } - lastGlobalResolvedTs = globalResolvedTs - atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs) - log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs)) - err = c.forEachSink(func(sink *partitionSink) error { - return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) - }) - if err != nil { - return errors.Trace(err) + if globalResolvedTs > lastGlobalResolvedTs { + lastGlobalResolvedTs = globalResolvedTs + atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs) + log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs)) + + err = c.forEachSink(func(sink *partitionSink) error { + return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) + }) + if err != nil { + return errors.Trace(err) + } } } } From fd5bad5b89d51bcc26c1390c6bb86fa474392455 Mon Sep 17 00:00:00 2001 From: Obliviate <756541536@qq.com> Date: Tue, 25 Jan 2022 11:51:13 +0800 Subject: [PATCH 19/72] fix: data race (#4419) Co-authored-by: Ti Chi Robot --- dm/syncer/checkpoint.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 3a47074999e..49d83a344b2 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -700,7 +700,9 @@ func (cp *RemoteCheckPoint) FlushPointsExcept( if snapshotCp.globalPoint != nil { cp.globalPoint.flushBy(*snapshotCp.globalPoint) + cp.Lock() cp.globalPointSaveTime = snapshotCp.globalPointSaveTime + cp.Unlock() } for _, point := range points { From 990327d5f7d1bbba7380c29297a5e29a4fa1e331 Mon Sep 17 00:00:00 2001 From: glorv Date: Tue, 25 Jan 2022 14:32:11 +0800 Subject: [PATCH 20/72] dm: adjust lightning loader config (#4034) close pingcap/tiflow#3519 --- dm/_utils/terror_gen/errors_release.txt | 3 +- dm/dm/config/subtask.go | 47 ++--------- dm/dm/config/task.go | 58 +++++++++++-- dm/dm/config/task_converters.go | 3 - dm/dm/config/task_test.go | 8 +- dm/dm/worker/subtask.go | 27 +++--- dm/dm/worker/subtask_test.go | 4 +- dm/errors.toml | 18 ++-- dm/loader/lightning.go | 83 +++++++++++++++---- dm/pkg/schema/tracker.go | 2 +- dm/pkg/terror/error_list.go | 6 +- dm/tests/_utils/run_dm_ctl_with_tls_and_retry | 3 + dm/tests/all_mode/conf/dm-task.yaml | 1 + dm/tests/all_mode/run.sh | 8 +- dm/tests/dmctl_basic/conf/get_task.yaml | 3 +- dm/tests/dmctl_command/conf/dm-task.yaml | 1 - .../import_goroutine_leak/conf/dm-task.yaml | 1 + dm/tests/import_v10x/conf/task.yaml | 3 +- .../lightning_load_task/conf/dm-task.yaml | 4 +- .../lightning_load_task/conf/dm-task2.yaml | 3 - .../lightning_load_task/conf/dm-task3.yaml | 3 - .../lightning_load_task/conf/dm-task4.yaml | 3 - dm/tests/lightning_load_task/run.sh | 14 ++-- dm/tests/lightning_mode/conf/dm-task.yaml | 4 +- dm/tests/load_interrupt/conf/dm-task.yaml | 1 + dm/tests/new_relay/configs/tasks/test.yaml | 2 + dm/tests/print_status/run.sh | 16 +--- dm/tests/tls/conf/dm-task-2.yaml | 3 - dm/tests/tls/conf/dm-task.yaml | 3 - go.mod | 6 +- go.sum | 40 ++++++--- 31 files changed, 227 insertions(+), 154 deletions(-) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index df7b8731364..63fd3f42052 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -182,6 +182,8 @@ ErrConfigOnlineDDLMistakeRegex,[code=20049:class=config:scope=internal:level=hig ErrOpenAPITaskConfigExist,[code=20050:class=config:scope=internal:level=low], "Message: the openapi task config for '%s' already exist, Workaround: If you want to override it, please use the overwrite flag." ErrOpenAPITaskConfigNotExist,[code=20051:class=config:scope=internal:level=low], "Message: the openapi task config for '%s' does not exist" ErrConfigCollationCompatibleNotSupport,[code=20052:class=config:scope=internal:level=medium], "Message: collation compatible %s not supported, Workaround: Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." +ErrConfigInvalidLoadMode,[code=20053:class=config:scope=internal:level=medium], "Message: invalid load mode '%s', Workaround: Please choose a valid value in ['sql', 'loader']" +ErrConfigInvalidDuplicateResolution,[code=20054:class=config:scope=internal:level=medium], "Message: invalid load on-duplicate '%s', Workaround: Please choose a valid value in ['replace', 'error', 'ignore']" ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] @@ -269,7 +271,6 @@ ErrLoadUnitDuplicateTableFile,[code=34015:class=load-unit:scope=internal:level=h ErrLoadUnitGenBAList,[code=34016:class=load-unit:scope=internal:level=high], "Message: generate block allow list, Workaround: Please check the `block-allow-list` config in task configuration file." ErrLoadTaskWorkerNotMatch,[code=34017:class=functional:scope=internal:level=high], "Message: different worker in load stage, previous worker: %s, current worker: %s, Workaround: Please check if the previous worker is online." ErrLoadTaskCheckPointNotMatch,[code=34018:class=functional:scope=internal:level=high], "Message: inconsistent checkpoints between loader and target database, Workaround: If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command." -ErrLoadBackendNotSupport,[code=34019:class=functional:scope=internal:level=high], "Message: DM do not support backend %s , Workaround: If you do not understand the configure `tidb.backend` you can just delete it." ErrSyncerUnitPanic,[code=36001:class=sync-unit:scope=internal:level=high], "Message: panic error: %v" ErrSyncUnitInvalidTableName,[code=36002:class=sync-unit:scope=internal:level=high], "Message: extract table name for DML error: %s" ErrSyncUnitTableNameQuery,[code=36003:class=sync-unit:scope=internal:level=high], "Message: table name parse error: %s" diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index e8ec8bbb964..4c63b5d2f33 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/dumpling" @@ -174,37 +173,6 @@ func GetDBConfigForTest() DBConfig { return DBConfig{Host: "localhost", User: "root", Password: "not a real password", Port: 3306} } -// TiDBExtraConfig is the extra DB configuration only for TiDB. -type TiDBExtraConfig struct { - StatusPort int `toml:"status-port" json:"status-port" yaml:"status-port"` - PdAddr string `toml:"pd-addr" json:"pd-addr" yaml:"pd-addr"` - Backend string `toml:"backend" json:"backend" yaml:"backend"` -} - -func (db *TiDBExtraConfig) String() string { - cfg, err := json.Marshal(db) - if err != nil { - log.L().Error("fail to marshal config to json", log.ShortError(err)) - } - return string(cfg) -} - -// Toml returns TOML format representation of config. -func (db *TiDBExtraConfig) Toml() (string, error) { - var b bytes.Buffer - enc := toml.NewEncoder(&b) - if err := enc.Encode(db); err != nil { - return "", terror.ErrConfigTomlTransform.Delegate(err, "encode db config to toml") - } - return b.String(), nil -} - -// Decode loads config from file data. -func (db *TiDBExtraConfig) Decode(data string) error { - _, err := toml.Decode(data, db) - return terror.ErrConfigTomlTransform.Delegate(err, "decode db config") -} - // SubTaskConfig is the configuration for SubTask. type SubTaskConfig struct { // BurntSushi/toml seems have a bug for flag "-" @@ -256,10 +224,9 @@ type SubTaskConfig struct { RelayDir string `toml:"relay-dir" json:"relay-dir"` // UseRelay get value from dm-worker's relayEnabled - UseRelay bool `toml:"use-relay" json:"use-relay"` - From DBConfig `toml:"from" json:"from"` - To DBConfig `toml:"to" json:"to"` - TiDB TiDBExtraConfig `toml:"tidb" json:"tidb"` + UseRelay bool `toml:"use-relay" json:"use-relay"` + From DBConfig `toml:"from" json:"from"` + To DBConfig `toml:"to" json:"to"` RouteRules []*router.TableRule `toml:"route-rules" json:"route-rules"` FilterRules []*bf.BinlogEventRule `toml:"filter-rules" json:"filter-rules"` @@ -469,12 +436,12 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { return terror.ErrConfigInvalidChunkFileSize.Generate(c.MydumperConfig.ChunkFilesize) } - if c.TiDB.Backend != "" && c.TiDB.Backend != lcfg.BackendLocal && c.TiDB.Backend != lcfg.BackendTiDB { - return terror.ErrLoadBackendNotSupport.Generate(c.TiDB.Backend) - } if _, err := bf.NewBinlogEvent(c.CaseSensitive, c.FilterRules); err != nil { return terror.ErrConfigBinlogEventFilter.Delegate(err) } + if err := c.LoaderConfig.adjust(); err != nil { + return err + } // TODO: check every member // TODO: since we checked here, we could remove other terror like ErrSyncerUnitGenBAList @@ -558,5 +525,5 @@ func (c *SubTaskConfig) Clone() (*SubTaskConfig, error) { // NeedUseLightning returns whether need to use lightning loader. func (c *SubTaskConfig) NeedUseLightning() bool { - return (c.Mode == ModeAll || c.Mode == ModeFull) && c.TiDB.Backend != "" + return (c.Mode == ModeAll || c.Mode == ModeFull) && c.ImportMode == LoadModeSQL } diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 6c6cedcefdb..0ce58d97d55 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -207,11 +207,35 @@ func (m *MydumperConfig) UnmarshalYAML(unmarshal func(interface{}) error) error return nil } +// LoadMode defines different mode used in load phase. +type LoadMode string + +const ( + // LoadModeSQL means write data by sql statements, uses tidb-lightning tidb backend to load data. + LoadModeSQL LoadMode = "sql" + // LoadModeLoader is the legacy sql mode, use loader to load data. this should be replaced by sql mode in new version. + LoadModeLoader = "loader" +) + +// DuplicateResolveType defines the duplication resolution when meet duplicate rows. +type DuplicateResolveType string + +const ( + // OnDuplicateReplace represents replace the old row with new data. + OnDuplicateReplace DuplicateResolveType = "replace" + // OnDuplicateError represents return an error when meet duplicate row. + OnDuplicateError = "error" + // OnDuplicateIgnore represents ignore the new data when meet duplicate row. + OnDuplicateIgnore = "ignore" +) + // LoaderConfig represents loader process unit's specific config. type LoaderConfig struct { - PoolSize int `yaml:"pool-size" toml:"pool-size" json:"pool-size"` - Dir string `yaml:"dir" toml:"dir" json:"dir"` - SQLMode string `yaml:"-" toml:"-" json:"-"` // wrote by dump unit + PoolSize int `yaml:"pool-size" toml:"pool-size" json:"pool-size"` + Dir string `yaml:"dir" toml:"dir" json:"dir"` + SQLMode string `yaml:"-" toml:"-" json:"-"` // wrote by dump unit + ImportMode LoadMode `yaml:"import-mode" toml:"import-mode" json:"import-mode"` + OnDuplicate DuplicateResolveType `yaml:"on-duplicate" toml:"on-duplicate" json:"on-duplicate"` } // DefaultLoaderConfig return default loader config for task. @@ -235,6 +259,26 @@ func (m *LoaderConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { return nil } +func (m *LoaderConfig) adjust() error { + if m.ImportMode == "" { + m.ImportMode = LoadModeSQL + } + m.ImportMode = LoadMode(strings.ToLower(string(m.ImportMode))) + if m.ImportMode != LoadModeSQL && m.ImportMode != LoadModeLoader { + return terror.ErrConfigInvalidLoadMode.Generate(m.ImportMode) + } + + if m.OnDuplicate == "" { + m.OnDuplicate = OnDuplicateReplace + } + m.OnDuplicate = DuplicateResolveType(strings.ToLower(string(m.OnDuplicate))) + if m.OnDuplicate != OnDuplicateReplace && m.OnDuplicate != OnDuplicateError && m.OnDuplicate != OnDuplicateIgnore { + return terror.ErrConfigInvalidDuplicateResolution.Generate(m.OnDuplicate) + } + + return nil +} + // SyncerConfig represents syncer process unit's specific config. type SyncerConfig struct { MetaFile string `yaml:"meta-file" toml:"meta-file" json:"meta-file"` // meta filename, used only when load SubConfig directly @@ -344,9 +388,6 @@ type TaskConfig struct { // deprecated, replaced by `start-task --remove-meta` RemoveMeta bool `yaml:"remove-meta"` - // extra config when target db is TiDB - TiDB *TiDBExtraConfig `yaml:"tidb" toml:"tidb" json:"tidb"` - // task experimental configs Experimental struct { AsyncCheckpointFlush bool `yaml:"async-checkpoint-flush" toml:"async-checkpoint-flush" json:"async-checkpoint-flush"` @@ -696,7 +737,10 @@ func (c *TaskConfig) adjust() error { unusedConfigs = append(unusedConfigs, mydumper) } } - for loader := range c.Loaders { + for loader, cfg := range c.Loaders { + if err1 := cfg.adjust(); err1 != nil { + return err1 + } if globalConfigReferCount[configRefPrefixes[loaderIdx]+loader] == 0 { unusedConfigs = append(unusedConfigs, loader) } diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 7d0eb289be7..99c3c8ef7a9 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -99,9 +99,6 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]DBConfig) ([]* if err := cfg.Adjust(true); err != nil { return nil, terror.Annotatef(err, "source %s", inst.SourceID) } - if c.TiDB != nil { - cfg.TiDB = *c.TiDB - } cfgs[i] = cfg } if c.EnableHeartbeat { diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index fec4f7a078e..71571d8c455 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -690,8 +690,10 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { ExtraArgs: "--escape-backslash", }, LoaderConfig: LoaderConfig{ - PoolSize: 32, - Dir: "./dumpped_data", + PoolSize: 32, + Dir: "./dumpped_data", + ImportMode: LoadModeSQL, + OnDuplicate: OnDuplicateReplace, }, SyncerConfig: SyncerConfig{ WorkerCount: 32, @@ -1029,7 +1031,7 @@ func (t *testConfig) TestTaskConfigForDowngrade(c *C) { // make sure all new field were added cfgReflect := reflect.Indirect(reflect.ValueOf(cfg)) cfgForDowngradeReflect := reflect.Indirect(reflect.ValueOf(cfgForDowngrade)) - c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+4) // without flag, tidb, collation_compatible and experimental + c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+3) // without flag, collation_compatible and experimental // make sure all field were copied cfgForClone := &TaskConfigForDowngrade{} diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index 8615e76010b..ca04012059a 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -61,20 +61,12 @@ func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, wor switch cfg.Mode { case config.ModeAll: us = append(us, dumpling.NewDumpling(cfg)) - if cfg.NeedUseLightning() { - us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) - } else { - us = append(us, loader.NewLoader(cfg, etcdClient, workerName)) - } + us = append(us, newLoadUnit(cfg, etcdClient, workerName)) us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) case config.ModeFull: // NOTE: maybe need another checker in the future? us = append(us, dumpling.NewDumpling(cfg)) - if cfg.NeedUseLightning() { - us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) - } else { - us = append(us, loader.NewLoader(cfg, etcdClient, workerName)) - } + us = append(us, newLoadUnit(cfg, etcdClient, workerName)) case config.ModeIncrement: us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) default: @@ -83,6 +75,21 @@ func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, wor return us } +func newLoadUnit(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, workerName string) unit.Unit { + hasAutoGenColumn := false + for _, rule := range cfg.RouteRules { + if rule.SchemaExtractor != nil || rule.TableExtractor != nil || rule.SourceExtractor != nil { + hasAutoGenColumn = true + break + } + } + // tidb-lightning doesn't support column mapping currently + if cfg.ImportMode == config.LoadModeLoader || cfg.OnDuplicate == config.OnDuplicateError || hasAutoGenColumn || len(cfg.ColumnMappingRules) > 0 { + return loader.NewLoader(cfg, etcdClient, workerName) + } + return loader.NewLightning(cfg, etcdClient, workerName) +} + // SubTask represents a sub task of data migration. type SubTask struct { cfg *config.SubTaskConfig diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 8fb5006f03a..41e7a5a2152 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -55,7 +55,7 @@ func (t *testSubTask) TestCreateUnits(c *C) { c.Assert(unitsFull, HasLen, 2) _, ok := unitsFull[0].(*dumpling.Dumpling) c.Assert(ok, IsTrue) - _, ok = unitsFull[1].(*loader.Loader) + _, ok = unitsFull[1].(*loader.LightningLoader) c.Assert(ok, IsTrue) cfg.Mode = config.ModeIncrement @@ -69,7 +69,7 @@ func (t *testSubTask) TestCreateUnits(c *C) { c.Assert(unitsAll, HasLen, 3) _, ok = unitsAll[0].(*dumpling.Dumpling) c.Assert(ok, IsTrue) - _, ok = unitsAll[1].(*loader.Loader) + _, ok = unitsAll[1].(*loader.LightningLoader) c.Assert(ok, IsTrue) _, ok = unitsAll[2].(*syncer.Syncer) c.Assert(ok, IsTrue) diff --git a/dm/errors.toml b/dm/errors.toml index 22f1bbe6bc7..ede767d93e9 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1102,6 +1102,18 @@ description = "" workaround = "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." tags = ["internal", "medium"] +[error.DM-config-20053] +message = "invalid load mode '%s'" +description = "" +workaround = "Please choose a valid value in ['sql', 'loader']" +tags = ["internal", "medium"] + +[error.DM-config-20054] +message = "invalid load on-duplicate '%s'" +description = "" +workaround = "Please choose a valid value in ['replace', 'error', 'ignore']" +tags = ["internal", "medium"] + [error.DM-binlog-op-22001] message = "" description = "" @@ -1624,12 +1636,6 @@ description = "" workaround = "If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command." tags = ["internal", "high"] -[error.DM-functional-34019] -message = "DM do not support backend %s " -description = "" -workaround = "If you do not understand the configure `tidb.backend` you can just delete it." -tags = ["internal", "high"] - [error.DM-sync-unit-36001] message = "panic error: %v" description = "" diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index 66b41fdf347..08cc1bc5df1 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" @@ -48,6 +49,7 @@ type LightningLoader struct { sync.RWMutex timeZone string + sqlMode string lightningGlobalConfig *lcfg.GlobalConfig cfg *config.SubTaskConfig @@ -95,12 +97,9 @@ func makeGlobalConfig(cfg *config.SubTaskConfig) *lcfg.GlobalConfig { lightningCfg.TiDB.Psw = cfg.To.Password lightningCfg.TiDB.User = cfg.To.User lightningCfg.TiDB.Port = cfg.To.Port - lightningCfg.TiDB.StatusPort = cfg.TiDB.StatusPort - lightningCfg.TiDB.PdAddr = cfg.TiDB.PdAddr - lightningCfg.TiDB.LogLevel = cfg.LogLevel - lightningCfg.TikvImporter.Backend = cfg.TiDB.Backend + lightningCfg.TikvImporter.Backend = lcfg.BackendTiDB lightningCfg.PostRestore.Checksum = lcfg.OpLevelOff - if cfg.TiDB.Backend == lcfg.BackendLocal { + if lightningCfg.TikvImporter.Backend == lcfg.BackendLocal { lightningCfg.TikvImporter.SortedKVDir = cfg.Dir } lightningCfg.Mydumper.SourceDir = cfg.Dir @@ -148,25 +147,64 @@ func (l *LightningLoader) Init(ctx context.Context) (err error) { } } l.timeZone = timeZone + + for k, v := range l.cfg.To.Session { + if strings.ToLower(k) == "sql_mode" { + l.sqlMode = v + break + } + } + + if len(l.sqlMode) == 0 { + sqlModes, err3 := utils.AdjustSQLModeCompatible(l.cfg.LoaderConfig.SQLMode) + if err3 != nil { + l.logger.Warn("cannot adjust sql_mode compatible, the sql_mode will stay the same", log.ShortError(err3)) + } + l.sqlMode = sqlModes + } return nil } +func (l *LightningLoader) ignoreCheckpointError(ctx context.Context, cfg *lcfg.Config) error { + status, err := l.checkPointList.taskStatus(ctx) + if err != nil { + return err + } + if status != lightningStatusRunning { + return nil + } + cpdb, err := checkpoints.OpenCheckpointsDB(ctx, cfg) + if err != nil { + return err + } + defer func() { + _ = cpdb.Close() + }() + return errors.Trace(cpdb.IgnoreErrorCheckpoint(ctx, "all")) +} + func (l *LightningLoader) runLightning(ctx context.Context, cfg *lcfg.Config) error { taskCtx, cancel := context.WithCancel(ctx) l.Lock() l.cancel = cancel l.Unlock() - if err := l.checkPointList.UpdateStatus(ctx, lightningStatusRunning); err != nil { + + // always try to skill all checkpoint errors so we can resume this phase. + err := l.ignoreCheckpointError(ctx, cfg) + if err != nil { + l.logger.Warn("check lightning checkpoint status failed, skip this error", log.ShortError(err)) + } + if err = l.checkPointList.UpdateStatus(ctx, lightningStatusRunning); err != nil { return err } - err := l.core.RunOnce(taskCtx, cfg, nil) - failpoint.Inject("LightningLoadDataSlowDown", nil) - failpoint.Inject("LightningLoadDataSlowDownByTask", func(val failpoint.Value) { + err = l.core.RunOnce(taskCtx, cfg, nil) + failpoint.Inject("LoadDataSlowDown", nil) + failpoint.Inject("LoadDataSlowDownByTask", func(val failpoint.Value) { tasks := val.(string) taskNames := strings.Split(tasks, ",") for _, taskName := range taskNames { if l.cfg.Name == taskName { - l.logger.Info("inject failpoint LightningLoadDataSlowDownByTask", zap.String("task", taskName)) + l.logger.Info("inject failpoint LoadDataSlowDownByTask in lightning loader", zap.String("task", taskName)) <-taskCtx.Done() } } @@ -197,14 +235,20 @@ func (l *LightningLoader) restore(ctx context.Context) error { cpPath := filepath.Join(l.cfg.LoaderConfig.Dir, lightningCheckpointFileName) cfg.Checkpoint.DSN = cpPath cfg.Checkpoint.KeepAfterSuccess = lcfg.CheckpointOrigin + cfg.TikvImporter.OnDuplicate = string(l.cfg.OnDuplicate) cfg.TiDB.Vars = make(map[string]string) + cfg.Routes = l.cfg.RouteRules if l.cfg.To.Session != nil { for k, v := range l.cfg.To.Session { cfg.TiDB.Vars[k] = v } } - cfg.TiDB.StrSQLMode = l.cfg.LoaderConfig.SQLMode - cfg.TiDB.Vars = map[string]string{"time_zone": l.timeZone} + cfg.TiDB.StrSQLMode = l.sqlMode + cfg.TiDB.Vars = map[string]string{ + "time_zone": l.timeZone, + // always set transaction mode to optimistic + "tidb_txn_mode": "optimistic", + } err = l.runLightning(ctx, cfg) if err == nil { l.finish.Store(true) @@ -264,7 +308,12 @@ func (l *LightningLoader) Process(ctx context.Context, pr chan pb.ProcessResult) isCanceled = true default: } - l.logger.Info("lightning load end", zap.Bool("IsCanceled", isCanceled)) + s := l.status() + l.logger.Info("lightning load end", + zap.Bool("IsCanceled", isCanceled), + zap.Int64("finished_bytes", s.FinishedBytes), + zap.Int64("total_bytes", s.TotalBytes), + zap.String("progress", s.Progress)) pr <- pb.ProcessResult{IsCanceled: isCanceled, Errors: errs} } @@ -324,8 +373,7 @@ func (l *LightningLoader) Update(ctx context.Context, cfg *config.SubTaskConfig) return nil } -// Status returns the unit's current status. -func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { +func (l *LightningLoader) status() *pb.LoadStatus { finished, total := l.core.Status() progress := percent(finished, total, l.finish.Load()) s := &pb.LoadStatus{ @@ -337,3 +385,8 @@ func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { } return s } + +// Status returns the unit's current status. +func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { + return l.status() +} diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 3c15c3eb05f..1d2e0052bdb 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -345,7 +345,7 @@ func (tr *Tracker) CreateSchemaIfNotExists(db string) error { if tr.dom.InfoSchema().SchemaExists(dbName) { return nil } - return tr.dom.DDL().CreateSchema(tr.se, dbName, nil, nil, nil) + return tr.dom.DDL().CreateSchema(tr.se, dbName, nil, nil) } // cloneTableInfo creates a clone of the TableInfo. diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 5d92eff424a..99e4e70488c 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -247,6 +247,8 @@ const ( codeConfigOpenAPITaskConfigExist codeConfigOpenAPITaskConfigNotExist codeCollationCompatibleNotSupport + codeConfigInvalidLoadMode + codeConfigInvalidLoadDuplicateResolution ) // Binlog operation error code list. @@ -362,7 +364,6 @@ const ( codeLoadUnitGenBAList codeLoadTaskWorkerNotMatch codeLoadCheckPointNotMatch - codeLoadBackendNotMatch ) // Sync unit error code. @@ -904,6 +905,8 @@ var ( ErrOpenAPITaskConfigExist = New(codeConfigOpenAPITaskConfigExist, ClassConfig, ScopeInternal, LevelLow, "the openapi task config for '%s' already exist", "If you want to override it, please use the overwrite flag.") ErrOpenAPITaskConfigNotExist = New(codeConfigOpenAPITaskConfigNotExist, ClassConfig, ScopeInternal, LevelLow, "the openapi task config for '%s' does not exist", "") ErrConfigCollationCompatibleNotSupport = New(codeCollationCompatibleNotSupport, ClassConfig, ScopeInternal, LevelMedium, "collation compatible %s not supported", "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`.") + ErrConfigInvalidLoadMode = New(codeConfigInvalidLoadMode, ClassConfig, ScopeInternal, LevelMedium, "invalid load mode '%s'", "Please choose a valid value in ['sql', 'loader']") + ErrConfigInvalidDuplicateResolution = New(codeConfigInvalidLoadDuplicateResolution, ClassConfig, ScopeInternal, LevelMedium, "invalid load on-duplicate '%s'", "Please choose a valid value in ['replace', 'error', 'ignore']") // Binlog operation error. ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") @@ -1005,7 +1008,6 @@ var ( ErrLoadUnitGenBAList = New(codeLoadUnitGenBAList, ClassLoadUnit, ScopeInternal, LevelHigh, "generate block allow list", "Please check the `block-allow-list` config in task configuration file.") ErrLoadTaskWorkerNotMatch = New(codeLoadTaskWorkerNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "different worker in load stage, previous worker: %s, current worker: %s", "Please check if the previous worker is online.") ErrLoadTaskCheckPointNotMatch = New(codeLoadCheckPointNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "inconsistent checkpoints between loader and target database", "If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command.") - ErrLoadBackendNotSupport = New(codeLoadBackendNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "DM do not support backend %s ", "If you do not understand the configure `tidb.backend` you can just delete it.") // Sync unit error. ErrSyncerUnitPanic = New(codeSyncerUnitPanic, ClassSyncUnit, ScopeInternal, LevelHigh, "panic error: %v", "") diff --git a/dm/tests/_utils/run_dm_ctl_with_tls_and_retry b/dm/tests/_utils/run_dm_ctl_with_tls_and_retry index cdc049defce..5f7ed3e9df0 100755 --- a/dm/tests/_utils/run_dm_ctl_with_tls_and_retry +++ b/dm/tests/_utils/run_dm_ctl_with_tls_and_retry @@ -39,6 +39,9 @@ for ((k = 0; k < 10; k++)); do break fi done + if [ $# -eq 0 ]; then + cat $dmctl_log + fi if $all_matched; then exit 0 fi diff --git a/dm/tests/all_mode/conf/dm-task.yaml b/dm/tests/all_mode/conf/dm-task.yaml index 17553c1b0f7..851519b4760 100644 --- a/dm/tests/all_mode/conf/dm-task.yaml +++ b/dm/tests/all_mode/conf/dm-task.yaml @@ -56,6 +56,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: sql syncers: global: diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 043511d9f43..d933b763d1b 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -176,7 +176,9 @@ function test_stop_task_before_checkpoint() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # generate uncomplete checkpoint - dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml + sed -i "s/import-mode: sql/import-mode: loader/" $WORK_DIR/dm-task.yaml + dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" check_log_contain_with_retry 'wait loader stop after init checkpoint' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'wait loader stop after init checkpoint' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -195,14 +197,14 @@ function test_stop_task_before_checkpoint() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # stop-task before load checkpoint - dmctl_start_task "$cur/conf/dm-task.yaml" + dmctl_start_task $WORK_DIR/dm-task.yaml check_log_contain_with_retry 'wait loader stop before load checkpoint' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'wait loader stop before load checkpoint' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ "\"result\": true" 3 - dmctl_start_task "$cur/conf/dm-task.yaml" + dmctl_start_task $WORK_DIR/dm-task.yaml check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index 4cc9cfac480..30aa6e566a5 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -142,6 +142,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" @@ -160,6 +162,5 @@ syncers: clean-dump-file: true ansi-quotes: false remove-meta: false -tidb: null experimental: async-checkpoint-flush: false diff --git a/dm/tests/dmctl_command/conf/dm-task.yaml b/dm/tests/dmctl_command/conf/dm-task.yaml index 702e88919e3..21b6540b32b 100644 --- a/dm/tests/dmctl_command/conf/dm-task.yaml +++ b/dm/tests/dmctl_command/conf/dm-task.yaml @@ -7,7 +7,6 @@ meta-schema: "dm_meta" heartbeat-update-interval: 1 heartbeat-report-interval: 1 remove-meta: true -tidb: null target-database: host: "127.0.0.1" diff --git a/dm/tests/import_goroutine_leak/conf/dm-task.yaml b/dm/tests/import_goroutine_leak/conf/dm-task.yaml index 210bca31e2f..d10c5591722 100644 --- a/dm/tests/import_goroutine_leak/conf/dm-task.yaml +++ b/dm/tests/import_goroutine_leak/conf/dm-task.yaml @@ -34,6 +34,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "loader" syncers: global: diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index c5d449f1810..a8756979ed2 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -87,6 +87,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" @@ -119,6 +121,5 @@ syncers: clean-dump-file: false ansi-quotes: false remove-meta: false -tidb: null experimental: async-checkpoint-flush: false diff --git a/dm/tests/lightning_load_task/conf/dm-task.yaml b/dm/tests/lightning_load_task/conf/dm-task.yaml index b29aa864d12..a8f2e93d964 100644 --- a/dm/tests/lightning_load_task/conf/dm-task.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task.yaml @@ -25,9 +25,6 @@ mysql-instances: loader-config-name: "global" syncer-config-name: "global" -tidb: - backend: "tidb" - block-allow-list: instance: do-dbs: ["load_task1"] @@ -43,6 +40,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "sql" syncers: global: diff --git a/dm/tests/lightning_load_task/conf/dm-task2.yaml b/dm/tests/lightning_load_task/conf/dm-task2.yaml index a751da96d85..14c8b07645d 100644 --- a/dm/tests/lightning_load_task/conf/dm-task2.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task2.yaml @@ -25,9 +25,6 @@ mysql-instances: loader-config-name: "global" syncer-config-name: "global" -tidb: - backend: "tidb" - block-allow-list: instance: do-dbs: ["load_task2"] diff --git a/dm/tests/lightning_load_task/conf/dm-task3.yaml b/dm/tests/lightning_load_task/conf/dm-task3.yaml index e6395139e8e..0a06a4eabfb 100644 --- a/dm/tests/lightning_load_task/conf/dm-task3.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task3.yaml @@ -23,9 +23,6 @@ block-allow-list: instance: do-dbs: ["load_task3"] -tidb: - backend: "tidb" - mydumpers: global: threads: 4 diff --git a/dm/tests/lightning_load_task/conf/dm-task4.yaml b/dm/tests/lightning_load_task/conf/dm-task4.yaml index d2abd5092f3..7e153eb4c49 100644 --- a/dm/tests/lightning_load_task/conf/dm-task4.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task4.yaml @@ -30,9 +30,6 @@ mydumpers: skip-tz-utc: true extra-args: "" -tidb: - backend: "tidb" - loaders: global: pool-size: 16 diff --git a/dm/tests/lightning_load_task/run.sh b/dm/tests/lightning_load_task/run.sh index 6848fc344da..6e130f06623 100755 --- a/dm/tests/lightning_load_task/run.sh +++ b/dm/tests/lightning_load_task/run.sh @@ -33,7 +33,7 @@ function test_worker_restart() { "Please check if the previous worker is online." 1 # worker1 online - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -82,7 +82,7 @@ function test_transfer_two_sources() { "\"unit\": \"Load\"" 1 # worker2 online - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDown=sleep(15000)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(15000)" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT @@ -184,7 +184,7 @@ function run() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 # worker1 loading load_task1 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT cp $CONF_DIR/source1.yaml $WORK_DIR/source1.yaml @@ -192,7 +192,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 # worker2 loading load_task2 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task2\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task2\")" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT cp $CONF_DIR/source2.yaml $WORK_DIR/source2.yaml @@ -200,7 +200,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # worker3 loading load_task3 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task3\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task3\")" run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $CONF_DIR/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT @@ -208,8 +208,8 @@ function run() { dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" dmctl_start_task "$cur/conf/dm-task2.yaml" "--remove-meta" - check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker1/log/dm-worker.log - check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker2/log/dm-worker.log + check_log_contain_with_retry 'inject failpoint LoadDataSlowDownByTask in lightning loader' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'inject failpoint LoadDataSlowDownByTask in lightning loader' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status load_task1" \ "\"unit\": \"Load\"" 1 \ diff --git a/dm/tests/lightning_mode/conf/dm-task.yaml b/dm/tests/lightning_mode/conf/dm-task.yaml index 2b3d43261c0..d33b6ed3157 100644 --- a/dm/tests/lightning_mode/conf/dm-task.yaml +++ b/dm/tests/lightning_mode/conf/dm-task.yaml @@ -37,9 +37,6 @@ black-white-list: # compatible with deprecated config do-dbs: ["lightning_mode"] ignore-dbs: ["ignore_db"] -tidb: - backend: "tidb" - filters: # filter rules, mysql instance can ref rules in it user-filter-1: schema-pattern: "*" # pattern of the upstream schema name, wildcard characters (*?) are supported @@ -58,6 +55,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "sql" syncers: global: diff --git a/dm/tests/load_interrupt/conf/dm-task.yaml b/dm/tests/load_interrupt/conf/dm-task.yaml index d09b9d04347..833ba15b113 100644 --- a/dm/tests/load_interrupt/conf/dm-task.yaml +++ b/dm/tests/load_interrupt/conf/dm-task.yaml @@ -34,6 +34,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "loader" syncers: global: diff --git a/dm/tests/new_relay/configs/tasks/test.yaml b/dm/tests/new_relay/configs/tasks/test.yaml index 8c174e41f7b..3040c12592c 100644 --- a/dm/tests/new_relay/configs/tasks/test.yaml +++ b/dm/tests/new_relay/configs/tasks/test.yaml @@ -43,6 +43,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" diff --git a/dm/tests/print_status/run.sh b/dm/tests/print_status/run.sh index 67469c5ecf4..7f11f1b3d66 100755 --- a/dm/tests/print_status/run.sh +++ b/dm/tests/print_status/run.sh @@ -64,21 +64,9 @@ function check_print_status() { echo "checking print status" # check load unit print status status_file=$WORK_DIR/worker1/log/loader_status.log - grep -oP "\[unit=load\] \[finished_bytes=[0-9]+\] \[total_bytes=59637\] \[total_file_count=3\] \[progress=.*\]" $WORK_DIR/worker1/log/dm-worker.log >$status_file - #grep -oP "loader.*\Kfinished_bytes = [0-9]+, total_bytes = [0-9]+, total_file_count = [0-9]+, progress = .*" $WORK_DIR/worker1/log/dm-worker.log > $status_file + grep -oP "\[unit=lightning-load\] \[IsCanceled=false\] \[finished_bytes=59637\] \[total_bytes=59637\] \[progress=.*\]" $WORK_DIR/worker1/log/dm-worker.log >$status_file status_count=$(wc -l $status_file | awk '{print $1}') - [ $status_count -ge 2 ] - count=0 - cat $status_file | while read -r line; do - total_file_count=$(echo "$line" | awk '{print $(NF-2)}' | tr -d "[total_file_count=" | tr -d "]") - [ $total_file_count -eq 3 ] - count=$((count + 1)) - if [ $count -eq $status_count ]; then - finished_bytes=$(echo "$line" | awk '{print $2}' | tr -d "[finished_bytes=" | tr -d "]") - total_bytes=$(echo "$line" | awk '{print $3}' | tr -d "[total_file_count" | tr -d "]") - [[ "$finished_bytes" -eq "$total_bytes" ]] - fi - done + [ $status_count -eq 1 ] echo "check load unit print status success" # check sync unit print status diff --git a/dm/tests/tls/conf/dm-task-2.yaml b/dm/tests/tls/conf/dm-task-2.yaml index 7c31d4ed9a2..27a0de091a3 100644 --- a/dm/tests/tls/conf/dm-task-2.yaml +++ b/dm/tests/tls/conf/dm-task-2.yaml @@ -47,6 +47,3 @@ syncers: global: worker-count: 16 batch: 100 - -tidb: - backend: "tidb" diff --git a/dm/tests/tls/conf/dm-task.yaml b/dm/tests/tls/conf/dm-task.yaml index eaae18aebd9..26e885afc9d 100644 --- a/dm/tests/tls/conf/dm-task.yaml +++ b/dm/tests/tls/conf/dm-task.yaml @@ -41,6 +41,3 @@ syncers: global: worker-count: 16 batch: 100 - -tidb: - backend: "tidb" diff --git a/go.mod b/go.mod index 5d6bed4fd4e..15724e773d3 100644 --- a/go.mod +++ b/go.mod @@ -53,12 +53,12 @@ require ( github.com/philhofer/fwd v1.0.0 // indirect github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd + github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c github.com/pingcap/log v0.0.0-20211207084639-71a2e5860834 - github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c + github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible - github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f + github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/r3labs/diff v1.1.0 diff --git a/go.sum b/go.sum index ba80675aeb9..23277d93f7b 100644 --- a/go.sum +++ b/go.sum @@ -89,7 +89,6 @@ github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -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= @@ -334,8 +333,9 @@ github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7 github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 h1:0IsP4ViNmA7ZElbCE4/lINdTppdw3jdcAiJaPDyeHx8= github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= -github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= +github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= @@ -671,6 +671,8 @@ github.com/lib/pq v1.3.0 h1:/qkRGz8zljWiDcFvgpwUpwIAPu3r07TDvs3Rws+o/pU= github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/linkedin/goavro/v2 v2.9.8 h1:jN50elxBsGBDGVDEKqUlDuU1cFwJ11K/yrJCBMe/7Wg= github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -815,8 +817,9 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd h1:I8IeI8MNiZVKnwuXhcIIzz6pREcOSbq18Q31KYIzFVM= github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd/go.mod h1:IVF+ijPSMZVtx2oIqxAg7ur6EyixtTYfOHwpfmlhqI4= +github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= +github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZLmhahmvHm7n9DUxGRQT00208= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= @@ -844,11 +847,11 @@ github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug 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/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= -github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d h1:k3/APKZjXOyJrFy8VyYwRlZhMelpD3qBLJNsw3bPl/g= -github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d/go.mod h1:7j18ezaWTao2LHOyMlsc2Dg1vW+mDY9dEbPzVyOlaeM= +github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= +github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= github.com/pingcap/tidb v1.1.0-beta.0.20211023132847-efa94595c071/go.mod h1:Ci7ABF58a4jn6YtaHi7655jP409edqC2JxWWFRqOubg= -github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c h1:NcpItOfJ3X0OSPUIY6DGS6QC79Ovcdb0TRXwbKHm15A= -github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c/go.mod h1:z/k01yGpWuIx2qnM3/gPSOZ95JHAMBx51WXyWWUPH1w= +github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d h1:A7JcevlsUn4wd++wBF3re1rrEPzGdYYuq0zRtUTwRmE= +github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d/go.mod h1:VNhK7vWI4TEQhI8M9Pk06YAp1slz8iymaUKaVUrfqWE= github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= @@ -859,8 +862,8 @@ github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= github.com/pingcap/tidb/parser v0.0.0-20211023132847-efa94595c071/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= -github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f h1:ZR9DHZXg02aFzmSVIqMEYwCG2VyTFk9gSk85u+RK4u8= -github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= +github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d h1:QjRWklG/hLNibuK3jBk7uhnwGB3GM87f9MBw37dqOnw= +github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 h1:DHU4vw0o15qdKsf7d/Pyhun4YtX8FwoDQxG0plPByUg= github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= @@ -876,6 +879,8 @@ github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZ 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/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= 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= @@ -938,9 +943,10 @@ github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg 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= +github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.3+incompatible h1:uenXGGa8ESCQq+dbgtl916dmg6PSAz2cXov0uORQ9v8= -github.com/shirou/gopsutil v3.21.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil/v3 v3.21.12 h1:VoGxEW2hpmz0Vt3wUvHIl9fquzYLNpVpgNNB7pGJimA= +github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= github.com/shopspring/decimal v1.3.0 h1:KK3gWIXskZ2O1U/JNTisNcvH+jveJxZYrjbTsrbbnh8= @@ -1038,10 +1044,12 @@ github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr9 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= github.com/tinylib/msgp v1.1.0 h1:9fQd+ICuRIu/ue4vxJZu6/LzxN0HwMds2nq/0cFvxHU= github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= -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= +github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= +github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= +github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2biQ= +github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= @@ -1116,6 +1124,8 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= @@ -1356,6 +1366,7 @@ golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7w 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-20190916202348-b4ddaad3f8a3/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-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1386,6 +1397,7 @@ golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7w 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-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1403,8 +1415,10 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211031064116-611d5d643895/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= From 180fa4e001b3f4ab3b72ce70176bbade542c3ed9 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Tue, 25 Jan 2022 18:32:11 +0800 Subject: [PATCH 21/72] Revert "redo (ticdc): make FlushLogs in manager asynchronously (#4301)" (#4475) ref pingcap/tiflow#4222 --- cdc/redo/manager.go | 58 +++++++-------------------------------------- 1 file changed, 8 insertions(+), 50 deletions(-) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 011477eb9be..96b979dde41 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -54,13 +54,9 @@ const ( ) const ( - // supposing to replicate 10k tables, each table has one cached change averagely. - // approximate 156.25KB + // supposing to replicate 10k tables, each table has one cached changce averagely. logBufferChanSize = 10000 - // supposing to replicate 10k tables, each table has one resolvedTs change averagely. - // approximate 156.25KB - flushBufferChanSize = 10000 - logBufferTimeout = time.Minute * 10 + logBufferTimeout = time.Minute * 10 ) // IsValidConsistentLevel checks whether a give consistent level is valid @@ -127,12 +123,6 @@ type cacheRows struct { rows []*model.RowChangedEvent } -// resolvedEvent represents a resolvedTs event -type resolvedEvent struct { - tableID model.TableID - resolvedTs model.Ts -} - // ManagerImpl manages redo log writer, buffers un-persistent redo logs, calculates // redo log resolved ts. It implements LogManager interface. type ManagerImpl struct { @@ -140,9 +130,8 @@ type ManagerImpl struct { level ConsistentLevelType storageType consistentStorage - flushBuffer chan resolvedEvent - logBuffer chan cacheRows - writer writer.RedoLogWriter + logBuffer chan cacheRows + writer writer.RedoLogWriter minResolvedTs uint64 tableIDs []model.TableID @@ -168,8 +157,7 @@ func NewManager(ctx context.Context, cfg *config.ConsistentConfig, opts *Manager level: ConsistentLevelType(cfg.Level), storageType: consistentStorage(uri.Scheme), rtsMap: make(map[model.TableID]uint64), - logBuffer: make(chan cacheRows, logBufferChanSize), /* approximate 0.228MB */ - flushBuffer: make(chan resolvedEvent, flushBufferChanSize), /* approximate 0.152MB */ + logBuffer: make(chan cacheRows, logBufferChanSize), } switch m.storageType { @@ -208,7 +196,6 @@ func NewManager(ctx context.Context, cfg *config.ConsistentConfig, opts *Manager if opts.EnableBgRunner { go m.bgUpdateResolvedTs(ctx, opts.ErrCh) - go m.bgFlushLog(ctx, opts.ErrCh) go m.bgWriteLog(ctx, opts.ErrCh) } return m, nil @@ -291,16 +278,7 @@ func (m *ManagerImpl) FlushLog( return nil } defer atomic.StoreInt64(&m.flushing, 0) - - select { - case <-ctx.Done(): - return ctx.Err() - case m.flushBuffer <- resolvedEvent{ - tableID: tableID, - resolvedTs: resolvedTs, - }: - return nil - } + return m.writer.FlushLog(ctx, tableID, resolvedTs) } // EmitDDLEvent sends DDL event to redo log writer @@ -397,7 +375,7 @@ func (m *ManagerImpl) bgUpdateResolvedTs(ctx context.Context, errCh chan<- error select { case errCh <- err: default: - log.Error("redo log manager err channel is full", zap.Error(err)) + log.Error("err channel is full", zap.Error(err)) } return } @@ -420,30 +398,10 @@ func (m *ManagerImpl) bgWriteLog(ctx context.Context, errCh chan<- error) { select { case errCh <- err: default: - log.Error("redo log manager err channel is full", zap.Error(err)) - } - return - } - } - } -} - -func (m *ManagerImpl) bgFlushLog(ctx context.Context, errCh chan<- error) { - for { - select { - case <-ctx.Done(): - return - case event := <-m.flushBuffer: - err := m.writer.FlushLog(ctx, event.tableID, event.resolvedTs) - if err != nil { - select { - case errCh <- err: - default: - log.Error("redo log manager err channel is full", zap.Error(err)) + log.Error("err channel is full", zap.Error(err)) } return } - } } } From f51e410e50477298805eac3f3a4d5b5d6f4a5292 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Tue, 25 Jan 2022 19:58:11 +0800 Subject: [PATCH 22/72] errors,version(ticdc): refine ErrCheckClusterVersionFromPD error (#4435) close pingcap/tiflow#4341 --- errors.toml | 2 +- pkg/errors/errors.go | 2 +- pkg/version/check.go | 21 +++++++++++---------- pkg/version/check_test.go | 20 +++++++++++++++++++- tools/check/check-errdoc.sh | 2 +- 5 files changed, 33 insertions(+), 14 deletions(-) diff --git a/errors.toml b/errors.toml index 77e74567dae..8726dc63a35 100755 --- a/errors.toml +++ b/errors.toml @@ -143,7 +143,7 @@ changefeed update error: %s ["CDC:ErrCheckClusterVersionFromPD"] error = ''' -failed to request PD +failed to request PD %s, please try again later ''' ["CDC:ErrCheckDataDirSatisfied"] diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 6b97fb1f893..5acab118024 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -117,7 +117,7 @@ var ( // utilities related errors ErrToTLSConfigFailed = errors.Normalize("generate tls config failed", errors.RFCCodeText("CDC:ErrToTLSConfigFailed")) - ErrCheckClusterVersionFromPD = errors.Normalize("failed to request PD", errors.RFCCodeText("CDC:ErrCheckClusterVersionFromPD")) + ErrCheckClusterVersionFromPD = errors.Normalize("failed to request PD %s, please try again later", errors.RFCCodeText("CDC:ErrCheckClusterVersionFromPD")) ErrNewSemVersion = errors.Normalize("create sem version", errors.RFCCodeText("CDC:ErrNewSemVersion")) ErrCheckDirWritable = errors.Normalize("check dir writable failed", errors.RFCCodeText("CDC:ErrCheckDirWritable")) ErrCheckDirReadable = errors.Normalize("check dir readable failed", errors.RFCCodeText("CDC:ErrCheckDirReadable")) diff --git a/pkg/version/check.go b/pkg/version/check.go index ccc5d064fb0..436e6306927 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -110,28 +110,29 @@ func CheckPDVersion(ctx context.Context, pdAddr string, credential *security.Cre req, err := http.NewRequestWithContext( ctx, http.MethodGet, fmt.Sprintf("%s/pd/api/v1/version", pdAddr), nil) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } resp, err := httpClient.Do(req) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } defer resp.Body.Close() - if resp.StatusCode < 200 || resp.StatusCode >= 300 { - arg := fmt.Sprintf("response status: %s", resp.Status) - return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) - } - content, err := io.ReadAll(resp.Body) - if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + if err != nil || resp.StatusCode < 200 || resp.StatusCode >= 300 { + var arg string + if err != nil { + arg = fmt.Sprintf("%s %s %s", resp.Status, content, err) + } else { + arg = fmt.Sprintf("%s %s", resp.Status, content) + } + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) } err = json.Unmarshal(content, &pdVer) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } ver, err := semver.NewVersion(removeVAndHash(pdVer.Version)) diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index 38118c49ca2..99878761e32 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "net/http" + "net/http/httptest" "net/url" "testing" "time" @@ -171,7 +172,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, pdAddrs, nil, false) - require.Regexp(t, ".*response status: .*", err) + require.Regexp(t, ".*400 Bad Request.*", err) } } @@ -361,3 +362,20 @@ func TestCheckTiCDCClusterVersion(t *testing.T) { } } } + +func TestCheckPDVersionError(t *testing.T) { + t.Parallel() + + var resp func(w http.ResponseWriter, r *http.Request) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + resp(w, r) + })) + defer ts.Close() + + resp = func(w http.ResponseWriter, _ *http.Request) { + w.WriteHeader(http.StatusInternalServerError) + } + require.Contains(t, CheckPDVersion(context.TODO(), ts.URL, nil).Error(), + "[CDC:ErrCheckClusterVersionFromPD]failed to request PD 500 Internal Server Error , please try again later", + ) +} diff --git a/tools/check/check-errdoc.sh b/tools/check/check-errdoc.sh index 3ebf7269eca..5d77180a65f 100755 --- a/tools/check/check-errdoc.sh +++ b/tools/check/check-errdoc.sh @@ -17,5 +17,5 @@ set -euo pipefail cd -P . cp errors.toml /tmp/errors.toml.before -./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm +./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm,deployments diff -q errors.toml /tmp/errors.toml.before From 72c5fab5af0edfa13e5b1cdfa355a7153bac6c33 Mon Sep 17 00:00:00 2001 From: dsdashun Date: Wed, 26 Jan 2022 11:46:44 +0800 Subject: [PATCH 23/72] relay/meta(dm): fix potential data races after saving GTID (#4455) close pingcap/tiflow#4166 --- dm/relay/meta.go | 4 +-- dm/relay/meta_test.go | 63 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 65 insertions(+), 2 deletions(-) diff --git a/dm/relay/meta.go b/dm/relay/meta.go index c48f2c5ae2b..38e50110940 100644 --- a/dm/relay/meta.go +++ b/dm/relay/meta.go @@ -212,7 +212,7 @@ func (lm *LocalMeta) Save(pos mysql.Position, gset gtid.Set) error { lm.BinlogGTID = "" } else { lm.BinlogGTID = gset.String() - lm.gset = gset + lm.gset = gset.Clone() // need to clone and set, in order to avoid the local meta's gset and the input gset referencing the same object, causing contentions later } lm.dirty = true @@ -328,7 +328,7 @@ func (lm *LocalMeta) AddDir(serverUUID string, newPos *mysql.Position, newGTID g } if newGTID != nil { - lm.gset = newGTID + lm.gset = newGTID.Clone() // need to clone and set, in order to avoid the local meta's gset and the input newGTID referencing the same object, causing contentions later lm.BinlogGTID = newGTID.String() } // if newGTID == nil, keep GTID not changed diff --git a/dm/relay/meta_test.go b/dm/relay/meta_test.go index 56c25f69875..558e812b67d 100644 --- a/dm/relay/meta_test.go +++ b/dm/relay/meta_test.go @@ -14,6 +14,7 @@ package relay import ( + "fmt" "os" "path" "strings" @@ -234,3 +235,65 @@ func (r *testMetaSuite) TestLocalMeta(c *C) { currentDir := lm.Dir() c.Assert(strings.HasSuffix(currentDir, cs.uuidWithSuffix), IsTrue) } + +func (r *testMetaSuite) TestLocalMetaPotentialDataRace(c *C) { + var err error + lm := NewLocalMeta("mysql", "/FAKE_DIR") + uuidStr := "85ab69d1-b21f-11e6-9c5e-64006a8978d2" + initGSet, _ := gtid.ParserGTID("mysql", fmt.Sprintf("%s:1", uuidStr)) + lm.(*LocalMeta).currentUUID = uuidStr + err = lm.Save( + mysql.Position{Name: "mysql-bin.000001", Pos: 234}, + initGSet, + ) + c.Assert(err, IsNil) + + ch1 := make(chan error) + ch2 := make(chan error) + pendingCh := make(chan struct{}) + go func() { + <-pendingCh + var err error + defer func() { + ch1 <- err + }() + _, lastGTID := lm.GTID() + var theMGSet mysql.GTIDSet + for i := 2; i < 100; i++ { + theMGSet, err = mysql.ParseGTIDSet("mysql", fmt.Sprintf("%s:1-%d", uuidStr, i*10)) + if err != nil { + return + } + + err = lastGTID.Set(theMGSet) + if err != nil { + return + } + err = lm.Save( + mysql.Position{Name: fmt.Sprintf("mysql-bin.%06d", i), Pos: 123}, + lastGTID, + ) + if err != nil { + return + } + } + }() + var gtidString string + go func() { + <-pendingCh + var err error + defer func() { + ch2 <- err + }() + for i := 0; i < 100; i++ { + _, currentGTID := lm.GTID() + gtidString = currentGTID.String() + } + }() + close(pendingCh) + ch1Err := <-ch1 + ch2Err := <-ch2 + c.Assert(ch1Err, IsNil) + c.Assert(ch2Err, IsNil) + c.Logf("GTID string from the go routine: %s", gtidString) +} From 9d7ae48d3fee0ea02bfa9dd504e06166bc25b45e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 26 Jan 2022 14:34:45 +0800 Subject: [PATCH 24/72] *(dm): add `--start-time` in DM master (#4261) ref pingcap/tiflow#4106 --- dm/dm/config/task_cli_args.go | 16 + dm/dm/config/task_cli_args_test.go | 11 + dm/dm/ctl/common/config.go | 14 + dm/dm/ctl/common/config_test.go | 53 +++ dm/dm/ctl/ctl.go | 2 +- dm/dm/ctl/master/check_task.go | 12 +- dm/dm/ctl/master/start_task.go | 7 + dm/dm/master/server.go | 71 +++- dm/dm/master/server_test.go | 2 + dm/dm/pb/dmmaster.pb.go | 367 +++++++++++------- dm/dm/proto/dmmaster.proto | 4 +- dm/tests/dmctl_basic/check_list/check_task.sh | 7 + dm/tests/dmctl_basic/check_list/start_task.sh | 7 + dm/tests/dmctl_basic/run.sh | 2 + go.mod | 1 + go.sum | 2 + tools/check/go.sum | 1 + 17 files changed, 425 insertions(+), 154 deletions(-) create mode 100644 dm/dm/ctl/common/config_test.go diff --git a/dm/dm/config/task_cli_args.go b/dm/dm/config/task_cli_args.go index f561b8eebd1..9f1bcedf420 100644 --- a/dm/dm/config/task_cli_args.go +++ b/dm/dm/config/task_cli_args.go @@ -15,6 +15,9 @@ package config import ( "encoding/json" + "time" + + "github.com/pingcap/tiflow/dm/pkg/terror" ) // TaskCliArgs is the task command line arguments, these arguments have higher priority than the config file and @@ -37,3 +40,16 @@ func (t *TaskCliArgs) Decode(data []byte) error { err := json.Unmarshal(data, t) return err } + +// Verify checks if all fields are legal. +func (t *TaskCliArgs) Verify() error { + if t.StartTime == "" { + return nil + } + _, err := time.Parse("2006-01-02 15:04:05", t.StartTime) + if err == nil { + return nil + } + _, err = time.Parse("2006-01-02T15:04:05", t.StartTime) + return terror.Annotate(err, "error while parse start-time, expected in the format like '2006-01-02 15:04:05'") +} diff --git a/dm/dm/config/task_cli_args_test.go b/dm/dm/config/task_cli_args_test.go index 8bdd0f52399..731ffd4866d 100644 --- a/dm/dm/config/task_cli_args_test.go +++ b/dm/dm/config/task_cli_args_test.go @@ -43,3 +43,14 @@ func (t *testConfig) TestTaskCliArgsDowngrade(c *C) { c.Assert(afterDowngrade.Decode([]byte(data)), IsNil) c.Assert(afterDowngrade.StartTime, Equals, "123") } + +func (t *testConfig) TestTaskCliArgsVerify(c *C) { + empty := TaskCliArgs{} + c.Assert(empty.Verify(), IsNil) + rightStartTime := TaskCliArgs{StartTime: "2006-01-02T15:04:05"} + c.Assert(rightStartTime.Verify(), IsNil) + rightStartTime = TaskCliArgs{StartTime: "2006-01-02 15:04:05"} + c.Assert(rightStartTime.Verify(), IsNil) + wrongStartTime := TaskCliArgs{StartTime: "15:04:05"} + c.Assert(wrongStartTime.Verify(), NotNil) +} diff --git a/dm/dm/ctl/common/config.go b/dm/dm/ctl/common/config.go index a98cb59ed44..6d73fbfdf4c 100644 --- a/dm/dm/ctl/common/config.go +++ b/dm/dm/ctl/common/config.go @@ -21,7 +21,11 @@ import ( "strings" "time" + "github.com/google/shlex" + "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/BurntSushi/toml" @@ -215,3 +219,13 @@ func validateAddr(addr string) error { } return nil } + +// SplitArgsRespectQuote splits args by space, but won't split space inside single or double quotes. +func SplitArgsRespectQuote(line string) []string { + ret, err := shlex.Split(line) + if err != nil { + log.L().Error("split args error", zap.Error(err)) + return []string{line} + } + return ret +} diff --git a/dm/dm/ctl/common/config_test.go b/dm/dm/ctl/common/config_test.go new file mode 100644 index 00000000000..08a794804e9 --- /dev/null +++ b/dm/dm/ctl/common/config_test.go @@ -0,0 +1,53 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "testing" + + . "github.com/pingcap/check" +) + +func TestConfig(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testConfigSuite{}) + +type testConfigSuite struct{} + +func (t *testConfigSuite) TestInteractiveQuotes(c *C) { + cases := []struct { + input string + expected []string + }{ + {`123`, []string{`123`}}, + {`"123"`, []string{`123`}}, + {`'123'`, []string{`123`}}, + {`123 456`, []string{`123`, `456`}}, + {`'123 456'`, []string{`123 456`}}, + {`"123 456"`, []string{`123 456`}}, + {`"123 456" 789`, []string{`123 456`, `789`}}, + {`0 '123"456 789'`, []string{`0`, `123"456 789`}}, + {`0'123"456 789'`, []string{`0123"456 789`}}, + {`"123""456" 7 "89"`, []string{`123456`, `7`, `89`}}, + // return original string when failed to split + {`123"456`, []string{`123"456`}}, + } + + for _, ca := range cases { + got := SplitArgsRespectQuote(ca.input) + c.Assert(got, DeepEquals, ca.expected) + } +} diff --git a/dm/dm/ctl/ctl.go b/dm/dm/ctl/ctl.go index 94f40e1cf41..aad9b046b9f 100644 --- a/dm/dm/ctl/ctl.go +++ b/dm/dm/ctl/ctl.go @@ -154,7 +154,7 @@ func loop() error { continue } - args := strings.Fields(line) + args := common.SplitArgsRespectQuote(line) c, err := Start(args) if err != nil { fmt.Println("fail to run:", args) diff --git a/dm/dm/ctl/master/check_task.go b/dm/dm/ctl/master/check_task.go index b3536f66c37..52c9b2c78f2 100644 --- a/dm/dm/ctl/master/check_task.go +++ b/dm/dm/ctl/master/check_task.go @@ -34,6 +34,7 @@ func NewCheckTaskCmd() *cobra.Command { } cmd.Flags().Int64P("error", "e", common.DefaultErrorCnt, "max count of errors to display") cmd.Flags().Int64P("warn", "w", common.DefaultWarnCnt, "max count of warns to display") + cmd.Flags().String("start-time", "", "specify the start time of binlog replication, e.g. '2021-10-21 00:01:00' or 2021-10-21T00:01:00") return cmd } @@ -57,6 +58,10 @@ func checkTaskFunc(cmd *cobra.Command, _ []string) error { if err != nil { return err } + startTime, err := cmd.Flags().GetString("start-time") + if err != nil { + return err + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -67,9 +72,10 @@ func checkTaskFunc(cmd *cobra.Command, _ []string) error { ctx, "CheckTask", &pb.CheckTaskRequest{ - Task: string(content), - ErrCnt: errCnt, - WarnCnt: warnCnt, + Task: string(content), + ErrCnt: errCnt, + WarnCnt: warnCnt, + StartTime: startTime, }, &resp, ) diff --git a/dm/dm/ctl/master/start_task.go b/dm/dm/ctl/master/start_task.go index c748c7419e5..6d70bd41999 100644 --- a/dm/dm/ctl/master/start_task.go +++ b/dm/dm/ctl/master/start_task.go @@ -37,6 +37,7 @@ func NewStartTaskCmd() *cobra.Command { RunE: startTaskFunc, } cmd.Flags().BoolP("remove-meta", "", false, "whether to remove task's meta data") + cmd.Flags().String("start-time", "", "specify the start time of binlog replication, e.g. '2021-10-21 00:01:00' or 2021-10-21T00:01:00") return cmd } @@ -76,6 +77,11 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { common.PrintLinesf("error in parse `--remove-meta`") return err } + startTime, err := cmd.Flags().GetString("start-time") + if err != nil { + common.PrintLinesf("error in parse `--start-time`") + return err + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -89,6 +95,7 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { Task: string(content), Sources: sources, RemoveMeta: removeMeta, + StartTime: startTime, }, &resp, ) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index b9de171296b..14c13fb72fb 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -428,12 +428,23 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S } resp := &pb.StartTaskResponse{} - cfg, stCfgs, err := s.generateSubTask(ctx, req.Task) - if err != nil { - resp.Msg = err.Error() + respWithErr := func(err error) (*pb.StartTaskResponse, error) { + resp.Msg += err.Error() // nolint:nilerr return resp, nil } + + cliArgs := config.TaskCliArgs{ + StartTime: req.StartTime, + } + if err := cliArgs.Verify(); err != nil { + return respWithErr(err) + } + + cfg, stCfgs, err := s.generateSubTask(ctx, req.Task, &cliArgs) + if err != nil { + return respWithErr(err) + } msg, err := checker.CheckSyncConfigFunc(ctx, stCfgs, ctlcommon.DefaultErrorCnt, ctlcommon.DefaultWarnCnt) if err != nil { resp.Msg = terror.WithClass(err, terror.ClassDMMaster).Error() @@ -481,29 +492,35 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S // use same latch for remove-meta and start-task release, err3 = s.scheduler.AcquireSubtaskLatch(cfg.Name) if err3 != nil { - resp.Msg += terror.ErrSchedulerLatchInUse.Generate("RemoveMeta", cfg.Name).Error() - // nolint:nilerr - return resp, nil + return respWithErr(terror.ErrSchedulerLatchInUse.Generate("RemoveMeta", cfg.Name)) } defer release() latched = true if scm := s.scheduler.GetSubTaskCfgsByTask(cfg.Name); len(scm) > 0 { - resp.Msg += terror.Annotate(terror.ErrSchedulerSubTaskExist.Generate(cfg.Name, sources), - "while remove-meta is true").Error() - return resp, nil + return respWithErr(terror.Annotate(terror.ErrSchedulerSubTaskExist.Generate(cfg.Name, sources), + "while remove-meta is true")) } err = s.removeMetaData(ctx, cfg.Name, cfg.MetaSchema, cfg.TargetDB) if err != nil { - resp.Msg += terror.Annotate(err, "while removing metadata").Error() - return resp, nil + return respWithErr(terror.Annotate(err, "while removing metadata")) + } + } + + if req.StartTime == "" { + err = ha.DeleteAllTaskCliArgs(s.etcdClient, cfg.Name) + if err != nil { + return respWithErr(terror.Annotate(err, "while removing task command line arguments")) + } + } else { + err = ha.PutTaskCliArgs(s.etcdClient, cfg.Name, sources, cliArgs) + if err != nil { + return respWithErr(terror.Annotate(err, "while putting task command line arguments")) } } err = s.scheduler.AddSubTasks(latched, subtaskCfgPointersToInstances(stCfgs...)...) if err != nil { - resp.Msg += err.Error() - // nolint:nilerr - return resp, nil + return respWithErr(err) } if release != nil { @@ -627,8 +644,8 @@ func (s *Server) UpdateTask(ctx context.Context, req *pb.UpdateTaskRequest) (*pb return resp2, err2 } + cfg, stCfgs, err := s.generateSubTask(ctx, req.Task, nil) resp := &pb.UpdateTaskResponse{} - cfg, stCfgs, err := s.generateSubTask(ctx, req.Task) if err != nil { resp.Msg = err.Error() // nolint:nilerr @@ -1190,8 +1207,18 @@ func (s *Server) CheckTask(ctx context.Context, req *pb.CheckTaskRequest) (*pb.C return resp2, err2 } + cliArgs := config.TaskCliArgs{ + StartTime: req.StartTime, + } + if err := cliArgs.Verify(); err != nil { + // nolint:nilerr + return &pb.CheckTaskResponse{ + Result: false, + Msg: err.Error(), + }, nil + } resp := &pb.CheckTaskResponse{} - _, stCfgs, err := s.generateSubTask(ctx, req.Task) + _, stCfgs, err := s.generateSubTask(ctx, req.Task, nil) if err != nil { resp.Msg = err.Error() // nolint:nilerr @@ -1453,8 +1480,18 @@ func (s *Server) OperateLeader(ctx context.Context, req *pb.OperateLeaderRequest }, nil } -func (s *Server) generateSubTask(ctx context.Context, task string) (*config.TaskConfig, []*config.SubTaskConfig, error) { +func (s *Server) generateSubTask( + ctx context.Context, + task string, + cliArgs *config.TaskCliArgs, +) (*config.TaskConfig, []*config.SubTaskConfig, error) { cfg := config.NewTaskConfig() + // bypass the meta check by set any value. If start-time is specified, DM-worker will not use meta field. + if cliArgs != nil && cliArgs.StartTime != "" { + for _, inst := range cfg.MySQLInstances { + inst.Meta = &config.Meta{BinLogName: cliArgs.StartTime} + } + } err := cfg.Decode(task) if err != nil { return nil, nil, terror.WithClass(err, terror.ClassDMMaster) diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index f6e7fe9f370..73d79b41275 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -857,6 +857,7 @@ func (t *testMaster) TestStartTask(c *check.C) { defer ctrl.Finish() server := testDefaultMasterServer(c) + server.etcdClient = t.etcdTestCli sources, workers := defaultWorkerSource() // s.generateSubTask with error @@ -1138,6 +1139,7 @@ func (t *testMaster) TestOperateTask(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() server := testDefaultMasterServer(c) + server.etcdClient = t.etcdTestCli sources, workers := defaultWorkerSource() // test operate-task with invalid task name diff --git a/dm/dm/pb/dmmaster.pb.go b/dm/dm/pb/dmmaster.pb.go index 3ace56ccdef..790788ca74c 100644 --- a/dm/dm/pb/dmmaster.pb.go +++ b/dm/dm/pb/dmmaster.pb.go @@ -158,6 +158,7 @@ type StartTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` RemoveMeta bool `protobuf:"varint,3,opt,name=removeMeta,proto3" json:"removeMeta,omitempty"` + StartTime string `protobuf:"bytes,4,opt,name=startTime,proto3" json:"startTime,omitempty"` } func (m *StartTaskRequest) Reset() { *m = StartTaskRequest{} } @@ -214,6 +215,13 @@ func (m *StartTaskRequest) GetRemoveMeta() bool { return false } +func (m *StartTaskRequest) GetStartTime() string { + if m != nil { + return m.StartTime + } + return "" +} + type StartTaskResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -1229,9 +1237,10 @@ func (m *PurgeWorkerRelayResponse) GetSources() []*CommonWorkerResponse { } type CheckTaskRequest struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - ErrCnt int64 `protobuf:"varint,2,opt,name=errCnt,proto3" json:"errCnt,omitempty"` - WarnCnt int64 `protobuf:"varint,3,opt,name=warnCnt,proto3" json:"warnCnt,omitempty"` + Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` + ErrCnt int64 `protobuf:"varint,2,opt,name=errCnt,proto3" json:"errCnt,omitempty"` + WarnCnt int64 `protobuf:"varint,3,opt,name=warnCnt,proto3" json:"warnCnt,omitempty"` + StartTime string `protobuf:"bytes,4,opt,name=startTime,proto3" json:"startTime,omitempty"` } func (m *CheckTaskRequest) Reset() { *m = CheckTaskRequest{} } @@ -1288,6 +1297,13 @@ func (m *CheckTaskRequest) GetWarnCnt() int64 { return 0 } +func (m *CheckTaskRequest) GetStartTime() string { + if m != nil { + return m.StartTime + } + return "" +} + type CheckTaskResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -3190,137 +3206,138 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 2074 bytes of a gzipped FileDescriptorProto + // 2090 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xe3, 0xc6, - 0x11, 0x17, 0x25, 0x5b, 0x96, 0x47, 0xb6, 0x22, 0xaf, 0x65, 0x99, 0xc7, 0xf3, 0xe9, 0x1c, 0x36, + 0x11, 0x17, 0x25, 0x59, 0x96, 0x46, 0xb6, 0x22, 0xaf, 0x65, 0x99, 0xc7, 0xf3, 0xe9, 0x1c, 0x36, 0x09, 0x0c, 0xa3, 0x38, 0xe3, 0xdc, 0x3e, 0x05, 0x48, 0x81, 0x9c, 0x74, 0xb9, 0x18, 0xf5, 0xd5, - 0x29, 0xed, 0x4b, 0x1b, 0x14, 0x28, 0x42, 0x49, 0x2b, 0x59, 0x30, 0x45, 0xf2, 0x48, 0xca, 0xae, - 0x71, 0xc8, 0x4b, 0x3f, 0x40, 0xff, 0xa0, 0x40, 0xf3, 0xd8, 0x87, 0x7e, 0x93, 0x3e, 0xf5, 0x31, - 0x40, 0x5f, 0xfa, 0x58, 0xdc, 0xf5, 0x83, 0x14, 0x3b, 0xb3, 0x24, 0x97, 0x14, 0xe5, 0x56, 0x01, - 0xea, 0x37, 0xce, 0xcc, 0x6a, 0xe6, 0x37, 0x7f, 0x76, 0x76, 0x76, 0x05, 0x8d, 0xe1, 0x74, 0x6a, - 0x87, 0x11, 0x0f, 0x9e, 0xf8, 0x81, 0x17, 0x79, 0xac, 0xec, 0xf7, 0x8d, 0xc6, 0x70, 0x7a, 0xe3, - 0x05, 0x57, 0x31, 0xcf, 0xd8, 0x1b, 0x7b, 0xde, 0xd8, 0xe1, 0x47, 0xb6, 0x3f, 0x39, 0xb2, 0x5d, - 0xd7, 0x8b, 0xec, 0x68, 0xe2, 0xb9, 0x21, 0x49, 0xcd, 0xaf, 0xa1, 0x79, 0x1e, 0xd9, 0x41, 0x74, - 0x61, 0x87, 0x57, 0x16, 0x7f, 0x3d, 0xe3, 0x61, 0xc4, 0x18, 0xac, 0x44, 0x76, 0x78, 0xa5, 0x6b, - 0xfb, 0xda, 0xc1, 0xba, 0x85, 0xdf, 0x4c, 0x87, 0xb5, 0xd0, 0x9b, 0x05, 0x03, 0x1e, 0xea, 0xe5, - 0xfd, 0xca, 0xc1, 0xba, 0x15, 0x93, 0xac, 0x03, 0x10, 0xf0, 0xa9, 0x77, 0xcd, 0x5f, 0xf2, 0xc8, - 0xd6, 0x2b, 0xfb, 0xda, 0x41, 0xcd, 0x52, 0x38, 0xe6, 0x6b, 0xd8, 0x52, 0x2c, 0x84, 0xbe, 0xe7, - 0x86, 0x9c, 0xb5, 0xa1, 0x1a, 0xf0, 0x70, 0xe6, 0x44, 0x68, 0xa4, 0x66, 0x49, 0x8a, 0x35, 0xa1, - 0x32, 0x0d, 0xc7, 0x7a, 0x19, 0x2d, 0x8b, 0x4f, 0x76, 0x9c, 0x1a, 0xae, 0xec, 0x57, 0x0e, 0xea, - 0xc7, 0xfa, 0x13, 0xbf, 0xff, 0xa4, 0xeb, 0x4d, 0xa7, 0x9e, 0xfb, 0x0b, 0xf4, 0x33, 0x56, 0x9a, - 0x40, 0x32, 0x7f, 0x0d, 0xec, 0xcc, 0xe7, 0x81, 0x1d, 0x71, 0xd5, 0x2d, 0x03, 0xca, 0x9e, 0x8f, - 0xf6, 0x1a, 0xc7, 0x20, 0x94, 0x08, 0xe1, 0x99, 0x6f, 0x95, 0x3d, 0x5f, 0xb8, 0xec, 0xda, 0x53, - 0x2e, 0x0d, 0xe3, 0xb7, 0xea, 0x72, 0x25, 0xe3, 0xb2, 0xf9, 0x7b, 0x0d, 0xb6, 0x33, 0x06, 0xa4, - 0x57, 0x77, 0x59, 0x48, 0x3d, 0x2e, 0x17, 0x79, 0x5c, 0x29, 0xf4, 0x78, 0xe5, 0x7f, 0xf5, 0xf8, - 0x53, 0xd8, 0x7a, 0xe5, 0x0f, 0x73, 0x0e, 0x2f, 0x95, 0x47, 0x33, 0x00, 0xa6, 0xaa, 0xb8, 0x97, - 0x44, 0x7d, 0x06, 0xed, 0x9f, 0xcf, 0x78, 0x70, 0x7b, 0x1e, 0xd9, 0xd1, 0x2c, 0x3c, 0x9d, 0x84, - 0x91, 0x82, 0x1d, 0x13, 0xa2, 0x15, 0x27, 0x24, 0x87, 0xfd, 0x1a, 0x76, 0xe7, 0xf4, 0x2c, 0xed, - 0xc0, 0xd3, 0xbc, 0x03, 0xbb, 0xc2, 0x01, 0x45, 0xef, 0x3c, 0xfe, 0x2e, 0x6c, 0x9f, 0x5f, 0x7a, - 0x37, 0xbd, 0xde, 0xe9, 0xa9, 0x37, 0xb8, 0x0a, 0xbf, 0x5f, 0xe0, 0xff, 0xa2, 0xc1, 0x9a, 0xd4, - 0xc0, 0x1a, 0x50, 0x3e, 0xe9, 0xc9, 0xdf, 0x95, 0x4f, 0x7a, 0x89, 0xa6, 0xb2, 0xa2, 0x89, 0xc1, - 0xca, 0xd4, 0x1b, 0x72, 0x59, 0x32, 0xf8, 0xcd, 0x5a, 0xb0, 0xea, 0xdd, 0xb8, 0x3c, 0xd0, 0x57, - 0x90, 0x49, 0x84, 0x58, 0xd9, 0xeb, 0x9d, 0x86, 0xfa, 0x2a, 0x1a, 0xc4, 0x6f, 0x11, 0x8f, 0xf0, - 0xd6, 0x1d, 0xf0, 0xa1, 0x5e, 0x45, 0xae, 0xa4, 0x98, 0x01, 0xb5, 0x99, 0x2b, 0x25, 0x6b, 0x28, - 0x49, 0x68, 0x73, 0x00, 0xad, 0xac, 0x9b, 0x4b, 0xc7, 0xf6, 0x7d, 0x58, 0x75, 0xc4, 0x4f, 0x65, - 0x64, 0xeb, 0x22, 0xb2, 0x52, 0x9d, 0x45, 0x12, 0xd3, 0x81, 0xd6, 0x2b, 0x57, 0x7c, 0xc6, 0x7c, - 0x19, 0xcc, 0x7c, 0x48, 0x4c, 0xd8, 0x08, 0xb8, 0xef, 0xd8, 0x03, 0x7e, 0x86, 0x1e, 0x93, 0x95, - 0x0c, 0x8f, 0xed, 0x43, 0x7d, 0xe4, 0x05, 0x03, 0x6e, 0x61, 0x1b, 0x92, 0x4d, 0x49, 0x65, 0x99, - 0x9f, 0xc2, 0x4e, 0xce, 0xda, 0xb2, 0x3e, 0x99, 0x16, 0x3c, 0x90, 0x4d, 0x20, 0x2e, 0x6f, 0xc7, - 0xbe, 0x8d, 0x51, 0x3f, 0x54, 0x5a, 0x01, 0x7a, 0x8b, 0x52, 0xd9, 0x0b, 0x16, 0xd7, 0xc2, 0xb7, - 0x1a, 0x18, 0x45, 0x4a, 0x25, 0xb8, 0x3b, 0xb5, 0xfe, 0x7f, 0x3b, 0xcc, 0xb7, 0x1a, 0xec, 0x7e, - 0x31, 0x0b, 0xc6, 0x45, 0xce, 0x2a, 0xfe, 0x68, 0xd9, 0xc3, 0xc1, 0x80, 0xda, 0xc4, 0xb5, 0x07, - 0xd1, 0xe4, 0x9a, 0x4b, 0x54, 0x09, 0x8d, 0xb5, 0x3d, 0x99, 0x52, 0x76, 0x2a, 0x16, 0x7e, 0x8b, - 0xf5, 0xa3, 0x89, 0xc3, 0x71, 0xeb, 0x53, 0x29, 0x27, 0x34, 0x56, 0xee, 0xac, 0xdf, 0x9b, 0x04, - 0xfa, 0x2a, 0x4a, 0x24, 0x65, 0xfe, 0x06, 0xf4, 0x79, 0x60, 0xf7, 0xd2, 0xbe, 0x7e, 0x09, 0xcd, - 0xee, 0x25, 0x1f, 0x5c, 0xfd, 0xb7, 0xa6, 0xdb, 0x86, 0x2a, 0x0f, 0x82, 0xae, 0x4b, 0x99, 0xa9, - 0x58, 0x92, 0x12, 0x71, 0xbb, 0xb1, 0x03, 0x57, 0x08, 0x28, 0x08, 0x31, 0x69, 0x7e, 0x02, 0x5b, - 0x8a, 0xe6, 0xa5, 0x4b, 0xf3, 0x12, 0x5a, 0xb2, 0x8a, 0xce, 0x11, 0x6a, 0x0c, 0x6e, 0x4f, 0xa9, - 0x9f, 0x0d, 0xe1, 0x1f, 0x89, 0xd3, 0x02, 0x1a, 0x78, 0xee, 0x68, 0x32, 0x96, 0x55, 0x29, 0x29, - 0x91, 0x14, 0xf2, 0xf8, 0xa4, 0x27, 0x4f, 0xc2, 0x84, 0x36, 0x67, 0xb0, 0x93, 0xb3, 0x74, 0x2f, - 0x91, 0x7f, 0x0e, 0x3b, 0x16, 0x1f, 0x4f, 0xc4, 0xe8, 0x13, 0x2f, 0xb9, 0xf3, 0xdc, 0xb0, 0x87, - 0xc3, 0x80, 0x87, 0xa1, 0x34, 0x1b, 0x93, 0xe6, 0x33, 0x68, 0xe7, 0xd5, 0x2c, 0x1d, 0xeb, 0x9f, - 0x40, 0xeb, 0x6c, 0x34, 0x72, 0x26, 0x2e, 0x7f, 0xc9, 0xa7, 0xfd, 0x0c, 0x92, 0xe8, 0xd6, 0x4f, - 0x90, 0x88, 0xef, 0xa2, 0x31, 0x43, 0x74, 0xa2, 0xdc, 0xef, 0x97, 0x86, 0xf0, 0xe3, 0x24, 0xdd, - 0xa7, 0xdc, 0x1e, 0xa6, 0x10, 0xe6, 0xd2, 0x4d, 0x62, 0x4a, 0x37, 0x1a, 0xce, 0xfe, 0x6a, 0x69, - 0xc3, 0xbf, 0xd3, 0x00, 0x5e, 0xe2, 0x00, 0x7a, 0xe2, 0x8e, 0xbc, 0xc2, 0xe0, 0x1b, 0x50, 0x9b, - 0xa2, 0x5f, 0x27, 0x3d, 0xfc, 0xe5, 0x8a, 0x95, 0xd0, 0xe2, 0xd4, 0xb2, 0x9d, 0x49, 0xd2, 0xa0, - 0x89, 0x10, 0xbf, 0xf0, 0x39, 0x0f, 0x5e, 0x59, 0xa7, 0xd4, 0x9e, 0xd6, 0xad, 0x84, 0x16, 0xc3, - 0xe6, 0xc0, 0x99, 0x70, 0x37, 0x42, 0x29, 0x9d, 0x6b, 0x0a, 0xc7, 0xec, 0x03, 0x50, 0x22, 0x17, - 0xe2, 0x61, 0xb0, 0x22, 0xb2, 0x1f, 0xa7, 0x40, 0x7c, 0x0b, 0x1c, 0x61, 0x64, 0x8f, 0xe3, 0x23, - 0x95, 0x08, 0xec, 0x37, 0x58, 0x6e, 0xb2, 0x13, 0x49, 0xca, 0x3c, 0x85, 0xa6, 0x98, 0x30, 0x28, - 0x68, 0x94, 0xb3, 0x38, 0x34, 0x5a, 0x5a, 0xd5, 0x45, 0x13, 0x65, 0x6c, 0xbb, 0x92, 0xda, 0x36, - 0x7f, 0x46, 0xda, 0x28, 0x8a, 0x0b, 0xb5, 0x1d, 0xc0, 0x1a, 0x0d, 0xfa, 0x74, 0x62, 0xd4, 0x8f, - 0x1b, 0x22, 0x9d, 0x69, 0xe8, 0xad, 0x58, 0x1c, 0xeb, 0xa3, 0x28, 0xdc, 0xa5, 0x8f, 0x2e, 0x09, - 0x19, 0x7d, 0x69, 0xe8, 0xac, 0x58, 0x6c, 0xfe, 0x55, 0x83, 0x35, 0x52, 0x13, 0xb2, 0x27, 0x50, - 0x75, 0xd0, 0x6b, 0x54, 0x55, 0x3f, 0x6e, 0x61, 0x4d, 0xe5, 0x62, 0xf1, 0x79, 0xc9, 0x92, 0xab, - 0xc4, 0x7a, 0x82, 0x85, 0x51, 0x50, 0xd6, 0xab, 0xde, 0x8a, 0xf5, 0xb4, 0x4a, 0xac, 0x27, 0xb3, - 0x18, 0x21, 0x65, 0xbd, 0xea, 0x8d, 0x58, 0x4f, 0xab, 0x9e, 0xd5, 0xa0, 0x4a, 0xb5, 0x24, 0x2e, - 0x19, 0xa8, 0x37, 0xb3, 0x03, 0xdb, 0x19, 0xb8, 0xb5, 0x04, 0x56, 0x3b, 0x03, 0xab, 0x96, 0x98, - 0x6f, 0x67, 0xcc, 0xd7, 0x62, 0x33, 0xa2, 0x3c, 0x44, 0xfa, 0xe2, 0x6a, 0x24, 0xc2, 0xe4, 0xc0, - 0x54, 0x93, 0x4b, 0xb7, 0xbd, 0x0f, 0x61, 0x8d, 0xc0, 0x67, 0x86, 0x22, 0x19, 0x6a, 0x2b, 0x96, - 0x99, 0x7f, 0x2e, 0xa7, 0xbd, 0x7c, 0x70, 0xc9, 0xa7, 0xf6, 0xe2, 0x5e, 0x8e, 0xe2, 0xf4, 0x42, - 0x33, 0x37, 0x38, 0x2e, 0xbc, 0xd0, 0x88, 0x2d, 0x37, 0xb4, 0x23, 0xbb, 0x6f, 0x87, 0xc9, 0xb1, - 0x1b, 0xd3, 0xc2, 0xfb, 0xc8, 0xee, 0x3b, 0x5c, 0x9e, 0xba, 0x44, 0xe0, 0xe6, 0x40, 0x7b, 0x7a, - 0x55, 0x6e, 0x0e, 0xa4, 0xc4, 0xea, 0x91, 0x33, 0x0b, 0x2f, 0xf5, 0x35, 0xda, 0xd2, 0x48, 0x08, - 0x34, 0x62, 0x94, 0xd4, 0x6b, 0xc8, 0xc4, 0x6f, 0xb1, 0x95, 0x47, 0x81, 0x37, 0xa5, 0x63, 0x43, - 0x5f, 0xa7, 0x7b, 0x63, 0xca, 0x89, 0xe5, 0x17, 0x76, 0x30, 0xe6, 0x91, 0x0e, 0xa9, 0x9c, 0x38, - 0xea, 0xc9, 0x23, 0xe3, 0x72, 0x2f, 0x27, 0xcf, 0x21, 0xb4, 0x5e, 0xf0, 0xe8, 0x7c, 0xd6, 0x17, - 0x47, 0x73, 0x77, 0x34, 0xbe, 0xe3, 0xe0, 0x31, 0x5f, 0xc1, 0x4e, 0x6e, 0xed, 0xd2, 0x10, 0x19, - 0xac, 0x0c, 0x46, 0xe3, 0x38, 0x61, 0xf8, 0x6d, 0xf6, 0x60, 0xf3, 0x05, 0x8f, 0x14, 0xdb, 0x8f, - 0x95, 0xa3, 0x46, 0x0e, 0x86, 0xdd, 0xd1, 0xf8, 0xe2, 0xd6, 0xe7, 0x77, 0x9c, 0x3b, 0xa7, 0xd0, - 0x88, 0xb5, 0x2c, 0x8d, 0xaa, 0x09, 0x95, 0xc1, 0x28, 0x19, 0x29, 0x07, 0xa3, 0xb1, 0xb9, 0x03, - 0xdb, 0x2f, 0xb8, 0xdc, 0xd7, 0x29, 0x32, 0xf3, 0x00, 0xa3, 0xa5, 0xb0, 0xa5, 0x29, 0xa9, 0x40, - 0x4b, 0x15, 0xfc, 0x51, 0x03, 0xf6, 0xb9, 0xed, 0x0e, 0x1d, 0xfe, 0x3c, 0x08, 0xbc, 0x60, 0xe1, - 0x1c, 0x8d, 0xd2, 0xef, 0x55, 0xe4, 0x7b, 0xb0, 0xde, 0x9f, 0xb8, 0x8e, 0x37, 0xfe, 0xc2, 0x0b, - 0x65, 0x95, 0xa7, 0x0c, 0x2c, 0xd1, 0xd7, 0x4e, 0x72, 0x57, 0x12, 0xdf, 0x66, 0x08, 0xdb, 0x19, - 0x48, 0xf7, 0x52, 0x60, 0x2f, 0x60, 0xe7, 0x22, 0xb0, 0xdd, 0x70, 0xc4, 0x83, 0xec, 0xf0, 0x96, - 0x9e, 0x47, 0x9a, 0x7a, 0x1e, 0x29, 0x6d, 0x8b, 0x2c, 0x4b, 0x4a, 0x0c, 0x37, 0x79, 0x45, 0x4b, - 0x1f, 0xf0, 0xc3, 0xe4, 0xa1, 0x23, 0x33, 0xf0, 0x3f, 0x52, 0xb2, 0xb2, 0xa9, 0xdc, 0x43, 0xbe, - 0x3c, 0x8e, 0x07, 0x49, 0x89, 0xb4, 0xbc, 0x00, 0x29, 0xa5, 0x26, 0x46, 0x1a, 0x25, 0x2d, 0xee, - 0x1e, 0xa7, 0xf7, 0xc3, 0x3e, 0xd4, 0xe2, 0xf1, 0x97, 0x6d, 0xc3, 0x7b, 0x27, 0xee, 0xb5, 0xed, - 0x4c, 0x86, 0x31, 0xab, 0x59, 0x62, 0xef, 0x41, 0x1d, 0x5f, 0xae, 0x88, 0xd5, 0xd4, 0x58, 0x13, - 0x36, 0xe8, 0x89, 0x44, 0x72, 0xca, 0xac, 0x01, 0x70, 0x1e, 0x79, 0xbe, 0xa4, 0x2b, 0x48, 0x5f, - 0x7a, 0x37, 0x92, 0x5e, 0x39, 0xfc, 0x29, 0xd4, 0xe2, 0x99, 0x4b, 0xb1, 0x11, 0xb3, 0x9a, 0x25, - 0xb6, 0x05, 0x9b, 0xcf, 0xaf, 0x27, 0x83, 0x28, 0x61, 0x69, 0x6c, 0x17, 0xb6, 0xbb, 0xb6, 0x3b, - 0xe0, 0x4e, 0x56, 0x50, 0x3e, 0x74, 0x61, 0x4d, 0x6e, 0x6b, 0x01, 0x4d, 0xea, 0x12, 0x64, 0xb3, - 0xc4, 0x36, 0xa0, 0x26, 0x9a, 0x0c, 0x52, 0x9a, 0x80, 0x41, 0x7b, 0x0e, 0x69, 0x84, 0x49, 0x51, - 0x40, 0x9a, 0x60, 0x22, 0x44, 0xa4, 0x57, 0x58, 0x0b, 0x9a, 0xf8, 0x6b, 0x3e, 0xf5, 0x1d, 0x3b, - 0x22, 0xee, 0xea, 0x61, 0x0f, 0xd6, 0x93, 0xbc, 0x8a, 0x25, 0xd2, 0x62, 0xc2, 0x6b, 0x96, 0x44, - 0x44, 0x30, 0x44, 0xc8, 0xfb, 0xf2, 0xb8, 0xa9, 0x51, 0xd0, 0x3c, 0x3f, 0x66, 0x94, 0x8f, 0xff, - 0xd6, 0x80, 0x2a, 0x81, 0x61, 0x5f, 0xc1, 0x7a, 0xf2, 0x14, 0xc8, 0xf0, 0x70, 0xcf, 0xbf, 0x3d, - 0x1a, 0x3b, 0x39, 0x2e, 0x25, 0xcd, 0x7c, 0xfc, 0xdb, 0x7f, 0xfc, 0xfb, 0x4f, 0xe5, 0x07, 0x66, - 0xeb, 0xc8, 0xf6, 0x27, 0xe1, 0xd1, 0xf5, 0x53, 0xdb, 0xf1, 0x2f, 0xed, 0xa7, 0x47, 0x62, 0xcb, - 0x87, 0x1f, 0x6b, 0x87, 0x6c, 0x04, 0x75, 0xe5, 0x45, 0x8e, 0xb5, 0x85, 0x9a, 0xf9, 0x37, 0x40, - 0x63, 0x77, 0x8e, 0x2f, 0x0d, 0x7c, 0x84, 0x06, 0xf6, 0x8d, 0x87, 0x45, 0x06, 0x8e, 0xde, 0x88, - 0x8e, 0xf9, 0x8d, 0xb0, 0xf3, 0x09, 0x40, 0xfa, 0x4a, 0xc6, 0x10, 0xed, 0xdc, 0xc3, 0x9b, 0xd1, - 0xce, 0xb3, 0xa5, 0x91, 0x12, 0x73, 0xa0, 0xae, 0x3c, 0x28, 0x31, 0x23, 0xf7, 0xc2, 0xa4, 0xbc, - 0x80, 0x19, 0x0f, 0x0b, 0x65, 0x52, 0xd3, 0x07, 0x08, 0xb7, 0xc3, 0xf6, 0x72, 0x70, 0x43, 0x5c, - 0x2a, 0xf1, 0xb2, 0x2e, 0x6c, 0xa8, 0xef, 0x36, 0x0c, 0xbd, 0x2f, 0x78, 0xb0, 0x32, 0xf4, 0x79, - 0x41, 0x02, 0xf9, 0x33, 0xd8, 0xcc, 0xbc, 0x94, 0x30, 0x5c, 0x5c, 0xf4, 0x54, 0x63, 0x3c, 0x28, - 0x90, 0x24, 0x7a, 0xbe, 0x82, 0xf6, 0xfc, 0xcb, 0x06, 0x46, 0xf1, 0x91, 0x92, 0x94, 0xf9, 0xd7, - 0x05, 0xa3, 0xb3, 0x48, 0x9c, 0xa8, 0x3e, 0x83, 0x66, 0xfe, 0x05, 0x80, 0x61, 0xf8, 0x16, 0x3c, - 0x58, 0x18, 0x7b, 0xc5, 0xc2, 0x44, 0xe1, 0xc7, 0xb0, 0x9e, 0x5c, 0xbf, 0xa9, 0x50, 0xf3, 0xf7, - 0x7c, 0x2a, 0xd4, 0xb9, 0x3b, 0xba, 0x59, 0x62, 0x63, 0xd8, 0xcc, 0xdc, 0x88, 0x29, 0x5e, 0x45, - 0xd7, 0x71, 0x8a, 0x57, 0xe1, 0xf5, 0xd9, 0x7c, 0x1f, 0x13, 0xfc, 0xd0, 0x68, 0xe7, 0x13, 0x4c, - 0xcd, 0x4b, 0x94, 0xe2, 0x09, 0x34, 0xb2, 0x97, 0x57, 0xf6, 0x80, 0x5a, 0x71, 0xc1, 0xbd, 0xd8, - 0x30, 0x8a, 0x44, 0x09, 0xe6, 0x00, 0x36, 0x33, 0x77, 0x50, 0x89, 0xb9, 0xe0, 0x5a, 0x2b, 0x31, - 0x17, 0x5d, 0x58, 0xcd, 0x1f, 0x22, 0xe6, 0x8f, 0x0e, 0x3f, 0xc8, 0x61, 0x96, 0xa3, 0xec, 0xd1, - 0x1b, 0x31, 0x8b, 0x7c, 0x13, 0x17, 0xe7, 0x55, 0x12, 0x27, 0x6a, 0x71, 0x99, 0x38, 0x65, 0xee, - 0xb1, 0x99, 0x38, 0x65, 0xef, 0xaa, 0xe6, 0x87, 0x68, 0xf3, 0xb1, 0x61, 0xe4, 0x6c, 0xd2, 0xa8, - 0x7f, 0xf4, 0xc6, 0xf3, 0x71, 0xdb, 0xfe, 0x0a, 0x20, 0x1d, 0xd6, 0x69, 0xdb, 0xce, 0xdd, 0x17, - 0x68, 0xdb, 0xce, 0xcf, 0xf4, 0x66, 0x07, 0x6d, 0xe8, 0xac, 0x5d, 0xec, 0x17, 0x1b, 0xa5, 0x19, - 0xa7, 0x21, 0x38, 0x93, 0x71, 0x75, 0x68, 0xcf, 0x66, 0x3c, 0x33, 0xb6, 0x9a, 0xfb, 0x68, 0xc5, - 0x30, 0x76, 0xf2, 0x19, 0xc7, 0x65, 0xc2, 0x09, 0x07, 0xe7, 0xbe, 0x74, 0x9c, 0x24, 0x3b, 0x45, - 0xd3, 0x28, 0xd9, 0x29, 0x9c, 0x3d, 0xe3, 0x4e, 0xc7, 0x3a, 0x79, 0x3b, 0xb3, 0xbe, 0xda, 0xec, - 0xd8, 0x05, 0x54, 0x69, 0x3e, 0x64, 0x5b, 0x52, 0x99, 0xa2, 0x9f, 0xa9, 0x2c, 0xa9, 0xf8, 0x07, - 0xa8, 0xf8, 0x11, 0xbb, 0xab, 0x85, 0xb2, 0xaf, 0xa1, 0xae, 0x8c, 0x54, 0xd4, 0xa7, 0xe7, 0xc7, - 0x3e, 0xea, 0xd3, 0x05, 0xb3, 0xd7, 0xc2, 0x28, 0x71, 0xb1, 0x0a, 0xb7, 0x45, 0x17, 0x36, 0xd4, - 0x91, 0x93, 0x9a, 0x5e, 0xc1, 0x6c, 0x6a, 0xe8, 0xf3, 0x82, 0x64, 0x43, 0x9c, 0x40, 0x23, 0x3b, - 0x3b, 0xd1, 0xde, 0x2a, 0x1c, 0xcc, 0x68, 0x6f, 0x15, 0x8f, 0x5a, 0x66, 0x49, 0xe0, 0x51, 0x87, - 0x1b, 0xa6, 0x1e, 0x41, 0x99, 0xa6, 0xa4, 0xcf, 0x0b, 0x62, 0x25, 0xcf, 0xf4, 0xbf, 0xbf, 0xed, - 0x68, 0xdf, 0xbd, 0xed, 0x68, 0xff, 0x7a, 0xdb, 0xd1, 0xfe, 0xf0, 0xae, 0x53, 0xfa, 0xee, 0x5d, - 0xa7, 0xf4, 0xcf, 0x77, 0x9d, 0x52, 0xbf, 0x8a, 0xff, 0xe3, 0xfd, 0xe8, 0x3f, 0x01, 0x00, 0x00, - 0xff, 0xff, 0xa2, 0xd2, 0xac, 0xe5, 0x0b, 0x1c, 0x00, 0x00, + 0x29, 0xed, 0x4b, 0x11, 0x14, 0x28, 0x4a, 0x49, 0x2b, 0x59, 0x30, 0x45, 0xf2, 0x48, 0xca, 0xae, + 0x71, 0x48, 0x1f, 0xfa, 0x01, 0xfa, 0x07, 0x05, 0x9a, 0xc7, 0x3e, 0xf4, 0x9b, 0xf4, 0xa9, 0x8f, + 0x01, 0xfa, 0xd2, 0xc7, 0xe2, 0xae, 0x1f, 0xa4, 0xd8, 0xd9, 0x5d, 0x72, 0xf9, 0x47, 0x4e, 0x15, + 0xa0, 0x7e, 0xe3, 0xcc, 0xac, 0x66, 0x7e, 0xf3, 0x67, 0x67, 0x67, 0x57, 0xd0, 0x1a, 0xcf, 0xe7, + 0x76, 0x18, 0xd1, 0xe0, 0x89, 0x1f, 0x78, 0x91, 0x47, 0xca, 0xfe, 0xd0, 0x68, 0x8d, 0xe7, 0x37, + 0x5e, 0x70, 0x25, 0x79, 0xc6, 0xde, 0xd4, 0xf3, 0xa6, 0x0e, 0x3d, 0xb2, 0xfd, 0xd9, 0x91, 0xed, + 0xba, 0x5e, 0x64, 0x47, 0x33, 0xcf, 0x0d, 0xb9, 0xd4, 0xfc, 0x2d, 0xb4, 0xcf, 0x23, 0x3b, 0x88, + 0x2e, 0xec, 0xf0, 0xca, 0xa2, 0xaf, 0x17, 0x34, 0x8c, 0x08, 0x81, 0x6a, 0x64, 0x87, 0x57, 0xba, + 0xb6, 0xaf, 0x1d, 0x34, 0x2c, 0xfc, 0x26, 0x3a, 0xac, 0x87, 0xde, 0x22, 0x18, 0xd1, 0x50, 0x2f, + 0xef, 0x57, 0x0e, 0x1a, 0x96, 0x24, 0x49, 0x0f, 0x20, 0xa0, 0x73, 0xef, 0x9a, 0xbe, 0xa4, 0x91, + 0xad, 0x57, 0xf6, 0xb5, 0x83, 0xba, 0xa5, 0x70, 0xc8, 0x1e, 0x34, 0x42, 0xb4, 0x30, 0x9b, 0x53, + 0xbd, 0x8a, 0x2a, 0x13, 0x86, 0xf9, 0x1a, 0xb6, 0x14, 0xfb, 0xa1, 0xef, 0xb9, 0x21, 0x25, 0x5d, + 0xa8, 0x05, 0x34, 0x5c, 0x38, 0x11, 0x42, 0xa8, 0x5b, 0x82, 0x22, 0x6d, 0xa8, 0xcc, 0xc3, 0xa9, + 0x5e, 0x46, 0x25, 0xec, 0x93, 0x1c, 0x27, 0xb0, 0x2a, 0xfb, 0x95, 0x83, 0xe6, 0xb1, 0xfe, 0xc4, + 0x1f, 0x3e, 0xe9, 0x7b, 0xf3, 0xb9, 0xe7, 0xfe, 0x02, 0xa3, 0x20, 0x95, 0xc6, 0x80, 0xcd, 0x5f, + 0x01, 0x39, 0xf3, 0x69, 0x60, 0x47, 0x54, 0x75, 0xda, 0x80, 0xb2, 0xe7, 0xa3, 0xbd, 0xd6, 0x31, + 0x30, 0x25, 0x4c, 0x78, 0xe6, 0x5b, 0x65, 0xcf, 0x67, 0x01, 0x71, 0xed, 0x39, 0x15, 0x86, 0xf1, + 0x5b, 0x0d, 0x48, 0x25, 0x15, 0x10, 0xf3, 0x0f, 0x1a, 0x6c, 0xa7, 0x0c, 0x08, 0xaf, 0xee, 0xb2, + 0x90, 0x78, 0x5c, 0x2e, 0xf2, 0xb8, 0x52, 0xe8, 0x71, 0xf5, 0x7f, 0xf5, 0xf8, 0x53, 0xd8, 0x7a, + 0xe5, 0x8f, 0x33, 0x0e, 0xaf, 0x94, 0x65, 0x33, 0x00, 0xa2, 0xaa, 0xb8, 0x97, 0x44, 0x7d, 0x06, + 0xdd, 0x9f, 0x2f, 0x68, 0x70, 0x7b, 0x1e, 0xd9, 0xd1, 0x22, 0x3c, 0x9d, 0x85, 0x91, 0x82, 0x1d, + 0x13, 0xa2, 0x15, 0x27, 0x24, 0x83, 0xfd, 0x1a, 0x76, 0x73, 0x7a, 0x56, 0x76, 0xe0, 0x69, 0xd6, + 0x81, 0x5d, 0xe6, 0x80, 0xa2, 0x37, 0x8f, 0xbf, 0x0f, 0xdb, 0xe7, 0x97, 0xde, 0xcd, 0x60, 0x70, + 0x7a, 0xea, 0x8d, 0xae, 0xc2, 0xef, 0x17, 0xf8, 0xbf, 0x6a, 0xb0, 0x2e, 0x34, 0x90, 0x16, 0x94, + 0x4f, 0x06, 0xe2, 0x77, 0xe5, 0x93, 0x41, 0xac, 0xa9, 0xac, 0x68, 0x22, 0x50, 0x9d, 0x7b, 0x63, + 0x2a, 0x4a, 0x06, 0xbf, 0x49, 0x07, 0xd6, 0xbc, 0x1b, 0x97, 0x06, 0x62, 0xfb, 0x71, 0x82, 0xad, + 0x1c, 0x0c, 0x4e, 0x43, 0x7d, 0x0d, 0x0d, 0xe2, 0x37, 0x8b, 0x47, 0x78, 0xeb, 0x8e, 0xe8, 0x58, + 0xaf, 0x21, 0x57, 0x50, 0xc4, 0x80, 0xfa, 0xc2, 0x15, 0x92, 0x75, 0x94, 0xc4, 0xb4, 0x39, 0x82, + 0x4e, 0xda, 0xcd, 0x95, 0x63, 0xfb, 0x3e, 0xac, 0x39, 0xec, 0xa7, 0x22, 0xb2, 0x4d, 0x16, 0x59, + 0xa1, 0xce, 0xe2, 0x12, 0xd3, 0x81, 0xce, 0x2b, 0x97, 0x7d, 0x4a, 0xbe, 0x08, 0x66, 0x36, 0x24, + 0x26, 0x6c, 0x04, 0xd4, 0x77, 0xec, 0x11, 0x3d, 0x43, 0x8f, 0xb9, 0x95, 0x14, 0x8f, 0xec, 0x43, + 0x73, 0xe2, 0x05, 0x23, 0x6a, 0x61, 0x93, 0x12, 0x2d, 0x4b, 0x65, 0x99, 0x9f, 0xc2, 0x4e, 0xc6, + 0xda, 0xaa, 0x3e, 0x99, 0x16, 0x3c, 0x10, 0x4d, 0x40, 0x96, 0xb7, 0x63, 0xdf, 0x4a, 0xd4, 0x0f, + 0x95, 0x56, 0x80, 0xde, 0xa2, 0x54, 0xf4, 0x82, 0xe5, 0xb5, 0xf0, 0x8d, 0x06, 0x46, 0x91, 0x52, + 0x01, 0xee, 0x4e, 0xad, 0xff, 0xdf, 0x0e, 0xf3, 0x8d, 0x06, 0xbb, 0x5f, 0x2c, 0x82, 0x69, 0x91, + 0xb3, 0x8a, 0x3f, 0x5a, 0xfa, 0xe8, 0x30, 0xa0, 0x3e, 0x73, 0xed, 0x51, 0x34, 0xbb, 0xa6, 0x02, + 0x55, 0x4c, 0x63, 0x6d, 0xb3, 0x13, 0x83, 0x01, 0xab, 0x58, 0xf8, 0xcd, 0xd6, 0x4f, 0x66, 0x0e, + 0xc5, 0xad, 0xcf, 0x4b, 0x39, 0xa6, 0xb1, 0x72, 0x17, 0xc3, 0xc1, 0x2c, 0xd0, 0xd7, 0x50, 0x22, + 0x28, 0xf3, 0x37, 0xa0, 0xe7, 0x81, 0xdd, 0x4b, 0xfb, 0xba, 0x86, 0x76, 0xff, 0x92, 0x8e, 0xae, + 0xbe, 0xab, 0xe9, 0x76, 0xa1, 0x46, 0x83, 0xa0, 0xef, 0xf2, 0xcc, 0x54, 0x2c, 0x41, 0xb1, 0xb8, + 0xdd, 0xd8, 0x81, 0xcb, 0x04, 0x3c, 0x08, 0x92, 0xfc, 0x8e, 0x23, 0xf5, 0x13, 0xd8, 0x52, 0xec, + 0xae, 0x5c, 0xb8, 0x97, 0xd0, 0x11, 0x35, 0x76, 0x8e, 0x8e, 0x48, 0xe8, 0x7b, 0x4a, 0x75, 0x6d, + 0x30, 0xef, 0xb9, 0x38, 0x29, 0xaf, 0x91, 0xe7, 0x4e, 0x66, 0x53, 0x51, 0xb3, 0x82, 0x62, 0x29, + 0xe3, 0xf1, 0x38, 0x19, 0x88, 0x73, 0x32, 0xa6, 0xcd, 0x05, 0xec, 0x64, 0x2c, 0xdd, 0x4b, 0x5e, + 0x9e, 0xc3, 0x8e, 0x45, 0xa7, 0x33, 0x36, 0x36, 0xc9, 0x25, 0x77, 0x9e, 0x2a, 0xf6, 0x78, 0x1c, + 0xd0, 0x30, 0x14, 0x66, 0x25, 0x69, 0x3e, 0x83, 0x6e, 0x56, 0xcd, 0xca, 0xb1, 0xfe, 0x09, 0x74, + 0xce, 0x26, 0x13, 0x67, 0xe6, 0xd2, 0x97, 0x74, 0x3e, 0x4c, 0x21, 0x89, 0x6e, 0xfd, 0x18, 0x09, + 0xfb, 0x2e, 0x1a, 0x42, 0x58, 0x9f, 0xca, 0xfc, 0x7e, 0x65, 0x08, 0x3f, 0x8e, 0xd3, 0x7d, 0x4a, + 0xed, 0x71, 0x02, 0x21, 0x97, 0x6e, 0x2e, 0xe6, 0xe9, 0x46, 0xc3, 0xe9, 0x5f, 0xad, 0x6c, 0xf8, + 0xf7, 0x1a, 0xc0, 0x4b, 0x1c, 0x5e, 0x4f, 0xdc, 0x89, 0x57, 0x18, 0x7c, 0x03, 0xea, 0x73, 0xf4, + 0xeb, 0x64, 0x80, 0xbf, 0xac, 0x5a, 0x31, 0xcd, 0xce, 0x34, 0xdb, 0x99, 0xc5, 0xed, 0x9b, 0x13, + 0xec, 0x17, 0x3e, 0xa5, 0xc1, 0x2b, 0xeb, 0x94, 0x37, 0xaf, 0x86, 0x15, 0xd3, 0x6c, 0x50, 0x1d, + 0x39, 0x33, 0xea, 0x46, 0x28, 0xe5, 0xa7, 0x9e, 0xc2, 0x31, 0x87, 0x00, 0x3c, 0x91, 0x4b, 0xf1, + 0x10, 0xa8, 0xb2, 0xec, 0xcb, 0x14, 0xb0, 0x6f, 0x86, 0x23, 0x8c, 0xec, 0xa9, 0x3c, 0x70, 0x39, + 0x81, 0xdd, 0x08, 0xcb, 0x4d, 0x6c, 0x4f, 0x41, 0x99, 0xa7, 0xd0, 0x66, 0xf3, 0x07, 0x0f, 0x1a, + 0xcf, 0x99, 0x0c, 0x8d, 0x96, 0x54, 0x75, 0xd1, 0xbc, 0x29, 0x6d, 0x57, 0x12, 0xdb, 0xe6, 0xcf, + 0xb8, 0x36, 0x1e, 0xc5, 0xa5, 0xda, 0x0e, 0x60, 0x9d, 0x5f, 0x12, 0xf8, 0x79, 0xd2, 0x3c, 0x6e, + 0xb1, 0x74, 0x26, 0xa1, 0xb7, 0xa4, 0x58, 0xea, 0xe3, 0x51, 0xb8, 0x4b, 0x1f, 0xbf, 0x60, 0xa4, + 0xf4, 0x25, 0xa1, 0xb3, 0xa4, 0xd8, 0xfc, 0x9b, 0x06, 0xeb, 0x5c, 0x4d, 0x48, 0x9e, 0x40, 0xcd, + 0x41, 0xaf, 0x51, 0x55, 0xf3, 0xb8, 0x83, 0x35, 0x95, 0x89, 0xc5, 0xe7, 0x25, 0x4b, 0xac, 0x62, + 0xeb, 0x39, 0x2c, 0x8c, 0x82, 0xb2, 0x5e, 0xf5, 0x96, 0xad, 0xe7, 0xab, 0xd8, 0x7a, 0x6e, 0x16, + 0x23, 0xa4, 0xac, 0x57, 0xbd, 0x61, 0xeb, 0xf9, 0xaa, 0x67, 0x75, 0xa8, 0xf1, 0x5a, 0x62, 0x57, + 0x10, 0xd4, 0x9b, 0xda, 0x81, 0xdd, 0x14, 0xdc, 0x7a, 0x0c, 0xab, 0x9b, 0x82, 0x55, 0x8f, 0xcd, + 0x77, 0x53, 0xe6, 0xeb, 0xd2, 0x0c, 0x2b, 0x0f, 0x96, 0x3e, 0x59, 0x8d, 0x9c, 0x30, 0x29, 0x10, + 0xd5, 0xe4, 0xca, 0x6d, 0xef, 0x43, 0x58, 0xe7, 0xe0, 0x53, 0x23, 0x93, 0x08, 0xb5, 0x25, 0x65, + 0xe6, 0x5f, 0xca, 0x49, 0x2f, 0x1f, 0x5d, 0xd2, 0xb9, 0xbd, 0xbc, 0x97, 0xa3, 0x38, 0xb9, 0xee, + 0xe4, 0xc6, 0xca, 0xa5, 0xd7, 0x1d, 0xb6, 0xe5, 0xc6, 0x76, 0x64, 0x0f, 0xed, 0x30, 0x3e, 0x94, + 0x25, 0xcd, 0xbc, 0x8f, 0xec, 0xa1, 0x43, 0xc5, 0x99, 0xcc, 0x09, 0xdc, 0x1c, 0x68, 0x4f, 0xaf, + 0x89, 0xcd, 0x81, 0x14, 0x5b, 0x3d, 0x71, 0x16, 0xe1, 0xa5, 0xbe, 0xce, 0xb7, 0x34, 0x12, 0x0c, + 0x0d, 0x1b, 0x34, 0xf5, 0x3a, 0x32, 0xf1, 0x9b, 0x6d, 0xe5, 0x49, 0xe0, 0xcd, 0xf9, 0xb1, 0xa1, + 0x37, 0xf8, 0x9d, 0x33, 0xe1, 0x48, 0xf9, 0x85, 0x1d, 0x4c, 0x69, 0xa4, 0x43, 0x22, 0xe7, 0x1c, + 0xf5, 0xe4, 0x11, 0x71, 0xb9, 0x97, 0x93, 0xe7, 0x10, 0x3a, 0x2f, 0x68, 0x74, 0xbe, 0x18, 0xb2, + 0xa3, 0xb9, 0x3f, 0x99, 0xde, 0x71, 0xf0, 0x98, 0xaf, 0x60, 0x27, 0xb3, 0x76, 0x65, 0x88, 0x04, + 0xaa, 0xa3, 0xc9, 0x54, 0x26, 0x0c, 0xbf, 0xcd, 0x01, 0x6c, 0xbe, 0xa0, 0x91, 0x62, 0xfb, 0xb1, + 0x72, 0xd4, 0x88, 0xb1, 0xb1, 0x3f, 0x99, 0x5e, 0xdc, 0xfa, 0xf4, 0x8e, 0x73, 0xe7, 0x14, 0x5a, + 0x52, 0xcb, 0xca, 0xa8, 0xda, 0x50, 0x19, 0x4d, 0xe2, 0x81, 0x73, 0x34, 0x99, 0x9a, 0x3b, 0xb0, + 0xfd, 0x82, 0x8a, 0x7d, 0x9d, 0x20, 0x33, 0x0f, 0x30, 0x5a, 0x0a, 0x5b, 0x98, 0x12, 0x0a, 0xb4, + 0x44, 0xc1, 0x9f, 0x34, 0x20, 0x9f, 0xdb, 0xee, 0xd8, 0xa1, 0xcf, 0x83, 0xc0, 0x0b, 0x96, 0x4e, + 0xd9, 0x28, 0xfd, 0x5e, 0x45, 0xbe, 0x07, 0x8d, 0xe1, 0xcc, 0x75, 0xbc, 0xe9, 0x17, 0x5e, 0x28, + 0x27, 0xae, 0x98, 0x81, 0x25, 0xfa, 0xda, 0x89, 0x6f, 0x52, 0xec, 0xdb, 0x0c, 0x61, 0x3b, 0x05, + 0xe9, 0x5e, 0x0a, 0xec, 0x05, 0xec, 0x5c, 0x04, 0xb6, 0x1b, 0x4e, 0x68, 0x90, 0x1e, 0xde, 0x92, + 0xf3, 0x48, 0x53, 0xcf, 0x23, 0xa5, 0x6d, 0x71, 0xcb, 0x82, 0x62, 0xc3, 0x4d, 0x56, 0xd1, 0xca, + 0x07, 0xfc, 0x38, 0x7e, 0x06, 0x49, 0x5d, 0x07, 0x1e, 0x29, 0x59, 0xd9, 0x54, 0x6e, 0x29, 0x5f, + 0x1e, 0xcb, 0x41, 0x52, 0x20, 0x2d, 0x2f, 0x41, 0xca, 0x53, 0x23, 0x91, 0x46, 0x71, 0x8b, 0xbb, + 0xc7, 0xd9, 0xfe, 0x70, 0x08, 0x75, 0x39, 0xfe, 0x92, 0x6d, 0x78, 0xef, 0xc4, 0xbd, 0xb6, 0x9d, + 0xd9, 0x58, 0xb2, 0xda, 0x25, 0xf2, 0x1e, 0x34, 0xf1, 0x5d, 0x8b, 0xb3, 0xda, 0x1a, 0x69, 0xc3, + 0x06, 0x7f, 0x40, 0x11, 0x9c, 0x32, 0x69, 0x01, 0x9c, 0x47, 0x9e, 0x2f, 0xe8, 0x0a, 0xd2, 0x97, + 0xde, 0x8d, 0xa0, 0xab, 0x87, 0x3f, 0x85, 0xba, 0x9c, 0xb9, 0x14, 0x1b, 0x92, 0xd5, 0x2e, 0x91, + 0x2d, 0xd8, 0x7c, 0x7e, 0x3d, 0x1b, 0x45, 0x31, 0x4b, 0x23, 0xbb, 0xb0, 0xdd, 0xb7, 0xdd, 0x11, + 0x75, 0xd2, 0x82, 0xf2, 0xa1, 0x0b, 0xeb, 0x62, 0x5b, 0x33, 0x68, 0x42, 0x17, 0x23, 0xdb, 0x25, + 0xb2, 0x01, 0x75, 0xd6, 0x64, 0x90, 0xd2, 0x18, 0x0c, 0xbe, 0xe7, 0x90, 0x46, 0x98, 0x3c, 0x0a, + 0x48, 0x73, 0x98, 0x08, 0x11, 0xe9, 0x2a, 0xe9, 0x40, 0x1b, 0x7f, 0x4d, 0xe7, 0xbe, 0x63, 0x47, + 0x9c, 0xbb, 0x76, 0x38, 0x80, 0x46, 0x9c, 0x57, 0xb6, 0x44, 0x58, 0x8c, 0x79, 0xed, 0x12, 0x8b, + 0x08, 0x86, 0x08, 0x79, 0x5f, 0x1e, 0xb7, 0x35, 0x1e, 0x34, 0xcf, 0x97, 0x8c, 0xf2, 0xf1, 0xdf, + 0x5b, 0x50, 0xe3, 0x60, 0xc8, 0x57, 0xd0, 0x88, 0x1f, 0x0a, 0x09, 0x1e, 0xee, 0xd9, 0x77, 0x4b, + 0x63, 0x27, 0xc3, 0xe5, 0x49, 0x33, 0x1f, 0xff, 0xee, 0x9f, 0xff, 0xf9, 0x73, 0xf9, 0x81, 0xd9, + 0x39, 0xb2, 0xfd, 0x59, 0x78, 0x74, 0xfd, 0xd4, 0x76, 0xfc, 0x4b, 0xfb, 0xe9, 0x11, 0xdb, 0xf2, + 0xe1, 0xc7, 0xda, 0x21, 0x99, 0x40, 0x53, 0x79, 0xaf, 0x23, 0x5d, 0xa6, 0x26, 0xff, 0x42, 0x68, + 0xec, 0xe6, 0xf8, 0xc2, 0xc0, 0x47, 0x68, 0x60, 0xdf, 0x78, 0x58, 0x64, 0xe0, 0xe8, 0x0d, 0xeb, + 0x98, 0x5f, 0x33, 0x3b, 0x9f, 0x00, 0x24, 0x6f, 0x68, 0x04, 0xd1, 0xe6, 0x9e, 0xe5, 0x8c, 0x6e, + 0x96, 0x2d, 0x8c, 0x94, 0x88, 0x03, 0x4d, 0xe5, 0xb9, 0x89, 0x18, 0x99, 0xf7, 0x27, 0xe5, 0x7d, + 0xcc, 0x78, 0x58, 0x28, 0x13, 0x9a, 0x3e, 0x40, 0xb8, 0x3d, 0xb2, 0x97, 0x81, 0x1b, 0xe2, 0x52, + 0x81, 0x97, 0xf4, 0x61, 0x43, 0x7d, 0xd5, 0x21, 0xe8, 0x7d, 0xc1, 0x73, 0x96, 0xa1, 0xe7, 0x05, + 0x31, 0xe4, 0xcf, 0x60, 0x33, 0xf5, 0x8e, 0x42, 0x70, 0x71, 0xd1, 0x43, 0x8e, 0xf1, 0xa0, 0x40, + 0x12, 0xeb, 0xf9, 0x0a, 0xba, 0xf9, 0x77, 0x0f, 0x8c, 0xe2, 0x23, 0x25, 0x29, 0xf9, 0xb7, 0x07, + 0xa3, 0xb7, 0x4c, 0x1c, 0xab, 0x3e, 0x83, 0x76, 0xf6, 0x7d, 0x80, 0x60, 0xf8, 0x96, 0x3c, 0x67, + 0x18, 0x7b, 0xc5, 0xc2, 0x58, 0xe1, 0xc7, 0xd0, 0x88, 0xaf, 0xdf, 0xbc, 0x50, 0xb3, 0xaf, 0x00, + 0xbc, 0x50, 0x73, 0x77, 0x74, 0xb3, 0x44, 0xa6, 0xb0, 0x99, 0xba, 0x11, 0xf3, 0x78, 0x15, 0x5d, + 0xc7, 0x79, 0xbc, 0x0a, 0xaf, 0xcf, 0xe6, 0xfb, 0x98, 0xe0, 0x87, 0x46, 0x37, 0x9b, 0x60, 0xde, + 0xbc, 0x58, 0x29, 0x9e, 0x40, 0x2b, 0x7d, 0x79, 0x25, 0x0f, 0x78, 0x2b, 0x2e, 0xb8, 0x17, 0x1b, + 0x46, 0x91, 0x28, 0xc6, 0x1c, 0xc0, 0x66, 0xea, 0x0e, 0x2a, 0x30, 0x17, 0x5c, 0x6b, 0x05, 0xe6, + 0xa2, 0x0b, 0xab, 0xf9, 0x43, 0xc4, 0xfc, 0xd1, 0xe1, 0x07, 0x19, 0xcc, 0x62, 0x94, 0x3d, 0x7a, + 0xc3, 0x66, 0x91, 0xaf, 0x65, 0x71, 0x5e, 0xc5, 0x71, 0xe2, 0x2d, 0x2e, 0x15, 0xa7, 0xd4, 0x3d, + 0x36, 0x15, 0xa7, 0xf4, 0x5d, 0xd5, 0xfc, 0x10, 0x6d, 0x3e, 0x36, 0x8c, 0x8c, 0x4d, 0x3e, 0xea, + 0x1f, 0xbd, 0xf1, 0x7c, 0xdc, 0xb6, 0xbf, 0x04, 0x48, 0x86, 0x75, 0xbe, 0x6d, 0x73, 0xf7, 0x05, + 0xbe, 0x6d, 0xf3, 0x33, 0xbd, 0xd9, 0x43, 0x1b, 0x3a, 0xe9, 0x16, 0xfb, 0x45, 0x26, 0x49, 0xc6, + 0xf9, 0x10, 0x9c, 0xca, 0xb8, 0x3a, 0xb4, 0xa7, 0x33, 0x9e, 0x1a, 0x5b, 0xcd, 0x7d, 0xb4, 0x62, + 0x18, 0x3b, 0xd9, 0x8c, 0xe3, 0x32, 0xe6, 0x84, 0x83, 0x73, 0x5f, 0x32, 0x4e, 0x72, 0x3b, 0x45, + 0xd3, 0x28, 0xb7, 0x53, 0x38, 0x7b, 0xca, 0x4e, 0x47, 0x7a, 0x59, 0x3b, 0x8b, 0xa1, 0xda, 0xec, + 0xc8, 0x05, 0xd4, 0xf8, 0x7c, 0x48, 0xb6, 0x84, 0x32, 0x45, 0x3f, 0x51, 0x59, 0x42, 0xf1, 0x0f, + 0x50, 0xf1, 0x23, 0x72, 0x57, 0x0b, 0x25, 0xbf, 0x86, 0xa6, 0x32, 0x52, 0xf1, 0x3e, 0x9d, 0x1f, + 0xfb, 0x78, 0x9f, 0x2e, 0x98, 0xbd, 0x96, 0x46, 0x89, 0xb2, 0x55, 0xb8, 0x2d, 0xfa, 0xb0, 0xa1, + 0x8e, 0x9c, 0xbc, 0xe9, 0x15, 0xcc, 0xa6, 0x86, 0x9e, 0x17, 0xc4, 0x1b, 0xe2, 0x04, 0x5a, 0xe9, + 0xd9, 0x89, 0xef, 0xad, 0xc2, 0xc1, 0x8c, 0xef, 0xad, 0xe2, 0x51, 0xcb, 0x2c, 0x31, 0x3c, 0xea, + 0x70, 0x43, 0xd4, 0x23, 0x28, 0xd5, 0x94, 0xf4, 0xbc, 0x40, 0x2a, 0x79, 0xa6, 0xff, 0xe3, 0x6d, + 0x4f, 0xfb, 0xf6, 0x6d, 0x4f, 0xfb, 0xf7, 0xdb, 0x9e, 0xf6, 0xc7, 0x77, 0xbd, 0xd2, 0xb7, 0xef, + 0x7a, 0xa5, 0x7f, 0xbd, 0xeb, 0x95, 0x86, 0x35, 0xfc, 0x0f, 0xf0, 0x47, 0xff, 0x0d, 0x00, 0x00, + 0xff, 0xff, 0x62, 0x01, 0x57, 0x61, 0x47, 0x1c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4169,6 +4186,13 @@ func (m *StartTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StartTime) > 0 { + i -= len(m.StartTime) + copy(dAtA[i:], m.StartTime) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.StartTime))) + i-- + dAtA[i] = 0x22 + } if m.RemoveMeta { i-- if m.RemoveMeta { @@ -5030,6 +5054,13 @@ func (m *CheckTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StartTime) > 0 { + i -= len(m.StartTime) + copy(dAtA[i:], m.StartTime) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.StartTime))) + i-- + dAtA[i] = 0x22 + } if m.WarnCnt != 0 { i = encodeVarintDmmaster(dAtA, i, uint64(m.WarnCnt)) i-- @@ -6561,6 +6592,10 @@ func (m *StartTaskRequest) Size() (n int) { if m.RemoveMeta { n += 2 } + l = len(m.StartTime) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -6945,6 +6980,10 @@ func (m *CheckTaskRequest) Size() (n int) { if m.WarnCnt != 0 { n += 1 + sovDmmaster(uint64(m.WarnCnt)) } + l = len(m.StartTime) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -7724,6 +7763,38 @@ func (m *StartTaskRequest) Unmarshal(dAtA []byte) error { } } m.RemoveMeta = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StartTime = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) @@ -10105,6 +10176,38 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StartTime = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) diff --git a/dm/dm/proto/dmmaster.proto b/dm/dm/proto/dmmaster.proto index a4504375923..f97bc200730 100644 --- a/dm/dm/proto/dmmaster.proto +++ b/dm/dm/proto/dmmaster.proto @@ -112,8 +112,9 @@ service Master { message StartTaskRequest { string task = 1; // task's configuration, yaml format - repeated string sources = 2; // mysql source need to do start task, empty for all sources defiend in the task config + repeated string sources = 2; // mysql source need to do start task, empty for all sources defined in the task config bool removeMeta = 3; // whether to remove meta data for this task or not + string startTime = 4; // a highest priority field to specify starting of binlog replication } message StartTaskResponse { @@ -251,6 +252,7 @@ message CheckTaskRequest { string task = 1; // task's configuration, yaml format int64 errCnt = 2; // max error count to display int64 warnCnt = 3; // max warn count to display + string startTime = 4; // a highest priority field to specify starting of binlog replication } message CheckTaskResponse { diff --git a/dm/tests/dmctl_basic/check_list/check_task.sh b/dm/tests/dmctl_basic/check_list/check_task.sh index 08016f50796..d3a628c95af 100644 --- a/dm/tests/dmctl_basic/check_list/check_task.sh +++ b/dm/tests/dmctl_basic/check_list/check_task.sh @@ -35,6 +35,13 @@ function check_task_error_database_config() { "Please check the database connection and the database config in configuration file" 1 } +function check_task_wrong_start_time_format() { + task_conf=$1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "check-task $task_conf --start-time '20060102 150405'" \ + "error while parse start-time" 1 +} + function check_task_error_count() { task_conf=$1 # 10 errors diff --git a/dm/tests/dmctl_basic/check_list/start_task.sh b/dm/tests/dmctl_basic/check_list/start_task.sh index acbb79cdf3c..6e41a0b7b70 100644 --- a/dm/tests/dmctl_basic/check_list/start_task.sh +++ b/dm/tests/dmctl_basic/check_list/start_task.sh @@ -11,3 +11,10 @@ function start_task_wrong_config_file() { "start-task not_exists_config_file" \ "error in get file content" 1 } + +function start_task_wrong_start_time_format() { + task_conf=$1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $task_conf --start-time '20060102 150405'" \ + "error while parse start-time" 1 +} diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index a1d82ee7dea..858713ee737 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -279,6 +279,7 @@ function run() { echo "dmctl_check_task" check_task_pass $TASK_CONF + check_task_wrong_start_time_format $cur/conf/dm-task3.yaml check_task_not_pass $cur/conf/dm-task2.yaml check_task_error_count $cur/conf/dm-task3.yaml @@ -296,6 +297,7 @@ function run() { check_task_error_database_config $WORK_DIR/dm-task-error-database-config.yaml echo "dmctl_start_task" + start_task_wrong_start_time_format $cur/conf/dm-task3.yaml dmctl_start_task check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/go.mod b/go.mod index 15724e773d3..bc9edb33616 100644 --- a/go.mod +++ b/go.mod @@ -35,6 +35,7 @@ require ( github.com/golang/protobuf v1.5.2 github.com/google/btree v1.0.0 github.com/google/go-cmp v0.5.6 + github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 github.com/google/uuid v1.1.2 github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 diff --git a/go.sum b/go.sum index 23277d93f7b..1f031ae2e48 100644 --- a/go.sum +++ b/go.sum @@ -486,6 +486,8 @@ github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 h1:K6RDEckDVWvDI9JAJY github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/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/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= 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= diff --git a/tools/check/go.sum b/tools/check/go.sum index ea23d874225..0903b3f2a69 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -296,6 +296,7 @@ github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +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 v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= From 03c5232e26e14e3bf6ef5b226a283d409d9ee3bb Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 26 Jan 2022 15:30:45 +0800 Subject: [PATCH 25/72] tests(ticdc): Migrate several test cases to testify (#4199) close pingcap/tiflow#2901 --- cdc/entry/schema_test_helper.go | 24 +- cdc/owner/barrier_test.go | 39 +-- cdc/owner/changefeed_test.go | 145 ++++---- cdc/owner/ddl_puller_test.go | 104 +++--- cdc/owner/ddl_sink_test.go | 37 +- cdc/owner/feed_state_manager_test.go | 183 +++++----- cdc/owner/main_test.go | 24 ++ cdc/owner/owner_test.go | 196 +++++------ cdc/owner/scheduler_v1_test.go | 179 +++++----- cdc/owner/schema_test.go | 83 +++-- cdc/processor/main_test.go | 24 ++ cdc/processor/manager_test.go | 74 ++-- cdc/processor/processor_test.go | 392 ++++++++++------------ pkg/orchestrator/etcd_worker_bank_test.go | 26 +- pkg/orchestrator/etcd_worker_test.go | 168 ++++------ pkg/orchestrator/main_test.go | 24 ++ pkg/orchestrator/reactor_state_test.go | 139 ++++---- pkg/orchestrator/reactor_state_tester.go | 48 +-- 18 files changed, 946 insertions(+), 963 deletions(-) create mode 100644 cdc/owner/main_test.go create mode 100644 cdc/processor/main_test.go create mode 100644 pkg/orchestrator/main_test.go diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go index cf58c893719..ca07814a7a0 100644 --- a/cdc/entry/schema_test_helper.go +++ b/cdc/entry/schema_test_helper.go @@ -14,7 +14,8 @@ package entry import ( - "github.com/pingcap/check" + "testing" + ticonfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -22,33 +23,34 @@ import ( timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) // SchemaTestHelper is a test helper for schema which creates an internal tidb instance to generate DDL jobs with meta information type SchemaTestHelper struct { - c *check.C + t *testing.T tk *testkit.TestKit storage kv.Storage domain *domain.Domain } // NewSchemaTestHelper creates a SchemaTestHelper -func NewSchemaTestHelper(c *check.C) *SchemaTestHelper { +func NewSchemaTestHelper(t *testing.T) *SchemaTestHelper { store, err := mockstore.NewMockStore() - c.Assert(err, check.IsNil) + require.Nil(t, err) ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) session.SetSchemaLease(0) session.DisableStats4Test() domain, err := session.BootstrapSession(store) - c.Assert(err, check.IsNil) + require.Nil(t, err) domain.SetStatsUpdating(true) - tk := testkit.NewTestKit(c, store) + tk := testkit.NewTestKit(t, store) return &SchemaTestHelper{ - c: c, + t: t, tk: tk, storage: store, domain: domain, @@ -59,8 +61,8 @@ func NewSchemaTestHelper(c *check.C) *SchemaTestHelper { func (s *SchemaTestHelper) DDL2Job(ddl string) *timodel.Job { s.tk.MustExec(ddl) jobs, err := s.GetCurrentMeta().GetLastNHistoryDDLJobs(1) - s.c.Assert(err, check.IsNil) - s.c.Assert(jobs, check.HasLen, 1) + require.Nil(s.t, err) + require.Len(s.t, jobs, 1) return jobs[0] } @@ -72,7 +74,7 @@ func (s *SchemaTestHelper) Storage() kv.Storage { // GetCurrentMeta return the current meta snapshot func (s *SchemaTestHelper) GetCurrentMeta() *timeta.Meta { ver, err := s.storage.CurrentVersion(oracle.GlobalTxnScope) - s.c.Assert(err, check.IsNil) + require.Nil(s.t, err) return timeta.NewSnapshotMeta(s.storage.GetSnapshot(ver)) } diff --git a/cdc/owner/barrier_test.go b/cdc/owner/barrier_test.go index 5b09cd030ba..3524b80eb48 100644 --- a/cdc/owner/barrier_test.go +++ b/cdc/owner/barrier_test.go @@ -18,50 +18,41 @@ import ( "math/rand" "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&barrierSuite{}) - -type barrierSuite struct{} - -func (s *barrierSuite) TestBarrier(c *check.C) { - defer testleak.AfterTest(c)() +func TestBarrier(t *testing.T) { b := newBarriers() b.Update(ddlJobBarrier, 2) b.Update(syncPointBarrier, 3) b.Update(finishBarrier, 1) tp, ts := b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) b.Update(finishBarrier, 4) tp, ts = b.Min() - c.Assert(tp, check.Equals, ddlJobBarrier) - c.Assert(ts, check.Equals, uint64(2)) + require.Equal(t, tp, ddlJobBarrier) + require.Equal(t, ts, uint64(2)) b.Remove(ddlJobBarrier) tp, ts = b.Min() - c.Assert(tp, check.Equals, syncPointBarrier) - c.Assert(ts, check.Equals, uint64(3)) + require.Equal(t, tp, syncPointBarrier) + require.Equal(t, ts, uint64(3)) b.Update(finishBarrier, 1) tp, ts = b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) b.Update(ddlJobBarrier, 5) tp, ts = b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) } -func (s *barrierSuite) TestBarrierRandom(c *check.C) { - defer testleak.AfterTest(c)() +func TestBarrierRandom(t *testing.T) { maxBarrierType := 50 maxBarrierTs := 1000000 b := newBarriers() @@ -90,7 +81,7 @@ func (s *barrierSuite) TestBarrierRandom(c *check.C) { } } tp, ts := b.Min() - c.Assert(ts, check.Equals, expectedMinTs) - c.Assert(expectedBarriers[tp], check.Equals, expectedMinTs) + require.Equal(t, ts, expectedMinTs) + require.Equal(t, expectedBarriers[tp], expectedMinTs) } } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index e31908a18d5..427d1abf49e 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -15,13 +15,14 @@ package owner import ( "context" + "io/ioutil" "os" "path/filepath" "sync" "sync/atomic" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tiflow/cdc/entry" @@ -31,8 +32,8 @@ import ( "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/pdtime" "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/pingcap/tiflow/pkg/version" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) @@ -112,11 +113,7 @@ func (m *mockDDLSink) Barrier(ctx context.Context) error { return nil } -var _ = check.Suite(&changefeedSuite{}) - -type changefeedSuite struct{} - -func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *orchestrator.ChangefeedReactorState, +func createChangefeed4Test(ctx cdcContext.Context, t *testing.T) (*changefeed, *orchestrator.ChangefeedReactorState, map[model.CaptureID]*model.CaptureInfo, *orchestrator.ReactorStateTester) { ctx.GlobalVars().PDClient = &gc.MockPDClient{ UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { @@ -130,9 +127,9 @@ func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *or return &mockDDLSink{} }) state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) info = ctx.ChangefeedVars().Info return info, true, nil }) @@ -142,14 +139,13 @@ func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *or return cf, state, captures, tester } -func (s *changefeedSuite) TestPreCheck(c *check.C) { - defer testleak.AfterTest(c)() +func TestPreCheck(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status, check.NotNil) - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + require.NotNil(t, state.Status) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) // test clean the meta data of offline capture offlineCaputreID := "offline-capture" @@ -166,17 +162,16 @@ func (s *changefeedSuite) TestPreCheck(c *check.C) { cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status, check.NotNil) - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskStatuses, check.Not(check.HasKey), offlineCaputreID) - c.Assert(state.TaskPositions, check.Not(check.HasKey), offlineCaputreID) - c.Assert(state.Workloads, check.Not(check.HasKey), offlineCaputreID) + require.NotNil(t, state.Status) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.TaskStatuses, offlineCaputreID) + require.NotContains(t, state.TaskPositions, offlineCaputreID) + require.NotContains(t, state.Workloads, offlineCaputreID) } -func (s *changefeedSuite) TestInitialize(c *check.C) { - defer testleak.AfterTest(c)() +func TestInitialize(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) @@ -185,13 +180,12 @@ func (s *changefeedSuite) TestInitialize(c *check.C) { // initialize cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs) + require.Equal(t, state.Status.CheckpointTs, ctx.ChangefeedVars().Info.StartTs) } -func (s *changefeedSuite) TestHandleError(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedHandleError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) @@ -205,14 +199,12 @@ func (s *changefeedSuite) TestHandleError(c *check.C) { // handle error cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs) - c.Assert(state.Info.Error.Message, check.Equals, "fake error") + require.Equal(t, state.Status.CheckpointTs, ctx.ChangefeedVars().Info.StartTs) + require.Equal(t, state.Info.Error.Message, "fake error") } -func (s *changefeedSuite) TestExecDDL(c *check.C) { - defer testleak.AfterTest(c)() - - helper := entry.NewSchemaTestHelper(c) +func TestExecDDL(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() // Creates a table, which will be deleted at the start-ts of the changefeed. // It is expected that the changefeed DOES NOT replicate this table. @@ -237,7 +229,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { }, }) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) tickThreeTime := func() { cf.Tick(ctx, state, captures) @@ -249,13 +241,12 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { } // pre check and initialize tickThreeTime() - - c.Assert(cf.schema.AllPhysicalTables(), check.HasLen, 1) - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Operation, check.HasLen, 0) - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasLen, 0) + require.Len(t, cf.schema.AllPhysicalTables(), 1) + require.Len(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Operation, 0) + require.Len(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, 0) job = helper.DDL2Job("drop table test0.table0") - // ddl puller resolved ts grow uo + // ddl puller resolved ts grow up mockDDLPuller := cf.ddlPuller.(*mockDDLPuller) mockDDLPuller.resolvedTs = startTs mockDDLSink := cf.sink.(*mockDDLSink) @@ -263,15 +254,15 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) // three tick to make sure all barriers set in initialize is handled tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // The ephemeral table should have left no trace in the schema cache - c.Assert(cf.schema.AllPhysicalTables(), check.HasLen, 0) + require.Len(t, cf.schema.AllPhysicalTables(), 0) // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // handle create database job = helper.DDL2Job("create database test1") @@ -279,14 +270,14 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockDDLSink.ddlExecuting.Query, check.Equals, "CREATE DATABASE `test1`") + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) + require.Equal(t, mockDDLSink.ddlExecuting.Query, "CREATE DATABASE `test1`") // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // handle create table job = helper.DDL2Job("create table test1.test1(id int primary key)") @@ -294,22 +285,22 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockDDLSink.ddlExecuting.Query, check.Equals, "CREATE TABLE `test1`.`test1` (`id` INT PRIMARY KEY)") + + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) + require.Equal(t, mockDDLSink.ddlExecuting.Query, "CREATE TABLE `test1`.`test1` (`id` INT PRIMARY KEY)") // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasKey, job.TableID) + require.Contains(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, job.TableID) } -func (s *changefeedSuite) TestSyncPoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestSyncPoint(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) ctx.ChangefeedVars().Info.SyncPointEnabled = true ctx.ChangefeedVars().Info.SyncPointInterval = 1 * time.Second - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check @@ -331,16 +322,15 @@ func (s *changefeedSuite) TestSyncPoint(c *check.C) { } for i := 1; i < len(mockDDLSink.syncPointHis); i++ { // check the time interval between adjacent sync points is less or equal than one second - c.Assert(mockDDLSink.syncPointHis[i]-mockDDLSink.syncPointHis[i-1], check.LessEqual, uint64(1000<<18)) + require.LessOrEqual(t, mockDDLSink.syncPointHis[i]-mockDDLSink.syncPointHis[i-1], uint64(1000<<18)) } - c.Assert(len(mockDDLSink.syncPointHis), check.GreaterEqual, 5) + require.GreaterOrEqual(t, len(mockDDLSink.syncPointHis), 5) } -func (s *changefeedSuite) TestFinished(c *check.C) { - defer testleak.AfterTest(c)() +func TestFinished(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) ctx.ChangefeedVars().Info.TargetTs = ctx.ChangefeedVars().Info.StartTs + 1000 - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check @@ -359,17 +349,17 @@ func (s *changefeedSuite) TestFinished(c *check.C) { tester.MustApplyPatches() } - c.Assert(state.Status.CheckpointTs, check.Equals, state.Info.TargetTs) - c.Assert(state.Info.State, check.Equals, model.StateFinished) + require.Equal(t, state.Status.CheckpointTs, state.Info.TargetTs) + require.Equal(t, state.Info.State, model.StateFinished) } -func (s *changefeedSuite) TestRemoveChangefeed(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRemoveChangefeed(t *testing.T) { baseCtx, cancel := context.WithCancel(context.Background()) ctx := cdcContext.NewContext4Test(baseCtx, true) info := ctx.ChangefeedVars().Info - dir := c.MkDir() + dir, err := ioutil.TempDir("", "remove-changefeed-test") + require.NoError(t, err) + defer os.RemoveAll(dir) info.Config.Consistent = &config.ConsistentConfig{ Level: "eventual", Storage: filepath.Join("nfs://", dir), @@ -378,17 +368,17 @@ func (s *changefeedSuite) TestRemoveChangefeed(c *check.C) { ID: ctx.ChangefeedVars().ID, Info: info, }) - testChangefeedReleaseResource(c, ctx, cancel, dir, true /*expectedInitialized*/) + testChangefeedReleaseResource(t, ctx, cancel, dir, true /*expectedInitialized*/) } -func (s *changefeedSuite) TestRemovePausedChangefeed(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRemovePausedChangefeed(t *testing.T) { baseCtx, cancel := context.WithCancel(context.Background()) ctx := cdcContext.NewContext4Test(baseCtx, true) info := ctx.ChangefeedVars().Info info.State = model.StateStopped - dir := c.MkDir() + dir, err := ioutil.TempDir("", "remove-paused-changefeed-test") + require.NoError(t, err) + defer os.RemoveAll(dir) info.Config.Consistent = &config.ConsistentConfig{ Level: "eventual", Storage: filepath.Join("nfs://", dir), @@ -397,17 +387,17 @@ func (s *changefeedSuite) TestRemovePausedChangefeed(c *check.C) { ID: ctx.ChangefeedVars().ID, Info: info, }) - testChangefeedReleaseResource(c, ctx, cancel, dir, false /*expectedInitialized*/) + testChangefeedReleaseResource(t, ctx, cancel, dir, false /*expectedInitialized*/) } func testChangefeedReleaseResource( - c *check.C, + t *testing.T, ctx cdcContext.Context, cancel context.CancelFunc, redoLogDir string, expectedInitialized bool, ) { - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) // pre check cf.Tick(ctx, state, captures) @@ -416,7 +406,7 @@ func testChangefeedReleaseResource( // initialize cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(cf.initialized, check.Equals, expectedInitialized) + require.Equal(t, cf.initialized, expectedInitialized) // remove changefeed from state manager by admin job cf.feedStateManager.PushAdminJob(&model.AdminJob{ @@ -425,15 +415,14 @@ func testChangefeedReleaseResource( }) // changefeed tick will release resources err := cf.tick(ctx, state, captures) - c.Assert(err, check.IsNil) + require.Nil(t, err) cancel() // check redo log dir is deleted _, err = os.Stat(redoLogDir) - c.Assert(os.IsNotExist(err), check.IsTrue) + require.True(t, os.IsNotExist(err)) } -func (s *changefeedSuite) TestAddSpecialComment(c *check.C) { - defer testleak.AfterTest(c)() +func TestAddSpecialComment(t *testing.T) { testCase := []struct { input string result string @@ -565,10 +554,10 @@ func (s *changefeedSuite) TestAddSpecialComment(c *check.C) { } for _, ca := range testCase { re, err := addSpecialComment(ca.input) - c.Check(err, check.IsNil) - c.Check(re, check.Equals, ca.result) + require.Nil(t, err) + require.Equal(t, re, ca.result) } - c.Assert(func() { + require.Panics(t, func() { _, _ = addSpecialComment("alter table t force, auto_increment = 12;alter table t force, auto_increment = 12;") - }, check.Panics, "invalid ddlQuery statement size") + }, "invalid ddlQuery statement size") } diff --git a/cdc/owner/ddl_puller_test.go b/cdc/owner/ddl_puller_test.go index 654fa371060..08f5a7bbd26 100644 --- a/cdc/owner/ddl_puller_test.go +++ b/cdc/owner/ddl_puller_test.go @@ -18,10 +18,10 @@ import ( "encoding/json" "sync" "sync/atomic" + "testing" "time" "github.com/benbjohnson/clock" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/parser/model" @@ -29,25 +29,21 @@ import ( "github.com/pingcap/tiflow/cdc/model" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zaptest/observer" ) -var _ = check.Suite(&ddlPullerSuite{}) - -type ddlPullerSuite struct{} - type mockPuller struct { - c *check.C + t *testing.T inCh chan *model.RawKVEntry outCh chan *model.RawKVEntry resolvedTs model.Ts } -func newMockPuller(c *check.C, startTs model.Ts) *mockPuller { +func newMockPuller(t *testing.T, startTs model.Ts) *mockPuller { return &mockPuller{ - c: c, + t: t, inCh: make(chan *model.RawKVEntry), outCh: make(chan *model.RawKVEntry), resolvedTs: startTs - 1, @@ -84,7 +80,7 @@ func (m *mockPuller) append(e *model.RawKVEntry) { func (m *mockPuller) appendDDL(job *timodel.Job) { b, err := json.Marshal(job) - m.c.Assert(err, check.IsNil) + require.Nil(m.t, err) ek := []byte("m") ek = codec.EncodeBytes(ek, []byte("DDLJobList")) ek = codec.EncodeUint(ek, uint64('l')) @@ -106,13 +102,12 @@ func (m *mockPuller) appendResolvedTs(ts model.Ts) { }) } -func (s *ddlPullerSuite) TestPuller(c *check.C) { - defer testleak.AfterTest(c)() +func TestPuller(t *testing.T) { startTs := uint64(10) - mockPuller := newMockPuller(c, startTs) + mockPuller := newMockPuller(t, startTs) ctx := cdcContext.NewBackendContext4Test(true) p, err := newDDLPuller(ctx, startTs) - c.Assert(err, check.IsNil) + require.Nil(t, err) p.(*ddlPullerImpl).puller = mockPuller var wg sync.WaitGroup wg.Add(1) @@ -122,22 +117,22 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { if errors.Cause(err) == context.Canceled { err = nil } - c.Assert(err, check.IsNil) + require.Nil(t, err) }() defer wg.Wait() defer p.Close() // test initialize state resolvedTs, ddl := p.FrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) // test send resolvedTs mockPuller.appendResolvedTs(15) - waitResolvedTsGrowing(c, p, 15) + waitResolvedTsGrowing(t, p, 15) // test send ddl job out of order mockPuller.appendDDL(&timodel.Job{ @@ -155,23 +150,23 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 16}, }) resolvedTs, ddl = p.FrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(15)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(15)) + require.Nil(t, ddl) mockPuller.appendResolvedTs(20) - waitResolvedTsGrowing(c, p, 16) + waitResolvedTsGrowing(t, p, 16) resolvedTs, ddl = p.FrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(16)) - c.Assert(ddl.ID, check.Equals, int64(1)) + require.Equal(t, resolvedTs, uint64(16)) + require.Equal(t, ddl.ID, int64(1)) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(16)) - c.Assert(ddl.ID, check.Equals, int64(1)) + require.Equal(t, resolvedTs, uint64(16)) + require.Equal(t, ddl.ID, int64(1)) // DDL could be processed with a delay, wait here for a pending DDL job is added - waitResolvedTsGrowing(c, p, 18) + waitResolvedTsGrowing(t, p, 18) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(18)) - c.Assert(ddl.ID, check.Equals, int64(2)) + require.Equal(t, resolvedTs, uint64(18)) + require.Equal(t, ddl.ID, int64(2)) // test add ddl job repeated mockPuller.appendDDL(&timodel.Job{ @@ -189,18 +184,18 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, }) mockPuller.appendResolvedTs(30) - waitResolvedTsGrowing(c, p, 25) + waitResolvedTsGrowing(t, p, 25) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(25)) - c.Assert(ddl.ID, check.Equals, int64(3)) + require.Equal(t, resolvedTs, uint64(25)) + require.Equal(t, ddl.ID, int64(3)) _, ddl = p.PopFrontDDL() - c.Assert(ddl, check.IsNil) + require.Nil(t, ddl) - waitResolvedTsGrowing(c, p, 30) + waitResolvedTsGrowing(t, p, 30) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(30)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(30)) + require.Nil(t, ddl) // test add invalid ddl job mockPuller.appendDDL(&timodel.Job{ @@ -218,15 +213,14 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 36}, }) mockPuller.appendResolvedTs(40) - waitResolvedTsGrowing(c, p, 40) + waitResolvedTsGrowing(t, p, 40) resolvedTs, ddl = p.PopFrontDDL() // no ddl should be received - c.Assert(resolvedTs, check.Equals, uint64(40)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(40)) + require.Nil(t, ddl) } -func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { - defer testleak.AfterTest(c)() +func TestResolvedTsStuck(t *testing.T) { // For observing the logs zapcore, logs := observer.New(zap.WarnLevel) conf := &log.Config{Level: "warn", File: log.FileLogConfig{}} @@ -236,10 +230,10 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { defer restoreFn() startTs := uint64(10) - mockPuller := newMockPuller(c, startTs) + mockPuller := newMockPuller(t, startTs) ctx := cdcContext.NewBackendContext4Test(true) p, err := newDDLPuller(ctx, startTs) - c.Assert(err, check.IsNil) + require.Nil(t, err) mockClock := clock.NewMock() p.(*ddlPullerImpl).clock = mockClock @@ -253,22 +247,22 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { if errors.Cause(err) == context.Canceled { err = nil } - c.Assert(err, check.IsNil) + require.Nil(t, err) }() defer wg.Wait() defer p.Close() // test initialize state resolvedTs, ddl := p.FrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) mockPuller.appendResolvedTs(30) - waitResolvedTsGrowing(c, p, 30) - c.Assert(logs.Len(), check.Equals, 0) + waitResolvedTsGrowing(t, p, 30) + require.Equal(t, logs.Len(), 0) mockClock.Add(2 * ownerDDLPullerStuckWarnTimeout) for i := 0; i < 20; i++ { @@ -278,17 +272,17 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { } time.Sleep(10 * time.Millisecond) if i == 19 { - c.Fatal("warning log not printed") + t.Fatal("warning log not printed") } } mockPuller.appendResolvedTs(40) - waitResolvedTsGrowing(c, p, 40) + waitResolvedTsGrowing(t, p, 40) } // waitResolvedTsGrowing can wait the first DDL reaches targetTs or if no pending // DDL, DDL resolved ts reaches targetTs. -func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { +func waitResolvedTsGrowing(t *testing.T, p DDLPuller, targetTs model.Ts) { err := retry.Do(context.Background(), func() error { resolvedTs, _ := p.FrontDDL() if resolvedTs < targetTs { @@ -296,5 +290,5 @@ func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { } return nil }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(100)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } diff --git a/cdc/owner/ddl_sink_test.go b/cdc/owner/ddl_sink_test.go index 94dff72381b..3a6d2137b66 100644 --- a/cdc/owner/ddl_sink_test.go +++ b/cdc/owner/ddl_sink_test.go @@ -17,22 +17,18 @@ import ( "context" "sync" "sync/atomic" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&ddlSinkSuite{}) - -type ddlSinkSuite struct{} - type mockSink struct { sink.Sink checkpointTs model.Ts @@ -78,8 +74,7 @@ func newDDLSink4Test() (DDLSink, *mockSink) { return ddlSink, mockSink } -func (s *ddlSinkSuite) TestCheckpoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckpoint(t *testing.T) { ddlSink, mSink := newDDLSink4Test() ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) @@ -98,13 +93,12 @@ func (s *ddlSinkSuite) TestCheckpoint(c *check.C) { }, retry.WithBackoffBaseDelay(100), retry.WithMaxTries(30)) } ddlSink.emitCheckpointTs(ctx, 1) - c.Assert(waitCheckpointGrowingUp(mSink, 1), check.IsNil) + require.Nil(t, waitCheckpointGrowingUp(mSink, 1)) ddlSink.emitCheckpointTs(ctx, 10) - c.Assert(waitCheckpointGrowingUp(mSink, 10), check.IsNil) + require.Nil(t, waitCheckpointGrowingUp(mSink, 10)) } -func (s *ddlSinkSuite) TestExecDDL(c *check.C) { - defer testleak.AfterTest(c)() +func TestExecDDLEvents(t *testing.T) { ddlSink, mSink := newDDLSink4Test() ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) @@ -123,17 +117,16 @@ func (s *ddlSinkSuite) TestExecDDL(c *check.C) { for _, event := range ddlEvents { for { done, err := ddlSink.emitDDLEvent(ctx, event) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done { - c.Assert(mSink.GetDDL(), check.DeepEquals, event) + require.Equal(t, mSink.GetDDL(), event) break } } } } -func (s *ddlSinkSuite) TestExecDDLError(c *check.C) { - defer testleak.AfterTest(c)() +func TestExecDDLError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) var ( @@ -165,24 +158,24 @@ func (s *ddlSinkSuite) TestExecDDLError(c *check.C) { ddl1 := &model.DDLEvent{CommitTs: 1} for { done, err := ddlSink.emitDDLEvent(ctx, ddl1) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done { - c.Assert(mSink.GetDDL(), check.DeepEquals, ddl1) + require.Equal(t, mSink.GetDDL(), ddl1) break } } - c.Assert(resultErr, check.IsNil) + require.Nil(t, resultErr) mSink.ddlError = cerror.ErrExecDDLFailed.GenWithStackByArgs() ddl2 := &model.DDLEvent{CommitTs: 2} for { done, err := ddlSink.emitDDLEvent(ctx, ddl2) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done || readResultErr() != nil { - c.Assert(mSink.GetDDL(), check.DeepEquals, ddl2) + require.Equal(t, mSink.GetDDL(), ddl2) break } } - c.Assert(cerror.ErrExecDDLFailed.Equal(readResultErr()), check.IsTrue) + require.True(t, cerror.ErrExecDDLFailed.Equal(readResultErr())) } diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index cccb24b9833..3eb931722e4 100644 --- a/cdc/owner/feed_state_manager_test.go +++ b/cdc/owner/feed_state_manager_test.go @@ -14,38 +14,33 @@ package owner import ( + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&feedStateManagerSuite{}) - -type feedStateManagerSuite struct{} - -func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleJob(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // an admin job which of changefeed is not match manager.PushAdminJob(&model.AdminJob{ @@ -54,7 +49,7 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // a running can not be resume manager.PushAdminJob(&model.AdminJob{ @@ -63,7 +58,7 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // stop a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -72,11 +67,12 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateStopped) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) + + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateStopped) + require.Equal(t, state.Info.AdminJobType, model.AdminStop) + require.Equal(t, state.Status.AdminJobType, model.AdminStop) // resume a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -85,11 +81,11 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateNormal) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminNone) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminNone) + require.True(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateNormal) + require.Equal(t, state.Info.AdminJobType, model.AdminNone) + require.Equal(t, state.Status.AdminJobType, model.AdminNone) // remove a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -98,51 +94,51 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsTrue) - c.Assert(state.Exist(), check.IsFalse) + + require.False(t, manager.ShouldRunning()) + require.True(t, manager.ShouldRemoved()) + require.False(t, state.Exist()) } -func (s *feedStateManagerSuite) TestMarkFinished(c *check.C) { - defer testleak.AfterTest(c)() +func TestMarkFinished(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) manager.MarkFinished() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFinished) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) + + require.False(t, manager.ShouldRunning()) + require.Equal(t, state.Info.State, model.StateFinished) + require.Equal(t, state.Info.AdminJobType, model.AdminFinish) + require.Equal(t, state.Status.AdminJobType, model.AdminFinish) } -func (s *feedStateManagerSuite) TestCleanUpInfos(c *check.C) { - defer testleak.AfterTest(c)() +func TestCleanUpInfos(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -155,37 +151,36 @@ func (s *feedStateManagerSuite) TestCleanUpInfos(c *check.C) { return model.TaskWorkload{}, true, nil }) tester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskPositions, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.Workloads, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.TaskPositions, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.Workloads, ctx.GlobalVars().CaptureInfo.ID) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) manager.MarkFinished() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFinished) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.TaskStatuses, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskPositions, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.Workloads, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) + require.False(t, manager.ShouldRunning()) + require.Equal(t, state.Info.State, model.StateFinished) + require.Equal(t, state.Info.AdminJobType, model.AdminFinish) + require.Equal(t, state.Status.AdminJobType, model.AdminFinish) + require.NotContains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.TaskPositions, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.Workloads, ctx.GlobalVars().CaptureInfo.ID) } -func (s *feedStateManagerSuite) TestHandleError(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -206,7 +201,7 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { } for _, d := range intervals { - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { return &model.TaskPosition{Error: &model.RunningError{ Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, @@ -217,17 +212,17 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) + require.False(t, manager.ShouldRunning()) time.Sleep(d) manager.Tick(state) tester.MustApplyPatches() } - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFailed) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateFailed) + require.Equal(t, state.Info.AdminJobType, model.AdminStop) + require.Equal(t, state.Status.AdminJobType, model.AdminStop) // admin resume must retry changefeed immediately. manager.PushAdminJob(&model.AdminJob{ @@ -237,26 +232,56 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateNormal) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminNone) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminNone) + require.True(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateNormal) + require.Equal(t, state.Info.AdminJobType, model.AdminNone) + require.Equal(t, state.Status.AdminJobType, model.AdminNone) +} + +func TestHandleFastFailError(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(t, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + require.Nil(t, info) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Error: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "CDC:ErrGCTTLExceeded", + Message: "fake error for test", + }}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + // test handling fast failed error with non-nil ChangeFeedInfo + tester.MustApplyPatches() + // test handling fast failed error with nil ChangeFeedInfo + // set info to nil when this patch is applied + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return nil, true, nil + }) + manager.Tick(state) + // When the patches are applied, the callback function of PatchInfo in feedStateManager.HandleError will be called. + // At that time, the nil pointer will be checked instead of throwing a panic. See issue #3128 for more detail. + tester.MustApplyPatches() } -func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedStatusNotExist(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, map[string]string{ + tester := orchestrator.NewReactorStateTester(t, state, map[string]string{ "/tidb/cdc/capture/d563bfc0-f406-4f34-bc7d-6dc2e35a44e5": `{"id":"d563bfc0-f406-4f34-bc7d-6dc2e35a44e5","address":"172.16.6.147:8300","version":"v5.0.0-master-dirty"}`, "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole:///","opts":{},"create-time":"2021-06-05T00:44:15.065939487+08:00","start-ts":425381670108266496,"target-ts":0,"admin-job-type":1,"sort-engine":"unified","config":{"case-sensitive":true,"enable-old-value":true,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"open-protocol"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"failed","history":[],"error":{"addr":"172.16.6.147:8300","code":"CDC:ErrSnapshotLostByGC","message":"[CDC:ErrSnapshotLostByGC]fail to create or maintain changefeed due to snapshot loss caused by GC. checkpoint-ts 425381670108266496 is earlier than GC safepoint at 0"},"sync-point-enabled":false,"sync-point-interval":600000000000,"creator-version":"v5.0.0-master-dirty"}`, "/tidb/cdc/owner/156579d017f84a68": "d563bfc0-f406-4f34-bc7d-6dc2e35a44e5", }) manager.Tick(state) - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) tester.MustApplyPatches() manager.PushAdminJob(&model.AdminJob{ @@ -265,9 +290,9 @@ func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { Opts: &model.AdminJobOption{ForceRemove: true}, }) manager.Tick(state) - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsTrue) + require.False(t, manager.ShouldRunning()) + require.True(t, manager.ShouldRemoved()) tester.MustApplyPatches() - c.Assert(state.Info, check.IsNil) - c.Assert(state.Exist(), check.IsFalse) + require.Nil(t, state.Info) + require.False(t, state.Exist()) } diff --git a/cdc/owner/main_test.go b/cdc/owner/main_test.go new file mode 100644 index 00000000000..2019d2834ec --- /dev/null +++ b/cdc/owner/main_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package owner + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index b836d174e32..bcca13f5dad 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -18,9 +18,9 @@ import ( "context" "fmt" "math" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" @@ -28,14 +28,10 @@ import ( "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -var _ = check.Suite(&ownerSuite{}) - -type ownerSuite struct{} - type mockManager struct { gc.Manager } @@ -48,7 +44,7 @@ func (m *mockManager) CheckStaleCheckpointTs( var _ gc.Manager = (*mockManager)(nil) -func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { +func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*Owner, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { ctx.GlobalVars().PDClient = &gc.MockPDClient{ UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { return safePoint, nil @@ -62,7 +58,7 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator ctx.GlobalVars().PDClient, ) state := orchestrator.NewGlobalState() - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) // set captures cdcKey := etcd.CDCKey{ @@ -70,18 +66,17 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator CaptureID: ctx.GlobalVars().CaptureInfo.ID, } captureBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(cdcKey.String(), captureBytes) return owner, state, tester } -func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestCreateRemoveChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) changefeedID := "test-changefeed" changefeedInfo := &model.ChangeFeedInfo{ @@ -89,7 +84,7 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -97,29 +92,28 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // delete changefeed info key to remove changefeed tester.MustUpdate(cdcKey.String(), nil) // this tick to clean the leak info fo the removed changefeed _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // this tick to remove the changefeed state in memory tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) + + require.NotContains(t, owner.changefeeds, changefeedID) + require.NotContains(t, state.Changefeeds, changefeedID) tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) - + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) removeJob := model.AdminJob{ CfID: changefeedID, Type: model.AdminRemove, @@ -131,24 +125,23 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { mockedManager := &mockManager{Manager: owner.gcManager} owner.gcManager = mockedManager err = owner.gcManager.CheckStaleCheckpointTs(ctx, changefeedID, 0) - c.Assert(err, check.NotNil) + require.NotNil(t, err) // this tick create remove changefeed patches owner.EnqueueJob(removeJob) _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // apply patches and update owner's in memory changefeed states tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) } -func (s *ownerSuite) TestStopChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestStopChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() @@ -158,7 +151,7 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -166,9 +159,8 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) - + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // remove changefeed forcibly owner.EnqueueJob(model.AdminJob{ CfID: changefeedID, @@ -180,23 +172,20 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { // this tick to clean the leak info fo the removed changefeed _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(err, check.IsNil) + require.Nil(t, err) // this tick to remove the changefeed state in memory tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) + require.NotContains(t, state.Changefeeds, changefeedID) } -func (s *ownerSuite) TestFixChangefeedState(c *check.C) { - defer testleak.AfterTest(c)() +func TestFixChangefeedState(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) // We need to do bootstrap. owner.bootstrapped = false changefeedID := "test-changefeed" @@ -208,7 +197,7 @@ func (s *ownerSuite) TestFixChangefeedState(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -217,23 +206,21 @@ func (s *ownerSuite) TestFixChangefeedState(c *check.C) { // For the first tick, we do a bootstrap, and it tries to fix the meta information. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.bootstrapped, check.IsTrue) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - + require.Nil(t, err) + require.True(t, owner.bootstrapped) + require.NotContains(t, owner.changefeeds, changefeedID) // Start tick normally. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // The meta information is fixed correctly. - c.Assert(owner.changefeeds[changefeedID].state.Info.State, check.Equals, model.StateStopped) + require.Equal(t, owner.changefeeds[changefeedID].state.Info.State, model.StateStopped) } -func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { - defer testleak.AfterTest(c)() +func TestFixChangefeedSinkProtocol(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) // We need to do bootstrap. owner.bootstrapped = false changefeedID := "test-changefeed" @@ -249,7 +236,7 @@ func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { }, } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -258,25 +245,23 @@ func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { // For the first tick, we do a bootstrap, and it tries to fix the meta information. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.bootstrapped, check.IsTrue) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.True(t, owner.bootstrapped) + require.NotContains(t, owner.changefeeds, changefeedID) // Start tick normally. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // The meta information is fixed correctly. - c.Assert(owner.changefeeds[changefeedID].state.Info.SinkURI, - check.Equals, + require.Equal(t, owner.changefeeds[changefeedID].state.Info.SinkURI, "kafka://127.0.0.1:9092/ticdc-test2?protocol=open-protocol") } -func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckClusterVersion(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() @@ -288,7 +273,7 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -298,8 +283,8 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { // check the tick is skipped and the changefeed will not be handled _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) tester.MustUpdate("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300","version":"`+ctx.GlobalVars().CaptureInfo.Version+`"}`)) @@ -307,17 +292,16 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { // check the tick is not skipped and the changefeed will be handled normally _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) } -func (s *ownerSuite) TestAdminJob(c *check.C) { - defer testleak.AfterTest(c)() +func TestAdminJob(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, _, _ := createOwner4Test(ctx, c) + owner, _, _ := createOwner4Test(ctx, t) owner.EnqueueJob(model.AdminJob{ CfID: "test-changefeed1", Type: model.AdminResume, @@ -330,11 +314,11 @@ func (s *ownerSuite) TestAdminJob(c *check.C) { // remove job.done, it's hard to check deep equals jobs := owner.takeOwnerJobs() for _, job := range jobs { - c.Assert(job.done, check.NotNil) + require.NotNil(t, job.done) close(job.done) job.done = nil } - c.Assert(jobs, check.DeepEquals, []*ownerJob{ + require.Equal(t, jobs, []*ownerJob{ { tp: ownerJobTypeAdminJob, adminJob: &model.AdminJob{ @@ -355,11 +339,10 @@ func (s *ownerSuite) TestAdminJob(c *check.C) { debugInfoWriter: &buf, }, }) - c.Assert(owner.takeOwnerJobs(), check.HasLen, 0) + require.Len(t, owner.takeOwnerJobs(), 0) } -func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestUpdateGCSafePoint(t *testing.T) { mockPDClient := &gc.MockPDClient{} o := NewOwner(mockPDClient) o.gcManager = gc.NewManager(mockPDClient) @@ -367,23 +350,23 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() state := orchestrator.NewGlobalState() - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) // no changefeed, the gc safe point should be max uint64 mockPDClient.UpdateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(math.MaxUint64-1)) + require.Equal(t, safePoint, uint64(math.MaxUint64-1)) return 0, nil } err := o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add a failed changefeed, it must not trigger update GC safepoint. mockPDClient.UpdateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - c.Fatal("must not update") + t.Fatal("must not update") return 0, nil } changefeedID1 := "changefeed-test1" @@ -397,7 +380,7 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { }) tester.MustApplyPatches() err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // switch the state of changefeed to normal, it must update GC safepoint to // 1 (checkpoint Ts of changefeed-test1). @@ -406,8 +389,8 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(1)) - c.Assert(serviceID, check.Equals, gc.CDCServiceSafePointID) + require.Equal(t, safePoint, uint64(1)) + require.Equal(t, serviceID, gc.CDCServiceSafePointID) ch <- struct{}{} return 0, nil } @@ -418,10 +401,10 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { }) tester.MustApplyPatches() err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) select { case <-time.After(5 * time.Second): - c.Fatal("timeout") + t.Fatal("timeout") case <-ch: } @@ -444,28 +427,27 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(19)) - c.Assert(serviceID, check.Equals, gc.CDCServiceSafePointID) + require.Equal(t, safePoint, uint64(19)) + require.Equal(t, serviceID, gc.CDCServiceSafePointID) ch <- struct{}{} return 0, nil } err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) select { case <-time.After(5 * time.Second): - c.Fatal("timeout") + t.Fatal("timeout") case <-ch: } } // make sure handleJobs works well even if there is two different // version of captures in the cluster -func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleJobsDontBlock(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) statusProvider := owner.StatusProvider() // work well @@ -476,7 +458,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { State: model.StateNormal, } changefeedStr, err := cfInfo1.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: cf1, @@ -484,9 +466,9 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) + require.Nil(t, err) - c.Assert(owner.changefeeds, check.HasKey, cf1) + require.Contains(t, owner.changefeeds, cf1) // add an non-consistent version capture captureInfo := &model.CaptureInfo{ @@ -499,7 +481,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { CaptureID: captureInfo.ID, } v, err := captureInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(cdcKey.String(), v) // try to add another changefeed @@ -510,7 +492,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { State: model.StateNormal, } changefeedStr1, err := cfInfo2.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey = etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: cf2, @@ -518,10 +500,10 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr1)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) + require.Nil(t, err) // make sure this changefeed add failed, which means that owner are return // in clusterVersionConsistent check - c.Assert(owner.changefeeds[cf2], check.IsNil) + require.Nil(t, owner.changefeeds[cf2]) // make sure statusProvider works well ctx1, cancel := context.WithTimeout(context.Background(), time.Second*5) @@ -543,13 +525,13 @@ WorkLoop: case <-done: break WorkLoop case <-ctx1.Done(): - c.Fatal(ctx1.Err()) + t.Fatal(ctx1.Err()) case <-ticker.C: _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) } } - c.Assert(errIn, check.IsNil) - c.Assert(infos[cf1], check.NotNil) - c.Assert(infos[cf2], check.IsNil) + require.Nil(t, errIn) + require.NotNil(t, infos[cf1]) + require.Nil(t, infos[cf2]) } diff --git a/cdc/owner/scheduler_v1_test.go b/cdc/owner/scheduler_v1_test.go index eb2950b7190..289c951b3db 100644 --- a/cdc/owner/scheduler_v1_test.go +++ b/cdc/owner/scheduler_v1_test.go @@ -16,17 +16,15 @@ package owner import ( "fmt" "math/rand" + "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&schedulerSuite{}) - -type schedulerSuite struct { +type schedulerTester struct { changefeedID model.ChangeFeedID state *orchestrator.ChangefeedReactorState tester *orchestrator.ReactorStateTester @@ -34,10 +32,10 @@ type schedulerSuite struct { scheduler *oldScheduler } -func (s *schedulerSuite) reset(c *check.C) { +func (s *schedulerTester) reset(t *testing.T) { s.changefeedID = fmt.Sprintf("test-changefeed-%x", rand.Uint32()) s.state = orchestrator.NewChangefeedReactorState("test-changefeed") - s.tester = orchestrator.NewReactorStateTester(c, s.state, nil) + s.tester = orchestrator.NewReactorStateTester(t, s.state, nil) s.scheduler = newSchedulerV1().(*schedulerV1CompatWrapper).inner s.captures = make(map[model.CaptureID]*model.CaptureInfo) s.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { @@ -46,7 +44,7 @@ func (s *schedulerSuite) reset(c *check.C) { s.tester.MustApplyPatches() } -func (s *schedulerSuite) addCapture(captureID model.CaptureID) { +func (s *schedulerTester) addCapture(captureID model.CaptureID) { captureInfo := &model.CaptureInfo{ ID: captureID, } @@ -57,7 +55,7 @@ func (s *schedulerSuite) addCapture(captureID model.CaptureID) { s.tester.MustApplyPatches() } -func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableIDs ...model.TableID) { +func (s *schedulerTester) finishTableOperation(captureID model.CaptureID, tableIDs ...model.TableID) { s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { for _, tableID := range tableIDs { status.Operation[tableID].Status = model.OperFinished @@ -82,10 +80,9 @@ func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableID s.tester.MustApplyPatches() } -func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { - defer testleak.AfterTest(c)() - - s.reset(c) +func TestScheduleOneCapture(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID := "test-capture-0" s.addCapture(captureID) @@ -100,27 +97,27 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.tester.MustUpdate(key.String(), nil) s.tester.MustApplyPatches() - s.reset(c) + s.reset(t) captureID = "test-capture-1" s.addCapture(captureID) // add three tables shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, 2: {StartTs: 0}, 3: {StartTs: 0}, 4: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 3: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 4: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() // two tables finish adding operation @@ -128,13 +125,13 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { // remove table 1,2 and add table 4,5 shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 4: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, @@ -147,13 +144,13 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.scheduler.MoveTable(3, "fake-capture") s.scheduler.MoveTable(4, "fake-capture") shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 3: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, @@ -165,77 +162,77 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.finishTableOperation(captureID, 1, 2, 3, 4, 5) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{}) // table 3 is missing by expected, because the table was trying to move to a invalid capture // and the move will failed, the table 3 will be add in next tick shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{}) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 3: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) } -func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { - defer testleak.AfterTest(c)() - s.reset(c) +func TestScheduleMoveTable(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID1 := "test-capture-1" captureID2 := "test-capture-2" s.addCapture(captureID1) // add a table shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) s.finishTableOperation(captureID1, 1) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() s.addCapture(captureID2) // add a table shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{ 2: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) @@ -243,48 +240,48 @@ func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { s.scheduler.MoveTable(2, captureID1) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, }) s.finishTableOperation(captureID2, 2) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{}) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, 2: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{}) } -func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { - defer testleak.AfterTest(c)() - s.reset(c) +func TestScheduleRebalance(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID1 := "test-capture-1" captureID2 := "test-capture-2" captureID3 := "test-capture-3" @@ -306,13 +303,13 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { // rebalance table shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables remove in capture 1, this 4 tables will be added to another capture in next tick - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 0) - c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 0) + require.Len(t, s.state.TaskStatuses[captureID1].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID2].Tables, 0) + require.Len(t, s.state.TaskStatuses[captureID3].Tables, 0) s.state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { for _, opt := range status.Operation { @@ -321,7 +318,7 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { return status, true, nil }) s.state.PatchTaskWorkload(captureID1, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) workload = make(model.TaskWorkload) for tableID := range s.state.TaskStatuses[captureID1].Tables { workload[tableID] = model.WorkloadInfo{Workload: 1} @@ -332,26 +329,26 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { // clean finished operation shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables add to another capture in this tick - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.HasLen, 0) + require.Len(t, s.state.TaskStatuses[captureID1].Operation, 0) // rebalance table shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables add to another capture in this tick - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 2) + require.Len(t, s.state.TaskStatuses[captureID1].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID2].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID3].Tables, 2) tableIDs := make(map[model.TableID]struct{}) for _, status := range s.state.TaskStatuses { for tableID := range status.Tables { tableIDs[tableID] = struct{}{} } } - c.Assert(tableIDs, check.DeepEquals, map[model.TableID]struct{}{1: {}, 2: {}, 3: {}, 4: {}, 5: {}, 6: {}}) + require.Equal(t, tableIDs, map[model.TableID]struct{}{1: {}, 2: {}, 3: {}, 4: {}, 5: {}, 6: {}}) } diff --git a/cdc/owner/schema_test.go b/cdc/owner/schema_test.go index 9ed7e8ec14f..9c8abb71316 100644 --- a/cdc/owner/schema_test.go +++ b/cdc/owner/schema_test.go @@ -15,38 +15,33 @@ package owner import ( "sort" + "testing" - "github.com/pingcap/check" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -var _ = check.Suite(&schemaSuite{}) - -type schemaSuite struct{} - -func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestAllPhysicalTables(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.HasLen, 0) + require.Nil(t, err) + require.Len(t, schema.AllPhysicalTables(), 0) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + require.Nil(t, schema.HandleDDL(job)) + require.Equal(t, schema.AllPhysicalTables(), []model.TableID{tableIDT1}) // add ineligible table - c.Assert(schema.HandleDDL(helper.DDL2Job("create table test.t2(id int)")), check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + require.Nil(t, schema.HandleDDL(helper.DDL2Job("create table test.t2(id int)"))) + require.Equal(t, schema.AllPhysicalTables(), []model.TableID{tableIDT1}) // add partition table job = helper.DDL2Job(`CREATE TABLE test.employees ( id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, @@ -62,7 +57,7 @@ func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { PARTITION p2 VALUES LESS THAN (15), PARTITION p3 VALUES LESS THAN (20) )`) - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) expectedTableIDs := []model.TableID{tableIDT1} for _, p := range job.BinlogInfo.TableInfo.GetPartitionInfo().Definitions { expectedTableIDs = append(expectedTableIDs, p.ID) @@ -74,42 +69,41 @@ func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { } sortTableIDs(expectedTableIDs) sortTableIDs(schema.AllPhysicalTables()) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, expectedTableIDs) + require.Equal(t, schema.AllPhysicalTables(), expectedTableIDs) } -func (s *schemaSuite) TestIsIneligibleTableID(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestIsIneligibleTableID(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) // add ineligible table job = helper.DDL2Job("create table test.t2(id int)") tableIDT2 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.IsIneligibleTableID(tableIDT1), check.IsFalse) - c.Assert(schema.IsIneligibleTableID(tableIDT2), check.IsTrue) + + require.Nil(t, schema.HandleDDL(job)) + require.False(t, schema.IsIneligibleTableID(tableIDT1)) + require.True(t, schema.IsIneligibleTableID(tableIDT2)) } -func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestBuildDDLEvent(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") event, err := schema.BuildDDLEvent(job) - c.Assert(err, check.IsNil) - c.Assert(event, check.DeepEquals, &model.DDLEvent{ + require.Nil(t, err) + require.Equal(t, event, &model.DDLEvent{ StartTs: job.StartTS, CommitTs: job.BinlogInfo.FinishedTS, Query: "create table test.t1(id int primary key)", @@ -122,11 +116,11 @@ func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { }, PreTableInfo: nil, }) - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) job = helper.DDL2Job("ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL") event, err = schema.BuildDDLEvent(job) - c.Assert(err, check.IsNil) - c.Assert(event, check.DeepEquals, &model.DDLEvent{ + require.Nil(t, err) + require.Equal(t, event, &model.DDLEvent{ StartTs: job.StartTS, CommitTs: job.BinlogInfo.FinishedTS, Query: "ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL", @@ -146,22 +140,21 @@ func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { }) } -func (s *schemaSuite) TestSinkTableInfos(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestSinkTableInfos(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) // add ineligible table job = helper.DDL2Job("create table test.t2(id int)") - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.SinkTableInfos(), check.DeepEquals, []*model.SimpleTableInfo{ + require.Nil(t, schema.HandleDDL(job)) + require.Equal(t, schema.SinkTableInfos(), []*model.SimpleTableInfo{ { Schema: "test", Table: "t1", diff --git a/cdc/processor/main_test.go b/cdc/processor/main_test.go new file mode 100644 index 00000000000..5d7ddb7ad5e --- /dev/null +++ b/cdc/processor/main_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package processor + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index 075fbca5fc3..8415af305aa 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -17,9 +17,9 @@ import ( "bytes" "fmt" "math" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" @@ -27,31 +27,29 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type managerSuite struct { +type managerTester struct { manager *Manager state *orchestrator.GlobalReactorState tester *orchestrator.ReactorStateTester } -var _ = check.Suite(&managerSuite{}) - // NewManager4Test creates a new processor manager for test func NewManager4Test( - c *check.C, + t *testing.T, createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), ) *Manager { m := NewManager() m.newProcessor = func(ctx cdcContext.Context) *processor { - return newProcessor4Test(ctx, c, createTablePipeline) + return newProcessor4Test(ctx, t, createTablePipeline) } return m } -func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { - s.manager = NewManager4Test(c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { +func (s *managerTester) resetSuit(ctx cdcContext.Context, t *testing.T) { + s.manager = NewManager4Test(t, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), @@ -62,28 +60,28 @@ func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { }) s.state = orchestrator.NewGlobalState() captureInfoBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() - c.Assert(err, check.IsNil) - s.tester = orchestrator.NewReactorStateTester(c, s.state, map[string]string{ + require.Nil(t, err) + s.tester = orchestrator.NewReactorStateTester(t, s.state, map[string]string{ fmt.Sprintf("/tidb/cdc/capture/%s", ctx.GlobalVars().CaptureInfo.ID): string(captureInfoBytes), }) } -func (s *managerSuite) TestChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an inactive changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, s.manager.processors, 0) // an active changefeed s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { @@ -106,8 +104,8 @@ func (s *managerSuite) TestChangefeed(c *check.C) { s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 1) + require.Nil(t, err) + require.Len(t, s.manager.processors, 1) // processor return errors s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { @@ -121,19 +119,19 @@ func (s *managerSuite) TestChangefeed(c *check.C) { s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, s.manager.processors, 0) } -func (s *managerSuite) TestDebugInfo(c *check.C) { - defer testleak.AfterTest(c)() +func TestDebugInfo(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an active changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") @@ -156,38 +154,38 @@ func (s *managerSuite) TestDebugInfo(c *check.C) { }) s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 1) + require.Len(t, s.manager.processors, 1) done := make(chan struct{}) go func() { defer close(done) for { _, err = s.manager.Tick(ctx, s.state) if err != nil { - c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerrors.ErrReactorFinished.Equal(errors.Cause(err))) return } - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() } }() buf := bytes.NewBufferString("") s.manager.WriteDebugInfo(buf) - c.Assert(len(buf.String()), check.Greater, 0) + require.Greater(t, len(buf.String()), 0) s.manager.AsyncClose() <-done } -func (s *managerSuite) TestClose(c *check.C) { - defer testleak.AfterTest(c)() +func TestClose(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an active changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") @@ -210,13 +208,13 @@ func (s *managerSuite) TestClose(c *check.C) { }) s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 1) + require.Len(t, s.manager.processors, 1) s.manager.AsyncClose() _, err = s.manager.Tick(ctx, s.state) - c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerrors.ErrReactorFinished.Equal(errors.Cause(err))) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 0) + require.Len(t, s.manager.processors, 0) } diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index b171473d167..611befc491c 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -21,7 +21,6 @@ import ( "sync/atomic" "testing" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/entry" @@ -34,21 +33,15 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } - -type processorSuite struct{} - -var _ = check.Suite(&processorSuite{}) - // processor needs to implement TableExecutor. var _ scheduler.TableExecutor = (*processor)(nil) func newProcessor4Test( ctx cdcContext.Context, - c *check.C, + t *testing.T, createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), ) *processor { p := newProcessor(ctx) @@ -59,12 +52,12 @@ func newProcessor4Test( p.sinkManager = &sink.Manager{} p.redoManager = redo.NewDisabledManager() p.createTablePipeline = createTablePipeline - p.schemaStorage = &mockSchemaStorage{c: c, resolvedTs: math.MaxUint64} + p.schemaStorage = &mockSchemaStorage{t: t, resolvedTs: math.MaxUint64} return p } -func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orchestrator.ReactorStateTester) { - p := newProcessor4Test(ctx, c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { +func initProcessor4Test(ctx cdcContext.Context, t *testing.T) (*processor, *orchestrator.ReactorStateTester) { + p := newProcessor4Test(ctx, t, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), @@ -74,7 +67,7 @@ func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orches }, nil }) p.changefeed = orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - return p, orchestrator.NewReactorStateTester(c, p.changefeed, map[string]string{ + return p, orchestrator.NewReactorStateTester(t, p.changefeed, map[string]string{ "/tidb/cdc/capture/" + ctx.GlobalVars().CaptureInfo.ID: `{"id":"` + ctx.GlobalVars().CaptureInfo.ID + `","address":"127.0.0.1:8300"}`, "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":0,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":".","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"open-protocol"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, "/tidb/cdc/job/" + ctx.ChangefeedVars().ID: `{"resolved-ts":0,"checkpoint-ts":0,"admin-job-type":0}`, @@ -142,7 +135,7 @@ type mockSchemaStorage struct { // as we only need ResolvedTs() and DoGC() in unit tests. entry.SchemaStorage - c *check.C + t *testing.T lastGcTs uint64 resolvedTs uint64 } @@ -152,7 +145,7 @@ func (s *mockSchemaStorage) ResolvedTs() uint64 { } func (s *mockSchemaStorage) DoGC(ts uint64) uint64 { - s.c.Assert(s.lastGcTs, check.LessEqual, ts) + require.LessOrEqual(s.t, s.lastGcTs, ts) atomic.StoreUint64(&s.lastGcTs, ts) return ts } @@ -183,15 +176,14 @@ func (a *mockAgent) Close() error { return nil } -func (s *processorSuite) TestCheckTablesNum(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckTablesNum(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, @@ -199,13 +191,13 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { Error: nil, }) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) p.changefeed.Info.StartTs = 66 p.changefeed.Status.CheckpointTs = 88 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 88, ResolvedTs: 88, @@ -214,14 +206,13 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { }) } -func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleTableOperation4SingleTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 90 @@ -236,7 +227,7 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, in processing @@ -247,9 +238,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -260,9 +251,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // add table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -275,9 +266,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { table66 := p.tables[66].(*mockTablePipeline) table66.resolvedTs = 101 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -285,13 +276,13 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { 66: {Delete: false, BoundaryTs: 60, Status: model.OperProcessed}, }, }) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(101)) // finish the operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -310,21 +301,21 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, }, }) - c.Assert(table66.stopTs, check.Equals, uint64(120)) + require.Equal(t, table66.stopTs, uint64(120)) // remove table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, @@ -335,26 +326,25 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { table66.status = tablepipeline.TableStatusStopped table66.checkpointTs = 121 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 121, Status: model.OperFinished}, }, }) - c.Assert(table66.canceled, check.IsTrue) - c.Assert(p.tables[66], check.IsNil) + require.True(t, table66.canceled) + require.Nil(t, p.tables[66]) } -func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleTableOperation4MultiTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 20 @@ -370,7 +360,7 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, in processing @@ -384,9 +374,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -399,9 +389,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: false, BoundaryTs: 40, Status: model.OperProcessed}, }, }) - c.Assert(p.tables, check.HasLen, 4) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs, check.Equals, uint64(30)) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(30)) + require.Len(t, p.tables, 4) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs, uint64(30)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(30)) // add table, push the resolvedTs, finished add table table1 := p.tables[1].(*mockTablePipeline) @@ -419,9 +409,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -433,18 +423,18 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: true, BoundaryTs: 60, Status: model.OperProcessed}, }, }) - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) - c.Assert(table3.stopTs, check.Equals, uint64(60)) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) + require.Equal(t, table3.stopTs, uint64(60)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(101)) // finish remove operations table3.status = tablepipeline.TableStatusStopped table3.checkpointTs = 65 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -456,8 +446,8 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: true, BoundaryTs: 65, Status: model.OperFinished}, }, }) - c.Assert(p.tables, check.HasLen, 3) - c.Assert(table3.canceled, check.IsTrue) + require.Len(t, p.tables, 3) + require.True(t, table3.canceled) // clear finished operations cleanUpFinishedOpOperation(p.changefeed, p.captureInfo.ID, tester) @@ -471,25 +461,25 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, 4: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, }, }) - c.Assert(table1.stopTs, check.Equals, uint64(120)) - c.Assert(table4.stopTs, check.Equals, uint64(120)) - c.Assert(table2.canceled, check.IsTrue) - c.Assert(p.tables, check.HasLen, 2) + require.Equal(t, table1.stopTs, uint64(120)) + require.Equal(t, table4.stopTs, uint64(120)) + require.True(t, table2.canceled) + require.Len(t, p.tables, 2) // remove table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, @@ -503,24 +493,23 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { table4.status = tablepipeline.TableStatusStopped table4.checkpointTs = 122 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 121, Status: model.OperFinished}, 4: {Delete: true, BoundaryTs: 122, Status: model.OperFinished}, }, }) - c.Assert(table1.canceled, check.IsTrue) - c.Assert(table4.canceled, check.IsTrue) - c.Assert(p.tables, check.HasLen, 0) + require.True(t, table1.canceled) + require.True(t, table4.canceled) + require.Len(t, p.tables, 0) } -func (s *processorSuite) TestTableExecutor(c *check.C) { - defer testleak.AfterTest(c)() +func TestTableExecutor(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.newSchedulerEnabled = true p.lazyInit = func(ctx cdcContext.Context) error { p.agent = &mockAgent{executor: p} @@ -530,7 +519,7 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 20 @@ -546,40 +535,38 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() ok, err := p.AddTable(ctx, 1) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 2) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 3) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 4) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) - - c.Assert(p.tables, check.HasLen, 4) + require.Nil(t, err) + require.True(t, ok) + require.Len(t, p.tables, 4) checkpointTs := p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(0)) + require.Equal(t, checkpointTs, uint64(0)) done := p.IsAddTableFinished(ctx, 1) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 2) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 3) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 4) - c.Check(done, check.IsFalse) - - c.Assert(p.tables, check.HasLen, 4) + require.False(t, done) + require.Len(t, p.tables, 4) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, push the resolvedTs, finished add table @@ -598,16 +585,16 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { table4.checkpointTs = 30 done = p.IsAddTableFinished(ctx, 1) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 2) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 3) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 4) - c.Check(done, check.IsTrue) + require.True(t, done) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() table1.checkpointTs = 75 @@ -616,77 +603,78 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { table4.checkpointTs = 75 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(60)) + require.Equal(t, checkpointTs, uint64(60)) - updateChangeFeedPosition(c, tester, ctx.ChangefeedVars().ID, 103, 60) + updateChangeFeedPosition(t, tester, ctx.ChangefeedVars().ID, 103, 60) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() ok, err = p.RemoveTable(ctx, 3) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) + tester.MustApplyPatches() - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) - c.Assert(table3.stopTs, check.Equals, uint64(60)) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) + require.Equal(t, table3.stopTs, uint64(60)) done = p.IsRemoveTableFinished(ctx, 3) - c.Assert(done, check.IsFalse) + require.False(t, done) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(60)) + require.Equal(t, checkpointTs, uint64(60)) // finish remove operations table3.status = tablepipeline.TableStatusStopped table3.checkpointTs = 65 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) + tester.MustApplyPatches() - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) done = p.IsRemoveTableFinished(ctx, 3) - c.Assert(done, check.IsTrue) + require.True(t, done) - c.Assert(p.tables, check.HasLen, 3) - c.Assert(table3.canceled, check.IsTrue) + require.Len(t, p.tables, 3) + require.True(t, table3.canceled) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(75)) + require.Equal(t, checkpointTs, uint64(75)) err = p.Close() - c.Assert(err, check.IsNil) - c.Assert(p.agent, check.IsNil) + require.Nil(t, err) + require.Nil(t, p.agent) } -func (s *processorSuite) TestInitTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestInitTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -696,28 +684,27 @@ func (s *processorSuite) TestInitTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.tables[1], check.Not(check.IsNil)) - c.Assert(p.tables[2], check.Not(check.IsNil)) + require.NotNil(t, p.tables[1]) + require.NotNil(t, p.tables[2]) } -func (s *processorSuite) TestProcessorError(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send a abnormal error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) tester.MustApplyPatches() - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: &model.RunningError{ Addr: "127.0.0.1:0000", Code: "CDC:ErrSinkURIInvalid", @@ -725,30 +712,29 @@ func (s *processorSuite) TestProcessorError(c *check.C) { }, }) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send a normal error p.sendError(context.Canceled) _, err = p.Tick(ctx, p.changefeed) tester.MustApplyPatches() - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: nil, }) } -func (s *processorSuite) TestProcessorExit(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorExit(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // stop the changefeed @@ -762,21 +748,20 @@ func (s *processorSuite) TestProcessorExit(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: nil, }) } -func (s *processorSuite) TestProcessorClose(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorClose(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add tables @@ -787,7 +772,7 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // push the resolvedTs and checkpointTs @@ -801,27 +786,27 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { p.tables[1].(*mockTablePipeline).checkpointTs = 90 p.tables[2].(*mockTablePipeline).checkpointTs = 95 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 90, ResolvedTs: 90, Error: nil, }) - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{1: {StartTs: 20}, 2: {StartTs: 30}}, }) - c.Assert(p.changefeed.Workloads[p.captureInfo.ID], check.DeepEquals, model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) + require.Equal(t, p.changefeed.Workloads[p.captureInfo.ID], model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) - c.Assert(p.Close(), check.IsNil) + require.Nil(t, p.Close()) tester.MustApplyPatches() - c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) - c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) + require.True(t, p.tables[1].(*mockTablePipeline).canceled) + require.True(t, p.tables[2].(*mockTablePipeline).canceled) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add tables @@ -832,30 +817,29 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) tester.MustApplyPatches() - c.Assert(p.Close(), check.IsNil) + require.Nil(t, p.Close()) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].Error, check.DeepEquals, &model.RunningError{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].Error, &model.RunningError{ Addr: "127.0.0.1:0000", Code: "CDC:ErrSinkURIInvalid", Message: "[CDC:ErrSinkURIInvalid]sink uri invalid", }) - c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) - c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) + require.True(t, p.tables[1].(*mockTablePipeline).canceled) + require.True(t, p.tables[2].(*mockTablePipeline).canceled) } -func (s *processorSuite) TestPositionDeleted(c *check.C) { - defer testleak.AfterTest(c)() +func TestPositionDeleted(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} @@ -864,14 +848,14 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // cal position _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) @@ -883,27 +867,26 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { tester.MustApplyPatches() // position created again _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, }) // cal position _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) } -func (s *processorSuite) TestSchemaGC(c *check.C) { - defer testleak.AfterTest(c)() +func TestSchemaGC(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} @@ -913,17 +896,17 @@ func (s *processorSuite) TestSchemaGC(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - updateChangeFeedPosition(c, tester, "changefeed-id-test", 50, 50) + updateChangeFeedPosition(t, tester, "changefeed-id-test", 50, 50) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // GC Ts should be (checkpoint - 1). - c.Assert(p.schemaStorage.(*mockSchemaStorage).lastGcTs, check.Equals, uint64(49)) - c.Assert(p.lastSchemaTs, check.Equals, uint64(49)) + require.Equal(t, p.schemaStorage.(*mockSchemaStorage).lastGcTs, uint64(49)) + require.Equal(t, p.lastSchemaTs, uint64(49)) } func cleanUpFinishedOpOperation(state *orchestrator.ChangefeedReactorState, captureID model.CaptureID, tester *orchestrator.ReactorStateTester) { @@ -941,7 +924,7 @@ func cleanUpFinishedOpOperation(state *orchestrator.ChangefeedReactorState, capt tester.MustApplyPatches() } -func updateChangeFeedPosition(c *check.C, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, resolvedTs, checkpointTs model.Ts) { +func updateChangeFeedPosition(t *testing.T, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, resolvedTs, checkpointTs model.Ts) { key := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangeFeedStatus, ChangefeedID: cfID, @@ -953,14 +936,12 @@ func updateChangeFeedPosition(c *check.C, tester *orchestrator.ReactorStateTeste CheckpointTs: checkpointTs, } valueBytes, err := json.Marshal(cfStatus) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(keyStr, valueBytes) } -func (s *processorSuite) TestIgnorableError(c *check.C) { - defer testleak.AfterTest(c)() - +func TestIgnorableError(t *testing.T) { testCases := []struct { err error ignorable bool @@ -974,14 +955,13 @@ func (s *processorSuite) TestIgnorableError(c *check.C) { {errors.New("test error"), false}, } for _, tc := range testCases { - c.Assert(isProcessorIgnorableError(tc.err), check.Equals, tc.ignorable) + require.Equal(t, isProcessorIgnorableError(tc.err), tc.ignorable) } } -func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { - defer testleak.AfterTest(c)() +func TestUpdateBarrierTs(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 5 status.ResolvedTs = 10 @@ -995,11 +975,11 @@ func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { // init tick, add table OperDispatched. _, err := p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // tick again, add table OperProcessed. _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // Global resolved ts has advanced while schema storage stalls. @@ -1008,16 +988,16 @@ func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { return status, true, nil }) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() tb := p.tables[model.TableID(1)].(*mockTablePipeline) - c.Assert(tb.barrierTs, check.Equals, uint64(10)) + require.Equal(t, tb.barrierTs, uint64(10)) // Schema storage has advanced too. p.schemaStorage.(*mockSchemaStorage).resolvedTs = 15 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() tb = p.tables[model.TableID(1)].(*mockTablePipeline) - c.Assert(tb.barrierTs, check.Equals, uint64(15)) + require.Equal(t, tb.barrierTs, uint64(15)) } diff --git a/pkg/orchestrator/etcd_worker_bank_test.go b/pkg/orchestrator/etcd_worker_bank_test.go index fbbcec988c5..f79f3c18edd 100644 --- a/pkg/orchestrator/etcd_worker_bank_test.go +++ b/pkg/orchestrator/etcd_worker_bank_test.go @@ -20,19 +20,19 @@ import ( "strconv" "strings" "sync" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/log" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) type bankReactorState struct { - c *check.C + t *testing.T account []int pendingPatch [][]DataPatch index int @@ -42,7 +42,7 @@ type bankReactorState struct { const bankTestPrefix = "/ticdc/test/bank/" func (b *bankReactorState) Update(key util.EtcdKey, value []byte, isInit bool) error { - b.c.Assert(strings.HasPrefix(key.String(), bankTestPrefix), check.IsTrue) + require.True(b.t, strings.HasPrefix(key.String(), bankTestPrefix)) indexStr := key.String()[len(bankTestPrefix):] b.account[b.atoi(indexStr)] = b.atoi(string(value)) return nil @@ -62,12 +62,12 @@ func (b *bankReactorState) Check() { if sum != 0 { log.Info("show account", zap.Int("index", b.index), zap.Int("sum", sum), zap.Ints("account", b.account)) } - b.c.Assert(sum, check.Equals, 0, check.Commentf("not ft:%t", b.notFirstTick)) + require.Equal(b.t, sum, 0, fmt.Sprintf("not ft:%t", b.notFirstTick)) } func (b *bankReactorState) atoi(value string) int { i, err := strconv.Atoi(value) - b.c.Assert(err, check.IsNil) + require.Nil(b.t, err) return i } @@ -120,9 +120,7 @@ func (b *bankReactor) Tick(ctx context.Context, state ReactorState) (nextState R return state, err } -func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEtcdBank(t *testing.T) { _ = failpoint.Enable("github.com/pingcap/tiflow/pkg/orchestrator/InjectProgressRequestAfterCommit", "10%return(true)") defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/orchestrator/InjectProgressRequestAfterCommit") @@ -134,7 +132,7 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -144,7 +142,7 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { for i := 0; i < totalAccountNumber; i++ { _, err := cli.Put(ctx, fmt.Sprintf("%s%d", bankTestPrefix, i), "0") - c.Assert(err, check.IsNil) + require.Nil(t, err) } for i := 0; i < workerNumber; i++ { @@ -155,13 +153,13 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { for { worker, err := NewEtcdWorker(cli, bankTestPrefix, &bankReactor{ accountNumber: totalAccountNumber, - }, &bankReactorState{c: c, index: i, account: make([]int, totalAccountNumber)}) - c.Assert(err, check.IsNil) + }, &bankReactorState{t: t, index: i, account: make([]int, totalAccountNumber)}) + require.Nil(t, err) err = worker.Run(ctx, nil, 100*time.Millisecond, "127.0.0.1", "") if err == nil || err.Error() == "etcdserver: request timed out" { continue } - c.Assert(err, check.ErrorMatches, ".*context deadline exceeded.*") + require.Contains(t, err.Error(), "context deadline exceeded") return } }() diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index e65b463c54e..6d77450a012 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -16,6 +16,8 @@ package orchestrator import ( "context" "encoding/json" + "io/ioutil" + "os" "regexp" "strconv" "strings" @@ -23,14 +25,13 @@ import ( "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -43,12 +44,6 @@ const ( totalTicksPerReactor = 1000 ) -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&etcdWorkerSuite{}) - -type etcdWorkerSuite struct{} - type simpleReactor struct { state *simpleReactorState tickCount int @@ -199,26 +194,27 @@ func (s *simpleReactorState) GetPatches() [][]DataPatch { return [][]DataPatch{ret} } -func setUpTest(c *check.C) (func() *etcd.Client, func()) { - dir := c.MkDir() +func setUpTest(t *testing.T) (func() *etcd.Client, func()) { + dir, err := ioutil.TempDir("", "etcd-test") + require.Nil(t, err) url, server, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) + require.Nil(t, err) endpoints := []string{url.String()} return func() *etcd.Client { rawCli, err := clientv3.NewFromURLs(endpoints) - c.Check(err, check.IsNil) + require.Nil(t, err) return etcd.Wrap(rawCli, map[string]prometheus.Counter{}) }, func() { server.Close() + os.RemoveAll(dir) } } -func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { - defer testleak.AfterTest(c)() +func TestEtcdSum(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -226,15 +222,15 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { _ = cli.Unwrap().Close() }() _, err := cli.Put(ctx, testEtcdKeyPrefix+"/sum", "0") - c.Check(err, check.IsNil) + require.Nil(t, err) initArray := make([]int, numValuesPerGroup) jsonStr, err := json.Marshal(initArray) - c.Check(err, check.IsNil) + require.Nil(t, err) for i := 0; i < numGroups; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/"+strconv.Itoa(i), string(jsonStr)) - c.Check(err, check.IsNil) + require.Nil(t, err) } errg, ctx := errgroup.WithContext(ctx) @@ -278,7 +274,7 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { strings.Contains(err.Error(), "etcdserver: request timeout")) { return } - c.Check(err, check.IsNil) + require.Nil(t, err) } type intReactorState struct { @@ -327,20 +323,18 @@ func (r *linearizabilityReactor) Tick(ctx context.Context, state ReactorState) ( return r.state, nil } -func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { - defer testleak.AfterTest(c)() - +func TestLinearizability(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli0 := newClient() cli := newClient() for i := 0; i < 1000; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/lin", strconv.Itoa(i)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } reactor, err := NewEtcdWorker(cli0, testEtcdKeyPrefix+"/lin", &linearizabilityReactor{ @@ -350,7 +344,7 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { val: 0, isUpdated: false, }) - c.Assert(err, check.IsNil) + require.Nil(t, err) errg := &errgroup.Group{} errg.Go(func() error { return reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") @@ -359,16 +353,16 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { time.Sleep(500 * time.Millisecond) for i := 999; i < 2000; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/lin", strconv.Itoa(i)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } err = errg.Wait() - c.Assert(err, check.IsNil) + require.Nil(t, err) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) err = cli0.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type commonReactorState struct { @@ -421,13 +415,11 @@ func (r *finishedReactor) Tick(ctx context.Context, state ReactorState) (nextSta return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestFinished(c *check.C) { - defer testleak.AfterTest(c)() - +func TestFinished(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -437,19 +429,18 @@ func (s *etcdWorkerSuite) TestFinished(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/finished/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "abcabcfin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/finished/key1") + require.Equal(t, string(resp.Kvs[0].Value), "abcabcfin") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type coverReactor struct { @@ -491,13 +482,11 @@ func (r *coverReactor) Tick(ctx context.Context, state ReactorState) (nextState return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestCover(c *check.C) { - defer testleak.AfterTest(c)() - +func TestCover(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -507,20 +496,19 @@ func (s *etcdWorkerSuite) TestCover(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/cover/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "abccbaabccbafinfin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/cover/key1") + require.Equal(t, string(resp.Kvs[0].Value), "abccbaabccbafinfin") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/cover/key2") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "fin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/cover/key2") + require.Equal(t, string(resp.Kvs[0].Value), "fin") err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type emptyTxnReactor struct { @@ -570,13 +558,11 @@ func (r *emptyTxnReactor) Tick(ctx context.Context, state ReactorState) (nextSta return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestEmptyTxn(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEmptyTxn(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -587,19 +573,18 @@ func (s *etcdWorkerSuite) TestEmptyTxn(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/empty_txn/key2") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "123") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/empty_txn/key2") + require.Equal(t, string(resp.Kvs[0].Value), "123") err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type emptyOrNilReactor struct { @@ -639,13 +624,11 @@ func (r *emptyOrNilReactor) Tick(ctx context.Context, state ReactorState) (nextS return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEmptyOrNil(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -655,19 +638,18 @@ func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/emptyOrNil/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/emptyOrNil/key1") + require.Equal(t, string(resp.Kvs[0].Value), "") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type modifyOneReactor struct { @@ -709,20 +691,18 @@ func (r *modifyOneReactor) Tick(ctx context.Context, state ReactorState) (nextSt // TestModifyAfterDelete tests snapshot isolation when there is one modifying transaction delayed in the middle while a deleting transaction // commits. The first transaction should be aborted and retried, and isolation should not be violated. -func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { - defer testleak.AfterTest(c)() - +func TestModifyAfterDelete(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli1 := newClient() cli2 := newClient() _, err := cli1.Put(ctx, "/test/key1", "original value") - c.Assert(err, check.IsNil) + require.Nil(t, err) modifyReactor := &modifyOneReactor{ key: []byte("/test/key1"), @@ -732,14 +712,14 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { worker1, err := NewEtcdWorker(cli1, "/test", modifyReactor, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) + require.Nil(t, err) var wg sync.WaitGroup wg.Add(1) go func() { defer wg.Done() err := worker1.Run(ctx, nil, time.Millisecond*100, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) }() modifyReactor.waitOnCh <- struct{}{} @@ -751,18 +731,18 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { worker2, err := NewEtcdWorker(cli2, "/test", deleteReactor, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = worker2.Run(ctx, nil, time.Millisecond*100, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) modifyReactor.waitOnCh <- struct{}{} wg.Wait() resp, err := cli1.Get(ctx, "/test/key1") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) - c.Assert(worker1.deleteCounter, check.Equals, int64(1)) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) + require.Equal(t, worker1.deleteCounter, int64(1)) _ = cli1.Unwrap().Close() _ = cli2.Unwrap().Close() diff --git a/pkg/orchestrator/main_test.go b/pkg/orchestrator/main_test.go new file mode 100644 index 00000000000..5b44e017491 --- /dev/null +++ b/pkg/orchestrator/main_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package orchestrator + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index 8d51eaa6f4d..442cfec0386 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -15,38 +15,33 @@ package orchestrator import ( "encoding/json" + "fmt" + "testing" "time" "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type stateSuite struct{} - -var _ = check.Suite(&stateSuite{}) - -func (s *stateSuite) TestCheckCaptureAlive(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckCaptureAlive(t *testing.T) { state := NewChangefeedReactorState("test") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") - c.Assert(stateTester.ApplyPatches(), check.ErrorMatches, ".*[CDC:ErrLeaseExpired].*") + require.Contains(t, stateTester.ApplyPatches().Error(), "[CDC:ErrLeaseExpired]") err := stateTester.Update("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`)) - c.Assert(err, check.IsNil) + require.Nil(t, err) state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") stateTester.MustApplyPatches() } -func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedStateUpdate(t *testing.T) { createTime, err := time.Parse("2006-01-02", "2020-02-02") - c.Assert(err, check.IsNil) + require.Nil(t, err) testCases := []struct { changefeedID string updateKey []string @@ -320,24 +315,23 @@ func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { value = nil } err = state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) + require.Nil(t, err) } - c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})), check.IsTrue, - check.Commentf("%d,%s", i, cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})))) + require.True(t, cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})), + fmt.Sprintf("%d,%s", i, cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})))) } } -func (s *stateSuite) TestPatchInfo(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchInfo(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) stateTester.MustApplyPatches() defaultConfig := config.GetDefaultReplicaConfig() - c.Assert(state.Info, check.DeepEquals, &model.ChangeFeedInfo{ + require.Equal(t, state.Info, &model.ChangeFeedInfo{ SinkURI: "123", Engine: model.SortUnified, Config: &config.ReplicaConfig{ @@ -354,7 +348,7 @@ func (s *stateSuite) TestPatchInfo(c *check.C) { return info, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Info, check.DeepEquals, &model.ChangeFeedInfo{ + require.Equal(t, state.Info, &model.ChangeFeedInfo{ SinkURI: "123", StartTs: 6, Engine: model.SortUnified, @@ -371,52 +365,50 @@ func (s *stateSuite) TestPatchInfo(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Info, check.IsNil) + require.Nil(t, state.Info) } -func (s *stateSuite) TestPatchStatus(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchStatus(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{CheckpointTs: 5}, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.DeepEquals, &model.ChangeFeedStatus{CheckpointTs: 5}) + require.Equal(t, state.Status, &model.ChangeFeedStatus{CheckpointTs: 5}) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.ResolvedTs = 6 return status, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.DeepEquals, &model.ChangeFeedStatus{CheckpointTs: 5, ResolvedTs: 6}) + require.Equal(t, state.Status, &model.ChangeFeedStatus{CheckpointTs: 5, ResolvedTs: 6}) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.IsNil) + require.Nil(t, state.Status) } -func (s *stateSuite) TestPatchTaskPosition(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskPosition(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskPosition(captureID1, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { - c.Assert(position, check.IsNil) + require.Nil(t, position) return &model.TaskPosition{ CheckPointTs: 1, }, true, nil }) state.PatchTaskPosition(captureID2, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { - c.Assert(position, check.IsNil) + require.Nil(t, position) return &model.TaskPosition{ CheckPointTs: 2, }, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 1, }, @@ -433,7 +425,7 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { return position, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 3, }, @@ -453,7 +445,7 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { return position, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 3, Count: 6, @@ -461,26 +453,25 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { }) } -func (s *stateSuite) TestPatchTaskStatus(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskStatus(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.TaskStatus{ Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}, }, true, nil }) state.PatchTaskStatus(captureID2, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.TaskStatus{ Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 1}}, }, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}}, captureID2: {Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 1}}}, }) @@ -493,7 +484,7 @@ func (s *stateSuite) TestPatchTaskStatus(c *check.C) { return status, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, captureID2: {Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 2}}}, }) @@ -501,27 +492,26 @@ func (s *stateSuite) TestPatchTaskStatus(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, }) } -func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskWorkload(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskWorkload(captureID1, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) return model.TaskWorkload{45: {Workload: 1}}, true, nil }) state.PatchTaskWorkload(captureID2, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) return model.TaskWorkload{46: {Workload: 1}}, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}}, captureID2: {46: {Workload: 1}}, }) @@ -534,7 +524,7 @@ func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { return workload, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, captureID2: {45: {Workload: 3}, 46: {Workload: 1}}, }) @@ -542,13 +532,12 @@ func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, }) } -func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() +func TestGlobalStateUpdate(t *testing.T) { testCases := []struct { updateKey []string updateValue []string @@ -647,26 +636,25 @@ func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { value = nil } err := state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) + require.Nil(t, err) } - c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})), check.IsTrue, - check.Commentf("%s", cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})))) + require.True(t, cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})), + cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{}))) } } -func (s *stateSuite) TestCaptureChangeHooks(c *check.C) { - defer testleak.AfterTest(c)() +func TestCaptureChangeHooks(t *testing.T) { state := NewGlobalState() var callCount int state.onCaptureAdded = func(captureID model.CaptureID, addr string) { callCount++ - c.Check(captureID, check.Equals, "capture-1") - c.Check(addr, check.Equals, "ip-1:8300") + require.Equal(t, captureID, "capture-1") + require.Equal(t, addr, "ip-1:8300") } state.onCaptureRemoved = func(captureID model.CaptureID) { callCount++ - c.Check(captureID, check.Equals, "capture-1") + require.Equal(t, captureID, "capture-1") } captureInfo := &model.CaptureInfo{ @@ -674,21 +662,20 @@ func (s *stateSuite) TestCaptureChangeHooks(c *check.C) { AdvertiseAddr: "ip-1:8300", } captureInfoBytes, err := json.Marshal(captureInfo) - c.Check(err, check.IsNil) + require.Nil(t, err) err = state.Update(util.NewEtcdKey(etcd.CaptureInfoKeyPrefix+"/capture-1"), captureInfoBytes, false) - c.Check(err, check.IsNil) - c.Check(callCount, check.Equals, 1) + require.Nil(t, err) + require.Equal(t, callCount, 1) err = state.Update(util.NewEtcdKey(etcd.CaptureInfoKeyPrefix+"/capture-1"), nil /* delete */, false) - c.Check(err, check.IsNil) - c.Check(callCount, check.Equals, 2) + require.Nil(t, err) + require.Equal(t, callCount, 2) } -func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckChangefeedNormal(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.CheckChangefeedNormal() stateTester.MustApplyPatches() state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { @@ -699,7 +686,7 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + require.Equal(t, state.Status.ResolvedTs, uint64(1)) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { info.AdminJobType = model.AdminStop @@ -711,7 +698,7 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + require.Equal(t, state.Status.ResolvedTs, uint64(1)) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.ResolvedTs = 2 @@ -719,5 +706,5 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(2)) + require.Equal(t, state.Status.ResolvedTs, uint64(2)) } diff --git a/pkg/orchestrator/reactor_state_tester.go b/pkg/orchestrator/reactor_state_tester.go index 72b5f5e2c0a..5e724616697 100644 --- a/pkg/orchestrator/reactor_state_tester.go +++ b/pkg/orchestrator/reactor_state_tester.go @@ -14,55 +14,57 @@ package orchestrator import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/errors" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/stretchr/testify/require" ) // ReactorStateTester is a helper struct for unit-testing an implementer of ReactorState type ReactorStateTester struct { - c *check.C + t *testing.T state ReactorState kvEntries map[string]string } // NewReactorStateTester creates a new ReactorStateTester -func NewReactorStateTester(c *check.C, state ReactorState, initKVEntries map[string]string) *ReactorStateTester { +func NewReactorStateTester(t *testing.T, state ReactorState, initKVEntries map[string]string) *ReactorStateTester { if initKVEntries == nil { initKVEntries = make(map[string]string) } for k, v := range initKVEntries { err := state.Update(util.NewEtcdKey(k), []byte(v), true) - c.Assert(err, check.IsNil) + require.NoError(t, err) } return &ReactorStateTester{ - c: c, + t: t, state: state, kvEntries: initKVEntries, } } // Update is used to update keys in the mocked kv-store. -func (t *ReactorStateTester) Update(key string, value []byte) error { +func (r *ReactorStateTester) Update(key string, value []byte) error { k := util.NewEtcdKey(key) - err := t.state.Update(k, value, false) + err := r.state.Update(k, value, false) if err != nil { return errors.Trace(err) } if value != nil { - t.kvEntries[key] = string(value) + r.kvEntries[key] = string(value) } else { - delete(t.kvEntries, key) + delete(r.kvEntries, key) } return nil } // ApplyPatches calls the GetPatches method on the ReactorState and apply the changes to the mocked kv-store. -func (t *ReactorStateTester) ApplyPatches() error { - patchGroups := t.state.GetPatches() +func (r *ReactorStateTester) ApplyPatches() error { + patchGroups := r.state.GetPatches() for _, patches := range patchGroups { - err := t.applyPatches(patches) + err := r.applyPatches(patches) if err != nil { return err } @@ -70,11 +72,11 @@ func (t *ReactorStateTester) ApplyPatches() error { return nil } -func (t *ReactorStateTester) applyPatches(patches []DataPatch) error { +func (r *ReactorStateTester) applyPatches(patches []DataPatch) error { RetryLoop: for { tmpKVEntries := make(map[util.EtcdKey][]byte) - for k, v := range t.kvEntries { + for k, v := range r.kvEntries { tmpKVEntries[util.NewEtcdKey(k)] = []byte(v) } changedSet := make(map[util.EtcdKey]struct{}) @@ -89,14 +91,14 @@ RetryLoop: } } for k := range changedSet { - err := t.state.Update(k, tmpKVEntries[k], false) + err := r.state.Update(k, tmpKVEntries[k], false) if err != nil { return err } if value := tmpKVEntries[k]; value != nil { - t.kvEntries[k.String()] = string(value) + r.kvEntries[k.String()] = string(value) } else { - delete(t.kvEntries, k.String()) + delete(r.kvEntries, k.String()) } } return nil @@ -104,16 +106,16 @@ RetryLoop: } // MustApplyPatches calls ApplyPatches and must successfully -func (t *ReactorStateTester) MustApplyPatches() { - t.c.Assert(t.ApplyPatches(), check.IsNil) +func (r *ReactorStateTester) MustApplyPatches() { + require.Nil(r.t, r.ApplyPatches()) } // MustUpdate calls Update and must successfully -func (t *ReactorStateTester) MustUpdate(key string, value []byte) { - t.c.Assert(t.Update(key, value), check.IsNil) +func (r *ReactorStateTester) MustUpdate(key string, value []byte) { + require.Nil(r.t, r.Update(key, value)) } // KVEntries returns the contents of the mocked KV store. -func (t *ReactorStateTester) KVEntries() map[string]string { - return t.kvEntries +func (r *ReactorStateTester) KVEntries() map[string]string { + return r.kvEntries } From 462074a08c09b3e376479682e9c51f4232450663 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Wed, 26 Jan 2022 16:56:46 +0800 Subject: [PATCH 26/72] docs(ticdc): add design doc for multiple topics (#4422) ref pingcap/tiflow#4423 --- ...022-01-20-ticdc-mq-sink-multiple-topics.md | 146 ++++++++++++++++++ 1 file changed, 146 insertions(+) create mode 100644 docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md diff --git a/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md b/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md new file mode 100644 index 00000000000..318fecc0bd7 --- /dev/null +++ b/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md @@ -0,0 +1,146 @@ +# TiCDC supports multi-topic dispatch + +- Author(s): [hi-rustin](https://github.com/hi-rustin) +- Tracking Issue: https://github.com/pingcap/tiflow/issues/4423 + +## Table of Contents + +- [Introduction](#introduction) +- [Motivation or Background](#motivation-or-background) +- [Detailed Design](#detailed-design) +- [Test Design](#test-design) + - [Functional Tests](#functional-tests) + - [Scenario Tests](#scenario-tests) + - [Compatibility Tests](#compatibility-tests) + - [Benchmark Tests](#benchmark-tests) +- [Impacts & Risks](#impacts--risks) +- [Investigation & Alternatives](#investigation--alternatives) +- [Unresolved Questions](#unresolved-questions) + +## Introduction + +This document provides a complete design on implementing multi-topic support in TiCDC MQ Sink. + +## Motivation or Background + +TiCDC MQ Sink only supports sending messages to a single topic, but in the MQ Sink usage scenario, we send data to +systems like [Flink], which requires us to support multiple topics, each topic as a data source. + +## Detailed Design + +This solution will introduce a new configuration to the configuration file that specifies which topic the sink will send +the table data to. + +We will continue to keep the original topic configuration in the sinkURI, which serves two purposes. + +1. when there is no new configuration or the configuration does not match, the data will be sent to that default topic. +2. DDLs of the schema level will be sent to this topic by default. + +### Topic dispatch configuration format + +This configuration will be added to the TiCDC changefeed configuration file. + +```toml +[sink] +dispatchers = [ + { matcher = ['test1.*', 'test2.*'], dispatcher = "ts", topic = "Topic dispatch expression" }, + { matcher = ['test3.*', 'test4.*'], dispatcher = "rowid", topic = "Topic dispatch expression" }, + { matcher = ['test1.*', 'test5.*'], dispatcher = "ts", topic = "Topic dispatch expression" }, +] +``` + +A new topic field has been added to dispatchers that will specify the topic dispatching rules for these tables. + +### Topic dispatch expression details + +The expression format looks like `flink_{schema}{table}`. This expression consists of two keywords and the `flink_` +prefix. + +Two keywords(case-insensitive): + +| Keyword | Description | Required | +| -------- | ---------------------- | -------- | +| {schema} | the name of the schema | no | +| {table} | the name of the table | no | + +> When neither keyword is filled in, it is equivalent to sending the data to a fixed topic. + +`flink_` is the user-definable part, where the user can fill in the expression with a custom string. + +Some examples: + +```toml +[sink] +dispatchers = [ + { matcher = ['test1.table1', 'test2.table1'], topic = "{schema}_{table}" }, + { matcher = ['test3.*', 'test4.*'], topic = "flink{schema}" }, + { matcher = ['test1.*', 'test5.*'], topic = "test-cdc" }, +] +``` + +- matcher = ['test1.*', 'test2.*'], topic = "{schema}\_{table}" + - Send the data from `test1.table1` and `test2.table1` to the `test1_table1` and `test2_table1` topics, respectively +- matcher = ['test3.*', 'test4.*'], topic = "flink\_{schema}" + - Send the data from all tables in `test3` and `test4` to `flinktest3` and `flinktest4` topics, respectively +- matcher = ['test1.*', 'test5.*'], topic = "test-cdc" + - Send the data of all the tables in `test1` (except `test1.table1`) and `test5` to the `test-cdc` topic + - The `table1` in `test1` is sent to the `test1_table1` topic, because for tables matching multiple matcher rules, the + topic expression corresponding to the top matcher prevails + +### DDL dispatch rules + +- schema-level DDLs that are sent to the default topic +- table-level DDLs, will be sent to the matching topic, if there is no topic match, it will be sent to the default topic + +## Test Design + +This functionality will be mainly covered by unit and integration tests. + +### Functional Tests + +#### Unit test + +Coverage should be more than 75% in new added code. + +#### Integration test + +Can pass all existing integration tests when changefeed without topic dispatch configuration. In addition, we will +integrate [Flink] into our integration tests to verify multi-topic functionality. + +### Scenario Tests + +We will test the scenario of using `canal-json` format to connect data to [Flink]. + +### Compatibility Tests + +#### Compatibility with other features/components + +For TiCDC's original support of only a single topic, we're not going to break it this time. When you pass only the +default topic in the sinkURI and there is no topic expression configuration, it will work as is. + +#### Upgrade compatibility + +When not configured, it works as a single topic, so just add the configuration and create a new changefeed after the +upgrade. + +#### Downgrade compatibility + +The new configuration is not recognized by the old TiCDC, so you need to remove the changefeed before downgrading. + +### Benchmark Tests + +N/A + +## Impacts & Risks + +N/A + +## Investigation & Alternatives + +N/A + +## Unresolved Questions + +N/A + +[flink]: https://flink.apache.org/ From c56c1c83de0c51e6e84a39116fcdd242247a33df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Wed, 26 Jan 2022 18:12:45 +0800 Subject: [PATCH 27/72] github(dm): add release-note-none for robot pull request (#4481) ref pingcap/tiflow#4478 --- .github/dependabot.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 .github/dependabot.yml diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 00000000000..666116a420f --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,12 @@ +version: 2 +updates: + - package-ecosystem: "npm" + directory: "/dm/ui" + # Specify labels for npm pull requests. + labels: + - "dependencies" + - "javascript" + - "release-note-none" + - "skip-issue-check" + # We use this option to make these configurations valid only for PRs with security issues. + open-pull-requests-limit: 0 From f917d3651530db961d97cf7b704a306d11e46ed5 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 26 Jan 2022 19:34:47 +0800 Subject: [PATCH 28/72] cmd,sink(ticdc): validate changefeed params and revise error message (#4482) close pingcap/tiflow#1716, close pingcap/tiflow#1718, close pingcap/tiflow#1719, close pingcap/tiflow#4472 --- cdc/sink/mysql.go | 1 + cdc/sink/mysql_params.go | 44 ++++++++++++++++++++- cdc/sink/mysql_params_test.go | 48 ++++++++++++++++++++++- pkg/cmd/cli/cli_changefeed_create.go | 23 ++++++++--- pkg/cmd/cli/cli_changefeed_create_test.go | 34 ++++++++++++++++ pkg/cmd/factory/factory_impl.go | 9 +++-- pkg/version/check.go | 11 ++---- pkg/version/check_test.go | 2 +- 8 files changed, 152 insertions(+), 20 deletions(-) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index d2946f85197..1b2dc9f5602 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -116,6 +116,7 @@ func newMySQLSink( } // create test db used for parameter detection + // Refer https://github.com/go-sql-driver/mysql#parameters if dsn.Params == nil { dsn.Params = make(map[string]string, 1) } diff --git a/cdc/sink/mysql_params.go b/cdc/sink/mysql_params.go index a374dbb2538..9f0fec92a41 100644 --- a/cdc/sink/mysql_params.go +++ b/cdc/sink/mysql_params.go @@ -35,6 +35,10 @@ const ( // expose these two variables for redo log applier DefaultWorkerCount = 16 DefaultMaxTxnRow = 256 + // The upper limit of max worker counts. + maxWorkerCount = 1024 + // The upper limit of max txn rows. + maxMaxTxnRow = 2048 defaultDMLMaxRetryTime = 8 defaultDDLMaxRetryTime = 20 @@ -113,9 +117,16 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if err != nil { return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) } - if c > 0 { - params.workerCount = c + if c <= 0 { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, + fmt.Errorf("invalid worker-count %d, which must be greater than 0", c)) } + if c > maxWorkerCount { + log.Warn("worker-count too large", + zap.Int("original", c), zap.Int("override", maxWorkerCount)) + c = maxWorkerCount + } + params.workerCount = c } s = sinkURI.Query().Get("max-txn-row") if s != "" { @@ -123,6 +134,15 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if err != nil { return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) } + if c <= 0 { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, + fmt.Errorf("invalid max-txn-row %d, which must be greater than 0", c)) + } + if c > maxMaxTxnRow { + log.Warn("max-txn-row too large", + zap.Int("original", c), zap.Int("override", maxMaxTxnRow)) + c = maxMaxTxnRow + } params.maxTxnRow = c } s = sinkURI.Query().Get("tidb-txn-mode") @@ -182,6 +202,14 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if s == "" { params.timezone = "" } else { + value, err := url.QueryUnescape(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } + _, err = time.LoadLocation(value) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.timezone = fmt.Sprintf(`"%s"`, s) } } else { @@ -195,14 +223,26 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string // To keep the same style with other sink parameters, we use dash as word separator. s = sinkURI.Query().Get("read-timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.readTimeout = s } s = sinkURI.Query().Get("write-timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.writeTimeout = s } s = sinkURI.Query().Get("timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.dialTimeout = s } diff --git a/cdc/sink/mysql_params_test.go b/cdc/sink/mysql_params_test.go index 91f8a0e6d00..075e5d41d26 100644 --- a/cdc/sink/mysql_params_test.go +++ b/cdc/sink/mysql_params_test.go @@ -172,17 +172,63 @@ func TestParseSinkURITimezone(t *testing.T) { } } +func TestParseSinkURIOverride(t *testing.T) { + defer testleak.AfterTestT(t)() + cases := []struct { + uri string + checker func(*sinkParams) + }{{ + uri: "mysql://127.0.0.1:3306/?worker-count=2147483648", // int32 max + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.workerCount, maxWorkerCount) + }, + }, { + uri: "mysql://127.0.0.1:3306/?max-txn-row=2147483648", // int32 max + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.maxTxnRow, maxMaxTxnRow) + }, + }, { + uri: "mysql://127.0.0.1:3306/?tidb-txn-mode=badmode", + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.tidbTxnMode, defaultTiDBTxnMode) + }, + }} + ctx := context.TODO() + opts := map[string]string{OptChangefeedID: "changefeed-01"} + var uri *url.URL + var err error + for _, cs := range cases { + if cs.uri != "" { + uri, err = url.Parse(cs.uri) + require.Nil(t, err) + } else { + uri = nil + } + p, err := parseSinkURIToParams(ctx, uri, opts) + require.Nil(t, err) + cs.checker(p) + } +} + func TestParseSinkURIBadQueryString(t *testing.T) { defer testleak.AfterTestT(t)() uris := []string{ "", "postgre://127.0.0.1:3306", "mysql://127.0.0.1:3306/?worker-count=not-number", + "mysql://127.0.0.1:3306/?worker-count=-1", + "mysql://127.0.0.1:3306/?worker-count=0", "mysql://127.0.0.1:3306/?max-txn-row=not-number", + "mysql://127.0.0.1:3306/?max-txn-row=-1", + "mysql://127.0.0.1:3306/?max-txn-row=0", "mysql://127.0.0.1:3306/?ssl-ca=only-ca-exists", "mysql://127.0.0.1:3306/?batch-replace-enable=not-bool", "mysql://127.0.0.1:3306/?batch-replace-enable=true&batch-replace-size=not-number", "mysql://127.0.0.1:3306/?safe-mode=not-bool", + "mysql://127.0.0.1:3306/?time-zone=badtz", + "mysql://127.0.0.1:3306/?write-timeout=badduration", + "mysql://127.0.0.1:3306/?read-timeout=badduration", + "mysql://127.0.0.1:3306/?timeout=badduration", } ctx := context.TODO() opts := map[string]string{OptChangefeedID: "changefeed-01"} @@ -196,7 +242,7 @@ func TestParseSinkURIBadQueryString(t *testing.T) { uri = nil } _, err = parseSinkURIToParams(ctx, uri, opts) - require.NotNil(t, err) + require.Error(t, err) } } diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index 9f8667026f0..1f819a5ced8 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -163,17 +163,18 @@ func (o *createChangefeedOptions) complete(ctx context.Context, f factory.Factor } o.startTs = oracle.ComposeTS(ts, logical) } - - return o.completeCfg(ctx, cmd) -} - -// completeCfg complete the replica config from file and cmd flags. -func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Command) error { _, captureInfos, err := o.etcdClient.GetCaptures(ctx) if err != nil { return err } + return o.completeCfg(cmd, captureInfos) +} + +// completeCfg complete the replica config from file and cmd flags. +func (o *createChangefeedOptions) completeCfg( + cmd *cobra.Command, captureInfos []*model.CaptureInfo, +) error { cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return errors.Trace(err) @@ -227,6 +228,16 @@ func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Co } } + switch o.commonChangefeedOptions.sortEngine { + case model.SortInMemory: + case model.SortInFile: + case model.SortUnified: + default: + log.Warn("invalid sort-engine, use Unified Sorter by default", + zap.String("invalidSortEngine", o.commonChangefeedOptions.sortEngine)) + o.commonChangefeedOptions.sortEngine = model.SortUnified + } + if o.commonChangefeedOptions.sortEngine == model.SortUnified && !cdcClusterVer.ShouldEnableUnifiedSorterByDefault() { o.commonChangefeedOptions.sortEngine = model.SortInMemory log.Warn("The TiCDC cluster is built from an older version, disabling Unified Sorter by default", diff --git a/pkg/cmd/cli/cli_changefeed_create_test.go b/pkg/cmd/cli/cli_changefeed_create_test.go index 658346d38e6..6e877f617a9 100644 --- a/pkg/cmd/cli/cli_changefeed_create_test.go +++ b/pkg/cmd/cli/cli_changefeed_create_test.go @@ -20,8 +20,10 @@ import ( "testing" "github.com/pingcap/check" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" ) @@ -65,3 +67,35 @@ func (s *changefeedSuite) TestStrictDecodeConfig(c *check.C) { c.Assert(err, check.NotNil) c.Assert(err, check.ErrorMatches, ".*CDC:ErrFilterRuleInvalid.*") } + +func (s *changefeedSuite) TestInvalidSortEngine(c *check.C) { + defer testleak.AfterTest(c)() + + cases := []struct { + input string + expect model.SortEngine + }{{ + input: "invalid", + expect: model.SortUnified, + }, { + input: "memory", + expect: model.SortInMemory, + }, { + input: "file", + expect: model.SortInFile, + }, { + input: "unified", + expect: model.SortUnified, + }} + for _, cs := range cases { + cmd := new(cobra.Command) + o := newChangefeedCommonOptions() + o.addFlags(cmd) + c.Assert(cmd.ParseFlags([]string{"--sort-engine=" + cs.input}), check.IsNil) + opt := newCreateChangefeedOptions(o) + err := opt.completeCfg(cmd, + []*model.CaptureInfo{{Version: version.MinTiCDCVersion.String()}}) + c.Assert(err, check.IsNil) + c.Assert(opt.commonChangefeedOptions.sortEngine, check.Equals, cs.expect) + } +} diff --git a/pkg/cmd/factory/factory_impl.go b/pkg/cmd/factory/factory_impl.go index fc4e9c77518..737e5e8f3ed 100644 --- a/pkg/cmd/factory/factory_impl.go +++ b/pkg/cmd/factory/factory_impl.go @@ -93,7 +93,8 @@ func (f *factoryImpl) EtcdClient() (*etcd.CDCEtcdClient, error) { } logConfig.Level = logLevel - pdEndpoints := strings.Split(f.GetPdAddr(), ",") + pdAddr := f.GetPdAddr() + pdEndpoints := strings.Split(pdAddr, ",") etcdClient, err := clientv3.New(clientv3.Config{ Context: ctx, @@ -118,7 +119,8 @@ func (f *factoryImpl) EtcdClient() (*etcd.CDCEtcdClient, error) { }, }) if err != nil { - return nil, err + return nil, errors.Annotatef(err, + "fail to open PD client, please check pd address \"%s\"", pdAddr) } client := etcd.NewCDCEtcdClient(ctx, etcdClient) @@ -156,7 +158,8 @@ func (f factoryImpl) PdClient() (pd.Client, error) { }), )) if err != nil { - return nil, errors.Annotatef(err, "fail to open PD client, pd=\"%s\"", pdAddr) + return nil, errors.Annotatef(err, + "fail to open PD client, please check pd address \"%s\"", pdAddr) } err = version.CheckClusterVersion(ctx, pdClient, pdEndpoints, credential, true) diff --git a/pkg/version/check.go b/pkg/version/check.go index 436e6306927..6647bc10ec0 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -35,7 +35,6 @@ import ( var ( // minPDVersion is the version of the minimal compatible PD. - // TODO bump 5.2.0-alpha once PD releases. minPDVersion *semver.Version = semver.New("5.1.0-alpha") // maxPDVersion is the version of the maximum compatible PD. // Compatible versions are in [minPDVersion, maxPDVersion) @@ -43,16 +42,14 @@ var ( maxPDVersion *semver.Version = semver.New("9999.0.0") // MinTiKVVersion is the version of the minimal compatible TiKV. - // TODO bump 5.2.0-alpha once TiKV releases. MinTiKVVersion *semver.Version = semver.New("5.1.0-alpha") // maxTiKVVersion is the version of the maximum compatible TiKV. // Compatible versions are in [MinTiKVVersion, maxTiKVVersion) // 9999.0.0 disables the check effectively in the master branch. maxTiKVVersion *semver.Version = semver.New("9999.0.0") - // minTiCDCVersion is the version of the minimal compatible TiCDC. - // TODO bump 5.2.0-alpha once TiCDC releases. - minTiCDCVersion *semver.Version = semver.New("5.1.0-alpha") + // MinTiCDCVersion is the version of the minimal compatible TiCDC. + MinTiCDCVersion *semver.Version = semver.New("5.1.0-alpha") // Compatible versions are in [MinTiCDCVersion, MaxTiCDCVersion) // 9999.0.0 disables the check effectively in the master branch. maxTiCDCVersion *semver.Version = semver.New("9999.0.0") @@ -266,11 +263,11 @@ func CheckTiCDCClusterVersion(cdcClusterVer TiCDCClusterVersion) (unknown bool, return true, nil } ver := cdcClusterVer.Version - minOrd := ver.Compare(*minTiCDCVersion) + minOrd := ver.Compare(*MinTiCDCVersion) if minOrd < 0 { arg := fmt.Sprintf("TiCDC %s is not supported, the minimal compatible version is %s"+ "try tiup ctl:%s cdc [COMMAND]", - ver, minTiCDCVersion, ver) + ver, MinTiCDCVersion, ver) return false, cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) } maxOrd := ver.Compare(*maxTiCDCVersion) diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index 99878761e32..feaae574ca3 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -338,7 +338,7 @@ func TestCheckTiCDCClusterVersion(t *testing.T) { expectedUnknown: true, }, { - cdcClusterVersion: TiCDCClusterVersion{Version: minTiCDCVersion}, + cdcClusterVersion: TiCDCClusterVersion{Version: MinTiCDCVersion}, expectedErr: "", expectedUnknown: false, }, From cdc9fa5524731d9dce557faa784f5c45ddb598db Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 27 Jan 2022 11:59:13 +0800 Subject: [PATCH 29/72] build(deps): bump node-fetch from 2.6.6 to 2.6.7 in /dm/ui (#4441) ref pingcap/tiflow#4478 --- dm/ui/yarn.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dm/ui/yarn.lock b/dm/ui/yarn.lock index e2d36417eab..3bf47b4aa40 100644 --- a/dm/ui/yarn.lock +++ b/dm/ui/yarn.lock @@ -3167,9 +3167,9 @@ node-fetch-h2@^2.3.0: http2-client "^1.2.5" node-fetch@^2.6.1: - version "2.6.6" - resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.6.tgz#1751a7c01834e8e1697758732e9efb6eeadfaf89" - integrity sha512-Z8/6vRlTUChSdIgMa51jxQ4lrw/Jy5SOW10ObaA47/RElsAN2c5Pn8bTgFGWn/ibwzXTE8qwr1Yzx28vsecXEA== + version "2.6.7" + resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.7.tgz#24de9fba827e3b4ae44dc8b20256a379160052ad" + integrity sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ== dependencies: whatwg-url "^5.0.0" From 650d748c9a7309276a2f758a4fae441de7f5cb88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Thu, 27 Jan 2022 12:49:12 +0800 Subject: [PATCH 30/72] github(dm): add missing schedule property (#4487) ref pingcap/tiflow#4478 --- .github/dependabot.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 666116a420f..6fcbe6d6bd6 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -2,6 +2,8 @@ version: 2 updates: - package-ecosystem: "npm" directory: "/dm/ui" + schedule: + interval: "weekly" # Specify labels for npm pull requests. labels: - "dependencies" From 757decd8aa030febe8536c9fdea15d35c6bfb224 Mon Sep 17 00:00:00 2001 From: Ehco Date: Thu, 27 Jan 2022 17:59:12 +0800 Subject: [PATCH 31/72] webui(dm): always redircet to index page (#4350) ref pingcap/tiflow#3583 --- dm/ui/server.go | 26 ++++++++++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/dm/ui/server.go b/dm/ui/server.go index baeb5fd8c1f..722a6b7d2fb 100644 --- a/dm/ui/server.go +++ b/dm/ui/server.go @@ -18,6 +18,7 @@ package ui import ( "io/fs" "net/http" + "strings" "github.com/gin-gonic/gin" "github.com/pingcap/tiflow/dm/openapi" @@ -25,20 +26,41 @@ import ( "go.uber.org/zap" ) +const ( + buildPath = "dist" + assetsPath = "assets" + indexPath = "/dashboard/" +) + // WebUIAssetsHandler returns a http handler for serving static files. func WebUIAssetsHandler() http.FileSystem { - stripped, err := fs.Sub(WebUIAssets, "dist") + stripped, err := fs.Sub(WebUIAssets, buildPath) if err != nil { panic(err) // this should never happen } return http.FS(stripped) } +// we need this to handle this case: user want to access /dashboard/source.html/ but webui is a single page app, +// and it only can handle requests in index page, so we need to redirect to index page. +func alwaysRedirect(path string) gin.HandlerFunc { + return func(c *gin.Context) { + // note that static file like css and js under the assets folder should not be redirected. + if c.Request.URL.Path != path && !strings.Contains(c.Request.URL.Path, assetsPath) { + c.Redirect(http.StatusPermanentRedirect, path) + c.AbortWithStatus(http.StatusPermanentRedirect) + } else { + c.Next() + } + } +} + // InitWebUIRouter initializes the webUI router. func InitWebUIRouter() *gin.Engine { router := gin.New() router.Use(gin.Recovery()) + router.Use(alwaysRedirect(indexPath)) router.Use(openapi.ZapLogger(log.L().WithFields(zap.String("component", "webui")).Logger)) - router.StaticFS("/dashboard/", WebUIAssetsHandler()) + router.StaticFS(indexPath, WebUIAssetsHandler()) return router } From 14f8c4fd7526bf24bb112b3ee30c2d1ac63c4b40 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Fri, 28 Jan 2022 00:07:12 +0800 Subject: [PATCH 32/72] sink (ticdc): reduce the memory consumption by remove redundant eventBuffer in sinkNode (#4452) ref pingcap/tiflow#4222 --- cdc/processor/pipeline/sink.go | 41 +++++++++++------------------ cdc/processor/pipeline/sink_test.go | 36 ++++++++++++------------- cdc/processor/pipeline/table.go | 4 +-- 3 files changed, 36 insertions(+), 45 deletions(-) diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index a0f72e7b68a..914d92bb6f5 100755 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -75,8 +75,7 @@ type sinkNode struct { targetTs model.Ts barrierTs model.Ts - eventBuffer []*model.PolymorphicEvent - rowBuffer []*model.RowChangedEvent + rowBuffer []*model.RowChangedEvent flowController tableFlowController @@ -128,6 +127,8 @@ func (n *sinkNode) stop(ctx context.Context) (err error) { return } +// flushSink emits all rows in rowBuffer to the backend sink and flushes +// the backend sink. func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err error) { defer func() { if err != nil { @@ -147,7 +148,7 @@ func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err erro if resolvedTs <= n.checkpointTs { return nil } - if err := n.emitRow2Sink(ctx); err != nil { + if err := n.emitRowToSink(ctx); err != nil { return errors.Trace(err) } checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, n.tableID, resolvedTs) @@ -172,7 +173,8 @@ func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err erro return nil } -func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) error { +// addRowToBuffer checks event and adds event.Row to rowBuffer. +func (n *sinkNode) addRowToBuffer(ctx context.Context, event *model.PolymorphicEvent) error { if event == nil || event.Row == nil { log.Warn("skip emit nil event", zap.Any("event", event)) return nil @@ -182,7 +184,7 @@ func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) preColLen := len(event.Row.PreColumns) // Some transactions could generate empty row change event, such as // begin; insert into t (id) values (1); delete from t where id=1; commit; - // Just ignore these row changed events + // Just ignore these row changed events. if colLen == 0 && preColLen == 0 { log.Warn("skip emit empty row event", zap.Any("event", event)) return nil @@ -198,18 +200,18 @@ func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) return errors.Trace(err) } // NOTICE: Please do not change the order, the delete event always comes before the insert event. - n.eventBuffer = append(n.eventBuffer, deleteEvent, insertEvent) + n.rowBuffer = append(n.rowBuffer, deleteEvent.Row, insertEvent.Row) } else { // If the handle key columns are not updated, PreColumns is directly ignored. event.Row.PreColumns = nil - n.eventBuffer = append(n.eventBuffer, event) + n.rowBuffer = append(n.rowBuffer, event.Row) } } else { - n.eventBuffer = append(n.eventBuffer, event) + n.rowBuffer = append(n.rowBuffer, event.Row) } - if len(n.eventBuffer) >= defaultSyncResolvedBatch { - if err := n.emitRow2Sink(ctx); err != nil { + if len(n.rowBuffer) >= defaultSyncResolvedBatch { + if err := n.emitRowToSink(ctx); err != nil { return errors.Trace(err) } } @@ -280,7 +282,7 @@ func splitUpdateEvent(updateEvent *model.PolymorphicEvent) (*model.PolymorphicEv return &deleteEvent, &insertEvent, nil } -// clear event buffer and row buffer. +// clearBuffers clears rowBuffer. // Also, it dereferences data that are held by buffers. func (n *sinkNode) clearBuffers() { // Do not hog memory. @@ -292,21 +294,10 @@ func (n *sinkNode) clearBuffers() { } n.rowBuffer = n.rowBuffer[:0] } - - if cap(n.eventBuffer) > defaultSyncResolvedBatch { - n.eventBuffer = make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) - } else { - for i := range n.eventBuffer { - n.eventBuffer[i] = nil - } - n.eventBuffer = n.eventBuffer[:0] - } } -func (n *sinkNode) emitRow2Sink(ctx context.Context) error { - for _, ev := range n.eventBuffer { - n.rowBuffer = append(n.rowBuffer, ev.Row) - } +// emitRowToSink emits the rows in rowBuffer to backend sink. +func (n *sinkNode) emitRowToSink(ctx context.Context) error { failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") time.Sleep(10 * time.Second) @@ -346,7 +337,7 @@ func (n *sinkNode) HandleMessage(ctx context.Context, msg pipeline.Message) (boo atomic.StoreUint64(&n.resolvedTs, msg.PolymorphicEvent.CRTs) return true, nil } - if err := n.emitEvent(ctx, event); err != nil { + if err := n.addRowToBuffer(ctx, event); err != nil { return false, errors.Trace(err) } case pipeline.MessageTypeTick: diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index b4227fe392c..172c14dc0c8 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -372,7 +372,7 @@ func TestIgnoreEmptyRowChangeEvent(t *testing.T) { // empty row, no Columns and PreColumns. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) } func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { @@ -391,7 +391,7 @@ func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { // nil row. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) columns := []*model.Column{ { @@ -424,9 +424,9 @@ func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) - require.Equal(t, 1, len(node.eventBuffer)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.Columns)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.PreColumns)) + require.Equal(t, 1, len(node.rowBuffer)) + require.Equal(t, 2, len(node.rowBuffer[0].Columns)) + require.Equal(t, 2, len(node.rowBuffer[0].PreColumns)) } func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { @@ -447,7 +447,7 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { // nil row. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) // No update to the handle key column. columns := []*model.Column{ @@ -482,12 +482,12 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) - require.Equal(t, 1, len(node.eventBuffer)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.Columns)) - require.Equal(t, 0, len(node.eventBuffer[0].Row.PreColumns)) + require.Equal(t, 1, len(node.rowBuffer)) + require.Equal(t, 2, len(node.rowBuffer[0].Columns)) + require.Equal(t, 0, len(node.rowBuffer[0].PreColumns)) // Cleanup. - node.eventBuffer = []*model.PolymorphicEvent{} + node.rowBuffer = []*model.RowChangedEvent{} // Update to the handle key column. columns = []*model.Column{ { @@ -522,21 +522,21 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) // Split an update event into a delete and an insert event. - require.Equal(t, 2, len(node.eventBuffer)) + require.Equal(t, 2, len(node.rowBuffer)) deleteEventIndex := 0 - require.Equal(t, 0, len(node.eventBuffer[deleteEventIndex].Row.Columns)) - require.Equal(t, 2, len(node.eventBuffer[deleteEventIndex].Row.PreColumns)) + require.Equal(t, 0, len(node.rowBuffer[deleteEventIndex].Columns)) + require.Equal(t, 2, len(node.rowBuffer[deleteEventIndex].PreColumns)) nonHandleKeyColIndex := 0 handleKeyColIndex := 1 // NOTICE: When old value disabled, we only keep the handle key pre cols. - require.Nil(t, node.eventBuffer[deleteEventIndex].Row.PreColumns[nonHandleKeyColIndex]) - require.Equal(t, "col2", node.eventBuffer[deleteEventIndex].Row.PreColumns[handleKeyColIndex].Name) - require.True(t, node.eventBuffer[deleteEventIndex].Row.PreColumns[handleKeyColIndex].Flag.IsHandleKey()) + require.Nil(t, node.rowBuffer[deleteEventIndex].PreColumns[nonHandleKeyColIndex]) + require.Equal(t, "col2", node.rowBuffer[deleteEventIndex].PreColumns[handleKeyColIndex].Name) + require.True(t, node.rowBuffer[deleteEventIndex].PreColumns[handleKeyColIndex].Flag.IsHandleKey()) insertEventIndex := 1 - require.Equal(t, 2, len(node.eventBuffer[insertEventIndex].Row.Columns)) - require.Equal(t, 0, len(node.eventBuffer[insertEventIndex].Row.PreColumns)) + require.Equal(t, 2, len(node.rowBuffer[insertEventIndex].Columns)) + require.Equal(t, 0, len(node.rowBuffer[insertEventIndex].PreColumns)) } type flushFlowController struct { diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index b0a45e269d5..725eeba2bb8 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -65,7 +65,7 @@ type tablePipelineImpl struct { tableID int64 markTableID int64 - tableName string // quoted schema and table, used in metircs only + tableName string // quoted schema and table, used in metrics only sorterNode *sorterNode sinkNode *sinkNode @@ -108,7 +108,7 @@ func (t *tablePipelineImpl) UpdateBarrierTs(ts model.Ts) { } } -// AsyncStop tells the pipeline to stop, and returns true is the pipeline is already stopped. +// AsyncStop tells the pipeline to stop, and returns true if the pipeline is already stopped. func (t *tablePipelineImpl) AsyncStop(targetTs model.Ts) bool { err := t.p.SendToFirstNode(pipeline.CommandMessage(&pipeline.Command{ Tp: pipeline.CommandTypeStop, From 2b45869e995dfdbcd9a1785b143d8efff5b3c217 Mon Sep 17 00:00:00 2001 From: jerrylisl <31952107+jerrylisl@users.noreply.github.com> Date: Fri, 28 Jan 2022 11:45:13 +0800 Subject: [PATCH 33/72] master(dm): specify worker when create upstream source (#4167) close pingcap/tiflow#4169 --- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/ctl/master/operate_source.go | 24 +- dm/dm/master/scheduler/scheduler.go | 47 +- dm/dm/master/scheduler/scheduler_test.go | 37 +- dm/dm/master/server.go | 7 +- dm/dm/pb/dmmaster.pb.go | 322 ++++++----- dm/dm/proto/dmmaster.proto | 535 +++++++++--------- dm/errors.toml | 6 + dm/pkg/terror/error_list.go | 3 +- dm/tests/_utils/env_variables | 2 + .../dmctl_basic/check_list/operate_source.sh | 2 +- dm/tests/dmctl_command/run.sh | 17 +- 12 files changed, 585 insertions(+), 418 deletions(-) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 63fd3f42052..9e115d4ce3a 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -535,6 +535,7 @@ ErrSchedulerStopRelayOnSpecified,[code=46029:class=scheduler:scope=internal:leve ErrSchedulerStartRelayOnBound,[code=46030:class=scheduler:scope=internal:level=low], "Message: the source has `start-relay` automatically for bound worker, so it can't `start-relay` with worker name now, Workaround: Please stop relay by `stop-relay` without worker name first." ErrSchedulerStopRelayOnBound,[code=46031:class=scheduler:scope=internal:level=low], "Message: the source has `start-relay` automatically for bound worker, so it can't `stop-relay` with worker name now, Workaround: Please use `stop-relay` without worker name." ErrSchedulerPauseTaskForTransferSource,[code=46032:class=scheduler:scope=internal:level=low], "Message: failed to auto pause tasks %s when transfer-source, Workaround: Please pause task by `dmctl pause-task`." +ErrSchedulerWorkerNotFree,[code=46033:class=scheduler:scope=internal:level=low], "Message: dm-worker with name %s not free" ErrCtlGRPCCreateConn,[code=48001:class=dmctl:scope=internal:level=high], "Message: can not create grpc connection, Workaround: Please check your network connection." ErrCtlInvalidTLSCfg,[code=48002:class=dmctl:scope=internal:level=medium], "Message: invalid TLS config, Workaround: Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in command line." ErrCtlLoadTLSCfg,[code=48003:class=dmctl:scope=internal:level=high], "Message: can not load tls config, Workaround: Please ensure that the tls certificate is accessible on the node currently running dmctl." diff --git a/dm/dm/ctl/master/operate_source.go b/dm/dm/ctl/master/operate_source.go index c50e405217d..bba20d1c7b8 100644 --- a/dm/dm/ctl/master/operate_source.go +++ b/dm/dm/ctl/master/operate_source.go @@ -32,11 +32,12 @@ import ( // NewOperateSourceCmd creates a OperateSource command. func NewOperateSourceCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "operate-source [config-file ...] [--print-sample-config]", + Use: "operate-source [config-file ...] [-w worker] [--print-sample-config]", Short: "`create`/`update`/`stop`/`show` upstream MySQL/MariaDB source", RunE: operateSourceFunc, } cmd.Flags().BoolP("print-sample-config", "p", false, "print sample config file of source") + cmd.Flags().StringP("worker", "w", "", "specify bound worker for created source") return cmd } @@ -85,6 +86,20 @@ func operateSourceFunc(cmd *cobra.Command, _ []string) error { return errors.New("please check output to see error") } + var specifyWorker string + if op == pb.SourceOp_StartSource { + specifyWorker, err = cmd.Flags().GetString("worker") + if err != nil { + common.PrintLinesf("error in parse `--worker`") + return err + } + if specifyWorker != "" { + if len(cmd.Flags().Args()) > 2 { + common.PrintLinesf("operate-source create can't create multiple sources when specify worker") + } + } + } + contents := make([]string, 0, len(cmd.Flags().Args())-1) sourceID := make([]string, 0, len(cmd.Flags().Args())-1) sources, err := common.GetSourceArgs(cmd) @@ -132,9 +147,10 @@ func operateSourceFunc(cmd *cobra.Command, _ []string) error { ctx, "OperateSource", &pb.OperateSourceRequest{ - Config: contents, - Op: op, - SourceID: sourceID, + Config: contents, + Op: op, + SourceID: sourceID, + WorkerName: specifyWorker, }, &resp, ) diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index fd7ed6555d0..e5b899a4837 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -317,7 +317,7 @@ func (s *Scheduler) CloseAllWorkers() { } } -// AddSourceCfg adds the upstream source config to the cluster. +// AddSourceCfg adds the upstream source config to the cluster, and try to bound source to worker // NOTE: please verify the config before call this. func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { s.mu.Lock() @@ -327,11 +327,49 @@ func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { return terror.ErrSchedulerNotStarted.Generate() } + err := s.addSource(cfg) + if err != nil { + return err + } + + // try to bound it to a Free worker. + _, err = s.tryBoundForSource(cfg.SourceID) + return err +} + +// AddSourceCfgWithWorker adds the upstream source config to the cluster, and try to bound source to specify worker +// NOTE: please verify the config before call this. +func (s *Scheduler) AddSourceCfgWithWorker(cfg *config.SourceConfig, workerName string) error { + s.mu.Lock() + defer s.mu.Unlock() + + if !s.started.Load() { + return terror.ErrSchedulerNotStarted.Generate() + } + + // check whether worker exists. + w, ok := s.workers[workerName] + if !ok { + return terror.ErrSchedulerWorkerNotExist.Generate(workerName) + } + + if w.stage != WorkerFree { + return terror.ErrSchedulerWorkerNotFree.Generate(workerName) + } + + if err := s.addSource(cfg); err != nil { + return err + } + + return s.boundSourceToWorker(cfg.SourceID, w) +} + +// addSource adds the upstream source config to the cluster. +func (s *Scheduler) addSource(cfg *config.SourceConfig) error { // 1. check whether exists. if _, ok := s.sourceCfgs[cfg.SourceID]; ok { return terror.ErrSchedulerSourceCfgExist.Generate(cfg.SourceID) } - // 2. put the config into etcd. _, err := ha.PutSourceCfg(s.etcdCli, cfg) if err != nil { @@ -341,10 +379,7 @@ func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { // 3. record the config in the scheduler. s.sourceCfgs[cfg.SourceID] = cfg s.unbounds[cfg.SourceID] = struct{}{} - - // 4. try to bound it to a Free worker. - _, err = s.tryBoundForSource(cfg.SourceID) - return err + return nil } // UpdateSourceCfg update the upstream source config to the cluster. diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index f2d254bf0f0..f045456407b 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -135,6 +135,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { // not started scheduler can't do anything. c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfg(sourceCfg1)), IsTrue) + c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfgWithWorker(sourceCfg1, workerName1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.UpdateSourceCfg(sourceCfg1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.RemoveSourceCfg(sourceID1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSubTasks(false, subtaskCfg1)), IsTrue) @@ -406,7 +407,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.relayStageMatch(c, s, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) - // CASE 4.4: start a task with two sources. + // CASE 4.4.1: start a task with two sources. // can't add more than one tasks at a time now. c.Assert(terror.ErrSchedulerMultiTask.Equal(s.AddSubTasks(false, subtaskCfg1, subtaskCfg21)), IsTrue) // task2' config and stage not exists before. @@ -423,7 +424,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) - // CASE 4.4.1 fail to stop any task. + // CASE 4.4.2 fail to stop any task. // can call without tasks or sources, return without error, but take no effect. c.Assert(s.RemoveSubTasks("", sourceID1), IsNil) c.Assert(s.RemoveSubTasks(taskName1), IsNil) @@ -481,6 +482,38 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.relayStageMatch(c, s, sourceID2, pb.Stage_InvalidStage) rebuildScheduler(ctx) + // CASE 4.7.1: add source2 with specify worker1 + // source2 not exist, worker1 is bound + t.sourceCfgNotExist(c, s, sourceID2) + t.workerBound(c, s, ha.NewSourceBound(sourceID1, workerName1)) + c.Assert(terror.ErrSchedulerWorkerNotFree.Equal(s.AddSourceCfgWithWorker(&sourceCfg2, workerName1)), IsTrue) + // source2 is not created because expected worker1 is already bound + t.sourceCfgNotExist(c, s, sourceID2) + rebuildScheduler(ctx) + + // CASE 4.7.2: add source2 with specify worker2 + // source2 not exist, worker2 should be free + t.sourceCfgNotExist(c, s, sourceID2) + t.workerFree(c, s, workerName2) + c.Assert(s.AddSourceCfgWithWorker(&sourceCfg2, workerName2), IsNil) + t.workerBound(c, s, ha.NewSourceBound(sourceID2, workerName2)) + t.sourceBounds(c, s, []string{sourceID1, sourceID2}, []string{}) + c.Assert(s.StartRelay(sourceID2, []string{workerName2}), IsNil) + t.relayStageMatch(c, s, sourceID2, pb.Stage_Running) + rebuildScheduler(ctx) + + // CASE 4.7.3: remove source2 again. + c.Assert(s.StopRelay(sourceID2, []string{workerName2}), IsNil) + c.Assert(s.RemoveSourceCfg(sourceID2), IsNil) + c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.RemoveSourceCfg(sourceID2)), IsTrue) // already removed. + // source2 removed. + t.sourceCfgNotExist(c, s, sourceID2) + // worker2 become Free now. + t.workerFree(c, s, workerName2) + t.sourceBounds(c, s, []string{sourceID1}, []string{}) + t.relayStageMatch(c, s, sourceID2, pb.Stage_InvalidStage) + rebuildScheduler(ctx) + // CASE 4.8: worker1 become offline. // before shutdown, worker1 bound source t.workerBound(c, s, ha.NewSourceBound(sourceID1, workerName1)) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 14c13fb72fb..4e24e10cfb9 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -1352,7 +1352,12 @@ func (s *Server) OperateSource(ctx context.Context, req *pb.OperateSourceRequest err error ) for _, cfg := range cfgs { - err = s.scheduler.AddSourceCfg(cfg) + // add source with worker when specify a worker name + if req.WorkerName != "" { + err = s.scheduler.AddSourceCfgWithWorker(cfg, req.WorkerName) + } else { + err = s.scheduler.AddSourceCfg(cfg) + } // return first error and try to revert, so user could copy-paste same start command after error if err != nil { resp.Msg = err.Error() diff --git a/dm/dm/pb/dmmaster.pb.go b/dm/dm/pb/dmmaster.pb.go index 790788ca74c..b70beb74973 100644 --- a/dm/dm/pb/dmmaster.pb.go +++ b/dm/dm/pb/dmmaster.pb.go @@ -1357,9 +1357,10 @@ func (m *CheckTaskResponse) GetMsg() string { } type OperateSourceRequest struct { - Op SourceOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.SourceOp" json:"op,omitempty"` - Config []string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty"` - SourceID []string `protobuf:"bytes,3,rep,name=sourceID,proto3" json:"sourceID,omitempty"` + Op SourceOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.SourceOp" json:"op,omitempty"` + Config []string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty"` + SourceID []string `protobuf:"bytes,3,rep,name=sourceID,proto3" json:"sourceID,omitempty"` + WorkerName string `protobuf:"bytes,4,opt,name=workerName,proto3" json:"workerName,omitempty"` } func (m *OperateSourceRequest) Reset() { *m = OperateSourceRequest{} } @@ -1416,6 +1417,13 @@ func (m *OperateSourceRequest) GetSourceID() []string { return nil } +func (m *OperateSourceRequest) GetWorkerName() string { + if m != nil { + return m.WorkerName + } + return "" +} + type OperateSourceResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -3206,138 +3214,139 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 2090 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xe3, 0xc6, - 0x11, 0x17, 0x25, 0x59, 0x96, 0x46, 0xb6, 0x22, 0xaf, 0x65, 0x99, 0xc7, 0xf3, 0xe9, 0x1c, 0x36, - 0x09, 0x0c, 0xa3, 0x38, 0xe3, 0xdc, 0x3e, 0x05, 0x48, 0x81, 0x9c, 0x74, 0xb9, 0x18, 0xf5, 0xd5, - 0x29, 0xed, 0x4b, 0x11, 0x14, 0x28, 0x4a, 0x49, 0x2b, 0x59, 0x30, 0x45, 0xf2, 0x48, 0xca, 0xae, - 0x71, 0x48, 0x1f, 0xfa, 0x01, 0xfa, 0x07, 0x05, 0x9a, 0xc7, 0x3e, 0xf4, 0x9b, 0xf4, 0xa9, 0x8f, - 0x01, 0xfa, 0xd2, 0xc7, 0xe2, 0xae, 0x1f, 0xa4, 0xd8, 0xd9, 0x5d, 0x72, 0xf9, 0x47, 0x4e, 0x15, - 0xa0, 0x7e, 0xe3, 0xcc, 0xac, 0x66, 0x7e, 0xf3, 0x67, 0x67, 0x67, 0x57, 0xd0, 0x1a, 0xcf, 0xe7, - 0x76, 0x18, 0xd1, 0xe0, 0x89, 0x1f, 0x78, 0x91, 0x47, 0xca, 0xfe, 0xd0, 0x68, 0x8d, 0xe7, 0x37, - 0x5e, 0x70, 0x25, 0x79, 0xc6, 0xde, 0xd4, 0xf3, 0xa6, 0x0e, 0x3d, 0xb2, 0xfd, 0xd9, 0x91, 0xed, - 0xba, 0x5e, 0x64, 0x47, 0x33, 0xcf, 0x0d, 0xb9, 0xd4, 0xfc, 0x2d, 0xb4, 0xcf, 0x23, 0x3b, 0x88, - 0x2e, 0xec, 0xf0, 0xca, 0xa2, 0xaf, 0x17, 0x34, 0x8c, 0x08, 0x81, 0x6a, 0x64, 0x87, 0x57, 0xba, - 0xb6, 0xaf, 0x1d, 0x34, 0x2c, 0xfc, 0x26, 0x3a, 0xac, 0x87, 0xde, 0x22, 0x18, 0xd1, 0x50, 0x2f, - 0xef, 0x57, 0x0e, 0x1a, 0x96, 0x24, 0x49, 0x0f, 0x20, 0xa0, 0x73, 0xef, 0x9a, 0xbe, 0xa4, 0x91, - 0xad, 0x57, 0xf6, 0xb5, 0x83, 0xba, 0xa5, 0x70, 0xc8, 0x1e, 0x34, 0x42, 0xb4, 0x30, 0x9b, 0x53, - 0xbd, 0x8a, 0x2a, 0x13, 0x86, 0xf9, 0x1a, 0xb6, 0x14, 0xfb, 0xa1, 0xef, 0xb9, 0x21, 0x25, 0x5d, - 0xa8, 0x05, 0x34, 0x5c, 0x38, 0x11, 0x42, 0xa8, 0x5b, 0x82, 0x22, 0x6d, 0xa8, 0xcc, 0xc3, 0xa9, - 0x5e, 0x46, 0x25, 0xec, 0x93, 0x1c, 0x27, 0xb0, 0x2a, 0xfb, 0x95, 0x83, 0xe6, 0xb1, 0xfe, 0xc4, - 0x1f, 0x3e, 0xe9, 0x7b, 0xf3, 0xb9, 0xe7, 0xfe, 0x02, 0xa3, 0x20, 0x95, 0xc6, 0x80, 0xcd, 0x5f, - 0x01, 0x39, 0xf3, 0x69, 0x60, 0x47, 0x54, 0x75, 0xda, 0x80, 0xb2, 0xe7, 0xa3, 0xbd, 0xd6, 0x31, - 0x30, 0x25, 0x4c, 0x78, 0xe6, 0x5b, 0x65, 0xcf, 0x67, 0x01, 0x71, 0xed, 0x39, 0x15, 0x86, 0xf1, - 0x5b, 0x0d, 0x48, 0x25, 0x15, 0x10, 0xf3, 0x0f, 0x1a, 0x6c, 0xa7, 0x0c, 0x08, 0xaf, 0xee, 0xb2, - 0x90, 0x78, 0x5c, 0x2e, 0xf2, 0xb8, 0x52, 0xe8, 0x71, 0xf5, 0x7f, 0xf5, 0xf8, 0x53, 0xd8, 0x7a, - 0xe5, 0x8f, 0x33, 0x0e, 0xaf, 0x94, 0x65, 0x33, 0x00, 0xa2, 0xaa, 0xb8, 0x97, 0x44, 0x7d, 0x06, - 0xdd, 0x9f, 0x2f, 0x68, 0x70, 0x7b, 0x1e, 0xd9, 0xd1, 0x22, 0x3c, 0x9d, 0x85, 0x91, 0x82, 0x1d, - 0x13, 0xa2, 0x15, 0x27, 0x24, 0x83, 0xfd, 0x1a, 0x76, 0x73, 0x7a, 0x56, 0x76, 0xe0, 0x69, 0xd6, - 0x81, 0x5d, 0xe6, 0x80, 0xa2, 0x37, 0x8f, 0xbf, 0x0f, 0xdb, 0xe7, 0x97, 0xde, 0xcd, 0x60, 0x70, - 0x7a, 0xea, 0x8d, 0xae, 0xc2, 0xef, 0x17, 0xf8, 0xbf, 0x6a, 0xb0, 0x2e, 0x34, 0x90, 0x16, 0x94, - 0x4f, 0x06, 0xe2, 0x77, 0xe5, 0x93, 0x41, 0xac, 0xa9, 0xac, 0x68, 0x22, 0x50, 0x9d, 0x7b, 0x63, - 0x2a, 0x4a, 0x06, 0xbf, 0x49, 0x07, 0xd6, 0xbc, 0x1b, 0x97, 0x06, 0x62, 0xfb, 0x71, 0x82, 0xad, - 0x1c, 0x0c, 0x4e, 0x43, 0x7d, 0x0d, 0x0d, 0xe2, 0x37, 0x8b, 0x47, 0x78, 0xeb, 0x8e, 0xe8, 0x58, - 0xaf, 0x21, 0x57, 0x50, 0xc4, 0x80, 0xfa, 0xc2, 0x15, 0x92, 0x75, 0x94, 0xc4, 0xb4, 0x39, 0x82, - 0x4e, 0xda, 0xcd, 0x95, 0x63, 0xfb, 0x3e, 0xac, 0x39, 0xec, 0xa7, 0x22, 0xb2, 0x4d, 0x16, 0x59, - 0xa1, 0xce, 0xe2, 0x12, 0xd3, 0x81, 0xce, 0x2b, 0x97, 0x7d, 0x4a, 0xbe, 0x08, 0x66, 0x36, 0x24, - 0x26, 0x6c, 0x04, 0xd4, 0x77, 0xec, 0x11, 0x3d, 0x43, 0x8f, 0xb9, 0x95, 0x14, 0x8f, 0xec, 0x43, - 0x73, 0xe2, 0x05, 0x23, 0x6a, 0x61, 0x93, 0x12, 0x2d, 0x4b, 0x65, 0x99, 0x9f, 0xc2, 0x4e, 0xc6, - 0xda, 0xaa, 0x3e, 0x99, 0x16, 0x3c, 0x10, 0x4d, 0x40, 0x96, 0xb7, 0x63, 0xdf, 0x4a, 0xd4, 0x0f, - 0x95, 0x56, 0x80, 0xde, 0xa2, 0x54, 0xf4, 0x82, 0xe5, 0xb5, 0xf0, 0x8d, 0x06, 0x46, 0x91, 0x52, - 0x01, 0xee, 0x4e, 0xad, 0xff, 0xdf, 0x0e, 0xf3, 0x8d, 0x06, 0xbb, 0x5f, 0x2c, 0x82, 0x69, 0x91, - 0xb3, 0x8a, 0x3f, 0x5a, 0xfa, 0xe8, 0x30, 0xa0, 0x3e, 0x73, 0xed, 0x51, 0x34, 0xbb, 0xa6, 0x02, - 0x55, 0x4c, 0x63, 0x6d, 0xb3, 0x13, 0x83, 0x01, 0xab, 0x58, 0xf8, 0xcd, 0xd6, 0x4f, 0x66, 0x0e, - 0xc5, 0xad, 0xcf, 0x4b, 0x39, 0xa6, 0xb1, 0x72, 0x17, 0xc3, 0xc1, 0x2c, 0xd0, 0xd7, 0x50, 0x22, - 0x28, 0xf3, 0x37, 0xa0, 0xe7, 0x81, 0xdd, 0x4b, 0xfb, 0xba, 0x86, 0x76, 0xff, 0x92, 0x8e, 0xae, - 0xbe, 0xab, 0xe9, 0x76, 0xa1, 0x46, 0x83, 0xa0, 0xef, 0xf2, 0xcc, 0x54, 0x2c, 0x41, 0xb1, 0xb8, - 0xdd, 0xd8, 0x81, 0xcb, 0x04, 0x3c, 0x08, 0x92, 0xfc, 0x8e, 0x23, 0xf5, 0x13, 0xd8, 0x52, 0xec, - 0xae, 0x5c, 0xb8, 0x97, 0xd0, 0x11, 0x35, 0x76, 0x8e, 0x8e, 0x48, 0xe8, 0x7b, 0x4a, 0x75, 0x6d, - 0x30, 0xef, 0xb9, 0x38, 0x29, 0xaf, 0x91, 0xe7, 0x4e, 0x66, 0x53, 0x51, 0xb3, 0x82, 0x62, 0x29, - 0xe3, 0xf1, 0x38, 0x19, 0x88, 0x73, 0x32, 0xa6, 0xcd, 0x05, 0xec, 0x64, 0x2c, 0xdd, 0x4b, 0x5e, - 0x9e, 0xc3, 0x8e, 0x45, 0xa7, 0x33, 0x36, 0x36, 0xc9, 0x25, 0x77, 0x9e, 0x2a, 0xf6, 0x78, 0x1c, - 0xd0, 0x30, 0x14, 0x66, 0x25, 0x69, 0x3e, 0x83, 0x6e, 0x56, 0xcd, 0xca, 0xb1, 0xfe, 0x09, 0x74, - 0xce, 0x26, 0x13, 0x67, 0xe6, 0xd2, 0x97, 0x74, 0x3e, 0x4c, 0x21, 0x89, 0x6e, 0xfd, 0x18, 0x09, - 0xfb, 0x2e, 0x1a, 0x42, 0x58, 0x9f, 0xca, 0xfc, 0x7e, 0x65, 0x08, 0x3f, 0x8e, 0xd3, 0x7d, 0x4a, - 0xed, 0x71, 0x02, 0x21, 0x97, 0x6e, 0x2e, 0xe6, 0xe9, 0x46, 0xc3, 0xe9, 0x5f, 0xad, 0x6c, 0xf8, - 0xf7, 0x1a, 0xc0, 0x4b, 0x1c, 0x5e, 0x4f, 0xdc, 0x89, 0x57, 0x18, 0x7c, 0x03, 0xea, 0x73, 0xf4, - 0xeb, 0x64, 0x80, 0xbf, 0xac, 0x5a, 0x31, 0xcd, 0xce, 0x34, 0xdb, 0x99, 0xc5, 0xed, 0x9b, 0x13, - 0xec, 0x17, 0x3e, 0xa5, 0xc1, 0x2b, 0xeb, 0x94, 0x37, 0xaf, 0x86, 0x15, 0xd3, 0x6c, 0x50, 0x1d, - 0x39, 0x33, 0xea, 0x46, 0x28, 0xe5, 0xa7, 0x9e, 0xc2, 0x31, 0x87, 0x00, 0x3c, 0x91, 0x4b, 0xf1, - 0x10, 0xa8, 0xb2, 0xec, 0xcb, 0x14, 0xb0, 0x6f, 0x86, 0x23, 0x8c, 0xec, 0xa9, 0x3c, 0x70, 0x39, - 0x81, 0xdd, 0x08, 0xcb, 0x4d, 0x6c, 0x4f, 0x41, 0x99, 0xa7, 0xd0, 0x66, 0xf3, 0x07, 0x0f, 0x1a, - 0xcf, 0x99, 0x0c, 0x8d, 0x96, 0x54, 0x75, 0xd1, 0xbc, 0x29, 0x6d, 0x57, 0x12, 0xdb, 0xe6, 0xcf, - 0xb8, 0x36, 0x1e, 0xc5, 0xa5, 0xda, 0x0e, 0x60, 0x9d, 0x5f, 0x12, 0xf8, 0x79, 0xd2, 0x3c, 0x6e, - 0xb1, 0x74, 0x26, 0xa1, 0xb7, 0xa4, 0x58, 0xea, 0xe3, 0x51, 0xb8, 0x4b, 0x1f, 0xbf, 0x60, 0xa4, - 0xf4, 0x25, 0xa1, 0xb3, 0xa4, 0xd8, 0xfc, 0x9b, 0x06, 0xeb, 0x5c, 0x4d, 0x48, 0x9e, 0x40, 0xcd, - 0x41, 0xaf, 0x51, 0x55, 0xf3, 0xb8, 0x83, 0x35, 0x95, 0x89, 0xc5, 0xe7, 0x25, 0x4b, 0xac, 0x62, - 0xeb, 0x39, 0x2c, 0x8c, 0x82, 0xb2, 0x5e, 0xf5, 0x96, 0xad, 0xe7, 0xab, 0xd8, 0x7a, 0x6e, 0x16, - 0x23, 0xa4, 0xac, 0x57, 0xbd, 0x61, 0xeb, 0xf9, 0xaa, 0x67, 0x75, 0xa8, 0xf1, 0x5a, 0x62, 0x57, - 0x10, 0xd4, 0x9b, 0xda, 0x81, 0xdd, 0x14, 0xdc, 0x7a, 0x0c, 0xab, 0x9b, 0x82, 0x55, 0x8f, 0xcd, - 0x77, 0x53, 0xe6, 0xeb, 0xd2, 0x0c, 0x2b, 0x0f, 0x96, 0x3e, 0x59, 0x8d, 0x9c, 0x30, 0x29, 0x10, - 0xd5, 0xe4, 0xca, 0x6d, 0xef, 0x43, 0x58, 0xe7, 0xe0, 0x53, 0x23, 0x93, 0x08, 0xb5, 0x25, 0x65, - 0xe6, 0x5f, 0xca, 0x49, 0x2f, 0x1f, 0x5d, 0xd2, 0xb9, 0xbd, 0xbc, 0x97, 0xa3, 0x38, 0xb9, 0xee, - 0xe4, 0xc6, 0xca, 0xa5, 0xd7, 0x1d, 0xb6, 0xe5, 0xc6, 0x76, 0x64, 0x0f, 0xed, 0x30, 0x3e, 0x94, - 0x25, 0xcd, 0xbc, 0x8f, 0xec, 0xa1, 0x43, 0xc5, 0x99, 0xcc, 0x09, 0xdc, 0x1c, 0x68, 0x4f, 0xaf, - 0x89, 0xcd, 0x81, 0x14, 0x5b, 0x3d, 0x71, 0x16, 0xe1, 0xa5, 0xbe, 0xce, 0xb7, 0x34, 0x12, 0x0c, - 0x0d, 0x1b, 0x34, 0xf5, 0x3a, 0x32, 0xf1, 0x9b, 0x6d, 0xe5, 0x49, 0xe0, 0xcd, 0xf9, 0xb1, 0xa1, - 0x37, 0xf8, 0x9d, 0x33, 0xe1, 0x48, 0xf9, 0x85, 0x1d, 0x4c, 0x69, 0xa4, 0x43, 0x22, 0xe7, 0x1c, - 0xf5, 0xe4, 0x11, 0x71, 0xb9, 0x97, 0x93, 0xe7, 0x10, 0x3a, 0x2f, 0x68, 0x74, 0xbe, 0x18, 0xb2, - 0xa3, 0xb9, 0x3f, 0x99, 0xde, 0x71, 0xf0, 0x98, 0xaf, 0x60, 0x27, 0xb3, 0x76, 0x65, 0x88, 0x04, - 0xaa, 0xa3, 0xc9, 0x54, 0x26, 0x0c, 0xbf, 0xcd, 0x01, 0x6c, 0xbe, 0xa0, 0x91, 0x62, 0xfb, 0xb1, - 0x72, 0xd4, 0x88, 0xb1, 0xb1, 0x3f, 0x99, 0x5e, 0xdc, 0xfa, 0xf4, 0x8e, 0x73, 0xe7, 0x14, 0x5a, - 0x52, 0xcb, 0xca, 0xa8, 0xda, 0x50, 0x19, 0x4d, 0xe2, 0x81, 0x73, 0x34, 0x99, 0x9a, 0x3b, 0xb0, - 0xfd, 0x82, 0x8a, 0x7d, 0x9d, 0x20, 0x33, 0x0f, 0x30, 0x5a, 0x0a, 0x5b, 0x98, 0x12, 0x0a, 0xb4, - 0x44, 0xc1, 0x9f, 0x34, 0x20, 0x9f, 0xdb, 0xee, 0xd8, 0xa1, 0xcf, 0x83, 0xc0, 0x0b, 0x96, 0x4e, - 0xd9, 0x28, 0xfd, 0x5e, 0x45, 0xbe, 0x07, 0x8d, 0xe1, 0xcc, 0x75, 0xbc, 0xe9, 0x17, 0x5e, 0x28, - 0x27, 0xae, 0x98, 0x81, 0x25, 0xfa, 0xda, 0x89, 0x6f, 0x52, 0xec, 0xdb, 0x0c, 0x61, 0x3b, 0x05, - 0xe9, 0x5e, 0x0a, 0xec, 0x05, 0xec, 0x5c, 0x04, 0xb6, 0x1b, 0x4e, 0x68, 0x90, 0x1e, 0xde, 0x92, - 0xf3, 0x48, 0x53, 0xcf, 0x23, 0xa5, 0x6d, 0x71, 0xcb, 0x82, 0x62, 0xc3, 0x4d, 0x56, 0xd1, 0xca, - 0x07, 0xfc, 0x38, 0x7e, 0x06, 0x49, 0x5d, 0x07, 0x1e, 0x29, 0x59, 0xd9, 0x54, 0x6e, 0x29, 0x5f, - 0x1e, 0xcb, 0x41, 0x52, 0x20, 0x2d, 0x2f, 0x41, 0xca, 0x53, 0x23, 0x91, 0x46, 0x71, 0x8b, 0xbb, - 0xc7, 0xd9, 0xfe, 0x70, 0x08, 0x75, 0x39, 0xfe, 0x92, 0x6d, 0x78, 0xef, 0xc4, 0xbd, 0xb6, 0x9d, - 0xd9, 0x58, 0xb2, 0xda, 0x25, 0xf2, 0x1e, 0x34, 0xf1, 0x5d, 0x8b, 0xb3, 0xda, 0x1a, 0x69, 0xc3, - 0x06, 0x7f, 0x40, 0x11, 0x9c, 0x32, 0x69, 0x01, 0x9c, 0x47, 0x9e, 0x2f, 0xe8, 0x0a, 0xd2, 0x97, - 0xde, 0x8d, 0xa0, 0xab, 0x87, 0x3f, 0x85, 0xba, 0x9c, 0xb9, 0x14, 0x1b, 0x92, 0xd5, 0x2e, 0x91, - 0x2d, 0xd8, 0x7c, 0x7e, 0x3d, 0x1b, 0x45, 0x31, 0x4b, 0x23, 0xbb, 0xb0, 0xdd, 0xb7, 0xdd, 0x11, - 0x75, 0xd2, 0x82, 0xf2, 0xa1, 0x0b, 0xeb, 0x62, 0x5b, 0x33, 0x68, 0x42, 0x17, 0x23, 0xdb, 0x25, - 0xb2, 0x01, 0x75, 0xd6, 0x64, 0x90, 0xd2, 0x18, 0x0c, 0xbe, 0xe7, 0x90, 0x46, 0x98, 0x3c, 0x0a, - 0x48, 0x73, 0x98, 0x08, 0x11, 0xe9, 0x2a, 0xe9, 0x40, 0x1b, 0x7f, 0x4d, 0xe7, 0xbe, 0x63, 0x47, - 0x9c, 0xbb, 0x76, 0x38, 0x80, 0x46, 0x9c, 0x57, 0xb6, 0x44, 0x58, 0x8c, 0x79, 0xed, 0x12, 0x8b, - 0x08, 0x86, 0x08, 0x79, 0x5f, 0x1e, 0xb7, 0x35, 0x1e, 0x34, 0xcf, 0x97, 0x8c, 0xf2, 0xf1, 0xdf, - 0x5b, 0x50, 0xe3, 0x60, 0xc8, 0x57, 0xd0, 0x88, 0x1f, 0x0a, 0x09, 0x1e, 0xee, 0xd9, 0x77, 0x4b, - 0x63, 0x27, 0xc3, 0xe5, 0x49, 0x33, 0x1f, 0xff, 0xee, 0x9f, 0xff, 0xf9, 0x73, 0xf9, 0x81, 0xd9, - 0x39, 0xb2, 0xfd, 0x59, 0x78, 0x74, 0xfd, 0xd4, 0x76, 0xfc, 0x4b, 0xfb, 0xe9, 0x11, 0xdb, 0xf2, - 0xe1, 0xc7, 0xda, 0x21, 0x99, 0x40, 0x53, 0x79, 0xaf, 0x23, 0x5d, 0xa6, 0x26, 0xff, 0x42, 0x68, - 0xec, 0xe6, 0xf8, 0xc2, 0xc0, 0x47, 0x68, 0x60, 0xdf, 0x78, 0x58, 0x64, 0xe0, 0xe8, 0x0d, 0xeb, - 0x98, 0x5f, 0x33, 0x3b, 0x9f, 0x00, 0x24, 0x6f, 0x68, 0x04, 0xd1, 0xe6, 0x9e, 0xe5, 0x8c, 0x6e, - 0x96, 0x2d, 0x8c, 0x94, 0x88, 0x03, 0x4d, 0xe5, 0xb9, 0x89, 0x18, 0x99, 0xf7, 0x27, 0xe5, 0x7d, - 0xcc, 0x78, 0x58, 0x28, 0x13, 0x9a, 0x3e, 0x40, 0xb8, 0x3d, 0xb2, 0x97, 0x81, 0x1b, 0xe2, 0x52, - 0x81, 0x97, 0xf4, 0x61, 0x43, 0x7d, 0xd5, 0x21, 0xe8, 0x7d, 0xc1, 0x73, 0x96, 0xa1, 0xe7, 0x05, - 0x31, 0xe4, 0xcf, 0x60, 0x33, 0xf5, 0x8e, 0x42, 0x70, 0x71, 0xd1, 0x43, 0x8e, 0xf1, 0xa0, 0x40, - 0x12, 0xeb, 0xf9, 0x0a, 0xba, 0xf9, 0x77, 0x0f, 0x8c, 0xe2, 0x23, 0x25, 0x29, 0xf9, 0xb7, 0x07, - 0xa3, 0xb7, 0x4c, 0x1c, 0xab, 0x3e, 0x83, 0x76, 0xf6, 0x7d, 0x80, 0x60, 0xf8, 0x96, 0x3c, 0x67, - 0x18, 0x7b, 0xc5, 0xc2, 0x58, 0xe1, 0xc7, 0xd0, 0x88, 0xaf, 0xdf, 0xbc, 0x50, 0xb3, 0xaf, 0x00, - 0xbc, 0x50, 0x73, 0x77, 0x74, 0xb3, 0x44, 0xa6, 0xb0, 0x99, 0xba, 0x11, 0xf3, 0x78, 0x15, 0x5d, - 0xc7, 0x79, 0xbc, 0x0a, 0xaf, 0xcf, 0xe6, 0xfb, 0x98, 0xe0, 0x87, 0x46, 0x37, 0x9b, 0x60, 0xde, - 0xbc, 0x58, 0x29, 0x9e, 0x40, 0x2b, 0x7d, 0x79, 0x25, 0x0f, 0x78, 0x2b, 0x2e, 0xb8, 0x17, 0x1b, - 0x46, 0x91, 0x28, 0xc6, 0x1c, 0xc0, 0x66, 0xea, 0x0e, 0x2a, 0x30, 0x17, 0x5c, 0x6b, 0x05, 0xe6, - 0xa2, 0x0b, 0xab, 0xf9, 0x43, 0xc4, 0xfc, 0xd1, 0xe1, 0x07, 0x19, 0xcc, 0x62, 0x94, 0x3d, 0x7a, - 0xc3, 0x66, 0x91, 0xaf, 0x65, 0x71, 0x5e, 0xc5, 0x71, 0xe2, 0x2d, 0x2e, 0x15, 0xa7, 0xd4, 0x3d, - 0x36, 0x15, 0xa7, 0xf4, 0x5d, 0xd5, 0xfc, 0x10, 0x6d, 0x3e, 0x36, 0x8c, 0x8c, 0x4d, 0x3e, 0xea, - 0x1f, 0xbd, 0xf1, 0x7c, 0xdc, 0xb6, 0xbf, 0x04, 0x48, 0x86, 0x75, 0xbe, 0x6d, 0x73, 0xf7, 0x05, - 0xbe, 0x6d, 0xf3, 0x33, 0xbd, 0xd9, 0x43, 0x1b, 0x3a, 0xe9, 0x16, 0xfb, 0x45, 0x26, 0x49, 0xc6, - 0xf9, 0x10, 0x9c, 0xca, 0xb8, 0x3a, 0xb4, 0xa7, 0x33, 0x9e, 0x1a, 0x5b, 0xcd, 0x7d, 0xb4, 0x62, - 0x18, 0x3b, 0xd9, 0x8c, 0xe3, 0x32, 0xe6, 0x84, 0x83, 0x73, 0x5f, 0x32, 0x4e, 0x72, 0x3b, 0x45, - 0xd3, 0x28, 0xb7, 0x53, 0x38, 0x7b, 0xca, 0x4e, 0x47, 0x7a, 0x59, 0x3b, 0x8b, 0xa1, 0xda, 0xec, - 0xc8, 0x05, 0xd4, 0xf8, 0x7c, 0x48, 0xb6, 0x84, 0x32, 0x45, 0x3f, 0x51, 0x59, 0x42, 0xf1, 0x0f, - 0x50, 0xf1, 0x23, 0x72, 0x57, 0x0b, 0x25, 0xbf, 0x86, 0xa6, 0x32, 0x52, 0xf1, 0x3e, 0x9d, 0x1f, - 0xfb, 0x78, 0x9f, 0x2e, 0x98, 0xbd, 0x96, 0x46, 0x89, 0xb2, 0x55, 0xb8, 0x2d, 0xfa, 0xb0, 0xa1, - 0x8e, 0x9c, 0xbc, 0xe9, 0x15, 0xcc, 0xa6, 0x86, 0x9e, 0x17, 0xc4, 0x1b, 0xe2, 0x04, 0x5a, 0xe9, - 0xd9, 0x89, 0xef, 0xad, 0xc2, 0xc1, 0x8c, 0xef, 0xad, 0xe2, 0x51, 0xcb, 0x2c, 0x31, 0x3c, 0xea, - 0x70, 0x43, 0xd4, 0x23, 0x28, 0xd5, 0x94, 0xf4, 0xbc, 0x40, 0x2a, 0x79, 0xa6, 0xff, 0xe3, 0x6d, - 0x4f, 0xfb, 0xf6, 0x6d, 0x4f, 0xfb, 0xf7, 0xdb, 0x9e, 0xf6, 0xc7, 0x77, 0xbd, 0xd2, 0xb7, 0xef, - 0x7a, 0xa5, 0x7f, 0xbd, 0xeb, 0x95, 0x86, 0x35, 0xfc, 0x0f, 0xf0, 0x47, 0xff, 0x0d, 0x00, 0x00, - 0xff, 0xff, 0x62, 0x01, 0x57, 0x61, 0x47, 0x1c, 0x00, 0x00, + // 2101 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0xcd, 0x6f, 0xe3, 0xc6, + 0x15, 0x17, 0x25, 0x59, 0x96, 0x9e, 0x6c, 0x45, 0x1e, 0xcb, 0x32, 0x97, 0xeb, 0x68, 0x1d, 0x36, + 0x09, 0x0c, 0xa3, 0x58, 0x63, 0xdd, 0x9e, 0x02, 0xa4, 0x40, 0x56, 0xda, 0x6c, 0x8c, 0x7a, 0xe3, + 0x94, 0xf6, 0xa6, 0x08, 0x0a, 0x14, 0xa5, 0xa4, 0x91, 0x2c, 0x98, 0x22, 0xb9, 0x24, 0x65, 0xd7, + 0x58, 0xa4, 0x87, 0x9e, 0x7a, 0xea, 0x07, 0x0a, 0x34, 0xc7, 0x1e, 0xfa, 0x9f, 0xf4, 0xd4, 0x63, + 0x80, 0x5e, 0x7a, 0x2c, 0x76, 0xfb, 0x87, 0x14, 0xf3, 0x66, 0x86, 0x1c, 0x7e, 0xc8, 0xa9, 0x02, + 0xd4, 0x37, 0xbe, 0xf7, 0x46, 0xef, 0xfd, 0xde, 0xc7, 0xbc, 0x79, 0x33, 0x82, 0xd6, 0x78, 0x3e, + 0xb7, 0xc3, 0x88, 0x06, 0x8f, 0xfd, 0xc0, 0x8b, 0x3c, 0x52, 0xf6, 0x87, 0x46, 0x6b, 0x3c, 0xbf, + 0xf1, 0x82, 0x2b, 0xc9, 0x33, 0xf6, 0xa6, 0x9e, 0x37, 0x75, 0xe8, 0x91, 0xed, 0xcf, 0x8e, 0x6c, + 0xd7, 0xf5, 0x22, 0x3b, 0x9a, 0x79, 0x6e, 0xc8, 0xa5, 0xe6, 0x6f, 0xa0, 0x7d, 0x1e, 0xd9, 0x41, + 0x74, 0x61, 0x87, 0x57, 0x16, 0x7d, 0xb5, 0xa0, 0x61, 0x44, 0x08, 0x54, 0x23, 0x3b, 0xbc, 0xd2, + 0xb5, 0x7d, 0xed, 0xa0, 0x61, 0xe1, 0x37, 0xd1, 0x61, 0x3d, 0xf4, 0x16, 0xc1, 0x88, 0x86, 0x7a, + 0x79, 0xbf, 0x72, 0xd0, 0xb0, 0x24, 0x49, 0x7a, 0x00, 0x01, 0x9d, 0x7b, 0xd7, 0xf4, 0x05, 0x8d, + 0x6c, 0xbd, 0xb2, 0xaf, 0x1d, 0xd4, 0x2d, 0x85, 0x43, 0xf6, 0xa0, 0x11, 0xa2, 0x85, 0xd9, 0x9c, + 0xea, 0x55, 0x54, 0x99, 0x30, 0xcc, 0x57, 0xb0, 0xa5, 0xd8, 0x0f, 0x7d, 0xcf, 0x0d, 0x29, 0xe9, + 0x42, 0x2d, 0xa0, 0xe1, 0xc2, 0x89, 0x10, 0x42, 0xdd, 0x12, 0x14, 0x69, 0x43, 0x65, 0x1e, 0x4e, + 0xf5, 0x32, 0x2a, 0x61, 0x9f, 0xe4, 0x38, 0x81, 0x55, 0xd9, 0xaf, 0x1c, 0x34, 0x8f, 0xf5, 0xc7, + 0xfe, 0xf0, 0x71, 0xdf, 0x9b, 0xcf, 0x3d, 0xf7, 0xe7, 0x18, 0x05, 0xa9, 0x34, 0x06, 0x6c, 0xfe, + 0x12, 0xc8, 0x99, 0x4f, 0x03, 0x3b, 0xa2, 0xaa, 0xd3, 0x06, 0x94, 0x3d, 0x1f, 0xed, 0xb5, 0x8e, + 0x81, 0x29, 0x61, 0xc2, 0x33, 0xdf, 0x2a, 0x7b, 0x3e, 0x0b, 0x88, 0x6b, 0xcf, 0xa9, 0x30, 0x8c, + 0xdf, 0x6a, 0x40, 0x2a, 0xa9, 0x80, 0x98, 0x7f, 0xd0, 0x60, 0x3b, 0x65, 0x40, 0x78, 0x75, 0x97, + 0x85, 0xc4, 0xe3, 0x72, 0x91, 0xc7, 0x95, 0x42, 0x8f, 0xab, 0xff, 0xab, 0xc7, 0x9f, 0xc0, 0xd6, + 0x4b, 0x7f, 0x9c, 0x71, 0x78, 0xa5, 0x2c, 0x9b, 0x01, 0x10, 0x55, 0xc5, 0xbd, 0x24, 0xea, 0x53, + 0xe8, 0xfe, 0x6c, 0x41, 0x83, 0xdb, 0xf3, 0xc8, 0x8e, 0x16, 0xe1, 0xe9, 0x2c, 0x8c, 0x14, 0xec, + 0x98, 0x10, 0xad, 0x38, 0x21, 0x19, 0xec, 0xd7, 0xb0, 0x9b, 0xd3, 0xb3, 0xb2, 0x03, 0x4f, 0xb2, + 0x0e, 0xec, 0x32, 0x07, 0x14, 0xbd, 0x79, 0xfc, 0x7d, 0xd8, 0x3e, 0xbf, 0xf4, 0x6e, 0x06, 0x83, + 0xd3, 0x53, 0x6f, 0x74, 0x15, 0x7e, 0xbf, 0xc0, 0xff, 0x55, 0x83, 0x75, 0xa1, 0x81, 0xb4, 0xa0, + 0x7c, 0x32, 0x10, 0xbf, 0x2b, 0x9f, 0x0c, 0x62, 0x4d, 0x65, 0x45, 0x13, 0x81, 0xea, 0xdc, 0x1b, + 0x53, 0x51, 0x32, 0xf8, 0x4d, 0x3a, 0xb0, 0xe6, 0xdd, 0xb8, 0x34, 0x10, 0xdb, 0x8f, 0x13, 0x6c, + 0xe5, 0x60, 0x70, 0x1a, 0xea, 0x6b, 0x68, 0x10, 0xbf, 0x59, 0x3c, 0xc2, 0x5b, 0x77, 0x44, 0xc7, + 0x7a, 0x0d, 0xb9, 0x82, 0x22, 0x06, 0xd4, 0x17, 0xae, 0x90, 0xac, 0xa3, 0x24, 0xa6, 0xcd, 0x11, + 0x74, 0xd2, 0x6e, 0xae, 0x1c, 0xdb, 0xf7, 0x60, 0xcd, 0x61, 0x3f, 0x15, 0x91, 0x6d, 0xb2, 0xc8, + 0x0a, 0x75, 0x16, 0x97, 0x98, 0x0e, 0x74, 0x5e, 0xba, 0xec, 0x53, 0xf2, 0x45, 0x30, 0xb3, 0x21, + 0x31, 0x61, 0x23, 0xa0, 0xbe, 0x63, 0x8f, 0xe8, 0x19, 0x7a, 0xcc, 0xad, 0xa4, 0x78, 0x64, 0x1f, + 0x9a, 0x13, 0x2f, 0x18, 0x51, 0x0b, 0x9b, 0x94, 0x68, 0x59, 0x2a, 0xcb, 0xfc, 0x04, 0x76, 0x32, + 0xd6, 0x56, 0xf5, 0xc9, 0xb4, 0xe0, 0x81, 0x68, 0x02, 0xb2, 0xbc, 0x1d, 0xfb, 0x56, 0xa2, 0x7e, + 0xa8, 0xb4, 0x02, 0xf4, 0x16, 0xa5, 0xa2, 0x17, 0x2c, 0xaf, 0x85, 0x6f, 0x34, 0x30, 0x8a, 0x94, + 0x0a, 0x70, 0x77, 0x6a, 0xfd, 0xff, 0x76, 0x98, 0x6f, 0x34, 0xd8, 0xfd, 0x62, 0x11, 0x4c, 0x8b, + 0x9c, 0x55, 0xfc, 0xd1, 0xd2, 0x47, 0x87, 0x01, 0xf5, 0x99, 0x6b, 0x8f, 0xa2, 0xd9, 0x35, 0x15, + 0xa8, 0x62, 0x1a, 0x6b, 0x9b, 0x9d, 0x18, 0x0c, 0x58, 0xc5, 0xc2, 0x6f, 0xb6, 0x7e, 0x32, 0x73, + 0x28, 0x6e, 0x7d, 0x5e, 0xca, 0x31, 0x8d, 0x95, 0xbb, 0x18, 0x0e, 0x66, 0x81, 0xbe, 0x86, 0x12, + 0x41, 0x99, 0xbf, 0x06, 0x3d, 0x0f, 0xec, 0x5e, 0xda, 0xd7, 0x35, 0xb4, 0xfb, 0x97, 0x74, 0x74, + 0xf5, 0x5d, 0x4d, 0xb7, 0x0b, 0x35, 0x1a, 0x04, 0x7d, 0x97, 0x67, 0xa6, 0x62, 0x09, 0x8a, 0xc5, + 0xed, 0xc6, 0x0e, 0x5c, 0x26, 0xe0, 0x41, 0x90, 0xe4, 0x77, 0x1c, 0xa9, 0x1f, 0xc3, 0x96, 0x62, + 0x77, 0xe5, 0xc2, 0xfd, 0x9d, 0x06, 0x1d, 0x51, 0x64, 0xe7, 0xe8, 0x89, 0xc4, 0xbe, 0xa7, 0x94, + 0xd7, 0x06, 0x73, 0x9f, 0x8b, 0x93, 0xfa, 0x1a, 0x79, 0xee, 0x64, 0x36, 0x15, 0x45, 0x2b, 0x28, + 0x96, 0x33, 0x1e, 0x90, 0x93, 0x81, 0x38, 0x28, 0x63, 0x9a, 0x8d, 0x0e, 0x7c, 0x54, 0xf9, 0x3c, + 0xc9, 0xa8, 0xc2, 0x31, 0x17, 0xb0, 0x93, 0x41, 0x72, 0x2f, 0x89, 0x7b, 0x06, 0x3b, 0x16, 0x9d, + 0xce, 0xd8, 0x5c, 0x25, 0x97, 0xdc, 0x79, 0xec, 0xd8, 0xe3, 0x71, 0x40, 0xc3, 0x50, 0x98, 0x95, + 0xa4, 0xf9, 0x14, 0xba, 0x59, 0x35, 0x2b, 0x27, 0xe3, 0x27, 0xd0, 0x39, 0x9b, 0x4c, 0x9c, 0x99, + 0x4b, 0x5f, 0xd0, 0xf9, 0x30, 0x85, 0x24, 0xba, 0xf5, 0x63, 0x24, 0xec, 0xbb, 0x68, 0x4a, 0x61, + 0x8d, 0x2c, 0xf3, 0xfb, 0x95, 0x21, 0xfc, 0x38, 0x2e, 0x87, 0x53, 0x6a, 0x8f, 0x13, 0x08, 0xb9, + 0x72, 0xe0, 0x62, 0x5e, 0x0e, 0x68, 0x38, 0xfd, 0xab, 0x95, 0x0d, 0xff, 0x5e, 0x03, 0x78, 0x81, + 0xd3, 0xed, 0x89, 0x3b, 0xf1, 0x0a, 0x83, 0x6f, 0x40, 0x7d, 0x8e, 0x7e, 0x9d, 0x0c, 0xf0, 0x97, + 0x55, 0x2b, 0xa6, 0xd9, 0xa1, 0x67, 0x3b, 0xb3, 0xb8, 0xbf, 0x73, 0x82, 0xfd, 0xc2, 0xa7, 0x34, + 0x78, 0x69, 0x9d, 0xf2, 0xee, 0xd6, 0xb0, 0x62, 0x9a, 0x95, 0xe3, 0xc8, 0x99, 0x51, 0x37, 0x42, + 0x29, 0x3f, 0x16, 0x15, 0x8e, 0x39, 0x04, 0xe0, 0x89, 0x5c, 0x8a, 0x87, 0x40, 0x95, 0x65, 0x5f, + 0xa6, 0x80, 0x7d, 0x33, 0x1c, 0x61, 0x64, 0x4f, 0xe5, 0x89, 0xcc, 0x09, 0x6c, 0x57, 0x58, 0x6e, + 0xa2, 0xec, 0x05, 0x65, 0x9e, 0x42, 0x9b, 0x0d, 0x28, 0x3c, 0x68, 0x3c, 0x67, 0x32, 0x34, 0x5a, + 0x52, 0xd5, 0x45, 0x03, 0xa9, 0xb4, 0x5d, 0x49, 0x6c, 0x9b, 0x9f, 0x73, 0x6d, 0x3c, 0x8a, 0x4b, + 0xb5, 0x1d, 0xc0, 0x3a, 0xbf, 0x45, 0xf0, 0x03, 0xa7, 0x79, 0xdc, 0x62, 0xe9, 0x4c, 0x42, 0x6f, + 0x49, 0xb1, 0xd4, 0xc7, 0xa3, 0x70, 0x97, 0x3e, 0xbe, 0x89, 0x53, 0xfa, 0x92, 0xd0, 0x59, 0x52, + 0x6c, 0xfe, 0x4d, 0x83, 0x75, 0xae, 0x26, 0x24, 0x8f, 0xa1, 0xe6, 0xa0, 0xd7, 0xa8, 0xaa, 0x79, + 0xdc, 0xc1, 0x9a, 0xca, 0xc4, 0xe2, 0xb3, 0x92, 0x25, 0x56, 0xb1, 0xf5, 0x1c, 0x16, 0x46, 0x41, + 0x59, 0xaf, 0x7a, 0xcb, 0xd6, 0xf3, 0x55, 0x6c, 0x3d, 0x37, 0x8b, 0x11, 0x52, 0xd6, 0xab, 0xde, + 0xb0, 0xf5, 0x7c, 0xd5, 0xd3, 0x3a, 0xd4, 0x78, 0x2d, 0xb1, 0x3b, 0x0a, 0xea, 0x4d, 0xed, 0xc0, + 0x6e, 0x0a, 0x6e, 0x3d, 0x86, 0xd5, 0x4d, 0xc1, 0xaa, 0xc7, 0xe6, 0xbb, 0x29, 0xf3, 0x75, 0x69, + 0x86, 0x95, 0x07, 0x4b, 0x9f, 0xac, 0x46, 0x4e, 0x98, 0x14, 0x88, 0x6a, 0x72, 0xe5, 0xb6, 0xf7, + 0x01, 0xac, 0x73, 0xf0, 0xa9, 0x99, 0x4a, 0x84, 0xda, 0x92, 0x32, 0xf3, 0x2f, 0xe5, 0xa4, 0xd7, + 0x8f, 0x2e, 0xe9, 0xdc, 0x5e, 0xde, 0xeb, 0x51, 0x9c, 0xdc, 0x87, 0x72, 0x73, 0xe7, 0xd2, 0xfb, + 0x10, 0xdb, 0x72, 0x63, 0x3b, 0xb2, 0x87, 0x76, 0x18, 0x9f, 0xda, 0x92, 0x66, 0xde, 0x47, 0xf6, + 0xd0, 0xa1, 0xe2, 0xd0, 0xe6, 0x04, 0x6e, 0x0e, 0xb4, 0xa7, 0xd7, 0xc4, 0xe6, 0x40, 0x8a, 0xad, + 0x9e, 0x38, 0x8b, 0xf0, 0x52, 0x5f, 0xe7, 0x5b, 0x1a, 0x09, 0x86, 0x86, 0x4d, 0xa2, 0x7a, 0x1d, + 0x99, 0xf8, 0xcd, 0xb6, 0xf2, 0x24, 0xf0, 0xe6, 0xfc, 0xd8, 0xd0, 0x1b, 0xfc, 0x52, 0x9a, 0x70, + 0xa4, 0xfc, 0xc2, 0x0e, 0xa6, 0x34, 0xd2, 0x21, 0x91, 0x73, 0x8e, 0x7a, 0xf2, 0x88, 0xb8, 0xdc, + 0xcb, 0xc9, 0x73, 0x08, 0x9d, 0xe7, 0x34, 0x3a, 0x5f, 0x0c, 0xd9, 0xd9, 0xdd, 0x9f, 0x4c, 0xef, + 0x38, 0x78, 0xcc, 0x97, 0xb0, 0x93, 0x59, 0xbb, 0x32, 0x44, 0x02, 0xd5, 0xd1, 0x64, 0x2a, 0x13, + 0x86, 0xdf, 0xe6, 0x00, 0x36, 0x9f, 0xd3, 0x48, 0xb1, 0xfd, 0x48, 0x39, 0x6a, 0xc4, 0x5c, 0xd9, + 0x9f, 0x4c, 0x2f, 0x6e, 0x7d, 0x7a, 0xc7, 0xb9, 0x73, 0x0a, 0x2d, 0xa9, 0x65, 0x65, 0x54, 0x6d, + 0xa8, 0x8c, 0x26, 0xf1, 0x44, 0x3a, 0x9a, 0x4c, 0xcd, 0x1d, 0xd8, 0x7e, 0x4e, 0xc5, 0xbe, 0x4e, + 0x90, 0x99, 0x07, 0x18, 0x2d, 0x85, 0x2d, 0x4c, 0x09, 0x05, 0x5a, 0xa2, 0xe0, 0x4f, 0x1a, 0x90, + 0xcf, 0x6c, 0x77, 0xec, 0xd0, 0x67, 0x41, 0xe0, 0x05, 0x4b, 0xc7, 0x70, 0x94, 0x7e, 0xaf, 0x22, + 0xdf, 0x83, 0xc6, 0x70, 0xe6, 0x3a, 0xde, 0xf4, 0x0b, 0x2f, 0x94, 0x23, 0x59, 0xcc, 0xc0, 0x12, + 0x7d, 0xe5, 0xc4, 0x57, 0x2d, 0xf6, 0x6d, 0x86, 0xb0, 0x9d, 0x82, 0x74, 0x2f, 0x05, 0xf6, 0x1c, + 0x76, 0x2e, 0x02, 0xdb, 0x0d, 0x27, 0x34, 0x48, 0x0f, 0x77, 0xc9, 0x79, 0xa4, 0xa9, 0xe7, 0x91, + 0xd2, 0xb6, 0xb8, 0x65, 0x41, 0xb1, 0xe1, 0x26, 0xab, 0x68, 0xe5, 0x03, 0x7e, 0x1c, 0xbf, 0x93, + 0xa4, 0xee, 0x0b, 0xef, 0x2a, 0x59, 0xd9, 0x54, 0xae, 0x31, 0x5f, 0x1e, 0xcb, 0x41, 0x53, 0x20, + 0x2d, 0x2f, 0x41, 0xca, 0x53, 0x23, 0x91, 0x46, 0x71, 0x8b, 0xbb, 0xc7, 0xe1, 0xff, 0x70, 0x08, + 0x75, 0x39, 0x1e, 0x93, 0x6d, 0x78, 0xe7, 0xc4, 0xbd, 0xb6, 0x9d, 0xd9, 0x58, 0xb2, 0xda, 0x25, + 0xf2, 0x0e, 0x34, 0xf1, 0xe1, 0x8b, 0xb3, 0xda, 0x1a, 0x69, 0xc3, 0x06, 0x7f, 0x61, 0x11, 0x9c, + 0x32, 0x69, 0x01, 0x9c, 0x47, 0x9e, 0x2f, 0xe8, 0x0a, 0xd2, 0x97, 0xde, 0x8d, 0xa0, 0xab, 0x87, + 0x3f, 0x85, 0xba, 0x9c, 0xb9, 0x14, 0x1b, 0x92, 0xd5, 0x2e, 0x91, 0x2d, 0xd8, 0x7c, 0x76, 0x3d, + 0x1b, 0x45, 0x31, 0x4b, 0x23, 0xbb, 0xb0, 0xdd, 0xb7, 0xdd, 0x11, 0x75, 0xd2, 0x82, 0xf2, 0xa1, + 0x0b, 0xeb, 0x62, 0x5b, 0x33, 0x68, 0x42, 0x17, 0x23, 0xdb, 0x25, 0xb2, 0x01, 0x75, 0xd6, 0x64, + 0x90, 0xd2, 0x18, 0x0c, 0xbe, 0xe7, 0x90, 0x46, 0x98, 0x3c, 0x0a, 0x48, 0x73, 0x98, 0x08, 0x11, + 0xe9, 0x2a, 0xe9, 0x40, 0x1b, 0x7f, 0x4d, 0xe7, 0xbe, 0x63, 0x47, 0x9c, 0xbb, 0x76, 0x38, 0x80, + 0x46, 0x9c, 0x57, 0xb6, 0x44, 0x58, 0x8c, 0x79, 0xed, 0x12, 0x8b, 0x08, 0x86, 0x08, 0x79, 0x5f, + 0x1e, 0xb7, 0x35, 0x1e, 0x34, 0xcf, 0x97, 0x8c, 0xf2, 0xf1, 0xdf, 0x5b, 0x50, 0xe3, 0x60, 0xc8, + 0x57, 0xd0, 0x88, 0x5f, 0x12, 0x09, 0x1e, 0xee, 0xd9, 0x87, 0x4d, 0x63, 0x27, 0xc3, 0xe5, 0x49, + 0x33, 0x1f, 0xfd, 0xf6, 0x9f, 0xff, 0xf9, 0x73, 0xf9, 0x81, 0xd9, 0x39, 0xb2, 0xfd, 0x59, 0x78, + 0x74, 0xfd, 0xc4, 0x76, 0xfc, 0x4b, 0xfb, 0xc9, 0x11, 0xdb, 0xf2, 0xe1, 0x47, 0xda, 0x21, 0x99, + 0x40, 0x53, 0x79, 0xd0, 0x23, 0x5d, 0xa6, 0x26, 0xff, 0x84, 0x68, 0xec, 0xe6, 0xf8, 0xc2, 0xc0, + 0x87, 0x68, 0x60, 0xdf, 0x78, 0x58, 0x64, 0xe0, 0xe8, 0x35, 0xeb, 0x98, 0x5f, 0x33, 0x3b, 0x1f, + 0x03, 0x24, 0x8f, 0x6c, 0x04, 0xd1, 0xe6, 0xde, 0xed, 0x8c, 0x6e, 0x96, 0x2d, 0x8c, 0x94, 0x88, + 0x03, 0x4d, 0xe5, 0x3d, 0x8a, 0x18, 0x99, 0x07, 0x2a, 0xe5, 0x01, 0xcd, 0x78, 0x58, 0x28, 0x13, + 0x9a, 0xde, 0x47, 0xb8, 0x3d, 0xb2, 0x97, 0x81, 0x1b, 0xe2, 0x52, 0x81, 0x97, 0xf4, 0x61, 0x43, + 0x7d, 0xf6, 0x21, 0xe8, 0x7d, 0xc1, 0x7b, 0x97, 0xa1, 0xe7, 0x05, 0x31, 0xe4, 0x4f, 0x61, 0x33, + 0xf5, 0xd0, 0x42, 0x70, 0x71, 0xd1, 0x4b, 0x8f, 0xf1, 0xa0, 0x40, 0x12, 0xeb, 0xf9, 0x0a, 0xba, + 0xf9, 0x87, 0x11, 0x8c, 0xe2, 0xbb, 0x4a, 0x52, 0xf2, 0x8f, 0x13, 0x46, 0x6f, 0x99, 0x38, 0x56, + 0x7d, 0x06, 0xed, 0xec, 0x03, 0x02, 0xc1, 0xf0, 0x2d, 0x79, 0xef, 0x30, 0xf6, 0x8a, 0x85, 0xb1, + 0xc2, 0x8f, 0xa0, 0x11, 0xdf, 0xcf, 0x79, 0xa1, 0x66, 0x9f, 0x09, 0x78, 0xa1, 0xe6, 0x2e, 0xf1, + 0x66, 0x89, 0x4c, 0x61, 0x33, 0x75, 0x23, 0xe6, 0xf1, 0x2a, 0xba, 0xae, 0xf3, 0x78, 0x15, 0x5e, + 0x9f, 0xcd, 0xf7, 0x30, 0xc1, 0x0f, 0x8d, 0x6e, 0x36, 0xc1, 0xbc, 0x79, 0xb1, 0x52, 0x3c, 0x81, + 0x56, 0xfa, 0xf2, 0x4a, 0x1e, 0xf0, 0x56, 0x5c, 0x70, 0x2f, 0x36, 0x8c, 0x22, 0x51, 0x8c, 0x39, + 0x80, 0xcd, 0xd4, 0x1d, 0x54, 0x60, 0x2e, 0xb8, 0xd6, 0x0a, 0xcc, 0x45, 0x17, 0x56, 0xf3, 0x87, + 0x88, 0xf9, 0xc3, 0xc3, 0xf7, 0x33, 0x98, 0xc5, 0x28, 0x7b, 0xf4, 0x9a, 0xcd, 0x22, 0x5f, 0xcb, + 0xe2, 0xbc, 0x8a, 0xe3, 0xc4, 0x5b, 0x5c, 0x2a, 0x4e, 0xa9, 0x7b, 0x6c, 0x2a, 0x4e, 0xe9, 0xbb, + 0xaa, 0xf9, 0x01, 0xda, 0x7c, 0x64, 0x18, 0x19, 0x9b, 0x7c, 0xd4, 0x3f, 0x7a, 0xed, 0xf9, 0xb8, + 0x6d, 0x7f, 0x01, 0x90, 0x0c, 0xeb, 0x7c, 0xdb, 0xe6, 0xee, 0x0b, 0x7c, 0xdb, 0xe6, 0x67, 0x7a, + 0xb3, 0x87, 0x36, 0x74, 0xd2, 0x2d, 0xf6, 0x8b, 0x4c, 0x92, 0x8c, 0xf3, 0x21, 0x38, 0x95, 0x71, + 0x75, 0x68, 0x4f, 0x67, 0x3c, 0x35, 0xb6, 0x9a, 0xfb, 0x68, 0xc5, 0x30, 0x76, 0xb2, 0x19, 0xc7, + 0x65, 0xcc, 0x09, 0x07, 0xe7, 0xbe, 0x64, 0x9c, 0xe4, 0x76, 0x8a, 0xa6, 0x51, 0x6e, 0xa7, 0x70, + 0xf6, 0x94, 0x9d, 0x8e, 0xf4, 0xb2, 0x76, 0x16, 0x43, 0xb5, 0xd9, 0x91, 0x0b, 0xa8, 0xf1, 0xf9, + 0x90, 0x6c, 0x09, 0x65, 0x8a, 0x7e, 0xa2, 0xb2, 0x84, 0xe2, 0x1f, 0xa0, 0xe2, 0x77, 0xc9, 0x5d, + 0x2d, 0x94, 0xfc, 0x0a, 0x9a, 0xca, 0x48, 0xc5, 0xfb, 0x74, 0x7e, 0xec, 0xe3, 0x7d, 0xba, 0x60, + 0xf6, 0x5a, 0x1a, 0x25, 0xca, 0x56, 0xe1, 0xb6, 0xe8, 0xc3, 0x86, 0x3a, 0x72, 0xf2, 0xa6, 0x57, + 0x30, 0x9b, 0x1a, 0x7a, 0x5e, 0x10, 0x6f, 0x88, 0x13, 0x68, 0xa5, 0x67, 0x27, 0xbe, 0xb7, 0x0a, + 0x07, 0x33, 0xbe, 0xb7, 0x8a, 0x47, 0x2d, 0xb3, 0xc4, 0xf0, 0xa8, 0xc3, 0x0d, 0x51, 0x8f, 0xa0, + 0x54, 0x53, 0xd2, 0xf3, 0x02, 0xa9, 0xe4, 0xa9, 0xfe, 0x8f, 0x37, 0x3d, 0xed, 0xdb, 0x37, 0x3d, + 0xed, 0xdf, 0x6f, 0x7a, 0xda, 0x1f, 0xdf, 0xf6, 0x4a, 0xdf, 0xbe, 0xed, 0x95, 0xfe, 0xf5, 0xb6, + 0x57, 0x1a, 0xd6, 0xf0, 0x4f, 0xc2, 0x1f, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0xab, 0x51, 0x94, + 0xcf, 0x68, 0x1c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5141,6 +5150,13 @@ func (m *OperateSourceRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.WorkerName) > 0 { + i -= len(m.WorkerName) + copy(dAtA[i:], m.WorkerName) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.WorkerName))) + i-- + dAtA[i] = 0x22 + } if len(m.SourceID) > 0 { for iNdEx := len(m.SourceID) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.SourceID[iNdEx]) @@ -7024,6 +7040,10 @@ func (m *OperateSourceRequest) Size() (n int) { n += 1 + l + sovDmmaster(uint64(l)) } } + l = len(m.WorkerName) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -10443,6 +10463,38 @@ func (m *OperateSourceRequest) Unmarshal(dAtA []byte) error { } m.SourceID = append(m.SourceID, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkerName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WorkerName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) diff --git a/dm/dm/proto/dmmaster.proto b/dm/dm/proto/dmmaster.proto index f97bc200730..0ba158c8b8f 100644 --- a/dm/dm/proto/dmmaster.proto +++ b/dm/dm/proto/dmmaster.proto @@ -6,134 +6,134 @@ import "dmworker.proto"; // refine if needed import "google/api/annotations.proto"; service Master { - rpc StartTask (StartTaskRequest) returns (StartTaskResponse) { - option (google.api.http) = { - post: "/apis/v1alpha1/tasks" - body: "*" - }; - } - - rpc OperateTask (OperateTaskRequest) returns (OperateTaskResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/tasks/{name}" - body: "*" - }; - } - rpc UpdateTask (UpdateTaskRequest) returns (UpdateTaskResponse) {} - - rpc QueryStatus (QueryStatusListRequest) returns (QueryStatusListResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/status/{name}" - }; - } - - // show un-resolved DDL locks - rpc ShowDDLLocks (ShowDDLLocksRequest) returns (ShowDDLLocksResponse) {} - // used by dmctl to manually unlock DDL lock - rpc UnlockDDLLock (UnlockDDLLockRequest) returns (UnlockDDLLockResponse) {} - - // OperateWorkerRelayTask requests some dm-workers to operate relay unit - rpc OperateWorkerRelayTask (OperateWorkerRelayRequest) returns (OperateWorkerRelayResponse) {} - - // PurgeWorkerRelay purges relay log files for some dm-workers - rpc PurgeWorkerRelay(PurgeWorkerRelayRequest) returns (PurgeWorkerRelayResponse) {} - - // CheckTask checks legality of task configuration - rpc CheckTask(CheckTaskRequest) returns (CheckTaskResponse) {} - - // Operate an upstream MySQL source. - rpc OperateSource(OperateSourceRequest) returns (OperateSourceResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/sources" - body: "*" - }; - } - - // RegisterWorker register the dm-workers. - rpc RegisterWorker(RegisterWorkerRequest) returns(RegisterWorkerResponse) {} - - // OfflineMember offline the dm cluster's members (master/worker). - rpc OfflineMember(OfflineMemberRequest) returns(OfflineMemberResponse) { - option (google.api.http) = { - delete: "/apis/v1alpha1/members/{type}/{name}" - }; - } - - // OperateLeader do some operate on master: - // - evict leader: make the master resign if it is leader, and will not campaign the leader again - // - cancel evict leader: the master can campaign leader again. - rpc OperateLeader(OperateLeaderRequest) returns(OperateLeaderResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/leader/{op}" - body: "*" - }; - } - - // ListMember list member information - rpc ListMember(ListMemberRequest) returns(ListMemberResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/members" - }; - } - - rpc OperateSchema(OperateSchemaRequest) returns(OperateSchemaResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/schema" - body: "*" - }; - } - - rpc GetSubTaskCfg(GetSubTaskCfgRequest) returns(GetSubTaskCfgResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/subtasks/{name}" - }; - } - - // GetCfg get config - rpc GetCfg(GetCfgRequest) returns(GetCfgResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/tasks/{name}" - }; - } - - rpc HandleError(HandleErrorRequest) returns(HandleErrorResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/errors" - body: "*" - }; - } - - rpc GetMasterCfg(GetMasterCfgRequest) returns(GetMasterCfgResponse) {} - - rpc TransferSource(TransferSourceRequest) returns(TransferSourceResponse) {} - - rpc OperateRelay(OperateRelayRequest) returns(OperateRelayResponse) {} + rpc StartTask (StartTaskRequest) returns (StartTaskResponse) { + option (google.api.http) = { + post: "/apis/v1alpha1/tasks" + body: "*" + }; + } + + rpc OperateTask (OperateTaskRequest) returns (OperateTaskResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/tasks/{name}" + body: "*" + }; + } + rpc UpdateTask (UpdateTaskRequest) returns (UpdateTaskResponse) {} + + rpc QueryStatus (QueryStatusListRequest) returns (QueryStatusListResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/status/{name}" + }; + } + + // show un-resolved DDL locks + rpc ShowDDLLocks (ShowDDLLocksRequest) returns (ShowDDLLocksResponse) {} + // used by dmctl to manually unlock DDL lock + rpc UnlockDDLLock (UnlockDDLLockRequest) returns (UnlockDDLLockResponse) {} + + // OperateWorkerRelayTask requests some dm-workers to operate relay unit + rpc OperateWorkerRelayTask (OperateWorkerRelayRequest) returns (OperateWorkerRelayResponse) {} + + // PurgeWorkerRelay purges relay log files for some dm-workers + rpc PurgeWorkerRelay(PurgeWorkerRelayRequest) returns (PurgeWorkerRelayResponse) {} + + // CheckTask checks legality of task configuration + rpc CheckTask(CheckTaskRequest) returns (CheckTaskResponse) {} + + // Operate an upstream MySQL source. + rpc OperateSource(OperateSourceRequest) returns (OperateSourceResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/sources" + body: "*" + }; + } + + // RegisterWorker register the dm-workers. + rpc RegisterWorker(RegisterWorkerRequest) returns(RegisterWorkerResponse) {} + + // OfflineMember offline the dm cluster's members (master/worker). + rpc OfflineMember(OfflineMemberRequest) returns(OfflineMemberResponse) { + option (google.api.http) = { + delete: "/apis/v1alpha1/members/{type}/{name}" + }; + } + + // OperateLeader do some operate on master: + // - evict leader: make the master resign if it is leader, and will not campaign the leader again + // - cancel evict leader: the master can campaign leader again. + rpc OperateLeader(OperateLeaderRequest) returns(OperateLeaderResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/leader/{op}" + body: "*" + }; + } + + // ListMember list member information + rpc ListMember(ListMemberRequest) returns(ListMemberResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/members" + }; + } + + rpc OperateSchema(OperateSchemaRequest) returns(OperateSchemaResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/schema" + body: "*" + }; + } + + rpc GetSubTaskCfg(GetSubTaskCfgRequest) returns(GetSubTaskCfgResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/subtasks/{name}" + }; + } + + // GetCfg get config + rpc GetCfg(GetCfgRequest) returns(GetCfgResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/tasks/{name}" + }; + } + + rpc HandleError(HandleErrorRequest) returns(HandleErrorResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/errors" + body: "*" + }; + } + + rpc GetMasterCfg(GetMasterCfgRequest) returns(GetMasterCfgResponse) {} + + rpc TransferSource(TransferSourceRequest) returns(TransferSourceResponse) {} + + rpc OperateRelay(OperateRelayRequest) returns(OperateRelayResponse) {} } message StartTaskRequest { - string task = 1; // task's configuration, yaml format - repeated string sources = 2; // mysql source need to do start task, empty for all sources defined in the task config - bool removeMeta = 3; // whether to remove meta data for this task or not - string startTime = 4; // a highest priority field to specify starting of binlog replication + string task = 1; // task's configuration, yaml format + repeated string sources = 2; // mysql source need to do start task, empty for all sources defined in the task config + bool removeMeta = 3; // whether to remove meta data for this task or not + string startTime = 4; // a highest priority field to specify starting of binlog replication } message StartTaskResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message OperateTaskRequest { - TaskOp op = 1; // Stop / Pause / Resume - string name = 2; // task's name - repeated string sources = 3; // sources need to do operation, empty for matched sources in processing the task + TaskOp op = 1; // Stop / Pause / Resume + string name = 2; // task's name + repeated string sources = 3; // sources need to do operation, empty for matched sources in processing the task } message OperateTaskResponse { - TaskOp op = 1; - bool result = 2; - string msg = 3; - repeated CommonWorkerResponse sources = 4; + TaskOp op = 1; + bool result = 2; + string msg = 3; + repeated CommonWorkerResponse sources = 4; } @@ -143,26 +143,26 @@ message OperateTaskResponse { // support update partial config for syncer, loader, etc later // sources need to do update, empty for all sources in processing the task message UpdateTaskRequest { - string task = 1; - repeated string sources = 2; + string task = 1; + repeated string sources = 2; } message UpdateTaskResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message QueryStatusListRequest { - string name = 1; // task's name, empty for all tasks - repeated string sources = 2; // sources need to query, empty for all sources + string name = 1; // task's name, empty for all tasks + repeated string sources = 2; // sources need to query, empty for all sources } message QueryStatusListResponse { - bool result = 1; - string msg = 2; - repeated QueryStatusResponse sources = 3; + bool result = 1; + string msg = 2; + repeated QueryStatusResponse sources = 3; } // ShowDDLLocksRequest used to query DDL locks which are un-resolved @@ -171,8 +171,8 @@ message QueryStatusListResponse { // any DDL lock in which the source is synced or unsynced will return // if specify task and sources both, and sources not doing the task , it will return empty DDL locks message ShowDDLLocksRequest { - string task = 1; - repeated string sources = 2; // sources need to query, empty for all sources + string task = 1; + repeated string sources = 2; // sources need to query, empty for all sources } // DDLLock represents a DDL lock info (I known the name confused with DDLLockInfo, any suggestion?) @@ -185,19 +185,19 @@ message ShowDDLLocksRequest { // synced: already synced dm-workers // unsynced: pending to sync dm-workers message DDLLock { - string ID = 1; - string task = 2; - string mode = 3; - string owner = 4; - repeated string DDLs = 5; - repeated string synced = 6; - repeated string unsynced = 7; + string ID = 1; + string task = 2; + string mode = 3; + string owner = 4; + repeated string DDLs = 5; + repeated string synced = 6; + repeated string unsynced = 7; } message ShowDDLLocksResponse { - bool result = 1; - string msg = 2; - repeated DDLLock locks = 3; // all un-resolved DDL locks + bool result = 1; + string msg = 2; + repeated DDLLock locks = 3; // all un-resolved DDL locks } // UnlockDDLLockRequest used to unlock (resolve) DDL lock manually @@ -205,27 +205,27 @@ message ShowDDLLocksResponse { // replaceOwner: dm-worker used to replace the original DDL lock's owner // forceRemove: force to remove the DDL lock even fail to execute the DDL for the owner. message UnlockDDLLockRequest { - string ID = 1; - string replaceOwner = 2; - bool forceRemove = 3; + string ID = 1; + string replaceOwner = 2; + bool forceRemove = 3; } message UnlockDDLLockResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } // OperateWorkerRelayRequest represents a request for some dm-workers to operate relay unit message OperateWorkerRelayRequest { - RelayOp op = 1; // Stop / Pause / Resume - repeated string sources = 2; + RelayOp op = 1; // Stop / Pause / Resume + repeated string sources = 2; } message OperateWorkerRelayResponse { - RelayOp op = 1; - bool result = 2; - string msg = 3; - repeated CommonWorkerResponse sources = 4; + RelayOp op = 1; + bool result = 2; + string msg = 3; + repeated CommonWorkerResponse sources = 4; } // PurgeWorkerRelayRequest represents a request to purge relay log files for some dm-workers @@ -235,233 +235,234 @@ message OperateWorkerRelayResponse { // filename: whether purge relay log files before this filename // subDir: specify relay sub directory for @filename message PurgeWorkerRelayRequest { - repeated string sources = 1; - bool inactive = 2; - int64 time = 3; - string filename = 4; - string subDir = 5; + repeated string sources = 1; + bool inactive = 2; + int64 time = 3; + string filename = 4; + string subDir = 5; } message PurgeWorkerRelayResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message CheckTaskRequest { - string task = 1; // task's configuration, yaml format - int64 errCnt = 2; // max error count to display - int64 warnCnt = 3; // max warn count to display - string startTime = 4; // a highest priority field to specify starting of binlog replication + string task = 1; // task's configuration, yaml format + int64 errCnt = 2; // max error count to display + int64 warnCnt = 3; // max warn count to display + string startTime = 4; // a highest priority field to specify starting of binlog replication } message CheckTaskResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } enum SourceOp { - InvalidSourceOp = 0; - StartSource = 1; - UpdateSource = 2; - StopSource = 3; - ShowSource = 4; + InvalidSourceOp = 0; + StartSource = 1; + UpdateSource = 2; + StopSource = 3; + ShowSource = 4; } message OperateSourceRequest { - SourceOp op = 1; - repeated string config = 2; - repeated string sourceID = 3; + SourceOp op = 1; + repeated string config = 2; + repeated string sourceID = 3; + string workerName = 4; } message OperateSourceResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message RegisterWorkerRequest { - string name = 1; - string address = 2; + string name = 1; + string address = 2; } message RegisterWorkerResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message OfflineMemberRequest { - string type = 1; - string name = 2; + string type = 1; + string name = 2; } message OfflineMemberResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } enum LeaderOp { - InvalidLeaderOp = 0; - EvictLeaderOp = 1; - CancelEvictLeaderOp = 2; + InvalidLeaderOp = 0; + EvictLeaderOp = 1; + CancelEvictLeaderOp = 2; } message OperateLeaderRequest { - LeaderOp op = 1; + LeaderOp op = 1; } message OperateLeaderResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message MasterInfo { - string name = 1; - uint64 memberID = 2; - bool alive = 3; - repeated string peerURLs = 4; - repeated string clientURLs = 5; + string name = 1; + uint64 memberID = 2; + bool alive = 3; + repeated string peerURLs = 4; + repeated string clientURLs = 5; } message WorkerInfo { - string name = 1; - string addr = 2; - string stage = 3; - string source = 4; + string name = 1; + string addr = 2; + string stage = 3; + string source = 4; } message ListLeaderMember { - string msg = 1; - string name = 2; - string addr = 3; + string msg = 1; + string name = 2; + string addr = 3; } message ListMasterMember { - string msg = 1; - repeated MasterInfo masters = 2; + string msg = 1; + repeated MasterInfo masters = 2; } message ListWorkerMember { - string msg = 1; - repeated WorkerInfo workers = 2; + string msg = 1; + repeated WorkerInfo workers = 2; } message Members { - oneof member { - ListLeaderMember leader = 1; - ListMasterMember master = 2; - ListWorkerMember worker = 3; - } + oneof member { + ListLeaderMember leader = 1; + ListMasterMember master = 2; + ListWorkerMember worker = 3; + } } message ListMemberRequest { - bool leader = 1; - bool master = 2; - bool worker = 3; - repeated string names = 4; + bool leader = 1; + bool master = 2; + bool worker = 3; + repeated string names = 4; } message ListMemberResponse { - bool result = 1; - string msg = 2; - repeated Members members = 3; + bool result = 1; + string msg = 2; + repeated Members members = 3; } message OperateSchemaRequest { - SchemaOp op = 1; // operation type - string task = 2; // task name - repeated string sources = 3; // source ID list - string database = 4; // database name - string table = 5; // table name - string schema = 6; // schema content, a `CREATE TABLE` statement - bool flush = 7; // flush table info and checkpoint - bool sync = 8; // sync the table info to master - bool fromSource = 9; // update schema from source schema - bool fromTarget = 10; // update schema from target schema + SchemaOp op = 1; // operation type + string task = 2; // task name + repeated string sources = 3; // source ID list + string database = 4; // database name + string table = 5; // table name + string schema = 6; // schema content, a `CREATE TABLE` statement + bool flush = 7; // flush table info and checkpoint + bool sync = 8; // sync the table info to master + bool fromSource = 9; // update schema from source schema + bool fromTarget = 10; // update schema from target schema } message OperateSchemaResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message GetSubTaskCfgRequest { - // the task name - string name = 1; + // the task name + string name = 1; } message GetSubTaskCfgResponse { - bool result = 1; - string msg = 2; - repeated string cfgs = 3; + bool result = 1; + string msg = 2; + repeated string cfgs = 3; } enum CfgType { - InvalidType = 0; - TaskType = 1; - MasterType = 2; - WorkerType = 3; - SourceType = 4; - TaskTemplateType = 5; + InvalidType = 0; + TaskType = 1; + MasterType = 2; + WorkerType = 3; + SourceType = 4; + TaskTemplateType = 5; } message GetCfgRequest { - CfgType type = 1; // the config type - string name = 2; // the config name + CfgType type = 1; // the config type + string name = 2; // the config name } message GetCfgResponse { - bool result = 1; - string msg = 2; - string cfg = 3; + bool result = 1; + string msg = 2; + string cfg = 3; } message GetMasterCfgRequest { } message GetMasterCfgResponse { - string cfg = 1; + string cfg = 1; } message HandleErrorRequest { - ErrorOp op = 1; // operation type - string task = 2; // the task name - repeated string sources = 3; // source ID list - string binlogPos = 4; // binlog-pos (that's file:pos format) - repeated string sqls = 5; // sqls (use for replace) + ErrorOp op = 1; // operation type + string task = 2; // the task name + repeated string sources = 3; // source ID list + string binlogPos = 4; // binlog-pos (that's file:pos format) + repeated string sqls = 5; // sqls (use for replace) } message HandleErrorResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message TransferSourceRequest { - string source = 1; - string worker = 2; + string source = 1; + string worker = 2; } message TransferSourceResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message OperateRelayRequest { - RelayOpV2 op = 1; - string source = 2; - repeated string worker = 3; + RelayOpV2 op = 1; + string source = 2; + repeated string worker = 3; } message OperateRelayResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } enum RelayOpV2 { - InvalidRelayOpV2 = 0; - StartRelayV2 = 1; - StopRelayV2 = 2; + InvalidRelayOpV2 = 0; + StartRelayV2 = 1; + StopRelayV2 = 2; } diff --git a/dm/errors.toml b/dm/errors.toml index ede767d93e9..da1ebb27a3e 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -3220,6 +3220,12 @@ description = "" workaround = "Please pause task by `dmctl pause-task`." tags = ["internal", "low"] +[error.DM-scheduler-46033] +message = "dm-worker with name %s not free" +description = "" +workaround = "" +tags = ["internal", "low"] + [error.DM-dmctl-48001] message = "can not create grpc connection" description = "" diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 99e4e70488c..06c4c958ec2 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -652,6 +652,7 @@ const ( codeSchedulerStartRelayOnBound codeSchedulerStopRelayOnBound codeSchedulerPauseTaskForTransferSource + codeSchedulerWorkerNotFree ) // dmctl error code. @@ -1303,7 +1304,7 @@ var ( ErrSchedulerStartRelayOnBound = New(codeSchedulerStartRelayOnBound, ClassScheduler, ScopeInternal, LevelLow, "the source has `start-relay` automatically for bound worker, so it can't `start-relay` with worker name now", "Please stop relay by `stop-relay` without worker name first.") ErrSchedulerStopRelayOnBound = New(codeSchedulerStopRelayOnBound, ClassScheduler, ScopeInternal, LevelLow, "the source has `start-relay` automatically for bound worker, so it can't `stop-relay` with worker name now", "Please use `stop-relay` without worker name.") ErrSchedulerPauseTaskForTransferSource = New(codeSchedulerPauseTaskForTransferSource, ClassScheduler, ScopeInternal, LevelLow, "failed to auto pause tasks %s when transfer-source", "Please pause task by `dmctl pause-task`.") - + ErrSchedulerWorkerNotFree = New(codeSchedulerWorkerNotFree, ClassScheduler, ScopeInternal, LevelLow, "dm-worker with name %s not free", "") // dmctl. ErrCtlGRPCCreateConn = New(codeCtlGRPCCreateConn, ClassDMCtl, ScopeInternal, LevelHigh, "can not create grpc connection", "Please check your network connection.") ErrCtlInvalidTLSCfg = New(codeCtlInvalidTLSCfg, ClassDMCtl, ScopeInternal, LevelMedium, "invalid TLS config", "Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in command line.") diff --git a/dm/tests/_utils/env_variables b/dm/tests/_utils/env_variables index 2818e72294c..a0ae74c9ef1 100755 --- a/dm/tests/_utils/env_variables +++ b/dm/tests/_utils/env_variables @@ -27,6 +27,8 @@ WORKER2_PORT=8263 WORKER3_PORT=8264 WORKER4_PORT=18262 WORKER5_PORT=18263 +WORKER1_NAME=worker1 +WORKER2_NAME=worker2 SOURCE_ID1="mysql-replica-01" SOURCE_ID2="mysql-replica-02" RESET_MASTER=${RESET_MASTER:-true} diff --git a/dm/tests/dmctl_basic/check_list/operate_source.sh b/dm/tests/dmctl_basic/check_list/operate_source.sh index 58e82590db3..5c02f31443c 100644 --- a/dm/tests/dmctl_basic/check_list/operate_source.sh +++ b/dm/tests/dmctl_basic/check_list/operate_source.sh @@ -3,7 +3,7 @@ function operate_source_empty_arg() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "operate-source" \ - "operate-source \[config-file ...\] \[--print-sample-config\] \[flags\]" 1 + "operate-source \[config-file ...\] \[-w worker\] \[--print-sample-config\] \[flags\]" 1 } function operate_source_wrong_config_file() { diff --git a/dm/tests/dmctl_command/run.sh b/dm/tests/dmctl_command/run.sh index 8d9ba8efbd4..2b73bb8ac55 100644 --- a/dm/tests/dmctl_command/run.sh +++ b/dm/tests/dmctl_command/run.sh @@ -81,7 +81,22 @@ function run() { cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml - dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source1.yaml -w $WORKER1_NAME" \ + "\"result\": true" 2 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --name worker1" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source2.yaml -w wrong-worker" \ + "\"result\": false" 1 \ + "not exists" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source2.yaml -w worker1" \ + "\"result\": false" 1 \ + "not free" 1 dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # check wrong do-tables From 596f5588471713800041bd55b0caeba61dc0b035 Mon Sep 17 00:00:00 2001 From: Obliviate <756541536@qq.com> Date: Fri, 28 Jan 2022 13:01:13 +0800 Subject: [PATCH 34/72] checkpoint(dm): batch create table (#4477) close pingcap/tiflow#4309 --- dm/pkg/schema/tracker.go | 16 +++++ dm/pkg/schema/tracker_test.go | 112 ++++++++++++++++++++++++++++++++++ dm/syncer/checkpoint.go | 10 ++- 3 files changed, 136 insertions(+), 2 deletions(-) diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 1d2e0052bdb..1d0640cc6f9 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -370,6 +370,22 @@ func (tr *Tracker) CreateTableIfNotExists(table *filter.Table, ti *model.TableIn return tr.dom.DDL().CreateTableWithInfo(tr.se, schemaName, ti, ddl.OnExistIgnore) } +func (tr *Tracker) BatchCreateTableIfNotExist(tablesToCreate map[string]map[string]*model.TableInfo) error { + for schema, tableNameInfo := range tablesToCreate { + var cloneTis []*model.TableInfo + for table, ti := range tableNameInfo { + cloneTi := cloneTableInfo(ti) // clone TableInfo w.r.t the warning of the CreateTable function + cloneTi.Name = model.NewCIStr(table) // TableInfo has no `TableName` + cloneTis = append(cloneTis, cloneTi) + } + schemaName := model.NewCIStr(schema) + if err := tr.dom.DDL().BatchCreateTableWithInfo(tr.se, schemaName, cloneTis, ddl.OnExistIgnore); err != nil { + return err + } + } + return nil +} + // GetSystemVar gets a variable from schema tracker. func (tr *Tracker) GetSystemVar(name string) (string, bool) { return tr.se.GetSessionVars().GetSystemVar(name) diff --git a/dm/pkg/schema/tracker_test.go b/dm/pkg/schema/tracker_test.go index 5e6d9f12e97..b67ac50483c 100644 --- a/dm/pkg/schema/tracker_test.go +++ b/dm/pkg/schema/tracker_test.go @@ -445,6 +445,118 @@ func (s *trackerSuite) TestCreateTableIfNotExists(c *C) { c.Assert(duration.Seconds(), Less, float64(30)) } +func (s *trackerSuite) TestBatchCreateTableIfNotExist(c *C) { + log.SetLevel(zapcore.ErrorLevel) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) + c.Assert(err, IsNil) + err = tracker.CreateSchemaIfNotExists("testdb") + c.Assert(err, IsNil) + err = tracker.CreateSchemaIfNotExists("testdb2") + c.Assert(err, IsNil) + + tables := []*filter.Table{ + { + Schema: "testdb", + Name: "foo", + }, + { + Schema: "testdb", + Name: "foo1", + }, + { + Schema: "testdb2", + Name: "foo3", + }, + } + execStmt := []string{ + `create table foo( + a int primary key auto_increment, + b int as (c+1) not null, + c int comment 'some cmt', + d text, + key dk(d(255)) + ) comment 'more cmt' partition by range columns (a) ( + partition x41 values less than (41), + partition x82 values less than (82), + partition rest values less than maxvalue comment 'part cmt' + );`, + `create table foo1( + a int primary key, + b text not null, + d datetime, + e varchar(5) + );`, + `create table foo3( + a int, + b int, + primary key(a));`, + } + tiInfos := make([]*model.TableInfo, len(tables)) + for i := range tables { + ctx := context.Background() + err = tracker.Exec(ctx, tables[i].Schema, execStmt[i]) + c.Assert(err, IsNil) + tiInfos[i], err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + c.Assert(tiInfos[i], NotNil) + c.Assert(tiInfos[i].Name.O, Equals, tables[i].Name) + tiInfos[i] = tiInfos[i].Clone() + clearVolatileInfo(tiInfos[i]) + } + // drop all tables and recover + // 1. drop + for i := range tables { + err = tracker.DropTable(tables[i]) + c.Assert(err, IsNil) + _, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, ErrorMatches, `.*Table 'testdb.*\.foo.*' doesn't exist`) // drop table success + } + // 2. recover + tablesToCreate := map[string]map[string]*model.TableInfo{} + tablesToCreate["testdb"] = map[string]*model.TableInfo{} + tablesToCreate["testdb2"] = map[string]*model.TableInfo{} + for i := range tables { + tablesToCreate[tables[i].Schema][tables[i].Name] = tiInfos[i] + } + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, IsNil) + // 3. check all create success + for i := range tables { + var ti *model.TableInfo + ti, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + cloneTi := ti.Clone() + clearVolatileInfo(cloneTi) + c.Assert(cloneTi, DeepEquals, tiInfos[i]) + } + + // drop two tables and create all three + // expect: silently succeed + // 1. drop table + err = tracker.DropTable(tables[2]) + c.Assert(err, IsNil) + err = tracker.DropTable(tables[0]) + c.Assert(err, IsNil) + // 2. batch create + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, IsNil) + // 3. check + for i := range tables { + var ti *model.TableInfo + ti, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + clearVolatileInfo(ti) + c.Assert(ti, DeepEquals, tiInfos[i]) + } + + // drop schema and raise error + ctx := context.Background() + err = tracker.Exec(ctx, "", `drop database testdb`) + c.Assert(err, IsNil) + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, NotNil) +} + func (s *trackerSuite) TestAllSchemas(c *C) { log.SetLevel(zapcore.ErrorLevel) ctx := context.Background() diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 49d83a344b2..0873d607c27 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -844,6 +844,7 @@ func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { cp.RLock() defer cp.RUnlock() cp.globalPoint.rollback(schemaTracker, "") + tablesToCreate := make(map[string]map[string]*model.TableInfo) for schemaName, mSchema := range cp.points { for tableName, point := range mSchema { table := &filter.Table{ @@ -864,13 +865,18 @@ func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { if err := schemaTracker.CreateSchemaIfNotExists(schemaName); err != nil { logger.Error("failed to rollback schema on schema tracker: cannot create schema", log.ShortError(err)) } - if err := schemaTracker.CreateTableIfNotExists(table, point.savedPoint.ti); err != nil { - logger.Error("failed to rollback schema on schema tracker: cannot create table", log.ShortError(err)) + if _, ok := tablesToCreate[schemaName]; !ok { + tablesToCreate[schemaName] = map[string]*model.TableInfo{} } + tablesToCreate[schemaName][tableName] = point.savedPoint.ti } } } } + logger := cp.logCtx.L().WithFields(zap.Reflect("batch create table", tablesToCreate)) + if err := schemaTracker.BatchCreateTableIfNotExist(tablesToCreate); err != nil { + logger.Error("failed to rollback schema on schema tracker: cannot create table", log.ShortError(err)) + } // drop any tables in the tracker if no corresponding checkpoint exists. for _, schema := range schemaTracker.AllSchemas() { From f710abba6417971370e0d0711d9cd8ca3fa784c4 Mon Sep 17 00:00:00 2001 From: amyangfei Date: Mon, 7 Feb 2022 12:23:35 +0800 Subject: [PATCH 35/72] sink(ticdc): fix duplicated replace when batch-replace is disabled (#4502) close pingcap/tiflow#4501 --- cdc/sink/mysql.go | 2 - cdc/sink/mysql_test.go | 85 ++++++++++++++++++++++++++++-------------- 2 files changed, 56 insertions(+), 31 deletions(-) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 1b2dc9f5602..7b81d0a24b0 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -678,8 +678,6 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, } } else { query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */, translateToInsert) - sqls = append(sqls, query) - values = append(values, args) if query != "" { sqls = append(sqls, query) values = append(values, args) diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index cf42a6cb8c8..cc0b119350c 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -58,41 +58,68 @@ func TestPrepareDML(t *testing.T) { testCases := []struct { input []*model.RowChangedEvent expected *preparedDMLs - }{{ - input: []*model.RowChangedEvent{}, - expected: &preparedDMLs{sqls: []string{}, values: [][]interface{}{}}, - }, { - input: []*model.RowChangedEvent{ - { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, - PreColumns: []*model.Column{nil, { - Name: "a1", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }, { - Name: "a3", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }}, - IndexColumns: [][]int{{1, 2}}, + }{ + { + input: []*model.RowChangedEvent{}, + expected: &preparedDMLs{sqls: []string{}, values: [][]interface{}{}}, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + expected: &preparedDMLs{ + sqls: []string{"DELETE FROM `common_1`.`uk_without_pk` WHERE `a1` = ? AND `a3` = ? LIMIT 1;"}, + values: [][]interface{}{{1, 1}}, + rowCount: 1, + }, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813516, + CommitTs: 418658114257813517, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + Columns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + expected: &preparedDMLs{ + sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk`(`a1`,`a3`) VALUES (?,?);"}, + values: [][]interface{}{{2, 2}}, + rowCount: 1, }, }, - expected: &preparedDMLs{ - sqls: []string{"DELETE FROM `common_1`.`uk_without_pk` WHERE `a1` = ? AND `a3` = ? LIMIT 1;"}, - values: [][]interface{}{{1, 1}}, - rowCount: 1, - }, - }} + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() ms := newMySQLSink4Test(ctx, t) - for i, tc := range testCases { + for _, tc := range testCases { dmls := ms.prepareDMLs(tc.input, 0, 0) - require.Equal(t, tc.expected, dmls, tc.expected, fmt.Sprintf("%d", i)) + require.Equal(t, tc.expected, dmls) } } From 5c7d2388fef19f49cdc517a03acd303649fa4e5a Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Mon, 7 Feb 2022 19:05:35 +0800 Subject: [PATCH 36/72] Optimistic: support start task with inconsistent upstream table schema (#3903) close pingcap/tiflow#3629 --- Makefile | 13 +- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/common/common.go | 5 +- dm/dm/common/common_test.go | 23 - dm/dm/master/scheduler/scheduler.go | 13 + dm/dm/master/scheduler/scheduler_test.go | 16 + dm/dm/master/server.go | 16 +- dm/dm/master/server_test.go | 4 +- dm/dm/master/shardddl/optimist.go | 233 ++++------ dm/dm/master/shardddl/optimist_test.go | 86 +--- dm/errors.toml | 6 + dm/pkg/shardddl/optimism/info.go | 7 +- dm/pkg/shardddl/optimism/info_test.go | 2 +- dm/pkg/shardddl/optimism/keeper.go | 160 ++++--- dm/pkg/shardddl/optimism/keeper_test.go | 174 ++++--- dm/pkg/shardddl/optimism/lock.go | 160 ++++++- dm/pkg/shardddl/optimism/lock_test.go | 194 +++++++- dm/pkg/shardddl/optimism/ops.go | 34 +- dm/pkg/shardddl/optimism/ops_test.go | 18 +- dm/pkg/shardddl/optimism/schema.go | 151 ------- dm/pkg/shardddl/optimism/schema_test.go | 109 ----- dm/pkg/terror/error_list.go | 2 + dm/syncer/checkpoint.go | 88 ++-- dm/syncer/optimist.go | 15 - dm/syncer/schema.go | 2 +- dm/syncer/shardddl/optimist.go | 38 +- dm/syncer/shardddl/optimist_test.go | 47 +- dm/syncer/syncer.go | 39 +- dm/tests/_utils/shardddl_lib.sh | 36 ++ dm/tests/_utils/test_prepare | 2 +- dm/tests/downstream_more_column/run.sh | 2 +- dm/tests/others_integration_2.txt | 1 + dm/tests/sequence_sharding_optimistic/run.sh | 4 +- dm/tests/shardddl1/run.sh | 9 +- .../shardddl_optimistic/conf/diff_config.toml | 44 ++ .../shardddl_optimistic/conf/dm-master.toml | 10 + .../shardddl_optimistic/conf/dm-worker1.toml | 2 + .../shardddl_optimistic/conf/dm-worker2.toml | 2 + .../conf/double-source-optimistic-incr.yaml | 63 +++ .../conf/double-source-optimistic.yaml | 58 +++ .../shardddl_optimistic/conf/source1.yaml | 11 + .../shardddl_optimistic/conf/source2.yaml | 11 + dm/tests/shardddl_optimistic/run.sh | 426 ++++++++++++++++++ 43 files changed, 1528 insertions(+), 809 deletions(-) delete mode 100644 dm/pkg/shardddl/optimism/schema.go delete mode 100644 dm/pkg/shardddl/optimism/schema_test.go create mode 100644 dm/tests/shardddl_optimistic/conf/diff_config.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-master.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-worker1.toml create mode 100644 dm/tests/shardddl_optimistic/conf/dm-worker2.toml create mode 100644 dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml create mode 100644 dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml create mode 100644 dm/tests/shardddl_optimistic/conf/source1.yaml create mode 100644 dm/tests/shardddl_optimistic/conf/source2.yaml create mode 100644 dm/tests/shardddl_optimistic/run.sh diff --git a/Makefile b/Makefile index 2b12c40d678..c338ba4dac1 100644 --- a/Makefile +++ b/Makefile @@ -277,13 +277,22 @@ dm_generate_openapi: tools/bin/oapi-codegen cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/types-gen-cfg.yaml openapi/spec/dm.yaml cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/client-gen-cfg.yaml openapi/spec/dm.yaml -dm_unit_test: check_failpoint_ctl +define run_dm_unit_test + @echo "running unit test for packages:" $(1) mkdir -p $(DM_TEST_DIR) $(FAILPOINT_ENABLE) @export log_level=error; \ - $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(DM_PACKAGES) \ + $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(1) \ || { $(FAILPOINT_DISABLE); exit 1; } $(FAILPOINT_DISABLE) +endef + +dm_unit_test: check_failpoint_ctl + $(call run_dm_unit_test,$(DM_PACKAGES)) + +# run unit test for the specified pkg only, like `make dm_unit_test_pkg PKG=github.com/pingcap/tiflow/dm/dm/master` +dm_unit_test_pkg: check_failpoint_ctl + $(call run_dm_unit_test,$(PKG)) dm_unit_test_in_verify_ci: check_failpoint_ctl tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml mkdir -p $(DM_TEST_DIR) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 9e115d4ce3a..2697655c99c 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -395,6 +395,7 @@ ErrMasterBoundChanging,[code=38052:class=dm-master:scope=internal:level=low], "M ErrMasterFailToImportFromV10x,[code=38053:class=dm-master:scope=internal:level=high], "Message: fail to import DM cluster from v1.0.x, Workaround: Please confirm that you have not violated any restrictions in the upgrade documentation." ErrMasterInconsistentOptimisticDDLsAndInfo,[code=38054:class=dm-master:scope=internal:level=high], "Message: inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d" ErrMasterOptimisticTableInfoBeforeNotExist,[code=38055:class=dm-master:scope=internal:level=high], "Message: table-info-before not exist in optimistic ddls: %v" +ErrMasterOptimisticDownstreamMetaNotFound,[code=38056:class=dm-master:scope=internal:level=high], "Message: downstream database config and meta for task %s not found" ErrWorkerParseFlagSet,[code=40001:class=dm-worker:scope=internal:level=medium], "Message: parse dm-worker config flag set" ErrWorkerInvalidFlag,[code=40002:class=dm-worker:scope=internal:level=medium], "Message: '%s' is an invalid flag" ErrWorkerDecodeConfigFromFile,[code=40003:class=dm-worker:scope=internal:level=medium], "Message: toml decode file, Workaround: Please check the configuration file has correct TOML format." diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index 4464dbb5b2b..a447c8ee373 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -78,7 +78,8 @@ var ( ShardDDLOptimismOperationKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/operation/") // ShardDDLOptimismInitSchemaKeyAdapter is used to store the initial schema (before constructed the lock) of merged tables. // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. - ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") + // TODO: prune in etcd when upgrade + // ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped // k/v: Encode(lock-id, column-name, source-id, upstream-schema-name, upstream-table-name) -> int // If we don't identify different upstream tables, we may report an error for tb2 in the following case. @@ -112,8 +113,6 @@ func keyAdapterKeysLen(s KeyAdapter) int { ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter, TaskCliArgsKeyAdapter: return 2 - case ShardDDLOptimismInitSchemaKeyAdapter: - return 3 case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 case ShardDDLOptimismDroppedColumnsKeyAdapter: diff --git a/dm/dm/common/common_test.go b/dm/dm/common/common_test.go index b2669ca5cc9..0dbebb690d3 100644 --- a/dm/dm/common/common_test.go +++ b/dm/dm/common/common_test.go @@ -16,7 +16,6 @@ package common import ( "net" "path" - "strings" "testing" . "github.com/pingcap/check" @@ -61,11 +60,6 @@ func (t *testCommon) TestKeyAdapter(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/74657374", }, - { - keys: []string{"test", "target_db", "target_table"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/7461726765745f7461626c65", - }, { keys: []string{"test", "mysql_replica_01", "target_db", "target_table"}, adapter: ShardDDLOptimismInfoKeyAdapter, @@ -108,11 +102,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/", }, - { - keys: []string{"test", "target_db"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/", - }, } for _, ca := range testCases { @@ -121,18 +110,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { _, err := ca.adapter.Decode(encKey) c.Assert(err, NotNil) } - - keys := []string{"test", "target_db", "target_table"} - fullEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys...) - prefixEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys[:len(keys)-1]...) - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey), IsTrue) - - keys2 := []string{"test", "target_db_2", "target_table_2"} - fullEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2...) - prefixEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2[:len(keys2)-1]...) - - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey2), IsFalse) - c.Assert(strings.HasPrefix(fullEncodedKey2, prefixEncodedKey), IsFalse) } func (t *testCommon) TestIsErrNetClosing(c *C) { diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index e5b899a4837..c0784c57392 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -1020,6 +1020,19 @@ func (s *Scheduler) getSubTaskCfgByTaskSource(task, source string) *config.SubTa return &clone } +// GetDownstreamMetaByTask gets downstream db config and meta config by task name. +func (s *Scheduler) GetDownstreamMetaByTask(task string) (*config.DBConfig, string) { + v, ok := s.subTaskCfgs.Load(task) + if !ok { + return nil, "" + } + cfgM := v.(map[string]config.SubTaskConfig) + for _, cfg := range cfgM { + return cfg.To.Clone(), cfg.MetaSchema + } + return nil, "" +} + // GetSubTaskCfgsByTask gets subtask configs' map by task name. func (s *Scheduler) GetSubTaskCfgsByTask(task string) map[string]*config.SubTaskConfig { v, ok := s.subTaskCfgs.Load(task) diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index f045456407b..e9eb1627e6d 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -254,12 +254,15 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { c.Assert(s.AddSubTasks(false), IsNil) // can call without configs, return without error, but take no effect. t.subTaskCfgNotExist(c, s, taskName1, sourceID1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_InvalidStage) + t.downstreamMetaNotExist(c, s, taskName1) // start the task. c.Assert(s.AddSubTasks(false, subtaskCfg1), IsNil) c.Assert(terror.ErrSchedulerSubTaskExist.Equal(s.AddSubTasks(false, subtaskCfg1)), IsTrue) // add again. // subtask config and stage exist. t.subTaskCfgExist(c, s, subtaskCfg1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_Running) + t.downstreamMetaExist(c, s, taskName1, subtaskCfg1.To, subtaskCfg1.MetaSchema) + t.downstreamMetaNotExist(c, s, taskName2) // update source config when task already started will failed c.Assert(terror.ErrSchedulerSourceOpTaskExist.Equal(s.UpdateSourceCfg(sourceCfg1)), IsTrue) @@ -629,6 +632,19 @@ func (t *testScheduler) subTaskCfgExist(c *C, s *Scheduler, expectCfg config.Sub c.Assert(cfgM[expectCfg.Name], DeepEquals, expectCfg) } +func (t *testScheduler) downstreamMetaNotExist(c *C, s *Scheduler, task string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, IsNil) + c.Assert(metaConfig, Equals, "") +} + +func (t *testScheduler) downstreamMetaExist(c *C, s *Scheduler, task string, expectDBCfg config.DBConfig, expectMetaConfig string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, NotNil) + c.Assert(dbConfig, DeepEquals, &expectDBCfg) + c.Assert(metaConfig, Equals, expectMetaConfig) +} + func (t *testScheduler) workerNotExist(c *C, s *Scheduler, worker string) { c.Assert(s.GetWorkerByName(worker), IsNil) wm, _, err := ha.GetAllWorkerInfo(etcdTestCli) diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 4e24e10cfb9..48691c15c8e 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -132,7 +132,7 @@ func NewServer(cfg *Config) *Server { ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), } server.pessimist = shardddl.NewPessimist(&logger, server.getTaskResources) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) server.closed.Store(true) setUseTLS(&cfg.Security) @@ -590,6 +590,18 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* resp.Result = true resp.Sources = s.getSourceRespsAfterOperation(ctx, req.Name, sources, []string{}, req) + + if expect == pb.Stage_Stopped { + // delete meta data for optimist + if len(req.Sources) == 0 { + err2 = s.optimist.RemoveMetaDataWithTask(req.Name) + } else { + err2 = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + } + if err2 != nil { + log.L().Error("failed to delete metadata for task", zap.String("task name", req.Name), log.ShortError(err2)) + } + } return resp, nil } @@ -1558,7 +1570,7 @@ func (s *Server) removeMetaData(ctx context.Context, taskName, metaSchema string if err != nil { return err } - err = s.optimist.RemoveMetaData(taskName) + err = s.optimist.RemoveMetaDataWithTask(taskName) if err != nil { return err } diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index 73d79b41275..1438246ef44 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -960,7 +960,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} @@ -1045,7 +1045,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( p = parser.New() diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index bab8ef7e86d..58a899b4c05 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -53,11 +52,11 @@ type Optimist struct { } // NewOptimist creates a new Optimist instance. -func NewOptimist(pLogger *log.Logger) *Optimist { +func NewOptimist(pLogger *log.Logger, getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *Optimist { return &Optimist{ logger: pLogger.WithFields(zap.String("component", "shard DDL optimist")), closed: true, - lk: optimism.NewLockKeeper(), + lk: optimism.NewLockKeeper(getDownstreamMetaFunc), tk: optimism.NewTableKeeper(), } } @@ -162,9 +161,11 @@ func (o *Optimist) ShowLocks(task string, sources []string) []*pb.DDLLock { return ret } -// RemoveMetaData removes meta data for a specified task +// RemoveMetaDataWithTask removes meta data for a specified task // NOTE: this function can only be used when the specified task is not running. -func (o *Optimist) RemoveMetaData(task string) error { +// This function only be used when --remove-meta or stop-task +// NOTE: For stop-task, we still delete drop columns in etcd though user may restart the task again later. +func (o *Optimist) RemoveMetaDataWithTask(task string) error { o.mu.Lock() defer o.mu.Unlock() if o.closed { @@ -185,10 +186,43 @@ func (o *Optimist) RemoveMetaData(task string) error { o.lk.RemoveLock(op.ID) } + o.lk.RemoveDownstreamMeta(task) o.tk.RemoveTableByTask(task) // clear meta data in etcd - _, err = optimism.DeleteInfosOperationsTablesSchemasByTask(o.cli, task, lockIDSet) + _, err = optimism.DeleteInfosOperationsTablesByTask(o.cli, task, lockIDSet) + return err +} + +// RemoveMetaDataWithTaskAndSources removes meta data for a specified task and sources +// NOTE: this function can only be used when the specified task for source is not running. +func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...string) error { + o.mu.Lock() + defer o.mu.Unlock() + if o.closed { + return terror.ErrMasterOptimistNotStarted.Generate() + } + + dropColumns := make(map[string][]string) + + // gets all locks for this task + locks := o.lk.FindLocksByTask(task) + for _, lock := range locks { + // remove table by sources for related lock + cols := lock.TryRemoveTableBySources(sources) + dropColumns[lock.ID] = cols + o.logger.Debug("the tables removed from the lock", zap.String("task", task), zap.Strings("sources", sources)) + if !lock.HasTables() { + o.lk.RemoveLock(lock.ID) + } + } + + o.lk.RemoveDownstreamMeta(task) + // remove source table in table keeper + o.tk.RemoveTableByTaskAndSources(task, sources) + o.logger.Debug("the tables removed from the table keeper", zap.String("task", task), zap.Strings("source", sources)) + // clear meta data in etcd + _, err := optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, sources, dropColumns) return err } @@ -250,26 +284,22 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) - initSchemas, revInitSchemas, err := optimism.GetAllInitSchemas(o.cli) - if err != nil { - return 0, 0, 0, err - } - o.logger.Info("get all init schemas", zap.Int64("revision", revInitSchemas)) - colm, _, err := optimism.GetAllDroppedColumns(o.cli) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected columns can be handled by the user. o.logger.Error("fail to recover colms", log.ShortError(err)) } + o.lk.SetDropColumns(colm) // recover the shard DDL lock based on history shard DDL info & lock operation. - err = o.recoverLocks(ifm, opm, colm, initSchemas) + err = o.recoverLocks(ifm, opm) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected locks can be handled by the user. o.logger.Error("fail to recover locks", log.ShortError(err)) } + o.lk.SetDropColumns(nil) return revSource, revInfo, revOperation, nil } @@ -295,96 +325,11 @@ func sortInfos(ifm map[string]map[string]map[string]map[string]optimism.Info) [] return infos } -// buildLockJoinedAndTTS build joined table and target table slice for lock by history infos. -func (o *Optimist) buildLockJoinedAndTTS( - ifm map[string]map[string]map[string]map[string]optimism.Info, - initSchemas map[string]map[string]map[string]optimism.InitSchema) ( - map[string]schemacmp.Table, map[string][]optimism.TargetTable, - map[string]map[string]map[string]map[string]schemacmp.Table) { - type infoKey struct { - lockID string - source string - upSchema string - upTable string - } - infos := make(map[infoKey]optimism.Info) - lockTTS := make(map[string][]optimism.TargetTable) - for _, taskInfos := range ifm { - for _, sourceInfos := range taskInfos { - for _, schemaInfos := range sourceInfos { - for _, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if _, ok := lockTTS[lockID]; !ok { - lockTTS[lockID] = o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) - } - infos[infoKey{lockID, info.Source, info.UpSchema, info.UpTable}] = info - } - } - } - } - - lockJoined := make(map[string]schemacmp.Table) - missTable := make(map[string]map[string]map[string]map[string]schemacmp.Table) - for lockID, tts := range lockTTS { - for _, tt := range tts { - for upSchema, tables := range tt.UpTables { - for upTable := range tables { - var table schemacmp.Table - if info, ok := infos[infoKey{lockID, tt.Source, upSchema, upTable}]; ok && info.TableInfoBefore != nil { - table = schemacmp.Encode(info.TableInfoBefore) - } else if initSchema, ok := initSchemas[tt.Task][tt.DownSchema][tt.DownTable]; ok { - // If there is no optimism.Info for a upstream table, it indicates the table structure - // hasn't been changed since last removeLock. So the init schema should be its table info. - table = schemacmp.Encode(initSchema.TableInfo) - if _, ok := missTable[lockID]; !ok { - missTable[lockID] = make(map[string]map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source]; !ok { - missTable[lockID][tt.Source] = make(map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source][upSchema]; !ok { - missTable[lockID][tt.Source][upSchema] = make(map[string]schemacmp.Table) - } - missTable[lockID][tt.Source][upSchema][upTable] = table - } else { - o.logger.Error( - "can not find table info for upstream table", - zap.String("source", tt.Source), - zap.String("up-schema", upSchema), - zap.String("up-table", upTable), - ) - continue - } - if joined, ok := lockJoined[lockID]; !ok { - lockJoined[lockID] = table - } else { - newJoined, err := joined.Join(table) - // ignore error, will report it in TrySync later - if err != nil { - o.logger.Error(fmt.Sprintf("fail to join table info %s with %s, lockID: %s in recover lock", joined, newJoined, lockID), log.ShortError(err)) - } else { - lockJoined[lockID] = newJoined - } - } - } - } - } - } - return lockJoined, lockTTS, missTable -} - // recoverLocks recovers shard DDL locks based on shard DDL info and shard DDL lock operation. func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, - colm map[string]map[string]map[string]map[string]map[string]optimism.DropColumnStage, - initSchemas map[string]map[string]map[string]optimism.InitSchema) error { - // construct joined table based on the shard DDL info. - o.logger.Info("build lock joined and tts") - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - // build lock and restore table info - o.logger.Info("rebuild locks and tables") - o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS, missTable) +) error { // sort infos by revision infos := sortInfos(ifm) var firstErr error @@ -395,12 +340,18 @@ func (o *Optimist) recoverLocks( } for _, info := range infos { + if info.IsDeleted { + // TODO: handle drop table + continue + } + if !o.tk.SourceTableExist(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) { + continue + } // never mark the lock operation from `done` to `not-done` when recovering. err := o.handleInfo(info, true) if err != nil { o.logger.Error("fail to handle info while recovering locks", zap.Error(err)) setFirstErr(err) - continue } } @@ -419,7 +370,6 @@ func (o *Optimist) recoverLocks( err := lock.DeleteColumnsByOp(op) if err != nil { o.logger.Error("fail to update lock columns", zap.Error(err)) - continue } } } @@ -595,43 +545,45 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. // avoid new ddl added while previous ddl resolved and remove lock // change lock granularity if needed o.mu.Lock() - lock := o.lk.FindLock(op.ID) - if lock == nil { - o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - err := lock.DeleteColumnsByOp(op) - if err != nil { - o.logger.Error("fail to update lock columns", zap.Error(err)) - } - // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. - // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, - // because these tables may have different schemas. - done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) - o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) - if !lock.IsResolved() { - o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - // the lock has done, remove the lock. - o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) - deleted, err := o.removeLock(lock) - if err != nil { - o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) - metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) - } - if deleted { - o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) - } + o.handleOperation(op) o.mu.Unlock() } } } +func (o *Optimist) handleOperation(op optimism.Operation) { + lock := o.lk.FindLock(op.ID) + if lock == nil { + o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) + return + } + + err := lock.DeleteColumnsByOp(op) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + } + // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. + // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, + // because these tables may have different schemas. + done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) + o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) + if !lock.IsResolved() { + o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) + return + } + + // the lock has done, remove the lock. + o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) + deleted, err := o.removeLock(lock) + if err != nil { + o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) + metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) + } + if deleted { + o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) + } +} + // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { cfStage := optimism.ConflictNone @@ -649,18 +601,6 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk default: o.logger.Info("the shard DDL lock returned some DDLs", zap.String("lock", lockID), zap.Strings("ddls", newDDLs), zap.Strings("cols", cols), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted)) - - // try to record the init schema before applied the DDL to the downstream. - initSchema := optimism.NewInitSchema(info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore) - rev, putted, err2 := optimism.PutInitSchemaIfNotExist(o.cli, initSchema) - switch { - case err2 != nil: - return err2 - case putted: - o.logger.Info("recorded the initial schema", zap.String("info", info.ShortString())) - default: - o.logger.Debug("skip to record the initial schema", zap.String("info", info.ShortString()), zap.Int64("revision", rev)) - } } lock := o.lk.FindLock(lockID) @@ -761,8 +701,7 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. - initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) - rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, initSchema) + rev, deleted, err := optimism.DeleteInfosOperationsColumns(o.cli, infos, ops, lock.ID) if err != nil { return deleted, err } diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index d8226ee996c..b3e6c84e13e 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -20,7 +20,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -43,7 +42,7 @@ var _ = SerialSuites(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -104,7 +103,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -164,7 +163,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { o.Close() // CASE 4: create (not re-start) a new optimist with previous source tables. - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, etcdTestCli), IsNil) tts = o.tk.FindTables(task, downSchema, downTable) c.Assert(tts, HasLen, 2) @@ -197,14 +196,14 @@ func (t *testOptimist) TestOptimist(c *C) { func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( backOff = 30 waitTime = 100 * time.Millisecond logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) rebuildOptimist = func(ctx context.Context) { switch restart { @@ -213,7 +212,7 @@ func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { c.Assert(o.Start(ctx, cli), IsNil) case restartNewInstance: o.Close() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, cli), IsNil) } } @@ -615,7 +614,7 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { var ( watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task-test-optimist" source1 = "mysql-replica-1" downSchema = "foo" @@ -724,7 +723,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-lock-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -919,7 +918,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-init-schema" source = "mysql-replica-1" upSchema = "foo" @@ -954,11 +953,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { c.Assert(o.Start(ctx, etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) - // no init schema exist now. - is, _, err := optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) - // PUT i11, will creat a lock. _, err = optimism.PutInfo(etcdTestCli, i11) c.Assert(err, IsNil) @@ -967,11 +961,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti0) // the init schema. - // PUT i12, the lock will be synced. rev1, err := optimism.PutInfo(etcdTestCli, i12) c.Assert(err, IsNil) @@ -1011,11 +1000,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 0 }), IsTrue) - // the init schema should also be deleted. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) - // PUT i21 to create the lock again. _, err = optimism.PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) @@ -1023,16 +1007,11 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 1 }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - - // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. } func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( @@ -1104,7 +1083,7 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1146,19 +1125,6 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) - - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) - - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, nil) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err := joined.Compare(schemacmp.Encode(ti2)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) } func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { @@ -1166,7 +1132,7 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1182,11 +1148,10 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY, b INT)`) ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY)`) - ddlDropB = "ALTER TABLE bar DROP COLUMN b" - ddlDropC = "ALTER TABLE bar DROP COLUMN c" - infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) - infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) - initSchema = optimism.NewInitSchema(task, downSchema, downTable, ti0) + ddlDropB = "ALTER TABLE bar DROP COLUMN b" + ddlDropC = "ALTER TABLE bar DROP COLUMN c" + infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) + infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) ) ctx, cancel := context.WithCancel(context.Background()) @@ -1209,21 +1174,8 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) +} - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) - - initSchemas := map[string]map[string]map[string]optimism.InitSchema{task: {downSchema: {downTable: initSchema}}} - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 1) - cmp, err := missTable[utils.GenDDLLockID(task, downSchema, downTable)][source2]["foo"]["bar-1"].Compare(schemacmp.Encode(initSchema.TableInfo)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err = joined.Compare(schemacmp.Encode(ti0)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" } diff --git a/dm/errors.toml b/dm/errors.toml index da1ebb27a3e..96bab495cf8 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2380,6 +2380,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38056] +message = "downstream database config and meta for task %s not found" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-worker-40001] message = "parse dm-worker config flag set" description = "" diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index eb0179a7a58..bdb43c067ce 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -298,15 +298,14 @@ func deleteInfoOp(info Info) clientv3.Op { info.Task, info.Source, info.UpSchema, info.UpTable)) } -// ClearTestInfoOperationSchema is used to clear all shard DDL information in optimism mode. +// ClearTestInfoOperationColumns is used to clear all shard DDL information in optimism mode. // it only used for testing now. -func ClearTestInfoOperationSchema(cli *clientv3.Client) error { +func ClearTestInfoOperationColumn(cli *clientv3.Client) error { clearSource := clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Path(), clientv3.WithPrefix()) clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) - clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp, clearColumns).Commit() + _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearColumns).Commit() return err } diff --git a/dm/pkg/shardddl/optimism/info_test.go b/dm/pkg/shardddl/optimism/info_test.go index 63df2a422a1..44a35c1c41d 100644 --- a/dm/pkg/shardddl/optimism/info_test.go +++ b/dm/pkg/shardddl/optimism/info_test.go @@ -47,7 +47,7 @@ func TestInfo(t *testing.T) { // clear keys in etcd test cluster. func clearTestInfoOperation(c *C) { - c.Assert(ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index b88493e78f1..f1f0d3dc57c 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -20,79 +20,63 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" ) +// DownstreamMeta used to fetch table info from downstream. +type DownstreamMeta struct { + dbConfig *config.DBConfig + meta string +} + // LockKeeper used to keep and handle DDL lock conveniently. // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. +// But the drop columns should be persistent. type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock + + downstreamMetaMap map[string]*DownstreamMeta + getDownstreamMetaFunc func(string) (*config.DBConfig, string) + // lockID -> column name -> source -> upSchema -> upTable -> int + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. -func NewLockKeeper() *LockKeeper { +func NewLockKeeper(getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *LockKeeper { return &LockKeeper{ - locks: make(map[string]*Lock), + locks: make(map[string]*Lock), + downstreamMetaMap: make(map[string]*DownstreamMeta), + getDownstreamMetaFunc: getDownstreamMetaFunc, } } -// RebuildLocksAndTables rebuild the locks and tables. -func (lk *LockKeeper) RebuildLocksAndTables( - cli *clientv3.Client, - ifm map[string]map[string]map[string]map[string]Info, - colm map[string]map[string]map[string]map[string]map[string]DropColumnStage, - lockJoined map[string]schemacmp.Table, - lockTTS map[string][]TargetTable, - missTable map[string]map[string]map[string]map[string]schemacmp.Table, -) { - var ( - lock *Lock - ok bool - ) - for task, taskInfos := range ifm { - for source, sourceInfos := range taskInfos { - for schema, schemaInfos := range sourceInfos { - for table, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if lock, ok = lk.locks[lockID]; !ok { - lock = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID]) - } - // filter info which doesn't have SourceTable - // SourceTable will be changed after user update block-allow-list - // But old infos still remain in etcd. - // TODO: add a mechanism to remove all outdated infos in etcd. - if !lock.TableExist(info.Source, info.UpSchema, info.UpTable) { - delete(ifm[task][source][schema], table) - continue - } - lk.locks[lockID] = lock - lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) - if columns, ok := colm[lockID]; ok { - lock.columns = columns - } - } - } - } +// SetDropColumns set drop columns for lock keeper. +func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage) { + lk.dropColumns = dropColumns +} + +// getDownstreamMeta gets and cached downstream meta. +func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { + if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { + return downstreamMeta, nil } - // update missTable's table info for locks - for lockID, lockTable := range missTable { - for source, sourceTable := range lockTable { - for schema, schemaTable := range sourceTable { - for table, tableinfo := range schemaTable { - if _, ok := lk.locks[lockID]; !ok { - continue - } - if !lk.locks[lockID].TableExist(source, schema, table) { - continue - } - lk.locks[lockID].tables[source][schema][table] = tableinfo - } - } - } + dbConfig, meta := lk.getDownstreamMetaFunc(task) + if dbConfig == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } + downstreamMeta := &DownstreamMeta{dbConfig: dbConfig, meta: meta} + lk.downstreamMetaMap[task] = downstreamMeta + return downstreamMeta, nil +} + +// RemoveDownstreamMeta removes downstream mate by task. +func (lk *LockKeeper) RemoveDownstreamMeta(task string) { + delete(lk.downstreamMetaMap, task) } // TrySync tries to sync the lock. @@ -111,8 +95,20 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable } if l, ok = lk.locks[lockID]; !ok { - lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts) + downstreamMeta, err := lk.getDownstreamMeta(info.Task) + if err != nil { + log.L().Error("get downstream meta", log.ShortError(err)) + } + + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts, downstreamMeta) l = lk.locks[lockID] + + // set drop columns, only when recover locks + if lk.dropColumns != nil { + if cols, ok := lk.dropColumns[lockID]; ok { + l.columns = cols + } + } } newDDLs, cols, err := l.TrySync(info, tts) @@ -143,6 +139,21 @@ func (lk *LockKeeper) FindLock(lockID string) *Lock { return lk.locks[lockID] } +// FindLocksByTask finds locks by task. +func (lk *LockKeeper) FindLocksByTask(task string) []*Lock { + lk.mu.RLock() + defer lk.mu.RUnlock() + + locks := make([]*Lock, 0) + for _, lock := range lk.locks { + if lock.Task == task { + locks = append(locks, lock) + } + } + + return locks +} + // FindLockByInfo finds a lock with a shard DDL info. func (lk *LockKeeper) FindLockByInfo(info Info) *Lock { return lk.FindLock(genDDLLockID(info)) @@ -166,6 +177,7 @@ func (lk *LockKeeper) Clear() { defer lk.mu.Unlock() lk.locks = make(map[string]*Lock) + lk.downstreamMetaMap = make(map[string]*DownstreamMeta) } // genDDLLockID generates DDL lock ID from its info. @@ -245,6 +257,30 @@ func (tk *TableKeeper) AddTable(task, source, upSchema, upTable, downSchema, dow return added } +// SourceTableExist check whether a source table exist. +func (tk *TableKeeper) SourceTableExist(task, source, upSchema, upTable, downSchema, downTable string) bool { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return false + } + if _, ok := tk.tables[task][source]; !ok { + return false + } + st := tk.tables[task][source] + targetTable := st.TargetTable(downSchema, downTable) + + if targetTable.UpTables != nil { + if tables, ok := targetTable.UpTables[upSchema]; ok { + if _, ok2 := tables[upTable]; ok2 { + return true + } + } + } + return false +} + // RemoveTable removes a table from the source tables. // it returns whether removed (exit before). func (tk *TableKeeper) RemoveTable(task, source, upSchema, upTable, downSchema, downTable string) bool { @@ -276,6 +312,20 @@ func (tk *TableKeeper) RemoveTableByTask(task string) bool { return true } +// RemoveTableByTaskAndSource removes tables from the source tables through task name and sources. +func (tk *TableKeeper) RemoveTableByTaskAndSources(task string, sources []string) { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return + } + + for _, source := range sources { + delete(tk.tables[task], source) + } +} + // FindTables finds source tables by task name and downstream table name. func (tk *TableKeeper) FindTables(task, downSchema, downTable string) []TargetTable { tk.mu.RLock() diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 6b686872c66..5aefe4173c2 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -17,13 +17,14 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" - "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testKeeper struct{} @@ -41,7 +42,7 @@ func TestKeeper(t *testing.T) { func (t *testKeeper) TestLockKeeper(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) upSchema = "foo_1" upTable = "bar_1" downSchema = "foo" @@ -81,6 +82,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(lock1, NotNil) c.Assert(lock1.ID, Equals, lockID1) c.Assert(lk.FindLockByInfo(i11).ID, Equals, lockID1) + + lks := lk.FindLocksByTask("hahaha") + c.Assert(len(lks), Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + synced, remain := lock1.IsSynced() c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) @@ -110,6 +118,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(synced, IsTrue) c.Assert(remain, Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + lks = lk.FindLocksByTask(task2) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID2) + // try to find not-exists lock. lockIDNotExists := "lock-not-exists" c.Assert(lk.FindLock(lockIDNotExists), IsNil) @@ -134,7 +149,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) task = "test-lock-keeper-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -277,6 +292,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { // no tables exist before Init/Update. c.Assert(tk.FindTables(task1, downSchema, downTable), IsNil) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsFalse) + } + } // Init with `nil` is fine. tk.Init(nil) @@ -288,6 +308,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // adds new tables. c.Assert(tk.Update(st21), IsTrue) @@ -300,11 +325,21 @@ func (t *testKeeper) TestTableKeeper(c *C) { tts = tk.FindTables(task2, downSchema, downTable) c.Assert(tts, HasLen, 1) c.Assert(tts[0], DeepEquals, tt22) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsTrue) + } + } // deletes tables. st22.IsDeleted = true c.Assert(tk.Update(st22), IsTrue) c.Assert(tk.FindTables(task2, downSchema, downTable), IsNil) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsFalse) + } + } // try to delete, but not exist. c.Assert(tk.Update(st22), IsFalse) @@ -316,6 +351,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // add a table for st11. c.Assert(tk.AddTable(task1, st11.Source, "db-2", "tbl-3", downSchema, downTable), IsTrue) @@ -346,6 +386,19 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tk.RemoveTable(task1, "not-exit", "db", "tbl-1", downSchema, downTable), IsFalse) tts = tk.FindTables(task1, downSchema, downTable) c.Assert(tts[1], DeepEquals, tt12) + + c.Assert(tk.RemoveTableByTask("hahaha"), IsFalse) + tk.RemoveTableByTaskAndSources("hahaha", nil) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{"hahaha"}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{source1, source2}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 1) + c.Assert(tts[0].Source, Equals, "new-source") + c.Assert(tts[0].UpTables["db-2"], HasKey, "tbl-3") } func (t *testKeeper) TestTargetTablesForTask(c *C) { @@ -414,72 +467,65 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }) } -func (t *testKeeper) TestRebuildLocksAndTables(c *C) { - defer clearTestInfoOperation(c) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" +} + +func (t *testKeeper) TestGetDownstreamMeta(c *C) { var ( - lk = NewLockKeeper() - task = "task" - source1 = "mysql-replica-1" - source2 = "mysql-replica-2" - upSchema = "foo" - upTable = "bar" - downSchema = "db" - downTable = "tbl" - DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} - DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) - ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) - ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) - ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) - - i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) - i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) - - tts = []TargetTable{ - newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), - newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), + task1 = "hahaha" + task2 = "hihihi" + task3 = "hehehe" + ) + getDownstreamMetaFunc := func(task string) (*config.DBConfig, string) { + switch task { + case task1, task2: + return &config.DBConfig{}, "meta" + default: + return nil, "" } + } - lockID = utils.GenDDLLockID(task, downSchema, downTable) + conn.InitMockDB(c) + lk := NewLockKeeper(getDownstreamMetaFunc) + c.Assert(lk.downstreamMetaMap, HasLen, 0) - ifm = map[string]map[string]map[string]map[string]Info{ - task: { - source1: {upSchema: {upTable: i11}}, - source2: {upSchema: {upTable: i21}}, - }, - } - colm = map[string]map[string]map[string]map[string]map[string]DropColumnStage{ - lockID: { - "c3": { - source1: {upSchema: {upTable: DropNotDone}}, - source2: {upSchema: {upTable: DropNotDone}}, - }, - }, - } - lockJoined = map[string]schemacmp.Table{ - lockID: schemacmp.Encode(ti2), - } - lockTTS = map[string][]TargetTable{ - lockID: tts, - } - ) + downstreamMeta, err := lk.getDownstreamMeta(task3) + c.Assert(downstreamMeta, IsNil) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) - lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS, nil) - locks := lk.Locks() - c.Assert(len(locks), Equals, 1) - lock, ok := locks[lockID] - c.Assert(ok, IsTrue) - cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) + downstreamMeta, err = lk.getDownstreamMeta(task1) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + downstreamMeta2, err := lk.getDownstreamMeta(task1) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, downstreamMeta2) + + downstreamMeta3, err := lk.getDownstreamMeta(task2) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.RemoveDownstreamMeta(task3) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + + lk.RemoveDownstreamMeta(task1) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + downstreamMeta, err = lk.getDownstreamMeta(task1) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - c.Assert(lock.columns, DeepEquals, colm[lockID]) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.Clear() + c.Assert(lk.downstreamMetaMap, HasLen, 0) } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 7b9740ce126..d1b207a9f48 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -14,16 +14,23 @@ package optimism import ( + "bytes" + "encoding/json" "fmt" "sync" + "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" + "golang.org/x/net/context" "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -75,30 +82,94 @@ type Lock struct { // record the partially dropped columns // column name -> source -> upSchema -> upTable -> int columns map[string]map[string]map[string]map[string]DropColumnStage + + downstreamMeta *DownstreamMeta } // NewLock creates a new Lock instance. -// NOTE: we MUST give the initial table info when creating the lock now. -func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable) *Lock { +func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { l := &Lock{ - cli: cli, - ID: id, - Task: task, - DownSchema: downSchema, - DownTable: downTable, - joined: joined, - tables: make(map[string]map[string]map[string]schemacmp.Table), - done: make(map[string]map[string]map[string]bool), - synced: true, - versions: make(map[string]map[string]map[string]int64), - columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + cli: cli, + ID: id, + Task: task, + DownSchema: downSchema, + DownTable: downTable, + joined: joined, + tables: make(map[string]map[string]map[string]schemacmp.Table), + done: make(map[string]map[string]map[string]bool), + synced: true, + versions: make(map[string]map[string]map[string]int64), + columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + downstreamMeta: downstreamMeta, } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) - + // pre join because tables may have different schema at the beginning + l.joinTable() return l } +// FetchTableInfos fetch all table infos for a lock. +func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.TableInfo, error) { + if l.downstreamMeta == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + + db, err := conn.DefaultDBProvider.Apply(l.downstreamMeta.dbConfig) + if err != nil { + return nil, err + } + defer db.Close() + + ctx, cancel := context.WithTimeout(context.Background(), dbutil.DefaultTimeout) + defer cancel() + + query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?` + row := db.DB.QueryRowContext(ctx, query, source, schema, table) + if row.Err() != nil { + return nil, terror.ErrDBExecuteFailed.Delegate(row.Err(), query) + } + var tiBytes []byte + if err := row.Scan(&tiBytes); err != nil { + return nil, terror.ErrDBExecuteFailed.Delegate(err, query) + } + var ti *model.TableInfo + if bytes.Equal(tiBytes, []byte("null")) { + log.L().Warn("null table info", zap.String("query", query), zap.String("source", source), zap.String("schema", schema), zap.String("table", table)) + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + if err := json.Unmarshal(tiBytes, &ti); err != nil { + return nil, err + } + return ti, nil +} + +// joinTable join tables for a lock and update l.joined. +func (l *Lock) joinTable() { + var ( + joined = l.joined + firstTable = true + ) + for _, schemaTables := range l.tables { + for _, tables := range schemaTables { + for _, ti := range tables { + if firstTable { + joined = ti + firstTable = false + } else { + newJoined, err := joined.Join(ti) + if err != nil { + log.L().Error(fmt.Sprintf("fail to join table info %s with %s", joined, ti), zap.String("lockID", l.ID), log.ShortError(err)) + return + } + joined = newJoined + } + } + } + } + l.joined = joined +} + // TrySync tries to sync the lock, re-entrant. // new upstream sources may join when the DDL lock is in syncing, // so we need to merge these new sources. @@ -398,6 +469,56 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { return true } +// TryRemoveTable tries to remove tables in the lock by sources. +// return drop columns for later use. +func (l *Lock) TryRemoveTableBySources(sources []string) []string { + l.mu.Lock() + defer l.mu.Unlock() + + // record drop columns for sources + dropColumns := make([]string, 0) + for col, sourceColumns := range l.columns { + for _, source := range sources { + if _, ok := sourceColumns[source]; ok { + dropColumns = append(dropColumns, col) + break + } + } + } + + for _, source := range sources { + if _, ok := l.tables[source]; !ok { + continue + } + + delete(l.tables, source) + _, remain := l.syncStatus() + l.synced = remain == 0 + delete(l.done, source) + delete(l.versions, source) + for _, sourceColumns := range l.columns { + delete(sourceColumns, source) + } + log.L().Info("tables removed from the lock", zap.String("lock", l.ID), zap.String("source", source)) + } + return dropColumns +} + +// HasTables check whether a lock has tables. +func (l *Lock) HasTables() bool { + l.mu.Lock() + defer l.mu.Unlock() + + for _, schemas := range l.tables { + for _, tables := range schemas { + for range tables { + return true + } + } + } + return false +} + // IsSynced returns whether the lock has synced. // In the optimistic mode, we call it `synced` if table info of all tables are the same, // and we define `remain` as the table count which have different table info with the joined one, @@ -533,6 +654,7 @@ func (l *Lock) tryRevertDone(source, schema, table string) { } // addTables adds any not-existing tables into the lock. +// For a new table, try to fetch table info from downstream. func (l *Lock) addTables(tts []TargetTable) { for _, tt := range tts { if _, ok := l.tables[tt.Source]; !ok { @@ -548,7 +670,15 @@ func (l *Lock) addTables(tts []TargetTable) { } for table := range tables { if _, ok := l.tables[tt.Source][schema][table]; !ok { - l.tables[tt.Source][schema][table] = l.joined + ti, err := l.FetchTableInfos(tt.Task, tt.Source, schema, table) + if err != nil { + log.L().Error("source table info not found, use joined table info instead", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), log.ShortError(err)) + l.tables[tt.Source][schema][table] = l.joined + } else { + t := schemacmp.Encode(ti) + log.L().Debug("get source table info", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("info", t)) + l.tables[tt.Source][schema][table] = t + } l.done[tt.Source][schema][table] = false l.versions[tt.Source][schema][table] = 0 log.L().Info("table added to the lock", zap.String("lock", l.ID), diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 1ed4ebc64a0..5960b139491 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -14,8 +14,11 @@ package optimism import ( + "encoding/json" + "fmt" "testing" + "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" @@ -24,6 +27,9 @@ import ( "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -83,7 +89,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -385,7 +391,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -469,7 +475,7 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -540,7 +546,7 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -593,7 +599,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -636,7 +642,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { tables = map[string]map[string]struct{}{db1: {tbl1: struct{}{}}} tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source1: { db1: {tbl1: 0}, @@ -749,7 +755,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -914,7 +920,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1036,7 +1042,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1224,7 +1230,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -1367,7 +1373,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1437,6 +1443,103 @@ func (t *testLock) TestTryRemoveTable(c *C) { c.Assert(l.TryRemoveTable("not-exist", db, tbl1), IsFalse) } +func (t *testLock) TestTryRemoveTableWithSources(c *C) { + var ( + ID = "test_lock_try_remove_table-`foo`.`bar`" + task = "test_lock_try_remove_table" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + downSchema = "foo" + downTable = "bar" + db = "foo" + tbl1 = "bar1" + tbl2 = "bar2" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c1"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + + tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} + tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables), newTargetTable(task, source2, downSchema, downTable, tables)} + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + + vers = map[string]map[string]map[string]int64{ + source1: { + db: {tbl1: 0, tbl2: 0}, + }, + source2: { + db: {tbl1: 0, tbl2: 0}, + }, + } + ) + + // only one table exists before TrySync. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + + // TrySync for the first table. + info := newInfoWithVersion(task, source1, db, tbl1, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err := l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready := l.Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + c.Assert(ready[source2], HasLen, 1) + c.Assert(ready[source2][db], HasLen, 2) + c.Assert(ready[source2][db][tbl1], IsTrue) + c.Assert(ready[source2][db][tbl2], IsTrue) + + // TryRemoveTableBySources with nil + c.Assert(len(l.TryRemoveTableBySources(nil)), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with wrong source + tts = tts[:1] + c.Assert(len(l.TryRemoveTableBySources([]string{"hahaha"})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with source2 + c.Assert(len(l.TryRemoveTableBySources([]string{source2})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + delete(vers, source2) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.HasTables(), IsTrue) + + // TrySync with second table + info = newInfoWithVersion(task, source1, db, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsTrue) + c.Assert(ready[source1][db][tbl2], IsTrue) + + // TryRemoveTableBySources with source1,source2 + cols = l.TryRemoveTableBySources([]string{source1}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.HasTables(), IsFalse) +} + func (t *testLock) TestLockTryMarkDone(c *C) { var ( ID = "test_lock_try_mark_done-`foo`.`bar`" @@ -1459,7 +1562,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1567,7 +1670,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1609,7 +1712,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) // case 2: add a column with a smaller field length - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- @@ -1657,7 +1760,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1853,7 +1956,7 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1909,3 +2012,62 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { c.Assert(l.versions, DeepEquals, vers) t.checkLockSynced(c, l) } + +func (t *testLock) TestFetchTableInfo(c *C) { + var ( + meta = "meta" + ID = "test_lock_try_sync_index-`foo`.`bar`" + task = "test_lock_try_sync_index" + source = "mysql-replica-1" + downSchema = "db" + downTable = "bar" + schema = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, UNIQUE INDEX idx_c1(c1))`) + tables = map[string]map[string]struct{}{ + schema: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}, + } + tts = []TargetTable{ + newTargetTable(task, source, downSchema, downTable, tables), + } + query = fmt.Sprintf("SELECT table_info FROM `%s`.`%s` WHERE id = \\? AND cp_schema = \\? AND cp_table = \\?", meta, cputil.SyncerCheckpoint(task)) + ) + + // nil downstream meta + l := NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + ti, err := l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // table info not exist + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock := conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"})) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrDBExecuteFailed.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // null table info + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow("null")) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // succeed + tiBytes, err := json.Marshal(ti0) + c.Assert(err, IsNil) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow(tiBytes)) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + c.Assert(ti, DeepEquals, ti0) +} diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 7cd13bace1d..28c92f3e495 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" - "github.com/pingcap/tiflow/dm/pkg/utils" ) // PutSourceTablesInfo puts source tables and a shard DDL info. @@ -48,10 +47,10 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) return rev, err } -// DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. +// DeleteInfosOperationsColumns deletes the shard DDL infos, operations, and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { +func DeleteInfosOperationsColumns(cli *clientv3.Client, infos []Info, ops []Operation, lockID string) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,8 +61,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ for _, op := range ops { opsDel = append(opsDel, deleteOperationOp(op)) } - opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) - opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(schema.Task, schema.DownSchema, schema.DownTable))) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(lockID)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -71,16 +69,36 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ return rev, resp.Succeeded, nil } -// DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. -func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { +// DeleteInfosOperationsTablesByTask deletes the shard DDL infos and operations in etcd. +// This function should often be called by DM-master when stop a task for all sources. +func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) for lockID := range lockIDSet { opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID), clientv3.WithPrefix())) } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } + +// DeleteInfosOperationsTablesByTaskAndSource deletes the shard DDL infos and operations in etcd by task and source. +// This function should often be called by DM-master when stop a task for sources. +func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, sources []string, dropColumns map[string][]string) (int64, error) { + opsDel := make([]clientv3.Op, 0, 5) + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + for lockID, cols := range dropColumns { + for _, col := range cols { + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, col, source), clientv3.WithPrefix())) + } + } + } + } + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) + return rev, err +} diff --git a/dm/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 55270f0d5e8..da140184268 100644 --- a/dm/pkg/shardddl/optimism/ops_test.go +++ b/dm/pkg/shardddl/optimism/ops_test.go @@ -30,7 +30,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} info = NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, nil, nil) op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false, []string{}) - is = NewInitSchema(task, downSchema, downTable, nil) ) // put info. @@ -52,15 +51,8 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm, HasLen, 1) c.Assert(opm[task][source][upSchema][upTable], DeepEquals, op) - // put init schema. - _, _, err = PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - isc, _, err := GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc, DeepEquals, is) - // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, is) + _, deleted, err := DeleteInfosOperationsColumns(etcdTestCli, []Info{info}, []Operation{op}, genDDLLockID(info)) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -71,12 +63,9 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 1) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) + _, deleted, err = DeleteInfosOperationsColumns(etcdTestCli, []Info{infoWithVer}, []Operation{op}, genDDLLockID(infoWithVer)) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -87,9 +76,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsTrue) } func (t *testForEtcd) TestSourceTablesInfo(c *C) { diff --git a/dm/pkg/shardddl/optimism/schema.go b/dm/pkg/shardddl/optimism/schema.go deleted file mode 100644 index c4f4167e4df..00000000000 --- a/dm/pkg/shardddl/optimism/schema.go +++ /dev/null @@ -1,151 +0,0 @@ -// Copyright 2020 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 optimism - -import ( - "encoding/json" - - "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - - "github.com/pingcap/tiflow/dm/dm/common" - "github.com/pingcap/tiflow/dm/pkg/etcdutil" -) - -// InitSchema represents the initial schema (schema before the lock constructed) of a merged table. -// NOTE: `Task`, `DownSchema` and `DownTable` are redundant in the etcd key path for convenient. -type InitSchema struct { - Task string `json:"task"` // data migration task name - DownSchema string `json:"down-schema"` // downstream/target schema name - DownTable string `json:"down-table"` // downstream/target table name - TableInfo *model.TableInfo `json:"table-info"` // the initial table info (schema) -} - -// NewInitSchema creates a new InitSchema instance. -func NewInitSchema(task, downSchema, downTable string, tableInfo *model.TableInfo) InitSchema { - return InitSchema{ - Task: task, - DownSchema: downSchema, - DownTable: downTable, - TableInfo: tableInfo, - } -} - -// String implements Stringer interface. -func (is InitSchema) String() string { - s, _ := is.toJSON() - return s -} - -// toJSON returns the string of JSON represent. -func (is InitSchema) toJSON() (string, error) { - data, err := json.Marshal(is) - if err != nil { - return "", err - } - return string(data), nil -} - -// IsEmpty returns true when this InitSchema has no value. -func (is InitSchema) IsEmpty() bool { - var emptyIS InitSchema - return is == emptyIS -} - -// infoFromJSON constructs InitSchema from its JSON represent. -func initSchemaFromJSON(s string) (is InitSchema, err error) { - err = json.Unmarshal([]byte(s), &is) - return -} - -// GetInitSchema gets the InitSchema for the specified downstream table. -func GetInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (InitSchema, int64, error) { - var is InitSchema - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return is, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - if resp.Count > 0 { - is, err = initSchemaFromJSON(string(resp.Kvs[0].Value)) - if err != nil { - return is, 0, err - } - } - return is, rev, nil -} - -// GetAllInitSchemas gets all init schemas from etcd. -// This function should often be called by DM-master. -// k/k/k/v: task-name -> downstream-schema-name -> downstream-table-name -> InitSchema. -func GetAllInitSchemas(cli *clientv3.Client) (map[string]map[string]map[string]InitSchema, int64, error) { - initSchemas := make(map[string]map[string]map[string]InitSchema) - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return nil, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - for _, kv := range resp.Kvs { - schema, err := initSchemaFromJSON(string(kv.Value)) - if err != nil { - return nil, 0, err - } - if _, ok := initSchemas[schema.Task]; !ok { - initSchemas[schema.Task] = make(map[string]map[string]InitSchema) - } - if _, ok := initSchemas[schema.Task][schema.DownSchema]; !ok { - initSchemas[schema.Task][schema.DownSchema] = make(map[string]InitSchema) - } - initSchemas[schema.Task][schema.DownSchema][schema.DownTable] = schema - } - return initSchemas, rev, nil -} - -// PutInitSchemaIfNotExist puts the InitSchema into ectd if no previous one exists. -func PutInitSchemaIfNotExist(cli *clientv3.Client, is InitSchema) (rev int64, putted bool, err error) { - value, err := is.toJSON() - if err != nil { - return 0, false, err - } - key := common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(is.Task, is.DownSchema, is.DownTable) - - cmp := clientv3util.KeyMissing(key) - op := clientv3.OpPut(key, value) - - resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{cmp}, []clientv3.Op{op}, []clientv3.Op{}) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// DeleteInitSchema tries to delete the InitSchema for the specified downstream table. -func DeleteInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (rev int64, deleted bool, err error) { - op := deleteInitSchemaOp(task, downSchema, downTable) - resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// deleteInitSchemaOp returns a DELETE etcd operation for init schema. -func deleteInitSchemaOp(task, downSchema, downTable string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) -} diff --git a/dm/pkg/shardddl/optimism/schema_test.go b/dm/pkg/shardddl/optimism/schema_test.go deleted file mode 100644 index f5f3ba582a4..00000000000 --- a/dm/pkg/shardddl/optimism/schema_test.go +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2020 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 optimism - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/util/mock" -) - -func (t *testForEtcd) TestInitSchemaJSON(c *C) { - is1 := NewInitSchema("test", "foo", "bar", nil) - j, err := is1.toJSON() - c.Assert(err, IsNil) - c.Assert(j, Equals, `{"task":"test","down-schema":"foo","down-table":"bar","table-info":null}`) - c.Assert(j, Equals, is1.String()) - - is2, err := initSchemaFromJSON(j) - c.Assert(err, IsNil) - c.Assert(is2, DeepEquals, is1) -} - -func (t *testForEtcd) TestInitSchemaEtcd(c *C) { - defer clearTestInfoOperation(c) - - var ( - task = "test-init-schema-etcd" - downSchema = "foo" - downTable = "bar" - downTable2 = "bar2" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - tblI1 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY)") - tblI2 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)") - tblI3 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar2 (id INT PRIMARY KEY, c INT)") - is1 = NewInitSchema(task, downSchema, downTable, tblI1) - is2 = NewInitSchema(task, downSchema, downTable, tblI2) - is3 = NewInitSchema(task, downSchema, downTable2, tblI3) - ) - - // try to get, but no one exists. - isc, rev0, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev0, Greater, int64(0)) - c.Assert(isc.IsEmpty(), IsTrue) - - // put the init schema. - rev1, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev1, Greater, rev0) - c.Assert(putted, IsTrue) - - // get it back. - isc, rev2, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev2, Equals, rev1) - c.Assert(isc, DeepEquals, is1) - - // can't put again if a previous one exist. - rev3, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - rev3, putted, err = PutInitSchemaIfNotExist(etcdTestCli, is2) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - - // put new init schema with different downstream info. - rev4, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is3) - c.Assert(err, IsNil) - c.Assert(rev4, Greater, rev3) - c.Assert(putted, IsTrue) - - // get all init schemas. - initSchemas, rev5, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev5, Equals, rev4) - c.Assert(initSchemas[is1.Task][is1.DownSchema][is1.DownTable], DeepEquals, is1) - c.Assert(initSchemas[is3.Task][is3.DownSchema][is3.DownTable], DeepEquals, is3) - - // delete the schema. - rev6, deleted, err := DeleteInitSchema(etcdTestCli, is1.Task, is1.DownSchema, is1.DownTable) - c.Assert(err, IsNil) - c.Assert(rev6, Greater, rev5) - c.Assert(deleted, IsTrue) - rev7, deleted, err := DeleteInitSchema(etcdTestCli, is3.Task, is3.DownSchema, is3.DownTable) - c.Assert(err, IsNil) - c.Assert(rev7, Greater, rev6) - c.Assert(deleted, IsTrue) - - // not exist now. - initSchemas, rev8, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev8, Equals, rev7) - c.Assert(initSchemas, HasLen, 0) -} diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 06c4c958ec2..6c95627d941 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -496,6 +496,7 @@ const ( codeMasterFailToImportFromV10x codeMasterInconsistentOptimistDDLsAndInfo codeMasterOptimisticTableInfobeforeNotExist + codeMasterOptimisticDownstreamMetaNotFound ) // DM-worker error code. @@ -1143,6 +1144,7 @@ var ( ErrMasterInconsistentOptimisticDDLsAndInfo = New(codeMasterInconsistentOptimistDDLsAndInfo, ClassDMMaster, ScopeInternal, LevelHigh, "inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d", "") ErrMasterOptimisticTableInfoBeforeNotExist = New(codeMasterOptimisticTableInfobeforeNotExist, ClassDMMaster, ScopeInternal, LevelHigh, "table-info-before not exist in optimistic ddls: %v", "") + ErrMasterOptimisticDownstreamMetaNotFound = New(codeMasterOptimisticDownstreamMetaNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "downstream database config and meta for task %s not found", "") // DM-worker error. ErrWorkerParseFlagSet = New(codeWorkerParseFlagSet, ClassDMWorker, ScopeInternal, LevelMedium, "parse dm-worker config flag set", "") diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 0873d607c27..833bde9f798 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -24,6 +24,7 @@ import ( "sync" "time" + "github.com/pingcap/errors" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -61,6 +62,7 @@ var ( globalCpSchema = "" // global checkpoint's cp_schema globalCpTable = "" // global checkpoint's cp_table maxCheckPointTimeout = "1m" + batchFlushPoints = 100 ) type tablePoint struct { @@ -245,8 +247,8 @@ type CheckPoint interface { // corresponding to Meta.Flush FlushPointsExcept(tctx *tcontext.Context, snapshotID int, exceptTables []*filter.Table, extraSQLs []string, extraArgs [][]interface{}) error - // FlushPointWithTableInfo flushed the table point with given table info - FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error + // FlushPointsWithTableInfos flushed the table points with given table infos + FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error // FlushSafeModeExitPoint flushed the global checkpoint's with given table info FlushSafeModeExitPoint(tctx *tcontext.Context) error @@ -712,45 +714,63 @@ func (cp *RemoteCheckPoint) FlushPointsExcept( return nil } -// FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo. -func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error { +// FlushPointsWithTableInfos implements CheckPoint.FlushPointsWithTableInfos. +func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error { cp.Lock() defer cp.Unlock() - sourceSchema, sourceTable := table.Schema, table.Name - sqls := make([]string, 0, 1) - args := make([][]interface{}, 0, 10) - point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) - - if tablePoints, ok := cp.points[sourceSchema]; ok { - if p, ok2 := tablePoints[sourceTable]; ok2 { - point = p - } - } - - tiBytes, err := json.Marshal(ti) - if err != nil { - return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + // should not happened + if len(tables) != len(tis) { + return errors.Errorf("the length of the tables is not equal to the length of the table infos, left: %d, right: %d", len(tables), len(tis)) } - location := point.MySQLLocation() - sql2, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) - sqls = append(sqls, sql2) - args = append(args, arg) + for i := 0; i < len(tables); i += batchFlushPoints { + end := i + batchFlushPoints + if end > len(tables) { + end = len(tables) + } - // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update - tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) - defer cancel() - _, err = cp.dbConn.ExecuteSQL(tctx2, sqls, args...) - if err != nil { - return err - } + sqls := make([]string, 0, batchFlushPoints) + args := make([][]interface{}, 0, batchFlushPoints) + points := make([]*binlogPoint, 0, batchFlushPoints) + for j := i; j < end; j++ { + table := tables[j] + ti := tis[j] + sourceSchema, sourceTable := table.Schema, table.Name + + var point *binlogPoint + // if point already in memory, use it + if tablePoints, ok := cp.points[sourceSchema]; ok { + if p, ok2 := tablePoints[sourceTable]; ok2 { + point = p + } + } + // create new point + if point == nil { + cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), ti) + point = cp.points[sourceSchema][sourceTable] + } + tiBytes, err := json.Marshal(ti) + if err != nil { + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + } + location := point.MySQLLocation() + sql, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) + sqls = append(sqls, sql) + args = append(args, arg) + points = append(points, point) + } + // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) + defer cancel() + _, err := cp.dbConn.ExecuteSQL(tctx2, sqls, args...) + if err != nil { + return err + } - err = point.save(point.savedPoint.location, ti) - if err != nil { - return err + for _, point := range points { + point.flush() + } } - point.flush() - return nil } diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index 6fed273462c..575dcdb5473 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -218,18 +218,3 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { s.tctx.L().Info("finish to handle ddls in optimistic shard mode", zap.String("event", "query"), zap.Stringer("queryEventContext", qec)) return nil } - -// trackInitTableInfoOptimistic tries to get the initial table info (before modified by other tables) and track it in optimistic shard mode. -func (s *Syncer) trackInitTableInfoOptimistic(sourceTable, targetTable *filter.Table) (*model.TableInfo, error) { - ti, err := s.optimist.GetTableInfo(targetTable.Schema, targetTable.Name) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotGetTable.Delegate(err, sourceTable) - } - if ti != nil { - err = s.schemaTracker.CreateTableIfNotExists(sourceTable, ti) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) - } - } - return ti, nil -} diff --git a/dm/syncer/schema.go b/dm/syncer/schema.go index 979d814783e..eca48fb41d0 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -138,7 +138,7 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR if req.Flush { log.L().Info("flush table info", zap.String("table info", newSQL)) - err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), sourceTable, ti) + err = s.checkpoint.FlushPointsWithTableInfos(tcontext.NewContext(ctx, log.L()), []*filter.Table{sourceTable}, []*model.TableInfo{ti}) if err != nil { return "", err } diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 027626d3726..4b076e96008 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -17,6 +17,7 @@ import ( "context" "sync" + filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -70,6 +71,25 @@ func (o *Optimist) Init(sourceTables map[string]map[string]map[string]map[string return err } +// Tables clone and return tables +// first one is sourceTable, second one is targetTable. +func (o *Optimist) Tables() [][]filter.Table { + o.mu.Lock() + defer o.mu.Unlock() + + tbls := make([][]filter.Table, 0) + for downSchema, downTables := range o.tables.Tables { + for downTable, upSchemas := range downTables { + for upSchema, upTables := range upSchemas { + for upTable := range upTables { + tbls = append(tbls, []filter.Table{{Schema: upSchema, Name: upTable}, {Schema: downSchema, Name: downTable}}) + } + } + } + } + return tbls +} + // Reset resets the internal state of the optimist. func (o *Optimist) Reset() { o.mu.Lock() @@ -161,24 +181,6 @@ func (o *Optimist) DoneOperation(op optimism.Operation) error { return nil } -// GetTableInfo tries to get the init schema of the downstream table. -func (o *Optimist) GetTableInfo(downSchema, downTable string) (*model.TableInfo, error) { - if downTable == "" { - return nil, nil - } - - is, rev, err := optimism.GetInitSchema(o.cli, o.task, downSchema, downTable) - if err != nil { - return nil, err - } - if is.IsEmpty() { - o.logger.Info("no init schema exists", zap.String("schema", downSchema), zap.String("table", downTable), zap.Int64("revision", rev)) - } else { - o.logger.Info("got init schema", zap.Stringer("init schema", is)) - } - return is.TableInfo, nil -} - // PendingInfo returns the shard DDL info which is pending to handle. func (o *Optimist) PendingInfo() *optimism.Info { o.mu.RLock() diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index fa616f51bae..7b2a76bfbed 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -36,7 +36,7 @@ var _ = Suite(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -95,6 +95,9 @@ func (t *testOptimist) TestOptimist(c *C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + tables := o.Tables() + c.Assert(len(tables), Equals, 0) + // init with some source tables. err := o.Init(sourceTables) c.Assert(err, IsNil) @@ -104,6 +107,9 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(stm[task], HasLen, 1) c.Assert(stm[task][source], DeepEquals, o.tables) + tables = o.Tables() + c.Assert(len(tables), Equals, 4) + // no info and operation in pending. c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) @@ -206,42 +212,3 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) } - -func (t *testOptimist) TestGetTableInfo(c *C) { - defer clearOptimistTestSourceInfoOperation(c) - - var ( - task = "test-get-table-info" - source = "mysql-replica-1" - logger = log.L() - o = NewOptimist(&logger, etcdTestCli, task, source) - - downSchema = "foo" - downTable = "bar" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - is = optimism.NewInitSchema(task, downSchema, downTable, - createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`)) - ) - - // no table info exist now - ti, err := o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, IsNil) - - // put the table schema. - _, putted, err := optimism.PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - c.Assert(putted, IsTrue) - - // can get the table info now. - ti, err = o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, DeepEquals, is.TableInfo) - - // no table info for database. - ti, err = o.GetTableInfo(downSchema, "") - c.Assert(err, IsNil) - c.Assert(ti, IsNil) -} diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 763d90efefb..b1ac38f52da 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -735,14 +735,6 @@ func (s *Syncer) getTableInfo(tctx *tcontext.Context, sourceTable, targetTable * return ti, nil } - // in optimistic shard mode, we should try to get the init schema (the one before modified by other tables) first. - if s.cfg.ShardMode == config.ShardOptimistic { - ti, err = s.trackInitTableInfoOptimistic(sourceTable, targetTable) - if err != nil { - return nil, err - } - } - // if the table does not exist (IsTableNotExists(err)), continue to fetch the table from downstream and create it. if ti == nil { err = s.trackTableInfoFromDownstream(tctx, sourceTable, targetTable) @@ -1492,7 +1484,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err != nil { return err } - if s.cfg.Mode == config.ModeAll && fresh { + if fresh && s.cfg.Mode == config.ModeAll { delLoadTask = true flushCheckpoint = true err = s.loadTableStructureFromDump(ctx) @@ -1500,7 +1492,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) cleanDumpFile = false } - } else { + if s.cfg.ShardMode == config.ShardOptimistic { + s.flushOptimisticTableInfos(tctx) + } + } + + if s.cfg.Mode == config.ModeIncrement || !fresh { cleanDumpFile = false } @@ -3172,7 +3169,7 @@ func (s *Syncer) loadTableStructureFromDump(ctx context.Context) error { continue } } - logger.Info("fetch table structure form dump files", + logger.Info("fetch table structure from dump files", zap.Strings("database", dbs), zap.Any("tables", tables)) for _, db := range dbs { @@ -3764,3 +3761,23 @@ func calculateChanSize(queueSize, workerCount int, compact bool) int { } return chanSize } + +func (s *Syncer) flushOptimisticTableInfos(tctx *tcontext.Context) { + tbls := s.optimist.Tables() + sourceTables := make([]*filter.Table, 0, len(tbls)) + tableInfos := make([]*model.TableInfo, 0, len(tbls)) + for _, tbl := range tbls { + sourceTable := tbl[0] + targetTable := tbl[1] + tableInfo, err := s.getTableInfo(tctx, &sourceTable, &targetTable) + if err != nil { + tctx.L().Error("failed to get table infos", log.ShortError(err)) + continue + } + sourceTables = append(sourceTables, &sourceTable) + tableInfos = append(tableInfos, tableInfo) + } + if err := s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos); err != nil { + tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) + } +} diff --git a/dm/tests/_utils/shardddl_lib.sh b/dm/tests/_utils/shardddl_lib.sh index 3860a971204..b5f1ffc9c30 100644 --- a/dm/tests/_utils/shardddl_lib.sh +++ b/dm/tests/_utils/shardddl_lib.sh @@ -54,3 +54,39 @@ function restart_master() { run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT } + +function restart_worker1() { + echo "restart dm-worker1" + ps aux | grep worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + 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 +} + +function restart_worker2() { + echo "restart dm-worker2" + ps aux | grep worker2 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER2_PORT 20 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT +} + +function restart_task() { + echo "restart task" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $1" + + if [[ "$task_conf" == *"single"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 1 + elif [[ "$task_conf" == *"double"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 2 + fi +} diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index f268325da4e..b4ea29bf170 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -324,7 +324,7 @@ function init_cluster(){ } function get_master_status() { - arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h127.0.0.1 -P3306 | awk 'NR==2') + arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h$1 -P$2 | awk 'NR==2') echo $arr } diff --git a/dm/tests/downstream_more_column/run.sh b/dm/tests/downstream_more_column/run.sh index eb8eca00da0..edfef1dabe1 100755 --- a/dm/tests/downstream_more_column/run.sh +++ b/dm/tests/downstream_more_column/run.sh @@ -43,7 +43,7 @@ function run() { # start DM task in incremental mode # schemaTracker create table from downstream - master_status=($(get_master_status)) + master_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) cp $cur/conf/dm-task-incremental.yaml $WORK_DIR/dm-task-incremental.yaml sed -i "s/binlog-gtid-placeholder/${master_status[2]}/g" $WORK_DIR/dm-task-incremental.yaml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/others_integration_2.txt b/dm/tests/others_integration_2.txt index e6f69950689..676318c3ce2 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -10,4 +10,5 @@ openapi duplicate_event tracker_ignored_ddl extend_column +shardddl_optimistic gbk diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index 58ea6fd456c..20f5f5a0d38 100755 --- a/dm/tests/sequence_sharding_optimistic/run.sh +++ b/dm/tests/sequence_sharding_optimistic/run.sh @@ -64,9 +64,9 @@ run() { "query-status $task_name" \ "\"stage\": \"Paused\"" 2 - # try to get schema for the table, but can't get because no DDL/DML replicated yet. + # try to get schema for the table, table exists for optimistic. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c1` varchar(20) DEFAULT NULL, `c2` varchar(20) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # resume task manually. run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 224390e17dc..101b9d4e7f8 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -394,10 +394,17 @@ function DM_UpdateBARule_CASE() { sed -i 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $WORK_DIR/task.yaml echo 'ignore-checking-items: ["schema_of_shard_tables"]' >>$WORK_DIR/task.yaml + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $WORK_DIR/task.yaml" \ "\"result\": true" 3 + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,13,13);" run_sql_source2 "insert into ${shardddl1}.${tb1} values(14);" run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 14" @@ -406,7 +413,7 @@ function DM_UpdateBARule_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "show-ddl-locks" \ - "\"ID\": \"test-\`shardddl\`.\`tb\`\"" 1 + "no DDL lock exists" 1 run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" diff --git a/dm/tests/shardddl_optimistic/conf/diff_config.toml b/dm/tests/shardddl_optimistic/conf/diff_config.toml new file mode 100644 index 00000000000..cf3abc02891 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/diff_config.toml @@ -0,0 +1,44 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + target-check-tables = ["shardddl.tb"] + +[routes.rule1] +schema-pattern = "shardddl[1-2]" +table-pattern = "tb*" +target-schema = "shardddl" +target-table = "tb" + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/shardddl_optimistic/conf/dm-master.toml b/dm/tests/shardddl_optimistic/conf/dm-master.toml new file mode 100644 index 00000000000..458b3e124f0 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-master.toml @@ -0,0 +1,10 @@ +# Master Configuration. +name = "master1" +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" +peer-urls = "127.0.0.1:8291" +initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" + +[experimental] +openapi = true diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker1.toml b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml new file mode 100644 index 00000000000..6f1d1b5344f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker2.toml b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml new file mode 100644 index 00000000000..83949162686 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml new file mode 100644 index 00000000000..541c103ddbc --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml @@ -0,0 +1,63 @@ +--- +name: test +task-mode: incremental +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-pos: pos-holder + binlog-name: name-holder + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-gtid: gtid-holder + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml new file mode 100644 index 00000000000..8e60eab721d --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml @@ -0,0 +1,58 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/shardddl_optimistic/conf/source1.yaml b/dm/tests/shardddl_optimistic/conf/source1.yaml new file mode 100644 index 00000000000..175e07df7a5 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source1.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-01 +flavor: '' +enable-gtid: false +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/conf/source2.yaml b/dm/tests/shardddl_optimistic/conf/source2.yaml new file mode 100644 index 00000000000..e6508fd9d4f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source2.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-02 +flavor: '' +enable-gtid: true +enable-relay: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh new file mode 100644 index 00000000000..09acecfa9bb --- /dev/null +++ b/dm/tests/shardddl_optimistic/run.sh @@ -0,0 +1,426 @@ +#!/bin/bash + +set -eu + +cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +source $cur/../_utils/shardddl_lib.sh + +function DM_DIFFERENT_SCHEMA_FULL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_FULL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_FULL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + + # get checkpoint + source1_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) + source2_status=($(get_master_status $MYSQL_HOST2 $MYSQL_PORT2)) + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "pause-task test" \ + "\"result\": true" 3 + + # save schema + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema11.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema12.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema21.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema22.sql + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + # incremental data + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + # start task with current checkpoint + sed "s/pos-holder/${source1_status[1]}/g" $cur/conf/double-source-optimistic-incr.yaml >$WORK_DIR/task.yaml + sed -i "s/name-holder/${source1_status[0]}/g" $WORK_DIR/task.yaml + sed -i "s/gtid-holder/${source2_status[2]}/g" $WORK_DIR/task.yaml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml --remove-meta" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Column count doesn't match" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema11.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema12.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema21.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema22.sql" \ + "\"result\": true" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_INCREMENTAL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_INCREMENTAL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function DM_RESTART_TASK_MASTER_WORKER_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_RESTART_TASK_MASTER_WORKER() { + # create table with different schema, init data, and create table in downstream manually + run_case RESTART_TASK_MASTER_WORKER "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function random_restart() { + mod=$(($RANDOM % 4)) + if [[ "$mod" == "0" ]]; then + echo "restart master" + restart_master + elif [[ "$mod" == "1" ]]; then + echo "restart worker1" + restart_worker1 + elif [[ "$mod" == "2" ]]; then + echo "restart worker2" + restart_worker2 + else + echo "restart task" + restart_task $cur/conf/double-source-optimistic.yaml + fi +} + +function DM_STOP_TASK_FOR_A_SOURCE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,'aaa');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'ccc');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'ddd');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'eee','eee');" + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'fff','fff');" + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 12" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'ggg');" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'hhh');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 14" + run_sql_tidb_with_retry "select count(1) from INFORMATION_SCHEMA.COLUMNS where TABLE_SCHEMA='${shardddl}' AND TABLE_NAME='${tb}';" \ + "count(1): 2" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(15,'iii');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'jjj');" + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'kkk');" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'lll');" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_STOP_TASK_FOR_A_SOURCE() { + run_case STOP_TASK_FOR_A_SOURCE "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" +} + +function DM_UPDATE_BA_ROUTE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col1 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,5);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(6,6);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,7);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(8,8);" + + # source1 db2.tb1 add column and then drop column + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col2 int" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(9,9,9);" + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col2" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(10,10);" + + # source1.db1.tb1, source2.db2.tb1 add column + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,11,11);" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col3 int" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(12,12,12);" + + # source2 db1.tb1 drop column + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(13);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 13" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + sed 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $cur/conf/double-source-optimistic.yaml >$WORK_DIR/task.yaml + + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml" \ + "\"result\": true" 3 + + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(14,14,14);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 15" + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,16);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,17);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 17" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml" \ + "\"result\": true" 3 + + # source1: db1.tb1(id,new_col3), db2.tb1(id,new_col1) + # source2: db1.tb1(id,new_col3), db2.tb1(id,new_col1,new_col3) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(18,18);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(19,19);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(20,20);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(21,21,21);" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unknown column 'new_col1' in 'field list'" 2 + + run_sql_tidb "alter table ${shardddl}.${tb} add column new_col1 int" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col3 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(22,22);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(23,23);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(24,24);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(25,25);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_UPDATE_BA_ROUTE() { + run_case UPDATE_BA_ROUTE "double-source-optimistic" "init_table 111 121 211 221" "clean_table" "optimistic" +} + +function run() { + init_cluster + init_database + + DM_DIFFERENT_SCHEMA_FULL + DM_DIFFERENT_SCHEMA_INCREMENTAL + DM_RESTART_TASK_MASTER_WORKER + DM_STOP_TASK_FOR_A_SOURCE + DM_UPDATE_BA_ROUTE +} + +cleanup_data $shardddl +cleanup_data $shardddl1 +cleanup_data $shardddl2 +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" From b0f52af9c3d805b0d90d421c24ea8c42c49391a8 Mon Sep 17 00:00:00 2001 From: will & database <87208113+db-will@users.noreply.github.com> Date: Mon, 7 Feb 2022 06:57:34 -0500 Subject: [PATCH 37/72] syncer(dm): Fix async flush log message (#4500) ref pingcap/tiflow#4287 --- dm/syncer/syncer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index b1ac38f52da..212a3fa9bb1 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1218,7 +1218,7 @@ func (s *Syncer) afterFlushCheckpoint(task *checkpointFlushTask) error { s.tctx.L().Info("after async flushed checkpoint, gc stale causality keys", zap.Int64("flush job seq", task.asyncflushJob.flushSeq)) s.addJob(newGCJob(task.asyncflushJob.flushSeq)) } else { - s.tctx.L().Info("after async flushed checkpoint, gc all causality keys") + s.tctx.L().Info("after sync flushed checkpoint, gc all causality keys") s.addJob(newGCJob(math.MaxInt64)) } From 95e751fb86b98b6fcf64a0d6bb4f792c5424d2d7 Mon Sep 17 00:00:00 2001 From: zoubingwu Date: Tue, 8 Feb 2022 10:41:35 +0800 Subject: [PATCH 38/72] dm(ui): support start/pause/resume/delete task (#4412) ref pingcap/tiflow#3583 --- dm/ui/locales/en.json | 16 +- dm/ui/locales/zh.json | 13 +- dm/ui/package.json | 2 +- .../BatchImportTaskConfig/index.tsx | 4 +- .../CreateTaskConfig/MigrateRule.tsx | 59 ++++---- .../CreateTaskConfig/TargetInfo.tsx | 16 +- .../src/components/CreateTaskConfig/index.tsx | 32 +++- .../StartTaskWithListSelection/index.tsx | 116 +++++++++++++++ dm/ui/src/layouts/SiderMenu.tsx | 2 +- dm/ui/src/models/index.ts | 5 +- dm/ui/src/models/task.ts | 48 +++++- dm/ui/src/models/taskConfig.ts | 12 +- dm/ui/src/pages/cluster/relay-log.tsx | 14 -- ...sync-detail.tsx => replication-detail.tsx} | 6 +- dm/ui/src/pages/migration/task-config.tsx | 137 +++++++++++++----- dm/ui/src/pages/migration/task.tsx | 47 +++++- dm/ui/yarn.lock | 8 +- 17 files changed, 405 insertions(+), 132 deletions(-) create mode 100644 dm/ui/src/components/StartTaskWithListSelection/index.tsx delete mode 100644 dm/ui/src/pages/cluster/relay-log.tsx rename dm/ui/src/pages/migration/{sync-detail.tsx => replication-detail.tsx} (91%) diff --git a/dm/ui/locales/en.json b/dm/ui/locales/en.json index 5ead2ec5ff6..101db00e61c 100644 --- a/dm/ui/locales/en.json +++ b/dm/ui/locales/en.json @@ -5,6 +5,7 @@ "add new source": "Add new source", "add source config": "Add source config", "address": "address", + "are you sure to delete task {{name}}": "Are you sure to delete task \"{{name}}\"?", "are you sure to offline this master": "Are you sure to offline this master", "are you sure to offline this worker": "Are you sure to offline this worker", "basic info": "Basics", @@ -33,6 +34,7 @@ "enable gtid": "Enable GTID", "enable relay": "Enable relay", "enabled": "Enabled", + "error": "Error", "event filter": "Event filter", "event filter ignore event": "Ignore event", "event filter ignore sql": "Ignore SQL", @@ -71,6 +73,7 @@ "password is required": "Password is required", "pause": "Pause", "please check the log": "Please check the log", + "please select at least one task": "Please select at least one task", "port": "Port", "port is required": "Port is required", "previous": "Previous", @@ -78,7 +81,7 @@ "reading task list": "Reading task list", "refresh": "Refresh", "relay config (optional)": "Relay log (optional)", - "relay log": "Relay Log", + "replication detail": "Replication Detail", "request success": "Request success", "requesting": "Requesting", "resume": "Resume", @@ -93,9 +96,10 @@ "source name": "Source name", "source name is required": "Source name is required", "start import": "Start import", + "start task": "Start", + "start task and remove meta data": "Remove meta and tart task", "status": "Status", "sync config": "Replication config", - "sync detail": "Sync Detail", "synchronous concurrency": "Sync concurrency", "table": "Table", "target": "Target", @@ -113,10 +117,6 @@ "user name": "User", "user name is required": "User name is required", "view": "View", - "{{count}} filters": "{{count}} filter", - "{{count}} filters_plural": "{{count}} filters", - "{{count}} rules": "{{count}} rule", - "{{count}} rules_plural": "{{count}} rules", - "{{val}} and other {{count}}": "{{val}}", - "{{val}} and other {{count}}_plural": "{{val}} and {{count}} others" + "{{val}} and {{count}} others": "{{val}} and {{count}} in total", + "{{val}} and {{count}} others_plural": "{{val}} and {{count}} in total" } diff --git a/dm/ui/locales/zh.json b/dm/ui/locales/zh.json index c1994751802..0edeac1136c 100644 --- a/dm/ui/locales/zh.json +++ b/dm/ui/locales/zh.json @@ -5,6 +5,7 @@ "add new source": "添加上游", "add source config": "添加上游配置", "address": "地址", + "are you sure to delete task {{name}}": "确定要删除任务 \"{{name}}\" 吗?", "are you sure to offline this master": "确定要下线此主节点吗?", "are you sure to offline this worker": "确定要下线此从节点吗?", "basic info": "基本信息", @@ -33,6 +34,7 @@ "enable gtid": "启用 GTID", "enable relay": "启用 Relay log", "enabled": "已启用", + "error": "错误", "event filter": "事件过滤", "event filter ignore event": "忽略事件", "event filter ignore sql": "忽略 SQL", @@ -71,6 +73,7 @@ "password is required": "密码不能为空", "pause": "暂停", "please check the log": "请查看日志", + "please select at least one task": "请选择至少一个任务", "port": "端口", "port is required": "端口不能为空", "previous": "上一步", @@ -78,7 +81,7 @@ "reading task list": "读取任务列表", "refresh": "刷新", "relay config (optional)": "Relay 配置 (可选)", - "relay log": "Relay 日志", + "replication detail": "同步详情", "request success": "请求成功", "requesting": "请求中", "resume": "恢复", @@ -93,9 +96,10 @@ "source name": "名称", "source name is required": "名称不能为空", "start import": "开始导入", + "start task": "启动任务", + "start task and remove meta data": "清理元数据启动", "status": "状态", "sync config": "同步配置", - "sync detail": "同步详情", "synchronous concurrency": "同步并发数", "table": "表", "target": "下游", @@ -113,7 +117,6 @@ "user name": "用户名", "user name is required": "用户名不能为空", "view": "查看", - "{{count}} filters_0": "{{count}} 条", - "{{count}} rules_0": "{{count}} 条", - "{{val}} and other {{count}}_0": "{{val}} 等 {{count}} 个" + "{{val}} and {{count}} others": "{{val}} 等总计 {{count}} 个", + "{{val}} and {{count}} others_plural": "{{val}} 等总计 {{count}} 个" } diff --git a/dm/ui/package.json b/dm/ui/package.json index 7eea4f1c5a3..4aad45c37a3 100644 --- a/dm/ui/package.json +++ b/dm/ui/package.json @@ -50,7 +50,7 @@ "less": "^4.1.2", "lint-staged": "^12.1.2", "msw": "^0.36.3", - "msw-auto-mock": "^0.5.0", + "msw-auto-mock": "^0.5.2", "prettier": "^2.5.1", "rollup-plugin-visualizer": "^5.5.2", "typescript": "4.5.4", diff --git a/dm/ui/src/components/BatchImportTaskConfig/index.tsx b/dm/ui/src/components/BatchImportTaskConfig/index.tsx index 468c72491af..cacfb0a417f 100644 --- a/dm/ui/src/components/BatchImportTaskConfig/index.tsx +++ b/dm/ui/src/components/BatchImportTaskConfig/index.tsx @@ -24,7 +24,7 @@ const BlinkingText: React.FC<{ text: string; active: boolean }> = ({ useEffect(() => { if (active) { - timer.current = window.setInterval(() => setCount(c => c + 1), 300) + timer.current = window.setInterval(() => setCount(c => (c + 1) % 4), 300) } return () => { if (timer.current) { @@ -49,7 +49,7 @@ const BatchImportTaskConfig: React.FC = () => { const [overwrite, setOverwrite] = useState(false) const [batchImportTaskConfig] = useDmapiBatchImportTaskConfigMutation() const timerId = useRef() - const stopTimer = useRef(false) + const stopTimer = useRef(false) const calcStepStatus = (thisStep: number): Pick => { if (currentStep === thisStep) { diff --git a/dm/ui/src/components/CreateTaskConfig/MigrateRule.tsx b/dm/ui/src/components/CreateTaskConfig/MigrateRule.tsx index 0aa04aa6871..f9886bc7e2e 100644 --- a/dm/ui/src/components/CreateTaskConfig/MigrateRule.tsx +++ b/dm/ui/src/components/CreateTaskConfig/MigrateRule.tsx @@ -21,12 +21,13 @@ const itemLayout = { wrapperCol: { span: 18 }, } -const createPattern = (name: string[]) => name.join('|') +const createPattern = (name: string[]) => + name.length === 0 ? '*' : name.join('|') const MigrateRule: StepCompnent = ({ prev, initialValues }) => { const [t] = useTranslation() const [currentSource, setCurrentSource] = useState('') - const [cascaderValue, setCascaderValue] = useState([]) + const [cascaderValue, setCascaderValue] = useState([]) const [form] = Form.useForm() const { data } = useDmapiGetSourceListQuery({ withStatus: false }) const [getSourceTable] = useDmapiGetSourceTableListMutation() @@ -59,6 +60,28 @@ const MigrateRule: StepCompnent = ({ prev, initialValues }) => { setCascaderOptions([...cascaderOptions]) } + const handelSchemaSelectInCascader = (value: string[][], field: any) => { + setCascaderValue(value) + const dbPattern = createPattern([ + ...new Set(value.map((item: string[]) => item[0])), + ] as string[]) + const tablePattern = createPattern([ + ...new Set(value.map((item: string[]) => item[1]).filter(Boolean)), + ] as string[]) + const newData = [...form.getFieldValue('table_migrate_rule')] + newData[field.key] = { + ...newData[field.key], + source: { + ...newData[field.key].source, + schema: dbPattern, + table: tablePattern, + }, + } + form.setFieldsValue({ + table_migrate_rule: newData, + }) + } + useEffect(() => { if (schemas) { setCascaderOptions( @@ -124,35 +147,9 @@ const MigrateRule: StepCompnent = ({ prev, initialValues }) => { multiple maxTagCount="responsive" value={cascaderValue} - onChange={(value: any) => { - setCascaderValue(value) - const dbPattern = createPattern([ - ...new Set( - value.map((item: string[]) => item[0]) - ), - ] as string[]) - const tablePattern = createPattern([ - ...new Set( - value - .map((item: string[]) => item[1]) - .filter(Boolean) - ), - ] as string[]) - const newData = [ - ...form.getFieldValue('table_migrate_rule'), - ] - newData[field.key] = { - ...newData[field.key], - source: { - ...newData[field.key].source, - schema: dbPattern, - table: tablePattern, - }, - } - form.setFieldsValue({ - table_migrate_rule: newData, - }) - }} + onChange={(val: any) => + handelSchemaSelectInCascader(val, field) + } > {isFetching ? ( diff --git a/dm/ui/src/components/CreateTaskConfig/TargetInfo.tsx b/dm/ui/src/components/CreateTaskConfig/TargetInfo.tsx index 231182b9ffb..7d9ddf34eb8 100644 --- a/dm/ui/src/components/CreateTaskConfig/TargetInfo.tsx +++ b/dm/ui/src/components/CreateTaskConfig/TargetInfo.tsx @@ -1,7 +1,7 @@ import React from 'react' import { useTranslation } from 'react-i18next' -import { Form, Input, Button } from '~/uikit' +import { Form, Input, Button, InputNumber } from '~/uikit' import { StepCompnent } from '~/components/CreateTaskConfig/shared' const IPv4Pattern = @@ -38,7 +38,7 @@ const TargetInfo: StepCompnent = ({ prev, initialValues }) => { name={['target_config', 'port']} rules={[{ required: true, message: t('port is required') }]} > - + { - + diff --git a/dm/ui/src/components/CreateTaskConfig/index.tsx b/dm/ui/src/components/CreateTaskConfig/index.tsx index 9d2e7122b95..324ace29c41 100644 --- a/dm/ui/src/components/CreateTaskConfig/index.tsx +++ b/dm/ui/src/components/CreateTaskConfig/index.tsx @@ -1,7 +1,7 @@ -import React, { useState, useCallback } from 'react' +import React, { useState, useCallback, useEffect } from 'react' import { useTranslation } from 'react-i18next' import { merge } from 'lodash' -import { useNavigate } from 'react-router-dom' +import { useNavigate, useLocation } from 'react-router-dom' import { Card, Form, Steps, message } from '~/uikit' import { @@ -15,7 +15,10 @@ import SourceInfo from '~/components/CreateTaskConfig/SourceInfo' import TargetInfo from '~/components/CreateTaskConfig/TargetInfo' import EventFilters from '~/components/CreateTaskConfig/EventFilter' import MigrateRule from '~/components/CreateTaskConfig/MigrateRule' -import { useDmapiCreateTaskConfigMutation } from '~/models/taskConfig' +import { + useDmapiCreateTaskConfigMutation, + useDmapiGetTaskConfigQuery, +} from '~/models/taskConfig' const { Step } = Steps @@ -48,10 +51,16 @@ const stepComponents = [ const CreateTaskConfig: React.FC = () => { const [t] = useTranslation() const navigate = useNavigate() + const loc = useLocation() + const shouldFetch = loc.hash.startsWith('#edit') + const [loading, setLoading] = useState(true) const [currentStep, setCurrentStep] = useState(0) const [taskData, setTaskData] = useState(defaultValue) const [createTaskConfig] = useDmapiCreateTaskConfigMutation() - + const { data } = useDmapiGetTaskConfigQuery( + { taskName: loc.hash.split('-')[1] }, + { skip: !shouldFetch } + ) const goNextStep = useCallback(() => { setCurrentStep(c => c + 1) }, []) @@ -94,8 +103,21 @@ const CreateTaskConfig: React.FC = () => { }) } + useEffect(() => { + if (shouldFetch && data) { + setTaskData( + merge({}, defaultValue, data, { + binlog_filter_rule_array: Object.entries( + data?.binlog_filter_rule ?? {} + ).map(([name, value]) => ({ name, ...value })), + }) + ) + setLoading(false) + } + }, [shouldFetch, data]) + return ( - + diff --git a/dm/ui/src/components/StartTaskWithListSelection/index.tsx b/dm/ui/src/components/StartTaskWithListSelection/index.tsx new file mode 100644 index 00000000000..f6f50702023 --- /dev/null +++ b/dm/ui/src/components/StartTaskWithListSelection/index.tsx @@ -0,0 +1,116 @@ +import React, { useMemo, useState } from 'react' +import { useTranslation } from 'react-i18next' + +import { Table, TableColumnsType, Button, Space, Modal, message } from '~/uikit' +import { useDmapiGetTaskConfigListQuery } from '~/models/taskConfig' +import { + Task, + useDmapiGetTaskListQuery, + useDmapiStartTaskMutation, +} from '~/models/task' + +const StartTaskWithListSelection: React.FC<{ + onCancel: () => void +}> = ({ onCancel }) => { + const [t] = useTranslation() + const { data: tasks, isFetching: isFetchingTask } = useDmapiGetTaskListQuery({ + withStatus: false, + }) + const { data: taskConfigs, isFetching } = useDmapiGetTaskConfigListQuery() + const [selectedTasks, setSelectedTasks] = useState([]) + const [startTask] = useDmapiStartTaskMutation() + + const selectableTaskConfigs = useMemo(() => { + const taskNames = new Set(tasks?.data?.map(task => task.name) || []) + return ( + taskConfigs?.data?.filter( + taskConfig => !taskNames.has(taskConfig.name) + ) || [] + ) + }, [taskConfigs, tasks]) + + const handleStartTask = async (removeMeta: boolean) => { + if (selectedTasks.length === 0) { + Modal.error({ + title: t('error'), + content: t('please select at least one task'), + }) + return + } + const key = 'startTask-' + Date.now() + message.loading({ content: t('requesting'), key }) + await Promise.all( + selectedTasks.map(task => { + return startTask({ task, remove_meta: removeMeta }).unwrap() + }) + ) + message.success({ content: t('request success'), key }) + } + + const columns: TableColumnsType = [ + { + title: t('task name'), + dataIndex: 'name', + }, + { + title: t('type'), + dataIndex: 'task_mode', + }, + { + title: t('source info'), + dataIndex: 'source_config', + render(sourceConfig) { + return sourceConfig.source_conf?.length > 0 + ? t('{{val}} and {{count}} others', { + val: `${sourceConfig.source_conf[0].source_name}`, + count: sourceConfig.source_conf.length, + }) + : '-' + }, + }, + { + title: t('target info'), + dataIndex: 'target_config', + render(targetConfig) { + return `${targetConfig.host}:${targetConfig.port}` + }, + }, + ] + + const rowSelection = { + selectedRowKeys: selectedTasks.map(i => i.name), + onChange: (selectedRowKeys: React.Key[], selectedRows: Task[]) => { + setSelectedTasks(selectedRows) + }, + } + + return ( +
+ + +
+ + + + + +
+ + ) +} + +export default StartTaskWithListSelection diff --git a/dm/ui/src/layouts/SiderMenu.tsx b/dm/ui/src/layouts/SiderMenu.tsx index caabb4667c9..df68794f889 100644 --- a/dm/ui/src/layouts/SiderMenu.tsx +++ b/dm/ui/src/layouts/SiderMenu.tsx @@ -82,7 +82,7 @@ const SiderMenu: React.FC<{ {!collapsed && (

- Data Sync Platform + Data Migration

)} diff --git a/dm/ui/src/models/index.ts b/dm/ui/src/models/index.ts index 0f2dd8cff1d..bee306f9c86 100644 --- a/dm/ui/src/models/index.ts +++ b/dm/ui/src/models/index.ts @@ -13,7 +13,10 @@ const rtkQueryErrorLogger: Middleware = () => next => action => { console.error('RTKQ error caught: ', action) // insert your own error handler here message.error({ - content: action.payload?.data?.error_msg ?? 'Oops, somthing went wrong', + content: + action.payload?.data?.error_msg ?? + action.payload?.data?.error ?? + 'Oops, somthing went wrong', }) } diff --git a/dm/ui/src/models/task.ts b/dm/ui/src/models/task.ts index a16d9f25d5d..d379c3453fd 100644 --- a/dm/ui/src/models/task.ts +++ b/dm/ui/src/models/task.ts @@ -240,8 +240,8 @@ export type SubTaskStatus = { name: string source_name: string worker_name: string - stage: string - unit: string + stage: TaskStage + unit: TaskUnit unresolved_ddl_lock_id?: string load_status?: LoadStatus | null sync_status?: SyncStatus | null @@ -257,3 +257,47 @@ export const { useDmapiGetSchemaListByTaskAndSourceQuery, useDmapiGetTableListByTaskAndSourceQuery, } = injectedRtkApi + +export enum TaskUnit { + InvalidUnit = 'InvalidUnit', + Check = 'Check', + Dump = 'Dump', + Load = 'Load', + Sync = 'Sync', + Relay = 'Relay', +} + +export enum TaskStage { + InvalidStage = 'InvalidStage', + New = 'New', + Running = 'Running', + Paused = 'Paused', + Stopped = 'Stopped', + Finished = 'Finished', + Pausing = 'Pausing', + Resuming = 'Resuming', + Stopping = 'Stopping', +} + +// https://github.com/pingcap/tiflow/blob/9261014edd93902d1b0bcb473aec911e80901721/dm/dm/ctl/master/query_status.go#L130 +export const calculateTaskStatus = (subtasks: SubTaskStatus[]) => { + // TODO Error status + + if (subtasks.some(subtask => subtask.stage === TaskStage.Paused)) { + return TaskStage.Paused + } + + if (subtasks.every(subtask => subtask.stage === TaskStage.New)) { + return TaskStage.New + } + + if (subtasks.every(subtask => subtask.stage === TaskStage.Finished)) { + return TaskStage.Finished + } + + if (subtasks.every(subtask => subtask.stage === TaskStage.Stopped)) { + return TaskStage.Stopped + } + + return TaskStage.Running +} diff --git a/dm/ui/src/models/taskConfig.ts b/dm/ui/src/models/taskConfig.ts index ee88087473c..136b9750d21 100644 --- a/dm/ui/src/models/taskConfig.ts +++ b/dm/ui/src/models/taskConfig.ts @@ -8,7 +8,7 @@ const injectedRtkApi = api.injectEndpoints({ { overwrite: boolean } >({ query: queryArg => ({ - url: `/task/templates/import`, + url: `/tasks/templates/import`, method: 'POST', body: queryArg, }), @@ -16,31 +16,31 @@ const injectedRtkApi = api.injectEndpoints({ }), dmapiCreateTaskConfig: build.mutation({ query: queryArg => ({ - url: `/task/templates`, + url: `/tasks/templates`, method: 'POST', body: queryArg, }), invalidatesTags: ['TaskConfig'], }), dmapiGetTaskConfigList: build.query, void>({ - query: () => ({ url: `/task/templates` }), + query: () => ({ url: `/tasks/templates` }), providesTags: ['TaskConfig'], }), dmapiGetTaskConfig: build.query({ query: queryArg => ({ - url: `/task/templates/${queryArg.taskName}`, + url: `/tasks/templates/${queryArg.taskName}`, }), }), dmapUpdateTaskConfig: build.mutation({ query: queryArg => ({ - url: `/task/templates/${queryArg.taskName}`, + url: `/tasks/templates/${queryArg.taskName}`, method: 'PUT', }), invalidatesTags: ['TaskConfig'], }), dmapiDeleteTaskConfig: build.mutation({ query: queryArg => ({ - url: `/task/templates/${queryArg.taskName}`, + url: `/tasks/templates/${queryArg.taskName}`, method: 'DELETE', }), invalidatesTags: ['TaskConfig'], diff --git a/dm/ui/src/pages/cluster/relay-log.tsx b/dm/ui/src/pages/cluster/relay-log.tsx deleted file mode 100644 index 44f577b4b0c..00000000000 --- a/dm/ui/src/pages/cluster/relay-log.tsx +++ /dev/null @@ -1,14 +0,0 @@ -import React from 'react' - -import i18n from '~/i18n' - -const RelayLog: React.FC = () => { - return
TODO
-} - -export const meta = { - title: () => i18n.t('relay log'), - index: 0, -} - -export default RelayLog diff --git a/dm/ui/src/pages/migration/sync-detail.tsx b/dm/ui/src/pages/migration/replication-detail.tsx similarity index 91% rename from dm/ui/src/pages/migration/sync-detail.tsx rename to dm/ui/src/pages/migration/replication-detail.tsx index 4cd0c9f51aa..d764dc35159 100644 --- a/dm/ui/src/pages/migration/sync-detail.tsx +++ b/dm/ui/src/pages/migration/replication-detail.tsx @@ -7,7 +7,7 @@ import { RedoOutlined, SearchOutlined } from '~/uikit/icons' import { useDmapiGetTaskListQuery } from '~/models/task' import { useFuseSearch } from '~/utils/search' -const SyncDetail: React.FC = () => { +const ReplicationDetail: React.FC = () => { const [t] = useTranslation() const { data, isFetching, refetch } = useDmapiGetTaskListQuery({ withStatus: true, @@ -55,8 +55,8 @@ const SyncDetail: React.FC = () => { } export const meta = { - title: () => i18n.t('sync detail'), + title: () => i18n.t('replication detail'), index: 3, } -export default SyncDetail +export default ReplicationDetail diff --git a/dm/ui/src/pages/migration/task-config.tsx b/dm/ui/src/pages/migration/task-config.tsx index 0728bede07d..c5e55e6da2f 100644 --- a/dm/ui/src/pages/migration/task-config.tsx +++ b/dm/ui/src/pages/migration/task-config.tsx @@ -12,6 +12,8 @@ import { Dropdown, Menu, Modal, + message, + TableColumnsType, } from '~/uikit' import { RedoOutlined, @@ -20,31 +22,61 @@ import { ImportOutlined, DownOutlined, SearchOutlined, + PlayCircleOutlined, + ExclamationCircleOutlined, } from '~/uikit/icons' import i18n from '~/i18n' -import { useDmapiGetTaskConfigListQuery } from '~/models/taskConfig' -import { Task } from '~/models/task' +import { + useDmapiDeleteTaskConfigMutation, + useDmapiGetTaskConfigListQuery, +} from '~/models/taskConfig' +import { Task, useDmapiStartTaskMutation } from '~/models/task' import { unimplemented } from '~/utils/unimplemented' +import { useFuseSearch } from '~/utils/search' import CreateTaskConfig from '~/components/CreateTaskConfig' import BatchImportTaskConfig from '~/components/BatchImportTaskConfig' -import { useFuseSearch } from '~/utils/search' const TaskConfig: React.FC = () => { const [t] = useTranslation() - const [isModalVisible, setIsModalVisible] = useState(false) + const [isImportTaskModalVisible, setIsImportTaskModalVisible] = + useState(false) + const [selected, setSelected] = useState([]) + const navigate = useNavigate() const loc = useLocation() const { data, isFetching, refetch } = useDmapiGetTaskConfigListQuery( undefined, - { skip: loc.hash === '#new' } + { skip: loc.hash === '#new' || loc.hash === '#edit' } ) + const [startTask] = useDmapiStartTaskMutation() + const [deleteTask] = useDmapiDeleteTaskConfigMutation() + + const handleDeleteTask = (taskName: string) => { + Modal.confirm({ + title: t('are you sure to delete task {{name}}', { name: taskName }), + icon: , + onOk: async () => { + const key = 'deleteTask-' + Date.now() + message.loading({ content: t('requesting'), key }) + await deleteTask({ taskName }).unwrap() + message.success({ content: t('request success'), key }) + }, + }) + } + + const rowSelection = { + selectedRowKeys: selected.map(i => i.name), + onChange: (selectedRowKeys: React.Key[], selectedRows: Task[]) => { + setSelected(selectedRows) + }, + } const dataSource = data?.data const { result, setKeyword } = useFuseSearch(dataSource, { keys: ['name'], }) - const columns = [ + const columns: TableColumnsType = [ { title: t('name'), dataIndex: 'name', @@ -55,50 +87,48 @@ const TaskConfig: React.FC = () => { }, { title: t('source info'), - render(data: Task) { - const { source_config } = data - - if (source_config?.source_conf?.length > 0) { - return t('{{val}} and other {{count}}', { - val: source_config.source_conf[0].source_name, - count: source_config.source_conf.length, - }) - } - - return '-' + dataIndex: 'source_config', + render(sourceConfig) { + return sourceConfig.source_conf?.length > 0 + ? t('{{val}} and {{count}} others', { + val: `${sourceConfig.source_conf[0].source_name}`, + count: sourceConfig.source_conf.length, + }) + : '-' }, }, { title: t('target info'), - render(data: Task) { - return data.target_config.host + dataIndex: 'target_config', + render(targetConfig) { + return `${targetConfig.host}:${targetConfig.port}` }, }, { title: t('migrate rules'), render(data: Task) { - return t('{{count}} rules', { - count: data.table_migrate_rule.length, - }) + return data.table_migrate_rule.length }, }, { title: t('event filter'), render(data: Task) { - return t('{{count}} filters', { - count: Object.keys(data.binlog_filter_rule || {}).length, - }) + return Object.keys(data.binlog_filter_rule || {}).length }, }, { title: t('operations'), - render() { + render(data) { return ( - - @@ -107,7 +137,25 @@ const TaskConfig: React.FC = () => { }, ] - if (loc.hash === '#new') { + const handleStartTask = async (removeMeta: boolean) => { + if (selected.length === 0) { + Modal.error({ + title: t('error'), + content: t('please select at least one task'), + }) + return + } + const key = 'startTask-' + Date.now() + message.loading({ content: t('requesting'), key }) + await Promise.all( + selected.map(task => { + return startTask({ task, remove_meta: removeMeta }).unwrap() + }) + ) + message.success({ content: t('request success'), key }) + } + + if (loc.hash === '#new' || loc.hash.startsWith('#edit')) { return } @@ -139,11 +187,30 @@ const TaskConfig: React.FC = () => { } > - + + + handleStartTask(true)} key="1"> + {t('start task and remove meta data')} + + + } + > + + @@ -163,17 +230,21 @@ const TaskConfig: React.FC = () => { columns={columns} loading={isFetching} rowKey="name" + rowSelection={rowSelection} pagination={{ total: data?.total, + onChange: () => { + setSelected([]) + }, }} /> setIsModalVisible(false)} + onCancel={() => setIsImportTaskModalVisible(false)} > diff --git a/dm/ui/src/pages/migration/task.tsx b/dm/ui/src/pages/migration/task.tsx index 329e3df4d49..310901af3f6 100644 --- a/dm/ui/src/pages/migration/task.tsx +++ b/dm/ui/src/pages/migration/task.tsx @@ -38,15 +38,18 @@ import { useDmapiPauseTaskMutation, useDmapiResumeTaskMutation, useDmapiGetTaskStatusQuery, + calculateTaskStatus, } from '~/models/task' import i18n from '~/i18n' import { useFuseSearch } from '~/utils/search' +import StartTaskWithListSelection from '~/components/StartTaskWithListSelection' const TaskList: React.FC = () => { const [t] = useTranslation() const [visible, setVisible] = useState(false) const [currentTaskName, setCurrentTaskName] = useState('') const [selectedSources, setSelectedSources] = useState([]) + const [isStartTaskModalVisible, setIsStartTaskModalVisible] = useState(false) const [displayedSubtaskOffset, setDisplayedSubtaskOffset] = useState({ start: 0, end: 10, @@ -132,18 +135,36 @@ const TaskList: React.FC = () => { keys: ['name'], }) - const columns: TableColumnsType | undefined = [ + const columns: TableColumnsType = [ { title: t('task name'), dataIndex: 'name', }, { title: t('source info'), - dataIndex: 'name', + dataIndex: 'source_config', + render(sourceConfig) { + return sourceConfig.source_conf?.length > 0 + ? t('{{val}} and {{count}} others', { + val: `${sourceConfig.source_conf[0].source_name}`, + count: sourceConfig.source_conf.length, + }) + : '-' + }, }, { title: t('target info'), - dataIndex: 'name', + dataIndex: 'target_config', + render(targetConfig) { + return `${targetConfig.host}:${targetConfig.port}` + }, + }, + { + title: t('status'), + dataIndex: 'status_list', + render(subtasks) { + return calculateTaskStatus(subtasks) + }, }, { title: t('operations'), @@ -203,6 +224,13 @@ const TaskList: React.FC = () => { {t('pause')} + + @@ -269,6 +297,19 @@ const TaskList: React.FC = () => { )} + + setIsStartTaskModalVisible(false)} + > + setIsStartTaskModalVisible(false)} + /> + ) } diff --git a/dm/ui/yarn.lock b/dm/ui/yarn.lock index 3bf47b4aa40..b7e8c06c80b 100644 --- a/dm/ui/yarn.lock +++ b/dm/ui/yarn.lock @@ -3090,10 +3090,10 @@ ms@^2.1.1: resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.3.tgz#574c8138ce1d2b5861f0b44579dbadd60c6615b2" integrity sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA== -msw-auto-mock@^0.5.0: - version "0.5.0" - resolved "https://registry.yarnpkg.com/msw-auto-mock/-/msw-auto-mock-0.5.0.tgz#c7e0eadf0be380f14b3cc5a568f106dee9cc8ddd" - integrity sha512-FYHckfLoaX0j9owt7nJnlCQKgAX+GFoUsfKhPD1BT53m5m/rJ4jwyGGxWyqas0Vj9/KESgSEWI+RXh+aTrz1Hg== +msw-auto-mock@^0.5.2: + version "0.5.2" + resolved "https://registry.yarnpkg.com/msw-auto-mock/-/msw-auto-mock-0.5.2.tgz#f58b8385e134f3414e1682a885120047ed9ed0fe" + integrity sha512-NNctZiHIFZZf4rDIBdq/x48F+JQclROW8jf96cAl36ZYkIT8DbWw9gCUk4SzJIEY8Z6tj0cP3lBX8RWYXnKgtg== dependencies: "@apidevtools/swagger-parser" "^10.0.3" cac "^6.7.12" From 3c410c5669ffa3f5d778be16452f2762ba6579f0 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 8 Feb 2022 11:41:35 +0800 Subject: [PATCH 39/72] pkg(both): a separate library for DML row change (#4376) ref pingcap/tiflow#3242 --- dm/pkg/log/log.go | 4 +- dm/pkg/schema/tracker.go | 25 ++- dm/pkg/utils/common.go | 7 + dm/syncer/causality_test.go | 4 +- dm/syncer/compactor_test.go | 4 +- dm/syncer/dml_test.go | 4 +- dm/syncer/syncer.go | 4 +- pkg/sqlmodel/causality.go | 166 ++++++++++++++ pkg/sqlmodel/causality_test.go | 73 +++++++ pkg/sqlmodel/multivalue.go | 202 +++++++++++++++++ pkg/sqlmodel/multivalue_test.go | 68 ++++++ pkg/sqlmodel/reduce.go | 154 +++++++++++++ pkg/sqlmodel/reduce_test.go | 130 +++++++++++ pkg/sqlmodel/row_change.go | 377 ++++++++++++++++++++++++++++++++ pkg/sqlmodel/row_change_test.go | 345 +++++++++++++++++++++++++++++ 15 files changed, 1549 insertions(+), 18 deletions(-) create mode 100644 pkg/sqlmodel/causality.go create mode 100644 pkg/sqlmodel/causality_test.go create mode 100644 pkg/sqlmodel/multivalue.go create mode 100644 pkg/sqlmodel/multivalue_test.go create mode 100644 pkg/sqlmodel/reduce.go create mode 100644 pkg/sqlmodel/reduce_test.go create mode 100644 pkg/sqlmodel/row_change.go create mode 100644 pkg/sqlmodel/row_change_test.go diff --git a/dm/pkg/log/log.go b/dm/pkg/log/log.go index 98920087430..62c63ca42d5 100644 --- a/dm/pkg/log/log.go +++ b/dm/pkg/log/log.go @@ -104,6 +104,7 @@ var ( // InitLogger initializes DM's and also the TiDB library's loggers. func InitLogger(cfg *Config) error { + inDev := strings.ToLower(cfg.Level) == "debug" // init DM logger logger, props, err := pclog.InitLogger(&pclog.Config{ Level: cfg.Level, @@ -114,6 +115,7 @@ func InitLogger(cfg *Config) error { MaxDays: cfg.FileMaxDays, MaxBackups: cfg.FileMaxBackups, }, + Development: inDev, }) if err != nil { return terror.ErrInitLoggerFail.Delegate(err) @@ -125,7 +127,7 @@ func InitLogger(cfg *Config) error { appLevel = props.Level appProps = props // init and set tidb slow query logger to stdout if log level is debug - if cfg.Level == "debug" { + if inDev { slowQueryLogger := zap.NewExample() slowQueryLogger = slowQueryLogger.With(zap.String("component", "slow query logger")) logutil.SlowQueryLogger = slowQueryLogger diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 1d0640cc6f9..1ee9d1015a4 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -403,7 +403,7 @@ func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string return nil, err } - dti = GetDownStreamTi(ti, originTi) + dti = GetDownStreamTI(ti, originTi) tr.dsTracker.tableInfos[tableID] = dti } return dti, nil @@ -412,9 +412,14 @@ func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string // GetAvailableDownStreamUKIndexInfo gets available downstream UK whose data is not null. // note. this function will not init downstreamTrack. func (tr *Tracker) GetAvailableDownStreamUKIndexInfo(tableID string, data []interface{}) *model.IndexInfo { - dti, ok := tr.dsTracker.tableInfos[tableID] + dti := tr.dsTracker.tableInfos[tableID] + + return GetIdentityUKByData(dti, data) +} - if !ok || len(dti.AvailableUKIndexList) == 0 { +// GetIdentityUKByData gets available downstream UK whose data is not null. +func GetIdentityUKByData(downstreamTI *DownstreamTableInfo, data []interface{}) *model.IndexInfo { + if downstreamTI == nil || len(downstreamTI.AvailableUKIndexList) == 0 { return nil } // func for check data is not null @@ -422,7 +427,7 @@ func (tr *Tracker) GetAvailableDownStreamUKIndexInfo(tableID string, data []inte return data[i] != nil } - for _, uk := range dti.AvailableUKIndexList { + for _, uk := range downstreamTI.AvailableUKIndexList { // check uk's column data is not null if isSpecifiedIndexColumn(uk, fn) { return uk @@ -499,8 +504,8 @@ func (tr *Tracker) initDownStreamSQLModeAndParser(tctx *tcontext.Context) error return nil } -// GetDownStreamTi constructs downstreamTable index cache by tableinfo. -func GetDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *DownstreamTableInfo { +// GetDownStreamTI constructs downstreamTable index cache by tableinfo. +func GetDownStreamTI(downstreamTI *model.TableInfo, originTi *model.TableInfo) *DownstreamTableInfo { var ( absoluteUKIndexInfo *model.IndexInfo availableUKIndexList = []*model.IndexInfo{} @@ -510,10 +515,10 @@ func GetDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *Downstream // func for check not null constraint fn := func(i int) bool { - return mysql.HasNotNullFlag(ti.Columns[i].Flag) + return mysql.HasNotNullFlag(downstreamTI.Columns[i].Flag) } - for i, idx := range ti.Indices { + for i, idx := range downstreamTI.Indices { if !idx.Primary && !idx.Unique { continue } @@ -536,7 +541,7 @@ func GetDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *Downstream // handle pk exceptional case. // e.g. "create table t(a int primary key, b int)". if !hasPk { - exPk := redirectIndexKeys(handlePkExCase(ti), originTi) + exPk := redirectIndexKeys(handlePkExCase(downstreamTI), originTi) if exPk != nil { absoluteUKIndexInfo = exPk absoluteUKPosition = len(availableUKIndexList) @@ -550,7 +555,7 @@ func GetDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *Downstream } return &DownstreamTableInfo{ - TableInfo: ti, + TableInfo: downstreamTI, AbsoluteUKIndexInfo: absoluteUKIndexInfo, AvailableUKIndexList: availableUKIndexList, } diff --git a/dm/pkg/utils/common.go b/dm/pkg/utils/common.go index 1718f2d3fb6..25c4ecb348a 100644 --- a/dm/pkg/utils/common.go +++ b/dm/pkg/utils/common.go @@ -37,6 +37,10 @@ import ( "github.com/pingcap/tiflow/dm/pkg/terror" ) +func init() { + ZeroSessionCtx = NewSessionCtx(nil) +} + // TrimCtrlChars returns a slice of the string s with all leading // and trailing control characters removed. func TrimCtrlChars(s string) string { @@ -322,6 +326,9 @@ func (se *session) GetBuiltinFunctionUsage() map[string]uint32 { return se.builtinFunctionUsage } +// ZeroSessionCtx is used when the session variables is not important. +var ZeroSessionCtx sessionctx.Context + // NewSessionCtx return a session context with specified session variables. func NewSessionCtx(vars map[string]string) sessionctx.Context { variables := variable.NewSessionVars() diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index 1354ca7a979..1109b8c1b80 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -83,7 +83,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { Length: types.UnspecifiedLength, }}, } - downTi := schema.GetDownStreamTi(ti, ti) + downTi := schema.GetDownStreamTI(ti, ti) c.Assert(downTi, NotNil) jobCh := make(chan *job, 10) @@ -152,7 +152,7 @@ func (s *testSyncerSuite) TestCasualityWithPrefixIndex(c *C) { schemaStr := "create table t (c1 text, c2 int unique, unique key c1(c1(3)));" ti, err := createTableInfo(p, se, int64(0), schemaStr) c.Assert(err, IsNil) - downTi := schema.GetDownStreamTi(ti, ti) + downTi := schema.GetDownStreamTI(ti, ti) c.Assert(downTi, NotNil) c.Assert(len(downTi.AvailableUKIndexList) == 2, IsTrue) tiIndex := downTi.AvailableUKIndexList[0] diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 06ad3993791..506f9581a7f 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -91,7 +91,7 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { Length: types.UnspecifiedLength, }}, } - downTi := schema.GetDownStreamTi(ti, ti) + downTi := schema.GetDownStreamTI(ti, ti) c.Assert(downTi, NotNil) var dml *DML @@ -208,7 +208,7 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { Length: types.UnspecifiedLength, }}, } - downTi := schema.GetDownStreamTi(ti, ti) + downTi := schema.GetDownStreamTI(ti, ti) c.Assert(downTi, NotNil) testCases := []struct { diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index a7748abb4bf..148c2538eff 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -224,7 +224,7 @@ func (s *testSyncerSuite) TestGenMultipleKeys(c *C) { ti, err := createTableInfo(p, se, int64(i+1), tc.schema) assert(err, IsNil) - dti := schema.GetDownStreamTi(ti, ti) + dti := schema.GetDownStreamTI(ti, ti) assert(dti, NotNil) keys := genMultipleKeys(sessCtx, dti, ti, tc.values, "table") assert(keys, DeepEquals, tc.keys) @@ -619,7 +619,7 @@ func (s *testSyncerSuite) TestTruncateIndexValues(c *C) { } ti, err := createTableInfo(p, se, int64(i+1), tc.schema) assert(err, IsNil) - dti := schema.GetDownStreamTi(ti, ti) + dti := schema.GetDownStreamTI(ti, ti) assert(dti, NotNil) assert(dti.AvailableUKIndexList, NotNil) cols := make([]*model.ColumnInfo, 0, len(dti.AvailableUKIndexList[0].Columns)) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 212a3fa9bb1..0c0ca3d9392 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -871,7 +871,9 @@ func (s *Syncer) updateReplicationLagMetric() { func (s *Syncer) saveTablePoint(table *filter.Table, location binlog.Location) { ti, err := s.schemaTracker.GetTableInfo(table) if err != nil && table.Name != "" { - s.tctx.L().DPanic("table info missing from schema tracker", + // TODO: if we RENAME tb1 TO tb2, the tracker will remove TableInfo of tb1 but we still save the table + // checkpoint for tb1. We can delete the table checkpoint in future. + s.tctx.L().Warn("table info missing from schema tracker", zap.Stringer("table", table), zap.Stringer("location", location), zap.Error(err)) diff --git a/pkg/sqlmodel/causality.go b/pkg/sqlmodel/causality.go new file mode 100644 index 00000000000..cc8d8f4572b --- /dev/null +++ b/pkg/sqlmodel/causality.go @@ -0,0 +1,166 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "fmt" + "strconv" + "strings" + + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/tablecodec" + "go.uber.org/zap" + + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" +) + +// CausalityKeys returns all string representation of causality keys. If two row +// changes has the same causality keys, they must be replicated sequentially. +func (r *RowChange) CausalityKeys() []string { + r.lazyInitIdentityInfo() + + ret := make([]string, 0, 1) + if r.preValues != nil { + ret = append(ret, r.getCausalityString(r.preValues)...) + } + if r.postValues != nil { + ret = append(ret, r.getCausalityString(r.postValues)...) + } + return ret +} + +func columnValue2String(value interface{}) string { + var data string + switch v := value.(type) { + case nil: + data = "null" + case bool: + if v { + data = "1" + } else { + data = "0" + } + case int: + data = strconv.FormatInt(int64(v), 10) + case int8: + data = strconv.FormatInt(int64(v), 10) + case int16: + data = strconv.FormatInt(int64(v), 10) + case int32: + data = strconv.FormatInt(int64(v), 10) + case int64: + data = strconv.FormatInt(v, 10) + case uint8: + data = strconv.FormatUint(uint64(v), 10) + case uint16: + data = strconv.FormatUint(uint64(v), 10) + case uint32: + data = strconv.FormatUint(uint64(v), 10) + case uint64: + data = strconv.FormatUint(v, 10) + case float32: + data = strconv.FormatFloat(float64(v), 'f', -1, 32) + case float64: + data = strconv.FormatFloat(v, 'f', -1, 64) + case string: + data = v + case []byte: + data = string(v) + default: + data = fmt.Sprintf("%v", v) + } + + return data +} + +func genKeyString( + table string, + columns []*timodel.ColumnInfo, + values []interface{}, +) string { + var buf strings.Builder + for i, data := range values { + if data == nil { + log.L().Debug("ignore null value", + zap.String("column", columns[i].Name.O), + zap.String("table", table)) + continue // ignore `null` value. + } + // one column key looks like:`column_val.column_name.` + buf.WriteString(columnValue2String(data)) + buf.WriteString(".") + buf.WriteString(columns[i].Name.L) + buf.WriteString(".") + } + if buf.Len() == 0 { + log.L().Debug("all value are nil, no key generated", + zap.String("table", table)) + return "" // all values are `null`. + } + buf.WriteString(table) + return buf.String() +} + +// truncateIndexValues truncate prefix index from data. +func truncateIndexValues( + ctx sessionctx.Context, + ti *timodel.TableInfo, + indexColumns *timodel.IndexInfo, + tiColumns []*timodel.ColumnInfo, + data []interface{}, +) []interface{} { + values := make([]interface{}, 0, len(indexColumns.Columns)) + datums, err := utils.AdjustBinaryProtocolForDatum(ctx, data, tiColumns) + if err != nil { + log.L().Warn("adjust binary protocol for datum error", zap.Error(err)) + return data + } + tablecodec.TruncateIndexValues(ti, indexColumns, datums) + for _, datum := range datums { + values = append(values, datum.GetValue()) + } + return values +} + +func (r *RowChange) getCausalityString(values []interface{}) []string { + pkAndUks := r.identityInfo.AvailableUKIndexList + if len(pkAndUks) == 0 { + // the table has no PK/UK, all values of the row consists the causality key + return []string{genKeyString(r.sourceTable.String(), r.sourceTableInfo.Columns, values)} + } + + ret := make([]string, 0, len(pkAndUks)) + + for _, indexCols := range pkAndUks { + cols, vals := getColsAndValuesOfIdx(r.sourceTableInfo.Columns, indexCols, values) + // handle prefix index + truncVals := truncateIndexValues(r.tiSessionCtx, r.sourceTableInfo, indexCols, cols, vals) + key := genKeyString(r.sourceTable.String(), cols, truncVals) + if len(key) > 0 { // ignore `null` value. + ret = append(ret, key) + } else { + log.L().Debug("ignore empty key", zap.String("table", r.sourceTable.String())) + } + } + + if len(ret) == 0 { + // the table has no PK/UK, or all UK are NULL. all values of the row + // consists the causality key + return []string{genKeyString(r.sourceTable.String(), r.sourceTableInfo.Columns, values)} + } + + return ret +} diff --git a/pkg/sqlmodel/causality_test.go b/pkg/sqlmodel/causality_test.go new file mode 100644 index 00000000000..4a49e95640e --- /dev/null +++ b/pkg/sqlmodel/causality_test.go @@ -0,0 +1,73 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "sync" + "testing" + + "github.com/stretchr/testify/require" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" +) + +func TestCausalityKeys(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + + cases := []struct { + createSQL string + preValue []interface{} + postValue []interface{} + + causalityKeys []string + }{ + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT, c3 VARCHAR(10) UNIQUE)", + []interface{}{1, 2, "abc"}, + []interface{}{3, 4, "abc"}, + []string{"1.c.db.tb1", "abc.c3.db.tb1", "3.c.db.tb1", "abc.c3.db.tb1"}, + }, + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT, c3 VARCHAR(10), UNIQUE INDEX(c3(1)))", + []interface{}{1, 2, "abc"}, + []interface{}{3, 4, "adef"}, + []string{"1.c.db.tb1", "a.c3.db.tb1", "3.c.db.tb1", "a.c3.db.tb1"}, + }, + } + + for _, ca := range cases { + ti := mockTableInfo(t, ca.createSQL) + change := NewRowChange(source, nil, ca.preValue, ca.postValue, ti, nil, nil) + require.Equal(t, ca.causalityKeys, change.CausalityKeys()) + } +} + +func TestCausalityKeysNoRace(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + ti := mockTableInfo(t, "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT, c3 VARCHAR(10) UNIQUE)") + var wg sync.WaitGroup + for i := 0; i < 100; i++ { + wg.Add(1) + go func() { + change := NewRowChange(source, nil, []interface{}{1, 2, "abc"}, []interface{}{3, 4, "abc"}, ti, nil, nil) + change.CausalityKeys() + wg.Done() + }() + } + wg.Wait() +} diff --git a/pkg/sqlmodel/multivalue.go b/pkg/sqlmodel/multivalue.go new file mode 100644 index 00000000000..8c77387b276 --- /dev/null +++ b/pkg/sqlmodel/multivalue.go @@ -0,0 +1,202 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "strings" + + "go.uber.org/zap" + + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/pkg/quotes" +) + +// SameTypeTargetAndColumns check whether two row changes have same type, target +// and columns, so they can be merged to a multi-value DML. +func SameTypeTargetAndColumns(lhs *RowChange, rhs *RowChange) bool { + if lhs.tp != rhs.tp { + return false + } + if lhs.sourceTable.Schema == rhs.sourceTable.Schema && + lhs.sourceTable.Table == rhs.sourceTable.Table { + return true + } + if lhs.targetTable.Schema != rhs.targetTable.Schema || + lhs.targetTable.Table != rhs.targetTable.Table { + return false + } + + // when the targets are the same and the sources are not the same (same + // group of shard tables), this piece of code is run. + var lhsCols, rhsCols []string + switch lhs.tp { + case RowChangeDelete: + lhsCols, _ = lhs.whereColumnsAndValues() + rhsCols, _ = rhs.whereColumnsAndValues() + case RowChangeUpdate: + // not supported yet + return false + case RowChangeInsert: + for _, col := range lhs.sourceTableInfo.Columns { + lhsCols = append(lhsCols, col.Name.L) + } + for _, col := range rhs.sourceTableInfo.Columns { + rhsCols = append(rhsCols, col.Name.L) + } + } + + if len(lhsCols) != len(rhsCols) { + return false + } + for i := 0; i < len(lhsCols); i++ { + if lhsCols[i] != rhsCols[i] { + return false + } + } + return true +} + +// GenDeleteSQL generates the DELETE SQL and its arguments. +// Input `changes` should have same target table and same columns for WHERE +// (typically same PK/NOT NULL UK), otherwise the behaviour is undefined. +func GenDeleteSQL(changes ...*RowChange) (string, []interface{}) { + if len(changes) == 0 { + log.L().DPanic("row changes is empty") + return "", nil + } + + first := changes[0] + + var buf strings.Builder + buf.Grow(1024) + buf.WriteString("DELETE FROM ") + buf.WriteString(first.targetTable.QuoteString()) + buf.WriteString(" WHERE (") + + whereColumns, _ := first.whereColumnsAndValues() + for i, column := range whereColumns { + if i != len(whereColumns)-1 { + buf.WriteString(quotes.QuoteName(column) + ",") + } else { + buf.WriteString(quotes.QuoteName(column) + ")") + } + } + buf.WriteString(" IN (") + // TODO: can't handle NULL by IS NULL, should use WHERE OR + args := make([]interface{}, 0, len(changes)*len(whereColumns)) + holder := valuesHolder(len(whereColumns)) + for i, change := range changes { + if i > 0 { + buf.WriteString(",") + } + buf.WriteString(holder) + _, whereValues := change.whereColumnsAndValues() + // a simple check about different number of WHERE values, not trying to + // cover all cases + if len(whereValues) != len(whereColumns) { + log.L().DPanic("len(whereValues) != len(whereColumns)", + zap.Int("len(whereValues)", len(whereValues)), + zap.Int("len(whereColumns)", len(whereColumns)), + zap.Any("whereValues", whereValues), + zap.Stringer("sourceTable", change.sourceTable)) + return "", nil + } + args = append(args, whereValues...) + } + buf.WriteString(")") + return buf.String(), args +} + +// TODO: support GenUpdateSQL(changes ...*RowChange) using UPDATE SET CASE WHEN + +// GenInsertSQL generates the INSERT SQL and its arguments. +// Input `changes` should have same target table and same modifiable columns, +// otherwise the behaviour is undefined. +func GenInsertSQL(tp DMLType, changes ...*RowChange) (string, []interface{}) { + if len(changes) == 0 { + log.L().DPanic("row changes is empty") + return "", nil + } + + first := changes[0] + + var buf strings.Builder + buf.Grow(1024) + if tp == DMLReplace { + buf.WriteString("REPLACE INTO ") + } else { + buf.WriteString("INSERT INTO ") + } + buf.WriteString(first.targetTable.QuoteString()) + buf.WriteString(" (") + columnNum := 0 + var skipColIdx []int + for i, col := range first.sourceTableInfo.Columns { + if isGenerated(first.targetTableInfo.Columns, col.Name) { + skipColIdx = append(skipColIdx, i) + continue + } + + if columnNum != 0 { + buf.WriteByte(',') + } + columnNum++ + buf.WriteString(quotes.QuoteName(col.Name.O)) + } + buf.WriteString(") VALUES ") + holder := valuesHolder(columnNum) + for i := range changes { + if i > 0 { + buf.WriteString(",") + } + buf.WriteString(holder) + } + if tp == DMLInsertOnDuplicateUpdate { + buf.WriteString(" ON DUPLICATE KEY UPDATE ") + i := 0 // used as index of skipColIdx + writtenFirstCol := false + + for j, col := range first.sourceTableInfo.Columns { + if i < len(skipColIdx) && skipColIdx[i] == j { + i++ + continue + } + + if writtenFirstCol { + buf.WriteByte(',') + } + writtenFirstCol = true + + colName := quotes.QuoteName(col.Name.O) + buf.WriteString(colName + "=VALUES(" + colName + ")") + } + } + + args := make([]interface{}, 0, len(changes)*(len(first.sourceTableInfo.Columns)-len(skipColIdx))) + for _, change := range changes { + i := 0 // used as index of skipColIdx + for j, val := range change.postValues { + if i >= len(skipColIdx) { + args = append(args, change.postValues[j:]...) + break + } + if skipColIdx[i] == j { + i++ + continue + } + args = append(args, val) + } + } + return buf.String(), args +} diff --git a/pkg/sqlmodel/multivalue_test.go b/pkg/sqlmodel/multivalue_test.go new file mode 100644 index 00000000000..a06326d5ee9 --- /dev/null +++ b/pkg/sqlmodel/multivalue_test.go @@ -0,0 +1,68 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "testing" + + "github.com/stretchr/testify/require" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" +) + +func TestGenDeleteMultiValue(t *testing.T) { + t.Parallel() + + source1 := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + source2 := &cdcmodel.TableName{Schema: "db", Table: "tb2"} + target := &cdcmodel.TableName{Schema: "db", Table: "tb"} + + sourceTI1 := mockTableInfo(t, "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)") + sourceTI2 := mockTableInfo(t, "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT)") + targetTI := mockTableInfo(t, "CREATE TABLE tb (c INT PRIMARY KEY, c2 INT)") + + change1 := NewRowChange(source1, target, []interface{}{1, 2}, nil, sourceTI1, targetTI, nil) + change2 := NewRowChange(source2, target, []interface{}{3, 4}, nil, sourceTI2, targetTI, nil) + sql, args := GenDeleteSQL(change1, change2) + + require.Equal(t, "DELETE FROM `db`.`tb` WHERE (`c`) IN ((?),(?))", sql) + require.Equal(t, []interface{}{1, 3}, args) +} + +func TestGenInsertMultiValue(t *testing.T) { + t.Parallel() + + source1 := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + source2 := &cdcmodel.TableName{Schema: "db", Table: "tb2"} + target := &cdcmodel.TableName{Schema: "db", Table: "tb"} + + sourceTI1 := mockTableInfo(t, "CREATE TABLE tb1 (gen INT AS (c+1), c INT PRIMARY KEY, c2 INT)") + sourceTI2 := mockTableInfo(t, "CREATE TABLE tb2 (gen INT AS (c+1), c INT PRIMARY KEY, c2 INT)") + targetTI := mockTableInfo(t, "CREATE TABLE tb (gen INT AS (c+1), c INT PRIMARY KEY, c2 INT)") + + change1 := NewRowChange(source1, target, nil, []interface{}{2, 1, 2}, sourceTI1, targetTI, nil) + change2 := NewRowChange(source2, target, nil, []interface{}{4, 3, 4}, sourceTI2, targetTI, nil) + + sql, args := GenInsertSQL(DMLInsert, change1, change2) + require.Equal(t, "INSERT INTO `db`.`tb` (`c`,`c2`) VALUES (?,?),(?,?)", sql) + require.Equal(t, []interface{}{1, 2, 3, 4}, args) + + sql, args = GenInsertSQL(DMLReplace, change1, change2) + require.Equal(t, "REPLACE INTO `db`.`tb` (`c`,`c2`) VALUES (?,?),(?,?)", sql) + require.Equal(t, []interface{}{1, 2, 3, 4}, args) + + sql, args = GenInsertSQL(DMLInsertOnDuplicateUpdate, change1, change2) + require.Equal(t, "INSERT INTO `db`.`tb` (`c`,`c2`) VALUES (?,?),(?,?) ON DUPLICATE KEY UPDATE `c`=VALUES(`c`),`c2`=VALUES(`c2`)", sql) + require.Equal(t, []interface{}{1, 2, 3, 4}, args) +} diff --git a/pkg/sqlmodel/reduce.go b/pkg/sqlmodel/reduce.go new file mode 100644 index 00000000000..6e146e2d54c --- /dev/null +++ b/pkg/sqlmodel/reduce.go @@ -0,0 +1,154 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "fmt" + "strings" + + "go.uber.org/zap" + + "github.com/pingcap/tiflow/dm/pkg/log" +) + +// HasNotNullUniqueIdx returns true when the target table structure has PK or UK +// whose columns are all NOT NULL. +func (r *RowChange) HasNotNullUniqueIdx() bool { + r.lazyInitIdentityInfo() + + return r.identityInfo.AbsoluteUKIndexInfo != nil +} + +// IdentityValues returns the two group of values that can be used to identify +// the row. That is to say, if two row changes has same IdentityValues, they are +// changes of the same row. We can use this property to only replicate latest +// changes of one row. +// We always use same index for same table structure to get IdentityValues. +// two groups returned are from preValues and postValues. +func (r *RowChange) IdentityValues() ([]interface{}, []interface{}) { + r.lazyInitIdentityInfo() + + indexInfo := r.identityInfo.AbsoluteUKIndexInfo + if indexInfo == nil { + return r.preValues, r.postValues + } + + pre := make([]interface{}, 0, len(indexInfo.Columns)) + post := make([]interface{}, 0, len(indexInfo.Columns)) + + for _, column := range indexInfo.Columns { + if r.preValues != nil { + pre = append(pre, r.preValues[column.Offset]) + } + if r.postValues != nil { + post = append(post, r.postValues[column.Offset]) + } + } + return pre, post +} + +func (r *RowChange) IsIdentityUpdated() bool { + if r.tp != RowChangeUpdate { + return false + } + + r.lazyInitIdentityInfo() + pre, post := r.IdentityValues() + if len(pre) != len(post) { + // should not happen + return true + } + for i := range pre { + if pre[i] != post[i] { + return true + } + } + return false +} + +// genKey gens key by values e.g. "a.1.b". +func genKey(values []interface{}) string { + builder := new(strings.Builder) + for i, v := range values { + if i != 0 { + builder.WriteString(".") + } + fmt.Fprintf(builder, "%v", v) + } + + return builder.String() +} + +// IdentityKey returns a string generated by IdentityValues. +// If RowChange.IsIdentityUpdated, the behaviour is undefined. +func (r *RowChange) IdentityKey() string { + pre, post := r.IdentityValues() + if len(pre) != 0 { + return genKey(pre) + } + return genKey(post) +} + +// Reduce will merge two row changes of same row into one row changes, +// e.g., INSERT{1} + UPDATE{1 -> 2} -> INSERT{2}. Receiver will be changed +// in-place. +func (r *RowChange) Reduce(preRowChange *RowChange) { + if r.IdentityKey() != preRowChange.IdentityKey() { + log.L().DPanic("reduce row change failed, identity key not match", + zap.String("preID", preRowChange.IdentityKey()), + zap.String("curID", r.IdentityKey())) + return + } + + // special handle INSERT + DELETE -> DELETE + if r.tp == RowChangeDelete && preRowChange.tp == RowChangeInsert { + return + } + + r.preValues = preRowChange.preValues + r.calculateType() +} + +// Split will split current RowChangeUpdate into two RowChangeDelete and +// RowChangeInsert one. The behaviour is undefined for other types of RowChange. +func (r *RowChange) Split() (*RowChange, *RowChange) { + if r.tp != RowChangeUpdate { + log.L().DPanic("Split should only be called on RowChangeUpdate", + zap.Stringer("rowChange", r)) + return nil, nil + } + + pre := &RowChange{ + sourceTable: r.sourceTable, + targetTable: r.targetTable, + preValues: r.preValues, + sourceTableInfo: r.sourceTableInfo, + targetTableInfo: r.targetTableInfo, + tiSessionCtx: r.tiSessionCtx, + tp: RowChangeDelete, + identityInfo: r.identityInfo, + } + post := &RowChange{ + sourceTable: r.sourceTable, + targetTable: r.targetTable, + postValues: r.postValues, + sourceTableInfo: r.sourceTableInfo, + targetTableInfo: r.targetTableInfo, + tiSessionCtx: r.tiSessionCtx, + tp: RowChangeInsert, + identityInfo: r.identityInfo, + } + + return pre, post +} diff --git a/pkg/sqlmodel/reduce_test.go b/pkg/sqlmodel/reduce_test.go new file mode 100644 index 00000000000..876e2089252 --- /dev/null +++ b/pkg/sqlmodel/reduce_test.go @@ -0,0 +1,130 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "testing" + + "github.com/stretchr/testify/require" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" +) + +func TestIdentity(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + sourceTI1 := mockTableInfo(t, "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)") + + change := NewRowChange(source, nil, []interface{}{1, 2}, nil, sourceTI1, nil, nil) + pre, post := change.IdentityValues() + require.Equal(t, []interface{}{1}, pre) + require.Len(t, post, 0) + + change = NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{1, 4}, sourceTI1, nil, nil) + pre, post = change.IdentityValues() + require.Equal(t, []interface{}{1}, pre) + require.Equal(t, []interface{}{1}, post) + require.False(t, change.IsIdentityUpdated()) + + sourceTI2 := mockTableInfo(t, "CREATE TABLE tb1 (c INT, c2 INT)") + change = NewRowChange(source, nil, nil, []interface{}{5, 6}, sourceTI2, nil, nil) + pre, post = change.IdentityValues() + require.Len(t, pre, 0) + require.Equal(t, []interface{}{5, 6}, post) +} + +func TestSplit(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + sourceTI1 := mockTableInfo(t, "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)") + + change := NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{3, 4}, sourceTI1, nil, nil) + require.True(t, change.IsIdentityUpdated()) + del, ins := change.Split() + delIDKey := del.IdentityKey() + require.NotZero(t, delIDKey) + insIDKey := ins.IdentityKey() + require.NotZero(t, insIDKey) + require.NotEqual(t, delIDKey, insIDKey) +} + +func (s *dpanicSuite) TestReduce() { + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + sourceTI := mockTableInfo(s.T(), "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)") + + cases := []struct { + pre1 []interface{} + post1 []interface{} + pre2 []interface{} + post2 []interface{} + preAfter []interface{} + postAfter []interface{} + }{ + // INSERT + UPDATE + { + nil, + []interface{}{1, 2}, + []interface{}{1, 2}, + []interface{}{3, 4}, + nil, + []interface{}{3, 4}, + }, + // INSERT + DELETE + { + nil, + []interface{}{1, 2}, + []interface{}{1, 2}, + nil, + []interface{}{1, 2}, + nil, + }, + // UPDATE + UPDATE + { + []interface{}{1, 2}, + []interface{}{1, 3}, + []interface{}{1, 3}, + []interface{}{1, 4}, + []interface{}{1, 2}, + []interface{}{1, 4}, + }, + // UPDATE + DELETE + { + []interface{}{1, 2}, + []interface{}{1, 3}, + []interface{}{1, 3}, + nil, + []interface{}{1, 2}, + nil, + }, + } + + for _, c := range cases { + change1 := NewRowChange(source, nil, c.pre1, c.post1, sourceTI, nil, nil) + change2 := NewRowChange(source, nil, c.pre2, c.post2, sourceTI, nil, nil) + changeAfter := NewRowChange(source, nil, c.preAfter, c.postAfter, sourceTI, nil, nil) + changeAfter.lazyInitIdentityInfo() + + change2.Reduce(change1) + s.Equal(changeAfter, change2) + } + + // test reduce on IdentityUpdated will DPanic + change1 := NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{3, 4}, sourceTI, nil, nil) + change2 := NewRowChange(source, nil, []interface{}{3, 4}, []interface{}{5, 6}, sourceTI, nil, nil) + s.Panics(func() { + change2.Reduce(change1) + }) +} diff --git a/pkg/sqlmodel/row_change.go b/pkg/sqlmodel/row_change.go new file mode 100644 index 00000000000..a4a3cbfd8ca --- /dev/null +++ b/pkg/sqlmodel/row_change.go @@ -0,0 +1,377 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "fmt" + "strings" + + "github.com/pingcap/failpoint" + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "go.uber.org/zap" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/quotes" +) + +type RowChangeType int + +// these constants represent types of row change. +const ( + RowChangeNull RowChangeType = iota + RowChangeInsert + RowChangeUpdate + RowChangeDelete +) + +// String implements fmt.Stringer interface. +func (t RowChangeType) String() string { + switch t { + case RowChangeInsert: + return "ChangeInsert" + case RowChangeUpdate: + return "ChangeUpdate" + case RowChangeDelete: + return "ChangeDelete" + } + + return "" +} + +// RowChange represents a row change, it can be further converted into DML SQL. +type RowChange struct { + sourceTable *cdcmodel.TableName + targetTable *cdcmodel.TableName + + preValues []interface{} + postValues []interface{} + + sourceTableInfo *timodel.TableInfo + targetTableInfo *timodel.TableInfo + + tiSessionCtx sessionctx.Context + + tp RowChangeType + identityInfo *schema.DownstreamTableInfo +} + +// NewRowChange creates a new RowChange. +// preValues stands for values exists before this change, postValues stands for +// values exists after this change. +// These parameters can be nil: +// - targetTable: when same as sourceTable or not applicable +// - preValues: when INSERT +// - postValues: when DELETE +// - targetTableInfo: when same as sourceTableInfo or not applicable +// - tiSessionCtx: will use default sessionCtx which is UTC timezone +// All arguments must not be changed after assigned to RowChange, any +// modification (like convert []byte to string) should be done before +// NewRowChange. +func NewRowChange( + sourceTable *cdcmodel.TableName, + targetTable *cdcmodel.TableName, + preValues []interface{}, + postValues []interface{}, + sourceTableInfo *timodel.TableInfo, + downstreamTableInfo *timodel.TableInfo, + tiCtx sessionctx.Context, +) *RowChange { + ret := &RowChange{ + sourceTable: sourceTable, + preValues: preValues, + postValues: postValues, + sourceTableInfo: sourceTableInfo, + } + + if preValues != nil && len(preValues) != len(sourceTableInfo.Columns) { + log.L().DPanic("preValues length not equal to sourceTableInfo columns", + zap.Int("preValues", len(preValues)), + zap.Int("sourceTableInfo", len(sourceTableInfo.Columns)), + zap.Stringer("sourceTable", sourceTable)) + } + if postValues != nil && len(postValues) != len(sourceTableInfo.Columns) { + log.L().DPanic("postValues length not equal to sourceTableInfo columns", + zap.Int("postValues", len(postValues)), + zap.Int("sourceTableInfo", len(sourceTableInfo.Columns)), + zap.Stringer("sourceTable", sourceTable)) + } + + if targetTable != nil { + ret.targetTable = targetTable + } else { + ret.targetTable = sourceTable + } + + if downstreamTableInfo != nil { + ret.targetTableInfo = downstreamTableInfo + } else { + ret.targetTableInfo = sourceTableInfo + } + + if tiCtx != nil { + ret.tiSessionCtx = tiCtx + } else { + ret.tiSessionCtx = utils.ZeroSessionCtx + } + + ret.calculateType() + + return ret +} + +func (r *RowChange) calculateType() { + switch { + case r.preValues == nil && r.postValues != nil: + r.tp = RowChangeInsert + case r.preValues != nil && r.postValues != nil: + r.tp = RowChangeUpdate + case r.preValues != nil && r.postValues == nil: + r.tp = RowChangeDelete + default: + log.L().DPanic("preValues and postValues can't both be nil", + zap.Stringer("sourceTable", r.sourceTable)) + } +} + +// Type returns the RowChangeType of this RowChange. Caller can future decide +// the DMLType when generate DML from it. +func (r *RowChange) Type() RowChangeType { + return r.tp +} + +// String implements Stringer interface. +func (r *RowChange) String() string { + return fmt.Sprintf("type: %s, source table: %s, target table: %s, preValues: %v, postValues: %v", + r.tp, r.sourceTable, r.targetTable, r.preValues, r.postValues) +} + +// TargetTableID returns a ID string for target table. +func (r *RowChange) TargetTableID() string { + return r.targetTable.QuoteString() +} + +// SetIdentifyInfo can be used when caller has calculated and cached +// identityInfo, to avoid every RowChange lazily initialize it. +func (r *RowChange) SetIdentifyInfo(info *schema.DownstreamTableInfo) { + r.identityInfo = info +} + +func (r *RowChange) lazyInitIdentityInfo() { + if r.identityInfo != nil { + return + } + + r.identityInfo = schema.GetDownStreamTI(r.targetTableInfo, r.sourceTableInfo) +} + +func getColsAndValuesOfIdx( + columns []*timodel.ColumnInfo, + indexColumns *timodel.IndexInfo, + data []interface{}, +) ([]*timodel.ColumnInfo, []interface{}) { + cols := make([]*timodel.ColumnInfo, 0, len(indexColumns.Columns)) + values := make([]interface{}, 0, len(indexColumns.Columns)) + for _, col := range indexColumns.Columns { + cols = append(cols, columns[col.Offset]) + values = append(values, data[col.Offset]) + } + + return cols, values +} + +// whereColumnsAndValues returns columns and values to identify the row, to form +// the WHERE clause. +func (r *RowChange) whereColumnsAndValues() ([]string, []interface{}) { + r.lazyInitIdentityInfo() + + uniqueIndex := r.identityInfo.AbsoluteUKIndexInfo + if uniqueIndex == nil { + uniqueIndex = schema.GetIdentityUKByData(r.identityInfo, r.preValues) + } + + columns, values := r.sourceTableInfo.Columns, r.preValues + if uniqueIndex != nil { + columns, values = getColsAndValuesOfIdx(r.sourceTableInfo.Columns, uniqueIndex, values) + } + + columnNames := make([]string, 0, len(columns)) + for _, column := range columns { + columnNames = append(columnNames, column.Name.O) + } + + failpoint.Inject("DownstreamTrackerWhereCheck", func() { + if r.tp == RowChangeUpdate { + log.L().Info("UpdateWhereColumnsCheck", + zap.String("Columns", fmt.Sprintf("%v", columnNames))) + } else if r.tp == RowChangeDelete { + log.L().Info("DeleteWhereColumnsCheck", + zap.String("Columns", fmt.Sprintf("%v", columnNames))) + } + }) + + return columnNames, values +} + +// genWhere generates WHERE clause for UPDATE and DELETE to identify the row. +// the SQL part is written to `buf` and the args part is returned. +func (r *RowChange) genWhere(buf *strings.Builder) []interface{} { + whereColumns, whereValues := r.whereColumnsAndValues() + + for i, col := range whereColumns { + if i != 0 { + buf.WriteString(" AND ") + } + buf.WriteString(quotes.QuoteName(col)) + if whereValues[i] == nil { + buf.WriteString(" IS ?") + } else { + buf.WriteString(" = ?") + } + } + return whereValues +} + +// valuesHolder gens values holder like (?,?,?). +func valuesHolder(n int) string { + var builder strings.Builder + builder.Grow((n-1)*2 + 3) + builder.WriteByte('(') + for i := 0; i < n; i++ { + if i > 0 { + builder.WriteString(",") + } + builder.WriteString("?") + } + builder.WriteByte(')') + return builder.String() +} + +func (r *RowChange) genDeleteSQL() (string, []interface{}) { + if r.tp != RowChangeDelete && r.tp != RowChangeUpdate { + log.L().DPanic("illegal type for genDeleteSQL", + zap.String("sourceTable", r.sourceTable.String()), + zap.Stringer("changeType", r.tp)) + return "", nil + } + + var buf strings.Builder + buf.Grow(1024) + buf.WriteString("DELETE FROM ") + buf.WriteString(r.targetTable.QuoteString()) + buf.WriteString(" WHERE ") + whereArgs := r.genWhere(&buf) + buf.WriteString(" LIMIT 1") + + return buf.String(), whereArgs +} + +func isGenerated(columns []*timodel.ColumnInfo, name timodel.CIStr) bool { + for _, col := range columns { + if col.Name.L == name.L { + return col.IsGenerated() + } + } + return false +} + +func (r *RowChange) genUpdateSQL() (string, []interface{}) { + if r.tp != RowChangeUpdate { + log.L().DPanic("illegal type for genUpdateSQL", + zap.String("sourceTable", r.sourceTable.String()), + zap.Stringer("changeType", r.tp)) + return "", nil + } + + var buf strings.Builder + buf.Grow(2048) + buf.WriteString("UPDATE ") + buf.WriteString(r.targetTable.QuoteString()) + buf.WriteString(" SET ") + + args := make([]interface{}, 0, len(r.preValues)+len(r.postValues)) + writtenFirstCol := false + for i, col := range r.sourceTableInfo.Columns { + if isGenerated(r.targetTableInfo.Columns, col.Name) { + continue + } + + if writtenFirstCol { + buf.WriteString(", ") + } + writtenFirstCol = true + fmt.Fprintf(&buf, "%s = ?", quotes.QuoteName(col.Name.O)) + args = append(args, r.postValues[i]) + } + + buf.WriteString(" WHERE ") + whereArgs := r.genWhere(&buf) + buf.WriteString(" LIMIT 1") + + args = append(args, whereArgs...) + return buf.String(), args +} + +func (r *RowChange) genInsertSQL(tp DMLType) (string, []interface{}) { + return GenInsertSQL(tp, r) +} + +type DMLType int + +// these constants represent types of row change. +const ( + DMLNull DMLType = iota + DMLInsert + DMLReplace + DMLInsertOnDuplicateUpdate + DMLUpdate + DMLDelete +) + +// String implements fmt.Stringer interface. +func (t DMLType) String() string { + switch t { + case DMLInsert: + return "DMLInsert" + case DMLReplace: + return "DMLReplace" + case DMLUpdate: + return "DMLUpdate" + case DMLInsertOnDuplicateUpdate: + return "DMLInsertOnDuplicateUpdate" + case DMLDelete: + return "DMLDelete" + } + + return "" +} + +// GenSQL generated a DML SQL for this RowChange. +func (r *RowChange) GenSQL(tp DMLType) (string, []interface{}) { + switch tp { + case DMLInsert, DMLReplace, DMLInsertOnDuplicateUpdate: + return r.genInsertSQL(tp) + case DMLUpdate: + return r.genUpdateSQL() + case DMLDelete: + return r.genDeleteSQL() + } + log.L().DPanic("illegal type for GenSQL", + zap.String("sourceTable", r.sourceTable.String()), + zap.Stringer("DMLType", tp)) + return "", nil +} diff --git a/pkg/sqlmodel/row_change_test.go b/pkg/sqlmodel/row_change_test.go new file mode 100644 index 00000000000..5d04a71930c --- /dev/null +++ b/pkg/sqlmodel/row_change_test.go @@ -0,0 +1,345 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlmodel + +import ( + "testing" + + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + timodel "github.com/pingcap/tidb/parser/model" + timock "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" +) + +func mockTableInfo(t *testing.T, sql string) *timodel.TableInfo { + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + ti, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + require.NoError(t, err) + return ti +} + +type dpanicSuite struct { + suite.Suite +} + +func (s *dpanicSuite) SetupSuite() { + err := log.InitLogger(&log.Config{Level: "debug"}) + s.NoError(err) +} + +func TestDpanicSuite(t *testing.T) { + suite.Run(t, new(dpanicSuite)) +} + +func TestNewRowChange(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tbl"} + target := &cdcmodel.TableName{Schema: "db", Table: "tbl_routed"} + sourceTI := mockTableInfo(t, "CREATE TABLE tbl (id INT PRIMARY KEY, name INT)") + targetTI := mockTableInfo(t, "CREATE TABLE tbl_routed (id INT PRIMARY KEY, name INT)") + tiSession := utils.NewSessionCtx(map[string]string{ + "time_zone": "+08:00", + }) + + expected := &RowChange{ + sourceTable: source, + targetTable: target, + preValues: []interface{}{1, 2}, + postValues: []interface{}{1, 3}, + sourceTableInfo: sourceTI, + targetTableInfo: targetTI, + tiSessionCtx: tiSession, + tp: RowChangeUpdate, + identityInfo: nil, + } + + actual := NewRowChange(source, target, []interface{}{1, 2}, []interface{}{1, 3}, sourceTI, targetTI, tiSession) + require.Equal(t, expected, actual) + + actual.lazyInitIdentityInfo() + require.NotNil(t, actual.identityInfo) + + // test some arguments of NewRowChange can be nil + + expected.targetTable = expected.sourceTable + expected.targetTableInfo = expected.sourceTableInfo + expected.tiSessionCtx = utils.ZeroSessionCtx + expected.identityInfo = nil + actual = NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{1, 3}, sourceTI, nil, nil) + require.Equal(t, expected, actual) +} + +func (s *dpanicSuite) TestRowChangeType() { + change := &RowChange{preValues: []interface{}{1}} + change.calculateType() + s.Equal(RowChangeDelete, change.tp) + change = &RowChange{preValues: []interface{}{1}, postValues: []interface{}{2}} + change.calculateType() + s.Equal(RowChangeUpdate, change.tp) + change = &RowChange{postValues: []interface{}{1}} + change.calculateType() + s.Equal(RowChangeInsert, change.tp) + + s.Panics(func() { + change = &RowChange{} + change.calculateType() + }) +} + +func (s *dpanicSuite) TestGenDelete() { + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + target := &cdcmodel.TableName{Schema: "db", Table: "tb2"} + + cases := []struct { + sourceCreateSQL string + targetCreateSQL string + preValues []interface{} + + expectedSQL string + expectedArgs []interface{} + }{ + { + "CREATE TABLE tb1 (id INT PRIMARY KEY, name INT)", + "CREATE TABLE tb2 (id INT PRIMARY KEY, name INT, extra VARCHAR(20))", + []interface{}{1, 2}, + + "DELETE FROM `db`.`tb2` WHERE `id` = ? LIMIT 1", + []interface{}{1}, + }, + { + "CREATE TABLE tb1 (c INT, c2 INT UNIQUE)", + "CREATE TABLE tb2 (c INT, c2 INT UNIQUE)", + []interface{}{1, 2}, + + "DELETE FROM `db`.`tb2` WHERE `c2` = ? LIMIT 1", + []interface{}{2}, + }, + // next 2 cases test NULL value + { + "CREATE TABLE tb1 (c INT, c2 INT UNIQUE)", + "CREATE TABLE tb2 (c INT, c2 INT UNIQUE)", + []interface{}{1, nil}, + + "DELETE FROM `db`.`tb2` WHERE `c` = ? AND `c2` IS ? LIMIT 1", + []interface{}{1, nil}, + }, + { + "CREATE TABLE tb1 (c INT, c2 INT)", + "CREATE TABLE tb2 (c INT, c2 INT)", + []interface{}{1, nil}, + + "DELETE FROM `db`.`tb2` WHERE `c` = ? AND `c2` IS ? LIMIT 1", + []interface{}{1, nil}, + }, + // next 2 cases test using downstream table to generate WHERE + { + "CREATE TABLE tb1 (id INT PRIMARY KEY, user_id INT NOT NULL UNIQUE)", + "CREATE TABLE tb2 (new_id INT PRIMARY KEY, id INT, user_id INT NOT NULL UNIQUE)", + []interface{}{1, 2}, + + "DELETE FROM `db`.`tb2` WHERE `user_id` = ? LIMIT 1", + []interface{}{2}, + }, + { + "CREATE TABLE tb1 (id INT PRIMARY KEY, c2 INT)", + "CREATE TABLE tb2 (new_id INT PRIMARY KEY, id INT, c2 INT)", + []interface{}{1, 2}, + + "DELETE FROM `db`.`tb2` WHERE `id` = ? AND `c2` = ? LIMIT 1", + []interface{}{1, 2}, + }, + } + + for _, c := range cases { + sourceTI := mockTableInfo(s.T(), c.sourceCreateSQL) + targetTI := mockTableInfo(s.T(), c.targetCreateSQL) + change := NewRowChange(source, target, c.preValues, nil, sourceTI, targetTI, nil) + sql, args := change.GenSQL(DMLDelete) + s.Equal(c.expectedSQL, sql) + s.Equal(c.expectedArgs, args) + } + + // a RowChangeUpdate can still generate DELETE SQL + sourceTI := mockTableInfo(s.T(), "CREATE TABLE tb1 (id INT PRIMARY KEY, name INT)") + change := NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{3, 4}, sourceTI, nil, nil) + sql, args := change.GenSQL(DMLDelete) + s.Equal("DELETE FROM `db`.`tb1` WHERE `id` = ? LIMIT 1", sql) + s.Equal([]interface{}{1}, args) + + change = NewRowChange(source, nil, nil, []interface{}{3, 4}, sourceTI, nil, nil) + s.Panics(func() { + change.GenSQL(DMLDelete) + }) +} + +func (s *dpanicSuite) TestGenUpdate() { + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + target := &cdcmodel.TableName{Schema: "db", Table: "tb2"} + + cases := []struct { + sourceCreateSQL string + targetCreateSQL string + preValues []interface{} + postValues []interface{} + + expectedSQL string + expectedArgs []interface{} + }{ + { + "CREATE TABLE tb1 (id INT PRIMARY KEY, name INT)", + "CREATE TABLE tb2 (id INT PRIMARY KEY, name INT, extra VARCHAR(20))", + []interface{}{1, 2}, + []interface{}{3, 4}, + + "UPDATE `db`.`tb2` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1", + []interface{}{3, 4, 1}, + }, + { + "CREATE TABLE tb1 (id INT UNIQUE, name INT)", + "CREATE TABLE tb2 (id INT UNIQUE, name INT)", + []interface{}{nil, 2}, + []interface{}{3, 4}, + + "UPDATE `db`.`tb2` SET `id` = ?, `name` = ? WHERE `id` IS ? AND `name` = ? LIMIT 1", + []interface{}{3, 4, nil, 2}, + }, + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)", + "CREATE TABLE tb2 (c INT, c2 INT)", + []interface{}{1, 2}, + []interface{}{3, 4}, + + "UPDATE `db`.`tb2` SET `c` = ?, `c2` = ? WHERE `c` = ? AND `c2` = ? LIMIT 1", + []interface{}{3, 4, 1, 2}, + }, + // next 2 cases test generated column + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT AS (c+1))", + "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT AS (c+1))", + []interface{}{1, 2}, + []interface{}{3, 4}, + + "UPDATE `db`.`tb2` SET `c` = ? WHERE `c` = ? LIMIT 1", + []interface{}{3, 1}, + }, + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT AS (c+1))", + "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT)", + []interface{}{1, 2}, + []interface{}{3, 4}, + + "UPDATE `db`.`tb2` SET `c` = ?, `c2` = ? WHERE `c` = ? LIMIT 1", + []interface{}{3, 4, 1}, + }, + } + + for _, c := range cases { + sourceTI := mockTableInfo(s.T(), c.sourceCreateSQL) + targetTI := mockTableInfo(s.T(), c.targetCreateSQL) + change := NewRowChange(source, target, c.preValues, c.postValues, sourceTI, targetTI, nil) + sql, args := change.GenSQL(DMLUpdate) + s.Equal(c.expectedSQL, sql) + s.Equal(c.expectedArgs, args) + } + + sourceTI := mockTableInfo(s.T(), "CREATE TABLE tb1 (id INT PRIMARY KEY, name INT)") + change := NewRowChange(source, nil, nil, []interface{}{3, 4}, sourceTI, nil, nil) + s.Panics(func() { + change.GenSQL(DMLUpdate) + }) +} + +func TestGenInsert(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb1"} + target := &cdcmodel.TableName{Schema: "db", Table: "tb2"} + + cases := []struct { + sourceCreateSQL string + targetCreateSQL string + postValues []interface{} + + expectedInsertSQL string + expectedReplaceSQL string + expectedInsertOnDupSQL string + expectedArgs []interface{} + }{ + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT)", + "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT, extra VARCHAR(20))", + []interface{}{1, 2}, + + "INSERT INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?)", + "REPLACE INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?)", + "INSERT INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?) ON DUPLICATE KEY UPDATE `c`=VALUES(`c`),`c2`=VALUES(`c2`)", + []interface{}{1, 2}, + }, + // next 2 cases test generated column + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT AS (c+1))", + "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT AS (c+1))", + []interface{}{1, 2}, + + "INSERT INTO `db`.`tb2` (`c`) VALUES (?)", + "REPLACE INTO `db`.`tb2` (`c`) VALUES (?)", + "INSERT INTO `db`.`tb2` (`c`) VALUES (?) ON DUPLICATE KEY UPDATE `c`=VALUES(`c`)", + []interface{}{1}, + }, + { + "CREATE TABLE tb1 (c INT PRIMARY KEY, c2 INT AS (c+1))", + "CREATE TABLE tb2 (c INT PRIMARY KEY, c2 INT)", + []interface{}{1, 2}, + + "INSERT INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?)", + "REPLACE INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?)", + "INSERT INTO `db`.`tb2` (`c`,`c2`) VALUES (?,?) ON DUPLICATE KEY UPDATE `c`=VALUES(`c`),`c2`=VALUES(`c2`)", + []interface{}{1, 2}, + }, + } + + for _, c := range cases { + sourceTI := mockTableInfo(t, c.sourceCreateSQL) + targetTI := mockTableInfo(t, c.targetCreateSQL) + change := NewRowChange(source, target, nil, c.postValues, sourceTI, targetTI, nil) + sql, args := change.GenSQL(DMLInsert) + require.Equal(t, c.expectedInsertSQL, sql) + require.Equal(t, c.expectedArgs, args) + sql, args = change.GenSQL(DMLReplace) + require.Equal(t, c.expectedReplaceSQL, sql) + require.Equal(t, c.expectedArgs, args) + sql, args = change.GenSQL(DMLInsertOnDuplicateUpdate) + require.Equal(t, c.expectedInsertOnDupSQL, sql) + require.Equal(t, c.expectedArgs, args) + } +} + +func TestValuesHolder(t *testing.T) { + t.Parallel() + + require.Equal(t, "()", valuesHolder(0)) + require.Equal(t, "(?)", valuesHolder(1)) + require.Equal(t, "(?,?)", valuesHolder(2)) +} From c4f8055178bfd35e728f145a9c27805b130441af Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 8 Feb 2022 13:25:35 +0800 Subject: [PATCH 40/72] =?UTF-8?q?metrics(ticdc):=20add=20some=20log=20and?= =?UTF-8?q?=20metrics=20to=20owner=20and=20processorManage=E2=80=A6=20(#44?= =?UTF-8?q?02)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit close pingcap/tiflow#3884 --- cdc/owner/changefeed.go | 33 ++++++++++++++++++++++++++++++++- cdc/owner/metrics.go | 29 ++++++++++++++++++++++++++++- cdc/processor/manager.go | 21 +++++++++++++++++---- cdc/processor/metrics.go | 18 ++++++++++++++++++ cdc/processor/processor.go | 12 ++++++++++++ cdc/sink/sink.go | 1 + pkg/orchestrator/interfaces.go | 8 -------- 7 files changed, 108 insertions(+), 14 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 5cead7f9937..c719d761ce4 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -17,6 +17,7 @@ import ( "context" "strings" "sync" + "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -72,6 +73,7 @@ type changefeed struct { metricsChangefeedCheckpointTsLagGauge prometheus.Gauge metricsChangefeedResolvedTsGauge prometheus.Gauge metricsChangefeedResolvedTsLagGauge prometheus.Gauge + metricChangefeedTickDuration prometheus.Observer newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) newSink func() DDLSink @@ -109,12 +111,25 @@ func newChangefeed4Test( } func (c *changefeed) Tick(ctx cdcContext.Context, state *orchestrator.ChangefeedReactorState, captures map[model.CaptureID]*model.CaptureInfo) { + startTime := time.Now() + ctx = cdcContext.WithErrorHandler(ctx, func(err error) error { c.errCh <- errors.Trace(err) return nil }) state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID) - if err := c.tick(ctx, state, captures); err != nil { + err := c.tick(ctx, state, captures) + + // The tick duration is recorded only if changefeed has completed initialization + if c.initialized { + costTime := time.Since(startTime) + if costTime > changefeedLogsWarnDuration { + log.Warn("changefeed tick took too long", zap.String("changefeed", c.id), zap.Duration("duration", costTime)) + } + c.metricChangefeedTickDuration.Observe(costTime.Seconds()) + } + + if err != nil { log.Error("an error occurred in Owner", zap.String("changefeed", c.state.ID), zap.Error(err)) var code string if rfcCode, ok := cerror.RFCCode(err); ok { @@ -185,7 +200,12 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *orchestrator.Changefeed // So we return here. return nil } + startTime := time.Now() newCheckpointTs, newResolvedTs, err := c.scheduler.Tick(ctx, c.state, c.schema.AllPhysicalTables(), captures) + costTime := time.Since(startTime) + if costTime > schedulerLogsWarnDuration { + log.Warn("scheduler tick took too long", zap.String("changefeed", c.id), zap.Duration("duration", costTime)) + } if err != nil { return errors.Trace(err) } @@ -296,6 +316,7 @@ LOOP: c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge.WithLabelValues(c.id) c.metricsChangefeedResolvedTsGauge = changefeedResolvedTsGauge.WithLabelValues(c.id) c.metricsChangefeedResolvedTsLagGauge = changefeedResolvedTsLagGauge.WithLabelValues(c.id) + c.metricChangefeedTickDuration = changefeedTickDuration.WithLabelValues(c.id) // create scheduler c.scheduler, err = c.newScheduler(ctx, checkpointTs) @@ -338,6 +359,9 @@ func (c *changefeed) releaseResources(ctx cdcContext.Context) { c.metricsChangefeedResolvedTsGauge = nil c.metricsChangefeedResolvedTsLagGauge = nil + changefeedTickDuration.DeleteLabelValues(c.id) + c.metricChangefeedTickDuration = nil + c.initialized = false } @@ -548,7 +572,14 @@ func (c *changefeed) updateStatus(currentTs int64, checkpointTs, resolvedTs mode } func (c *changefeed) Close(ctx cdcContext.Context) { + startTime := time.Now() + c.releaseResources(ctx) + costTime := time.Since(startTime) + if costTime > changefeedLogsWarnDuration { + log.Warn("changefeed close took too long", zap.String("changefeed", c.id), zap.Duration("duration", costTime)) + } + changefeedCloseDuration.Observe(costTime.Seconds()) } func (c *changefeed) GetInfoProvider() schedulerv2.InfoProvider { diff --git a/cdc/owner/metrics.go b/cdc/owner/metrics.go index bc1bff8d0aa..fa29fec9e84 100644 --- a/cdc/owner/metrics.go +++ b/cdc/owner/metrics.go @@ -13,7 +13,11 @@ package owner -import "github.com/prometheus/client_golang/prometheus" +import ( + "time" + + "github.com/prometheus/client_golang/prometheus" +) var ( changefeedCheckpointTsGauge = prometheus.NewGaugeVec( @@ -65,6 +69,22 @@ var ( Name: "status", Help: "The status of changefeeds", }, []string{"changefeed"}) + changefeedTickDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "changefeed_tick_duration", + Help: "Bucketed histogram of owner tick changefeed reactor time (s).", + Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), + }, []string{"changefeed"}) + changefeedCloseDuration = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "changefeed_close_duration", + Help: "Bucketed histogram of owner close changefeed reactor time (s).", + Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), + }) ) const ( @@ -72,6 +92,11 @@ const ( maintainTableTypeTotal string = "total" // tables that are dispatched to a processor and have not been finished yet maintainTableTypeWip string = "wip" + // When heavy operations (such as network IO and serialization) take too much time, the program + // should print a warning log, and if necessary, the timeout should be exposed externally through + // monitor. + changefeedLogsWarnDuration = 1 * time.Second + schedulerLogsWarnDuration = 1 * time.Second ) // InitMetrics registers all metrics used in owner @@ -83,4 +108,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(ownershipCounter) registry.MustRegister(ownerMaintainTableNumGauge) registry.MustRegister(changefeedStatusGauge) + registry.MustRegister(changefeedTickDuration) + registry.MustRegister(changefeedCloseDuration) } diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 935e484bb0e..e386f6cc995 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -27,6 +27,7 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -36,6 +37,7 @@ const ( commandTpUnknow commandTp = iota //nolint:varcheck,deadcode commandTpClose commandTpWriteDebugInfo + processorLogsWarnDuration = 1 * time.Second ) type command struct { @@ -53,16 +55,19 @@ type Manager struct { newProcessor func(cdcContext.Context) *processor enableNewScheduler bool + + metricProcessorCloseDuration prometheus.Observer } // NewManager creates a new processor manager func NewManager() *Manager { conf := config.GetGlobalServerConfig() return &Manager{ - processors: make(map[model.ChangeFeedID]*processor), - commandQueue: make(chan *command, 4), - newProcessor: newProcessor, - enableNewScheduler: conf.Debug.EnableNewScheduler, + processors: make(map[model.ChangeFeedID]*processor), + commandQueue: make(chan *command, 4), + newProcessor: newProcessor, + enableNewScheduler: conf.Debug.EnableNewScheduler, + metricProcessorCloseDuration: processorCloseDuration.WithLabelValues(conf.AdvertiseAddr), } } @@ -129,7 +134,15 @@ func (m *Manager) Tick(stdCtx context.Context, state orchestrator.ReactorState) func (m *Manager) closeProcessor(changefeedID model.ChangeFeedID) { if processor, exist := m.processors[changefeedID]; exist { + startTime := time.Now() + captureID := processor.captureInfo.ID err := processor.Close() + costTime := time.Since(startTime) + if costTime > processorLogsWarnDuration { + log.Warn("processor close took too long", zap.String("changefeed", changefeedID), + zap.String("capture", captureID), zap.Duration("duration", costTime)) + } + m.metricProcessorCloseDuration.Observe(costTime.Seconds()) if err != nil { log.Warn("failed to close processor", zap.String("changefeed", changefeedID), diff --git a/cdc/processor/metrics.go b/cdc/processor/metrics.go index 5c52b0ccba6..9875929b8d0 100644 --- a/cdc/processor/metrics.go +++ b/cdc/processor/metrics.go @@ -81,6 +81,22 @@ var ( Name: "schema_storage_gc_ts", Help: "the TS of the currently maintained oldest snapshot in SchemaStorage", }, []string{"changefeed", "capture"}) + processorTickDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "processor", + Name: "processor_tick_duration", + Help: "Bucketed histogram of processorManager tick processor time (s).", + Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), + }, []string{"changefeed", "capture"}) + processorCloseDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "processor", + Name: "processor_close_duration", + Help: "Bucketed histogram of processorManager close processor time (s).", + Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), + }, []string{"capture"}) ) // InitMetrics registers all metrics used in processor @@ -94,4 +110,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(syncTableNumGauge) registry.MustRegister(processorErrorCounter) registry.MustRegister(processorSchemaStorageGcTsGauge) + registry.MustRegister(processorTickDuration) + registry.MustRegister(processorCloseDuration) } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index b66f7831089..53199727e5b 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -92,6 +92,7 @@ type processor struct { metricSyncTableNumGauge prometheus.Gauge metricSchemaStorageGcTsGauge prometheus.Gauge metricProcessorErrorCounter prometheus.Counter + metricProcessorTickDuration prometheus.Observer } // checkReadyForMessages checks whether all necessary Etcd keys have been established. @@ -246,6 +247,7 @@ func newProcessor(ctx cdcContext.Context) *processor { metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, advertiseAddr), metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, advertiseAddr), metricSchemaStorageGcTsGauge: processorSchemaStorageGcTsGauge.WithLabelValues(changefeedID, advertiseAddr), + metricProcessorTickDuration: processorTickDuration.WithLabelValues(changefeedID, advertiseAddr), } p.createTablePipeline = p.createTablePipelineImpl p.lazyInit = p.lazyInitImpl @@ -280,6 +282,7 @@ func isProcessorIgnorableError(err error) bool { // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // The main logic of processor is in this function, including the calculation of many kinds of ts, maintain table pipeline, error handling, etc. func (p *processor) Tick(ctx cdcContext.Context, state *orchestrator.ChangefeedReactorState) (orchestrator.ReactorState, error) { + startTime := time.Now() p.changefeed = state state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID) ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ @@ -287,6 +290,14 @@ func (p *processor) Tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR Info: state.Info, }) _, err := p.tick(ctx, state) + + costTime := time.Since(startTime) + if costTime > processorLogsWarnDuration { + log.Warn("processor tick took too long", zap.String("changefeed", p.changefeedID), + zap.String("capture", ctx.GlobalVars().CaptureInfo.ID), zap.Duration("duration", costTime)) + } + p.metricProcessorTickDuration.Observe(costTime.Seconds()) + if err == nil { return state, nil } @@ -1058,6 +1069,7 @@ func (p *processor) Close() error { syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + processorTickDuration.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) if p.sinkManager != nil { // pass a canceled context is ok here, since we don't need to wait Close ctx, cancel := context.WithCancel(context.Background()) diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index e1254cc5ee1..c557c6e8a46 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -41,6 +41,7 @@ type Sink interface { // TryEmitRowChangedEvents is thread-safety and non-blocking. TryEmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) (bool, error) + // EmitDDLEvent sends DDL Event to Sink // EmitDDLEvent should execute DDL to downstream synchronously // diff --git a/pkg/orchestrator/interfaces.go b/pkg/orchestrator/interfaces.go index 5e74fab592b..20452cd6f26 100644 --- a/pkg/orchestrator/interfaces.go +++ b/pkg/orchestrator/interfaces.go @@ -67,11 +67,3 @@ func (s *SingleDataPatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map } return nil } - -// MultiDataPatch represents an update to many keys -type MultiDataPatch func(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error - -// Patch implements the DataPatch interface -func (m MultiDataPatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error { - return m(valueMap, changedSet) -} From 5459951cc902021092cbc3ce5f9c2bbc2c4e9bf4 Mon Sep 17 00:00:00 2001 From: WizardXiao <89761062+WizardXiao@users.noreply.github.com> Date: Tue, 8 Feb 2022 14:19:35 +0800 Subject: [PATCH 41/72] syncer(dm): enhance error handling about add inject at dml event (#4259) close pingcap/tiflow#4260 --- dm/syncer/err-operator/operator.go | 47 +++++++++---------------- dm/syncer/err-operator/operator_test.go | 26 +++++--------- dm/syncer/handle_error.go | 2 +- dm/syncer/syncer.go | 36 +++++++++---------- dm/tests/handle_error/run.sh | 11 +++--- 5 files changed, 49 insertions(+), 73 deletions(-) diff --git a/dm/syncer/err-operator/operator.go b/dm/syncer/err-operator/operator.go index 2ce26470ef9..e99e57c4e44 100644 --- a/dm/syncer/err-operator/operator.go +++ b/dm/syncer/err-operator/operator.go @@ -34,11 +34,10 @@ import ( // Operator contains an operation for specified binlog pos // used by `handle-error`. type Operator struct { - uuid string // add a UUID, make it more friendly to be traced in log - op pb.ErrorOp - isAllInjected bool // is all injected, used by inject - events []*replication.BinlogEvent // startLocation -> events - originReq *pb.HandleWorkerErrorRequest + uuid string // add a UUID, make it more friendly to be traced in log + op pb.ErrorOp + events []*replication.BinlogEvent // ddls -> events + originReq *pb.HandleWorkerErrorRequest } // newOperator creates a new operator with a random UUID. @@ -130,18 +129,6 @@ func (h *Holder) GetBehindCommands(pos string) []*pb.HandleWorkerErrorRequest { return res } -func (h *Holder) SetHasAllInjected(startLocation binlog.Location) { - h.mu.Lock() - defer h.mu.Unlock() - - key := startLocation.Position.String() - operator, ok := h.operators[key] - if !ok { - return - } - operator.isAllInjected = true -} - func (h *Holder) IsInject(startLocation binlog.Location) bool { h.mu.RLock() defer h.mu.RUnlock() @@ -186,7 +173,7 @@ func (h *Holder) GetEvent(startLocation binlog.Location) (*replication.BinlogEve } // MatchAndApply tries to match operation for event by location and apply it on replace events. -func (h *Holder) MatchAndApply(startLocation, endLocation binlog.Location, realEventHeaderTS uint32) (bool, pb.ErrorOp) { +func (h *Holder) MatchAndApply(startLocation, endLocation binlog.Location, currentEvent *replication.BinlogEvent) (bool, pb.ErrorOp) { h.mu.Lock() defer h.mu.Unlock() @@ -196,10 +183,6 @@ func (h *Holder) MatchAndApply(startLocation, endLocation binlog.Location, realE return false, pb.ErrorOp_InvalidErrorOp } - if operator.isAllInjected { - return false, pb.ErrorOp_InvalidErrorOp - } - if operator.op == pb.ErrorOp_Replace || operator.op == pb.ErrorOp_Inject { if len(operator.events) == 0 { // this should not happen @@ -209,7 +192,7 @@ func (h *Holder) MatchAndApply(startLocation, endLocation binlog.Location, realE // set LogPos as start position for _, ev := range operator.events { ev.Header.LogPos = startLocation.Position.Pos - ev.Header.Timestamp = realEventHeaderTS + ev.Header.Timestamp = currentEvent.Header.Timestamp if e, ok := ev.Event.(*replication.QueryEvent); ok { if startLocation.GetGTID() != nil { e.GSet = startLocation.GetGTID().Origin() @@ -217,14 +200,18 @@ func (h *Holder) MatchAndApply(startLocation, endLocation binlog.Location, realE } } - // set the last replace event as end position - e := operator.events[len(operator.events)-1] - e.Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos - e.Header.LogPos = endLocation.Position.Pos - if e, ok := e.Event.(*replication.QueryEvent); ok { - if endLocation.GetGTID() != nil { - e.GSet = endLocation.GetGTID().Origin() + if operator.op == pb.ErrorOp_Replace { + // set the last replace event as end position + e := operator.events[len(operator.events)-1] + e.Header.EventSize = endLocation.Position.Pos - startLocation.Position.Pos + e.Header.LogPos = endLocation.Position.Pos + if e, ok := e.Event.(*replication.QueryEvent); ok { + if endLocation.GetGTID() != nil { + e.GSet = endLocation.GetGTID().Origin() + } } + } else if operator.op == pb.ErrorOp_Inject { + operator.events = append(operator.events, currentEvent) } } diff --git a/dm/syncer/err-operator/operator_test.go b/dm/syncer/err-operator/operator_test.go index 929e91b113b..8dba3cccf63 100644 --- a/dm/syncer/err-operator/operator_test.go +++ b/dm/syncer/err-operator/operator_test.go @@ -86,14 +86,14 @@ func (o *testOperatorSuite) TestOperator(c *C) { // skip event err = h.Set(&pb.HandleWorkerErrorRequest{Op: pb.ErrorOp_Skip, BinlogPos: startLocation.Position.String()}, nil) c.Assert(err, IsNil) - apply, op := h.MatchAndApply(startLocation, endLocation, event1.Header.Timestamp) + apply, op := h.MatchAndApply(startLocation, endLocation, event1) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Skip) // overwrite operator err = h.Set(&pb.HandleWorkerErrorRequest{Op: pb.ErrorOp_Replace, BinlogPos: startLocation.Position.String()}, []*replication.BinlogEvent{event1, event2}) c.Assert(err, IsNil) - apply, op = h.MatchAndApply(startLocation, endLocation, event2.Header.Timestamp) + apply, op = h.MatchAndApply(startLocation, endLocation, event2) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) @@ -126,7 +126,7 @@ func (o *testOperatorSuite) TestOperator(c *C) { // revert exist operator err = h.Set(&pb.HandleWorkerErrorRequest{Op: pb.ErrorOp_Revert, BinlogPos: startLocation.Position.String()}, nil) c.Assert(err, IsNil) - apply, op = h.MatchAndApply(startLocation, endLocation, event1.Header.Timestamp) + apply, op = h.MatchAndApply(startLocation, endLocation, event1) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) @@ -139,17 +139,17 @@ func (o *testOperatorSuite) TestOperator(c *C) { // test removeOutdated flushLocation := startLocation c.Assert(h.RemoveOutdated(flushLocation), IsNil) - apply, op = h.MatchAndApply(startLocation, endLocation, event1.Header.Timestamp) + apply, op = h.MatchAndApply(startLocation, endLocation, event1) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) flushLocation = endLocation c.Assert(h.RemoveOutdated(flushLocation), IsNil) - apply, op = h.MatchAndApply(startLocation, endLocation, event1.Header.Timestamp) + apply, op = h.MatchAndApply(startLocation, endLocation, event1) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) - apply, op = h.MatchAndApply(endLocation, nextLocation, event1.Header.Timestamp) + apply, op = h.MatchAndApply(endLocation, nextLocation, event1) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Replace) } @@ -211,20 +211,10 @@ func (o *testOperatorSuite) TestInjectOperator(c *C) { c.Assert(isInject, IsFalse) // test MatchAndApply - apply, op := h.MatchAndApply(startLocation, endLocation, event2.Header.Timestamp) + apply, op := h.MatchAndApply(startLocation, endLocation, event2) c.Assert(apply, IsTrue) c.Assert(op, Equals, pb.ErrorOp_Inject) - apply, op = h.MatchAndApply(endLocation, nextLocation, event2.Header.Timestamp) - c.Assert(apply, IsFalse) - c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) - - // test set all injected - h.SetHasAllInjected(endLocation) - apply, op = h.MatchAndApply(startLocation, endLocation, event2.Header.Timestamp) - c.Assert(apply, IsTrue) - c.Assert(op, Equals, pb.ErrorOp_Inject) - h.SetHasAllInjected(startLocation) - apply, op = h.MatchAndApply(startLocation, endLocation, event2.Header.Timestamp) + apply, op = h.MatchAndApply(endLocation, nextLocation, event2) c.Assert(apply, IsFalse) c.Assert(op, Equals, pb.ErrorOp_InvalidErrorOp) } diff --git a/dm/syncer/handle_error.go b/dm/syncer/handle_error.go index 5103974aafa..413115ea171 100644 --- a/dm/syncer/handle_error.go +++ b/dm/syncer/handle_error.go @@ -39,7 +39,7 @@ func (s *Syncer) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReque return "", fmt.Errorf("source '%s' has no error", s.cfg.SourceID) } - if !isQueryEvent { + if !isQueryEvent && req.Op != pb.ErrorOp_Inject { return "", fmt.Errorf("only support to handle ddl error currently, see https://docs.pingcap.com/tidb-data-migration/stable/error-handling for other errors") } pos = startLocation.Position.String() diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 0c0ca3d9392..c1e9be052b6 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1877,11 +1877,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Debug("receive binlog event", zap.Reflect("header", e.Header)) - // TODO: support all event + // support QueryEvent and RowsEvent // we calculate startLocation and endLocation(currentLocation) for Query event here // set startLocation empty for other events to avoid misuse startLocation = binlog.Location{} - if ev, ok := e.Event.(*replication.QueryEvent); ok { + switch ev := e.Event.(type) { + case *replication.QueryEvent, *replication.RowsEvent: startLocation = binlog.InitLocation( mysql.Position{ Name: lastLocation.Position.Name, @@ -1904,19 +1905,22 @@ func (s *Syncer) Run(ctx context.Context) (err error) { ) currentLocation.Suffix = endSuffix - err = currentLocation.SetGTID(ev.GSet) - if err != nil { - return terror.Annotatef(err, "fail to record GTID %v", ev.GSet) + if queryEvent, ok := ev.(*replication.QueryEvent); ok { + err = currentLocation.SetGTID(queryEvent.GSet) + if err != nil { + return terror.Annotatef(err, "fail to record GTID %v", queryEvent.GSet) + } } if !s.isReplacingOrInjectingErr { - apply, op := s.errOperatorHolder.MatchAndApply(startLocation, currentLocation, e.Header.Timestamp) + apply, op := s.errOperatorHolder.MatchAndApply(startLocation, currentLocation, e) if apply { if op == pb.ErrorOp_Replace || op == pb.ErrorOp_Inject { s.isReplacingOrInjectingErr = true // revert currentLocation to startLocation currentLocation = startLocation } else if op == pb.ErrorOp_Skip { + queryEvent := ev.(*replication.QueryEvent) ec := eventContext{ tctx: tctx, header: e.Header, @@ -1925,9 +1929,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { lastLocation: &lastLocation, } var sourceTbls map[string]map[string]struct{} - sourceTbls, err = s.trackOriginDDL(ev, ec) + sourceTbls, err = s.trackOriginDDL(queryEvent, ec) if err != nil { - tctx.L().Warn("failed to track query when handle-error skip", zap.Error(err), zap.ByteString("sql", ev.Query)) + tctx.L().Warn("failed to track query when handle-error skip", zap.Error(err), zap.ByteString("sql", queryEvent.Query)) } s.saveGlobalPoint(currentLocation) @@ -1951,21 +1955,15 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } } // set endLocation.Suffix=0 of last replace or inject event - // also redirect stream to next event if currentLocation.Suffix > 0 && e.Header.EventSize > 0 { currentLocation.Suffix = 0 s.isReplacingOrInjectingErr = false s.locations.reset(currentLocation) - if s.errOperatorHolder.IsInject(startLocation) { - s.errOperatorHolder.SetHasAllInjected(startLocation) - // reset event as startLocation, avoid to be marked in checkpoint - currentLocation.Position.Pos = startLocation.Position.Pos - err = s.streamerController.RedirectStreamer(tctx, startLocation) - } else { - err = s.streamerController.RedirectStreamer(tctx, currentLocation) - } - if err != nil { - return err + if !s.errOperatorHolder.IsInject(startLocation) { + // replace operator need redirect to currentLocation + if err = s.streamerController.RedirectStreamer(tctx, currentLocation); err != nil { + return err + } } } } diff --git a/dm/tests/handle_error/run.sh b/dm/tests/handle_error/run.sh index 4e6b43ddbad..3b4417544e6 100644 --- a/dm/tests/handle_error/run.sh +++ b/dm/tests/handle_error/run.sh @@ -365,17 +365,18 @@ function DM_INJECT_DML_ERROR_CASE() { "binlog inject test alter table ${db}.${tb1} drop index b;alter table ${db}.${tb1} add unique(c);" \ "\"result\": true" 2 - run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where d = 2;" "count(1): 1" + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where b = 2;" "count(1): 2" } function DM_INJECT_ERROR() { + # inject at ddl run_case INJECT_DDL_ERROR "single-source-no-sharding" \ "run_sql_source1 \"create table ${db}.${tb1} (a int unique, b int);\"" \ "clean_table" "" - # TODO inject dml, because get dml error position is not supported. - # run_case INJECT_DML_ERROR "single-source-no-sharding" \ - # "run_sql_source1 \"create table ${db}.${tb1} (a int unique, b varchar(10));\"" \ - # "clean_table" "" + # inject at dml + run_case INJECT_DML_ERROR "single-source-no-sharding" \ + "run_sql_source1 \"create table ${db}.${tb1} (a int unique, b varchar(10));\"" \ + "clean_table" "" } function DM_LIST_ERROR_CASE() { From da0dda8898004123c31fdc41efa2d25172f70477 Mon Sep 17 00:00:00 2001 From: niubell Date: Tue, 8 Feb 2022 15:53:35 +0800 Subject: [PATCH 42/72] master(dm): add cluster id as unique identification of dm-cluster and optimize error message of code:30008 (#4493) close pingcap/tiflow#3830, close pingcap/tiflow#4370, close pingcap/tiflow#4456 --- dm/_utils/terror_gen/errors_release.txt | 3 +- dm/dm/common/common.go | 3 + dm/dm/master/election.go | 6 + dm/dm/master/election_test.go | 4 + dm/dm/master/openapi.go | 7 + dm/dm/master/openapi_test.go | 9 ++ dm/dm/master/server.go | 43 ++++++ dm/errors.toml | 8 +- dm/openapi/gen.client.go | 101 +++++++++++++ dm/openapi/gen.server.go | 185 +++++++++++++----------- dm/openapi/gen.types.go | 6 + dm/openapi/spec/dm.yaml | 22 +++ dm/pkg/retry/strategy.go | 4 + dm/pkg/terror/error_list.go | 4 +- 14 files changed, 317 insertions(+), 88 deletions(-) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 2697655c99c..1d732811c0b 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -212,7 +212,7 @@ ErrRelayBinlogNameNotValid,[code=30004:class=relay-unit:scope=internal:level=hig ErrRelayNoCurrentUUID,[code=30005:class=relay-unit:scope=internal:level=high], "Message: no current UUID set" ErrRelayFlushLocalMeta,[code=30006:class=relay-unit:scope=internal:level=high], "Message: flush local meta" ErrRelayUpdateIndexFile,[code=30007:class=relay-unit:scope=internal:level=high], "Message: update UUID index file %s" -ErrRelayLogDirpathEmpty,[code=30008:class=relay-unit:scope=internal:level=high], "Message: dirpath is empty, Workaround: Please check the `relay-dir` config in source config file." +ErrRelayLogDirpathEmpty,[code=30008:class=relay-unit:scope=internal:level=high], "Message: dirpath is empty, Workaround: Please check the `relay-dir` config in source config file or dm-worker config file." ErrRelayReaderNotStateNew,[code=30009:class=relay-unit:scope=internal:level=high], "Message: stage %s, expect %s, already started" ErrRelayReaderStateCannotClose,[code=30010:class=relay-unit:scope=internal:level=high], "Message: stage %s, expect %s, can not close" ErrRelayReaderNeedStart,[code=30011:class=relay-unit:scope=internal:level=high], "Message: stage %s, expect %s" @@ -396,6 +396,7 @@ ErrMasterFailToImportFromV10x,[code=38053:class=dm-master:scope=internal:level=h ErrMasterInconsistentOptimisticDDLsAndInfo,[code=38054:class=dm-master:scope=internal:level=high], "Message: inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d" ErrMasterOptimisticTableInfoBeforeNotExist,[code=38055:class=dm-master:scope=internal:level=high], "Message: table-info-before not exist in optimistic ddls: %v" ErrMasterOptimisticDownstreamMetaNotFound,[code=38056:class=dm-master:scope=internal:level=high], "Message: downstream database config and meta for task %s not found" +ErrMasterInvalidClusterID,[code=38057:class=dm-master:scope=internal:level=high], "Message: invalid cluster id: %v" ErrWorkerParseFlagSet,[code=40001:class=dm-worker:scope=internal:level=medium], "Message: parse dm-worker config flag set" ErrWorkerInvalidFlag,[code=40002:class=dm-worker:scope=internal:level=medium], "Message: '%s' is an invalid flag" ErrWorkerDecodeConfigFromFile,[code=40003:class=dm-worker:scope=internal:level=medium], "Message: toml decode file, Workaround: Please check the configuration file has correct TOML format." diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index a447c8ee373..ab4cecfd67a 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -26,6 +26,9 @@ var ( useOfClosedErrMsg = "use of closed network connection" // ClusterVersionKey is used to store the version of the cluster. ClusterVersionKey = "/dm-cluster/version" + // ClusterIDKey is used to store the cluster id of the whole dm cluster. Cluster id is the unique identification of dm cluster + // After leader of dm master bootstraped, the leader will get the id from etcd or generate fresh one, and backfill to etcd. + ClusterIDKey = "/dm-cluster/id" // WorkerRegisterKeyAdapter is used to encode and decode register key. // k/v: Encode(worker-name) -> the information of the DM-worker node. WorkerRegisterKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-worker/r/") diff --git a/dm/dm/master/election.go b/dm/dm/master/election.go index d6bd3989c42..8d9e59ebf40 100644 --- a/dm/dm/master/election.go +++ b/dm/dm/master/election.go @@ -188,6 +188,12 @@ func (s *Server) startLeaderComponent(ctx context.Context) bool { return false } + err = s.initClusterID(ctx) + if err != nil { + log.L().Error("init cluster id failed", zap.Error(err)) + return false + } + failpoint.Inject("FailToStartLeader", func(val failpoint.Value) { masterStrings := val.(string) if strings.Contains(masterStrings, s.cfg.Name) { diff --git a/dm/dm/master/election_test.go b/dm/dm/master/election_test.go index 11a6c62aa8e..a831726accd 100644 --- a/dm/dm/master/election_test.go +++ b/dm/dm/master/election_test.go @@ -76,6 +76,8 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { _, leaderID, _, err := s2.election.LeaderInfo(ctx) c.Assert(err, check.IsNil) c.Assert(leaderID, check.Equals, cfg1.Name) + c.Assert(s1.ClusterID(), check.Greater, uint64(0)) + c.Assert(s2.ClusterID(), check.Equals, uint64(0)) // fail to start scheduler/pessimism/optimism c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/FailToStartLeader", `return("dm-master-2")`), check.IsNil) @@ -89,6 +91,7 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { _, leaderID, _, err = s2.election.LeaderInfo(ctx) c.Assert(err, check.IsNil) c.Assert(leaderID, check.Equals, cfg1.Name) + clusterID := s1.ClusterID() //nolint:errcheck failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/FailToStartLeader") @@ -99,6 +102,7 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { _, leaderID, _, err = s2.election.LeaderInfo(ctx) c.Assert(err, check.IsNil) c.Assert(leaderID, check.Equals, cfg2.Name) + c.Assert(clusterID, check.Equals, s2.ClusterID()) cancel() } diff --git a/dm/dm/master/openapi.go b/dm/dm/master/openapi.go index 41ce4e81165..ada0dee4700 100644 --- a/dm/dm/master/openapi.go +++ b/dm/dm/master/openapi.go @@ -901,6 +901,13 @@ func (s *Server) DMAPUpdateTaskTemplate(c *gin.Context, taskName string) { c.IndentedJSON(http.StatusOK, task) } +// DMAPIGetClusterInfo return cluster id of dm cluster. +func (s *Server) DMAPIGetClusterInfo(c *gin.Context) { + r := &openapi.GetClusterInfoResponse{} + r.ClusterId = s.ClusterID() + c.IndentedJSON(http.StatusOK, r) +} + func terrorHTTPErrorHandler() gin.HandlerFunc { return func(c *gin.Context) { c.Next() diff --git a/dm/dm/master/openapi_test.go b/dm/dm/master/openapi_test.go index a57f8b35521..3be3e748246 100644 --- a/dm/dm/master/openapi_test.go +++ b/dm/dm/master/openapi_test.go @@ -632,6 +632,15 @@ func (t *openAPISuite) TestClusterAPI(c *check.C) { c.Assert(resultMasters.Data[0].Leader, check.IsTrue) c.Assert(resultMasters.Data[0].Alive, check.IsTrue) + // check cluster id + clusterIDURL := baseURL + "info" + resp := testutil.NewRequest().Get(clusterIDURL).GoWithHTTPHandler(t.testT, s1.openapiHandles) + c.Assert(resp.Code(), check.Equals, http.StatusOK) + var clusterIDResp openapi.GetClusterInfoResponse + err = resp.UnmarshalBodyToObject(&clusterIDResp) + c.Assert(err, check.IsNil) + c.Assert(clusterIDResp.ClusterId, check.Greater, uint64(0)) + // offline master-2 with retry // operate etcd cluster may met `etcdserver: unhealthy cluster`, add some retry for i := 0; i < 20; i++ { diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 48691c15c8e..310a6836d7a 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -15,7 +15,9 @@ package master import ( "context" + "encoding/binary" "fmt" + "math/rand" "net" "net/http" "reflect" @@ -121,6 +123,8 @@ type Server struct { closed atomic.Bool openapiHandles *gin.Engine // injected in `InitOpenAPIHandles` + + clusterID atomic.Uint64 } // NewServer creates a new Server. @@ -405,6 +409,45 @@ func subtaskCfgPointersToInstances(stCfgPointers ...*config.SubTaskConfig) []con return stCfgs } +func (s *Server) initClusterID(ctx context.Context) error { + log.L().Info("init cluster id begin") + ctx1, cancel := context.WithTimeout(ctx, etcdutil.DefaultRequestTimeout) + defer cancel() + + resp, err := s.etcdClient.Get(ctx1, dmcommon.ClusterIDKey) + if err != nil { + return err + } + + // New cluster, generate a cluster id and backfill it to etcd + if len(resp.Kvs) == 0 { + ts := uint64(time.Now().Unix()) + clusterID := (ts << 32) + uint64(rand.Uint32()) + clusterIDBytes := make([]byte, 8) + binary.BigEndian.PutUint64(clusterIDBytes, clusterID) + _, err = s.etcdClient.Put(ctx1, dmcommon.ClusterIDKey, string(clusterIDBytes)) + if err != nil { + return err + } + s.clusterID.Store(clusterID) + log.L().Info("generate and init cluster id success", zap.Uint64("cluster_id", s.clusterID.Load())) + return nil + } + + if len(resp.Kvs[0].Value) != 8 { + return terror.ErrMasterInvalidClusterID.Generate(resp.Kvs[0].Value) + } + + s.clusterID.Store(binary.BigEndian.Uint64(resp.Kvs[0].Value)) + log.L().Info("init cluster id success", zap.Uint64("cluster_id", s.clusterID.Load())) + return nil +} + +// ClusterID return correct cluster id when as leader. +func (s *Server) ClusterID() uint64 { + return s.clusterID.Load() +} + // StartTask implements MasterServer.StartTask. func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.StartTaskResponse, error) { var ( diff --git a/dm/errors.toml b/dm/errors.toml index 96bab495cf8..beee203ee50 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1285,7 +1285,7 @@ tags = ["internal", "high"] [error.DM-relay-unit-30008] message = "dirpath is empty" description = "" -workaround = "Please check the `relay-dir` config in source config file." +workaround = "Please check the `relay-dir` config in source config file or dm-worker config file." tags = ["internal", "high"] [error.DM-relay-unit-30009] @@ -2386,6 +2386,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38057] +message = "invalid cluster id: %v" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-worker-40001] message = "parse dm-worker config flag set" description = "" diff --git a/dm/openapi/gen.client.go b/dm/openapi/gen.client.go index 3f1a57a84c5..f8f9c45317c 100644 --- a/dm/openapi/gen.client.go +++ b/dm/openapi/gen.client.go @@ -90,6 +90,9 @@ func WithRequestEditorFn(fn RequestEditorFn) ClientOption { // The interface specification for the client above. type ClientInterface interface { + // DMAPIGetClusterInfo request + DMAPIGetClusterInfo(ctx context.Context, reqEditors ...RequestEditorFn) (*http.Response, error) + // DMAPIGetClusterMasterList request DMAPIGetClusterMasterList(ctx context.Context, reqEditors ...RequestEditorFn) (*http.Response, error) @@ -213,6 +216,18 @@ type ClientInterface interface { DMAPIGetTaskStatus(ctx context.Context, taskName string, params *DMAPIGetTaskStatusParams, reqEditors ...RequestEditorFn) (*http.Response, error) } +func (c *Client) DMAPIGetClusterInfo(ctx context.Context, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewDMAPIGetClusterInfoRequest(c.Server) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + func (c *Client) DMAPIGetClusterMasterList(ctx context.Context, reqEditors ...RequestEditorFn) (*http.Response, error) { req, err := NewDMAPIGetClusterMasterListRequest(c.Server) if err != nil { @@ -741,6 +756,33 @@ func (c *Client) DMAPIGetTaskStatus(ctx context.Context, taskName string, params return c.Client.Do(req) } +// NewDMAPIGetClusterInfoRequest generates requests for DMAPIGetClusterInfo +func NewDMAPIGetClusterInfoRequest(server string) (*http.Request, error) { + var err error + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/api/v1/cluster/info") + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("GET", queryURL.String(), nil) + if err != nil { + return nil, err + } + + return req, nil +} + // NewDMAPIGetClusterMasterListRequest generates requests for DMAPIGetClusterMasterList func NewDMAPIGetClusterMasterListRequest(server string) (*http.Request, error) { var err error @@ -2174,6 +2216,9 @@ func WithBaseURL(baseURL string) ClientOption { // ClientWithResponsesInterface is the interface specification for the client with responses above. type ClientWithResponsesInterface interface { + // DMAPIGetClusterInfo request + DMAPIGetClusterInfoWithResponse(ctx context.Context, reqEditors ...RequestEditorFn) (*DMAPIGetClusterInfoResponse, error) + // DMAPIGetClusterMasterList request DMAPIGetClusterMasterListWithResponse(ctx context.Context, reqEditors ...RequestEditorFn) (*DMAPIGetClusterMasterListResponse, error) @@ -2297,6 +2342,28 @@ type ClientWithResponsesInterface interface { DMAPIGetTaskStatusWithResponse(ctx context.Context, taskName string, params *DMAPIGetTaskStatusParams, reqEditors ...RequestEditorFn) (*DMAPIGetTaskStatusResponse, error) } +type DMAPIGetClusterInfoResponse struct { + Body []byte + HTTPResponse *http.Response + JSON200 *GetClusterInfoResponse +} + +// Status returns HTTPResponse.Status +func (r DMAPIGetClusterInfoResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r DMAPIGetClusterInfoResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + type DMAPIGetClusterMasterListResponse struct { Body []byte HTTPResponse *http.Response @@ -3061,6 +3128,15 @@ func (r DMAPIGetTaskStatusResponse) StatusCode() int { return 0 } +// DMAPIGetClusterInfoWithResponse request returning *DMAPIGetClusterInfoResponse +func (c *ClientWithResponses) DMAPIGetClusterInfoWithResponse(ctx context.Context, reqEditors ...RequestEditorFn) (*DMAPIGetClusterInfoResponse, error) { + rsp, err := c.DMAPIGetClusterInfo(ctx, reqEditors...) + if err != nil { + return nil, err + } + return ParseDMAPIGetClusterInfoResponse(rsp) +} + // DMAPIGetClusterMasterListWithResponse request returning *DMAPIGetClusterMasterListResponse func (c *ClientWithResponses) DMAPIGetClusterMasterListWithResponse(ctx context.Context, reqEditors ...RequestEditorFn) (*DMAPIGetClusterMasterListResponse, error) { rsp, err := c.DMAPIGetClusterMasterList(ctx, reqEditors...) @@ -3447,6 +3523,31 @@ func (c *ClientWithResponses) DMAPIGetTaskStatusWithResponse(ctx context.Context return ParseDMAPIGetTaskStatusResponse(rsp) } +// ParseDMAPIGetClusterInfoResponse parses an HTTP response from a DMAPIGetClusterInfoWithResponse call +func ParseDMAPIGetClusterInfoResponse(rsp *http.Response) (*DMAPIGetClusterInfoResponse, error) { + bodyBytes, err := ioutil.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &DMAPIGetClusterInfoResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + switch { + case strings.Contains(rsp.Header.Get("Content-Type"), "json") && rsp.StatusCode == 200: + var dest GetClusterInfoResponse + if err := json.Unmarshal(bodyBytes, &dest); err != nil { + return nil, err + } + response.JSON200 = &dest + } + + return response, nil +} + // ParseDMAPIGetClusterMasterListResponse parses an HTTP response from a DMAPIGetClusterMasterListWithResponse call func ParseDMAPIGetClusterMasterListResponse(rsp *http.Response) (*DMAPIGetClusterMasterListResponse, error) { bodyBytes, err := ioutil.ReadAll(rsp.Body) diff --git a/dm/openapi/gen.server.go b/dm/openapi/gen.server.go index 434f551728e..2ec86b20155 100644 --- a/dm/openapi/gen.server.go +++ b/dm/openapi/gen.server.go @@ -20,6 +20,9 @@ import ( // ServerInterface represents all server handlers. type ServerInterface interface { + // get cluster info such as cluster id + // (GET /api/v1/cluster/info) + DMAPIGetClusterInfo(c *gin.Context) // get cluster master node list // (GET /api/v1/cluster/masters) DMAPIGetClusterMasterList(c *gin.Context) @@ -132,6 +135,15 @@ type ServerInterfaceWrapper struct { type MiddlewareFunc func(c *gin.Context) +// DMAPIGetClusterInfo operation middleware +func (siw *ServerInterfaceWrapper) DMAPIGetClusterInfo(c *gin.Context) { + for _, middleware := range siw.HandlerMiddlewares { + middleware(c) + } + + siw.Handler.DMAPIGetClusterInfo(c) +} + // DMAPIGetClusterMasterList operation middleware func (siw *ServerInterfaceWrapper) DMAPIGetClusterMasterList(c *gin.Context) { for _, middleware := range siw.HandlerMiddlewares { @@ -895,6 +907,8 @@ func RegisterHandlersWithOptions(router *gin.Engine, si ServerInterface, options HandlerMiddlewares: options.Middlewares, } + router.GET(options.BaseURL+"/api/v1/cluster/info", wrapper.DMAPIGetClusterInfo) + router.GET(options.BaseURL+"/api/v1/cluster/masters", wrapper.DMAPIGetClusterMasterList) router.DELETE(options.BaseURL+"/api/v1/cluster/masters/:master-name", wrapper.DMAPIOfflineMasterNode) @@ -969,91 +983,92 @@ func RegisterHandlersWithOptions(router *gin.Engine, si ServerInterface, options // Base64 encoded, gzipped, json marshaled Swagger object var swaggerSpec = []string{ - "H4sIAAAAAAAC/+w9WXPjNpp/BcvdhyQlWZLtvrw1D9220+Nd291lK5WdSvWqIRKSMCYBGgDtaLr836dw", - "kARJgKR8dKzEeUgcEceH775AfgtCmqSUICJ4cPAt4OEKJVD9eRhnXCB2BuW/5Q8poyliAiP1GEaR+jVC", - "PGQ4FZiS4ED9ijgHdAHECoEwYwwRARK1CCA0QsEgQL/DJI1RcBBMdt/sjHfGO5ODt7uvJ8EgEOtU/s4F", - "w2QZ3A0CGOMb1NyHkhgTBLiAIjO7YW62sXcQLEPFqnNKYwSJXDZGMEIO+DG3V1JnMEN7LEpgokAtz6eX", - "cRzsbhAwdJ1hhqLg4Dc9Mz9sAd1AI/lLMZvO/4lCIbcyxPmVsqs/kDhzmpFoxmnGQjTLT1/dUw0BegiQ", - "Qwpi3SrYm9sma34dD8dtGwq49G8lH3Zuosa6dmjSUC/Rn4YS9VVIXYhyEpUhKNAU8qsLdJ0hLpqEZSih", - "N2iWIAE1AhYwi0VwsIAxR4MaQm5XSKwkF1Og5wE5D0RQwDnkCGACInpLuGAIJsXPgYu1LdBnMdaQ/RdD", - "i+Ag+M9RqUJGRn+MLtX4c5igUzn6bhAIyK+6ZsmjN/BqH9ks40LeEYqRQHrfC8RTSjhq4k9O738KCU95", - "hjvXrlmSXiol1OTHUjlFWZKCjGARDGrwyE0l3NFMwHmsf1tQlkARHAQRzeaxRQ+SJXPE5LaIC5xAgWaC", - "ChjPGL3tO3OBCeYrFM3ma4E2nrTBRhoyx6kwEa/3yxmYCLSUU2pkr8wfNBHVOEodTDeWXKxzzBhlv2Kx", - "OkOcO1WLJBmUfwMkxzbIqH6dhVLLNOaqZyDUGqh+6IGZmvClb2ZigOrSP+VCAxse14E/IlGx7pK//TIj", - "FYP8LxYo4V0iU/UaSpGBjMF1wRdKFHvQPxjo3dsPoa3g4x/CWNcnPoRWWY8IvV7w+4Ctdd+jAm7U6ROD", - "L1X7I+JcW66nB/lx8Z3NyzW/B/RTqbovBctCkbEWK60BnIXKH5rx67jqkR1eHL+fHoPp+w+nx+CrmHwF", - "P3zF0VeAifhhMvkRnH+agvNfTk/B+1+mn2Yn54cXx2fH59PB54uTs/cX/wD/e/wPPeNHMPpp+h+/hVrc", - "UTTDJEK/fwGHp79cTo8vjo/AT6MfwfH5x5Pz47+dEEKPPoCj45/f/3I6BYd/f39xeTz9WyYWb5P5Pjj8", - "dHr6fnqc//9sjonLvzRHa7qZ0dzp8Spr5xiufu92Sq3p+VoWVl2kOqUw6vZoYgojt0fT4mD4LP8gkM6d", - "xFhMlxa7lViwns+WAkfOQSmjSxnfOB9qF6A/TDU8NnwNez1r6+pRHIC7UP5JeRfIJSFFGFCL5ELljAgK", - "tGeCgKItYGZCgyhxxleVgEHHsNVVf2VYIK5iQ82mcgMVKa5QeJVSTATg8hcowNEZCCHRfIAFgAsZQjLE", - "BWQCk6Wapnx1ZzRxHc9CSgQijrPx6xisaQZuIRHWCSvhm0MDgK/hpFQBuZRKNTAAX8Nd/6M996MHyP1/", - "OwV/TcLmYX9JI5jjnKYCJ5gLHAK+giySaJT8I7UquMVipUNaQxpK4jXIOIrA7QoRAI1rCmgYZozL2M63", - "5tHRKUgq7mhBmhrX23RyMe7njLm8ZYZiuAYxXYJQLpulIKUxDtcgpGSBl5l2pZtO9O8pZsaM5Ww6rvOo", - "GqRdcYF1NqHYToYAdbkmWRxL0ahlbSzdI/9kN9rOFfvuvR43tp6uZMisB0vGTBHDNMIhjOO1FhGAdWal", - "RADmQB8rGgCzOLiBcYYOgNpC0omjkJKI3w96hhKIyYynMESVE0xe1eE/wwQnWQIWDCEQYX4F1CwFw8cP", - "99neFRFfyLMfKkI3OUMpE/WsIFyTDYgyU/phqcot9aEe1c1Bw1JpPfRxenKUJ7yy1KQ6CvVcahT0Dk4W", - "4e7uEIXjt8PJBL0bzndhOBzv7u/CcDIZj8d7B5Phm7f77/yIKaW9AqI7M1aAuMAxKjNj7WDq5Ngck52x", - "/Ge3PywRZhX+CHZG+oHeokmnCDMUCsrWUsEw1GRsLqjUE50QeLmk282wRbvKJTppafkM1SVqOFQ4Bpho", - "DtfKp0TqDzWsTgZg8u7Nux9daryyr4f5XDz3AGZrZy43CBpxeYZXAvT4AIRQhKtZls6SokTgzT+qsSBL", - "tR0rqGP5TT4xL/j2/vxZnntnxLO5WtJlod1p5RyJmisry11khMjJXV54lVmdTGQf10VhH9JzsF3m+VJ5", - "CkUCsyln2pNQukelQwdl3NgdmNRixUsUZgyLdXMb5b+YCgDncdULGCi6LTCKI3CL4xjMEVjhKEJE+zVL", - "JAp/0l6osghYMJqoIco+L6QtbKqlWuYVMTGDcUxvUTQLSRPsQ5oklIBzo5kvL0+BnIMXOITa6y+Q1Ykc", - "zuNZCP0+r7WwVlX5SJvbnDwrF5Yn8S79s7WcPMfn4zOg1eDo/16N35m/60fr3vUKrf2bHpb7SaqkDN/I", - "o12hda6JgbV5x351p7SKSwcOmgA6pcP4wx8ZzVJHJiWKixpBf0IvMONiFtNQWxnXFBkIoGizZQVkSySc", - "QzOy+YKNJIFafVCeuXGQAmxrQydSde6xqWr07x5fr7RhPctZGUfas5NueCa1hlKVXCsCl801KzatzIo6", - "VaPWMqZi6i6LuiQjhZzfUhZ5VywGVJfc23/12rkeZX7o1ENrnb298WtXjJDmYVpbMlDHcqVxLzz4tkm2", - "sy9Z27IBrYnHfFy1sug9qMlf9asRazvdlN2HZJ4z7vJvDHTyYQNCRqnoVmXW2Q0nGpKbLS2GGlSExS97", - "LebeKsL7zb0eNexn8220+fYr/CZXyay77qXdAE4TJFbSEbhl1OVx5XzLC2A6+bYk9wN4kKE0xiH08KLu", - "HfAsPF2hvD/B+JjxGugmBpN3K7RmvRthONmQt2xAnLwjg1GFlR45RxW5Ami8bV/O8SVK36oovcIjvboj", - "dHm00h9hM2BjOTff0bQ/29G0k+v+kENU6mhNTzJL0p56yWonuRsEMYVRz4lW1cZqYqqxC+RXuRprWs3+", - "KvAegfOyEBtTE7AMZeYOobXH2fP4l2sSlsdXdSn38eUjoHayYVC5+YHLt2aI0/gGRTPlGtPwauYpPrVq", - "+rwLzYk/dxuZX33n+DbndDJkiY6W5Jo8NcgyRw3PaES9ruOwc4kJTJYSK64t7ErD7QqHqyIThTnIJ28U", - "QDfSfT0Tcw41GyIiZiLtW5o02fnZHK0wiaxcV5+5RWTmqIHJZ60nqozwn0hXItFN3sLbAy7TW9UbB5Yc", - "LGW03EZzPaBGdsgQyMgwX8UmfatYV0L0zjDWRoR9yArVB/2ycVXyOIlRlwMXnqy42RYqH1u5hFmVhB+a", - "xPO1CzQlbWr6NJvK06cmFjiW+GNZjEzvMZazYPy5MrqrfeYDJqd0+bNa7EKu5aobILKCJEQz3f89yxtF", - "VpAsUWd920og6FgK8CyV4RZYUN3rbdrKoygGaZwtMenT9q1q/BqSqu8WJUPTtVpLiDabbhUE0lXLi77e", - "YkW5qLd32W/2bYbgV+4gj5JZlKmgRjhWW9Fbib8VJJHOKy5iHAoUqZOoIDVLpDDSG8RuGdZ1e9Uz+cVl", - "4qWAzxJn36Skxy1cq+oBpVIPQIGkSbF2SRHnpr4dDIKy2O3eTJvUfpkN5dCpCVZ64z6Zhc4eKxXUJ3jJ", - "oECFENVJKJnVjAFqzKB/X5pSIGd6ck2wagnGDXAzVROOoIAfIEd5g7eHlDnkpusgp94ii2N5EBIylCCi", - "e8hgrPqSSk6FalAvp6kEoUNT1Li8fn4nVeoM5NbVDj3mSskLpCRdLswBFHmZMkY3KG7oWbwklCFt2RzZ", - "Evlz7tMWTNEypoJaECVxH7NgYDC9eM2OnRQKgZiK07Q98APjG17C9f9HjKbdUN15KPBzFseG36XwOiKT", - "SvGILoDkxEK+JBdxR6c+4ZgLREJHiUvpKCIYjUGutjAxPpCqWuk+EMqkplyodvBiNQA5z5jk1SptMkFd", - "KJDLuYui0nzISCvCrKnvd0b5/jOjqRsr6wEzsWIIRtU2nP26CVMI0xMk/kJKjKvn9B9x4l158tq5tJ7R", - "ubSPA05IyDbjAEsJeRiAoTSezaEIq410k2ajkL2WdP9WjBL8r2IrtQZAv6MwUz9JebjOIBFYbeXu8knj", - "nuirH+TeOPS7nIVH0epw+vwLl8NZWtpm+qQW/5RbjPcW4Xj39d5w9234ZjiZoDdD+PrV3vB1OJ6/3Y9e", - "vVvsjQ8mwzfj/cn+7t5g/Gr/zX60F1rD3+692h3ujvei+e7+6yjaiw4mw8mbsfPmWTVJaN0kUw9M40jL", - "zJRWMbTvDBifJhHdkhr2WbGK7+MBZchQDKVGa+/rkwJdmNLQ0LjLv6jr8DvtJ2y8Tl0TVP1AL5LrJ+rt", - "bFmc3BWv2nD4yNDw3fydTtpJFNS+02e7jLxn/FbTxuqhWiDnPIe0y8f9pJ23Foh7cpQdbHli4QG4xXEU", - "QhblQV41ipoPf3pgGrRRMvOlR4XOurud+h6wCiesreUeg6B8bxd3lS0FvuD0MYkRUcQBoaKIuPMT8xpZ", - "JvfEYM8NxLyHeuxCnhP1LSJciZRaEF5mA9oxvo0tC5t1LNynkeCJavTtVXkv0VGSSuHxXuQu8yOb9L0U", - "s7RrJ8wuxR/dLf3lvt2g+65nLSCO1UVcftVMhrRU+R1ybW5jO542epTyofaVWadiq1ucLAwR5x5wN+uU", - "aq41aGLDBVStzNhWIWpxqv3F/+axyx29pVVTQ+UgV/SCmoYE3lZX7apv3aNZob094U4Fp0JKcHxEQ0dK", - "4egMfEoRef/5BBx9OpRyyuLgIFgJkfKD0SiiId9JMVmGMN0JaTL612okcDQfSoU71E4SpmTEtcZXvuaC", - "KvbAQp2kscENYlzv/Wpnb2esEqcpIjDFwUGwJ7WtUhNipaAdwRSPbiYjc9NwpNP96pExwsUN85NIbff+", - "84nrlraqZWi5VLN3x2OTnMgbMWGqs1ryPP/kugexNNFtyrT1VrgiQk21amGQR99/RDAat/EdW2uZU2zE", - "sySBbB0cSEwCg2D7nSa5PAm45JLXzJDgi5ztIczom/5D+XR3mt9ipJW1g1KfFosYE6TRdq7zjylkMEGa", - "yr81EqIWeLlXLX+XDBPkifzAgiGw5UUXIkps9nnfzJcG4+w7jOUzoyjVeK29oaYXIXM91lPCylcIfB8J", - "c7yyYMskzHqzzkYSZggz+maMw0YSZoxaDwmzwfNLmAXDX1vCqu9JaiVklOzkwDkl6yMSRzT8n8tP5x5R", - "qoIl1yquOzTZLaIhUNuVUEU0rEFkfIIWcP4+PTvtBY4c2AHOSuiSiQ8cHXJ0q57yxR9dzCx3NoGMuj9V", - "tNQqlr7OEFtbPI3FalaMcPCwu47t4N9HVXyO15w4mNS+4hPnDXk1EtSHlKTII3EVhXIf6vVrti7z3lrj", - "BX+g0frRzpu/gqV5QLMbmMvt7hoon3wHEJ6bDtIvpAAE3dq0dZG1KWSjb1byrduM2C8J6xS6mM7V1e6M", - "4OusevvMb1GqucBeFsV7l+Fu0MjGUt1RT1OdAIAxN32xed+vCuNM/cqlHdQKD9QL+4/GM86Xtm0By2om", - "A/ChDDtKYcZ11lspnxat9VmOvMhvzD9zxv3Sx9Q+N6IqWljN84uM6N7zvDvsocRmiGdJP2pfqKEv5H5C", - "cmtqPCW9rZfp9nAE9W3tPu7gExDXf+vrSf3C2g31LQmB84ttusDl80H7ssfom/6jdGF6MIuqCz8/Xhm0", - "FAE925dn77m9s0b4pFxa7cDeLibVNdL786iATPSyWOUNwm0xWE8Q9jVuUd5VKx0S2LttNJamX/4pjWVx", - "X6mPrSzuFD8fRmttwPouyZXay0G3RFHZr6K33+f/GCxF0566y9xC/SurrtpF3D+L5oowf2rVJRgkfGG+", - "w+DnsqkZtjXppyditWZnwp+F13JGKJwvCqB+3aKur3Rwl87bdVnA/M3VfYoGKkm4tSWDxju6HXRQJ4zN", - "Nyaej00roCoprr9c0V6aUA7kVN+1fArRa35h5I8sUZjPfWxLgQLqr8swUbzYuErZuiSP8q68fjKd9919", - "hyaELResou3xHhJWSsC07Jl8ClHzMfeLdLmlq0LZTYRrpO+KdThfJ2rQd6J7vft3czbYfSJ4tic0NDcA", - "788W39QVtE3qwjXu2Mg9t2/ZO/zyApaeXrnv+tx2dhmZcqm/Z72uwHsby+0h0/gvp9ib9rqN5GnmIbn+", - "RMIL0beD6JmiVm+6N/T3/bT2c+WIQZ93cDoC8sbnFu19H/TSzq02IDoCM81PmzGT7rTp02PznPnpy1O2", - "K9oVzrvtbeC5B2/oVpBeLTkv3LGt3GH6fe7BHg/s7in6ej6sJfe8J9H9KgDPwWi99Bv9UU50a9PRg7l4", - "wyakov3ohaVf2qK2VpacvVGPLEpy3jxGG0Y09kcgX2TqmcnUwP8eEh/Kcw7ojXPPx123PvtXkTxusfim", - "KcAXCXmRkD+g0a7lK9ZbawBbxdCblj0pvlr8Ioobb/5XEcTHT0Z0fiv7z9JLVn7YewN5bfda+3VYW6+r", - "/itl1TfKgH0HK7OlzdyKW3PuqXOnerMbu8m5qfqSpjXNdiKaQEzUK5oCiWSzgPdrgu1vhYpo+MBXQY2u", - "MxxeDfUtGN2oMjSb39XYKnApW/Otm+8CpAGveDpU299VxM8BZP6Oj2Jc/sPdl7t/BwAA//8QGZ0BHpAA", - "AA==", + "H4sIAAAAAAAC/+w9W3Pbtpp/Bcvdh7YjWZLtOIl3zoNjuznetZ2MrU73TCerQCQk4ZgEaAC0q5Pxfz+D", + "C0mQBCjKt1iN+9C6Ii4fvvsN5LcgpElKCSKCB/vfAh4uUALVn4dxxgViZ1D+W/6QMpoiJjBSj2EUqV8j", + "xEOGU4EpCfbVr4hzQGdALBAIM8YQESBRiwBCIxT0AvQnTNIYBfvBaPvt1nBruDXaf7e9Nwp6gVim8ncu", + "GCbz4K4XwBjfoOY+lMSYIMAFFJnZDXOzjb2DYBkqVp1SGiNI5LIxghFywI+5vZI6gxnaYVECEwVqeT69", + "jONgd72AoesMMxQF+3/omflhC+h6Gslfitl0+k8UCrmVIc7vlF19R+JMaUaiCacZC9EkP311TzUE6CFA", + "DimIdatgb26bLPl13B+2bSjg3L+VfLhyEzXWtUOThnqJ7jSUqK9C6kKUk6gMQYHGkF9doOsMcdEkLEMJ", + "vUGTBAmoETCDWSyC/RmMOerVEHK7QGIhuZgCPQ/IeSCCAk4hRwATENFbwgVDMCl+DlysbYE+ibGG7L8Y", + "mgX7wX8OShUyMPpjcKnGn8MEncrRd71AQH61apY8egOv9pHNMi7kHaEYCaT3vUA8pYSjJv7k9O6nkPCU", + "Z7hz7Zol6aVSQk1+LJVTlCUpyAgWQa8Gj9xUwh1NBJzG+rcZZQkUwX4Q0WwaW/QgWTJFTG6LuMAJFGgi", + "qIDxhNHbrjNnmGC+QNFkuhRo7UlrbKQhc5wKE7G3W87ARKC5nFIje2V+r4moxlHqYLqx5GKdY8Yo+x2L", + "xRni3KlaJMmg/BsgObZBRvXrJJRapjFXPQOh1kD1Q/fM1ITPfTMTA9Qq/VMu1LPhcR34IxLGgJyQGfVL", + "S6gHTXDUBM48A1iq0YK4WUfqWiu3A6jdDymAfjCl5pL/xQIlfJVMV92aUqYhY3BZMK7SFR0YNOjp3dsP", + "oc304x/CmP8nPoTWqY8IvV7wecDWyvlRATf6/onBl7bnEXGuTevTg/y4+M6m5ZrPAf1Y2pZLwbJQZKzF", + "jdAATkLlsE34dVx1GQ8vjg/Gx2B88OH0GHwVo6/gp684+gowET+NRj+D809jcP7b6Sk4+G38aXJyfnhx", + "fHZ8Pu59vjg5O7j4B/jf43/oGT+DwS/j//jDaEsUTTCJ0J9fwOHpb5fj44vjI/DL4GdwfP7x5Pz4byeE", + "0KMP4Oj414PfTsfg8O8HF5fH479lYvYume6Cw0+npwfj4/z/J1NMXA6wOVrTD46mTpdcmWPHcPX7aq/Z", + "mp6vZWHVRapTCqPVLldMYeR2uVo8IJ/x6gXS+5QYi+ncYrcSC9bzyVxoe9kYlDI6lwGY86H2UbrDVMNj", + "wxmy17O2rh7FAbgL5Z+U+4NcElLEKbVQM1TekqBAu04IKNoCZiY0iBJnfFGJaHSQXV31d4YF4ip41Wwq", + "N1Ch7AKFVynFRAAuf4ECHJ2BEBLNB1gAOJPOCkNcQCYwmatpKphwhjvX8SSkRCDiOBu/jsGSZuAWEmGd", + "sBJfOjQA+BqOShWQS6lUAz3wNdz2P9pxP3qA3P+3U/CXJGwe9rc0gjnOaSpwgrnAIeALyCKJRsk/UquC", + "WywWOuY2pKEkXoKMowjcLhAB0PjOgIZhxrgMPn1rHh2dgqTiLxekqXG9TScX437OmMudZyiGSxDTOQjl", + "slkKUhrjcAlCSmZ4nmlfv+nl/5liZsxYzqbDOo+qQTpWEFinO4rtbE85l2uSxbEUjVpaydI98k92o+1c", + "se/O3rCx9XghY3o9WDJmihimEQ5hHC+1iACsUz8lAjAH+lhRD5jFwQ2MM7QP1BaSThyFlET8ftAzlEBM", + "JjyFIaqcYPSmDv8ZJjjJEjBjCIEI8yugZikYPn64z/aukP1Cnv1QEbrJGUqZqGcF4ZpsQJSZ0g9LVW6p", + "D/Wobg4alkrroY/jk6M8I5elJhdTqOdSo6D3cDQLt7f7KBy+649G6H1/ug3D/nB7dxuGo9FwONzZH/Xf", + "vtt970dMKe0VEN2puwLEGY5RmbprB1Nn76aYbA3lP9vdYYkwq/BHsDXQD/QWTTpFmKFQULaUCoahJmNz", + "QaWeWAmBl0tWuxm2aFe5RGdVLZ+hukQNhwrHABPN4Vr5lEj9qYbVUQ+M3r99/7NLjVf29TCfi+cewGzt", + "zOUGQSMuT0FLgB4fgBCKcDHJ0klS1DC8CVI1FmSptmMFdSy/ySfmBd/enz/Lc28NeDZVS7ostDvvnSNR", + "c2VluYuMEDl5lRdeZVYnE9nHdVHYh/QcbJd5vlSeQpFhbcqZ9iSU7lH52l4ZN64OTGqx4iUKM4bFsrmN", + "8l9MiYLzuOoF9BTdZhjFEbjFcQymCCxwFCGi/Zo5EoU/aS9UWQTMGE3UEGWfZ9IWNtVSLfmGmJjAOKa3", + "KJqEpAn2IU0SSsC50cyXl6dAzsEzHELt9RfIWokczuNJCP0+r7WwVlX5SJvbnDwrF5Yn8S79q7WcPMfn", + "4zOg1eDg/94M35u/60dbvesVWvo3PSz3k1RJGb6RR7tCy1wTA2vzFfvVndIqLh04aALolA7jD39kNEsd", + "mZQoLooY3Qk9w4yLSUxDbWVcU2QggKL1lhWQzZFwDs3I+gs2kgRq9V555sZBCrCtDZ1I1bnHpqrRv3t8", + "vdKGday3ZRxpz0664ZnUGkpVcq0IXDbXrNi0MgvqVI1ay5iSrrtu65KMFHJ+S1nkXbEYUF1yZ/fNnnM9", + "yvzQqYfWOjs7wz1XjJDmYVpbMlDHcqVxLzz4tkm2sy9Z27IBrYnHfFy19Ok9qMlfdStiazvdlN2HZJ4z", + "7vJvDHTyYQNCRqlYrcqssxtONCQ3W1oM1asIi1/2Wsy91SXgN/d6VL+bzbfR5tuv8JtcNb3VhTntBnCa", + "ILGQjsAtoy6PK+dbXgCzkm9Lcj+ABxlKYxxCDy/q5gbPwuMFyhsojI8ZL4HusjB5t0Jr1tsl+qM1ecsG", + "xMk7MhhVWOmQc1SRK4DG2/blHF+j9I2K0is80ql9Q5dHKw0cNgM2lnPzHU27sx1NV3LddzlEpY7W9CSz", + "JO2ol6x+l7teEFMYdZxoVW2sLqsau0B+lauxptXsrgLvETjPC7ExNQHLUGbuEFp7nB2Pf7kkYXl8VZdy", + "H18+AmonGwaVm++5fGuGOI1vUDRRrjENryae4lOrps/b5Jz4c/e5+dV3jm9zTidDluhoSa7JU4Msc9Tw", + "jEbU6zoOO5WYwGQuseLawq403C5wuCgyUZiDfPJaAXQj3dcxMedQsyEiYiLSrqVJk52fTNECk8jKdXWZ", + "W0RmjhqYfNZ6osoI/4l0JRLd5D3GHeAyzV+dcWDJwVxGy2001wNqZIcMgYz081Vs0reKdSVEXxnG2oiw", + "D1mheq9bNq5KHicx6nLgwpMVN9tC5WMrlzCrkvBDk3i+doGmpI1NI2lTefrUxAzHEn8si5FpjsZyFow/", + "V0avap/5gMkpnf+qFruQa7nqBogsIAnRRDeoT/JGkQUkc7Syvm0lEHQsBXiWynALzKhuRjd971EUgzTO", + "5ph06UtXNX4NSdV3i5K+aautJUSbXcEKAumq5UVfb7GiXNTbXO03+zZD8Ct3kEfJJMpUUCMcqy3orcTf", + "ApJI5xVnMQ4FitRJVJCaJVIY6Q1itwzrur1q6vziMvFSwCeJs7FT0uMWLlX1gFKpB6BA0qRYu6SIc1Pf", + "DnpBWex2b6ZNarfMhnLo1AQrvXGfzMLKHisV1Cd4zqBAhRDVSSiZ1YwBakyve1+aUiBnenJNsGoJxjVw", + "M1YTjqCAHyBHeQe6h5Q55KbrIKfeLItjeRASMpQgonvIYKz6kkpOhWpQJ6epBGGFpqhxef38TqrUGcit", + "qx16zJWSF0hJulyYAyjyMmWMblDc0LN4TihD2rI5siXy59ynLZiiZUwFtSBK4i5mwcBgevGaHTspFAIx", + "Fadpe+AHxje8hOv/jxhNV0N156HAr1kcG36XwuuITCrFIzoDkhML+ZJcxB1XCQjHXCASOkpcSkcRwWgM", + "crWFifGBVNVK94FQJjXlTPWrF6sByHnGJK9WaZMJ6kKBXM5dFJXmQ0ZaEWZNfb81yPefGE3dWFkPmIgF", + "QzCqtuHs1k2YQpieIPEXUmJcPaf/iBPvyqM959J6xsqlfRxwQkK2HgdYSsjDAAyl8WQKRVhtpBs1G4Xs", + "taT7t2CU4H8VW6k1APoThZn6ScrDdQaJwGord5dPGndEX/0g98ah3+UsPIpWh9PnX7gcztLSNtMntfin", + "3GK4MwuH23s7/e134dv+aITe9uHem53+XjicvtuN3ryf7Qz3R/23w93R7vZOb/hm9+1utBNaw9/tvNnu", + "bw93oun27l4U7UT7o/7o7dB5Na6aJLSuuqkHpnGkZWZKqxjadQaMT5OIbkkN+6xYxffxgNJnKIZSo7X3", + "9UmBLkxpaGi8yr+o6/A77SesvU5dE1T9QC+S6yfq7GxZnLwqXrXh8JGh4bv5O520kyiofenQdhl5x/it", + "po3VQ7VAznkOaZePu0k7by0Qd+QoO9jyxMI9cIvjKIQsyoO8ahQ17f/ywDRoo2TmS48KnXV3O/UdYBVO", + "WFvLPQZB+d4u7ipbCnzB6WMSI6KIA0JFEXHnJ+Y1sozuicGOG4hpB/W4CnlO1LeIcCVSakF4mQ1ox/gm", + "tiys17Fwn0aCJ6rRt1flvURHSSqFx3vTvMyPrNP3UszSrp0wuxR/rG7pL/ddDbrvetYM4ljdFOZXzWRI", + "S5XfIdfmurjjaaNHKR9q3+l1Kra6xcnCEHHuAXe9TqnmWr0mNlxA1cqMbRWiFqfaX/xvHrvc0VtaNTVU", + "DnJFL6hpSOBtddVV9a17NCu0tyfcqeBUSAmOj2joSCkcnYFPKSIHn0/A0adDKacsDvaDhRAp3x8MIhry", + "rRSTeQjTrZAmg38tBgJH075UuH3tJGFKBlxrfOVrzqhiDyzUSRob3CDG9d5vtna2hipxmiICUxzsBztS", + "2yo1IRYK2gFM8eBmNDA3DQf58sYCF/ffTyK118Hnk+odclXC0OKo1tseDk1OIu+/hKlOZslj/JPr1sPS", + "MrfpUM9tdYX1mi7V3K+ox7MkgWwZ7MszgOK2OplRwLNwASAHlSvsAs65dTM9+CIXqaNFV0F4V8yUl9ef", + "Bz+Oy/JtWOoFu48IRuMtCo6ttSpqoY/1LppczaxDmME3/Ydyde+0GMZI2zAHpT7NZjEmSKPtXKdlU8hg", + "gjSV/2jkiS3w8mBD/i7lKMjrG4EFQ2CrEV2fKbHZ5T1BXxqMs+vwIV4YRanGa+3NQp0Imav3jhJWvlnh", + "eSTM8SaHDZMw641Ia0mYIczgm7GZa0mYsfUdJMwGzy9hFgw/toRV32/VSsgo2cqBc0rWRySOaPg/l5/O", + "PaJUBUuuVdwCabJbREOgtiuhimhYg8i4Si3g/H18dtoJHDlwBTgLoStJPnB0JLZa9ZTvQ1nFzHJnE9+p", + "a2VFp7Fi6esMsaXF01gsJsUIBw+7y/sO/n1Uxed4+4uDSe2bT3Hep1gjQX1ISYo8QaGCc+5DvX492mXe", + "cmyCgw80Wj7aefM30zQPaHYDU7ndXQPlo2cA4aXpIP2eDkDQrU1bF1mbQjb4ZuUkV5sR++VuK4UuplN1", + "4z0j+DqrXsrzW5RqirSTRfFe8bjrNZLUVF80oKnOi8CYm3bhvB1aRbemrOfSDmqFB+qF3UfjGefL9jaA", + "ZTWTAfhQhh2kMOO6GKCUT4vW+ixHXuQvEnjhjPuli6l9aURVtLDuFMwyolvy86a5hxKbIZ4l3ah9oYa+", + "kvsJya2p8ZT0tl6C3MER1JfYu7iDT0Bc/2W4J/ULaxf3NyQEzu/76bqfzwftyh6Db/qP0oXpwCyqXP7y", + "eKXXUhv1bF+eveP2ztLpk3JptTF9s5hUl47vz6MCMtHJYpUXKzfFYD1B2Ne4XHpXLQBJYO820ViaawRP", + "aSyLa1xdbGVx1frlMFprX9qzJFdq70zdEEVlf0LA/g7DY7AUTTvqLnM590dWXbX7yX8VzRVh/tSqSzBI", + "+Mx8P8PPZWMzbGPST0/Eas2Gjb8Kr+WMUDhfFED9FkpdX1nBXTpvt8oC5i/07lI0UEnCjS0ZNF5d7qCD", + "OmFsvg3ycmxaAVVJcf3FkfbShHIgx/oK6lOIXvPLMN+zRGE+07IpBQqovwrERPG+5ypl65I8yJsVu8l0", + "3o74DE0IGy5YRTfoPSSslIBx2Ur6FKLmY+5X6XJLV4Wy6wjXQF+hW+F8nahBz0T3elP0+myw/UTwbE5o", + "aC5G3p8tvqmbeevUhWvcsZZ7br98wOGXF7B09Mp9two3s8vIlEv9rfx1Bd7ZWG4OmYY/nGJv2us2kqeZ", + "h+T6yxGvRN8MomeKWp3p3tDf99PaL5Ujel1eTeoIyBufybT3fdC7TDfagOgIzDQ/rcdMutOmS4/NS+an", + "L0/ZrmhXOO82t4HnHryhW0E6teS8csemcofp97kHezywu6fo6/mwlNxzQKL7VQBegtF67Tf6Xk50a9PR", + "g7l4zSakov3olaVf26I2VpacvVGPLEpy3jRGa0Y09rcxX2XqhclUz/96Fh/Kcw7ojHPPN283PvtXkTxu", + "sfi6KcBXCXmVkO/QaNfyce+NNYCtYuhNy54UH3N+FcW1N/9RBPHxkxErPyH+V+klK793voa8tnut3Tqs", + "rbd4/0hZ9bUyYM9gZTa0mVtxa849de5UL7xjNzk3Vd9dtaTZVkQTiIl6c1UgkWwW8H5ksf1lWRENH/iG", + "rMF1hsOrvr4FoxtV+mbzuxpbBS5laz4B9CxAGvCKp321/V1F/BxA5u/4KMblP9x9uft3AAAA///Luz+r", + "1pEAAA==", } // GetSwagger returns the content of the embedded swagger specification file diff --git a/dm/openapi/gen.types.go b/dm/openapi/gen.types.go index 6913035c0bf..ca478469ac9 100644 --- a/dm/openapi/gen.types.go +++ b/dm/openapi/gen.types.go @@ -93,6 +93,12 @@ type ErrorWithMessage struct { ErrorMsg string `json:"error_msg"` } +// GetClusterInfoResponse defines model for GetClusterInfoResponse. +type GetClusterInfoResponse struct { + // cluster id + ClusterId uint64 `json:"cluster_id"` +} + // GetClusterMasterListResponse defines model for GetClusterMasterListResponse. type GetClusterMasterListResponse struct { Data []ClusterMaster `json:"data"` diff --git a/dm/openapi/spec/dm.yaml b/dm/openapi/spec/dm.yaml index b1bfa185e96..7db4ce256f2 100644 --- a/dm/openapi/spec/dm.yaml +++ b/dm/openapi/spec/dm.yaml @@ -959,6 +959,19 @@ paths: "application/json": schema: $ref: "#/components/schemas/ErrorWithMessage" + /api/v1/cluster/info: + get: + tags: + - cluster + summary: "get cluster info such as cluster id" + operationId: "DMAPIGetClusterInfo" + responses: + "200": + description: "success" + content: + "application/json": + schema: + $ref: "#/components/schemas/GetClusterInfoResponse" components: schemas: @@ -1833,3 +1846,12 @@ components: required: - "success_task_list" - "failed_task_list" + GetClusterInfoResponse: + type: object + properties: + cluster_id: + type: integer + format: uint64 + description: "cluster id" + required: + - "cluster_id" diff --git a/dm/pkg/retry/strategy.go b/dm/pkg/retry/strategy.go index bbaceb65fb6..74a64849d39 100644 --- a/dm/pkg/retry/strategy.go +++ b/dm/pkg/retry/strategy.go @@ -17,6 +17,9 @@ import ( "time" tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + + "go.uber.org/zap" ) // backoffStrategy represents enum of retry wait interval. @@ -77,6 +80,7 @@ func (*FiniteRetryStrategy) Apply(ctx *tcontext.Context, params Params, duration = time.Duration(i+1) * params.FirstRetryDuration default: } + log.L().Warn("retry stratey takes effect", zap.Error(err), zap.Int("retry_times", i), zap.Int("retry_count", params.RetryCount)) select { case <-ctx.Context().Done(): diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 6c95627d941..04490f95306 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -497,6 +497,7 @@ const ( codeMasterInconsistentOptimistDDLsAndInfo codeMasterOptimisticTableInfobeforeNotExist codeMasterOptimisticDownstreamMetaNotFound + codeMasterInvalidClusterID ) // DM-worker error code. @@ -947,7 +948,7 @@ var ( ErrRelayNoCurrentUUID = New(codeRelayNoCurrentUUID, ClassRelayUnit, ScopeInternal, LevelHigh, "no current UUID set", "") ErrRelayFlushLocalMeta = New(codeRelayFlushLocalMeta, ClassRelayUnit, ScopeInternal, LevelHigh, "flush local meta", "") ErrRelayUpdateIndexFile = New(codeRelayUpdateIndexFile, ClassRelayUnit, ScopeInternal, LevelHigh, "update UUID index file %s", "") - ErrRelayLogDirpathEmpty = New(codeRelayLogDirpathEmpty, ClassRelayUnit, ScopeInternal, LevelHigh, "dirpath is empty", "Please check the `relay-dir` config in source config file.") + ErrRelayLogDirpathEmpty = New(codeRelayLogDirpathEmpty, ClassRelayUnit, ScopeInternal, LevelHigh, "dirpath is empty", "Please check the `relay-dir` config in source config file or dm-worker config file.") ErrRelayReaderNotStateNew = New(codeRelayReaderNotStateNew, ClassRelayUnit, ScopeInternal, LevelHigh, "stage %s, expect %s, already started", "") ErrRelayReaderStateCannotClose = New(codeRelayReaderStateCannotClose, ClassRelayUnit, ScopeInternal, LevelHigh, "stage %s, expect %s, can not close", "") ErrRelayReaderNeedStart = New(codeRelayReaderNeedStart, ClassRelayUnit, ScopeInternal, LevelHigh, "stage %s, expect %s", "") @@ -1145,6 +1146,7 @@ var ( ErrMasterInconsistentOptimisticDDLsAndInfo = New(codeMasterInconsistentOptimistDDLsAndInfo, ClassDMMaster, ScopeInternal, LevelHigh, "inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d", "") ErrMasterOptimisticTableInfoBeforeNotExist = New(codeMasterOptimisticTableInfobeforeNotExist, ClassDMMaster, ScopeInternal, LevelHigh, "table-info-before not exist in optimistic ddls: %v", "") ErrMasterOptimisticDownstreamMetaNotFound = New(codeMasterOptimisticDownstreamMetaNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "downstream database config and meta for task %s not found", "") + ErrMasterInvalidClusterID = New(codeMasterInvalidClusterID, ClassDMMaster, ScopeInternal, LevelHigh, "invalid cluster id: %v", "") // DM-worker error. ErrWorkerParseFlagSet = New(codeWorkerParseFlagSet, ClassDMWorker, ScopeInternal, LevelMedium, "parse dm-worker config flag set", "") From 801bb607615e42be6be7770a302d692bca037834 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 8 Feb 2022 16:41:35 +0800 Subject: [PATCH 43/72] cdc/sink: Kafka support user set configuration (#4512) close pingcap/tiflow#4385 --- cdc/sink/producer/kafka/config.go | 39 ++++++++++++++++++++++++++ cdc/sink/producer/kafka/config_test.go | 37 ++++++++++++++++++++++-- 2 files changed, 74 insertions(+), 2 deletions(-) diff --git a/cdc/sink/producer/kafka/config.go b/cdc/sink/producer/kafka/config.go index 4f9bee3af36..4f2bd6bfce7 100644 --- a/cdc/sink/producer/kafka/config.go +++ b/cdc/sink/producer/kafka/config.go @@ -50,6 +50,11 @@ type Config struct { SaslScram *security.SaslScram // control whether to create topic AutoCreate bool + + // Timeout for sarama `config.Net` configurations, default to `10s` + DialTimeout time.Duration + WriteTimeout time.Duration + ReadTimeout time.Duration } // NewConfig returns a default Kafka configuration @@ -63,6 +68,9 @@ func NewConfig() *Config { Credential: &security.Credential{}, SaslScram: &security.SaslScram{}, AutoCreate: true, + DialTimeout: 10 * time.Second, + WriteTimeout: 10 * time.Second, + ReadTimeout: 10 * time.Second, } } @@ -201,6 +209,33 @@ func CompleteConfigsAndOpts(sinkURI *url.URL, producerConfig *Config, replicaCon opts["enable-tidb-extension"] = s } + s = params.Get("dial-timeout") + if s != "" { + a, err := time.ParseDuration(s) + if err != nil { + return err + } + producerConfig.DialTimeout = a + } + + s = params.Get("write-timeout") + if s != "" { + a, err := time.ParseDuration(s) + if err != nil { + return err + } + producerConfig.WriteTimeout = a + } + + s = params.Get("read-timeout") + if s != "" { + a, err := time.ParseDuration(s) + if err != nil { + return err + } + producerConfig.ReadTimeout = a + } + return nil } @@ -243,6 +278,10 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { // and https://github.com/pingcap/tiflow/issues/3352. config.Metadata.Timeout = 1 * time.Minute + config.Net.DialTimeout = c.DialTimeout + config.Net.WriteTimeout = c.WriteTimeout + config.Net.ReadTimeout = c.ReadTimeout + config.Producer.Partitioner = sarama.NewManualPartitioner config.Producer.MaxMessageBytes = c.MaxMessageBytes config.Producer.Return.Successes = true diff --git a/cdc/sink/producer/kafka/config_test.go b/cdc/sink/producer/kafka/config_test.go index d34ef822879..2a10aaf4ab5 100644 --- a/cdc/sink/producer/kafka/config_test.go +++ b/cdc/sink/producer/kafka/config_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "net/url" + "time" "github.com/Shopify/sarama" "github.com/pingcap/check" @@ -35,7 +36,6 @@ func (s *kafkaSuite) TestNewSaramaConfig(c *check.C) { config.Version = "invalid" _, err := newSaramaConfigImpl(ctx, config) c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*") - ctx = util.SetOwnerInCtx(ctx) config.Version = "2.6.0" config.ClientID = "^invalid$" @@ -84,8 +84,41 @@ func (s *kafkaSuite) TestNewSaramaConfig(c *check.C) { c.Assert(cfg.Net.SASL.Mechanism, check.Equals, sarama.SASLMechanism("SCRAM-SHA-256")) } +func (s *kafkaSuite) TestConfigTimeouts(c *check.C) { + defer testleak.AfterTest(c)() + + cfg := NewConfig() + c.Assert(cfg.DialTimeout, check.Equals, 10*time.Second) + c.Assert(cfg.ReadTimeout, check.Equals, 10*time.Second) + c.Assert(cfg.WriteTimeout, check.Equals, 10*time.Second) + + saramaConfig, err := newSaramaConfig(context.Background(), cfg) + c.Assert(err, check.IsNil) + c.Assert(saramaConfig.Net.DialTimeout, check.Equals, cfg.DialTimeout) + c.Assert(saramaConfig.Net.WriteTimeout, check.Equals, cfg.WriteTimeout) + c.Assert(saramaConfig.Net.ReadTimeout, check.Equals, cfg.ReadTimeout) + + uri := "kafka://127.0.0.1:9092/kafka-test?dial-timeout=5s&read-timeout=1000ms" + + "&write-timeout=2m" + sinkURI, err := url.Parse(uri) + c.Assert(err, check.IsNil) + opts := make(map[string]string) + err = CompleteConfigsAndOpts(sinkURI, cfg, config.GetDefaultReplicaConfig(), opts) + c.Assert(err, check.IsNil) + + c.Assert(cfg.DialTimeout, check.Equals, 5*time.Second) + c.Assert(cfg.ReadTimeout, check.Equals, 1000*time.Millisecond) + c.Assert(cfg.WriteTimeout, check.Equals, 2*time.Minute) + + saramaConfig, err = newSaramaConfig(context.Background(), cfg) + c.Assert(err, check.IsNil) + c.Assert(saramaConfig.Net.DialTimeout, check.Equals, 5*time.Second) + c.Assert(saramaConfig.Net.ReadTimeout, check.Equals, 1000*time.Millisecond) + c.Assert(saramaConfig.Net.WriteTimeout, check.Equals, 2*time.Minute) +} + func (s *kafkaSuite) TestCompleteConfigByOpts(c *check.C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() cfg := NewConfig() // Normal config. From 8dce39fdf6055826302a11e1016dd51e51de353d Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 8 Feb 2022 17:59:35 +0800 Subject: [PATCH 44/72] etcd/client(ticdc): add retry operation for etcd transaction api (#4248) (#4474) close pingcap/tiflow#4248 --- dm/pkg/etcdutil/etcdutil.go | 18 +------ pkg/errorutil/ignore.go | 22 +++++++++ pkg/errorutil/ignore_test.go | 23 ++++++++- pkg/etcd/client.go | 50 ++++++++++++++----- pkg/etcd/client_test.go | 72 ++++++++++++++++++++++++++++ pkg/etcd/etcd.go | 17 +++++-- pkg/orchestrator/etcd_worker.go | 35 ++++++++------ pkg/orchestrator/etcd_worker_test.go | 8 ++++ 8 files changed, 195 insertions(+), 50 deletions(-) diff --git a/dm/pkg/etcdutil/etcdutil.go b/dm/pkg/etcdutil/etcdutil.go index 2907bbe9695..f90b329e391 100644 --- a/dm/pkg/etcdutil/etcdutil.go +++ b/dm/pkg/etcdutil/etcdutil.go @@ -29,6 +29,7 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/pkg/errorutil" ) const ( @@ -48,22 +49,7 @@ var etcdDefaultTxnRetryParam = retry.Params{ FirstRetryDuration: time.Second, BackoffStrategy: retry.Stable, IsRetryableFn: func(retryTime int, err error) bool { - switch err { - // Etcd ResourceExhausted errors, may recover after some time - case v3rpc.ErrNoSpace, v3rpc.ErrTooManyRequests: - return true - // Etcd Unavailable errors, may be available after some time - // https://github.com/etcd-io/etcd/pull/9934/files#diff-6d8785d0c9eaf96bc3e2b29c36493c04R162-R167 - // ErrStopped: - // one of the etcd nodes stopped from failure injection - // ErrNotCapable: - // capability check has not been done (in the beginning) - case v3rpc.ErrNoLeader, v3rpc.ErrLeaderChanged, v3rpc.ErrNotCapable, v3rpc.ErrStopped, v3rpc.ErrTimeout, - v3rpc.ErrTimeoutDueToLeaderFail, v3rpc.ErrGRPCTimeoutDueToConnectionLost, v3rpc.ErrUnhealthy: - return true - default: - return false - } + return errorutil.IsRetryableEtcdError(err) }, } diff --git a/pkg/errorutil/ignore.go b/pkg/errorutil/ignore.go index 6a48dd203f6..05ecf2ad36d 100644 --- a/pkg/errorutil/ignore.go +++ b/pkg/errorutil/ignore.go @@ -19,6 +19,7 @@ import ( tddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/mysql" + v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" ) // IsIgnorableMySQLDDLError is used to check what error can be ignored @@ -46,3 +47,24 @@ func IsIgnorableMySQLDDLError(err error) bool { return false } } + +func IsRetryableEtcdError(err error) bool { + etcdErr := errors.Cause(err) + + switch etcdErr { + // Etcd ResourceExhausted errors, may recover after some time + case v3rpc.ErrNoSpace, v3rpc.ErrTooManyRequests: + return true + // Etcd Unavailable errors, may be available after some time + // https://github.com/etcd-io/etcd/pull/9934/files#diff-6d8785d0c9eaf96bc3e2b29c36493c04R162-R167 + // ErrStopped: + // one of the etcd nodes stopped from failure injection + // ErrNotCapable: + // capability check has not been done (in the beginning) + case v3rpc.ErrNoLeader, v3rpc.ErrLeaderChanged, v3rpc.ErrNotCapable, v3rpc.ErrStopped, v3rpc.ErrTimeout, + v3rpc.ErrTimeoutDueToLeaderFail, v3rpc.ErrGRPCTimeoutDueToConnectionLost, v3rpc.ErrUnhealthy: + return true + default: + return false + } +} diff --git a/pkg/errorutil/ignore_test.go b/pkg/errorutil/ignore_test.go index 7c2dbd0f7e0..825bf7d91b6 100644 --- a/pkg/errorutil/ignore_test.go +++ b/pkg/errorutil/ignore_test.go @@ -20,7 +20,8 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/tidb/infoschema" tmysql "github.com/pingcap/tidb/parser/mysql" - "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" ) func newMysqlErr(number uint16, message string) *mysql.MySQLError { @@ -42,6 +43,24 @@ func TestIgnoreMysqlDDLError(t *testing.T) { } for _, item := range cases { - assert.Equal(t, item.ret, IsIgnorableMySQLDDLError(item.err)) + require.Equal(t, item.ret, IsIgnorableMySQLDDLError(item.err)) + } +} + +func TestIsRetryableEtcdError(t *testing.T) { + cases := []struct { + err error + ret bool + }{ + {nil, false}, + {v3rpc.ErrCorrupt, false}, + + {v3rpc.ErrGRPCTimeoutDueToConnectionLost, true}, + {v3rpc.ErrTimeoutDueToLeaderFail, true}, + {v3rpc.ErrNoSpace, true}, + } + + for _, item := range cases { + require.Equal(t, item.ret, IsRetryableEtcdError(item.err)) } } diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index b346ac6a3cb..bb4f50c197c 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -21,10 +21,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/retry" "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.uber.org/zap" "google.golang.org/grpc/codes" ) @@ -51,6 +52,15 @@ const ( etcdRequestProgressDuration = 1 * time.Second // etcdWatchChBufferSize is arbitrarily specified, it will be modified in the future etcdWatchChBufferSize = 16 + // etcdTxnTimeoutDuration represents the timeout duration for committing a + // transaction to Etcd + etcdTxnTimeoutDuration = 30 * time.Second +) + +var ( + TxnEmptyCmps = []clientv3.Cmp{} + TxnEmptyOpsThen = []clientv3.Op{} + TxnEmptyOpsElse = []clientv3.Op{} ) // set to var instead of const for mocking the value to speedup test @@ -121,12 +131,17 @@ func (c *Client) Delete(ctx context.Context, key string, opts ...clientv3.OpOpti return c.cli.Delete(ctx, key, opts...) } -// Txn delegates request to clientv3.KV.Txn -func (c *Client) Txn(ctx context.Context) clientv3.Txn { - if metric, ok := c.metrics[EtcdTxn]; ok { - metric.Inc() - } - return c.cli.Txn(ctx) +// Txn delegates request to clientv3.KV.Txn. The error returned can only be a non-retryable error, +// such as context.Canceled, context.DeadlineExceeded, errors.ErrReachMaxTry. +func (c *Client) Txn(ctx context.Context, cmps []clientv3.Cmp, opsThen, opsElse []clientv3.Op) (resp *clientv3.TxnResponse, err error) { + txnCtx, cancel := context.WithTimeout(ctx, etcdTxnTimeoutDuration) + defer cancel() + err = retryRPC(EtcdTxn, c.metrics[EtcdTxn], func() error { + var inErr error + resp, inErr = c.cli.Txn(txnCtx).If(cmps...).Then(opsThen...).Else(opsElse...).Commit() + return inErr + }) + return } // Grant delegates request to clientv3.Lease.Grant @@ -144,11 +159,17 @@ func isRetryableError(rpcName string) retry.IsRetryable { if !cerrors.IsRetryableError(err) { return false } - if rpcName == EtcdRevoke { - if etcdErr, ok := err.(rpctypes.EtcdError); ok && etcdErr.Code() == codes.NotFound { - // it means the etcd lease is already expired or revoked + + switch rpcName { + case EtcdRevoke: + if etcdErr, ok := err.(v3rpc.EtcdError); ok && etcdErr.Code() == codes.NotFound { + // It means the etcd lease is already expired or revoked return false } + case EtcdTxn: + return errorutil.IsRetryableEtcdError(err) + default: + // For other types of operation, we retry directly without handling errors } return true @@ -193,7 +214,10 @@ func (c *Client) WatchWithChan(ctx context.Context, outCh chan<- clientv3.WatchR lastRevision := getRevisionFromWatchOpts(opts...) watchCtx, cancel := context.WithCancel(ctx) - defer cancel() + defer func() { + // Using closures to handle changes to the cancel function + cancel() + }() watchCh := c.cli.Watch(watchCtx, key, opts...) ticker := c.clock.Ticker(etcdRequestProgressDuration) @@ -203,7 +227,6 @@ func (c *Client) WatchWithChan(ctx context.Context, outCh chan<- clientv3.WatchR for { select { case <-ctx.Done(): - cancel() return case response := <-watchCh: lastReceivedResponseTime = c.clock.Now() @@ -217,7 +240,6 @@ func (c *Client) WatchWithChan(ctx context.Context, outCh chan<- clientv3.WatchR for { select { case <-ctx.Done(): - cancel() return case outCh <- response: // it may block here break Loop @@ -243,6 +265,8 @@ func (c *Client) WatchWithChan(ctx context.Context, outCh chan<- clientv3.WatchR zap.String("role", role)) cancel() watchCtx, cancel = context.WithCancel(ctx) + // to avoid possible context leak warning from govet + _ = cancel watchCh = c.cli.Watch(watchCtx, key, clientv3.WithPrefix(), clientv3.WithRev(lastRevision)) // we need to reset lastReceivedResponseTime after reset Watch lastReceivedResponseTime = c.clock.Now() diff --git a/pkg/etcd/client_test.go b/pkg/etcd/client_test.go index efa03c6d795..42220d48e9d 100644 --- a/pkg/etcd/client_test.go +++ b/pkg/etcd/client_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" ) type mockClient struct { @@ -43,6 +44,10 @@ func (m *mockClient) Put(ctx context.Context, key, val string, opts ...clientv3. return nil, errors.New("mock error") } +func (m *mockClient) Txn(ctx context.Context) clientv3.Txn { + return &mockTxn{ctx: ctx} +} + type mockWatcher struct { clientv3.Watcher watchCh chan clientv3.WatchResponse @@ -82,6 +87,32 @@ func TestRetry(t *testing.T) { _, err = retrycli.Put(context.TODO(), "", "") require.NotNil(t, err) require.Containsf(t, errors.Cause(err).Error(), "mock error", "err:%v", err.Error()) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Test Txn case + // case 0: normal + rsp, err := retrycli.Txn(ctx, nil, nil, nil) + require.Nil(t, err) + require.False(t, rsp.Succeeded) + + // case 1: errors.ErrReachMaxTry + _, err = retrycli.Txn(ctx, TxnEmptyCmps, nil, nil) + require.Regexp(t, ".*CDC:ErrReachMaxTry.*", err) + + // case 2: errors.ErrReachMaxTry + _, err = retrycli.Txn(ctx, nil, TxnEmptyOpsThen, nil) + require.Regexp(t, ".*CDC:ErrReachMaxTry.*", err) + + // case 3: context.DeadlineExceeded + _, err = retrycli.Txn(ctx, TxnEmptyCmps, TxnEmptyOpsThen, nil) + require.Equal(t, context.DeadlineExceeded, err) + + // other case: mock error + _, err = retrycli.Txn(ctx, TxnEmptyCmps, TxnEmptyOpsThen, TxnEmptyOpsElse) + require.Containsf(t, errors.Cause(err).Error(), "mock error", "err:%v", err.Error()) + maxTries = originValue } @@ -276,3 +307,44 @@ func TestRevisionNotFallBack(t *testing.T) { // while WatchCh was reset require.Equal(t, *watcher.rev, revision) } + +type mockTxn struct { + ctx context.Context + mode int +} + +func (txn *mockTxn) If(cs ...clientv3.Cmp) clientv3.Txn { + if cs != nil { + txn.mode += 1 + } + return txn +} + +func (txn *mockTxn) Then(ops ...clientv3.Op) clientv3.Txn { + if ops != nil { + txn.mode += 1 << 1 + } + return txn +} + +func (txn *mockTxn) Else(ops ...clientv3.Op) clientv3.Txn { + if ops != nil { + txn.mode += 1 << 2 + } + return txn +} + +func (txn *mockTxn) Commit() (*clientv3.TxnResponse, error) { + switch txn.mode { + case 0: + return &clientv3.TxnResponse{}, nil + case 1: + return nil, rpctypes.ErrNoSpace + case 2: + return nil, rpctypes.ErrTimeoutDueToLeaderFail + case 3: + return nil, context.DeadlineExceeded + default: + return nil, errors.New("mock error") + } +} diff --git a/pkg/etcd/etcd.go b/pkg/etcd/etcd.go index bd7d1f12beb..ccad20a2176 100644 --- a/pkg/etcd/etcd.go +++ b/pkg/etcd/etcd.go @@ -304,12 +304,15 @@ func (c CDCEtcdClient) CreateChangefeedInfo(ctx context.Context, info *model.Cha if err != nil { return errors.Trace(err) } - resp, err := c.Client.Txn(ctx).If( + + cmps := []clientv3.Cmp{ clientv3.Compare(clientv3.ModRevision(infoKey), "=", 0), clientv3.Compare(clientv3.ModRevision(jobKey), "=", 0), - ).Then( + } + opsThen := []clientv3.Op{ clientv3.OpPut(infoKey, value), - ).Commit() + } + resp, err := c.Client.Txn(ctx, cmps, opsThen, TxnEmptyOpsElse) if err != nil { return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) } @@ -504,10 +507,14 @@ func (c CDCEtcdClient) PutTaskPositionOnChange( } key := GetEtcdKeyTaskPosition(changefeedID, captureID) - resp, err := c.Client.Txn(ctx).If( + cmps := []clientv3.Cmp{ clientv3.Compare(clientv3.ModRevision(key), ">", 0), clientv3.Compare(clientv3.Value(key), "=", data), - ).Else(clientv3.OpPut(key, data)).Commit() + } + opsElse := []clientv3.Op{ + clientv3.OpPut(key, data), + } + resp, err := c.Client.Txn(ctx, cmps, TxnEmptyOpsThen, opsElse) if err != nil { return false, cerror.WrapError(cerror.ErrPDEtcdAPIError, err) } diff --git a/pkg/orchestrator/etcd_worker.go b/pkg/orchestrator/etcd_worker.go index f5d47146445..d334ca12c1e 100644 --- a/pkg/orchestrator/etcd_worker.go +++ b/pkg/orchestrator/etcd_worker.go @@ -28,6 +28,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.etcd.io/etcd/etcdserver/etcdserverpb" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" @@ -36,9 +37,6 @@ import ( ) const ( - // etcdTxnTimeoutDuration represents the timeout duration for committing a - // transaction to Etcd - etcdTxnTimeoutDuration = 30 * time.Second // When EtcdWorker commits a txn to etcd or ticks its reactor // takes more than etcdWorkerLogsWarnDuration, it will print a log etcdWorkerLogsWarnDuration = 1 * time.Second @@ -205,10 +203,10 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, if len(pendingPatches) > 0 { // Here we have some patches yet to be uploaded to Etcd. pendingPatches, err = worker.applyPatchGroups(ctx, pendingPatches) + if isRetryableError(err) { + continue + } if err != nil { - if cerrors.ErrEtcdTryAgain.Equal(errors.Cause(err)) { - continue - } return errors.Trace(err) } } else { @@ -257,6 +255,18 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, } } +func isRetryableError(err error) bool { + err = errors.Cause(err) + if cerrors.ErrEtcdTryAgain.Equal(err) || + context.DeadlineExceeded == err { + return true + } + // When encountering an abnormal connection with etcd, the worker will keep retrying + // until the session is done. + _, ok := err.(rpctypes.EtcdError) + return ok +} + func (worker *EtcdWorker) handleEvent(_ context.Context, event *clientv3.Event) { if worker.isDeleteCounterKey(event.Kv.Key) { switch event.Type { @@ -351,7 +361,7 @@ func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState m } cmps := make([]clientv3.Cmp, 0, len(changedState)) - ops := make([]clientv3.Op, 0, len(changedState)) + opsThen := make([]clientv3.Op, 0, len(changedState)) hasDelete := false for key, value := range changedState { @@ -373,11 +383,11 @@ func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState m op = clientv3.OpDelete(key.String()) hasDelete = true } - ops = append(ops, op) + opsThen = append(opsThen, op) } if hasDelete { - ops = append(ops, clientv3.OpPut(worker.prefix.String()+deletionCounterKey, fmt.Sprint(worker.deleteCounter+1))) + opsThen = append(opsThen, clientv3.OpPut(worker.prefix.String()+deletionCounterKey, fmt.Sprint(worker.deleteCounter+1))) } if worker.deleteCounter > 0 { cmps = append(cmps, clientv3.Compare(clientv3.Value(worker.prefix.String()+deletionCounterKey), "=", fmt.Sprint(worker.deleteCounter))) @@ -389,10 +399,7 @@ func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState m worker.metrics.metricEtcdTxnSize.Observe(float64(size)) startTime := time.Now() - - txnCtx, cancel := context.WithTimeout(ctx, etcdTxnTimeoutDuration) - resp, err := worker.client.Txn(txnCtx).If(cmps...).Then(ops...).Commit() - cancel() + resp, err := worker.client.Txn(ctx, cmps, opsThen, etcd.TxnEmptyOpsElse) // For testing the situation where we have a progress notification that // has the same revision as the committed Etcd transaction. @@ -411,7 +418,7 @@ func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState m return errors.Trace(err) } - logEtcdOps(ops, resp.Succeeded) + logEtcdOps(opsThen, resp.Succeeded) if resp.Succeeded { worker.barrierRev = resp.Header.GetRevision() return nil diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index 6d77450a012..3a6ef683d3c 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -33,6 +33,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -747,3 +748,10 @@ func TestModifyAfterDelete(t *testing.T) { _ = cli1.Unwrap().Close() _ = cli2.Unwrap().Close() } + +func TestRetryableError(t *testing.T) { + require.True(t, isRetryableError(cerrors.ErrEtcdTryAgain)) + require.True(t, isRetryableError(cerrors.ErrReachMaxTry.Wrap(rpctypes.ErrTimeoutDueToLeaderFail))) + require.True(t, isRetryableError(errors.Trace(context.DeadlineExceeded))) + require.False(t, isRetryableError(context.Canceled)) +} From 7fd749e34d4bc0a9a9be40493e6274c4534f0fb1 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 9 Feb 2022 09:43:35 +0800 Subject: [PATCH 45/72] .github(ticdc): request TiKV version (#4535) close pingcap/tiflow#4534 --- .github/ISSUE_TEMPLATE/ticdc-bug-report.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/ticdc-bug-report.yml b/.github/ISSUE_TEMPLATE/ticdc-bug-report.yml index f02f9b87f90..35ddd4e521c 100644 --- a/.github/ISSUE_TEMPLATE/ticdc-bug-report.yml +++ b/.github/ISSUE_TEMPLATE/ticdc-bug-report.yml @@ -36,6 +36,12 @@ body: (paste TiDB cluster version here) ``` + Upstream TiKV version (execute `tikv-server --version`): + + ```console + (paste TiKV version here) + ``` + TiCDC version (execute `cdc version`): ```console From be7b7421228321ff69c598af0467185f0b89d1e8 Mon Sep 17 00:00:00 2001 From: Ehco Date: Wed, 9 Feb 2022 10:05:35 +0800 Subject: [PATCH 46/72] metrics(dm): fix lag metrics for ddl never updated (#4491) close pingcap/tiflow#4471 --- dm/syncer/syncer.go | 5 +++++ dm/tests/metrics/run.sh | 8 +++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index c1e9be052b6..4cd21535bde 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -866,6 +866,11 @@ func (s *Syncer) updateReplicationLagMetric() { if minTS == s.workerJobTSArray[skipJobIdx].Load() { s.workerJobTSArray[skipJobIdx].Store(0) } + + // reset ddl job TS in case of ddl job TS is never updated + if minTS == s.workerJobTSArray[ddlJobIdx].Load() { + s.workerJobTSArray[ddlJobIdx].Store(0) + } } func (s *Syncer) saveTablePoint(table *filter.Table, location binlog.Location) { diff --git a/dm/tests/metrics/run.sh b/dm/tests/metrics/run.sh index 9ecf992fc75..03395452fac 100755 --- a/dm/tests/metrics/run.sh +++ b/dm/tests/metrics/run.sh @@ -19,7 +19,7 @@ function run() { inject_points=( "github.com/pingcap/tiflow/dm/syncer/BlockDDLJob=return(1)" - "github.com/pingcap/tiflow/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 beacuse we inject BlockDDLJob(ddl) to sleep(1) + "github.com/pingcap/tiflow/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 because we inject BlockDDLJob(ddl) to sleep(1) "github.com/pingcap/tiflow/dm/dm/worker/PrintStatusCheckSeconds=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -70,6 +70,12 @@ function run() { check_metric $WORKER1_PORT 'dm_syncer_replication_lag_sum{source_id="mysql-replica-01",task="test",worker="worker1"}' 5 0 999 check_metric $WORKER2_PORT 'dm_syncer_replication_lag_sum{source_id="mysql-replica-02",task="test",worker="worker2"}' 5 0 999 + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + # check the after ddl query-status lag should be set to 0 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"secondsBehindMaster\": \"0\"" 2 echo "check ddl lag done!" # check new metric dm_syncer_flush_checkpoints_time_interval exists From 1c1015b01d5ad2c738968d763161abfed0b21fc0 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Wed, 9 Feb 2022 21:03:36 +0800 Subject: [PATCH 47/72] sink(cdc): kafka producer use default configuration. (#4359) close pingcap/tiflow#4383, close pingcap/tiflow#4499 --- cdc/model/schema_storage.go | 6 +- cdc/owner/ddl_sink.go | 5 +- cdc/processor/processor.go | 42 ++++----- cdc/sink/mq.go | 40 +++++++-- cdc/sink/producer/kafka/config.go | 55 ++++++------ cdc/sink/producer/kafka/kafka.go | 117 ++++++++++++++++++-------- cdc/sink/producer/kafka/kafka_test.go | 11 ++- cdc/sink/sink.go | 2 + cmd/kafka-consumer/main.go | 1 + pkg/applier/redo.go | 2 + pkg/util/ctx.go | 16 ++++ pkg/util/identity.go | 47 +++++++++++ 12 files changed, 248 insertions(+), 96 deletions(-) create mode 100644 pkg/util/identity.go diff --git a/cdc/model/schema_storage.go b/cdc/model/schema_storage.go index ed61c09e9a6..788b682e498 100644 --- a/cdc/model/schema_storage.go +++ b/cdc/model/schema_storage.go @@ -17,14 +17,12 @@ import ( "fmt" "github.com/pingcap/log" - - "go.uber.org/zap" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/util/rowcodec" + "go.uber.org/zap" ) const ( @@ -139,7 +137,7 @@ func WrapTableInfo(schemaID int64, schemaName string, version uint64, info *mode ti.findHandleIndex() ti.initColumnsFlag() - log.Debug("warpped table info", zap.Reflect("tableInfo", ti)) + log.Debug("warped table info", zap.Reflect("tableInfo", ti)) return ti } diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index 415bfcc5d88..ea89ff101de 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -156,7 +156,10 @@ func (s *ddlSinkImpl) run(ctx cdcContext.Context, id model.ChangeFeedID, info *m err = cerror.ErrExecDDLFailed.GenWithStackByArgs() }) if err == nil || cerror.ErrDDLEventIgnored.Equal(errors.Cause(err)) { - log.Info("Execute DDL succeeded", zap.String("changefeed", ctx.ChangefeedVars().ID), zap.Bool("ignored", err != nil), zap.Reflect("ddl", ddl)) + log.Info("Execute DDL succeeded", + zap.String("changefeed", ctx.ChangefeedVars().ID), + zap.Bool("ignored", err != nil), + zap.Reflect("ddl", ddl)) atomic.StoreUint64(&s.ddlFinishedTs, ddl.CommitTs) continue } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 53199727e5b..2dc10eea710 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -462,6 +462,7 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { stdCtx := util.PutChangefeedIDInCtx(ctx, p.changefeed.ID) stdCtx = util.PutCaptureAddrInCtx(stdCtx, p.captureInfo.AdvertiseAddr) + stdCtx = util.PutRoleInCtx(stdCtx, util.RoleProcessor) p.mounter = entry.NewMounter(p.schemaStorage, p.changefeed.Info.Config.Mounter.WorkerNum, p.changefeed.Info.Config.EnableOldValue) p.wg.Add(1) @@ -1060,16 +1061,18 @@ func (p *processor) Close() error { } p.cancel() p.wg.Wait() - // mark tables share the same cdcContext with its original table, don't need to cancel - failpoint.Inject("processorStopDelay", nil) - resolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - resolvedTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkpointTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkpointTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - processorTickDuration.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + + if p.newSchedulerEnabled { + if p.agent == nil { + return nil + } + if err := p.agent.Close(); err != nil { + return errors.Trace(err) + } + p.agent = nil + } + + // sink close might be time-consuming, do it the last. if p.sinkManager != nil { // pass a canceled context is ok here, since we don't need to wait Close ctx, cancel := context.WithCancel(context.Background()) @@ -1087,15 +1090,16 @@ func (p *processor) Close() error { zap.String("changefeed", p.changefeedID), zap.Duration("duration", time.Since(start))) } - if p.newSchedulerEnabled { - if p.agent == nil { - return nil - } - if err := p.agent.Close(); err != nil { - return errors.Trace(err) - } - p.agent = nil - } + + // mark tables share the same cdcContext with its original table, don't need to cancel + failpoint.Inject("processorStopDelay", nil) + resolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + resolvedTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + checkpointTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + checkpointTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) return nil } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 898cc45a204..cf69bdf70e4 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/notify" "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -75,11 +76,15 @@ type mqSink struct { resolvedReceiver *notify.Receiver statistics *Statistics + + role util.Role + id model.ChangeFeedID } func newMqSink( ctx context.Context, credential *security.Credential, mqProducer producer.Producer, - filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error, + filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, + errCh chan error, ) (*mqSink, error) { var protocol config.Protocol err := protocol.FromString(replicaConfig.Sink.Protocol) @@ -112,6 +117,9 @@ func newMqSink( return nil, errors.Trace(err) } + changefeedID := util.ChangefeedIDFromCtx(ctx) + role := util.RoleFromCtx(ctx) + s := &mqSink{ mqProducer: mqProducer, dispatcher: d, @@ -128,6 +136,9 @@ func newMqSink( resolvedReceiver: resolvedReceiver, statistics: NewStatistics(ctx, "MQ", opts), + + role: role, + id: changefeedID, } go func() { @@ -137,7 +148,8 @@ func newMqSink( return case errCh <- err: default: - log.Error("error channel is full", zap.Error(err)) + log.Error("error channel is full", zap.Error(err), + zap.String("changefeed", changefeedID), zap.Any("role", s.role)) } } }() @@ -159,7 +171,10 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha rowsCount := 0 for _, row := range rows { if k.filter.ShouldIgnoreDMLEvent(row.StartTs, row.Table.Schema, row.Table.Table) { - log.Info("Row changed event ignored", zap.Uint64("start-ts", row.StartTs)) + log.Info("Row changed event ignored", + zap.Uint64("start-ts", row.StartTs), + zap.String("changefeed", k.id), + zap.Any("role", k.role)) continue } partition := k.dispatcher.Dispatch(row) @@ -273,6 +288,8 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { zap.String("query", ddl.Query), zap.Uint64("startTs", ddl.StartTs), zap.Uint64("commitTs", ddl.CommitTs), + zap.String("changefeed", k.id), + zap.Any("role", k.role), ) return cerror.ErrDDLEventIgnored.GenWithStackByArgs() } @@ -300,7 +317,8 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { k.statistics.AddDDLCount() log.Debug("emit ddl event", zap.String("query", ddl.Query), - zap.Uint64("commitTs", ddl.CommitTs), zap.Int32("partition", partition)) + zap.Uint64("commitTs", ddl.CommitTs), zap.Int32("partition", partition), + zap.String("changefeed", k.id), zap.Any("role", k.role)) err = k.writeToProducer(ctx, msg, codec.EncoderNeedSyncWrite, partition) return errors.Trace(err) } @@ -364,7 +382,8 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { return 0, err } } - log.Debug("MQSink flushed", zap.Int("thisBatchSize", thisBatchSize)) + log.Debug("MQSink flushed", zap.Int("thisBatchSize", thisBatchSize), + zap.String("changefeed", k.id), zap.Any("role", k.role)) return thisBatchSize, nil }) } @@ -435,11 +454,15 @@ func (k *mqSink) writeToProducer(ctx context.Context, message *codec.MQMessage, log.Warn("writeToProducer called with no-op", zap.ByteString("key", message.Key), zap.ByteString("value", message.Value), - zap.Int32("partition", partition)) + zap.Int32("partition", partition), + zap.String("changefeed", k.id), + zap.Any("role", k.role)) return nil } -func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { +func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, + filter *filter.Filter, replicaConfig *config.ReplicaConfig, + opts map[string]string, errCh chan error) (*mqSink, error) { producerConfig := kafka.NewConfig() if err := kafka.CompleteConfigsAndOpts(sinkURI, producerConfig, replicaConfig, opts); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) @@ -468,7 +491,8 @@ func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Fi return sink, nil } -func newPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { +func newPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, + replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { producer, err := pulsar.NewProducer(sinkURI, errCh) if err != nil { return nil, errors.Trace(err) diff --git a/cdc/sink/producer/kafka/config.go b/cdc/sink/producer/kafka/config.go index 4f2bd6bfce7..9a3d1e029d1 100644 --- a/cdc/sink/producer/kafka/config.go +++ b/cdc/sink/producer/kafka/config.go @@ -30,10 +30,6 @@ import ( "go.uber.org/zap" ) -func init() { - sarama.MaxRequestSize = 1024 * 1024 * 1024 // 1GB -} - // Config stores user specified Kafka producer configuration type Config struct { BrokerEndpoints []string @@ -261,23 +257,36 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { return nil, errors.Trace(err) } config.Version = version - // See: https://kafka.apache.org/documentation/#replication - // When one of the brokers in a Kafka cluster is down, the partition leaders - // in this broker is broken, Kafka will election a new partition leader and - // replication logs, this process will last from a few seconds to a few minutes. - // Kafka cluster will not provide a writing service in this process. - // Time out in one minute. - config.Metadata.Retry.Max = 120 - config.Metadata.Retry.Backoff = 500 * time.Millisecond - // If it is not set, this means a metadata request against an unreachable - // cluster (all brokers are unreachable or unresponsive) can take up to - // `Net.[Dial|Read]Timeout * BrokerCount * (Metadata.Retry.Max + 1) + - // Metadata.Retry.Backoff * Metadata.Retry.Max` - // to fail. - // See: https://github.com/Shopify/sarama/issues/765 - // and https://github.com/pingcap/tiflow/issues/3352. + + // Producer fetch metadata from brokers frequently, if metadata cannot be + // refreshed easily, this would indicate the network condition between the + // capture server and kafka broker is not good. + // In the scenario that cannot get response from Kafka server, this default + // setting can help to get response more quickly. + config.Metadata.Retry.Max = 1 + config.Metadata.Retry.Backoff = 100 * time.Millisecond + // This Timeout is useless if the `RefreshMetadata` time cost is less than it. config.Metadata.Timeout = 1 * time.Minute + // Admin.Retry take effect on `ClusterAdmin` related operations, + // only `CreateTopic` for cdc now. set the `Timeout` to `1m` to make CI stable. + config.Admin.Retry.Max = 5 + config.Admin.Retry.Backoff = 100 * time.Millisecond + config.Admin.Timeout = 1 * time.Minute + + // Producer.Retry take effect when the producer try to send message to kafka + // brokers. If kafka cluster is healthy, just the default value should be enough. + // For kafka cluster with a bad network condition, producer should not try to + // waster too much time on sending a message, get response no matter success + // or fail as soon as possible is preferred. + config.Producer.Retry.Max = 3 + config.Producer.Retry.Backoff = 100 * time.Millisecond + + // make sure sarama producer flush messages as soon as possible. + config.Producer.Flush.Bytes = 0 + config.Producer.Flush.Messages = 0 + config.Producer.Flush.Frequency = time.Duration(0) + config.Net.DialTimeout = c.DialTimeout config.Net.WriteTimeout = c.WriteTimeout config.Net.ReadTimeout = c.ReadTimeout @@ -287,9 +296,6 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { config.Producer.Return.Successes = true config.Producer.Return.Errors = true config.Producer.RequiredAcks = sarama.WaitForAll - // Time out in five minutes(600 * 500ms). - config.Producer.Retry.Max = 600 - config.Producer.Retry.Backoff = 500 * time.Millisecond switch strings.ToLower(strings.TrimSpace(c.Compression)) { case "none": config.Producer.Compression = sarama.CompressionNone @@ -306,11 +312,6 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { config.Producer.Compression = sarama.CompressionNone } - // Time out in one minute(120 * 500ms). - config.Admin.Retry.Max = 120 - config.Admin.Retry.Backoff = 500 * time.Millisecond - config.Admin.Timeout = 1 * time.Minute - if c.Credential != nil && len(c.Credential.CAPath) != 0 { config.Net.TLS.Enable = true config.Net.TLS.Config, err = c.Credential.ToTLSConfig() diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index e921fa60382..027e277ae92 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -27,10 +27,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/kafka" "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -45,13 +47,15 @@ const ( ) type kafkaSaramaProducer struct { - // clientLock is used to protect concurrent access of asyncClient and syncClient. + // clientLock is used to protect concurrent access of asyncProducer and syncProducer. // Since we don't close these two clients (which have an input chan) from the // sender routine, data race or send on closed chan could happen. - clientLock sync.RWMutex - asyncClient sarama.AsyncProducer - syncClient sarama.SyncProducer - // producersReleased records whether asyncClient and syncClient have been closed properly + clientLock sync.RWMutex + client sarama.Client + asyncProducer sarama.AsyncProducer + syncProducer sarama.SyncProducer + + // producersReleased records whether asyncProducer and syncProducer have been closed properly producersReleased bool topic string partitionNum int32 @@ -68,6 +72,9 @@ type kafkaSaramaProducer struct { closeCh chan struct{} // atomic flag indicating whether the producer is closing closing kafkaProducerClosingFlag + + role util.Role + id model.ChangeFeedID } type kafkaProducerClosingFlag = int32 @@ -93,14 +100,15 @@ func (k *kafkaSaramaProducer) AsyncSendMessage(ctx context.Context, message *cod failpoint.Inject("KafkaSinkAsyncSendError", func() { // simulate sending message to input channel successfully but flushing // message to Kafka meets error - log.Info("failpoint error injected") + log.Info("failpoint error injected", zap.String("changefeed", k.id), zap.Any("role", k.role)) k.failpointCh <- errors.New("kafka sink injected error") failpoint.Return(nil) }) failpoint.Inject("SinkFlushDMLPanic", func() { time.Sleep(time.Second) - log.Panic("SinkFlushDMLPanic") + log.Panic("SinkFlushDMLPanic", + zap.String("changefeed", k.id), zap.Any("role", k.role)) }) select { @@ -108,7 +116,7 @@ func (k *kafkaSaramaProducer) AsyncSendMessage(ctx context.Context, message *cod return ctx.Err() case <-k.closeCh: return nil - case k.asyncClient.Input() <- msg: + case k.asyncProducer.Input() <- msg: } return nil } @@ -131,7 +139,7 @@ func (k *kafkaSaramaProducer) SyncBroadcastMessage(ctx context.Context, message case <-k.closeCh: return nil default: - err := k.syncClient.SendMessages(msgs) + err := k.syncProducer.SendMessages(msgs) return cerror.WrapError(cerror.ErrKafkaSendMessage, err) } } @@ -193,12 +201,13 @@ func (k *kafkaSaramaProducer) stop() { if atomic.SwapInt32(&k.closing, kafkaProducerClosing) == kafkaProducerClosing { return } - log.Info("kafka producer closing...") + log.Info("kafka producer closing...", zap.String("changefeed", k.id), zap.Any("role", k.role)) close(k.closeCh) } // Close closes the sync and async clients. func (k *kafkaSaramaProducer) Close() error { + log.Info("stop the kafka producer", zap.String("changefeed", k.id), zap.Any("role", k.role)) k.stop() k.clientLock.Lock() @@ -210,22 +219,42 @@ func (k *kafkaSaramaProducer) Close() error { return nil } k.producersReleased = true - // In fact close sarama sync client doesn't return any error. - // But close async client returns error if error channel is not empty, we - // don't populate this error to the upper caller, just add a log here. + + // `client` is mainly used by `asyncProducer` to fetch metadata and other related + // operations. When we close the `kafkaSaramaProducer`, TiCDC no need to make sure + // that buffered messages flushed. + // Consider the situation that the broker does not respond, If the client is not + // closed, `asyncProducer.Close()` would waste a mount of time to try flush all messages. + // To prevent the scenario mentioned above, close client first. start := time.Now() - err := k.asyncClient.Close() + if err := k.client.Close(); err != nil { + log.Error("close sarama client with error", zap.Error(err), + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } else { + log.Info("sarama client closed", zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } + + start = time.Now() + err := k.asyncProducer.Close() if err != nil { - log.Error("close async client with error", zap.Error(err), zap.Duration("duration", time.Since(start))) + log.Error("close async client with error", zap.Error(err), + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) } else { - log.Info("async client closed", zap.Duration("duration", time.Since(start))) + log.Info("async client closed", zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) } start = time.Now() - err = k.syncClient.Close() + err = k.syncProducer.Close() if err != nil { - log.Error("close sync client with error", zap.Error(err), zap.Duration("duration", time.Since(start))) + log.Error("close sync client with error", zap.Error(err), + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) } else { - log.Info("sync client closed", zap.Duration("duration", time.Since(start))) + log.Info("sync client closed", zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) } return nil } @@ -233,6 +262,8 @@ func (k *kafkaSaramaProducer) Close() error { func (k *kafkaSaramaProducer) run(ctx context.Context) error { defer func() { k.flushedReceiver.Stop() + log.Info("stop the kafka producer", + zap.String("changefeed", k.id), zap.Any("role", k.role)) k.stop() }() for { @@ -242,16 +273,17 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { case <-k.closeCh: return nil case err := <-k.failpointCh: - log.Warn("receive from failpoint chan", zap.Error(err)) + log.Warn("receive from failpoint chan", zap.Error(err), + zap.String("changefeed", k.id), zap.Any("role", k.role)) return err - case msg := <-k.asyncClient.Successes(): + case msg := <-k.asyncProducer.Successes(): if msg == nil || msg.Metadata == nil { continue } flushedOffset := msg.Metadata.(uint64) atomic.StoreUint64(&k.partitionOffset[msg.Partition].flushed, flushedOffset) k.flushedNotifier.Notify() - case err := <-k.asyncClient.Errors(): + case err := <-k.asyncProducer.Errors(): // We should not wrap a nil pointer if the pointer is of a subtype of `error` // because Go would store the type info and the resulted `error` variable would not be nil, // which will cause the pkg/error library to malfunction. @@ -269,8 +301,13 @@ var ( ) // NewKafkaSaramaProducer creates a kafka sarama producer -func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, opts map[string]string, errCh chan error) (*kafkaSaramaProducer, error) { - log.Info("Starting kafka sarama producer ...", zap.Reflect("config", config)) +func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, + opts map[string]string, errCh chan error) (*kafkaSaramaProducer, error) { + changefeedID := util.ChangefeedIDFromCtx(ctx) + role := util.RoleFromCtx(ctx) + log.Info("Starting kafka sarama producer ...", zap.Any("config", config), + zap.String("changefeed", changefeedID), zap.Any("role", role)) + cfg, err := newSaramaConfigImpl(ctx, config) if err != nil { return nil, err @@ -282,7 +319,8 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o } defer func() { if err := admin.Close(); err != nil { - log.Warn("close kafka cluster admin failed", zap.Error(err)) + log.Warn("close kafka cluster admin failed", zap.Error(err), + zap.String("changefeed", changefeedID), zap.Any("role", role)) } }() @@ -290,11 +328,17 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - asyncClient, err := sarama.NewAsyncProducer(config.BrokerEndpoints, cfg) + client, err := sarama.NewClient(config.BrokerEndpoints, cfg) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - syncClient, err := sarama.NewSyncProducer(config.BrokerEndpoints, cfg) + + asyncProducer, err := sarama.NewAsyncProducerFromClient(client) + if err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + syncProducer, err := sarama.NewSyncProducerFromClient(client) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } @@ -305,10 +349,11 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o return nil, err } k := &kafkaSaramaProducer{ - asyncClient: asyncClient, - syncClient: syncClient, - topic: topic, - partitionNum: config.PartitionNum, + client: client, + asyncProducer: asyncProducer, + syncProducer: syncProducer, + topic: topic, + partitionNum: config.PartitionNum, partitionOffset: make([]struct { flushed uint64 sent uint64 @@ -318,6 +363,9 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o closeCh: make(chan struct{}), failpointCh: make(chan error, 1), closing: kafkaProducerRunning, + + id: changefeedID, + role: role, } go func() { if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { @@ -326,7 +374,8 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o return case errCh <- err: default: - log.Error("error channel is full", zap.Error(err)) + log.Error("error channel is full", zap.Error(err), + zap.String("changefeed", k.id), zap.Any("role", role)) } } }() @@ -380,7 +429,7 @@ func validateAndCreateTopic(admin kafka.ClusterAdminClient, topic string, config } if topicMaxMessageBytes < config.MaxMessageBytes { - log.Warn("topic's `max.message.bytes` less than the user set `max-message-bytes`,"+ + log.Warn("topic's `max.message.bytes` less than the `max-message-bytes`,"+ "use topic's `max.message.bytes` to initialize the Kafka producer", zap.Int("max.message.bytes", topicMaxMessageBytes), zap.Int("max-message-bytes", config.MaxMessageBytes)) @@ -420,7 +469,7 @@ func validateAndCreateTopic(admin kafka.ClusterAdminClient, topic string, config // TiCDC need to make sure that the producer's `MaxMessageBytes` won't larger than // broker's `message.max.bytes`. if brokerMessageMaxBytes < config.MaxMessageBytes { - log.Warn("broker's `message.max.bytes` less than the user set `max-message-bytes`,"+ + log.Warn("broker's `message.max.bytes` less than the `max-message-bytes`,"+ "use broker's `message.max.bytes` to initialize the Kafka producer", zap.Int("message.max.bytes", brokerMessageMaxBytes), zap.Int("max-message-bytes", config.MaxMessageBytes)) diff --git a/cdc/sink/producer/kafka/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go index 79c1c99fdf7..a21412648fa 100644 --- a/cdc/sink/producer/kafka/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/pingcap/tiflow/pkg/kafka" + "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/util/testleak" ) @@ -73,7 +74,7 @@ func (s *kafkaSuite) TestNewSaramaProducer(c *check.C) { metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) metadataResponse.AddTopicPartition(topic, 1, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) - leader.Returns(metadataResponse) + // Response for `sarama.NewClient` leader.Returns(metadataResponse) prodSuccess := new(sarama.ProduceResponse) @@ -108,6 +109,7 @@ func (s *kafkaSuite) TestNewSaramaProducer(c *check.C) { }() opts := make(map[string]string) + ctx = util.PutRoleInCtx(ctx, util.RoleTester) producer, err := NewKafkaSaramaProducer(ctx, topic, config, opts, errCh) c.Assert(err, check.IsNil) c.Assert(producer.GetPartitionNum(), check.Equals, int32(2)) @@ -336,6 +338,7 @@ func (s *kafkaSuite) TestCreateProducerFailed(c *check.C) { NewAdminClientImpl = kafka.NewSaramaAdminClient }() opts := make(map[string]string) + ctx = util.PutRoleInCtx(ctx, util.RoleTester) _, err := NewKafkaSaramaProducer(ctx, topic, config, opts, errCh) c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*") } @@ -352,7 +355,7 @@ func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) metadataResponse.AddTopicPartition(topic, 1, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) - leader.Returns(metadataResponse) + // Response for `sarama.NewClient` leader.Returns(metadataResponse) config := NewConfig() @@ -384,6 +387,7 @@ func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { errCh := make(chan error, 1) opts := make(map[string]string) + ctx = util.PutRoleInCtx(ctx, util.RoleTester) producer, err := NewKafkaSaramaProducer(ctx, topic, config, opts, errCh) c.Assert(opts, check.HasKey, "max-message-bytes") defer func() { @@ -434,7 +438,7 @@ func (s *kafkaSuite) TestProducerDoubleClose(c *check.C) { metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) metadataResponse.AddTopicPartition(topic, 1, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError) - leader.Returns(metadataResponse) + // Response for `sarama.NewClient` leader.Returns(metadataResponse) config := NewConfig() @@ -453,6 +457,7 @@ func (s *kafkaSuite) TestProducerDoubleClose(c *check.C) { errCh := make(chan error, 1) opts := make(map[string]string) + ctx = util.PutRoleInCtx(ctx, util.RoleTester) producer, err := NewKafkaSaramaProducer(ctx, topic, config, opts, errCh) c.Assert(opts, check.HasKey, "max-message-bytes") defer func() { diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index c557c6e8a46..39e81767fb2 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/util" ) // Sink options keys @@ -135,6 +136,7 @@ func Validate(ctx context.Context, sinkURI string, cfg *config.ReplicaConfig, op return err } errCh := make(chan error) + ctx = util.PutRoleInCtx(ctx, util.RoleClient) // TODO: find a better way to verify a sinkURI is valid s, err := New(ctx, "sink-verify", sinkURI, sinkFilter, cfg, opts, errCh) if err != nil { diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 659e106323f..aa748a590ab 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -333,6 +333,7 @@ func NewConsumer(ctx context.Context) (*Consumer, error) { } c.sinks = make([]*partitionSink, kafkaPartitionNum) ctx, cancel := context.WithCancel(ctx) + ctx = util.PutRoleInCtx(ctx, util.RoleKafkaConsumer) errCh := make(chan error, 1) opts := map[string]string{} for i := 0; i < int(kafkaPartitionNum); i++ { diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index f8334159a67..59de745105b 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -111,6 +112,7 @@ func (ra *RedoApplier) consumeLogs(ctx context.Context) error { return err } opts := map[string]string{} + ctx = util.PutRoleInCtx(ctx, util.RoleRedoLogApplier) s, err := sink.New(ctx, applierChangefeed, ra.cfg.SinkURI, ft, replicaConfig, opts, ra.errCh) if err != nil { return err diff --git a/pkg/util/ctx.go b/pkg/util/ctx.go index 492b00f8b4e..4f68af2cf33 100644 --- a/pkg/util/ctx.go +++ b/pkg/util/ctx.go @@ -31,6 +31,7 @@ const ( ctxKeyIsOwner = ctxKey("isOwner") ctxKeyTimezone = ctxKey("timezone") ctxKeyKVStorage = ctxKey("kvStorage") + ctxKeyRole = ctxKey("role") ) // CaptureAddrFromCtx returns a capture ID stored in the specified context. @@ -121,6 +122,21 @@ func PutChangefeedIDInCtx(ctx context.Context, changefeedID string) context.Cont return context.WithValue(ctx, ctxKeyChangefeedID, changefeedID) } +// RoleFromCtx returns a role stored in the specified context. +// It returns RoleUnknown if there's no valid role found +func RoleFromCtx(ctx context.Context) Role { + role, ok := ctx.Value(ctxKeyRole).(Role) + if !ok { + return RoleUnknown + } + return role +} + +// PutRoleInCtx return a new child context with the specified role stored. +func PutRoleInCtx(ctx context.Context, role Role) context.Context { + return context.WithValue(ctx, ctxKeyRole, role) +} + // ZapFieldCapture returns a zap field containing capture address // TODO: log redact for capture address func ZapFieldCapture(ctx context.Context) zap.Field { diff --git a/pkg/util/identity.go b/pkg/util/identity.go new file mode 100644 index 00000000000..5b0abe38a80 --- /dev/null +++ b/pkg/util/identity.go @@ -0,0 +1,47 @@ +// Copyright 2020 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 util + +// Role is the operator role, mainly used for logging at the moment. +type Role int + +const ( + RoleOwner Role = iota + RoleProcessor + RoleClient + RoleRedoLogApplier + RoleKafkaConsumer + RoleTester + RoleUnknown +) + +func (r Role) String() string { + switch r { + case RoleOwner: + return "owner" + case RoleProcessor: + return "processor" + case RoleClient: + return "cdc-client" + case RoleKafkaConsumer: + return "kafka-consumer" + case RoleRedoLogApplier: + return "redo-applier" + case RoleTester: + return "tester" + case RoleUnknown: + return "unknown" + } + return "unknown" +} From 85116e4e6c973c8f82164908bc1041ec56d2fb1a Mon Sep 17 00:00:00 2001 From: Ehco Date: Thu, 10 Feb 2022 11:29:35 +0800 Subject: [PATCH 48/72] openapi(dm): add error_msg in task status (#4421) ref pingcap/tiflow#3583 --- dm/dm/master/openapi.go | 21 ++- dm/dm/master/openapi_test.go | 104 ++++++++----- dm/openapi/gen.server.go | 172 ++++++++++----------- dm/openapi/gen.types.go | 3 + dm/openapi/spec/dm.yaml | 3 + dm/tests/openapi/client/openapi_task_check | 13 ++ dm/tests/openapi/run.sh | 75 ++++----- 7 files changed, 217 insertions(+), 174 deletions(-) diff --git a/dm/dm/master/openapi.go b/dm/dm/master/openapi.go index ada0dee4700..0581bda261d 100644 --- a/dm/dm/master/openapi.go +++ b/dm/dm/master/openapi.go @@ -936,6 +936,16 @@ func getOpenAPISubtaskStatusByTaskName(taskName string, return nil, terror.ErrOpenAPICommonError.New("worker's query-status response is nil") } sourceStatus := workerStatus.SourceStatus + openapiSubTaskStatus := openapi.SubTaskStatus{ + Name: taskName, + SourceName: sourceStatus.GetSource(), + WorkerName: sourceStatus.GetWorker(), + } + if !workerStatus.Result { + openapiSubTaskStatus.ErrorMsg = &workerStatus.Msg + subTaskStatusList = append(subTaskStatusList, openapiSubTaskStatus) + continue + } // find right task name var subTaskStatus *pb.SubTaskStatus for _, cfg := range workerStatus.SubTaskStatus { @@ -947,14 +957,9 @@ func getOpenAPISubtaskStatusByTaskName(taskName string, // not find continue } - openapiSubTaskStatus := openapi.SubTaskStatus{ - Name: taskName, - SourceName: sourceStatus.GetSource(), - WorkerName: sourceStatus.GetWorker(), - Stage: subTaskStatus.GetStage().String(), - Unit: subTaskStatus.GetUnit().String(), - UnresolvedDdlLockId: &subTaskStatus.UnresolvedDDLLockID, - } + openapiSubTaskStatus.Stage = subTaskStatus.GetStage().String() + openapiSubTaskStatus.Unit = subTaskStatus.GetUnit().String() + openapiSubTaskStatus.UnresolvedDdlLockId = &subTaskStatus.UnresolvedDDLLockID // add load status if loadS := subTaskStatus.GetLoad(); loadS != nil { openapiSubTaskStatus.LoadStatus = &openapi.LoadStatus{ diff --git a/dm/dm/master/openapi_test.go b/dm/dm/master/openapi_test.go index 3be3e748246..9638136fa99 100644 --- a/dm/dm/master/openapi_test.go +++ b/dm/dm/master/openapi_test.go @@ -536,7 +536,7 @@ func (t *openAPISuite) TestTaskAPI(c *check.C) { // get task status mockWorkerClient := pbmock.NewMockWorkerClient(ctrl) - mockTaskQueryStatus(mockWorkerClient, task.Name, source1.SourceName, workerName1) + mockTaskQueryStatus(mockWorkerClient, task.Name, source1.SourceName, workerName1, false) s.scheduler.SetWorkerClientForTest(workerName1, newMockRPCClient(mockWorkerClient)) taskStatusURL := fmt.Sprintf("%s/%s/status", taskURL, task.Name) result = testutil.NewRequest().Get(taskStatusURL).GoWithHTTPHandler(t.testT, s.openapiHandles) @@ -576,6 +576,21 @@ func (t *openAPISuite) TestTaskAPI(c *check.C) { c.Assert(status.WorkerName, check.Equals, workerName1) c.Assert(status.Name, check.Equals, task.Name) + // test some error happened on worker + mockWorkerClient = pbmock.NewMockWorkerClient(ctrl) + mockTaskQueryStatus(mockWorkerClient, task.Name, source1.SourceName, workerName1, true) + s.scheduler.SetWorkerClientForTest(workerName1, newMockRPCClient(mockWorkerClient)) + result = testutil.NewRequest().Get(taskURL+"?with_status=true").GoWithHTTPHandler(t.testT, s.openapiHandles) + c.Assert(result.Code(), check.Equals, http.StatusOK) + c.Assert(result.UnmarshalBodyToObject(&resultListTask), check.IsNil) + c.Assert(resultListTask.Data, check.HasLen, 1) + c.Assert(resultListTask.Total, check.Equals, 1) + c.Assert(resultListTask.Data[0].StatusList, check.NotNil) + statusList = *resultListTask.Data[0].StatusList + c.Assert(statusList, check.HasLen, 1) + status = statusList[0] + c.Assert(status.ErrorMsg, check.NotNil) + // stop task result = testutil.NewRequest().Delete(fmt.Sprintf("%s/%s", taskURL, task.Name)).GoWithHTTPHandler(t.testT, s.openapiHandles) c.Assert(result.Code(), check.Equals, http.StatusNoContent) @@ -840,49 +855,62 @@ func mockRelayQueryStatus( } func mockTaskQueryStatus( - mockWorkerClient *pbmock.MockWorkerClient, taskName, sourceName, workerName string) { - queryResp := &pb.QueryStatusResponse{ - Result: true, - SourceStatus: &pb.SourceStatus{ - Worker: workerName, - Source: sourceName, - }, - SubTaskStatus: []*pb.SubTaskStatus{ - { - Stage: pb.Stage_Running, - Name: taskName, - Status: &pb.SubTaskStatus_Sync{ - Sync: &pb.SyncStatus{ - TotalEvents: 0, - TotalTps: 0, - RecentTps: 0, - MasterBinlog: "", - MasterBinlogGtid: "", - SyncerBinlog: "", - SyncerBinlogGtid: "", - BlockingDDLs: nil, - UnresolvedGroups: nil, - Synced: false, - BinlogType: "", - SecondsBehindMaster: 0, + mockWorkerClient *pbmock.MockWorkerClient, taskName, sourceName, workerName string, needError bool) { + var queryResp *pb.QueryStatusResponse + if needError { + queryResp = &pb.QueryStatusResponse{ + Result: false, + Msg: "some error happened", + SourceStatus: &pb.SourceStatus{ + Worker: workerName, + Source: sourceName, + }, + } + } else { + queryResp = &pb.QueryStatusResponse{ + Result: true, + SourceStatus: &pb.SourceStatus{ + Worker: workerName, + Source: sourceName, + }, + SubTaskStatus: []*pb.SubTaskStatus{ + { + Stage: pb.Stage_Running, + Name: taskName, + Status: &pb.SubTaskStatus_Sync{ + Sync: &pb.SyncStatus{ + TotalEvents: 0, + TotalTps: 0, + RecentTps: 0, + MasterBinlog: "", + MasterBinlogGtid: "", + SyncerBinlog: "", + SyncerBinlogGtid: "", + BlockingDDLs: nil, + UnresolvedGroups: nil, + Synced: false, + BinlogType: "", + SecondsBehindMaster: 0, + }, }, }, - }, - { - Stage: pb.Stage_Running, - Name: taskName, - Status: &pb.SubTaskStatus_Dump{ - Dump: &pb.DumpStatus{ - CompletedTables: 0.0, - EstimateTotalRows: 10.0, - FinishedBytes: 0.0, - FinishedRows: 5.0, - TotalTables: 1, + { + Stage: pb.Stage_Running, + Name: taskName, + Status: &pb.SubTaskStatus_Dump{ + Dump: &pb.DumpStatus{ + CompletedTables: 0.0, + EstimateTotalRows: 10.0, + FinishedBytes: 0.0, + FinishedRows: 5.0, + TotalTables: 1, + }, }, }, }, - }, + } } + mockWorkerClient.EXPECT().QueryStatus( gomock.Any(), gomock.Any(), diff --git a/dm/openapi/gen.server.go b/dm/openapi/gen.server.go index 2ec86b20155..aa4dcc064d5 100644 --- a/dm/openapi/gen.server.go +++ b/dm/openapi/gen.server.go @@ -983,92 +983,92 @@ func RegisterHandlersWithOptions(router *gin.Engine, si ServerInterface, options // Base64 encoded, gzipped, json marshaled Swagger object var swaggerSpec = []string{ - "H4sIAAAAAAAC/+w9W3Pbtpp/Bcvdh7YjWZLtOIl3zoNjuznetZ2MrU73TCerQCQk4ZgEaAC0q5Pxfz+D", - "C0mQBCjKt1iN+9C6Ii4fvvsN5LcgpElKCSKCB/vfAh4uUALVn4dxxgViZ1D+W/6QMpoiJjBSj2EUqV8j", - "xEOGU4EpCfbVr4hzQGdALBAIM8YQESBRiwBCIxT0AvQnTNIYBfvBaPvt1nBruDXaf7e9Nwp6gVim8ncu", - "GCbz4K4XwBjfoOY+lMSYIMAFFJnZDXOzjb2DYBkqVp1SGiNI5LIxghFywI+5vZI6gxnaYVECEwVqeT69", - "jONgd72AoesMMxQF+3/omflhC+h6Gslfitl0+k8UCrmVIc7vlF19R+JMaUaiCacZC9EkP311TzUE6CFA", - "DimIdatgb26bLPl13B+2bSjg3L+VfLhyEzXWtUOThnqJ7jSUqK9C6kKUk6gMQYHGkF9doOsMcdEkLEMJ", - "vUGTBAmoETCDWSyC/RmMOerVEHK7QGIhuZgCPQ/IeSCCAk4hRwATENFbwgVDMCl+DlysbYE+ibGG7L8Y", - "mgX7wX8OShUyMPpjcKnGn8MEncrRd71AQH61apY8egOv9pHNMi7kHaEYCaT3vUA8pYSjJv7k9O6nkPCU", - "Z7hz7Zol6aVSQk1+LJVTlCUpyAgWQa8Gj9xUwh1NBJzG+rcZZQkUwX4Q0WwaW/QgWTJFTG6LuMAJFGgi", - "qIDxhNHbrjNnmGC+QNFkuhRo7UlrbKQhc5wKE7G3W87ARKC5nFIje2V+r4moxlHqYLqx5GKdY8Yo+x2L", - "xRni3KlaJMmg/BsgObZBRvXrJJRapjFXPQOh1kD1Q/fM1ITPfTMTA9Qq/VMu1LPhcR34IxLGgJyQGfVL", - "S6gHTXDUBM48A1iq0YK4WUfqWiu3A6jdDymAfjCl5pL/xQIlfJVMV92aUqYhY3BZMK7SFR0YNOjp3dsP", - "oc304x/CmP8nPoTWqY8IvV7wecDWyvlRATf6/onBl7bnEXGuTevTg/y4+M6m5ZrPAf1Y2pZLwbJQZKzF", - "jdAATkLlsE34dVx1GQ8vjg/Gx2B88OH0GHwVo6/gp684+gowET+NRj+D809jcP7b6Sk4+G38aXJyfnhx", - "fHZ8Pu59vjg5O7j4B/jf43/oGT+DwS/j//jDaEsUTTCJ0J9fwOHpb5fj44vjI/DL4GdwfP7x5Pz4byeE", - "0KMP4Oj414PfTsfg8O8HF5fH479lYvYume6Cw0+npwfj4/z/J1NMXA6wOVrTD46mTpdcmWPHcPX7aq/Z", - "mp6vZWHVRapTCqPVLldMYeR2uVo8IJ/x6gXS+5QYi+ncYrcSC9bzyVxoe9kYlDI6lwGY86H2UbrDVMNj", - "wxmy17O2rh7FAbgL5Z+U+4NcElLEKbVQM1TekqBAu04IKNoCZiY0iBJnfFGJaHSQXV31d4YF4ip41Wwq", - "N1Ch7AKFVynFRAAuf4ECHJ2BEBLNB1gAOJPOCkNcQCYwmatpKphwhjvX8SSkRCDiOBu/jsGSZuAWEmGd", - "sBJfOjQA+BqOShWQS6lUAz3wNdz2P9pxP3qA3P+3U/CXJGwe9rc0gjnOaSpwgrnAIeALyCKJRsk/UquC", - "WywWOuY2pKEkXoKMowjcLhAB0PjOgIZhxrgMPn1rHh2dgqTiLxekqXG9TScX437OmMudZyiGSxDTOQjl", - "slkKUhrjcAlCSmZ4nmlfv+nl/5liZsxYzqbDOo+qQTpWEFinO4rtbE85l2uSxbEUjVpaydI98k92o+1c", - "se/O3rCx9XghY3o9WDJmihimEQ5hHC+1iACsUz8lAjAH+lhRD5jFwQ2MM7QP1BaSThyFlET8ftAzlEBM", - "JjyFIaqcYPSmDv8ZJjjJEjBjCIEI8yugZikYPn64z/aukP1Cnv1QEbrJGUqZqGcF4ZpsQJSZ0g9LVW6p", - "D/Wobg4alkrroY/jk6M8I5elJhdTqOdSo6D3cDQLt7f7KBy+649G6H1/ug3D/nB7dxuGo9FwONzZH/Xf", - "vtt970dMKe0VEN2puwLEGY5RmbprB1Nn76aYbA3lP9vdYYkwq/BHsDXQD/QWTTpFmKFQULaUCoahJmNz", - "QaWeWAmBl0tWuxm2aFe5RGdVLZ+hukQNhwrHABPN4Vr5lEj9qYbVUQ+M3r99/7NLjVf29TCfi+cewGzt", - "zOUGQSMuT0FLgB4fgBCKcDHJ0klS1DC8CVI1FmSptmMFdSy/ySfmBd/enz/Lc28NeDZVS7ostDvvnSNR", - "c2VluYuMEDl5lRdeZVYnE9nHdVHYh/QcbJd5vlSeQpFhbcqZ9iSU7lH52l4ZN64OTGqx4iUKM4bFsrmN", - "8l9MiYLzuOoF9BTdZhjFEbjFcQymCCxwFCGi/Zo5EoU/aS9UWQTMGE3UEGWfZ9IWNtVSLfmGmJjAOKa3", - "KJqEpAn2IU0SSsC50cyXl6dAzsEzHELt9RfIWokczuNJCP0+r7WwVlX5SJvbnDwrF5Yn8S79q7WcPMfn", - "4zOg1eDg/94M35u/60dbvesVWvo3PSz3k1RJGb6RR7tCy1wTA2vzFfvVndIqLh04aALolA7jD39kNEsd", - "mZQoLooY3Qk9w4yLSUxDbWVcU2QggKL1lhWQzZFwDs3I+gs2kgRq9V555sZBCrCtDZ1I1bnHpqrRv3t8", - "vdKGday3ZRxpz0664ZnUGkpVcq0IXDbXrNi0MgvqVI1ay5iSrrtu65KMFHJ+S1nkXbEYUF1yZ/fNnnM9", - "yvzQqYfWOjs7wz1XjJDmYVpbMlDHcqVxLzz4tkm2sy9Z27IBrYnHfFy19Ok9qMlfdStiazvdlN2HZJ4z", - "7vJvDHTyYQNCRqlYrcqssxtONCQ3W1oM1asIi1/2Wsy91SXgN/d6VL+bzbfR5tuv8JtcNb3VhTntBnCa", - "ILGQjsAtoy6PK+dbXgCzkm9Lcj+ABxlKYxxCDy/q5gbPwuMFyhsojI8ZL4HusjB5t0Jr1tsl+qM1ecsG", - "xMk7MhhVWOmQc1SRK4DG2/blHF+j9I2K0is80ql9Q5dHKw0cNgM2lnPzHU27sx1NV3LddzlEpY7W9CSz", - "JO2ol6x+l7teEFMYdZxoVW2sLqsau0B+lauxptXsrgLvETjPC7ExNQHLUGbuEFp7nB2Pf7kkYXl8VZdy", - "H18+AmonGwaVm++5fGuGOI1vUDRRrjENryae4lOrps/b5Jz4c/e5+dV3jm9zTidDluhoSa7JU4Msc9Tw", - "jEbU6zoOO5WYwGQuseLawq403C5wuCgyUZiDfPJaAXQj3dcxMedQsyEiYiLSrqVJk52fTNECk8jKdXWZ", - "W0RmjhqYfNZ6osoI/4l0JRLd5D3GHeAyzV+dcWDJwVxGy2001wNqZIcMgYz081Vs0reKdSVEXxnG2oiw", - "D1mheq9bNq5KHicx6nLgwpMVN9tC5WMrlzCrkvBDk3i+doGmpI1NI2lTefrUxAzHEn8si5FpjsZyFow/", - "V0avap/5gMkpnf+qFruQa7nqBogsIAnRRDeoT/JGkQUkc7Syvm0lEHQsBXiWynALzKhuRjd971EUgzTO", - "5ph06UtXNX4NSdV3i5K+aautJUSbXcEKAumq5UVfb7GiXNTbXO03+zZD8Ct3kEfJJMpUUCMcqy3orcTf", - "ApJI5xVnMQ4FitRJVJCaJVIY6Q1itwzrur1q6vziMvFSwCeJs7FT0uMWLlX1gFKpB6BA0qRYu6SIc1Pf", - "DnpBWex2b6ZNarfMhnLo1AQrvXGfzMLKHisV1Cd4zqBAhRDVSSiZ1YwBakyve1+aUiBnenJNsGoJxjVw", - "M1YTjqCAHyBHeQe6h5Q55KbrIKfeLItjeRASMpQgonvIYKz6kkpOhWpQJ6epBGGFpqhxef38TqrUGcit", - "qx16zJWSF0hJulyYAyjyMmWMblDc0LN4TihD2rI5siXy59ynLZiiZUwFtSBK4i5mwcBgevGaHTspFAIx", - "Fadpe+AHxje8hOv/jxhNV0N156HAr1kcG36XwuuITCrFIzoDkhML+ZJcxB1XCQjHXCASOkpcSkcRwWgM", - "crWFifGBVNVK94FQJjXlTPWrF6sByHnGJK9WaZMJ6kKBXM5dFJXmQ0ZaEWZNfb81yPefGE3dWFkPmIgF", - "QzCqtuHs1k2YQpieIPEXUmJcPaf/iBPvyqM959J6xsqlfRxwQkK2HgdYSsjDAAyl8WQKRVhtpBs1G4Xs", - "taT7t2CU4H8VW6k1APoThZn6ScrDdQaJwGord5dPGndEX/0g98ah3+UsPIpWh9PnX7gcztLSNtMntfin", - "3GK4MwuH23s7/e134dv+aITe9uHem53+XjicvtuN3ryf7Qz3R/23w93R7vZOb/hm9+1utBNaw9/tvNnu", - "bw93oun27l4U7UT7o/7o7dB5Na6aJLSuuqkHpnGkZWZKqxjadQaMT5OIbkkN+6xYxffxgNJnKIZSo7X3", - "9UmBLkxpaGi8yr+o6/A77SesvU5dE1T9QC+S6yfq7GxZnLwqXrXh8JGh4bv5O520kyiofenQdhl5x/it", - "po3VQ7VAznkOaZePu0k7by0Qd+QoO9jyxMI9cIvjKIQsyoO8ahQ17f/ywDRoo2TmS48KnXV3O/UdYBVO", - "WFvLPQZB+d4u7ipbCnzB6WMSI6KIA0JFEXHnJ+Y1sozuicGOG4hpB/W4CnlO1LeIcCVSakF4mQ1ox/gm", - "tiys17Fwn0aCJ6rRt1flvURHSSqFx3vTvMyPrNP3UszSrp0wuxR/rG7pL/ddDbrvetYM4ljdFOZXzWRI", - "S5XfIdfmurjjaaNHKR9q3+l1Kra6xcnCEHHuAXe9TqnmWr0mNlxA1cqMbRWiFqfaX/xvHrvc0VtaNTVU", - "DnJFL6hpSOBtddVV9a17NCu0tyfcqeBUSAmOj2joSCkcnYFPKSIHn0/A0adDKacsDvaDhRAp3x8MIhry", - "rRSTeQjTrZAmg38tBgJH075UuH3tJGFKBlxrfOVrzqhiDyzUSRob3CDG9d5vtna2hipxmiICUxzsBztS", - "2yo1IRYK2gFM8eBmNDA3DQf58sYCF/ffTyK118Hnk+odclXC0OKo1tseDk1OIu+/hKlOZslj/JPr1sPS", - "MrfpUM9tdYX1mi7V3K+ox7MkgWwZ7MszgOK2OplRwLNwASAHlSvsAs65dTM9+CIXqaNFV0F4V8yUl9ef", - "Bz+Oy/JtWOoFu48IRuMtCo6ttSpqoY/1LppczaxDmME3/Ydyde+0GMZI2zAHpT7NZjEmSKPtXKdlU8hg", - "gjSV/2jkiS3w8mBD/i7lKMjrG4EFQ2CrEV2fKbHZ5T1BXxqMs+vwIV4YRanGa+3NQp0Imav3jhJWvlnh", - "eSTM8SaHDZMw641Ia0mYIczgm7GZa0mYsfUdJMwGzy9hFgw/toRV32/VSsgo2cqBc0rWRySOaPg/l5/O", - "PaJUBUuuVdwCabJbREOgtiuhimhYg8i4Si3g/H18dtoJHDlwBTgLoStJPnB0JLZa9ZTvQ1nFzHJnE9+p", - "a2VFp7Fi6esMsaXF01gsJsUIBw+7y/sO/n1Uxed4+4uDSe2bT3Hep1gjQX1ISYo8QaGCc+5DvX492mXe", - "cmyCgw80Wj7aefM30zQPaHYDU7ndXQPlo2cA4aXpIP2eDkDQrU1bF1mbQjb4ZuUkV5sR++VuK4UuplN1", - "4z0j+DqrXsrzW5RqirSTRfFe8bjrNZLUVF80oKnOi8CYm3bhvB1aRbemrOfSDmqFB+qF3UfjGefL9jaA", - "ZTWTAfhQhh2kMOO6GKCUT4vW+ixHXuQvEnjhjPuli6l9aURVtLDuFMwyolvy86a5hxKbIZ4l3ah9oYa+", - "kvsJya2p8ZT0tl6C3MER1JfYu7iDT0Bc/2W4J/ULaxf3NyQEzu/76bqfzwftyh6Db/qP0oXpwCyqXP7y", - "eKXXUhv1bF+eveP2ztLpk3JptTF9s5hUl47vz6MCMtHJYpUXKzfFYD1B2Ne4XHpXLQBJYO820ViaawRP", - "aSyLa1xdbGVx1frlMFprX9qzJFdq70zdEEVlf0LA/g7DY7AUTTvqLnM590dWXbX7yX8VzRVh/tSqSzBI", - "+Mx8P8PPZWMzbGPST0/Eas2Gjb8Kr+WMUDhfFED9FkpdX1nBXTpvt8oC5i/07lI0UEnCjS0ZNF5d7qCD", - "OmFsvg3ycmxaAVVJcf3FkfbShHIgx/oK6lOIXvPLMN+zRGE+07IpBQqovwrERPG+5ypl65I8yJsVu8l0", - "3o74DE0IGy5YRTfoPSSslIBx2Ur6FKLmY+5X6XJLV4Wy6wjXQF+hW+F8nahBz0T3elP0+myw/UTwbE5o", - "aC5G3p8tvqmbeevUhWvcsZZ7br98wOGXF7B09Mp9two3s8vIlEv9rfx1Bd7ZWG4OmYY/nGJv2us2kqeZ", - "h+T6yxGvRN8MomeKWp3p3tDf99PaL5Ujel1eTeoIyBufybT3fdC7TDfagOgIzDQ/rcdMutOmS4/NS+an", - "L0/ZrmhXOO82t4HnHryhW0E6teS8csemcofp97kHezywu6fo6/mwlNxzQKL7VQBegtF67Tf6Xk50a9PR", - "g7l4zSakov3olaVf26I2VpacvVGPLEpy3jRGa0Y09rcxX2XqhclUz/96Fh/Kcw7ojHPPN283PvtXkTxu", - "sfi6KcBXCXmVkO/QaNfyce+NNYCtYuhNy54UH3N+FcW1N/9RBPHxkxErPyH+V+klK793voa8tnut3Tqs", - "rbd4/0hZ9bUyYM9gZTa0mVtxa849de5UL7xjNzk3Vd9dtaTZVkQTiIl6c1UgkWwW8H5ksf1lWRENH/iG", - "rMF1hsOrvr4FoxtV+mbzuxpbBS5laz4B9CxAGvCKp321/V1F/BxA5u/4KMblP9x9uft3AAAA///Luz+r", - "1pEAAA==", + "H4sIAAAAAAAC/+w9W3Pbtpp/Bavdh7YjWZLtOIl3zoNjuznetZ2MrU73TCerQCQk4ZgEaAC0q5Pxfz+D", + "C0mQBEjKt1iN+9C6Igh8+O43gN96AY0TShARvLf/rceDJYqh+vMwSrlA7AzKf8sfEkYTxARG6jEMQ/Vr", + "iHjAcCIwJb199SviHNA5EEsEgpQxRASI1SSA0BD1+j30J4yTCPX2e+Ptt1ujrdHWeP/d9t641++JVSJ/", + "54Jhsujd9Xswwjeovg4lESYIcAFFalbD3CxjryBYivJZZ5RGCBI5bYRgiBzwY27PpPZghnaYlMBYgVrs", + "T0/j2Nhdv8fQdYoZCnv7f+g3s83m0PU1kr/kb9PZP1Eg5FKGOL9TdvUdiTOjKQmnnKYsQNNs9+U11RCg", + "hwA5JCfWrYK9vmy84tfRYNS0oIAL/1LyYesiaqxrhToN9RTdaShRX4bUhSgnURmCAk0gv7pA1yniok5Y", + "hmJ6g6YxElAjYA7TSPT25zDiqF9ByO0SiaXkYgr0e0C+B0Io4AxyBDABIb0lXDAE4/znnou1LdCnEdaQ", + "/RdD895+7z+HhQoZGv0xvFTjz2GMTuXou35PQH7V9pbceg2v9pbNNC7kHaEICaTXvUA8oYSjOv7k6913", + "IeEp9nDnWjWNk0ulhOr8WCinMI0TkBIsev0KPHJRCXc4FXAW6d/mlMVQ9PZ7IU1nkUUPksYzxOSyiAsc", + "Q4GmggoYTRm97frmHBPMlyiczlYCrf3SGgtpyBy7wkTs7RZvYCLQQr5SIXvp/X4dUbWtVMF0Y8nFOseM", + "UfY7FsszxLlTtUiSQfk3QHJsjYzq12kgtUztXfUMBFoDVTfdN6/GfOF7MzZAtemfYqK+DY9rwx+RMAbk", + "hMypX1oCPWiKwzpw5hnAUo3mxE07UteauRlA7X5IAfSDKTWX/C8WKOZtMl12awqZhozBVc64Sld0YNBe", + "X6/evAltph9/E8b8P/EmtE59ROj1hM8DtlbOjwq40fdPDL60PY+Ic21anx7kx8V3OivmfA7oJ9K2XAqW", + "BiJlDW6EBnAaKIdtyq+jsst4eHF8MDkGk4MPp8fgqxh/BT99xeFXgIn4aTz+GZx/moDz305PwcFvk0/T", + "k/PDi+Oz4/NJ//PFydnBxT/A/x7/Q7/xMxj+MvmPP4y2ROEUkxD9+QUcnv52OTm+OD4Cvwx/BsfnH0/O", + "j/92Qgg9+gCOjn89+O10Ag7/fnBxeTz5Wyrm7+LZLjj8dHp6MDnO/n86w8TlAJut1f3gcOZ0yZU5dgxX", + "v7d7zdbr2VwWVl2kOqUwbHe5IgpDt8vV4AH5jFe/J71PibGILix2K7BgPZ8uhLaXtUEJowsZgDkfah+l", + "O0wVPNacIXs+a+nyVhyAu1D+Sbk/yCUheZxSCTUD5S0JCrTrhICiLWDmhRpRopQvSxGNDrLLs/7OsEBc", + "Ba+aTeUCKpRdouAqoZgIwOUvUICjMxBAovkACwDn0llhiAvIBCYL9ZoKJpzhznU0DSgRiDj2xq8jsKIp", + "uIVEWDssxZcODQC+BuNCBWRSKtVAH3wNtv2PdtyPHiD3/+0U/BUJ6pv9LQlhhnOaCBxjLnAA+BKyUKJR", + "8o/UquAWi6WOuQ1pKIlWIOUoBLdLRAA0vjOgQZAyLoNP35xHR6cgLvnLOWkqXG/TycW4n1PmcucZiuAK", + "RHQBAjltmoCERjhYgYCSOV6k2teve/l/JpgZM5ax6ajKo2qQjhUE1umOfDnbU87kmqRRJEWjklaydI/8", + "k91oO5evu7M3qi09WcqYXg+WjJkghmmIAxhFKy0iAOvUT4EAzIHeVtgHZnJwA6MU7QO1hKQTRwElIb8f", + "9AzFEJMpT2CASjsYv6nCf4YJjtMYzBlCIMT8Cqi3FAwfP9xneVfIfiH3fqgIXecMpUzUs5xwdTYgykzp", + "h4Uqt9SHelQ1BzVLpfXQx8nJUZaRSxOTi8nVc6FR0Hs4ngfb2wMUjN4NxmP0fjDbhsFgtL27DYPxeDQa", + "7eyPB2/f7b73I6aQ9hKI7tRdDuIcR6hI3TWDqbN3M0y2RvKf7e6whJiV+KO3NdQP9BJ1OoWYoUBQtpIK", + "hqE6Y3NBpZ5ohcDLJe1uhi3aZS7RWVXLZyhPUcGhwjHARHO4Vj4FUn+qYHXcB+P3b9//7FLjpXU9zOfi", + "uQcwWzNzuUHQiMtS0BKgxwcggCJYTtNkGuc1DG+CVI0FaaLtWE4dy2/yiXnOt/fnz2LfW0OeztSULgvt", + "zntnSNRcWZruIiVEvtzmhZeZ1clE9nZdFPYhPQPbZZ4vlaeQZ1jrcqY9CaV7VL62X8SN7YFJJVa8REHK", + "sFjVl1H+iylRcB6VvYC+otscoygEtziKwAyBJQ5DRLRfs0Ai9yftiUqTgDmjsRqi7PNc2sK6Wqok3xAT", + "UxhF9BaF04DUwT6kcUwJODea+fLyFMh38BwHUHv9ObJakcN5NA2g3+e1JtaqKhtpc5uTZ+XEcifeqX+1", + "ppP7+Hx8BrQaHP7fm9F783d1a+2rXqGVf9HDYj1JlYThG7m1K7TKNDGwFm9Zr+qUlnHpwEEdQKd0GH/4", + "I6Np4sikhFFexOhO6DlmXEwjGmgr43pFBgIoXG9aAdkCCefQlKw/YS1JoGbvF3uubSQH21rQiVSde6yr", + "Gv27x9crbFjHelvKkfbspBueSq2hVCXXisBlc82MdSuzpE7VqLWMKem667YuyUgg57eUhd4Z8wHlKXd2", + "3+w556PMD516aM2zszPac8UISRamNSUDdSxXGPfcg296yXb2JWtbNqAx8ZiNK5c+vRs1+atuRWxtp+uy", + "+5DMc8pd/o2BTj6sQcgoFe2qzNq74URDcrOkxVD9krD4Za/B3FtdAn5zr0cNutl8G22+9XK/yVXTay/M", + "aTeA0xiJpXQEbhl1eVwZ3/IcmFa+Lcj9AB5kKIlwAD28qJsbPBNPlihroDA+ZrQCusvC5N1yrVltlxiM", + "1+QtGxAn78hgVGGlQ85RRa4AGm/bl3N8jdI3Kkov8Uin9g1dHi01cNgMWJvOzXc06c52NGnluu+yiVId", + "re5JpnHSUS9Z/S5r9C50VpERhWFHSKwykNW2VeE/yK8yvVg3w9116j0i8UUuh6bIYFne1B2Taxe24/Yv", + "VyQotq8KXe7ty0dArWTDoJL9fZezzhCn0Q0Kp8rXpsHV1FPNajQdWd+dE3/uxjm/Pcjwbfbp5PACHQ3Z", + "OrlrkKaOoqBRsXpex2ZnEhOYLCRWXEvYpYvbJQ6WeWoLc5C9vFZEXssfdsz0OfR2gIiYiqRrrdOk+6cz", + "tMQktJJnXd7NQz1HUU0+a9xRaYR/R7q0iW6ypuUOcJluss44sORgIcPvJprrARWyQ4ZASgbZLDbpG8W6", + "FPO3xsU2IuxNlqje75beK5PHSYyqHLjwZAXitlD52MolzKrG/NCsoK//oC5pE9OZWleePjUxx5HEH0sj", + "ZLqtsXwLRp9Lo9v6cT5gckoXv6rJLuRcrkIEIktIAjTVHe/TrPNkCckCtRbMrYyEDs4ATxMZv4E51d3t", + "ppE+DCOQROkCky6N7qppQENSdgbDeGD6dCsZ1nqbsYJA+n5ZFdlb/Sgm9XZr+82+zRD8yh01UjINUxUl", + "CcdsS3or8beEJNSJynmEA4FCtRMV9aaxFEZ6g9gtw7oRQHWJfnGZeCng09jZKSrpcQtXqhxBqdQDUCBp", + "UqxVEsS5KZj3+r2ieu5eTJvUbqkS5SGqF6x8yX1SFa1NWypLEOMFgwLlQlQloWRWMwaoMf3ujW5KgZzp", + "lyuCVclYroGbiXrhCAr4AXKUtbR7SJlBbtoYMurN0yiSGyEBQzEiuikNRqrRqeBUqAZ1cpoKEFo0RYXL", + "q/t3UqXKQG5d7dBjrhy/QErS5cQcQJHVPSN0g6KansULQhnSls0RW8ifM582Z4qGMSXUgjCOupgFA4Np", + "7qu3ACVQCMRU4KftgR8Y3/ACrv8/YjRph+rOQ4Ff0ygy/C6F1xGZlKpRdA4kJ+byJbmIO84mEI65QCRw", + "1MyUjiKC0QhkagsT4wOpMphuLKFMasq5aoDPZwOQ85RJXi3TJhXUhQI5nbvKKs2HjLRCzOr6fmuYrT81", + "mro2sx4wFUuGYFju69mtmjCFMP2CxF9AiXH1nP4jjr0zj/ecU+s3Wqf2ccAJCdh6HGApIQ8DMJRE0xkU", + "Qbkzb1zvPLLnku7fklGC/5UvpeYA6E8UpOonKQ/XKSQCq6XcbUNJ1BF91Y3cG4d+lzP3KBodTp9/4XI4", + "C0tbz8dU4p9iidHOPBht7+0Mtt8FbwfjMXo7gHtvdgZ7wWj2bjd8836+M9ofD96Odse72zv90Zvdt7vh", + "TmANf7fzZnuwPdoJZ9u7e2G4E+6PB+O3I+dZu3LW0To7px6YTpSGNxNaxtCuM2B8msx2Q67ZZ8VKvo8H", + "lAFDEZQarblRUAp0bkoDQ+M2/6Kqw++0n7D2PFVNUPYDvUiu7qizs2Vxclu8asPhI0PNd/O3TmknUVD7", + "FKPtMvKO8VtFG6uHaoKM8xzSLh93k3beWHHuyFF2sOWJhfvgFkdhAFmYBXnlKGo2+OWBadBaDc6XHhU6", + "je926jvAKpywNtaPDIKytV3cVfQo+ILTxyRGSBEHhIo84s52zCtkGd8Tgx0XELMO6rENeU7UN4hwKVJq", + "QHiRDWjG+Cb2QKzXAnGfzoQnKvo3l/m9REdxIoXHe3S9yI+s00iTv6VdO2FWyf9oPyNQrNsOuu+81xzi", + "SB095lf1ZEhD24BDrs35c8fTWtNTNtQutDkVW9XipEGAOPeAu17rVX2ufh0bLqAqdcumClGDU+3vJqhv", + "u1jRW6s1RVkOMkUvqOlw4E2F2rb61j26H5r7He5UcCqkBEdHNHCkFI7OwKcEkYPPJ+Do06GUUxb19ntL", + "IRK+PxyGNOBbCSaLACZbAY2H/1oOBQ5nA6lwB9pJwpQMudb4ytecU8UeWKid1Ba4QYzrtd9s7WyNVOI0", + "QQQmuLff25HaVqkJsVTQDmGChzfjoTm6OMymNxY4P1B/Eqq1Dj6flA+lqxKGFkc13/ZoZHISWUMnTHQy", + "S27jn1z3MhaWuUmHeo6/K6xXdKnmfkU9nsYxZKvevtwDyI+/kzkFPA2WAHJQOhMv4IJbR917X+QkVbTo", + "KgjvipniNPzz4Mdx+r4JS/3e7iOCUbuWwbG0VkUN9LEut8nUzDqEGX7TfyhX906LYYS0DXNQ6tN8HmGC", + "NNrOdVo2gQzGSFP5j1qe2AIvCzbk71KOell9o2fB0LPViK7PFNjscvHQlxrj7Dp8iBdGUarxWrmqqBMh", + "M/XeUcKKqxqeR8IcV0NsmIRZVyytJWGGMMNvxmauJWHG1neQMBs8v4RZMPzYEla+MKuRkGG8lQHnlKyP", + "SBzR4H8uP517RKkMlpwrP1ZSZ7eQBkAtV0AV0qACkXGVGsD5++TstBM4cmALOEuhK0k+cHQk1q56igtW", + "2phZrmziO3VOLW9dVix9nSK2sngai+U0H+HgYXd538G/j6r4HNfJOJjUPkoVZY2PFRJUhxSkyBIUKjjn", + "PtTr+9Yusx5mExx8oOHq0fabXXVT36BZDczkcnc1lI+fAYSXpoP0xR+AoFubti6y1oVs+M3KSbabEfu2", + "uFahi+hMHaFPCb5Oy6f8/BalnCLtZFG8Z0bu+rUkNdUnF2ii8yIw4qb/OOuvVtGtKeu5tIOa4YF6YffR", + "eMZ5e98GsKxmMgAfyrDDBKZcFwOU8mnQWp/lyIvsZoIXzrhfupjal0ZURQvrkMI8JbrHP2uaeyixGeJp", + "3I3aF2roK7mfkNyaGk9Jb+tW5Q6OoD4V38UdfALi+k/XPalfWLkJYENC4OwAoa77+XzQruwx/Kb/KFyY", + "DsyiyuUvj1f6DbVRz/LF3jsu7yydPimXlhvTN4tJden4/jwqIBOdLFZxUnNTDNYThH2106p35QKQBPZu", + "E42lOUbwlMYyP8bVxVbmZ7dfDqM19qU9S3Klcgnrhigq+5sE9ocdHoOlaNJRd5nTvj+y6qoceP6raK4Q", + "86dWXYJBwufmgxx+LpuYYRuTfnoiVqs3bPxVeC1jhNz5ogDqay11faWFu3Ters0CZjeEdykaqCThxpYM", + "anehO+igdhiZj428HJuWQ1VQXH/CpLk0oRzIiT6C+hSiV//UzPcsUZjvvmxKgQLqzwwxkV8gXaZsVZKH", + "WbNiN5nO2hGfoQlhwwUr7wa9h4QVEjApWkmfQtR8zP0qXW7pKlF2HeEa6iN0Lc7XiRr0THSvNkWvzwbb", + "TwTP5oSG5mDk/dnimzqZt05duMIda7nn9uUDDr88h6WjV+47VbiZXUamXOpv5a8q8M7GcnPINPrhFHvd", + "XjeRPEk9JNefongl+mYQPVXU6kz3mv6+n9Z+qRzR73LXqSMgr3130173QZejbrQB0RGYaX5aj5l0p02X", + "HpuXzE9fnrJd0a5w3m1uA889eEO3gnRqyXnljk3lDtPvcw/2eGB3T97X82ElueeAhPerALwEo/Xab/S9", + "nOjGpqMHc/GaTUh5+9ErS7+2RW2sLDl7ox5ZlOR7switGdHYH9t8lakXJlN9//UsPpRnHNAZ556P6G58", + "9q8kedxi8XVTgK8S8ioh36HRruFr4RtrABvF0JuWPcm/Dv0qimsv/qMI4uMnI1q/Sf5X6SUrPqC+hrw2", + "e63dOqytW7x/pKz6WhmwZ7AyG9rMrbg1454qd6oL79hNxk3lu6tWNN0KaQwxUTdX9SSSzQTerzY2X5YV", + "0uCBN2QNr1McXA30KRjdqDIwi99V2KrnUrbmE0DPAqQBL386UMvflcTPAWR2x0c+Lvvh7svdvwMAAP//", + "cJ+7WyeSAAA=", } // GetSwagger returns the content of the embedded swagger specification file diff --git a/dm/openapi/gen.types.go b/dm/openapi/gen.types.go index ca478469ac9..8596087165d 100644 --- a/dm/openapi/gen.types.go +++ b/dm/openapi/gen.types.go @@ -312,6 +312,9 @@ type SubTaskStatus struct { // status of dump unit DumpStatus *DumpStatus `json:"dump_status,omitempty"` + // error message when something wrong + ErrorMsg *string `json:"error_msg,omitempty"` + // status of load unit LoadStatus *LoadStatus `json:"load_status,omitempty"` diff --git a/dm/openapi/spec/dm.yaml b/dm/openapi/spec/dm.yaml index 7db4ce256f2..e1b2187cb44 100644 --- a/dm/openapi/spec/dm.yaml +++ b/dm/openapi/spec/dm.yaml @@ -1301,6 +1301,9 @@ components: dump_status: nullable: true $ref: "#/components/schemas/DumpStatus" + error_msg: + type: string + description: "error message when something wrong" required: - "name" - "source_name" diff --git a/dm/tests/openapi/client/openapi_task_check b/dm/tests/openapi/client/openapi_task_check index 77e70d41264..8b7ab8c4262 100755 --- a/dm/tests/openapi/client/openapi_task_check +++ b/dm/tests/openapi/client/openapi_task_check @@ -184,6 +184,18 @@ def get_task_status_success(task_name, total): assert data["total"] == int(total) +def get_task_status_success_but_worker_meet_error(task_name, total): + url = API_ENDPOINT + "/" + task_name + "/status" + resp = requests.get(url=url) + data = resp.json() + assert resp.status_code == 200 + print("get_task_status_success_but_worker_meet_error resp=", data) + assert data["total"] == int(total) + for status in data["data"]: + assert status["name"] == task_name + assert status["error_msg"] is not None + + def get_task_list(task_count): url = API_ENDPOINT resp = requests.get(url=url) @@ -457,6 +469,7 @@ if __name__ == "__main__": "get_task_status_failed": get_task_status_failed, "get_illegal_char_task_status_failed": get_illegal_char_task_status_failed, "get_task_status_success": get_task_status_success, + "get_task_status_success_but_worker_meet_error": get_task_status_success_but_worker_meet_error, "operate_schema_and_table_success": operate_schema_and_table_success, "create_task_template_success": create_task_template_success, "create_task_template_failed": create_task_template_failed, diff --git a/dm/tests/openapi/run.sh b/dm/tests/openapi/run.sh index b2103cfe4a7..0286527b3bc 100644 --- a/dm/tests/openapi/run.sh +++ b/dm/tests/openapi/run.sh @@ -385,32 +385,33 @@ function test_task_templates() { function test_noshard_task_dump_status() { echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: NO SHARD TASK DUMP STATUS" + + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever=return()" + kill_dm_worker + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + + # run dm-worker1 + 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 + # run dm-worker2 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + prepare_database - task_name="test-no-shard" + task_name="test-no-shard-dump-status" target_table_name="" # empty means no route - # create source succesfully openapi_source_check "create_source1_success" openapi_source_check "list_source_success" 1 - - # get source status success openapi_source_check "get_source_status_success" "mysql-01" - - # create source succesfully openapi_source_check "create_source2_success" - # get source list success openapi_source_check "list_source_success" 2 - - # get source status success openapi_source_check "get_source_status_success" "mysql-02" - - # start task success: not vaild task create request openapi_task_check "start_task_failed" - # start no shard task success openapi_task_check "start_noshard_task_success" $task_name $target_table_name - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status $task_name" \ "\"unit\": \"Dump\"" 2 @@ -418,8 +419,24 @@ function test_noshard_task_dump_status() { # check noshard task dump status success openapi_task_check "check_noshard_task_dump_status_success" "$task_name" 0 - # delete source success and clean data for other test + kill_dm_worker + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + + openapi_task_check "get_task_status_success_but_worker_meet_error" "$task_name" 2 clean_cluster_sources_and_tasks + + export GO_FAILPOINTS="" + kill_dm_worker + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + + # run dm-worker1 + 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 + # run dm-worker2 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: NO SHARD TASK DUMP STATUS SUCCESS" } @@ -457,40 +474,14 @@ function run() { run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - test_source test_relay + test_source test_shard_task test_multi_tasks - - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever=return()" - kill_dm_worker - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 - - # run dm-worker1 - 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 - # run dm-worker2 - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - - test_noshard_task_dump_status - - export GO_FAILPOINTS="" - kill_dm_worker - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 - - # run dm-worker1 - 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 - # run dm-worker2 - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - test_noshard_task test_task_templates + test_noshard_task_dump_status # NOTE: this test case MUST running at last, because it will offline some members of cluster test_cluster From e2d529cc9887ab61063555dbbbf1d9f9765581e3 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Thu, 10 Feb 2022 16:39:36 +0800 Subject: [PATCH 49/72] api,owner(ticdc): return error when api fails (#4494) close pingcap/tiflow#1710, ref pingcap/tiflow#3456 --- Makefile | 7 +- cdc/api/open.go | 65 ++++---- cdc/api/open_test.go | 286 +++++++++++++++++++++++++++++----- cdc/api/owner.go | 28 +--- cdc/api/status.go | 5 +- cdc/api/util.go | 58 +++++++ cdc/capture/capture.go | 62 +++++--- cdc/http_test.go | 4 +- cdc/owner/mock/owner_mock.go | 126 +++++++++++++++ cdc/owner/owner.go | 206 +++++++++++++----------- cdc/owner/owner_test.go | 47 +++--- cdc/owner/status_provider.go | 97 ++++++------ cdc/processor/manager.go | 46 +++--- cdc/processor/manager_test.go | 21 ++- cdc/server_test.go | 4 +- errors.toml | 10 +- pkg/errors/errors.go | 10 +- scripts/check-copyright.sh | 2 +- 18 files changed, 776 insertions(+), 308 deletions(-) create mode 100644 cdc/owner/mock/owner_mock.go diff --git a/Makefile b/Makefile index c338ba4dac1..32abfbaea10 100644 --- a/Makefile +++ b/Makefile @@ -112,7 +112,7 @@ kafka_consumer: install: go install ./... -unit_test: check_failpoint_ctl +unit_test: check_failpoint_ctl generate_mock mkdir -p "$(TEST_DIR)" $(FAILPOINT_ENABLE) @export log_level=error;\ @@ -167,7 +167,7 @@ integration_test_mysql: integration_test_kafka: check_third_party_binary tests/integration_tests/run.sh kafka "$(CASE)" "$(START_AT)" -fmt: tools/bin/gofumports tools/bin/shfmt +fmt: tools/bin/gofumports tools/bin/shfmt generate_mock @echo "gofmt (simplify)" tools/bin/gofumports -l -w $(FILES) 2>&1 | $(FAIL_ON_STDOUT) @echo "run shfmt" @@ -234,6 +234,9 @@ data-flow-diagram: docs/data-flow.dot swagger-spec: tools/bin/swag tools/bin/swag init --parseVendor -generalInfo cdc/api/open.go --output docs/swagger +generate_mock: tools/bin/mockgen + tools/bin/mockgen -source cdc/owner/owner.go -destination cdc/owner/mock/owner_mock.go + clean: go clean -i ./... rm -rf *.out diff --git a/cdc/api/open.go b/cdc/api/open.go index 14f0b29c403..feacbd8525d 100644 --- a/cdc/api/open.go +++ b/cdc/api/open.go @@ -90,7 +90,7 @@ func RegisterOpenAPIRoutes(router *gin.Engine, api openAPI) { changefeedGroup.POST("/:changefeed_id/pause", api.PauseChangefeed) changefeedGroup.POST("/:changefeed_id/resume", api.ResumeChangefeed) changefeedGroup.DELETE("/:changefeed_id", api.RemoveChangefeed) - changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", api.RebalanceTable) + changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", api.RebalanceTables) changefeedGroup.POST("/:changefeed_id/tables/move_table", api.MoveTable) // owner API @@ -319,11 +319,10 @@ func (h *openAPI) PauseChangefeed(c *gin.Context) { Type: model.AdminStop, } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.EnqueueJob(job) - return nil - }) - + if err := handleOwnerJob(ctx, h.capture, job); err != nil { + _ = c.Error(err) + return + } c.Status(http.StatusAccepted) } @@ -361,11 +360,10 @@ func (h *openAPI) ResumeChangefeed(c *gin.Context) { Type: model.AdminResume, } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.EnqueueJob(job) - return nil - }) - + if err := handleOwnerJob(ctx, h.capture, job); err != nil { + _ = c.Error(err) + return + } c.Status(http.StatusAccepted) } @@ -465,15 +463,14 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { Type: model.AdminRemove, } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.EnqueueJob(job) - return nil - }) - + if err := handleOwnerJob(ctx, h.capture, job); err != nil { + _ = c.Error(err) + return + } c.Status(http.StatusAccepted) } -// RebalanceTable rebalances tables +// RebalanceTables rebalances tables // @Summary rebalance tables // @Description rebalance all tables of a changefeed // @Tags changefeed @@ -483,7 +480,7 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { // @Success 202 // @Failure 500,400 {object} model.HTTPError // @Router /api/v1/changefeeds/{changefeed_id}/tables/rebalance_table [post] -func (h *openAPI) RebalanceTable(c *gin.Context) { +func (h *openAPI) RebalanceTables(c *gin.Context) { if !h.capture.IsOwner() { h.forwardToOwner(c) return @@ -503,11 +500,10 @@ func (h *openAPI) RebalanceTable(c *gin.Context) { return } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.TriggerRebalance(changefeedID) - return nil - }) - + if err := handleOwnerRebalance(ctx, h.capture, changefeedID); err != nil { + _ = c.Error(err) + return + } c.Status(http.StatusAccepted) } @@ -557,11 +553,12 @@ func (h *openAPI) MoveTable(c *gin.Context) { return } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.ManualSchedule(changefeedID, data.CaptureID, data.TableID) - return nil - }) - + err = handleOwnerScheduleTable( + ctx, h.capture, changefeedID, data.CaptureID, data.TableID) + if err != nil { + _ = c.Error(err) + return + } c.Status(http.StatusAccepted) } @@ -580,10 +577,10 @@ func (h *openAPI) ResignOwner(c *gin.Context) { return } - _ = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.AsyncStop() - return nil - }) + o, _ := h.capture.GetOwner() + if o != nil { + o.AsyncStop() + } c.Status(http.StatusAccepted) } @@ -748,7 +745,7 @@ func (h *openAPI) ServerStatus(c *gin.Context) { func (h *openAPI) Health(c *gin.Context) { ctx := c.Request.Context() - if _, err := h.capture.GetOwner(ctx); err != nil { + if _, err := h.capture.GetOwnerCaptureInfo(ctx); err != nil { c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) return } @@ -798,7 +795,7 @@ func (h *openAPI) forwardToOwner(c *gin.Context) { var owner *model.CaptureInfo // get owner err := retry.Do(ctx, func() error { - o, err := h.capture.GetOwner(ctx) + o, err := h.capture.GetOwnerCaptureInfo(ctx) if err != nil { log.Info("get owner failed, retry later", zap.Error(err)) return err diff --git a/cdc/api/open_test.go b/cdc/api/open_test.go index 48a1fb53d4d..6541b63ba80 100644 --- a/cdc/api/open_test.go +++ b/cdc/api/open_test.go @@ -23,9 +23,11 @@ import ( "testing" "github.com/gin-gonic/gin" + "github.com/golang/mock/gomock" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" + mock_owner "github.com/pingcap/tiflow/cdc/owner/mock" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -88,8 +90,7 @@ func (p *mockStatusProvider) GetCaptures(ctx context.Context) ([]*model.CaptureI return args.Get(0).([]*model.CaptureInfo), args.Error(1) } -func newRouter(p *mockStatusProvider) *gin.Engine { - c := capture.NewCapture4Test(true) +func newRouter(c *capture.Capture, p *mockStatusProvider) *gin.Engine { router := gin.New() RegisterOpenAPIRoutes(router, NewOpenAPI4Test(c, p)) return router @@ -108,7 +109,9 @@ func newStatusProvider() *mockStatusProvider { Return(map[model.CaptureID]*model.TaskStatus{captureID: {}}, nil) statusProvider.On("GetTaskPositions", mock.Anything). - Return(map[model.CaptureID]*model.TaskPosition{captureID: {Error: &model.RunningError{Message: "test"}}}, nil) + Return(map[model.CaptureID]*model.TaskPosition{ + captureID: {Error: &model.RunningError{Message: "test"}}, + }, nil) statusProvider.On("GetAllChangeFeedStatuses", mock.Anything). Return(map[model.ChangeFeedID]*model.ChangeFeedStatus{ @@ -139,7 +142,10 @@ func newStatusProvider() *mockStatusProvider { func TestListChangefeed(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test list changefeed succeeded api := testCase{url: "/api/v1/changefeeds", method: "GET"} @@ -168,7 +174,10 @@ func TestListChangefeed(t *testing.T) { func TestGetChangefeed(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test get changefeed succeeded api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "GET"} @@ -195,16 +204,33 @@ func TestGetChangefeed(t *testing.T) { func TestPauseChangefeed(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) + // test pause changefeed succeeded + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + require.EqualValues(t, changeFeedID, adminJob.CfID) + require.EqualValues(t, model.AdminStop, adminJob.Type) + close(done) + }) api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", changeFeedID), method: "POST"} w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) require.Equal(t, 202, w.Code) - // test pause changefeed failed - api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", nonExistChangefeedID), method: "POST"} + // test pause changefeed failed from owner side + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(adminJob.CfID) + close(done) + }) + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", changeFeedID), method: "POST"} w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -213,20 +239,51 @@ func TestPauseChangefeed(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&respErr) require.Nil(t, err) require.Contains(t, respErr.Error, "changefeed not exists") + + // test pause changefeed failed + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", nonExistChangefeedID), + method: "POST", + } + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr = model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") } func TestResumeChangefeed(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) + // test resume changefeed succeeded + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + require.EqualValues(t, changeFeedID, adminJob.CfID) + require.EqualValues(t, model.AdminResume, adminJob.Type) + close(done) + }) api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", changeFeedID), method: "POST"} w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) require.Equal(t, 202, w.Code) - // test resume changefeed failed - api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", nonExistChangefeedID), method: "POST"} + // test resume changefeed failed from owner side. + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(adminJob.CfID) + close(done) + }) + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", changeFeedID), method: "POST"} w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -235,20 +292,51 @@ func TestResumeChangefeed(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&respErr) require.Nil(t, err) require.Contains(t, respErr.Error, "changefeed not exists") + + // test resume changefeed failed + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", nonExistChangefeedID), + method: "POST", + } + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr = model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") } func TestRemoveChangefeed(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) + // test remove changefeed succeeded + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + require.EqualValues(t, changeFeedID, adminJob.CfID) + require.EqualValues(t, model.AdminRemove, adminJob.Type) + close(done) + }) api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "DELETE"} w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) require.Equal(t, 202, w.Code) - // test remove changefeed failed - api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), method: "DELETE"} + // test remove changefeed failed from owner size + mo.EXPECT(). + EnqueueJob(gomock.Any(), gomock.Any()). + Do(func(adminJob model.AdminJob, done chan<- error) { + done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(adminJob.CfID) + close(done) + }) + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "DELETE"} w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -257,20 +345,56 @@ func TestRemoveChangefeed(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&respErr) require.Nil(t, err) require.Contains(t, respErr.Error, "changefeed not exists") + + // test remove changefeed failed + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), + method: "DELETE", + } + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr = model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") } -func TestRebalanceTable(t *testing.T) { +func TestRebalanceTables(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) + // test rebalance table succeeded - api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", changeFeedID), method: "POST"} + mo.EXPECT(). + RebalanceTables(gomock.Any(), gomock.Any()). + Do(func(cfID model.ChangeFeedID, done chan<- error) { + require.EqualValues(t, cfID, changeFeedID) + close(done) + }) + api := testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", changeFeedID), + method: "POST", + } w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) require.Equal(t, 202, w.Code) - // test rebalance table failed - api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", nonExistChangefeedID), method: "POST"} + // test rebalance table failed from owner side. + mo.EXPECT(). + RebalanceTables(gomock.Any(), gomock.Any()). + Do(func(cfID model.ChangeFeedID, done chan<- error) { + done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(cfID) + close(done) + }) + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", changeFeedID), + method: "POST", + } w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -279,11 +403,31 @@ func TestRebalanceTable(t *testing.T) { err := json.NewDecoder(w.Body).Decode(&respErr) require.Nil(t, err) require.Contains(t, respErr.Error, "changefeed not exists") + + // test rebalance table failed + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", nonExistChangefeedID), + method: "POST", + } + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr = model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") } func TestMoveTable(t *testing.T) { t.Parallel() + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) + + // test move table succeeded data := struct { CaptureID string `json:"capture_id"` TableID int64 `json:"table_id"` @@ -291,17 +435,50 @@ func TestMoveTable(t *testing.T) { b, err := json.Marshal(&data) require.Nil(t, err) body := bytes.NewReader(b) - - router := newRouter(newStatusProvider()) - // test move table succeeded - api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", changeFeedID), method: "POST"} + mo.EXPECT(). + ScheduleTable(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func( + cfID model.ChangeFeedID, toCapture model.CaptureID, + tableID model.TableID, done chan<- error, + ) { + require.EqualValues(t, cfID, changeFeedID) + require.EqualValues(t, toCapture, data.CaptureID) + require.EqualValues(t, tableID, data.TableID) + close(done) + }) + api := testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", changeFeedID), + method: "POST", + } w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, body) router.ServeHTTP(w, req) require.Equal(t, 202, w.Code) - // test move table failed - api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", nonExistChangefeedID), method: "POST"} + // test move table failed from owner side. + data = struct { + CaptureID string `json:"capture_id"` + TableID int64 `json:"table_id"` + }{captureID, 1} + b, err = json.Marshal(&data) + require.Nil(t, err) + body = bytes.NewReader(b) + mo.EXPECT(). + ScheduleTable(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + Do(func( + cfID model.ChangeFeedID, toCapture model.CaptureID, + tableID model.TableID, done chan<- error, + ) { + require.EqualValues(t, cfID, changeFeedID) + require.EqualValues(t, toCapture, data.CaptureID) + require.EqualValues(t, tableID, data.TableID) + done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(cfID) + close(done) + }) + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", changeFeedID), + method: "POST", + } w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, body) router.ServeHTTP(w, req) @@ -310,12 +487,30 @@ func TestMoveTable(t *testing.T) { err = json.NewDecoder(w.Body).Decode(&respErr) require.Nil(t, err) require.Contains(t, respErr.Error, "changefeed not exists") + + // test move table failed + api = testCase{ + url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", nonExistChangefeedID), + method: "POST", + } + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr = model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") } func TestResignOwner(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test resign owner succeeded + mo.EXPECT().AsyncStop() api := testCase{url: "/api/v1/owner/resign", method: "POST"} w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) @@ -325,9 +520,15 @@ func TestResignOwner(t *testing.T) { func TestGetProcessor(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test get processor succeeded - api := testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, captureID), method: "GET"} + api := testCase{ + url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, captureID), + method: "GET", + } w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -338,7 +539,10 @@ func TestGetProcessor(t *testing.T) { require.Equal(t, "test", processorDetail.Error.Message) // test get processor fail due to capture ID error - api = testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, "non-exist-capture"), method: "GET"} + api = testCase{ + url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, "non-exist-capture"), + method: "GET", + } w = httptest.NewRecorder() req, _ = http.NewRequest(api.method, api.url, nil) router.ServeHTTP(w, req) @@ -346,12 +550,15 @@ func TestGetProcessor(t *testing.T) { httpError := &model.HTTPError{} err = json.NewDecoder(w.Body).Decode(httpError) require.Nil(t, err) - require.Contains(t, httpError.Error, "capture not exists, key: non-exist-capture") + require.Contains(t, httpError.Error, "capture not exists, non-exist-capture") } func TestListProcessor(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test list processor succeeded api := testCase{url: "/api/v1/processors", method: "GET"} w := httptest.NewRecorder() @@ -366,7 +573,10 @@ func TestListProcessor(t *testing.T) { func TestListCapture(t *testing.T) { t.Parallel() - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) // test list processor succeeded api := testCase{url: "/api/v1/captures", method: "GET"} w := httptest.NewRecorder() @@ -382,7 +592,10 @@ func TestListCapture(t *testing.T) { func TestServerStatus(t *testing.T) { t.Parallel() // capture is owner - ownerRouter := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + ownerRouter := newRouter(cp, newStatusProvider()) api := testCase{url: "/api/v1/status", method: "GET"} w := httptest.NewRecorder() req, _ := http.NewRequest(api.method, api.url, nil) @@ -395,7 +608,7 @@ func TestServerStatus(t *testing.T) { require.True(t, resp.IsOwner) // capture is not owner - c := capture.NewCapture4Test(false) + c := capture.NewCapture4Test(nil) r := gin.New() RegisterOpenAPIRoutes(r, NewOpenAPI4Test(c, nil)) api = testCase{url: "/api/v1/status", method: "GET"} @@ -416,7 +629,10 @@ func TestSetLogLevel(t *testing.T) { data := struct { Level string `json:"log_level"` }{"warn"} - router := newRouter(newStatusProvider()) + ctrl := gomock.NewController(t) + mo := mock_owner.NewMockOwner(ctrl) + cp := capture.NewCapture4Test(mo) + router := newRouter(cp, newStatusProvider()) api := testCase{url: "/api/v1/log", method: "POST"} w := httptest.NewRecorder() b, err := json.Marshal(&data) diff --git a/cdc/api/owner.go b/cdc/api/owner.go index a8b565c4a1a..d1275d8ea35 100644 --- a/cdc/api/owner.go +++ b/cdc/api/owner.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/owner" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/logutil" "github.com/tikv/client-go/v2/oracle" @@ -112,10 +111,10 @@ func (h *ownerAPI) handleResignOwner(w http.ResponseWriter, req *http.Request) { handleOwnerResp(w, concurrency.ErrElectionNotLeader) return } - err := h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.AsyncStop() - return nil - }) + o, err := h.capture.GetOwner() + if o != nil { + o.AsyncStop() + } handleOwnerResp(w, err) } @@ -153,11 +152,7 @@ func (h *ownerAPI) handleChangefeedAdmin(w http.ResponseWriter, req *http.Reques Opts: opts, } - err = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.EnqueueJob(job) - return nil - }) - + err = handleOwnerJob(req.Context(), h.capture, job) handleOwnerResp(w, err) } @@ -180,11 +175,7 @@ func (h *ownerAPI) handleRebalanceTrigger(w http.ResponseWriter, req *http.Reque return } - err = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.TriggerRebalance(changefeedID) - return nil - }) - + err = handleOwnerRebalance(req.Context(), h.capture, changefeedID) handleOwnerResp(w, err) } @@ -221,11 +212,8 @@ func (h *ownerAPI) handleMoveTable(w http.ResponseWriter, req *http.Request) { return } - err = h.capture.OperateOwnerUnderLock(func(owner *owner.Owner) error { - owner.ManualSchedule(changefeedID, to, tableID) - return nil - }) - + err = handleOwnerScheduleTable( + req.Context(), h.capture, changefeedID, to, tableID) handleOwnerResp(w, err) } diff --git a/cdc/api/status.go b/cdc/api/status.go index eb93bfc5982..e71c1528c03 100644 --- a/cdc/api/status.go +++ b/cdc/api/status.go @@ -60,9 +60,10 @@ func (h *statusAPI) writeEtcdInfo(ctx context.Context, cli *etcd.CDCEtcdClient, } func (h *statusAPI) handleDebugInfo(w http.ResponseWriter, req *http.Request) { - h.capture.WriteDebugInfo(w) + ctx := req.Context() + h.capture.WriteDebugInfo(ctx, w) fmt.Fprintf(w, "\n\n*** etcd info ***:\n\n") - h.writeEtcdInfo(req.Context(), h.capture.EtcdClient, w) + h.writeEtcdInfo(ctx, h.capture.EtcdClient, w) } func (h *statusAPI) handleStatus(w http.ResponseWriter, req *http.Request) { diff --git a/cdc/api/util.go b/cdc/api/util.go index 4baebf54e64..7f9650972a0 100644 --- a/cdc/api/util.go +++ b/cdc/api/util.go @@ -14,12 +14,15 @@ package api import ( + "context" "encoding/json" "net/http" "strings" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) @@ -76,3 +79,58 @@ func writeData(w http.ResponseWriter, data interface{}) { log.Error("fail to write data", zap.Error(err)) } } + +func handleOwnerJob( + ctx context.Context, capture *capture.Capture, job model.AdminJob, +) error { + // Use buffered channel to prevernt blocking owner. + done := make(chan error, 1) + o, err := capture.GetOwner() + if err != nil { + return errors.Trace(err) + } + o.EnqueueJob(job, done) + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case err := <-done: + return errors.Trace(err) + } +} + +func handleOwnerRebalance( + ctx context.Context, capture *capture.Capture, changefeedID string, +) error { + // Use buffered channel to prevernt blocking owner. + done := make(chan error, 1) + o, err := capture.GetOwner() + if err != nil { + return errors.Trace(err) + } + o.RebalanceTables(changefeedID, done) + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case err := <-done: + return errors.Trace(err) + } +} + +func handleOwnerScheduleTable( + ctx context.Context, capture *capture.Capture, + changefeedID string, captureID string, tableID int64, +) error { + // Use buffered channel to prevernt blocking owner. + done := make(chan error, 1) + o, err := capture.GetOwner() + if err != nil { + return errors.Trace(err) + } + o.ScheduleTable(changefeedID, captureID, tableID, done) + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case err := <-done: + return errors.Trace(err) + } +} diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index b2f8fae43aa..aa5681e608f 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -53,7 +53,7 @@ type Capture struct { info *model.CaptureInfo ownerMu sync.Mutex - owner *owner.Owner + owner owner.Owner processorManager *processor.Manager // session keeps alive between the capture and etcd @@ -88,7 +88,7 @@ type Capture struct { cancel context.CancelFunc newProcessorManager func() *processor.Manager - newOwner func(pd.Client) *owner.Owner + newOwner func(pd.Client) owner.Owner } // NewCapture returns a new Capture instance @@ -107,13 +107,11 @@ func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *etcd.C } } -func NewCapture4Test(isOwner bool) *Capture { +func NewCapture4Test(o owner.Owner) *Capture { res := &Capture{ info: &model.CaptureInfo{ID: "capture-for-test", AdvertiseAddr: "127.0.0.1", Version: "test"}, } - if isOwner { - res.owner = &owner.Owner{} - } + res.owner = o return res } @@ -482,20 +480,20 @@ func (c *Capture) runEtcdWorker( return nil } -func (c *Capture) setOwner(owner *owner.Owner) { +func (c *Capture) setOwner(owner owner.Owner) { c.ownerMu.Lock() defer c.ownerMu.Unlock() c.owner = owner } -// OperateOwnerUnderLock operates the owner with lock -func (c *Capture) OperateOwnerUnderLock(fn func(*owner.Owner) error) error { +// GetOwner returns owner if it is the owner. +func (c *Capture) GetOwner() (owner.Owner, error) { c.ownerMu.Lock() defer c.ownerMu.Unlock() if c.owner == nil { - return cerror.ErrNotOwner.GenWithStackByArgs() + return nil, cerror.ErrNotOwner.GenWithStackByArgs() } - return fn(c.owner) + return c.owner, nil } // campaign to be an owner. @@ -529,10 +527,10 @@ func (c *Capture) AsyncClose() { defer c.cancel() // Safety: Here we mainly want to stop the owner // and ignore it if the owner does not exist or is not set. - _ = c.OperateOwnerUnderLock(func(o *owner.Owner) error { + o, _ := c.GetOwner() + if o != nil { o.AsyncStop() - return nil - }) + } c.captureMu.Lock() defer c.captureMu.Unlock() if c.processorManager != nil { @@ -571,20 +569,38 @@ func (c *Capture) AsyncClose() { } // WriteDebugInfo writes the debug info into writer. -func (c *Capture) WriteDebugInfo(w io.Writer) { +func (c *Capture) WriteDebugInfo(ctx context.Context, w io.Writer) { + wait := func(done <-chan error) { + var err error + select { + case <-ctx.Done(): + err = ctx.Err() + case err = <-done: + } + if err != nil { + log.Warn("write debug info failed", zap.Error(err)) + } + } // Safety: Because we are mainly outputting information about the owner here, // if the owner does not exist or is not set, the information will not be output. - _ = c.OperateOwnerUnderLock(func(o *owner.Owner) error { + o, _ := c.GetOwner() + if o != nil { + doneOwner := make(chan error, 1) fmt.Fprintf(w, "\n\n*** owner info ***:\n\n") - o.WriteDebugInfo(w) - return nil - }) + o.WriteDebugInfo(w, doneOwner) + // wait the debug info printed + wait(doneOwner) + } + + doneM := make(chan error, 1) c.captureMu.Lock() defer c.captureMu.Unlock() if c.processorManager != nil { fmt.Fprintf(w, "\n\n*** processors info ***:\n\n") - c.processorManager.WriteDebugInfo(w) + c.processorManager.WriteDebugInfo(ctx, w, doneM) } + // wait the debug info printed + wait(doneM) } // IsOwner returns whether the capture is an owner @@ -594,8 +610,8 @@ func (c *Capture) IsOwner() bool { return c.owner != nil } -// GetOwner return the owner of current TiCDC cluster -func (c *Capture) GetOwner(ctx context.Context) (*model.CaptureInfo, error) { +// GetOwnerCaptureInfo return the owner capture info of current TiCDC cluster +func (c *Capture) GetOwnerCaptureInfo(ctx context.Context) (*model.CaptureInfo, error) { _, captureInfos, err := c.EtcdClient.GetCaptures(ctx) if err != nil { return nil, err @@ -621,5 +637,5 @@ func (c *Capture) StatusProvider() owner.StatusProvider { if c.owner == nil { return nil } - return c.owner.StatusProvider() + return owner.NewStatusProvider(c.owner) } diff --git a/cdc/http_test.go b/cdc/http_test.go index 39398a82639..6aaa47f3605 100644 --- a/cdc/http_test.go +++ b/cdc/http_test.go @@ -38,7 +38,7 @@ func (a *testCase) String() string { func TestPProfPath(t *testing.T) { router := gin.New() - RegisterRoutes(router, capture.NewCapture4Test(false), nil) + RegisterRoutes(router, capture.NewCapture4Test(nil), nil) apis := []*testCase{ {"/debug/pprof/", http.MethodGet}, @@ -63,7 +63,7 @@ func TestPProfPath(t *testing.T) { func TestHandleFailpoint(t *testing.T) { router := gin.New() - RegisterRoutes(router, capture.NewCapture4Test(false), nil) + RegisterRoutes(router, capture.NewCapture4Test(nil), nil) fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" uri := fmt.Sprintf("/debug/fail/%s", fp) body := bytes.NewReader([]byte("return(true)")) diff --git a/cdc/owner/mock/owner_mock.go b/cdc/owner/mock/owner_mock.go new file mode 100644 index 00000000000..b99c3b1d38f --- /dev/null +++ b/cdc/owner/mock/owner_mock.go @@ -0,0 +1,126 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: cdc/owner/owner.go + +// Package mock_owner is a generated GoMock package. +package mock_owner + +import ( + context "context" + io "io" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + model "github.com/pingcap/tiflow/cdc/model" + owner "github.com/pingcap/tiflow/cdc/owner" + orchestrator "github.com/pingcap/tiflow/pkg/orchestrator" +) + +// MockOwner is a mock of Owner interface. +type MockOwner struct { + ctrl *gomock.Controller + recorder *MockOwnerMockRecorder +} + +// MockOwnerMockRecorder is the mock recorder for MockOwner. +type MockOwnerMockRecorder struct { + mock *MockOwner +} + +// NewMockOwner creates a new mock instance. +func NewMockOwner(ctrl *gomock.Controller) *MockOwner { + mock := &MockOwner{ctrl: ctrl} + mock.recorder = &MockOwnerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockOwner) EXPECT() *MockOwnerMockRecorder { + return m.recorder +} + +// AsyncStop mocks base method. +func (m *MockOwner) AsyncStop() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "AsyncStop") +} + +// AsyncStop indicates an expected call of AsyncStop. +func (mr *MockOwnerMockRecorder) AsyncStop() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AsyncStop", reflect.TypeOf((*MockOwner)(nil).AsyncStop)) +} + +// EnqueueJob mocks base method. +func (m *MockOwner) EnqueueJob(adminJob model.AdminJob, done chan<- error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "EnqueueJob", adminJob, done) +} + +// EnqueueJob indicates an expected call of EnqueueJob. +func (mr *MockOwnerMockRecorder) EnqueueJob(adminJob, done interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnqueueJob", reflect.TypeOf((*MockOwner)(nil).EnqueueJob), adminJob, done) +} + +// Query mocks base method. +func (m *MockOwner) Query(query *owner.Query, done chan<- error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Query", query, done) +} + +// Query indicates an expected call of Query. +func (mr *MockOwnerMockRecorder) Query(query, done interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Query", reflect.TypeOf((*MockOwner)(nil).Query), query, done) +} + +// RebalanceTables mocks base method. +func (m *MockOwner) RebalanceTables(cfID model.ChangeFeedID, done chan<- error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "RebalanceTables", cfID, done) +} + +// RebalanceTables indicates an expected call of RebalanceTables. +func (mr *MockOwnerMockRecorder) RebalanceTables(cfID, done interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RebalanceTables", reflect.TypeOf((*MockOwner)(nil).RebalanceTables), cfID, done) +} + +// ScheduleTable mocks base method. +func (m *MockOwner) ScheduleTable(cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID, done chan<- error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "ScheduleTable", cfID, toCapture, tableID, done) +} + +// ScheduleTable indicates an expected call of ScheduleTable. +func (mr *MockOwnerMockRecorder) ScheduleTable(cfID, toCapture, tableID, done interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ScheduleTable", reflect.TypeOf((*MockOwner)(nil).ScheduleTable), cfID, toCapture, tableID, done) +} + +// Tick mocks base method. +func (m *MockOwner) Tick(ctx context.Context, state orchestrator.ReactorState) (orchestrator.ReactorState, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Tick", ctx, state) + ret0, _ := ret[0].(orchestrator.ReactorState) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Tick indicates an expected call of Tick. +func (mr *MockOwnerMockRecorder) Tick(ctx, state interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Tick", reflect.TypeOf((*MockOwner)(nil).Tick), ctx, state) +} + +// WriteDebugInfo mocks base method. +func (m *MockOwner) WriteDebugInfo(w io.Writer, done chan<- error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "WriteDebugInfo", w, done) +} + +// WriteDebugInfo indicates an expected call of WriteDebugInfo. +func (mr *MockOwnerMockRecorder) WriteDebugInfo(w, done interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteDebugInfo", reflect.TypeOf((*MockOwner)(nil).WriteDebugInfo), w, done) +} diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 53af624c721..70c546e0a17 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -15,7 +15,6 @@ package owner import ( "context" - "fmt" "io" "math" "sync" @@ -41,7 +40,7 @@ type ownerJobType int // All OwnerJob types const ( ownerJobTypeRebalance ownerJobType = iota - ownerJobTypeManualSchedule + ownerJobTypeScheduleTable ownerJobTypeAdminJob ownerJobTypeDebugInfo ownerJobTypeQuery @@ -51,30 +50,45 @@ const ( // captures with versions different from that of the owner const versionInconsistentLogRate = 1 +// Export field names for pretty printing. type ownerJob struct { - tp ownerJobType - changefeedID model.ChangeFeedID + Tp ownerJobType + ChangefeedID model.ChangeFeedID - // for ManualSchedule only - targetCaptureID model.CaptureID - // for ManualSchedule only - tableID model.TableID + // for ScheduleTable only + TargetCaptureID model.CaptureID + // for ScheduleTable only + TableID model.TableID // for Admin Job only - adminJob *model.AdminJob + AdminJob *model.AdminJob // for debug info only debugInfoWriter io.Writer // for status provider - query *ownerQuery + query *Query - done chan struct{} + done chan<- error } -// Owner manages many changefeeds -// All public functions are THREAD-SAFE, except for Tick, Tick is only used for etcd worker -type Owner struct { +// Owner managers TiCDC cluster. +// +// The interface is thread-safe, except for Tick, it's only used by etcd worker. +type Owner interface { + orchestrator.Reactor + EnqueueJob(adminJob model.AdminJob, done chan<- error) + RebalanceTables(cfID model.ChangeFeedID, done chan<- error) + ScheduleTable( + cfID model.ChangeFeedID, toCapture model.CaptureID, + tableID model.TableID, done chan<- error, + ) + WriteDebugInfo(w io.Writer, done chan<- error) + Query(query *Query, done chan<- error) + AsyncStop() +} + +type ownerImpl struct { changefeeds map[model.ChangeFeedID]*changefeed captures map[model.CaptureID]*model.CaptureInfo @@ -88,15 +102,16 @@ type Owner struct { closed int32 // bootstrapped specifies whether the owner has been initialized. // This will only be done when the owner starts the first Tick. - // NOTICE: Do not use it in a method other than tick unexpectedly, as it is not a thread-safe value. + // NOTICE: Do not use it in a method other than tick unexpectedly, + // as it is not a thread-safe value. bootstrapped bool newChangefeed func(id model.ChangeFeedID, gcManager gc.Manager) *changefeed } // NewOwner creates a new Owner -func NewOwner(pdClient pd.Client) *Owner { - return &Owner{ +func NewOwner(pdClient pd.Client) Owner { + return &ownerImpl{ changefeeds: make(map[model.ChangeFeedID]*changefeed), gcManager: gc.NewManager(pdClient), lastTickTime: time.Now(), @@ -110,8 +125,8 @@ func NewOwner4Test( newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error), newSink func() DDLSink, pdClient pd.Client, -) *Owner { - o := NewOwner(pdClient) +) Owner { + o := NewOwner(pdClient).(*ownerImpl) // Most tests do not need to test bootstrap. o.bootstrapped = true o.newChangefeed = func(id model.ChangeFeedID, gcManager gc.Manager) *changefeed { @@ -121,7 +136,7 @@ func NewOwner4Test( } // Tick implements the Reactor interface -func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { +func (o *ownerImpl) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) (nextState orchestrator.ReactorState, err error) { failpoint.Inject("owner-run-with-error", func() { failpoint.Return(nil, errors.New("owner run with injected error")) }) @@ -201,58 +216,65 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) } // EnqueueJob enqueues an admin job into an internal queue, and the Owner will handle the job in the next tick -func (o *Owner) EnqueueJob(adminJob model.AdminJob) { +// `done` must be buffered to prevernt blocking owner. +func (o *ownerImpl) EnqueueJob(adminJob model.AdminJob, done chan<- error) { o.pushOwnerJob(&ownerJob{ - tp: ownerJobTypeAdminJob, - adminJob: &adminJob, - changefeedID: adminJob.CfID, - done: make(chan struct{}), + Tp: ownerJobTypeAdminJob, + AdminJob: &adminJob, + ChangefeedID: adminJob.CfID, + done: done, }) } -// TriggerRebalance triggers a rebalance for the specified changefeed -func (o *Owner) TriggerRebalance(cfID model.ChangeFeedID) { +// RebalanceTables triggers a rebalance for the specified changefeed +// `done` must be buffered to prevernt blocking owner. +func (o *ownerImpl) RebalanceTables(cfID model.ChangeFeedID, done chan<- error) { o.pushOwnerJob(&ownerJob{ - tp: ownerJobTypeRebalance, - changefeedID: cfID, - done: make(chan struct{}), + Tp: ownerJobTypeRebalance, + ChangefeedID: cfID, + done: done, }) } -// ManualSchedule moves a table from a capture to another capture -func (o *Owner) ManualSchedule(cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID) { +// ScheduleTable moves a table from a capture to another capture +// `done` must be buffered to prevernt blocking owner. +func (o *ownerImpl) ScheduleTable( + cfID model.ChangeFeedID, toCapture model.CaptureID, tableID model.TableID, + done chan<- error, +) { o.pushOwnerJob(&ownerJob{ - tp: ownerJobTypeManualSchedule, - changefeedID: cfID, - targetCaptureID: toCapture, - tableID: tableID, - done: make(chan struct{}), + Tp: ownerJobTypeScheduleTable, + ChangefeedID: cfID, + TargetCaptureID: toCapture, + TableID: tableID, + done: done, }) } // WriteDebugInfo writes debug info into the specified http writer -func (o *Owner) WriteDebugInfo(w io.Writer) { - timeout := time.Second * 3 - done := make(chan struct{}) +func (o *ownerImpl) WriteDebugInfo(w io.Writer, done chan<- error) { o.pushOwnerJob(&ownerJob{ - tp: ownerJobTypeDebugInfo, + Tp: ownerJobTypeDebugInfo, debugInfoWriter: w, done: done, }) - // wait the debug info printed - select { - case <-done: - case <-time.After(timeout): - fmt.Fprintf(w, "failed to print debug info for owner\n") - } +} + +// Query queries owner internal information. +func (o *ownerImpl) Query(query *Query, done chan<- error) { + o.pushOwnerJob(&ownerJob{ + Tp: ownerJobTypeQuery, + query: query, + done: done, + }) } // AsyncStop stops the owner asynchronously -func (o *Owner) AsyncStop() { +func (o *ownerImpl) AsyncStop() { atomic.StoreInt32(&o.closed, 1) } -func (o *Owner) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { +func (o *ownerImpl) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { return nil, info != nil, nil }) @@ -277,7 +299,7 @@ func (o *Owner) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { } // Bootstrap checks if the state contains incompatible or incorrect information and tries to fix it. -func (o *Owner) Bootstrap(state *orchestrator.GlobalReactorState) { +func (o *ownerImpl) Bootstrap(state *orchestrator.GlobalReactorState) { log.Info("Start bootstrapping") fixChangefeedInfos(state) } @@ -297,7 +319,7 @@ func fixChangefeedInfos(state *orchestrator.GlobalReactorState) { } } -func (o *Owner) updateMetrics(state *orchestrator.GlobalReactorState) { +func (o *ownerImpl) updateMetrics(state *orchestrator.GlobalReactorState) { // Keep the value of prometheus expression `rate(counter)` = 1 // Please also change alert rule in ticdc.rules.yml when change the expression value. now := time.Now() @@ -321,7 +343,7 @@ func (o *Owner) updateMetrics(state *orchestrator.GlobalReactorState) { } } -func (o *Owner) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { +func (o *ownerImpl) clusterVersionConsistent(captures map[model.CaptureID]*model.CaptureInfo) bool { myVersion := version.ReleaseVersion for _, capture := range captures { if myVersion != capture.Version { @@ -335,24 +357,26 @@ func (o *Owner) clusterVersionConsistent(captures map[model.CaptureID]*model.Cap return true } -func (o *Owner) handleJobs() { +func (o *ownerImpl) handleJobs() { jobs := o.takeOwnerJobs() for _, job := range jobs { - changefeedID := job.changefeedID + changefeedID := job.ChangefeedID cfReactor, exist := o.changefeeds[changefeedID] - if !exist && job.tp != ownerJobTypeQuery { + if !exist && job.Tp != ownerJobTypeQuery { log.Warn("changefeed not found when handle a job", zap.Reflect("job", job)) + job.done <- cerror.ErrChangeFeedNotExists.FastGenByArgs(job.ChangefeedID) + close(job.done) continue } - switch job.tp { + switch job.Tp { case ownerJobTypeAdminJob: - cfReactor.feedStateManager.PushAdminJob(job.adminJob) - case ownerJobTypeManualSchedule: - cfReactor.scheduler.MoveTable(job.tableID, job.targetCaptureID) + cfReactor.feedStateManager.PushAdminJob(job.AdminJob) + case ownerJobTypeScheduleTable: + cfReactor.scheduler.MoveTable(job.TableID, job.TargetCaptureID) case ownerJobTypeRebalance: cfReactor.scheduler.Rebalance() case ownerJobTypeQuery: - o.handleQueries(job.query) + job.done <- o.handleQueries(job.query) case ownerJobTypeDebugInfo: // TODO: implement this function } @@ -360,9 +384,9 @@ func (o *Owner) handleJobs() { } } -func (o *Owner) handleQueries(query *ownerQuery) { - switch query.tp { - case ownerQueryAllChangeFeedStatuses: +func (o *ownerImpl) handleQueries(query *Query) error { + switch query.Tp { + case QueryAllChangeFeedStatuses: ret := map[model.ChangeFeedID]*model.ChangeFeedStatus{} for cfID, cfReactor := range o.changefeeds { ret[cfID] = &model.ChangeFeedStatus{} @@ -376,8 +400,8 @@ func (o *Owner) handleQueries(query *ownerQuery) { ret[cfID].CheckpointTs = cfReactor.state.Status.CheckpointTs ret[cfID].AdminJobType = cfReactor.state.Status.AdminJobType } - query.data = ret - case ownerQueryAllChangeFeedInfo: + query.Data = ret + case QueryAllChangeFeedInfo: ret := map[model.ChangeFeedID]*model.ChangeFeedInfo{} for cfID, cfReactor := range o.changefeeds { if cfReactor.state == nil { @@ -390,20 +414,17 @@ func (o *Owner) handleQueries(query *ownerQuery) { var err error ret[cfID], err = cfReactor.state.Info.Clone() if err != nil { - query.err = errors.Trace(err) - return + return errors.Trace(err) } } - query.data = ret - case ownerQueryAllTaskStatuses: - cfReactor, ok := o.changefeeds[query.changeFeedID] + query.Data = ret + case QueryAllTaskStatuses: + cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { - query.err = cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.changeFeedID) - return + return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) } if cfReactor.state == nil { - query.err = cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.changeFeedID) - return + return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) } var ret map[model.CaptureID]*model.TaskStatus @@ -412,8 +433,7 @@ func (o *Owner) handleQueries(query *ownerQuery) { var err error ret, err = provider.GetTaskStatuses() if err != nil { - query.err = errors.Trace(err) - return + return errors.Trace(err) } } else { ret = map[model.CaptureID]*model.TaskStatus{} @@ -421,12 +441,11 @@ func (o *Owner) handleQueries(query *ownerQuery) { ret[captureID] = taskStatus.Clone() } } - query.data = ret - case ownerQueryTaskPositions: - cfReactor, ok := o.changefeeds[query.changeFeedID] + query.Data = ret + case QueryTaskPositions: + cfReactor, ok := o.changefeeds[query.ChangeFeedID] if !ok { - query.err = cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.changeFeedID) - return + return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) } var ret map[model.CaptureID]*model.TaskPosition @@ -435,21 +454,19 @@ func (o *Owner) handleQueries(query *ownerQuery) { var err error ret, err = provider.GetTaskPositions() if err != nil { - query.err = errors.Trace(err) - return + return errors.Trace(err) } } else { if cfReactor.state == nil { - query.err = cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.changeFeedID) - return + return cerror.ErrChangeFeedNotExists.GenWithStackByArgs(query.ChangeFeedID) } ret = map[model.CaptureID]*model.TaskPosition{} for captureID, taskPosition := range cfReactor.state.TaskPositions { ret[captureID] = taskPosition.Clone() } } - query.data = ret - case ownerQueryProcessors: + query.Data = ret + case QueryProcessors: var ret []*model.ProcInfoSnap for cfID, cfReactor := range o.changefeeds { if cfReactor.state == nil { @@ -462,8 +479,8 @@ func (o *Owner) handleQueries(query *ownerQuery) { }) } } - query.data = ret - case ownerQueryCaptures: + query.Data = ret + case QueryCaptures: var ret []*model.CaptureInfo for _, captureInfo := range o.captures { ret = append(ret, &model.CaptureInfo{ @@ -472,11 +489,12 @@ func (o *Owner) handleQueries(query *ownerQuery) { Version: captureInfo.Version, }) } - query.data = ret + query.Data = ret } + return nil } -func (o *Owner) takeOwnerJobs() []*ownerJob { +func (o *ownerImpl) takeOwnerJobs() []*ownerJob { o.ownerJobQueueMu.Lock() defer o.ownerJobQueueMu.Unlock() @@ -485,13 +503,13 @@ func (o *Owner) takeOwnerJobs() []*ownerJob { return jobs } -func (o *Owner) pushOwnerJob(job *ownerJob) { +func (o *ownerImpl) pushOwnerJob(job *ownerJob) { o.ownerJobQueueMu.Lock() defer o.ownerJobQueueMu.Unlock() o.ownerJobQueue = append(o.ownerJobQueue, job) } -func (o *Owner) updateGCSafepoint( +func (o *ownerImpl) updateGCSafepoint( ctx context.Context, state *orchestrator.GlobalReactorState, ) error { forceUpdate := false @@ -524,6 +542,6 @@ func (o *Owner) updateGCSafepoint( } // StatusProvider returns a StatusProvider -func (o *Owner) StatusProvider() StatusProvider { +func (o *ownerImpl) StatusProvider() StatusProvider { return &ownerStatusProvider{owner: o} } diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index bcca13f5dad..4bf12827ac3 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -44,7 +44,7 @@ func (m *mockManager) CheckStaleCheckpointTs( var _ gc.Manager = (*mockManager)(nil) -func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*Owner, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { +func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*ownerImpl, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { ctx.GlobalVars().PDClient = &gc.MockPDClient{ UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { return safePoint, nil @@ -68,7 +68,7 @@ func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*Owner, *orchestrat captureBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() require.Nil(t, err) tester.MustUpdate(cdcKey.String(), captureBytes) - return owner, state, tester + return owner.(*ownerImpl), state, tester } func TestCreateRemoveChangefeed(t *testing.T) { @@ -128,9 +128,11 @@ func TestCreateRemoveChangefeed(t *testing.T) { require.NotNil(t, err) // this tick create remove changefeed patches - owner.EnqueueJob(removeJob) + done := make(chan error, 1) + owner.EnqueueJob(removeJob, done) _, err = owner.Tick(ctx, state) require.Nil(t, err) + require.Nil(t, <-done) // apply patches and update owner's in memory changefeed states tester.MustApplyPatches() @@ -162,17 +164,20 @@ func TestStopChangefeed(t *testing.T) { require.Nil(t, err) require.Contains(t, owner.changefeeds, changefeedID) // remove changefeed forcibly + done := make(chan error, 1) owner.EnqueueJob(model.AdminJob{ CfID: changefeedID, Type: model.AdminRemove, Opts: &model.AdminJobOption{ ForceRemove: true, }, - }) + }, done) // this tick to clean the leak info fo the removed changefeed _, err = owner.Tick(ctx, state) require.Nil(t, err) + require.Nil(t, <-done) + // this tick to remove the changefeed state in memory tester.MustApplyPatches() _, err = owner.Tick(ctx, state) @@ -301,15 +306,19 @@ func TestAdminJob(t *testing.T) { ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() + done1 := make(chan error, 1) owner, _, _ := createOwner4Test(ctx, t) owner.EnqueueJob(model.AdminJob{ CfID: "test-changefeed1", Type: model.AdminResume, - }) - owner.TriggerRebalance("test-changefeed2") - owner.ManualSchedule("test-changefeed3", "test-caputre1", 10) + }, done1) + done2 := make(chan error, 1) + owner.RebalanceTables("test-changefeed2", done2) + done3 := make(chan error, 1) + owner.ScheduleTable("test-changefeed3", "test-caputre1", 10, done3) + done4 := make(chan error, 1) var buf bytes.Buffer - owner.WriteDebugInfo(&buf) + owner.WriteDebugInfo(&buf, done4) // remove job.done, it's hard to check deep equals jobs := owner.takeOwnerJobs() @@ -320,22 +329,22 @@ func TestAdminJob(t *testing.T) { } require.Equal(t, jobs, []*ownerJob{ { - tp: ownerJobTypeAdminJob, - adminJob: &model.AdminJob{ + Tp: ownerJobTypeAdminJob, + AdminJob: &model.AdminJob{ CfID: "test-changefeed1", Type: model.AdminResume, }, - changefeedID: "test-changefeed1", + ChangefeedID: "test-changefeed1", }, { - tp: ownerJobTypeRebalance, - changefeedID: "test-changefeed2", + Tp: ownerJobTypeRebalance, + ChangefeedID: "test-changefeed2", }, { - tp: ownerJobTypeManualSchedule, - changefeedID: "test-changefeed3", - targetCaptureID: "test-caputre1", - tableID: 10, + Tp: ownerJobTypeScheduleTable, + ChangefeedID: "test-changefeed3", + TargetCaptureID: "test-caputre1", + TableID: 10, }, { - tp: ownerJobTypeDebugInfo, + Tp: ownerJobTypeDebugInfo, debugInfoWriter: &buf, }, }) @@ -344,7 +353,7 @@ func TestAdminJob(t *testing.T) { func TestUpdateGCSafePoint(t *testing.T) { mockPDClient := &gc.MockPDClient{} - o := NewOwner(mockPDClient) + o := NewOwner(mockPDClient).(*ownerImpl) o.gcManager = gc.NewManager(mockPDClient) ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) diff --git a/cdc/owner/status_provider.go b/cdc/owner/status_provider.go index 22ecdb2c072..54337e796f7 100644 --- a/cdc/owner/status_provider.go +++ b/cdc/owner/status_provider.go @@ -50,37 +50,49 @@ type StatusProvider interface { GetCaptures(ctx context.Context) ([]*model.CaptureInfo, error) } -type ownerQueryType int32 +// QueryType is the type of different queries. +type QueryType int32 const ( - ownerQueryAllChangeFeedStatuses = iota - ownerQueryAllChangeFeedInfo - ownerQueryAllTaskStatuses - ownerQueryTaskPositions - ownerQueryProcessors - ownerQueryCaptures + // QueryAllChangeFeedStatuses query all changefeed status. + QueryAllChangeFeedStatuses QueryType = iota + // QueryAllChangeFeedInfo is the type of query all changefeed info. + QueryAllChangeFeedInfo + // QueryAllTaskStatuses is the type of query all task statuses. + QueryAllTaskStatuses + // QueryTaskPositions is the type of query task positions. + QueryTaskPositions + // QueryProcessors is the type of query processors. + QueryProcessors + // QueryCaptures is the type of query captures info. + QueryCaptures ) -type ownerQuery struct { - tp ownerQueryType - changeFeedID model.ChangeFeedID +// Query wraps query command and return results. +type Query struct { + Tp QueryType + ChangeFeedID model.ChangeFeedID - data interface{} - err error + Data interface{} +} + +// NewStatusProvider returns a new StatusProvider for the owner. +func NewStatusProvider(owner Owner) StatusProvider { + return &ownerStatusProvider{owner: owner} } type ownerStatusProvider struct { - owner *Owner + owner Owner } func (p *ownerStatusProvider) GetAllChangeFeedStatuses(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedStatus, error) { - query := &ownerQuery{ - tp: ownerQueryAllChangeFeedStatuses, + query := &Query{ + Tp: QueryAllChangeFeedStatuses, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.(map[model.ChangeFeedID]*model.ChangeFeedStatus), nil + return query.Data.(map[model.ChangeFeedID]*model.ChangeFeedStatus), nil } func (p *ownerStatusProvider) GetChangeFeedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedStatus, error) { @@ -96,13 +108,13 @@ func (p *ownerStatusProvider) GetChangeFeedStatus(ctx context.Context, changefee } func (p *ownerStatusProvider) GetAllChangeFeedInfo(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedInfo, error) { - query := &ownerQuery{ - tp: ownerQueryAllChangeFeedInfo, + query := &Query{ + Tp: QueryAllChangeFeedInfo, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.(map[model.ChangeFeedID]*model.ChangeFeedInfo), nil + return query.Data.(map[model.ChangeFeedID]*model.ChangeFeedInfo), nil } func (p *ownerStatusProvider) GetChangeFeedInfo(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedInfo, error) { @@ -118,64 +130,59 @@ func (p *ownerStatusProvider) GetChangeFeedInfo(ctx context.Context, changefeedI } func (p *ownerStatusProvider) GetAllTaskStatuses(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskStatus, error) { - query := &ownerQuery{ - tp: ownerQueryAllTaskStatuses, - changeFeedID: changefeedID, + query := &Query{ + Tp: QueryAllTaskStatuses, + ChangeFeedID: changefeedID, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.(map[model.CaptureID]*model.TaskStatus), nil + return query.Data.(map[model.CaptureID]*model.TaskStatus), nil } func (p *ownerStatusProvider) GetTaskPositions(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskPosition, error) { - query := &ownerQuery{ - tp: ownerQueryTaskPositions, - changeFeedID: changefeedID, + query := &Query{ + Tp: QueryTaskPositions, + ChangeFeedID: changefeedID, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.(map[model.CaptureID]*model.TaskPosition), nil + return query.Data.(map[model.CaptureID]*model.TaskPosition), nil } func (p *ownerStatusProvider) GetProcessors(ctx context.Context) ([]*model.ProcInfoSnap, error) { - query := &ownerQuery{ - tp: ownerQueryProcessors, + query := &Query{ + Tp: QueryProcessors, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.([]*model.ProcInfoSnap), nil + return query.Data.([]*model.ProcInfoSnap), nil } func (p *ownerStatusProvider) GetCaptures(ctx context.Context) ([]*model.CaptureInfo, error) { - query := &ownerQuery{ - tp: ownerQueryCaptures, + query := &Query{ + Tp: QueryCaptures, } if err := p.sendQueryToOwner(ctx, query); err != nil { return nil, errors.Trace(err) } - return query.data.([]*model.CaptureInfo), nil + return query.Data.([]*model.CaptureInfo), nil } -func (p *ownerStatusProvider) sendQueryToOwner(ctx context.Context, query *ownerQuery) error { - doneCh := make(chan struct{}) - job := &ownerJob{ - tp: ownerJobTypeQuery, - query: query, - done: doneCh, - } - p.owner.pushOwnerJob(job) +func (p *ownerStatusProvider) sendQueryToOwner(ctx context.Context, query *Query) error { + doneCh := make(chan error, 1) + p.owner.Query(query, doneCh) select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case <-doneCh: + case err := <-doneCh: + if err != nil { + return errors.Trace(err) + } } - if query.err != nil { - return errors.Trace(query.err) - } return nil } diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index e386f6cc995..a02ac106f86 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -43,7 +43,7 @@ const ( type command struct { tp commandTp payload interface{} - done chan struct{} + done chan<- error } // Manager is a manager of processor, which maintains the state and behavior of processors @@ -152,33 +152,42 @@ func (m *Manager) closeProcessor(changefeedID model.ChangeFeedID) { } } -// AsyncClose sends a close signal to Manager and closing all processors +// AsyncClose sends a signal to Manager to close all processors. func (m *Manager) AsyncClose() { - m.sendCommand(commandTpClose, nil) + timeout := 3 * time.Second + ctx, cancel := context.WithTimeout(context.TODO(), timeout) + defer cancel() + done := make(chan error, 1) + err := m.sendCommand(ctx, commandTpClose, nil, done) + if err != nil { + log.Warn("async close failed", zap.Error(err)) + } } // WriteDebugInfo write the debug info to Writer -func (m *Manager) WriteDebugInfo(w io.Writer) { - timeout := time.Second * 3 - done := m.sendCommand(commandTpWriteDebugInfo, w) - // wait the debug info printed - select { - case <-done: - case <-time.After(timeout): - fmt.Fprintf(w, "failed to print debug info for processor\n") +func (m *Manager) WriteDebugInfo( + ctx context.Context, w io.Writer, done chan<- error, +) { + err := m.sendCommand(ctx, commandTpWriteDebugInfo, w, done) + if err != nil { + log.Warn("send command commandTpWriteDebugInfo failed", zap.Error(err)) } } -func (m *Manager) sendCommand(tp commandTp, payload interface{}) chan struct{} { - timeout := time.Second * 3 - cmd := &command{tp: tp, payload: payload, done: make(chan struct{})} +// sendCommands sends command to manager. +// `done` is closed upon command completion or sendCommand returns error. +func (m *Manager) sendCommand( + ctx context.Context, tp commandTp, payload interface{}, done chan<- error, +) error { + cmd := &command{tp: tp, payload: payload, done: done} select { + case <-ctx.Done(): + close(done) + return errors.Trace(ctx.Err()) case m.commandQueue <- cmd: - case <-time.After(timeout): - close(cmd.done) - log.Warn("the command queue is full, ignore this command", zap.Any("command", cmd)) + // FIXME: signal EtcdWorker to handle commands ASAP. } - return cmd.done + return nil } func (m *Manager) handleCommand() error { @@ -194,6 +203,7 @@ func (m *Manager) handleCommand() error { for changefeedID := range m.processors { m.closeProcessor(changefeedID) } + // FIXME: we should drain command queue and signal callers an error. return cerrors.ErrReactorFinished case commandTpWriteDebugInfo: w := cmd.payload.(io.Writer) diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index 8415af305aa..8d0fe700995 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -15,6 +15,7 @@ package processor import ( "bytes" + "context" "fmt" "math" "testing" @@ -170,8 +171,10 @@ func TestDebugInfo(t *testing.T) { s.tester.MustApplyPatches() } }() + doneM := make(chan error, 1) buf := bytes.NewBufferString("") - s.manager.WriteDebugInfo(buf) + s.manager.WriteDebugInfo(ctx, buf, doneM) + <-doneM require.Greater(t, len(buf.String()), 0) s.manager.AsyncClose() <-done @@ -218,3 +221,19 @@ func TestClose(t *testing.T) { s.tester.MustApplyPatches() require.Len(t, s.manager.processors, 0) } + +func TestSendCommandError(t *testing.T) { + m := NewManager() + ctx, cancel := context.WithCancel(context.TODO()) + cancel() + // Use unbuffered channel to stable test. + m.commandQueue = make(chan *command) + done := make(chan error, 1) + err := m.sendCommand(ctx, commandTpClose, nil, done) + require.Error(t, err) + select { + case <-done: + case <-time.After(time.Second): + require.FailNow(t, "done must be closed") + } +} diff --git a/cdc/server_test.go b/cdc/server_test.go index b6eccf3c74b..784b7f15543 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -188,7 +188,7 @@ func TestServerTLSWithoutCommonName(t *testing.T) { config.StoreGlobalServerConfig(conf) server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test(false) + server.capture = capture.NewCapture4Test(nil) require.Nil(t, err) err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) require.Nil(t, err) @@ -264,7 +264,7 @@ func TestServerTLSWithCommonName(t *testing.T) { config.StoreGlobalServerConfig(conf) server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test(false) + server.capture = capture.NewCapture4Test(nil) require.Nil(t, err) err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) require.Nil(t, err) diff --git a/errors.toml b/errors.toml index 8726dc63a35..7e918972366 100755 --- a/errors.toml +++ b/errors.toml @@ -103,7 +103,7 @@ campaign owner failed ["CDC:ErrCaptureNotExist"] error = ''' -capture not exists, key: %s +capture not exists, %s ''' ["CDC:ErrCaptureRegister"] @@ -123,12 +123,12 @@ capture suicide ["CDC:ErrChangeFeedAlreadyExists"] error = ''' -changefeed already exists, key: %s +changefeed already exists, %s ''' ["CDC:ErrChangeFeedNotExists"] error = ''' -changefeed not exists, key: %s +changefeed not exists, %s ''' ["CDC:ErrChangefeedAbnormalState"] @@ -978,12 +978,12 @@ fail to create changefeed because target-ts %d is earlier than start-ts %d ["CDC:ErrTaskPositionNotExists"] error = ''' -task position not exists, key: %s +task position not exists, %s ''' ["CDC:ErrTaskStatusNotExists"] error = ''' -task status not exists, key: %s +task status not exists, %s ''' ["CDC:ErrTiKVEventFeed"] diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 5acab118024..94c26419d7d 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -21,11 +21,11 @@ import ( var ( // kv related errors ErrWriteTsConflict = errors.Normalize("write ts conflict", errors.RFCCodeText("CDC:ErrWriteTsConflict")) - ErrChangeFeedNotExists = errors.Normalize("changefeed not exists, key: %s", errors.RFCCodeText("CDC:ErrChangeFeedNotExists")) - ErrChangeFeedAlreadyExists = errors.Normalize("changefeed already exists, key: %s", errors.RFCCodeText("CDC:ErrChangeFeedAlreadyExists")) - ErrTaskStatusNotExists = errors.Normalize("task status not exists, key: %s", errors.RFCCodeText("CDC:ErrTaskStatusNotExists")) - ErrTaskPositionNotExists = errors.Normalize("task position not exists, key: %s", errors.RFCCodeText("CDC:ErrTaskPositionNotExists")) - ErrCaptureNotExist = errors.Normalize("capture not exists, key: %s", errors.RFCCodeText("CDC:ErrCaptureNotExist")) + ErrChangeFeedNotExists = errors.Normalize("changefeed not exists, %s", errors.RFCCodeText("CDC:ErrChangeFeedNotExists")) + ErrChangeFeedAlreadyExists = errors.Normalize("changefeed already exists, %s", errors.RFCCodeText("CDC:ErrChangeFeedAlreadyExists")) + ErrTaskStatusNotExists = errors.Normalize("task status not exists, %s", errors.RFCCodeText("CDC:ErrTaskStatusNotExists")) + ErrTaskPositionNotExists = errors.Normalize("task position not exists, %s", errors.RFCCodeText("CDC:ErrTaskPositionNotExists")) + ErrCaptureNotExist = errors.Normalize("capture not exists, %s", errors.RFCCodeText("CDC:ErrCaptureNotExist")) ErrGetAllStoresFailed = errors.Normalize("get stores from pd failed", errors.RFCCodeText("CDC:ErrGetAllStoresFailed")) ErrMetaListDatabases = errors.Normalize("meta store list databases", errors.RFCCodeText("CDC:ErrMetaListDatabases")) ErrGRPCDialFailed = errors.Normalize("grpc dial failed", errors.RFCCodeText("CDC:ErrGRPCDialFailed")) diff --git a/scripts/check-copyright.sh b/scripts/check-copyright.sh index 99b8848f15f..bd2d3387a78 100755 --- a/scripts/check-copyright.sh +++ b/scripts/check-copyright.sh @@ -1,4 +1,4 @@ -result=$(find ./ -name "*.go" | grep -vE '\.pb\.go|vendor/|leaktest.go|kv_gen|redo_gen|sink_gen|pbmock|\.pb\.gw\.go|statik.go|openapi/gen\..*\.go|embedded_asserts.go|empty_asserts.go|docs/swagger' | +result=$(find ./ -name "*.go" | grep -vE '\.pb\.go|vendor/|leaktest.go|kv_gen|redo_gen|sink_gen|pbmock|\.pb\.gw\.go|statik.go|openapi/gen\..*\.go|embedded_asserts.go|empty_asserts.go|docs/swagger|owner/mock' | while read -r file_path; do head=$(head -n 1 "$file_path") if [[ ! "$head" =~ Copyright\ 20[0-9][0-9]\ PingCAP,\ Inc\. ]]; then From edad7dabc73599d2932a5f9b042de18e935595bb Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 11 Feb 2022 17:59:38 +0800 Subject: [PATCH 50/72] syncer(dm): use DML library (#4313) ref pingcap/tiflow#3242 --- dm/pkg/schema/tracker.go | 4 +- dm/syncer/causality.go | 7 +- dm/syncer/causality_test.go | 143 +--- dm/syncer/checkpoint_flush_worker.go | 12 +- dm/syncer/compactor.go | 76 +-- dm/syncer/compactor_test.go | 205 +++--- dm/syncer/ddl_test.go | 12 +- dm/syncer/dml.go | 927 ++++---------------------- dm/syncer/dml_test.go | 631 +++++------------- dm/syncer/dml_worker.go | 107 +-- dm/syncer/dml_worker_test.go | 113 ++++ dm/syncer/job.go | 45 +- dm/syncer/job_test.go | 60 +- dm/syncer/syncer.go | 80 +-- dm/syncer/syncer_test.go | 123 ++-- dm/tests/downstream_diff_index/run.sh | 2 +- dm/tests/shardddl1/run.sh | 8 +- pkg/sqlmodel/causality_test.go | 132 ++++ pkg/sqlmodel/reduce.go | 6 +- pkg/sqlmodel/reduce_test.go | 2 +- pkg/sqlmodel/row_change.go | 20 + 21 files changed, 996 insertions(+), 1719 deletions(-) create mode 100644 dm/syncer/dml_worker_test.go diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 1ee9d1015a4..de696115e71 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -397,13 +397,13 @@ func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string dti, ok := tr.dsTracker.tableInfos[tableID] if !ok { tctx.Logger.Info("Downstream schema tracker init. ", zap.String("tableID", tableID)) - ti, err := tr.getTableInfoByCreateStmt(tctx, tableID) + downstreamTI, err := tr.getTableInfoByCreateStmt(tctx, tableID) if err != nil { tctx.Logger.Error("Init dowstream schema info error. ", zap.String("tableID", tableID), zap.Error(err)) return nil, err } - dti = GetDownStreamTI(ti, originTi) + dti = GetDownStreamTI(downstreamTI, originTi) tr.dsTracker.tableInfos[tableID] = dti } return dti, nil diff --git a/dm/syncer/causality.go b/dm/syncer/causality.go index ab5eb8e8699..72b7546797b 100644 --- a/dm/syncer/causality.go +++ b/dm/syncer/causality.go @@ -20,6 +20,7 @@ import ( "go.uber.org/zap" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/syncer/metrics" ) @@ -79,7 +80,7 @@ func (c *causality) run() { c.relation.gc(j.flushSeq) continue default: - keys := j.dml.identifyKeys(c.sessCtx) + keys := j.dml.CausalityKeys() // detectConflict before add if c.detectConflict(keys) { @@ -87,8 +88,8 @@ func (c *causality) run() { c.outCh <- newConflictJob(c.workerCount) c.relation.clear() } - j.dml.key = c.add(keys) - c.logger.Debug("key for keys", zap.String("key", j.dml.key), zap.Strings("keys", keys)) + j.dmlQueueKey = c.add(keys) + c.logger.Debug("key for keys", zap.String("key", j.dmlQueueKey), zap.Strings("keys", keys)) } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index 1109b8c1b80..b3acb6d0810 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -15,21 +15,19 @@ package syncer import ( "math" + "testing" "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) func (s *testSyncerSuite) TestDetectConflict(c *C) { @@ -65,26 +63,11 @@ func (s *testSyncerSuite) TestDetectConflict(c *C) { c.Assert(ca.relation.len(), Equals, 0) } -func (s *testSyncerSuite) TestCasuality(c *C) { - p := parser.New() - se := mock.NewContext() +func TestCausality(t *testing.T) { + t.Parallel() + schemaStr := "create table tb(a int primary key, b int unique);" - ti, err := createTableInfo(p, se, int64(0), schemaStr) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) + ti := mockTableInfo(t, schemaStr) jobCh := make(chan *job, 10) syncer := &Syncer{ @@ -100,124 +83,44 @@ func (s *testSyncerSuite) TestCasuality(c *C) { } causalityCh := causalityWrap(jobCh, syncer) testCases := []struct { - op opType - oldVals []interface{} - vals []interface{} + preVals []interface{} + postVals []interface{} }{ { - op: insert, - vals: []interface{}{1, 2}, + postVals: []interface{}{1, 2}, }, { - op: insert, - vals: []interface{}{2, 3}, + postVals: []interface{}{2, 3}, }, { - op: update, - oldVals: []interface{}{2, 3}, - vals: []interface{}{3, 4}, + preVals: []interface{}{2, 3}, + postVals: []interface{}{3, 4}, }, { - op: del, - vals: []interface{}{1, 2}, + preVals: []interface{}{1, 2}, }, { - op: insert, - vals: []interface{}{1, 3}, + postVals: []interface{}{1, 3}, }, } - results := []opType{insert, insert, update, del, conflict, insert} - table := &filter.Table{Schema: "test", Name: "t1"} + results := []opType{dml, dml, dml, dml, conflict, dml} + table := &cdcmodel.TableName{Schema: "test", Table: "t1"} location := binlog.NewLocation("") ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) + change := sqlmodel.NewRowChange(table, nil, tc.preVals, tc.postVals, ti, nil, nil) + job := newDMLJob(change, ec) jobCh <- job } - c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + require.Eventually(t, func() bool { return len(causalityCh) == len(results) - }), IsTrue) + }, 3*time.Second, 100*time.Millisecond) for _, op := range results { job := <-causalityCh - c.Assert(job.tp, Equals, op) - } -} - -func (s *testSyncerSuite) TestCasualityWithPrefixIndex(c *C) { - p := parser.New() - se := mock.NewContext() - schemaStr := "create table t (c1 text, c2 int unique, unique key c1(c1(3)));" - ti, err := createTableInfo(p, se, int64(0), schemaStr) - c.Assert(err, IsNil) - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) - c.Assert(len(downTi.AvailableUKIndexList) == 2, IsTrue) - tiIndex := downTi.AvailableUKIndexList[0] - - jobCh := make(chan *job, 10) - syncer := &Syncer{ - cfg: &config.SubTaskConfig{ - SyncerConfig: config.SyncerConfig{ - QueueSize: 1024, - }, - Name: "task", - SourceID: "source", - }, - tctx: tcontext.Background().WithLogger(log.L()), - sessCtx: utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}), - } - causalityCh := causalityWrap(jobCh, syncer) - testCases := []struct { - op opType - oldVals []interface{} - vals []interface{} - }{ - { - op: insert, - vals: []interface{}{"1234", 1}, - }, - { - op: insert, - vals: []interface{}{"2345", 2}, - }, - { - op: update, - oldVals: []interface{}{"2345", 2}, - vals: []interface{}{"2345", 3}, - }, - { - op: del, - vals: []interface{}{"1234", 1}, - }, - { - op: insert, - vals: []interface{}{"2345", 1}, - }, - } - results := []opType{insert, insert, update, del, conflict, insert} - resultKeys := []string{"123.c1.", "234.c1.", "234.c1.", "123.c1.", "conflict", "234.c1."} - table := &filter.Table{Schema: "test", Name: "t1"} - location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} - - for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) - jobCh <- job - } - - c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return len(causalityCh) == len(results) - }), IsTrue) - - for i, op := range results { - job := <-causalityCh - if job.tp != conflict { - c.Assert(job.dml.key, Equals, resultKeys[i]) - } - c.Assert(job.tp, Equals, op) + require.Equal(t, op, job.tp) } } diff --git a/dm/syncer/checkpoint_flush_worker.go b/dm/syncer/checkpoint_flush_worker.go index e781c4b49fd..6d7f12f3aff 100644 --- a/dm/syncer/checkpoint_flush_worker.go +++ b/dm/syncer/checkpoint_flush_worker.go @@ -37,11 +37,11 @@ type checkpointFlushTask struct { } type checkpointFlushWorker struct { - input chan *checkpointFlushTask - cp CheckPoint - execError *atomic.Error - afterFlushFn func(task *checkpointFlushTask) error - addCountFunc func(bool, string, opType, int64, *filter.Table) + input chan *checkpointFlushTask + cp CheckPoint + execError *atomic.Error + afterFlushFn func(task *checkpointFlushTask) error + updateJobMetricsFn func(bool, string, *job) } // Add add a new flush checkpoint job. @@ -60,7 +60,7 @@ func (w *checkpointFlushWorker) Run(ctx *tcontext.Context) { if isAsyncFlush { task.asyncflushJob.flushWg.Wait() - w.addCountFunc(true, adminQueueName, task.asyncflushJob.tp, 1, task.asyncflushJob.targetTable) + w.updateJobMetricsFn(true, adminQueueName, task.asyncflushJob) ctx.L().Info("async flush checkpoint snapshot job has been processed by dml worker, about to flush checkpoint snapshot", zap.Int64("job sequence", task.asyncflushJob.flushSeq), zap.Int("snapshot_id", task.snapshotInfo.id)) } else { ctx.L().Info("about to sync flush checkpoint snapshot", zap.Int("snapshot_id", task.snapshotInfo.id)) diff --git a/dm/syncer/compactor.go b/dm/syncer/compactor.go index 62c907f037d..65800a7a8f3 100644 --- a/dm/syncer/compactor.go +++ b/dm/syncer/compactor.go @@ -19,11 +19,11 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/syncer/metrics" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // compactor compacts multiple statements into one statement. @@ -38,9 +38,9 @@ type compactor struct { buffer []*job // for metrics - task string - source string - addCountFunc func(bool, string, opType, int64, *filter.Table) + task string + source string + updateJobMetricsFn func(bool, string, *job) } // compactorWrap creates and runs a compactor instance. @@ -49,15 +49,15 @@ func compactorWrap(inCh chan *job, syncer *Syncer) chan *job { // TODO: implement ping-pong buffer. bufferSize := syncer.cfg.QueueSize * syncer.cfg.WorkerCount / 4 compactor := &compactor{ - inCh: inCh, - outCh: make(chan *job, bufferSize), - bufferSize: bufferSize, - logger: syncer.tctx.Logger.WithFields(zap.String("component", "compactor")), - keyMap: make(map[string]map[string]int), - buffer: make([]*job, 0, bufferSize), - task: syncer.cfg.Name, - source: syncer.cfg.SourceID, - addCountFunc: syncer.addCount, + inCh: inCh, + outCh: make(chan *job, bufferSize), + bufferSize: bufferSize, + logger: syncer.tctx.Logger.WithFields(zap.String("component", "compactor")), + keyMap: make(map[string]map[string]int), + buffer: make([]*job, 0, bufferSize), + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + updateJobMetricsFn: syncer.updateJobMetrics, } go func() { compactor.run() @@ -89,23 +89,21 @@ func (c *compactor) run() { // set safeMode when receive first job if len(c.buffer) == 0 { - c.safeMode = j.dml.safeMode + c.safeMode = j.safeMode } // if dml has no PK/NOT NULL UK, do not compact it. - if j.dml.identifyColumns() == nil { + if !j.dml.HasNotNullUniqueIdx() { c.buffer = append(c.buffer, j) continue } // if update job update its identify keys, turn it into delete + insert - if j.dml.op == update && j.dml.updateIdentify() { - delDML, insertDML := updateToDelAndInsert(j.dml) + if j.dml.IsIdentityUpdated() { + delDML, insertDML := j.dml.SplitUpdate() delJob := j.clone() - delJob.tp = del delJob.dml = delDML insertJob := j.clone() - insertJob.tp = insert insertJob.dml = insertDML c.compactJob(delJob) @@ -142,7 +140,7 @@ func (c *compactor) flushBuffer() { if j != nil { // set safemode for all jobs by first job in buffer. // or safemode for insert(delete + insert = insert with safemode) - j.dml.safeMode = c.safeMode || j.dml.safeMode + j.safeMode = c.safeMode || j.safeMode c.outCh <- j } } @@ -162,7 +160,7 @@ func (c *compactor) flushBuffer() { // DELETE + UPDATE => X _| // . func (c *compactor) compactJob(j *job) { - tableName := j.dml.targetTableID + tableName := j.dml.TargetTableID() tableKeyMap, ok := c.keyMap[tableName] if !ok { // do not alloc a large buffersize, otherwise if the downstream latency is low @@ -171,7 +169,7 @@ func (c *compactor) compactJob(j *job) { tableKeyMap = c.keyMap[tableName] } - key := j.dml.identifyKey() + key := j.dml.IdentityKey() failpoint.Inject("DownstreamIdentifyKeyCheckInCompact", func(v failpoint.Value) { value, err := strconv.Atoi(key) @@ -192,28 +190,24 @@ func (c *compactor) compactJob(j *job) { prevJob := c.buffer[prevPos] c.logger.Debug("start to compact", zap.Stringer("previous dml", prevJob.dml), zap.Stringer("current dml", j.dml)) - switch j.tp { - case update: - if prevJob.tp == insert { - // INSERT + UPDATE => INSERT - j.tp = insert - j.dml.oldValues = nil - j.dml.originOldValues = nil - j.dml.op = insert + // adjust safemode + skipReduce := false + switch j.dml.Type() { + case sqlmodel.RowChangeUpdate: + if prevJob.dml.Type() == sqlmodel.RowChangeInsert { // DELETE + INSERT + UPDATE => INSERT with safemode - j.dml.safeMode = prevJob.dml.safeMode - } else if prevJob.tp == update { - // UPDATE + UPDATE => UPDATE - j.dml.oldValues = prevJob.dml.oldValues - j.dml.originOldValues = prevJob.dml.originOldValues + j.safeMode = prevJob.safeMode } - case insert: - if prevJob.tp == del { + case sqlmodel.RowChangeInsert: + if prevJob.dml.Type() == sqlmodel.RowChangeDelete { // DELETE + INSERT => INSERT with safemode - j.dml.safeMode = true + j.safeMode = true + skipReduce = true } - case del: - // do nothing because anything + DELETE => DELETE + } + + if !skipReduce { + j.dml.Reduce(prevJob.dml) } // mark previous job as compacted(nil), add new job @@ -221,5 +215,5 @@ func (c *compactor) compactJob(j *job) { tableKeyMap[key] = len(c.buffer) c.buffer = append(c.buffer, j) c.logger.Debug("finish to compact", zap.Stringer("dml", j.dml)) - c.addCountFunc(true, adminQueueName, compact, 1, prevJob.targetTable) + c.updateJobMetricsFn(true, adminQueueName, newCompactJob(prevJob.targetTable)) } diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 506f9581a7f..9118766ab1e 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -20,31 +20,29 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/mock" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // mockExecute mock a kv store. -func mockExecute(kv map[interface{}][]interface{}, dmls []*DML) map[interface{}][]interface{} { +func mockExecute(kv map[interface{}][]interface{}, dmls []*sqlmodel.RowChange) map[interface{}][]interface{} { for _, dml := range dmls { - switch dml.op { - case insert: - kv[dml.values[0]] = dml.values - case update: - delete(kv, dml.oldValues[0]) - kv[dml.values[0]] = dml.values - case del: - delete(kv, dml.values[0]) + switch dml.Type() { + case sqlmodel.RowChangeInsert: + kv[dml.GetPostValues()[0]] = dml.GetPostValues() + case sqlmodel.RowChangeUpdate: + delete(kv, dml.GetPreValues()[0]) + kv[dml.GetPostValues()[0]] = dml.GetPostValues() + case sqlmodel.RowChangeDelete: + delete(kv, dml.GetPreValues()[0]) } } @@ -62,40 +60,25 @@ func randString(n int) string { func (s *testSyncerSuite) TestCompactJob(c *C) { compactor := &compactor{ - bufferSize: 10000, - logger: log.L(), - keyMap: make(map[string]map[string]int), - buffer: make([]*job, 0, 10000), - addCountFunc: func(b bool, s string, ot opType, i int64, t *filter.Table) {}, + bufferSize: 10000, + logger: log.L(), + keyMap: make(map[string]map[string]int), + buffer: make([]*job, 0, 10000), + updateJobMetricsFn: func(bool, string, *job) {}, } location := binlog.NewLocation("") ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} p := parser.New() se := mock.NewContext() - targetTableID := "`test`.`tb`" - sourceTable := &filter.Table{Schema: "test", Name: "tb1"} - targetTable := &filter.Table{Schema: "test", Name: "tb"} + sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb1"} + targetTable := &cdcmodel.TableName{Schema: "test", Table: "tb"} schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) - var dml *DML - var dmls []*DML + var dml *sqlmodel.RowChange + var dmls []*sqlmodel.RowChange dmlNum := 1000000 maxID := 1000 batch := 1000 @@ -111,7 +94,7 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { oldValues, ok := kv[newID] if !ok { // insert - dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, nil, values, ti, nil, nil) } else { if rand.Int()%2 > 0 { // update @@ -125,14 +108,14 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } } } - dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, values, ti, nil, nil) } else { // delete - dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, nil, ti, nil, nil) } } - kv = mockExecute(kv, []*DML{dml}) + kv = mockExecute(kv, []*sqlmodel.RowChange{dml}) dmls = append(dmls, dml) } @@ -148,15 +131,14 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { kv = mockExecute(kv, dmls[i:end]) for _, dml := range dmls[i:end] { - j := newDMLJob(dml.op, sourceTable, targetTable, dml, ec) - if j.dml.op == update && j.dml.updateIdentify() { - delDML, insertDML := updateToDelAndInsert(j.dml) + j := newDMLJob(dml, ec) + // if update job update its identify keys, turn it into delete + insert + if j.dml.IsIdentityUpdated() { + delDML, insertDML := j.dml.SplitUpdate() delJob := j.clone() - delJob.tp = del delJob.dml = delDML insertJob := j.clone() - insertJob.tp = insert insertJob.dml = insertDML compactor.compactJob(delJob) @@ -173,12 +155,12 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } for _, j := range compactor.buffer { if j != nil { - compactKV = mockExecute(compactKV, []*DML{j.dml}) + compactKV = mockExecute(compactKV, []*sqlmodel.RowChange{j.dml}) compactNumber++ c.Logf("after compact, dml: %s", j.dml.String()) } } - c.Logf("before compcat: %d, after compact: %d", noCompactNumber, compactNumber) + c.Logf("before compact: %d, after compact: %d", noCompactNumber, compactNumber) c.Assert(compactKV, DeepEquals, kv) compactor.keyMap = make(map[string]map[string]int) compactor.buffer = compactor.buffer[0:0] @@ -186,65 +168,101 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { - location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} p := parser.New() se := mock.NewContext() - targetTableID := "`test`.`tb`" - sourceTable := &filter.Table{Schema: "test", Name: "tb1"} - targetTable := &filter.Table{Schema: "test", Name: "tb"} + sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb"} schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) testCases := []struct { - input []*DML - output []*DML + input []*job + output []*job }{ // nolint:dupl { - input: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + input: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), }, - output: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + output: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), }, }, // nolint:dupl { - input: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, false, targetTableID, sourceTable, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), + input: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti, nil, nil), + ec, + ), }, - output: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "c"}, nil, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), + output: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "c"}, ti, nil, nil), + ecWithSafeMode, + ), }, }, } @@ -269,8 +287,7 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { outCh := compactorWrap(inCh, syncer) for _, tc := range testCases { - for _, dml := range tc.input { - j := newDMLJob(dml.op, sourceTable, targetTable, dml, ec) + for _, j := range tc.input { inCh <- j } inCh <- newFlushJob(syncer.cfg.WorkerCount, 1) @@ -281,7 +298,7 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { for i := 0; i <= len(tc.output); i++ { j := <-outCh if i < len(tc.output) { - c.Assert(j.dml, DeepEquals, tc.output[i]) + c.Assert(j.String(), Equals, tc.output[i].String()) } else { c.Assert(j.tp, Equals, flush) } diff --git a/dm/syncer/ddl_test.go b/dm/syncer/ddl_test.go index 40342bc3b51..3b32ddc0f21 100644 --- a/dm/syncer/ddl_test.go +++ b/dm/syncer/ddl_test.go @@ -97,11 +97,11 @@ func (s *testDDLSuite) TestCommentQuote(c *C) { expectedSQL := "ALTER TABLE `schemadb`.`ep_edu_course_message_auto_reply` MODIFY COLUMN `answer` JSON COMMENT '回复的内容-格式为list,有两个字段:\"answerType\"://''发送客服消息类型:1-文本消息,2-图片,3-图文链接''; answer:回复内容'" tctx := tcontext.Background().WithLogger(log.With(zap.String("test", "TestCommentQuote"))) - ec := &eventContext{ + testEC := &eventContext{ tctx: tctx, } qec := &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "schemadb", originSQL: sql, p: parser.New(), @@ -228,14 +228,14 @@ func (s *testDDLSuite) TestResolveDDLSQL(c *C) { }) c.Assert(err, IsNil) - ec := &eventContext{ + testEC := &eventContext{ tctx: tctx, } statusVars := []byte{4, 0, 0, 0, 0, 46, 0} syncer.idAndCollationMap = map[int]string{46: "utf8mb4_bin"} for i, sql := range sqls { qec := &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "test", originSQL: sql, appliedDDLs: make([]string, 0), @@ -443,7 +443,7 @@ func (s *testDDLSuite) TestResolveOnlineDDL(c *C) { tctx := tcontext.Background().WithLogger(log.With(zap.String("test", "TestResolveOnlineDDL"))) p := parser.New() - ec := &eventContext{tctx: tctx} + testEC := &eventContext{tctx: tctx} cluster, err := conn.NewCluster() c.Assert(err, IsNil) c.Assert(cluster.Start(), IsNil) @@ -463,7 +463,7 @@ func (s *testDDLSuite) TestResolveOnlineDDL(c *C) { c.Assert(plugin.Clear(tctx), IsNil) c.Assert(syncer.genRouter(), IsNil) qec = &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "test", appliedDDLs: make([]string, 0), p: p, diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index e5f5e8acc8f..7bf8c16a3c1 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -15,67 +15,32 @@ package syncer import ( "encoding/binary" - "fmt" - "strconv" - "strings" - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/tablecodec" "github.com/shopspring/decimal" "go.uber.org/zap" + cdcmodel "github.com/pingcap/tiflow/cdc/model" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) -// this type is used to generate DML SQL, opType is used to mark type in binlog. -type dmlOpType int64 - -const ( - insertDML = dmlOpType(insert) - updateDML = dmlOpType(update) - deleteDML = dmlOpType(del) - insertOnDuplicateDML dmlOpType = iota + 1 - replaceDML -) - -func (op dmlOpType) String() (str string) { - switch op { - case insertDML: - return "insert" - case updateDML: - return "update" - case deleteDML: - return "delete" - case insertOnDuplicateDML: - return "insert on duplicate update" - case replaceDML: - return "replace" - } - return -} - -// genDMLParam stores pruned columns, data as well as the original columns, data, index. +// genDMLParam stores original data and table structure. type genDMLParam struct { - targetTableID string // as a key in map like `schema`.`table` - sourceTable *filter.Table // origin table - safeMode bool // only used in update - data [][]interface{} // pruned data - originalData [][]interface{} // all data - columns []*model.ColumnInfo // pruned columns - sourceTableInfo *model.TableInfo // all table info - extendData [][]interface{} // all data include extend data + sourceTable *filter.Table // origin table + targetTable *filter.Table + safeMode bool // only used in update + originalData [][]interface{} // all data + sourceTableInfo *model.TableInfo // all table info + extendData [][]interface{} // all data include extend data } // extractValueFromData adjust the values obtained from go-mysql so that @@ -117,15 +82,13 @@ func extractValueFromData(data []interface{}, columns []*model.ColumnInfo, sourc return value } -func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID - dataSeq = param.data + tableID = utils.GenTableID(param.targetTable) originalDataSeq = param.originalData - columns = param.columns ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(dataSeq)) + dmls = make([]*sqlmodel.RowChange, 0, len(originalDataSeq)) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -133,23 +96,18 @@ func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLPar if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { originalDataSeq = extendData } RowLoop: - for dataIdx, data := range dataSeq { - if len(data) != len(columns) { - return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(columns), len(data)) + for dataIdx, data := range originalDataSeq { + if len(data) != len(ti.Columns) { + return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(ti.Columns), len(data)) } - value := extractValueFromData(data, columns, ti) - originalValue := value - if len(columns) != len(ti.Columns) { - originalValue = extractValueFromData(originalDataSeq[dataIdx], ti.Columns, ti) - } + originalValue := extractValueFromData(originalDataSeq[dataIdx], ti.Columns, ti) for _, expr := range filterExprs { skip, err := SkipDMLByExpression(s.sessCtx, originalValue, expr, ti.Columns) @@ -162,11 +120,17 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) - } - - dmls = append(dmls, newDML(insert, param.safeMode, tableID, param.sourceTable, nil, value, nil, originalValue, columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + nil, + originalValue, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil @@ -177,15 +141,13 @@ func (s *Syncer) genAndFilterUpdateDMLs( param *genDMLParam, oldValueFilters []expression.Expression, newValueFilters []expression.Expression, -) ([]*DML, error) { +) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID - data = param.data + tableID = utils.GenTableID(param.targetTable) originalData = param.originalData - columns = param.columns ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(data)/2) + dmls = make([]*sqlmodel.RowChange, 0, len(originalData)/2) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -193,38 +155,26 @@ func (s *Syncer) genAndFilterUpdateDMLs( if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { originalData = extendData } RowLoop: - for i := 0; i < len(data); i += 2 { - oldData := data[i] - changedData := data[i+1] + for i := 0; i < len(originalData); i += 2 { oriOldData := originalData[i] oriChangedData := originalData[i+1] - if len(oldData) != len(changedData) { - return nil, terror.ErrSyncerUnitDMLOldNewValueMismatch.Generate(len(oldData), len(changedData)) + if len(oriOldData) != len(oriChangedData) { + return nil, terror.ErrSyncerUnitDMLOldNewValueMismatch.Generate(len(oriOldData), len(oriChangedData)) } - if len(oldData) != len(columns) { - return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(columns), len(oldData)) + if len(oriOldData) != len(ti.Columns) { + return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(ti.Columns), len(oriOldData)) } - oldValues := extractValueFromData(oldData, columns, ti) - changedValues := extractValueFromData(changedData, columns, ti) - - var oriOldValues, oriChangedValues []interface{} - if len(columns) == len(ti.Columns) { - oriOldValues = oldValues - oriChangedValues = changedValues - } else { - oriOldValues = extractValueFromData(oriOldData, ti.Columns, ti) - oriChangedValues = extractValueFromData(oriChangedData, ti.Columns, ti) - } + oriOldValues := extractValueFromData(oriOldData, ti.Columns, ti) + oriChangedValues := extractValueFromData(oriChangedData, ti.Columns, ti) for j := range oldValueFilters { // AND logic @@ -244,23 +194,29 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, oriOldValues) - } - - dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + oriOldValues, + oriChangedValues, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil } -func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID + tableID = utils.GenTableID(param.targetTable) dataSeq = param.originalData ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(dataSeq)) + dmls = make([]*sqlmodel.RowChange, 0, len(dataSeq)) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -268,7 +224,6 @@ func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLPar if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { dataSeq = extendData @@ -293,11 +248,17 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) - } - - dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + value, + nil, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil @@ -332,115 +293,6 @@ func castUnsigned(data interface{}, ft *types.FieldType) interface{} { return data } -func columnValue(value interface{}, ft *types.FieldType) string { - castValue := castUnsigned(value, ft) - - var data string - switch v := castValue.(type) { - case nil: - data = "null" - case bool: - if v { - data = "1" - } else { - data = "0" - } - case int: - data = strconv.FormatInt(int64(v), 10) - case int8: - data = strconv.FormatInt(int64(v), 10) - case int16: - data = strconv.FormatInt(int64(v), 10) - case int32: - data = strconv.FormatInt(int64(v), 10) - case int64: - data = strconv.FormatInt(v, 10) - case uint8: - data = strconv.FormatUint(uint64(v), 10) - case uint16: - data = strconv.FormatUint(uint64(v), 10) - case uint32: - data = strconv.FormatUint(uint64(v), 10) - case uint64: - data = strconv.FormatUint(v, 10) - case float32: - data = strconv.FormatFloat(float64(v), 'f', -1, 32) - case float64: - data = strconv.FormatFloat(v, 'f', -1, 64) - case string: - data = v - case []byte: - data = string(v) - default: - data = fmt.Sprintf("%v", v) - } - - return data -} - -func findFitIndex(ti *model.TableInfo) *model.IndexInfo { - for _, idx := range ti.Indices { - if idx.Primary { - return idx - } - } - - if pk := ti.GetPkColInfo(); pk != nil { - return &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: pk.Name, - Offset: pk.Offset, - Length: types.UnspecifiedLength, - }}, - } - } - - // second find not null unique key - fn := func(i int) bool { - return !mysql.HasNotNullFlag(ti.Columns[i].Flag) - } - - return getSpecifiedIndexColumn(ti, fn) -} - -func getSpecifiedIndexColumn(ti *model.TableInfo, fn func(i int) bool) *model.IndexInfo { - for _, indexCols := range ti.Indices { - if !indexCols.Unique { - continue - } - - findFitIndex := true - for _, col := range indexCols.Columns { - if fn(col.Offset) { - findFitIndex = false - break - } - } - - if findFitIndex { - return indexCols - } - } - - return nil -} - -func getColumnData(columns []*model.ColumnInfo, indexColumns *model.IndexInfo, data []interface{}) ([]*model.ColumnInfo, []interface{}) { - cols := make([]*model.ColumnInfo, 0, len(indexColumns.Columns)) - values := make([]interface{}, 0, len(indexColumns.Columns)) - for _, column := range indexColumns.Columns { - cols = append(cols, columns[column.Offset]) - values = append(values, data[column.Offset]) - } - - return cols, values -} - func (s *Syncer) mappingDML(table *filter.Table, ti *model.TableInfo, data [][]interface{}) ([][]interface{}, error) { if s.columnMapping == nil { return data, nil @@ -464,49 +316,6 @@ func (s *Syncer) mappingDML(table *filter.Table, ti *model.TableInfo, data [][]i return rows, nil } -// pruneGeneratedColumnDML filters columns list, data and index removing all -// generated column. because generated column is not support setting value -// directly in DML, we must remove generated column from DML, including column -// list and data list including generated columns. -func pruneGeneratedColumnDML(ti *model.TableInfo, data [][]interface{}) ([]*model.ColumnInfo, [][]interface{}, error) { - // search for generated columns. if none found, return everything as-is. - firstGeneratedColumnIndex := -1 - for i, c := range ti.Columns { - if c.IsGenerated() { - firstGeneratedColumnIndex = i - break - } - } - if firstGeneratedColumnIndex < 0 { - return ti.Columns, data, nil - } - - // remove generated columns from the list of columns - cols := make([]*model.ColumnInfo, 0, len(ti.Columns)) - cols = append(cols, ti.Columns[:firstGeneratedColumnIndex]...) - for _, c := range ti.Columns[(firstGeneratedColumnIndex + 1):] { - if !c.IsGenerated() { - cols = append(cols, c) - } - } - - // remove generated columns from the list of data. - rows := make([][]interface{}, 0, len(data)) - for _, row := range data { - if len(row) != len(ti.Columns) { - return nil, nil, terror.ErrSyncerUnitDMLPruneColumnMismatch.Generate(len(ti.Columns), len(data)) - } - value := make([]interface{}, 0, len(cols)) - for i := range row { - if !ti.Columns[i].IsGenerated() { - value = append(value, row[i]) - } - } - rows = append(rows, value) - } - return cols, rows, nil -} - // checkLogColumns returns error when not all rows in skipped is empty, which means the binlog doesn't contain all // columns. // TODO: don't return error when all skipped columns is non-PK. @@ -519,518 +328,20 @@ func checkLogColumns(skipped [][]int) error { return nil } -// DML stores param for DML. -type DML struct { - targetTableID string - sourceTable *filter.Table - op opType - oldValues []interface{} // only for update SQL - values []interface{} - columns []*model.ColumnInfo - sourceTableInfo *model.TableInfo - originOldValues []interface{} // only for update SQL - originValues []interface{} // use to gen key and `WHERE` - safeMode bool - key string // use to detect causality - pickedDownstreamIndexInfo *model.IndexInfo // pick an index from downstream which comes from pk/uk not null/uk value not null and is only used in genWhere - downstreamTableInfo *schema.DownstreamTableInfo // downstream table info -} - -// newDML creates DML. -func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo, pickedDownstreamIndexInfo *model.IndexInfo, downstreamTableInfo *schema.DownstreamTableInfo) *DML { - return &DML{ - op: op, - safeMode: safeMode, - targetTableID: targetTableID, - sourceTable: sourceTable, - oldValues: oldValues, - values: values, - columns: columns, - sourceTableInfo: sourceTableInfo, - originOldValues: originOldValues, - originValues: originValues, - pickedDownstreamIndexInfo: pickedDownstreamIndexInfo, - downstreamTableInfo: downstreamTableInfo, - } -} - -// String returns the DML's string. -func (dml *DML) String() string { - return fmt.Sprintf("[safemode: %t, targetTableID: %s, op: %s, columns: %v, oldValues: %v, values: %v]", dml.safeMode, dml.targetTableID, dml.op.String(), dml.columnNames(), dml.originOldValues, dml.originValues) -} - -// updateToDelAndInsert turns updateDML to delDML and insertDML. -func updateToDelAndInsert(updateDML *DML) (*DML, *DML) { - delDML := &DML{} - *delDML = *updateDML - delDML.op = del - // use oldValues of update as values of delete and reset oldValues - delDML.values = updateDML.oldValues - delDML.originValues = updateDML.originOldValues - delDML.oldValues = nil - delDML.originOldValues = nil - - insertDML := &DML{} - *insertDML = *updateDML - insertDML.op = insert - // reset oldValues - insertDML.oldValues = nil - insertDML.originOldValues = nil - - return delDML, insertDML -} - -// identifyColumns gets columns of unique not null index. -// This is used for compact. -func (dml *DML) identifyColumns() []string { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - columns := make([]string, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - columns = append(columns, column.Name.O) - } - return columns - } - return nil -} - -// identifyValues gets values of unique not null index. -// This is used for compact. -func (dml *DML) identifyValues() []interface{} { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - values = append(values, dml.values[column.Offset]) - } - return values - } - return nil -} - -// oldIdentifyValues gets old values of unique not null index. -// only for update SQL. -func (dml *DML) oldIdentifyValues() []interface{} { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - values = append(values, dml.oldValues[column.Offset]) - } - return values - } - return nil -} - -// identifyKey use identifyValues to gen key. -// This is used for compact. -// PK or (UK + NOT NULL). -func (dml *DML) identifyKey() string { - return genKey(dml.identifyValues()) -} - -// updateIdentify check whether a update sql update its identify values. -func (dml *DML) updateIdentify() bool { - if len(dml.oldValues) == 0 { - return false - } - - values := dml.identifyValues() - oldValues := dml.oldIdentifyValues() - - if len(values) != len(oldValues) { - return true - } - - for i := 0; i < len(values); i++ { - if values[i] != oldValues[i] { - return true - } - } - - return false -} - -// identifyKeys gens keys by unique not null value. -// This is used for causality. -// PK or (UK + NOT NULL) or (UK + NULL + NOT NULL VALUE). -func (dml *DML) identifyKeys(ctx sessionctx.Context) []string { - var keys []string - // for UPDATE statement - if dml.originOldValues != nil { - keys = append(keys, genMultipleKeys(ctx, dml.downstreamTableInfo, dml.sourceTableInfo, dml.originOldValues, dml.targetTableID)...) - } - - if dml.originValues != nil { - keys = append(keys, genMultipleKeys(ctx, dml.downstreamTableInfo, dml.sourceTableInfo, dml.originValues, dml.targetTableID)...) - } - return keys -} - -// columnNames return column names of DML. -func (dml *DML) columnNames() []string { - columnNames := make([]string, 0, len(dml.columns)) - for _, column := range dml.columns { - columnNames = append(columnNames, column.Name.O) - } - return columnNames -} - -// whereColumnsAndValues gets columns and values of unique column with not null value. -// This is used to generete where condition. -func (dml *DML) whereColumnsAndValues() ([]string, []interface{}) { - columns, values := dml.sourceTableInfo.Columns, dml.originValues - - if dml.op == update { - values = dml.originOldValues - } - - if dml.pickedDownstreamIndexInfo != nil { - columns, values = getColumnData(dml.sourceTableInfo.Columns, dml.pickedDownstreamIndexInfo, values) - } - - columnNames := make([]string, 0, len(columns)) - for _, column := range columns { - columnNames = append(columnNames, column.Name.O) - } - - failpoint.Inject("DownstreamTrackerWhereCheck", func() { - if dml.op == update { - log.L().Info("UpdateWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) - } else if dml.op == del { - log.L().Info("DeleteWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) - } - }) - - return columnNames, values -} - -// genKey gens key by values e.g. "a.1.b". -// This is used for compact. -func genKey(values []interface{}) string { - builder := new(strings.Builder) - for i, v := range values { - if i != 0 { - builder.WriteString(".") - } - fmt.Fprintf(builder, "%v", v) - } - - return builder.String() -} - -// genKeyList format keys. -func genKeyList(table string, columns []*model.ColumnInfo, dataSeq []interface{}) string { - var buf strings.Builder - for i, data := range dataSeq { - if data == nil { - log.L().Debug("ignore null value", zap.String("column", columns[i].Name.O), zap.String("table", table)) - continue // ignore `null` value. - } - // one column key looks like:`column_val.column_name.` - buf.WriteString(columnValue(data, &columns[i].FieldType)) - buf.WriteString(".") - buf.WriteString(columns[i].Name.O) - buf.WriteString(".") - } - if buf.Len() == 0 { - log.L().Debug("all value are nil, no key generated", zap.String("table", table)) - return "" // all values are `null`. - } - buf.WriteString(table) - return buf.String() -} - -// truncateIndexValues truncate prefix index from data. -func truncateIndexValues(ctx sessionctx.Context, ti *model.TableInfo, indexColumns *model.IndexInfo, tiColumns []*model.ColumnInfo, data []interface{}) []interface{} { - values := make([]interface{}, 0, len(indexColumns.Columns)) - datums, err := utils.AdjustBinaryProtocolForDatum(ctx, data, tiColumns) - if err != nil { - log.L().Warn("adjust binary protocol for datum error", zap.Error(err)) - return data - } - tablecodec.TruncateIndexValues(ti, indexColumns, datums) - for _, datum := range datums { - values = append(values, datum.GetValue()) - } - return values -} - -// genMultipleKeys gens keys with UNIQUE NOT NULL value. -// if not UNIQUE NOT NULL value, use table name instead. -func genMultipleKeys(ctx sessionctx.Context, downstreamTableInfo *schema.DownstreamTableInfo, ti *model.TableInfo, value []interface{}, table string) []string { - multipleKeys := make([]string, 0, len(downstreamTableInfo.AvailableUKIndexList)) - - for _, indexCols := range downstreamTableInfo.AvailableUKIndexList { - cols, vals := getColumnData(ti.Columns, indexCols, value) - // handle prefix index - truncVals := truncateIndexValues(ctx, ti, indexCols, cols, vals) - key := genKeyList(table, cols, truncVals) - if len(key) > 0 { // ignore `null` value. - multipleKeys = append(multipleKeys, key) - } else { - log.L().Debug("ignore empty key", zap.String("table", table)) - } - } - - if len(multipleKeys) == 0 { - // use table name as key if no key generated (no PK/UK), - // no concurrence for rows in the same table. - log.L().Debug("use table name as the key", zap.String("table", table)) - multipleKeys = append(multipleKeys, table) - } - - return multipleKeys -} - -// genWhere generates where condition. -func (dml *DML) genWhere(buf *strings.Builder) []interface{} { - whereColumns, whereValues := dml.whereColumnsAndValues() - - for i, col := range whereColumns { - if i != 0 { - buf.WriteString(" AND ") - } - buf.WriteString(dbutil.ColumnName(col)) - if whereValues[i] == nil { - buf.WriteString(" IS ?") - } else { - buf.WriteString(" = ?") - } - } - return whereValues -} - -// genSQL generates SQL for a DML. -func (dml *DML) genSQL() (sql []string, arg [][]interface{}) { - switch dml.op { - case insert: - return dml.genInsertSQL() - case del: - return dml.genDeleteSQL() - case update: - return dml.genUpdateSQL() - } - return -} - -// genUpdateSQL generates a `UPDATE` SQL with `WHERE`. -func (dml *DML) genUpdateSQL() ([]string, [][]interface{}) { - if dml.safeMode { - sqls, args := dml.genDeleteSQL() - insertSQLs, insertArgs := dml.genInsertSQL() - sqls = append(sqls, insertSQLs...) - args = append(args, insertArgs...) - return sqls, args - } - var buf strings.Builder - buf.Grow(2048) - buf.WriteString("UPDATE ") - buf.WriteString(dml.targetTableID) - buf.WriteString(" SET ") - - for i, column := range dml.columns { - if i == len(dml.columns)-1 { - fmt.Fprintf(&buf, "%s = ?", dbutil.ColumnName(column.Name.O)) - } else { - fmt.Fprintf(&buf, "%s = ?, ", dbutil.ColumnName(column.Name.O)) - } - } - - buf.WriteString(" WHERE ") - whereArgs := dml.genWhere(&buf) - buf.WriteString(" LIMIT 1") - - args := dml.values - args = append(args, whereArgs...) - return []string{buf.String()}, [][]interface{}{args} -} - -// genDeleteSQL generates a `DELETE FROM` SQL with `WHERE`. -func (dml *DML) genDeleteSQL() ([]string, [][]interface{}) { - var buf strings.Builder - buf.Grow(1024) - buf.WriteString("DELETE FROM ") - buf.WriteString(dml.targetTableID) - buf.WriteString(" WHERE ") - whereArgs := dml.genWhere(&buf) - buf.WriteString(" LIMIT 1") - - return []string{buf.String()}, [][]interface{}{whereArgs} -} - -// genInsertSQL generates a `INSERT`. -// if in safemode, generates a `REPLACE` statement. -func (dml *DML) genInsertSQL() ([]string, [][]interface{}) { - var buf strings.Builder - buf.Grow(1024) - if dml.safeMode { - buf.WriteString("REPLACE INTO ") - } else { - buf.WriteString("INSERT INTO ") - } - buf.WriteString(dml.targetTableID) - buf.WriteString(" (") - for i, column := range dml.columns { - buf.WriteString(dbutil.ColumnName(column.Name.O)) - if i != len(dml.columns)-1 { - buf.WriteByte(',') - } else { - buf.WriteByte(')') - } - } - buf.WriteString(" VALUES (") - - // placeholders - for i := range dml.columns { - if i != len(dml.columns)-1 { - buf.WriteString("?,") - } else { - buf.WriteString("?)") - } - } - return []string{buf.String()}, [][]interface{}{dml.values} -} - -// valuesHolder gens values holder like (?,?,?). -func valuesHolder(n int) string { - builder := new(strings.Builder) - builder.WriteByte('(') - for i := 0; i < n; i++ { - if i > 0 { - builder.WriteString(",") - } - builder.WriteString("?") - } - builder.WriteByte(')') - return builder.String() -} - -// genInsertSQLMultipleRows generates a `INSERT` with multiple rows like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2)' -// if replace, generates a `REPLACE' with multiple rows like 'REPLACE INTO tb(a,b) VALUES (1,1),(2,2)' -// if onDuplicate, generates a `INSERT ON DUPLICATE KEY UPDATE` statement like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2) ON DUPLICATE KEY UPDATE a=VALUES(a),b=VALUES(b)'. -func genInsertSQLMultipleRows(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { - if len(dmls) == 0 { - return nil, nil - } - - var buf strings.Builder - buf.Grow(1024) - if op == replaceDML { - buf.WriteString("REPLACE INTO") - } else { - buf.WriteString("INSERT INTO") - } - buf.WriteString(" " + dmls[0].targetTableID + " (") - for i, column := range dmls[0].columns { - buf.WriteString(dbutil.ColumnName(column.Name.O)) - if i != len(dmls[0].columns)-1 { - buf.WriteByte(',') - } else { - buf.WriteByte(')') - } - } - buf.WriteString(" VALUES ") - - holder := valuesHolder(len(dmls[0].columns)) - for i := range dmls { - if i > 0 { - buf.WriteString(",") - } - buf.WriteString(holder) - } - - if op == insertOnDuplicateDML { - buf.WriteString(" ON DUPLICATE KEY UPDATE ") - for i, column := range dmls[0].columns { - col := dbutil.ColumnName(column.Name.O) - buf.WriteString(col + "=VALUES(" + col + ")") - if i != len(dmls[0].columns)-1 { - buf.WriteByte(',') - } - } - } - - args := make([]interface{}, 0, len(dmls)*len(dmls[0].columns)) - for _, dml := range dmls { - args = append(args, dml.values...) - } - return []string{buf.String()}, [][]interface{}{args} -} - -// genDeleteSQLMultipleRows generates delete statement with multiple rows like 'DELETE FROM tb WHERE (a,b) IN (1,1),(2,2)'. -func genDeleteSQLMultipleRows(dmls []*DML) ([]string, [][]interface{}) { - if len(dmls) == 0 { - return nil, nil - } - - var buf strings.Builder - buf.Grow(1024) - buf.WriteString("DELETE FROM ") - buf.WriteString(dmls[0].targetTableID) - buf.WriteString(" WHERE (") - - whereColumns, _ := dmls[0].whereColumnsAndValues() - for i, column := range whereColumns { - if i != len(whereColumns)-1 { - buf.WriteString(dbutil.ColumnName(column) + ",") - } else { - buf.WriteString(dbutil.ColumnName(column) + ")") - } - } - buf.WriteString(" IN (") - - holder := valuesHolder(len(whereColumns)) - args := make([]interface{}, 0, len(dmls)*len(dmls[0].columns)) - for i, dml := range dmls { - if i > 0 { - buf.WriteString(",") - } - buf.WriteString(holder) - _, whereValues := dml.whereColumnsAndValues() - // whereValues will have same length because we have checked it in genDMLsWithSameCols. - args = append(args, whereValues...) - } - buf.WriteString(")") - return []string{buf.String()}, [][]interface{}{args} -} - // genSQLMultipleRows generates multiple rows SQL with different dmlOpType. -func genSQLMultipleRows(op dmlOpType, dmls []*DML) (queries []string, args [][]interface{}) { +func genSQLMultipleRows(op sqlmodel.DMLType, dmls []*sqlmodel.RowChange) (queries string, args []interface{}) { if len(dmls) > 1 { - log.L().Debug("generate DMLs with multiple rows", zap.Stringer("op", op), zap.Stringer("original op", dmls[0].op), zap.Int("rows", len(dmls))) + log.L().Debug("generate DMLs with multiple rows", zap.Stringer("op", op), zap.Stringer("original op", dmls[0].Type()), zap.Int("rows", len(dmls))) } switch op { - case insertDML, replaceDML, insertOnDuplicateDML: - return genInsertSQLMultipleRows(op, dmls) - case deleteDML: - return genDeleteSQLMultipleRows(dmls) + case sqlmodel.DMLInsert, sqlmodel.DMLReplace, sqlmodel.DMLInsertOnDuplicateUpdate: + return sqlmodel.GenInsertSQL(op, dmls...) + case sqlmodel.DMLDelete: + return sqlmodel.GenDeleteSQL(dmls...) } return } -// sameColumns check whether two DMLs have same columns. -func sameColumns(lhs *DML, rhs *DML) bool { - var lhsCols, rhsCols []string - if lhs.op == del { - lhsCols, _ = lhs.whereColumnsAndValues() - rhsCols, _ = rhs.whereColumnsAndValues() - } else { - // if source table is same, columns will be same. - if lhs.sourceTable.Schema == rhs.sourceTable.Schema && lhs.sourceTable.Name == rhs.sourceTable.Name { - return true - } - lhsCols = lhs.columnNames() - rhsCols = rhs.columnNames() - } - if len(lhsCols) != len(rhsCols) { - return false - } - for i := 0; i < len(lhsCols); i++ { - if lhsCols[i] != rhsCols[i] { - return false - } - } - return true -} - // genDMLsWithSameCols group and gen dmls by same columns. // in optimistic shard mode, different upstream tables may have different columns. // e.g. @@ -1038,23 +349,23 @@ func sameColumns(lhs *DML, rhs *DML) bool { // insert into tb(a,b,d) values(2,2,2) // we can only combine DMLs with same column names. // all dmls should have same dmlOpType and same tableName. -func genDMLsWithSameCols(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { +func genDMLsWithSameCols(op sqlmodel.DMLType, dmls []*sqlmodel.RowChange) ([]string, [][]interface{}) { queries := make([]string, 0, len(dmls)) args := make([][]interface{}, 0, len(dmls)) - var lastDML *DML - var query []string - var arg [][]interface{} - groupDMLs := make([]*DML, 0, len(dmls)) + var lastDML *sqlmodel.RowChange + var query string + var arg []interface{} + groupDMLs := make([]*sqlmodel.RowChange, 0, len(dmls)) // group dmls by same columns for i, dml := range dmls { if i == 0 { lastDML = dml } - if !sameColumns(lastDML, dml) { + if !sqlmodel.SameTypeTargetAndColumns(lastDML, dml) { query, arg = genSQLMultipleRows(op, groupDMLs) - queries = append(queries, query...) - args = append(args, arg...) + queries = append(queries, query) + args = append(args, arg) groupDMLs = groupDMLs[0:0] lastDML = dml @@ -1063,44 +374,53 @@ func genDMLsWithSameCols(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) } if len(groupDMLs) > 0 { query, arg = genSQLMultipleRows(op, groupDMLs) - queries = append(queries, query...) - args = append(args, arg...) + queries = append(queries, query) + args = append(args, arg) } return queries, args } // genDMLsWithSameTable groups and generates dmls with same table. // all the dmls should have same dmlOpType. -func genDMLsWithSameTable(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) +func genDMLsWithSameTable(op sqlmodel.DMLType, jobs []*job) ([]string, [][]interface{}) { + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) var lastTable string - groupDMLs := make([]*DML, 0, len(dmls)) + groupDMLs := make([]*sqlmodel.RowChange, 0, len(jobs)) // for updateDML, generate SQLs one by one - if op == updateDML { - for _, dml := range dmls { - query, arg := dml.genUpdateSQL() - queries = append(queries, query...) - args = append(args, arg...) + if op == sqlmodel.DMLUpdate { + for _, j := range jobs { + if j.safeMode { + query, arg := j.dml.GenSQL(sqlmodel.DMLDelete) + queries = append(queries, query) + args = append(args, arg) + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + queries = append(queries, query) + args = append(args, arg) + continue + } + query, arg := j.dml.GenSQL(op) + queries = append(queries, query) + args = append(args, arg) } return queries, args } // group dmls with same table - for i, dml := range dmls { + for i, j := range jobs { if i == 0 { - lastTable = dml.targetTableID + lastTable = j.dml.TargetTableID() } - if lastTable != dml.targetTableID { + if lastTable != j.dml.TargetTableID() { query, arg := genDMLsWithSameCols(op, groupDMLs) queries = append(queries, query...) args = append(args, arg...) groupDMLs = groupDMLs[0:0] - lastTable = dml.targetTableID + lastTable = j.dml.TargetTableID() } - groupDMLs = append(groupDMLs, dml) + groupDMLs = append(groupDMLs, j.dml) } if len(groupDMLs) > 0 { query, arg := genDMLsWithSameCols(op, groupDMLs) @@ -1112,21 +432,34 @@ func genDMLsWithSameTable(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) // genDMLsWithSameOp groups and generates dmls by dmlOpType. // TODO: implement a volcano iterator interface for genDMLsWithSameXXX. -func genDMLsWithSameOp(dmls []*DML) ([]string, [][]interface{}) { - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) - var lastOp dmlOpType - groupDMLs := make([]*DML, 0, len(dmls)) +func genDMLsWithSameOp(jobs []*job) ([]string, [][]interface{}) { + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + var lastOp sqlmodel.DMLType + jobsWithSameOp := make([]*job, 0, len(jobs)) // group dmls with same dmlOp - for i, dml := range dmls { - curOp := dmlOpType(dml.op) - if curOp == updateDML && !dml.updateIdentify() && !dml.safeMode { + for i, j := range jobs { + var curOp sqlmodel.DMLType + switch j.dml.Type() { + case sqlmodel.RowChangeUpdate: // if update statement didn't update identify values and not in safemode, regard it as insert on duplicate. - curOp = insertOnDuplicateDML - } else if curOp == insertDML && dml.safeMode { + if !j.dml.IsIdentityUpdated() && !j.safeMode { + curOp = sqlmodel.DMLInsertOnDuplicateUpdate + break + } + + curOp = sqlmodel.DMLUpdate + case sqlmodel.RowChangeInsert: // if insert with safemode, regard it as replace - curOp = replaceDML + if j.safeMode { + curOp = sqlmodel.DMLReplace + break + } + + curOp = sqlmodel.DMLInsert + case sqlmodel.RowChangeDelete: + curOp = sqlmodel.DMLDelete } if i == 0 { @@ -1135,17 +468,17 @@ func genDMLsWithSameOp(dmls []*DML) ([]string, [][]interface{}) { // now there are 5 situations: [insert, replace(insert with safemode), insert on duplicate(update without identify keys), update(update identify keys/update with safemode), delete] if lastOp != curOp { - query, arg := genDMLsWithSameTable(lastOp, groupDMLs) + query, arg := genDMLsWithSameTable(lastOp, jobsWithSameOp) queries = append(queries, query...) args = append(args, arg...) - groupDMLs = groupDMLs[0:0] + jobsWithSameOp = jobsWithSameOp[0:0] lastOp = curOp } - groupDMLs = append(groupDMLs, dml) + jobsWithSameOp = append(jobsWithSameOp, j) } - if len(groupDMLs) > 0 { - query, arg := genDMLsWithSameTable(lastOp, groupDMLs) + if len(jobsWithSameOp) > 0 { + query, arg := genDMLsWithSameTable(lastOp, jobsWithSameOp) queries = append(queries, query...) args = append(args, arg...) } diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index 148c2538eff..b6efe30695f 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -15,14 +15,15 @@ package syncer import ( "math" - "strings" + "testing" . "github.com/pingcap/check" + "github.com/stretchr/testify/require" - "github.com/pingcap/tiflow/dm/pkg/schema" - "github.com/pingcap/tiflow/dm/pkg/utils" + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" - "github.com/pingcap/tidb-tools/pkg/filter" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -33,6 +34,14 @@ import ( "github.com/pingcap/tidb/util/mock" ) +var ( + location = binlog.Location{ + Position: binlog.MinPosition, + } + ec = &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} + ecWithSafeMode = &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location, safeMode: true} +) + func (s *testSyncerSuite) TestCastUnsigned(c *C) { // ref: https://dev.mysql.com/doc/refman/5.7/en/integer-types.html cases := []struct { @@ -70,420 +79,174 @@ func createTableInfo(p *parser.Parser, se sessionctx.Context, tableID int64, sql return tiddl.MockTableInfo(se, node.(*ast.CreateTableStmt), tableID) } -func (s *testSyncerSuite) TestFindFitIndex(c *C) { - p := parser.New() - se := mock.NewContext() - - ti, err := createTableInfo(p, se, 1, ` - create table t1( - a int, - b int, - c int, - d int not null, - primary key(a, b), - unique key(c), - unique key(d) - ); - `) - c.Assert(err, IsNil) - - columns := findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 2) - c.Assert(columns.Columns[0].Name.L, Equals, "a") - c.Assert(columns.Columns[1].Name.L, Equals, "b") - - ti, err = createTableInfo(p, se, 2, `create table t2(c int unique);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, IsNil) - - ti, err = createTableInfo(p, se, 3, `create table t3(d int not null unique);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "d") - - ti, err = createTableInfo(p, se, 4, `create table t4(e int not null, key(e));`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, IsNil) +func TestGenDMLWithSameOp(t *testing.T) { + targetTable1 := &cdcmodel.TableName{Schema: "db1", Table: "tb1"} + targetTable2 := &cdcmodel.TableName{Schema: "db2", Table: "tb2"} + sourceTable11 := &cdcmodel.TableName{Schema: "dba", Table: "tba"} + sourceTable12 := &cdcmodel.TableName{Schema: "dba", Table: "tbb"} + sourceTable21 := &cdcmodel.TableName{Schema: "dbb", Table: "tba"} + sourceTable22 := &cdcmodel.TableName{Schema: "dbb", Table: "tbb"} - ti, err = createTableInfo(p, se, 5, `create table t5(f datetime primary key);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "f") + tableInfo11 := mockTableInfo(t, "create table db.tb(id int primary key, col1 int unique not null, name varchar(24))") + tableInfo12 := mockTableInfo(t, "create table db.tb(id int primary key, col1 int unique not null, name varchar(24))") + tableInfo21 := mockTableInfo(t, "create table db.tb(id int primary key, col2 int unique not null, name varchar(24))") + tableInfo22 := mockTableInfo(t, "create table db.tb(id int primary key, col3 int unique not null, name varchar(24))") - ti, err = createTableInfo(p, se, 6, `create table t6(g int primary key);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "g") -} - -func (s *testSyncerSuite) TestGenMultipleKeys(c *C) { - p := parser.New() - se := mock.NewContext() - - testCases := []struct { - schema string - values []interface{} - keys []string - }{ - { - // test no keys - schema: `create table t1(a int)`, - values: []interface{}{10}, - keys: []string{"table"}, - }, - { - // one primary key - schema: `create table t2(a int primary key, b double)`, - values: []interface{}{60, 70.5}, - keys: []string{"60.a.table"}, - }, - { - // one unique key - schema: `create table t3(a int unique, b double)`, - values: []interface{}{60, 70.5}, - keys: []string{"60.a.table"}, - }, - { - // one ordinary key - schema: `create table t4(a int, b double, key(b))`, - values: []interface{}{60, 70.5}, - keys: []string{"table"}, - }, - { - // multiple keys - schema: `create table t5(a int, b text, c int, key(a), key(b(3)))`, - values: []interface{}{13, "abcdef", 15}, - keys: []string{"table"}, - }, - { - // multiple keys with primary key - schema: `create table t6(a int primary key, b varchar(16) unique)`, - values: []interface{}{16, "xyz"}, - keys: []string{"16.a.table", "xyz.b.table"}, - }, - { - // non-integer primary key - schema: `create table t65(a int unique, b varchar(16) primary key)`, - values: []interface{}{16, "xyz"}, - keys: []string{"xyz.b.table", "16.a.table"}, - }, - { - // primary key of multiple columns - schema: `create table t7(a int, b int, primary key(a, b))`, - values: []interface{}{59, 69}, - keys: []string{"59.a.69.b.table"}, - }, - { - // ordinary key of multiple columns - schema: `create table t75(a int, b int, c int, key(a, b), key(c, b))`, - values: []interface{}{48, 58, 68}, - keys: []string{"table"}, - }, - { - // so many keys - schema: ` - create table t8( - a int, b int, c int, - primary key(a, b), - unique key(b, c), - key(a, b, c), - unique key(c, a) - ) - `, - values: []interface{}{27, 37, 47}, - keys: []string{"27.a.37.b.table", "37.b.47.c.table", "47.c.27.a.table"}, - }, - { - // `null` for unique key - schema: ` - create table t8( - a int, b int default null, - primary key(a), - unique key(b) - ) - `, - values: []interface{}{17, nil}, - keys: []string{"17.a.table"}, - }, - } - sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) - for i, tc := range testCases { - schemaStr := tc.schema - assert := func(obtained interface{}, checker Checker, args ...interface{}) { - c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) - } - - ti, err := createTableInfo(p, se, int64(i+1), tc.schema) - assert(err, IsNil) - dti := schema.GetDownStreamTI(ti, ti) - assert(dti, NotNil) - keys := genMultipleKeys(sessCtx, dti, ti, tc.values, "table") - assert(keys, DeepEquals, tc.keys) - } -} - -func (s *testSyncerSuite) TestGenWhere(c *C) { - p := parser.New() - se := mock.NewContext() - schema1 := "create table test.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" - ti1, err := createTableInfo(p, se, 0, schema1) - c.Assert(err, IsNil) - schema2 := "create table test.tb(id int, col1 int, col2 int, name varchar(24))" - ti2, err := createTableInfo(p, se, 0, schema2) - c.Assert(err, IsNil) - ti1Index := &model.IndexInfo{ - Table: ti1.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti1.Columns[0].Name, - Offset: ti1.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - - testCases := []struct { - dml *DML - sql string - values []interface{} - }{ - { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1, ti1Index, nil), - "`id` = ?", - []interface{}{1}, - }, - { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1, ti1Index, nil), - "`id` = ?", - []interface{}{1}, - }, - { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2, nil, nil), - "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", - []interface{}{1, 2, 3, "haha"}, - }, - { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2, nil, nil), - "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", - []interface{}{1, 2, 3, "haha"}, - }, - } - - for _, tc := range testCases { - var buf strings.Builder - whereValues := tc.dml.genWhere(&buf) - c.Assert(buf.String(), Equals, tc.sql) - c.Assert(whereValues, DeepEquals, tc.values) - } -} - -func (s *testSyncerSuite) TestGenSQL(c *C) { - p := parser.New() - se := mock.NewContext() - schema := "create table test.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" - ti, err := createTableInfo(p, se, 0, schema) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - - testCases := []struct { - dml *DML - queries []string - args [][]interface{} - }{ - { - newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1, 2, 3, "haha"}}, - }, - { - newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1, 2, 3, "haha"}}, - }, - { - newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, - [][]interface{}{{1}}, - }, - { - newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, - [][]interface{}{{4, 5, 6, "hihi", 1}}, - }, - { - newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1}, {4, 5, 6, "hihi"}}, - }, - } - for _, tc := range testCases { - queries, args := tc.dml.genSQL() - c.Assert(queries, DeepEquals, tc.queries) - c.Assert(args, DeepEquals, tc.args) - } -} - -func (s *testSyncerSuite) TestValueHolder(c *C) { - holder := valuesHolder(0) - c.Assert(holder, Equals, "()") - holder = valuesHolder(10) - c.Assert(holder, Equals, "(?,?,?,?,?,?,?,?,?,?)") -} - -func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { - targetTableID1 := "`db1`.`tb1`" - targetTableID2 := "`db2`.`tb2`" - sourceTable11 := &filter.Table{Schema: "dba", Name: "tba"} - sourceTable12 := &filter.Table{Schema: "dba", Name: "tbb"} - sourceTable21 := &filter.Table{Schema: "dbb", Name: "tba"} - sourceTable22 := &filter.Table{Schema: "dbb", Name: "tbb"} - - p := parser.New() - se := mock.NewContext() - schema11 := "create table dba.tba(id int primary key, col1 int unique not null, name varchar(24))" - schema12 := "create table dba.tbb(id int primary key, col1 int unique not null, name varchar(24))" - schema21 := "create table dbb.tba(id int primary key, col2 int unique not null, name varchar(24))" - schema22 := "create table dbb.tbb(id int primary key, col3 int unique not null, name varchar(24))" - ti11, err := createTableInfo(p, se, 0, schema11) - c.Assert(err, IsNil) - ti11Index := &model.IndexInfo{ - Table: ti11.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti11.Columns[0].Name, - Offset: ti11.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi11 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti11Index, - } - ti12, err := createTableInfo(p, se, 0, schema12) - c.Assert(err, IsNil) - ti12Index := &model.IndexInfo{ - Table: ti12.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti12.Columns[0].Name, - Offset: ti12.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi12 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti12Index, - } - ti21, err := createTableInfo(p, se, 0, schema21) - c.Assert(err, IsNil) - ti21Index := &model.IndexInfo{ - Table: ti21.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti21.Columns[0].Name, - Offset: ti21.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi21 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti21Index, - } - ti22, err := createTableInfo(p, se, 0, schema22) - c.Assert(err, IsNil) - ti22Index := &model.IndexInfo{ - Table: ti22.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti22.Columns[0].Name, - Offset: ti22.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi22 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti22Index, - } - - dmls := []*DML{ + dmls := []*job{ // insert - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, nil, []interface{}{1, 1, "a"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, nil, []interface{}{2, 2, "b"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, nil, []interface{}{3, 3, "c"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update no index but safemode - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update uk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update pk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), // delete - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{4, 4, "aa"}, nil, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{5, 5, "bb"}, nil, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{6, 6, "cc"}, nil, tableInfo12, nil, nil), + ecWithSafeMode, + ), // target table 2 // insert - newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, nil, []interface{}{1, 1, "a"}, tableInfo21, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, nil, []interface{}{2, 2, "b"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, nil, []interface{}{3, 3, "c"}, tableInfo22, nil, nil), + ec, + ), + // update no index - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, tableInfo22, nil, nil), + ec, + ), + // update uk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, tableInfo22, nil, nil), + ec, + ), + // update pk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, tableInfo22, nil, nil), + ec, + ), + // delete - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{4, 4, "aa"}, nil, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{5, 5, "bb"}, nil, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{6, 6, "cc"}, nil, tableInfo22, nil, nil), + ec, + ), // table1 // detele - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{44, 44, "aaa"}, nil, tableInfo11, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{55, 55, "bbb"}, nil, tableInfo11, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{66, 66, "ccc"}, nil, tableInfo12, nil, nil), + ec, + ), } expectQueries := []string{ @@ -567,70 +330,8 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { } queries, args := genDMLsWithSameOp(dmls) - c.Assert(queries, DeepEquals, expectQueries) - c.Assert(args, DeepEquals, expectArgs) -} - -func (s *testSyncerSuite) TestTruncateIndexValues(c *C) { - p := parser.New() - se := mock.NewContext() - - testCases := []struct { - schema string - values []interface{} - preValues []interface{} - }{ - { - // test not prefix key - schema: `create table t1(a int, b varchar(20), unique key b(b))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{"1234"}, - }, - { - // test not string key - schema: `create table t1(a int, b text, unique key a(a))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{int64(10)}, - }, - { - // test keys - schema: `create table t1(a int, b text, unique key b(b(3)))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{"123"}, - }, - { - // test multi keys - schema: `create table t1(a int, b text, unique key c2(a, b(3)))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{int64(10), "123"}, - }, - { - // value is nil - schema: `create table t1(a int, b text, unique key c2(a, b(3)))`, - values: []interface{}{10, nil}, - preValues: []interface{}{int64(10), nil}, - }, - } - sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) - for i, tc := range testCases { - schemaStr := tc.schema - assert := func(obtained interface{}, checker Checker, args ...interface{}) { - c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) - } - ti, err := createTableInfo(p, se, int64(i+1), tc.schema) - assert(err, IsNil) - dti := schema.GetDownStreamTI(ti, ti) - assert(dti, NotNil) - assert(dti.AvailableUKIndexList, NotNil) - cols := make([]*model.ColumnInfo, 0, len(dti.AvailableUKIndexList[0].Columns)) - values := make([]interface{}, 0, len(dti.AvailableUKIndexList[0].Columns)) - for _, column := range dti.AvailableUKIndexList[0].Columns { - cols = append(cols, ti.Columns[column.Offset]) - values = append(values, tc.values[column.Offset]) - } - realPreValue := truncateIndexValues(sessCtx, ti, dti.AvailableUKIndexList[0], cols, values) - assert(realPreValue, DeepEquals, tc.preValues) - } + require.Equal(t, expectQueries, queries) + require.Equal(t, expectArgs, args) } func (s *testSyncerSuite) TestGBKExtractValueFromData(c *C) { diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index 78c4e48dd1f..016a418969a 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" tcontext "github.com/pingcap/tiflow/dm/pkg/context" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/tiflow/dm/syncer/metrics" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // DMLWorker is used to sync dml. @@ -46,10 +46,10 @@ type DMLWorker struct { // callback func // TODO: refine callback func - successFunc func(int, int, []*job) - fatalFunc func(*job, error) - lagFunc func(*job, int) - addCountFunc func(bool, string, opType, int64, *filter.Table) + successFunc func(int, int, []*job) + fatalFunc func(*job, error) + lagFunc func(*job, int) + updateJobMetricsFunc func(bool, string, *job) // channel inCh chan *job @@ -63,22 +63,22 @@ func dmlWorkerWrap(inCh chan *job, syncer *Syncer) chan *job { chanSize /= 2 } dmlWorker := &DMLWorker{ - batch: syncer.cfg.Batch, - workerCount: syncer.cfg.WorkerCount, - chanSize: chanSize, - multipleRows: syncer.cfg.MultipleRows, - 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), + batch: syncer.cfg.Batch, + workerCount: syncer.cfg.WorkerCount, + chanSize: chanSize, + multipleRows: syncer.cfg.MultipleRows, + 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, + updateJobMetricsFunc: syncer.updateJobMetrics, + tctx: syncer.tctx, + toDBConns: syncer.toDBConns, + inCh: inCh, + flushCh: make(chan *job), } go func() { @@ -117,25 +117,25 @@ func (w *DMLWorker) run() { metrics.QueueSizeGauge.WithLabelValues(w.task, "dml_worker_input", w.source).Set(float64(len(w.inCh))) switch j.tp { case flush: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) j.flushWg.Wait() - w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(true, adminQueueName, j) w.flushCh <- j case asyncFlush: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) w.flushCh <- j case conflict: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) j.flushWg.Wait() - w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(true, adminQueueName, j) default: - queueBucket := int(utils.GenHashKey(j.dml.key)) % w.workerCount - w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetTable) + queueBucket := int(utils.GenHashKey(j.dmlQueueKey)) % w.workerCount + w.updateJobMetricsFunc(false, queueBucketMapping[queueBucket], j) startTime := time.Now() - w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.dml.key)) + w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.dmlQueueKey)) jobChs[queueBucket] <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) } @@ -201,7 +201,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { args [][]interface{} db = w.toDBConns[queueID] err error - dmls = make([]*DML, 0, len(jobs)) + dmls = make([]*sqlmodel.RowChange, 0, len(jobs)) ) defer func() { @@ -238,10 +238,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { } }) - for _, j := range jobs { - dmls = append(dmls, j.dml) - } - queries, args = w.genSQLs(dmls) + queries, args = w.genSQLs(jobs) failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { t := v.(int) time.Sleep(time.Duration(t) * time.Second) @@ -266,17 +263,43 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { } // genSQLs generate SQLs in single row mode or multiple rows mode. -func (w *DMLWorker) genSQLs(dmls []*DML) ([]string, [][]interface{}) { +func (w *DMLWorker) genSQLs(jobs []*job) ([]string, [][]interface{}) { if w.multipleRows { - return genDMLsWithSameOp(dmls) + return genDMLsWithSameOp(jobs) } - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) - for _, dml := range dmls { - query, arg := dml.genSQL() - queries = append(queries, query...) - args = append(args, arg...) + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + for _, j := range jobs { + var query string + var arg []interface{} + appendQueryAndArg := func() { + queries = append(queries, query) + args = append(args, arg) + } + + switch j.dml.Type() { + case sqlmodel.RowChangeInsert: + if j.safeMode { + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + } else { + query, arg = j.dml.GenSQL(sqlmodel.DMLInsert) + } + + case sqlmodel.RowChangeUpdate: + if j.safeMode { + query, arg = j.dml.GenSQL(sqlmodel.DMLDelete) + appendQueryAndArg() + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + } else { + query, arg = j.dml.GenSQL(sqlmodel.DMLUpdate) + } + + case sqlmodel.RowChangeDelete: + query, arg = j.dml.GenSQL(sqlmodel.DMLDelete) + } + + appendQueryAndArg() } return queries, args } diff --git a/dm/syncer/dml_worker_test.go b/dm/syncer/dml_worker_test.go new file mode 100644 index 00000000000..ffb51a553ab --- /dev/null +++ b/dm/syncer/dml_worker_test.go @@ -0,0 +1,113 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "testing" + + tiddl "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + timodel "github.com/pingcap/tidb/parser/model" + timock "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sqlmodel" +) + +func mockTableInfo(t *testing.T, sql string) *timodel.TableInfo { + t.Helper() + + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + ti, err := tiddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + require.NoError(t, err) + return ti +} + +func TestGenSQL(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb"} + target := &cdcmodel.TableName{Schema: "targetSchema", Table: "targetTable"} + createSQL := "create table db.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" + + cases := []struct { + preValues []interface{} + postValues []interface{} + safeMode bool + + expectedSQLs []string + expectedArgs [][]interface{} + }{ + { + nil, + []interface{}{1, 2, 3, "haha"}, + false, + + []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1, 2, 3, "haha"}}, + }, + { + nil, + []interface{}{1, 2, 3, "haha"}, + true, + + []string{"REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1, 2, 3, "haha"}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + nil, + false, + + []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, + [][]interface{}{{1}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + []interface{}{4, 5, 6, "hihi"}, + false, + + []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, + [][]interface{}{{4, 5, 6, "hihi", 1}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + []interface{}{4, 5, 6, "hihi"}, + true, + + []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1}, {4, 5, 6, "hihi"}}, + }, + } + + worker := &DMLWorker{} + + for _, c := range cases { + tableInfo := mockTableInfo(t, createSQL) + change := sqlmodel.NewRowChange(source, target, c.preValues, c.postValues, tableInfo, nil, nil) + testEC := ec + if c.safeMode { + testEC = ecWithSafeMode + } + dmlJob := newDMLJob(change, testEC) + queries, args := worker.genSQLs([]*job{dmlJob}) + require.Equal(t, c.expectedSQLs, queries) + require.Equal(t, c.expectedArgs, args) + } +} diff --git a/dm/syncer/job.go b/dm/syncer/job.go index d6161b587da..28f6efbbb43 100644 --- a/dm/syncer/job.go +++ b/dm/syncer/job.go @@ -22,15 +22,14 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) type opType byte const ( null opType = iota - insert - update - del + dml ddl xid flush @@ -44,12 +43,8 @@ const ( func (t opType) String() string { switch t { - case insert: - return "insert" - case update: - return "update" - case del: - return "delete" + case dml: + return "dml" case ddl: return "ddl" case xid: @@ -83,7 +78,9 @@ type job struct { // sql example: drop table `s1`.`t1`, `s2`.`t2`. sourceTbls map[string][]*filter.Table targetTable *filter.Table - dml *DML + dml *sqlmodel.RowChange + dmlQueueKey string + safeMode bool 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 @@ -109,16 +106,24 @@ func (j *job) String() string { if j.dml != nil { dmlStr = j.dml.String() } - return fmt.Sprintf("tp: %s, flushSeq: %d, dml: %s, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.flushSeq, dmlStr, j.ddls, j.location, j.startLocation, j.currentLocation) + return fmt.Sprintf("tp: %s, flushSeq: %d, dml: [%s], safemode: %v, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.flushSeq, dmlStr, j.safeMode, j.ddls, j.location, j.startLocation, j.currentLocation) } -func newDMLJob(tp opType, sourceTable, targetTable *filter.Table, dml *DML, ec *eventContext) *job { +func newDMLJob(rowChange *sqlmodel.RowChange, ec *eventContext) *job { + sourceTable := rowChange.GetSourceTable() + targetTable := rowChange.GetTargetTable() + // TODO: remove sourceTbls and targetTable for dml Job return &job{ - tp: tp, - sourceTbls: map[string][]*filter.Table{sourceTable.Schema: {sourceTable}}, - targetTable: targetTable, - dml: dml, + tp: dml, + sourceTbls: map[string][]*filter.Table{ + sourceTable.Schema: { + &filter.Table{Schema: sourceTable.Schema, Name: sourceTable.Table}, + }, + }, + targetTable: &filter.Table{Schema: targetTable.Schema, Name: targetTable.Table}, + dml: rowChange, retry: true, + safeMode: ec.safeMode, location: *ec.lastLocation, startLocation: *ec.startLocation, @@ -228,6 +233,14 @@ func newConflictJob(workerCount int) *job { } } +// newCompactJob is only used for metrics. +func newCompactJob(targetTable *filter.Table) *job { + return &job{ + tp: compact, + targetTable: targetTable, + } +} + // put queues into bucket to monitor them. func queueBucketName(queueID int) string { return fmt.Sprintf("q_%d", queueID%defaultBucketCount) diff --git a/dm/syncer/job_test.go b/dm/syncer/job_test.go index 8b839e189cc..46abaed1955 100644 --- a/dm/syncer/job_test.go +++ b/dm/syncer/job_test.go @@ -14,14 +14,15 @@ package syncer import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) var _ = Suite(&testJobSuite{}) @@ -37,14 +38,8 @@ func (t *testJobSuite) TestJobTypeString(c *C) { null, "", }, { - insert, - "insert", - }, { - update, - "update", - }, { - del, - "delete", + dml, + "dml", }, { ddl, "ddl", @@ -69,14 +64,16 @@ func (t *testJobSuite) TestJobTypeString(c *C) { } } -func (t *testJobSuite) TestJob(c *C) { +func TestJob(t *testing.T) { + t.Parallel() + ddlInfo := &ddlInfo{ sourceTables: []*filter.Table{{Schema: "test1", Name: "t1"}}, targetTables: []*filter.Table{{Schema: "test2", Name: "t2"}}, } - table := &filter.Table{Schema: "test", Name: "t1"} + table := &cdcmodel.TableName{Schema: "test", Table: "t1"} location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} + ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location, safeMode: true} qec := &queryEventContext{ eventContext: ec, originSQL: "create database test", @@ -85,47 +82,34 @@ func (t *testJobSuite) TestJob(c *C) { } schema := "create table test.tb(id int primary key, col1 int unique not null)" - p := parser.New() - se := mock.NewContext() - ti, err := createTableInfo(p, se, 0, schema) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } + ti := mockTableInfo(t, schema) + + exampleChange := sqlmodel.NewRowChange(table, nil, nil, []interface{}{2, 2}, ti, nil, nil) testCases := []struct { job *job jobStr string }{ { - newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti, tiIndex, nil), ec), - "tp: insert, flushSeq: 0, dml: [safemode: true, targetTableID: targetTable, op: insert, columns: [id col1], oldValues: [], values: [2 2]], ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + newDMLJob(exampleChange, ec), + "tp: dml, flushSeq: 0, dml: [type: ChangeInsert, source table: test.t1, target table: test.t1, preValues: [], postValues: [2 2]], safemode: true, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(qec), - "tp: ddl, flushSeq: 0, dml: , ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + "tp: ddl, flushSeq: 0, dml: [], safemode: false, ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newXIDJob(binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation("")), - "tp: xid, flushSeq: 0, dml: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + "tp: xid, flushSeq: 0, dml: [], safemode: false, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newFlushJob(16, 1), - "tp: flush, flushSeq: 1, dml: , ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: flush, flushSeq: 1, dml: [], safemode: false, ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, { newSkipJob(ec), - "tp: skip, flushSeq: 0, dml: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: skip, flushSeq: 0, dml: [], safemode: false, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, } for _, testCase := range testCases { - c.Assert(testCase.job.String(), Equals, testCase.jobStr) + require.Equal(t, testCase.jobStr, testCase.job.String()) } } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 4cd21535bde..9f005e7e993 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -73,6 +73,7 @@ import ( sm "github.com/pingcap/tiflow/dm/syncer/safe-mode" "github.com/pingcap/tiflow/dm/syncer/shardddl" "github.com/pingcap/tiflow/pkg/errorutil" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) var ( @@ -432,11 +433,11 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } } s.checkpointFlushWorker = &checkpointFlushWorker{ - input: nil, // will be created in s.reset() - cp: s.checkpoint, - execError: &s.execError, - afterFlushFn: s.afterFlushCheckpoint, - addCountFunc: s.addCount, + input: nil, // will be created in s.reset() + cp: s.checkpoint, + execError: &s.execError, + afterFlushFn: s.afterFlushCheckpoint, + updateJobMetricsFn: s.updateJobMetrics, } // when Init syncer, set active relay log info @@ -807,19 +808,34 @@ func (s *Syncer) trackTableInfoFromDownstream(tctx *tcontext.Context, sourceTabl return nil } -func (s *Syncer) addCount(isFinished bool, queueBucket string, tp opType, n int64, targetTable *filter.Table) { +var dmlMetric = map[sqlmodel.RowChangeType]string{ + sqlmodel.RowChangeInsert: "insert", + sqlmodel.RowChangeUpdate: "update", + sqlmodel.RowChangeDelete: "delete", +} + +func (s *Syncer) updateJobMetrics(isFinished bool, queueBucket string, j *job) { + tp := j.tp + targetTable := j.targetTable + count := 1 + if tp == ddl { + count = len(j.ddls) + } + m := metrics.AddedJobsTotal if isFinished { - s.count.Add(n) + s.count.Add(int64(count)) m = metrics.FinishedJobsTotal } switch tp { - case insert, update, del, ddl, flush, asyncFlush, conflict, compact: - m.WithLabelValues(tp.String(), s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(n)) + case dml: + m.WithLabelValues(dmlMetric[j.dml.Type()], s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(count)) + case ddl, flush, asyncFlush, conflict, compact: + m.WithLabelValues(tp.String(), s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(count)) case skip, xid: // ignore skip/xid jobs default: - s.tctx.L().Warn("unknown job operation type", zap.Stringer("type", tp)) + s.tctx.L().Warn("unknown job operation type", zap.Stringer("type", j.tp)) } } @@ -901,7 +917,7 @@ var ( // There should not be a second way to send jobs to DML queue or DDL queue. func (s *Syncer) addJob(job *job) { failpoint.Inject("countJobFromOneEvent", func() { - if job.tp == insert { + if job.tp == dml { if job.currentLocation.Position.Compare(lastLocation.Position) == 0 { lastLocationNum++ } else { @@ -921,7 +937,7 @@ func (s *Syncer) addJob(job *job) { } }) failpoint.Inject("countJobFromOneGTID", func() { - if job.tp == insert { + if job.tp == dml { if binlog.CompareLocation(job.currentLocation, lastLocation, true) == 0 { lastLocationNum++ } else { @@ -952,12 +968,12 @@ func (s *Syncer) addJob(job *job) { s.jobWg.Add(1) s.dmlJobCh <- job case ddl: - s.addCount(false, adminQueueName, job.tp, 1, job.targetTable) + s.updateJobMetrics(false, adminQueueName, job) s.jobWg.Add(1) startTime := time.Now() s.ddlJobCh <- job metrics.AddJobDurationHistogram.WithLabelValues("ddl", s.cfg.Name, adminQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - case insert, update, del: + case dml: s.dmlJobCh <- job failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { s.tctx.L().Info("receive dml job", zap.Any("dml job", job)) @@ -1033,7 +1049,7 @@ func (s *Syncer) handleJob(job *job) (added2Queue bool, err error) { // 3. after job is sent to queue switch job.tp { - case insert, update, del: + case dml: // DMl events will generate many jobs and only caller knows all jobs has been sent, so many logics are done by // caller s.isTransactionEnd = false @@ -1370,7 +1386,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. continue } s.jobWg.Done() - s.addCount(true, queueBucket, ddlJob.tp, int64(len(ddlJob.ddls)), ddlJob.targetTable) + s.updateJobMetrics(true, queueBucket, ddlJob) } } @@ -1384,7 +1400,7 @@ func (s *Syncer) successFunc(queueID int, statementsCnt int, jobs []*job) { 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: + case dml: 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() @@ -1392,7 +1408,7 @@ func (s *Syncer) successFunc(queueID int, statementsCnt int, jobs []*job) { } for _, sqlJob := range jobs { - s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetTable) + s.updateJobMetrics(true, queueBucket, sqlJob) } s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket, "statements").Observe(float64(statementsCnt)) @@ -2256,26 +2272,12 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } extRows := generateExtendColumn(originRows, s.tableRouter, sourceTable, s.cfg.SourceID) - rows := originRows - if extRows != nil { - rows = extRows - } - - prunedColumns, prunedRows, err := pruneGeneratedColumnDML(tableInfo, rows) - if err != nil { - return err - } - var ( - dmls []*DML - jobType opType - ) + var dmls []*sqlmodel.RowChange param := &genDMLParam{ - targetTableID: utils.GenTableID(targetTable), - data: prunedRows, + targetTable: targetTable, originalData: originRows, - columns: prunedColumns, sourceTableInfo: tableInfo, sourceTable: sourceTable, extendData: extRows, @@ -2294,7 +2296,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen insert sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenWriteRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = insert case replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2: oldExprFilter, newExprFilter, err2 := s.exprFilterGroup.GetUpdateExprs(sourceTable, tableInfo) @@ -2308,7 +2309,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen update sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenUpdateRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = update case replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2: exprFilter, err2 := s.exprFilterGroup.GetDeleteExprs(sourceTable, tableInfo) @@ -2321,7 +2321,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen delete sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenDeleteRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = del default: ec.tctx.L().Debug("ignoring unrecognized event", zap.String("event", "row"), zap.Stringer("type", ec.header.EventType)) @@ -2329,14 +2328,17 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } startTime := time.Now() + + metricTp := "" for i := range dmls { - job := newDMLJob(jobType, sourceTable, targetTable, dmls[i], &ec) + metricTp = dmlMetric[dmls[i].Type()] + job := newDMLJob(dmls[i], &ec) added2Queue, err2 := s.handleJobFunc(job) if err2 != nil || !added2Queue { return err2 } } - metrics.DispatchBinlogDurationHistogram.WithLabelValues(jobType.String(), s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) + metrics.DispatchBinlogDurationHistogram.WithLabelValues(metricTp, s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) if len(sourceTable.Schema) != 0 { // when in position-based replication, now events before table checkpoint is sent to queue. But in GTID-based diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 1eac0b668fa..c555fb766c6 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/tiflow/pkg/errorutil" + "github.com/pingcap/tiflow/pkg/sqlmodel" sqlmock "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" @@ -841,11 +842,11 @@ func (s *testSyncerSuite) TestRun(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } syncer.handleJobFunc = syncer.addJobToMemory @@ -882,7 +883,7 @@ func (s *testSyncerSuite) TestRun(c *C) { []string{"CREATE TABLE IF NOT EXISTS `test_1`.`t_2` (`id` INT PRIMARY KEY,`name` VARCHAR(24))"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838401), "a"}}, }, { @@ -894,16 +895,16 @@ func (s *testSyncerSuite) TestRun(c *C) { []string{"ALTER TABLE `test_1`.`t_1` ADD INDEX `index1`(`name`)"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402), "b"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(580981944116838401)}}, }, { // safe mode is true, will split update to delete + replace - update, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402)}, {int64(580981944116838401), "b"}}, }, { @@ -984,11 +985,11 @@ func (s *testSyncerSuite) TestRun(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } // When crossing safeModeExitPoint, will generate a flush sql @@ -1000,11 +1001,11 @@ func (s *testSyncerSuite) TestRun(c *C) { expectJobs2 := []*expectJob{ { - insert, + dml, []string{"INSERT INTO `test_1`.`t_2` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(3), "c"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_2` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(3)}}, }, { @@ -1122,11 +1123,11 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } syncer.handleJobFunc = syncer.addJobToMemory @@ -1161,28 +1162,28 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { []string{"CREATE TABLE IF NOT EXISTS `test_1`.`t_1` (`id` INT PRIMARY KEY,`name` VARCHAR(24))"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(1), "a"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1)}}, }, { - update, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(2)}, {int64(1), "b"}}, }, { // start from this event, location passes safeModeExitLocation and safe mode should exit - insert, + dml, []string{"INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(1), "a"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1)}}, }, { - update, + dml, []string{"UPDATE `test_1`.`t_1` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1), "b", int64(2)}}, }, { @@ -1444,16 +1445,46 @@ type expectJob struct { args [][]interface{} } +var defaultDMLType = map[sqlmodel.RowChangeType]sqlmodel.DMLType{ + sqlmodel.RowChangeInsert: sqlmodel.DMLInsert, + sqlmodel.RowChangeUpdate: sqlmodel.DMLUpdate, + sqlmodel.RowChangeDelete: sqlmodel.DMLDelete, +} + func checkJobs(c *C, jobs []*job, expectJobs []*expectJob) { c.Assert(len(jobs), Equals, len(expectJobs), Commentf("jobs = %q", jobs)) for i, job := range jobs { c.Assert(job.tp, Equals, expectJobs[i].tp) + if job.tp == ddl { c.Assert(job.ddls, DeepEquals, expectJobs[i].sqlInJob) - } else if job.tp == insert || job.tp == update || job.tp == del { - sqls, args := job.dml.genSQL() - c.Assert(sqls, DeepEquals, expectJobs[i].sqlInJob) - c.Assert(args, DeepEquals, expectJobs[i].args) + continue + } + + if job.tp == dml { + if !job.safeMode { + sql, args := job.dml.GenSQL(defaultDMLType[job.dml.Type()]) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + continue + } + + // safemode + switch job.dml.Type() { + case sqlmodel.RowChangeInsert: + sql, args := job.dml.GenSQL(sqlmodel.DMLReplace) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + case sqlmodel.RowChangeUpdate: + sql, args := job.dml.GenSQL(sqlmodel.DMLDelete) + sql2, args2 := job.dml.GenSQL(sqlmodel.DMLReplace) + c.Assert([]string{sql, sql2}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args, args2}, DeepEquals, expectJobs[i].args) + case sqlmodel.RowChangeDelete: + sql, args := job.dml.GenSQL(sqlmodel.DMLDelete) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + } } } } @@ -1463,11 +1494,21 @@ var testJobs struct { jobs []*job } +func newDummyJob(tp opType, targetTable *filter.Table, ddls ...string) *job { + return &job{ + tp: tp, + targetTable: targetTable, + ddls: ddls, + dml: &sqlmodel.RowChange{}, + } +} + func (s *Syncer) mockFinishJob(jobs []*expectJob) { for _, job := range jobs { switch job.tp { - case ddl, insert, update, del, flush: - s.addCount(true, "test", job.tp, 1, &filter.Table{}) + case ddl, dml, flush: + dummyJob := newDummyJob(job.tp, &filter.Table{}, job.sqlInJob...) + s.updateJobMetrics(true, "test", dummyJob) } } } @@ -1476,8 +1517,8 @@ func (s *Syncer) addJobToMemory(job *job) (bool, error) { log.L().Info("add job to memory", zap.Stringer("job", job)) switch job.tp { - case ddl, insert, update, del, flush: - s.addCount(false, "test", job.tp, 1, &filter.Table{}) + case ddl, dml, flush: + s.updateJobMetrics(false, "test", job) testJobs.Lock() testJobs.jobs = append(testJobs.jobs, job) testJobs.Unlock() @@ -1500,7 +1541,7 @@ func (s *Syncer) addJobToMemory(job *job) (bool, error) { } } s.resetShardingGroup(job.targetTable) - case insert, update, del: + case dml: for sourceSchema, tbs := range job.sourceTbls { if len(sourceSchema) == 0 { continue @@ -1527,11 +1568,11 @@ func (s *Syncer) setupMockCheckpoint(c *C, checkPointDBConn *sql.Conn, checkPoin s.checkpoint.(*RemoteCheckPoint).dbConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(checkPointDBConn, &retry.FiniteRetryStrategy{})} // mock syncer.flushCpWorker init s.checkpointFlushWorker = &checkpointFlushWorker{ - input: nil, - cp: s.checkpoint, - execError: &s.execError, - afterFlushFn: s.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: nil, + cp: s.checkpoint, + execError: &s.execError, + afterFlushFn: s.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } c.Assert(s.checkpoint.(*RemoteCheckPoint).prepare(tcontext.Background()), IsNil) // disable flush checkpoint periodically diff --git a/dm/tests/downstream_diff_index/run.sh b/dm/tests/downstream_diff_index/run.sh index eab1c4f87b9..99ecab92111 100755 --- a/dm/tests/downstream_diff_index/run.sh +++ b/dm/tests/downstream_diff_index/run.sh @@ -25,7 +25,7 @@ function run() { # worker will inject delete/update sql check inject_points=( - "github.com/pingcap/tiflow/dm/syncer/DownstreamTrackerWhereCheck=return()" + "github.com/pingcap/tiflow/pkg/sqlmodel/DownstreamTrackerWhereCheck=return()" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 101b9d4e7f8..1a9cb587b75 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -710,10 +710,10 @@ function DM_MULTIPLE_ROWS_CASE() { run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb} where a>100 and a<=200;" "count(1): 100" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 30 - insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=insert\]' | wc -l) - replaceMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=replace\]' | wc -l) - updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op="insert on duplicate update"\]' | wc -l) - deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=delete\]' | wc -l) + insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLInsert\]' | wc -l) + replaceMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLReplace\]' | wc -l) + updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLInsertOnDuplicateUpdate\]' | wc -l) + deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLDelete\]' | wc -l) echo $insertMergeCnt $replaceMergeCnt $updateMergeCnt $deleteMergeCnt if [[ "$insertMergeCnt" -le 5 || "$updateMergeCnt" -le 5 || "$deleteMergeCnt" -le 5 || "$replaceMergeCnt" -le 5 ]]; then echo "merge dmls less than 5, insertMergeCnt: $insertMergeCnt, replaceMergeCnt: $replaceMergeCnt, updateMergeCnt: $updateMergeCnt, deleteMergeCnt: $deleteMergeCnt" diff --git a/pkg/sqlmodel/causality_test.go b/pkg/sqlmodel/causality_test.go index 4a49e95640e..998239168df 100644 --- a/pkg/sqlmodel/causality_test.go +++ b/pkg/sqlmodel/causality_test.go @@ -46,6 +46,38 @@ func TestCausalityKeys(t *testing.T) { []interface{}{3, 4, "adef"}, []string{"1.c.db.tb1", "a.c3.db.tb1", "3.c.db.tb1", "a.c3.db.tb1"}, }, + + // test not string key + { + "CREATE TABLE tb1 (a INT, b INT, UNIQUE KEY a(a))", + []interface{}{100, 200}, + nil, + []string{"100.a.db.tb1"}, + }, + + // test text + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY b(b(3)))", + []interface{}{1, "1234"}, + nil, + []string{"123.b.db.tb1"}, + }, + + // test composite keys + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY c2(a, b(3)))", + []interface{}{1, "1234"}, + nil, + []string{"1.a.123.b.db.tb1"}, + }, + + // test value is null + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY c2(a, b(3)))", + []interface{}{1, nil}, + nil, + []string{"1.a.db.tb1"}, + }, } for _, ca := range cases { @@ -71,3 +103,103 @@ func TestCausalityKeysNoRace(t *testing.T) { } wg.Wait() } + +func TestGetCausalityString(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tbl"} + + testCases := []struct { + schema string + values []interface{} + keys []string + }{ + { + // test no keys will use full row data instead of table name + schema: `create table t1(a int)`, + values: []interface{}{10}, + keys: []string{"10.a.db.tbl"}, + }, + { + // one primary key + schema: `create table t2(a int primary key, b double)`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.db.tbl"}, + }, + { + // one unique key + schema: `create table t3(a int unique, b double)`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.db.tbl"}, + }, + { + // one ordinary key + schema: `create table t4(a int, b double, key(b))`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.70.5.b.db.tbl"}, + }, + { + // multiple keys + schema: `create table t5(a int, b text, c int, key(a), key(b(3)))`, + values: []interface{}{13, "abcdef", 15}, + keys: []string{"13.a.abcdef.b.15.c.db.tbl"}, + }, + { + // multiple keys with primary key + schema: `create table t6(a int primary key, b varchar(16) unique)`, + values: []interface{}{16, "xyz"}, + keys: []string{"16.a.db.tbl", "xyz.b.db.tbl"}, + }, + { + // non-integer primary key + schema: `create table t65(a int unique, b varchar(16) primary key)`, + values: []interface{}{16, "xyz"}, + keys: []string{"xyz.b.db.tbl", "16.a.db.tbl"}, + }, + { + // primary key of multiple columns + schema: `create table t7(a int, b int, primary key(a, b))`, + values: []interface{}{59, 69}, + keys: []string{"59.a.69.b.db.tbl"}, + }, + { + // ordinary key of multiple columns + schema: `create table t75(a int, b int, c int, key(a, b), key(c, b))`, + values: []interface{}{48, 58, 68}, + keys: []string{"48.a.58.b.68.c.db.tbl"}, + }, + { + // so many keys + schema: ` + create table t8( + a int, b int, c int, + primary key(a, b), + unique key(b, c), + key(a, b, c), + unique key(c, a) + ) + `, + values: []interface{}{27, 37, 47}, + keys: []string{"27.a.37.b.db.tbl", "37.b.47.c.db.tbl", "47.c.27.a.db.tbl"}, + }, + { + // `null` for unique key + schema: ` + create table t8( + a int, b int default null, + primary key(a), + unique key(b) + ) + `, + values: []interface{}{17, nil}, + keys: []string{"17.a.db.tbl"}, + }, + } + + for _, ca := range testCases { + ti := mockTableInfo(t, ca.schema) + change := NewRowChange(source, nil, nil, ca.values, ti, nil, nil) + change.lazyInitIdentityInfo() + require.Equal(t, ca.keys, change.getCausalityString(ca.values)) + } +} diff --git a/pkg/sqlmodel/reduce.go b/pkg/sqlmodel/reduce.go index 6e146e2d54c..807d8af64a8 100644 --- a/pkg/sqlmodel/reduce.go +++ b/pkg/sqlmodel/reduce.go @@ -120,11 +120,11 @@ func (r *RowChange) Reduce(preRowChange *RowChange) { r.calculateType() } -// Split will split current RowChangeUpdate into two RowChangeDelete and +// SplitUpdate will split current RowChangeUpdate into two RowChangeDelete and // RowChangeInsert one. The behaviour is undefined for other types of RowChange. -func (r *RowChange) Split() (*RowChange, *RowChange) { +func (r *RowChange) SplitUpdate() (*RowChange, *RowChange) { if r.tp != RowChangeUpdate { - log.L().DPanic("Split should only be called on RowChangeUpdate", + log.L().DPanic("SplitUpdate should only be called on RowChangeUpdate", zap.Stringer("rowChange", r)) return nil, nil } diff --git a/pkg/sqlmodel/reduce_test.go b/pkg/sqlmodel/reduce_test.go index 876e2089252..8592b91b48f 100644 --- a/pkg/sqlmodel/reduce_test.go +++ b/pkg/sqlmodel/reduce_test.go @@ -53,7 +53,7 @@ func TestSplit(t *testing.T) { change := NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{3, 4}, sourceTI1, nil, nil) require.True(t, change.IsIdentityUpdated()) - del, ins := change.Split() + del, ins := change.SplitUpdate() delIDKey := del.IdentityKey() require.NotZero(t, delIDKey) insIDKey := ins.IdentityKey() diff --git a/pkg/sqlmodel/row_change.go b/pkg/sqlmodel/row_change.go index a4a3cbfd8ca..7c29ce7dafa 100644 --- a/pkg/sqlmodel/row_change.go +++ b/pkg/sqlmodel/row_change.go @@ -375,3 +375,23 @@ func (r *RowChange) GenSQL(tp DMLType) (string, []interface{}) { zap.Stringer("DMLType", tp)) return "", nil } + +// GetPreValues is only used in tests. +func (r *RowChange) GetPreValues() []interface{} { + return r.preValues +} + +// GetPostValues is only used in tests. +func (r *RowChange) GetPostValues() []interface{} { + return r.postValues +} + +// GetSourceTable returns TableName of the source table. +func (r *RowChange) GetSourceTable() *cdcmodel.TableName { + return r.sourceTable +} + +// GetTargetTable returns TableName of the target table. +func (r *RowChange) GetTargetTable() *cdcmodel.TableName { + return r.targetTable +} From 7bcfae4e0368322dadcf4c8b2ba74a94c5ab6bf5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Fri, 11 Feb 2022 18:57:36 +0800 Subject: [PATCH 51/72] tests(ticdc): fix canal_json_basic charset issue (#4558) ref pingcap/tiflow#4555 --- scripts/download-integration-test-binaries.sh | 2 +- tests/integration_tests/_utils/check_sync_diff | 2 +- tests/integration_tests/canal_json_basic/data/data.sql | 8 +++----- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index 0689f24328d..da20d2de925 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Copyright 20201 PingCAP, Inc. +# 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. diff --git a/tests/integration_tests/_utils/check_sync_diff b/tests/integration_tests/_utils/check_sync_diff index 7f1b72e3f7c..b0824e1473f 100755 --- a/tests/integration_tests/_utils/check_sync_diff +++ b/tests/integration_tests/_utils/check_sync_diff @@ -21,7 +21,7 @@ cd $workdir i=0 while [ $i -lt $check_time ]; do rm -rf $workdir/sync_diff/ - $binary --config=$conf >>$LOG 2>&1 + $binary --log-level=debug --config=$conf >>$LOG 2>&1 ret=$? if [ "$ret" == 0 ]; then echo "check diff successfully" diff --git a/tests/integration_tests/canal_json_basic/data/data.sql b/tests/integration_tests/canal_json_basic/data/data.sql index dfc1f40d725..b6fc1c90e42 100644 --- a/tests/integration_tests/canal_json_basic/data/data.sql +++ b/tests/integration_tests/canal_json_basic/data/data.sql @@ -41,9 +41,7 @@ CREATE TABLE multi_data_type -- t_bit BIT(64), t_json JSON, PRIMARY KEY (id) -) ENGINE = InnoDB - DEFAULT CHARSET = utf8 - COLLATE = utf8_bin; +); INSERT INTO multi_data_type( t_tinyint, t_tinyint_unsigned, t_smallint, t_smallint_unsigned, t_mediumint , t_mediumint_unsigned, t_int, t_int_unsigned, t_bigint, t_bigint_unsigned @@ -55,7 +53,7 @@ INSERT INTO multi_data_type( t_tinyint, t_tinyint_unsigned, t_smallint, t_smalli , t_set, t_json) VALUES ( -1, 1, -129, 129, -65536, 65536, -16777216, 16777216, -2147483649, 2147483649 , true, 123.456, 123.123, 123456789012.123456789012 - , '测', '测试', x'89504E470D0A1A0A', x'89504E470D0A1A0A', '测试tinytext', '测试text', '测试mediumtext', '测试longtext' + , '测', '测试', '89504E470D0A1A0A', '89504E470D0A1A0A', '测试tinytext', '测试text', '测试mediumtext', '测试longtext' , 'tinyblob', 'blob', 'mediumblob', 'longblob' , '1977-01-01', '9999-12-31 23:59:59', '19731230153000', '23:59:59' , 'enum2' @@ -71,7 +69,7 @@ INSERT INTO multi_data_type( t_tinyint, t_tinyint_unsigned, t_smallint, t_smalli , t_set, t_json) VALUES ( -2, 2, -130, 130, -65537, 65537, -16777217, 16777217, -2147483650, 2147483650 , false, 123.4567, 123.1237, 123456789012.1234567890127 - , '2', '测试2', x'89504E470D0A1A0B', x'89504E470D0A1A0B', '测试2tinytext', '测试2text', '测试2mediumtext', '测试longtext' + , '2', '测试2', '89504E470D0A1A0B', '89504E470D0A1A0B', '测试2tinytext', '测试2text', '测试2mediumtext', '测试longtext' , 'tinyblob2', 'blob2', 'mediumblob2', 'longblob2' , '2021-01-01', '2021-12-31 23:59:59', '19731230153000', '22:59:59' , 'enum1' From 8a709d74866178a1ee85dc7aafcb0e70b6e1f494 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Fri, 11 Feb 2022 20:05:38 +0800 Subject: [PATCH 52/72] cdc/metrics: Integrate sarama producer metrics (#4520) close pingcap/tiflow#4561 --- cdc/metrics.go | 2 + cdc/owner/ddl_sink.go | 9 +- cdc/sink/producer/kafka/kafka.go | 15 + cdc/sink/producer/kafka/metrics.go | 118 +++ go.mod | 1 + metrics/grafana/ticdc.json | 1383 ++++++++++++++++++++++++---- pkg/etcd/etcd_test.go | 1 + 7 files changed, 1357 insertions(+), 172 deletions(-) create mode 100644 cdc/sink/producer/kafka/metrics.go diff --git a/cdc/metrics.go b/cdc/metrics.go index 1c998521731..1af38df21f0 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tiflow/cdc/puller" redowriter "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sink/producer/kafka" "github.com/pingcap/tiflow/cdc/sorter" "github.com/pingcap/tiflow/cdc/sorter/leveldb" "github.com/pingcap/tiflow/cdc/sorter/memory" @@ -59,4 +60,5 @@ func init() { leveldb.InitMetrics(registry) redowriter.InitMetrics(registry) db.InitMetrics(registry) + kafka.InitMetrics(registry) } diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index ea89ff101de..d4c17256f5b 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -27,6 +27,7 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -90,7 +91,9 @@ func ddlSinkInitializer(ctx cdcContext.Context, a *ddlSinkImpl, id model.ChangeF return errors.Trace(err) } - s, err := sink.New(ctx, id, info.SinkURI, filter, info.Config, info.Opts, a.errCh) + stdCtx := util.PutChangefeedIDInCtx(ctx, id) + stdCtx = util.PutRoleInCtx(stdCtx, util.RoleOwner) + s, err := sink.New(stdCtx, id, info.SinkURI, filter, info.Config, info.Opts, a.errCh) if err != nil { return errors.Trace(err) } @@ -99,13 +102,13 @@ func ddlSinkInitializer(ctx cdcContext.Context, a *ddlSinkImpl, id model.ChangeF if !info.SyncPointEnabled { return nil } - syncPointStore, err := sink.NewSyncpointStore(ctx, id, info.SinkURI) + syncPointStore, err := sink.NewSyncpointStore(stdCtx, id, info.SinkURI) if err != nil { return errors.Trace(err) } a.syncPointStore = syncPointStore - if err := a.syncPointStore.CreateSynctable(ctx); err != nil { + if err := a.syncPointStore.CreateSynctable(stdCtx); err != nil { return errors.Trace(err) } return nil diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 027e277ae92..dd13cfdfaba 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -39,6 +39,9 @@ import ( const ( // defaultPartitionNum specifies the default number of partitions when we create the topic. defaultPartitionNum = 3 + + // flushMetricsInterval specifies the interval of refresh sarama metrics. + flushMetricsInterval = 5 * time.Second ) const ( @@ -75,6 +78,8 @@ type kafkaSaramaProducer struct { role util.Role id model.ChangeFeedID + + metricsMonitor *saramaMetricsMonitor } type kafkaProducerClosingFlag = int32 @@ -256,6 +261,8 @@ func (k *kafkaSaramaProducer) Close() error { log.Info("sync client closed", zap.Duration("duration", time.Since(start)), zap.String("changefeed", k.id), zap.Any("role", k.role)) } + + k.metricsMonitor.Cleanup() return nil } @@ -266,12 +273,17 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { zap.String("changefeed", k.id), zap.Any("role", k.role)) k.stop() }() + + ticker := time.NewTicker(flushMetricsInterval) + defer ticker.Stop() for { select { case <-ctx.Done(): return ctx.Err() case <-k.closeCh: return nil + case <-ticker.C: + k.metricsMonitor.CollectMetrics() case err := <-k.failpointCh: log.Warn("receive from failpoint chan", zap.Error(err), zap.String("changefeed", k.id), zap.Any("role", k.role)) @@ -366,6 +378,9 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, id: changefeedID, role: role, + + metricsMonitor: NewSaramaMetricsMonitor(cfg.MetricRegistry, + util.CaptureAddrFromCtx(ctx), changefeedID), } go func() { if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { diff --git a/cdc/sink/producer/kafka/metrics.go b/cdc/sink/producer/kafka/metrics.go new file mode 100644 index 00000000000..600e2eb3940 --- /dev/null +++ b/cdc/sink/producer/kafka/metrics.go @@ -0,0 +1,118 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kafka + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/rcrowley/go-metrics" +) + +var ( + // batch-size + batchSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_batch_size", + Help: "the number of bytes sent per partition per request for all topics", + }, []string{"capture", "changefeed"}) + + // record-send-rate + recordSendRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_record_send_rate", + Help: "Records/second sent to all topics", + }, []string{"capture", "changefeed"}) + + // records-per-request + recordPerRequestGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_records_per_request", + Help: "the number of records sent per request for all topics", + }, []string{"capture", "changefeed"}) + + // compression-ratio + compressionRatioGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_compression_ratio", + Help: "the compression ratio times 100 of record batches for all topics", + }, []string{"capture", "changefeed"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(batchSizeGauge) + registry.MustRegister(recordSendRateGauge) + registry.MustRegister(recordPerRequestGauge) + registry.MustRegister(compressionRatioGauge) +} + +// sarama metrics names, see https://pkg.go.dev/github.com/Shopify/sarama#pkg-overview +const ( + batchSizeMetricName = "batch-size" + recordSendRateMetricName = "record-send-rate" + recordPerRequestMetricName = "records-per-request" + compressionRatioMetricName = "compression-ratio" +) + +type saramaMetricsMonitor struct { + captureAddr string + changefeedID string + + registry metrics.Registry +} + +// CollectMetrics collect all monitored metrics +func (sm *saramaMetricsMonitor) CollectMetrics() { + batchSizeMetric := sm.registry.Get(batchSizeMetricName) + if histogram, ok := batchSizeMetric.(metrics.Histogram); ok { + batchSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } + + recordSendRateMetric := sm.registry.Get(recordSendRateMetricName) + if meter, ok := recordSendRateMetric.(metrics.Meter); ok { + recordSendRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(meter.Rate1()) + } + + recordPerRequestMetric := sm.registry.Get(recordPerRequestMetricName) + if histogram, ok := recordPerRequestMetric.(metrics.Histogram); ok { + recordPerRequestGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } + + compressionRatioMetric := sm.registry.Get(compressionRatioMetricName) + if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { + compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } +} + +func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string) *saramaMetricsMonitor { + return &saramaMetricsMonitor{ + captureAddr: captureAddr, + changefeedID: changefeedID, + registry: registry, + } +} + +func (sm *saramaMetricsMonitor) Cleanup() { + batchSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordSendRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordPerRequestGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + compressionRatioGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) +} diff --git a/go.mod b/go.mod index bc9edb33616..c713e718dac 100644 --- a/go.mod +++ b/go.mod @@ -63,6 +63,7 @@ require ( github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/r3labs/diff v1.1.0 + github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 github.com/shopspring/decimal v1.3.0 github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.2.1 diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 91f5d961714..a1f00411376 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -124,8 +124,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": null, - "iteration": 1640792097028, + "id": 33, + "iteration": 1644481309661, "links": [], "panels": [ { @@ -942,7 +942,7 @@ "h": 5, "w": 7, "x": 0, - "y": 3 + "y": 2 }, "id": 4, "links": [], @@ -1005,7 +1005,7 @@ "h": 10, "w": 7, "x": 7, - "y": 3 + "y": 2 }, "id": 90, "links": [], @@ -1285,7 +1285,7 @@ "h": 5, "w": 7, "x": 0, - "y": 8 + "y": 7 }, "id": 138, "links": [], @@ -1357,7 +1357,7 @@ "h": 7, "w": 9, "x": 0, - "y": 13 + "y": 12 }, "id": 86, "legend": { @@ -1472,7 +1472,7 @@ "h": 7, "w": 8, "x": 9, - "y": 13 + "y": 12 }, "hiddenSeries": false, "id": 102, @@ -1568,7 +1568,7 @@ "h": 7, "w": 7, "x": 17, - "y": 13 + "y": 12 }, "id": 82, "legend": { @@ -1657,7 +1657,7 @@ "h": 7, "w": 12, "x": 0, - "y": 20 + "y": 19 }, "hiddenSeries": false, "id": 3, @@ -1776,7 +1776,7 @@ "h": 7, "w": 12, "x": 12, - "y": 20 + "y": 19 }, "hiddenSeries": false, "id": 2, @@ -1872,7 +1872,7 @@ "h": 7, "w": 12, "x": 0, - "y": 27 + "y": 26 }, "id": 163, "legend": { @@ -1967,7 +1967,7 @@ "h": 7, "w": 12, "x": 12, - "y": 27 + "y": 26 }, "hiddenSeries": false, "id": 253, @@ -2073,7 +2073,7 @@ "h": 7, "w": 12, "x": 0, - "y": 34 + "y": 33 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2144,7 +2144,7 @@ "h": 7, "w": 12, "x": 12, - "y": 34 + "y": 33 }, "hiddenSeries": false, "id": 35, @@ -2252,7 +2252,7 @@ "h": 7, "w": 12, "x": 0, - "y": 41 + "y": 40 }, "hiddenSeries": false, "id": 34, @@ -2355,7 +2355,7 @@ "h": 7, "w": 12, "x": 12, - "y": 41 + "y": 40 }, "hiddenSeries": false, "id": 36, @@ -2472,7 +2472,7 @@ "h": 7, "w": 12, "x": 0, - "y": 48 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2541,7 +2541,7 @@ "h": 7, "w": 12, "x": 12, - "y": 48 + "y": 47 }, "hiddenSeries": false, "id": 98, @@ -2657,7 +2657,7 @@ "h": 7, "w": 12, "x": 0, - "y": 55 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2727,7 +2727,7 @@ "h": 7, "w": 12, "x": 12, - "y": 55 + "y": 54 }, "hiddenSeries": false, "id": 95, @@ -2876,7 +2876,7 @@ "h": 7, "w": 12, "x": 0, - "y": 62 + "y": 61 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2946,7 +2946,7 @@ "h": 7, "w": 12, "x": 12, - "y": 62 + "y": 61 }, "hiddenSeries": false, "id": 83, @@ -3054,7 +3054,7 @@ "h": 7, "w": 12, "x": 0, - "y": 69 + "y": 68 }, "id": 149, "legend": { @@ -3141,7 +3141,7 @@ "h": 7, "w": 12, "x": 12, - "y": 69 + "y": 68 }, "hiddenSeries": false, "id": 166, @@ -3238,91 +3238,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 76 - }, - "id": 164, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture, changefeed)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{ capture }}-{{ changefeed }}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Processor Memory Consumption Per Changefeed", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "columns": [], "datasource": "${DS_TEST-CLUSTER}", @@ -3383,7 +3298,7 @@ "h": 8, "w": 12, "x": 0, - "y": 77 + "y": 75 }, "id": 200, "options": { @@ -3475,6 +3390,91 @@ } ], "type": "table" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 75 + }, + "id": 164, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture, changefeed)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{ capture }}-{{ changefeed }}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor Memory Consumption Per Changefeed", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "title": "Changefeed", @@ -6359,7 +6359,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 4 }, "id": 13, "panels": [ @@ -8129,7 +8129,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 5 }, "id": 130, "panels": [ @@ -8613,7 +8613,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 266, "panels": [ @@ -9561,7 +9561,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 58, "panels": [ @@ -11520,7 +11520,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 8 }, "id": 294, "panels": [ @@ -12293,7 +12293,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 9 }, "id": 155, "panels": [ @@ -12350,8 +12350,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" } }, "seriesOverrides": [ @@ -12536,8 +12536,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" } }, "seriesOverrides": [], @@ -12643,8 +12643,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" } }, "seriesOverrides": [ @@ -12765,8 +12765,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" } }, "seriesOverrides": [ @@ -12860,6 +12860,13 @@ } ], "repeat": "runtime_instance", + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" + } + }, "title": "Runtime $runtime_instance", "type": "row" }, @@ -12869,53 +12876,650 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 10 }, - "id": 187, + "id": 411, "panels": [ { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The latency distributions of fsync called by redo writer", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 8 + "y": 6 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 172, + "id": 412, "legend": { - "show": true - }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, - "targets": [ - { - "exemplar": true, - "expr": "max(rate(ticdc_redo_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "repeatIteration": 1644481309661, + "repeatPanelId": 157, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, + "seriesOverrides": [ + { + "alias": "alloc-from-os", + "fill": 3, + "lines": true, + "stack": false + }, + { + "alias": "gc-threshold", + "bars": false, + "color": "#C4162A", + "lines": true, + "linewidth": 2, + "stack": false + }, + { + "alias": "gc", + "bars": false, + "color": "#C4162A", + "hideTooltip": true, + "legend": false, + "pointradius": 3, + "points": true, + "stack": false + } + ], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "alloc-from-os", + "refId": "A" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-inuse", + "refId": "H" + }, + { + "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-garbage", + "refId": "C" + }, + { + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "reserved-by-go", + "refId": "B" + }, + { + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "used-by-go", + "refId": "D" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc-threshold", + "refId": "E" + }, + { + "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc", + "refId": "F" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory Usage", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Count of live objects.", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 6 + }, + "id": 413, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "repeatIteration": 1644481309661, + "repeatPanelId": 158, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "objects", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated Live Objects", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process Go garbage collection STW pause duration", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 414, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatIteration": 1644481309661, + "repeatPanelId": 160, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, + "seriesOverrides": [ + {} + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 1, + "legendFormat": "min", + "refId": "A", + "step": 40 + }, + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile!~\"0|1\"}", + "format": "time_series", + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{quantile}}", + "refId": "B" + }, + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"1\"}", + "format": "time_series", + "instant": false, + "intervalFactor": 1, + "legendFormat": "max", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GC STW Duration (last 256 GC cycles)", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 1, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The throughput of Go's memory allocator.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 415, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatIteration": 1644481309661, + "repeatPanelId": 161, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, + "seriesOverrides": [ + { + "alias": "sweep", + "transform": "negative-Y" + }, + { + "alias": "alloc-ops", + "yaxis": 2 + }, + { + "alias": "swepp-ops", + "transform": "negative-Y", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc", + "refId": "A" + }, + { + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep", + "refId": "B" + }, + { + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc-ops", + "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "swepp-ops", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Allocator Throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "Bps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": true, + "alignLevel": null + } + } + ], + "repeat": null, + "repeatIteration": 1644481309661, + "repeatPanelId": 155, + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, + "title": "Runtime $runtime_instance", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 11 + }, + "id": 187, + "panels": [ + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The latency distributions of fsync called by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 8 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 172, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "max(rate(ticdc_redo_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", "queryType": "randomWalk", "refId": "A" } @@ -13230,6 +13834,431 @@ ], "title": "Redo", "type": "row" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 12 + }, + "id": 402, + "panels": [], + "title": "Kafka Sink", + "type": "row" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of bytes sent per partition per request for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "hiddenSeries": false, + "id": 410, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "batch size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of records sent per request for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "hiddenSeries": false, + "id": 408, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records send rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "wps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of records sent per request for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 20 + }, + "hiddenSeries": false, + "id": 409, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records per request", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the compression ratio times 100 of record batches for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 20 + }, + "hiddenSeries": false, + "id": 416, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "compression ratio", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "refresh": "1m", @@ -13240,7 +14269,11 @@ "list": [ { "allValue": null, - "current": {}, + "current": { + "isNone": true, + "text": "None", + "value": "" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "", "hide": 0, @@ -13262,7 +14295,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(ticdc_processor_resolved_ts{tidb_cluster=\"$tidb_cluster\"}, changefeed)", "hide": 0, @@ -13284,7 +14320,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "hide": 0, @@ -13306,7 +14345,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(tikv_engine_size_bytes{tidb_cluster=\"$tidb_cluster\"}, instance)", "hide": 0, @@ -13381,7 +14423,10 @@ }, { "allValue": "", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "hide": 0, @@ -13435,5 +14480,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 32 -} \ No newline at end of file + "version": 33 +} diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index 79f4c60c8f6..1ccc678117d 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -408,6 +408,7 @@ func TestGetOwnerRevision(t *testing.T) { defer wg.Done() sess, err := concurrency.NewSession(s.client.Client.Unwrap(), concurrency.WithTTL(10 /* seconds */)) + require.Nil(t, err) election := concurrency.NewElection(sess, CaptureOwnerKey) mockCaptureID := fmt.Sprintf("capture-%d", i) From c48f208177e825fc52c905fed70a7bbaea263ba9 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Sun, 13 Feb 2022 21:59:37 +0800 Subject: [PATCH 53/72] syncer(dm): fix startGTID is equal to endGTID (#4386) ref pingcap/tiflow#4287 --- dm/pkg/binlog/position.go | 13 ++++- dm/pkg/binlog/position_test.go | 24 +++++++++ dm/syncer/binlog_locations.go | 85 ++++++++++++++++++++++++------ dm/syncer/binlog_locations_test.go | 31 +++++++---- dm/syncer/syncer.go | 6 --- 5 files changed, 126 insertions(+), 33 deletions(-) diff --git a/dm/pkg/binlog/position.go b/dm/pkg/binlog/position.go index 9573f2292fe..ad7cfb1e719 100644 --- a/dm/pkg/binlog/position.go +++ b/dm/pkg/binlog/position.go @@ -401,9 +401,18 @@ func (l *Location) ResetSuffix() { // SetGTID set new gtid for location // Use this func instead of GITSet.Set to avoid change other location. func (l *Location) SetGTID(gset gmysql.GTIDSet) error { - flavor := gmysql.MySQLFlavor - if _, ok := l.gtidSet.(*gtid.MariadbGTIDSet); ok { + var flavor string + + switch gset.(type) { + case *gmysql.MysqlGTIDSet: + flavor = gmysql.MySQLFlavor + case *gmysql.MariadbGTIDSet: flavor = gmysql.MariaDBFlavor + case nil: + l.gtidSet = nil + return nil + default: + return fmt.Errorf("unknown GTIDSet type: %T", gset) } newGTID := gtid.MinGTIDSet(flavor) diff --git a/dm/pkg/binlog/position_test.go b/dm/pkg/binlog/position_test.go index 715451718fb..a61996f94b7 100644 --- a/dm/pkg/binlog/position_test.go +++ b/dm/pkg/binlog/position_test.go @@ -763,6 +763,30 @@ func (t *testPositionSuite) TestSetGTID(c *C) { c.Assert(loc.gtidSet.String(), Equals, GTIDSetStr2) c.Assert(loc2.gtidSet.String(), Equals, GTIDSetStr) c.Assert(CompareLocation(loc, loc2, true), Equals, 1) + + loc2.gtidSet = nil + err = loc2.SetGTID(mysqlSet) + c.Assert(err, IsNil) + c.Assert(loc2.gtidSet.String(), Equals, GTIDSetStr) +} + +func (t *testPositionSuite) TestSetGTIDMariaDB(c *C) { + gSetStr := "1-1-1,2-2-2" + gSet, err := gtid.ParserGTID("mariadb", gSetStr) + c.Assert(err, IsNil) + gSetOrigin := gSet.Origin() + + loc := Location{ + Position: gmysql.Position{ + Name: "mysql-bin.00002", + Pos: 2333, + }, + gtidSet: nil, + Suffix: 0, + } + err = loc.SetGTID(gSetOrigin) + c.Assert(err, IsNil) + c.Assert(loc.gtidSet.String(), Equals, gSetStr) } func (t *testPositionSuite) TestExtractSuffix(c *C) { diff --git a/dm/syncer/binlog_locations.go b/dm/syncer/binlog_locations.go index 2d0a29c1cbe..f73630015fc 100644 --- a/dm/syncer/binlog_locations.go +++ b/dm/syncer/binlog_locations.go @@ -18,11 +18,12 @@ import ( "strings" "sync" - "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" "github.com/pingcap/tiflow/dm/pkg/log" ) @@ -33,7 +34,8 @@ type locationRecorder struct { // | | // curStartLocation curEndLocation // there may be more events between curStartLocation and curEndLocation due to the limitation of binlog or - // implementation of DM, but those events should always belong to one transaction. + // implementation of DM, but in such scenario, those events should always belong to one transaction. + // When curStartLocation is equal to curEndLocation, it means current event is not a data change. // // curStartLocation is used when // - display a meaningful location @@ -44,7 +46,7 @@ type locationRecorder struct { curStartLocation binlog.Location curEndLocation binlog.Location - // txnEndLocation is the end location of last transaction. If current event is the last event of a txn, + // txnEndLocation is the end location of last seen transaction. If current event is the last event of a txn, // txnEndLocation will be assigned from curEndLocation // it is used when // - reset binlog replication for a finer granularity @@ -55,15 +57,20 @@ type locationRecorder struct { // distinguish DML query event. inDML bool + // we assign startGTID := endGTID after COMMIT, so at COMMIT we turn on the flag. + needUpdateStartGTID bool + mu sync.Mutex // guard curEndLocation because Syncer.printStatus is reading it from another goroutine. } func (l *locationRecorder) reset(loc binlog.Location) { l.mu.Lock() defer l.mu.Unlock() - l.curStartLocation = loc - l.curEndLocation = loc - l.txnEndLocation = loc + // need to clone location to avoid the modification leaking outside + clone := loc.Clone() + l.curStartLocation = clone + l.curEndLocation = clone + l.txnEndLocation = clone } //nolint:unused @@ -102,8 +109,13 @@ func shouldUpdatePos(e *replication.BinlogEvent) bool { return true } -func (l *locationRecorder) setCurrentGTID(gset mysql.GTIDSet) { - err := l.curEndLocation.SetGTID(gset) +func (l *locationRecorder) updateCurStartGTID() { + gsetWrapped := l.curEndLocation.GetGTID() + if gsetWrapped == nil { + return + } + gset := gsetWrapped.Origin() + err := l.curStartLocation.SetGTID(gset) if err != nil { log.L().DPanic("failed to set GTID set", zap.Any("GTID set", gset), @@ -111,6 +123,40 @@ func (l *locationRecorder) setCurrentGTID(gset mysql.GTIDSet) { } } +func (l *locationRecorder) setCurEndGTID(e *replication.BinlogEvent) { + gtidStr, err := event.GetGTIDStr(e) + if err != nil { + log.L().DPanic("failed to get GTID from event", + zap.Any("event", e), + zap.Error(err)) + return + } + + gset := l.curEndLocation.GetGTID() + + if gset == nil { + gset, _ = gtid.ParserGTID("", gtidStr) + _ = l.curEndLocation.SetGTID(gset.Origin()) + return + } + + clone := gset.Clone() + err = clone.Update(gtidStr) + if err != nil { + log.L().DPanic("failed to update GTID set", + zap.String("GTID", gtidStr), + zap.Error(err)) + return + } + + err = l.curEndLocation.SetGTID(clone.Origin()) + if err != nil { + log.L().DPanic("failed to set GTID set", + zap.String("GTID", gtidStr), + zap.Error(err)) + } +} + // update maintains the member of locationRecorder as their definitions. // - curStartLocation is assigned to curEndLocation // - curEndLocation is tried to be updated in-place @@ -119,7 +165,13 @@ func (l *locationRecorder) update(e *replication.BinlogEvent) { l.mu.Lock() defer l.mu.Unlock() - l.curStartLocation = l.curEndLocation + // GTID part is maintained separately + l.curStartLocation.Position = l.curEndLocation.Position + + if l.needUpdateStartGTID { + l.updateCurStartGTID() + l.needUpdateStartGTID = false + } if !shouldUpdatePos(e) { return @@ -138,11 +190,18 @@ func (l *locationRecorder) update(e *replication.BinlogEvent) { l.curEndLocation.Position.Pos = e.Header.LogPos switch ev := e.Event.(type) { + case *replication.GTIDEvent: + l.setCurEndGTID(e) + case *replication.MariadbGTIDEvent: + l.setCurEndGTID(e) + if !ev.IsDDL() { + l.inDML = true + } case *replication.XIDEvent: // for transactional engines like InnoDB, COMMIT is xid event - l.setCurrentGTID(ev.GSet) l.saveTxnEndLocation() l.inDML = false + l.needUpdateStartGTID = true case *replication.QueryEvent: query := strings.TrimSpace(string(ev.Query)) switch query { @@ -159,13 +218,9 @@ func (l *locationRecorder) update(e *replication.BinlogEvent) { if l.inDML { return } + l.needUpdateStartGTID = true - l.setCurrentGTID(ev.GSet) l.saveTxnEndLocation() - case *replication.MariadbGTIDEvent: - if !ev.IsDDL() { - l.inDML = true - } } } diff --git a/dm/syncer/binlog_locations_test.go b/dm/syncer/binlog_locations_test.go index 326c3dd7626..4df4260a7c0 100644 --- a/dm/syncer/binlog_locations_test.go +++ b/dm/syncer/binlog_locations_test.go @@ -47,7 +47,7 @@ type testLocationSuite struct { currGSet gtid.Set } -func (s *testLocationSuite) SetUpSuite(c *C) { +func (s *testLocationSuite) SetUpTest(c *C) { s.serverID = 101 s.binlogFile = "mysql-bin.000001" s.nextBinlogFile = "mysql-bin.000002" @@ -55,7 +55,7 @@ func (s *testLocationSuite) SetUpSuite(c *C) { s.flavor = mysql.MySQLFlavor s.prevGSetStr = "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14" s.lastGTIDStr = "3ccc475b-2343-11e7-be21-6c0b84d59f30:14" - s.currGSetStr = "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14" + s.currGSetStr = "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-15" var err error s.prevGSet, err = gtid.ParserGTID(s.flavor, s.prevGSetStr) @@ -179,10 +179,20 @@ func (s *testLocationSuite) checkOneTxnEvents(c *C, events []*replication.Binlog c.Assert(r.curEndLocation, DeepEquals, expected[0]) c.Assert(r.txnEndLocation, DeepEquals, expected[0]) + seenGTID := false for i, e := range events { r.update(e) c.Assert(r.curStartLocation, DeepEquals, expected[i]) - c.Assert(r.curEndLocation, DeepEquals, expected[i+1]) + + if e.Header.EventType == replication.GTID_EVENT || e.Header.EventType == replication.MARIADB_GTID_EVENT { + seenGTID = true + } + if seenGTID { + c.Assert(r.curEndLocation.Position, DeepEquals, expected[i+1].Position) + c.Assert(r.curEndLocation.GetGTID(), DeepEquals, expected[len(expected)-1].GetGTID()) + } else { + c.Assert(r.curEndLocation, DeepEquals, expected[i+1]) + } if i == len(events)-1 { switch e.Header.EventType { @@ -197,7 +207,8 @@ func (s *testLocationSuite) checkOneTxnEvents(c *C, events []*replication.Binlog } } -func (s *testLocationSuite) generateExpectLocations( +// generateExpectedLocations generates binlog position part of location from given event. +func (s *testLocationSuite) generateExpectedLocations( initLoc binlog.Location, events []*replication.BinlogEvent, ) []binlog.Location { @@ -224,7 +235,7 @@ func (s *testLocationSuite) TestDMLUpdateLocationsGTID(c *C) { // we have 8 events c.Assert(events, HasLen, 8) - expected := s.generateExpectLocations(s.loc, events) + expected := s.generateExpectedLocations(s.loc, events) c.Assert(expected[8].SetGTID(s.currGSet.Origin()), IsNil) @@ -246,7 +257,7 @@ func (s *testLocationSuite) TestDMLUpdateLocationsPos(c *C) { events[7].Event.(*replication.XIDEvent).GSet = nil events = append(events[:2], events[4:]...) - expected := s.generateExpectLocations(loc, events) + expected := s.generateExpectedLocations(loc, events) s.checkOneTxnEvents(c, events, expected) } @@ -257,7 +268,7 @@ func (s *testLocationSuite) TestDDLUpdateLocationsGTID(c *C) { // we have 5 events c.Assert(events, HasLen, 5) - expected := s.generateExpectLocations(s.loc, events) + expected := s.generateExpectedLocations(s.loc, events) c.Assert(expected[5].SetGTID(s.currGSet.Origin()), IsNil) @@ -281,7 +292,7 @@ func (s *testLocationSuite) TestDDLUpdateLocationsPos(c *C) { events = append(events[:2], events[4:]...) // now we have 3 events, test about their 4 locations - expected := s.generateExpectLocations(loc, events) + expected := s.generateExpectedLocations(loc, events) s.checkOneTxnEvents(c, events, expected) } @@ -305,7 +316,7 @@ func (s *testLocationSuite) TestDMLQueryUpdateLocationsGTID(c *C) { // we have 7 events c.Assert(events, HasLen, 7) - expected := s.generateExpectLocations(s.loc, events) + expected := s.generateExpectedLocations(s.loc, events) c.Assert(expected[7].SetGTID(s.currGSet.Origin()), IsNil) @@ -334,7 +345,7 @@ func (s *testLocationSuite) TestRotateEvent(c *C) { nextLoc := s.loc nextLoc.Position.Name = s.nextBinlogFile - expected := s.generateExpectLocations(nextLoc, events) + expected := s.generateExpectedLocations(nextLoc, events) // reset events of first binlog file expected[0].Position.Name = s.binlogFile diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 9f005e7e993..6ba6cf5e487 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1794,10 +1794,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { startTime := time.Now() e, err = s.getEvent(tctx, currentLocation) - s.tctx.L().Debug("location refactor", - zap.Stringer("current", currentLocation), - zap.Stringer("start", startLocation), - zap.Stringer("last", lastLocation)) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 1 { @@ -3684,8 +3680,6 @@ func (s *Syncer) getEvent(tctx *tcontext.Context, startLocation binlog.Location) e, err := s.streamerController.GetEvent(tctx) if err == nil { s.locations.update(e) - // TODO: observe below log in integration test - s.tctx.L().Debug("location refactor", zap.Stringer("locations", s.locations)) } return e, err } From f69f1683c395264a85100b68709cb05d90722705 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Sun, 13 Feb 2022 22:53:38 +0800 Subject: [PATCH 54/72] docs/media(ticdc): update cdc architecture (#4536) close pingcap/tiflow#4564 --- docs/media/cdc_architecture.svg | 189 ++++++++++++++++++-------------- 1 file changed, 106 insertions(+), 83 deletions(-) diff --git a/docs/media/cdc_architecture.svg b/docs/media/cdc_architecture.svg index e5c2cdfda7e..6ba12810f93 100644 --- a/docs/media/cdc_architecture.svg +++ b/docs/media/cdc_architecture.svg @@ -1,84 +1,107 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From fc3b5ad53eaa51d67360a0efb66b1cd04e6fc01f Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Sun, 13 Feb 2022 23:59:37 +0800 Subject: [PATCH 55/72] etcd (ticdc): fix a data race in unit test (#4551) close pingcap/tiflow#4549 --- pkg/etcd/client_test.go | 33 +++++++++++++++++---------------- pkg/etcd/etcd_test.go | 5 ++--- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/pkg/etcd/client_test.go b/pkg/etcd/client_test.go index 42220d48e9d..7119f6c9cad 100644 --- a/pkg/etcd/client_test.go +++ b/pkg/etcd/client_test.go @@ -17,6 +17,7 @@ import ( "context" "io/ioutil" "os" + "sync/atomic" "testing" "time" @@ -51,23 +52,23 @@ func (m *mockClient) Txn(ctx context.Context) clientv3.Txn { type mockWatcher struct { clientv3.Watcher watchCh chan clientv3.WatchResponse - resetCount *int - requestCount *int + resetCount *int32 + requestCount *int32 rev *int64 } func (m mockWatcher) Watch(ctx context.Context, key string, opts ...clientv3.OpOption) clientv3.WatchChan { - *m.resetCount++ + atomic.AddInt32(m.resetCount, 1) op := &clientv3.Op{} for _, opt := range opts { opt(op) } - *m.rev = op.Rev() + atomic.StoreInt64(m.rev, op.Rev()) return m.watchCh } func (m mockWatcher) RequestProgress(ctx context.Context) error { - *m.requestCount++ + atomic.AddInt32(m.requestCount, 1) return nil } @@ -153,8 +154,8 @@ func TestDelegateLease(t *testing.T) { // test no data lost when WatchCh blocked func TestWatchChBlocked(t *testing.T) { cli := clientv3.NewCtxClient(context.TODO()) - resetCount := 0 - requestCount := 0 + resetCount := int32(0) + requestCount := int32(0) rev := int64(0) watchCh := make(chan clientv3.WatchResponse, 1) watcher := mockWatcher{watchCh: watchCh, resetCount: &resetCount, requestCount: &requestCount, rev: &rev} @@ -204,9 +205,9 @@ func TestWatchChBlocked(t *testing.T) { require.Equal(t, sentRes, receivedRes) // make sure watchCh has been reset since timeout - require.True(t, *watcher.resetCount > 1) + require.True(t, atomic.LoadInt32(watcher.resetCount) > 1) // make sure RequestProgress has been call since timeout - require.True(t, *watcher.requestCount > 1) + require.True(t, atomic.LoadInt32(watcher.requestCount) > 1) // make sure etcdRequestProgressDuration is less than etcdWatchChTimeoutDuration require.Less(t, etcdRequestProgressDuration, etcdWatchChTimeoutDuration) } @@ -214,8 +215,8 @@ func TestWatchChBlocked(t *testing.T) { // test no data lost when OutCh blocked func TestOutChBlocked(t *testing.T) { cli := clientv3.NewCtxClient(context.TODO()) - resetCount := 0 - requestCount := 0 + resetCount := int32(0) + requestCount := int32(0) rev := int64(0) watchCh := make(chan clientv3.WatchResponse, 1) watcher := mockWatcher{watchCh: watchCh, resetCount: &resetCount, requestCount: &requestCount, rev: &rev} @@ -266,8 +267,8 @@ func TestOutChBlocked(t *testing.T) { func TestRevisionNotFallBack(t *testing.T) { cli := clientv3.NewCtxClient(context.TODO()) - resetCount := 0 - requestCount := 0 + resetCount := int32(0) + requestCount := int32(0) rev := int64(0) watchCh := make(chan clientv3.WatchResponse, 1) watcher := mockWatcher{watchCh: watchCh, resetCount: &resetCount, requestCount: &requestCount, rev: &rev} @@ -301,11 +302,11 @@ func TestRevisionNotFallBack(t *testing.T) { // move time forward mockClock.Add(time.Second * 30) // make sure watchCh has been reset since timeout - require.True(t, *watcher.resetCount > 1) - // make suer revision in WatchWitchChan does not fall back + require.True(t, atomic.LoadInt32(watcher.resetCount) > 1) + // make sure revision in WatchWitchChan does not fall back // even if there has not any response been received from WatchCh // while WatchCh was reset - require.Equal(t, *watcher.rev, revision) + require.Equal(t, atomic.LoadInt64(watcher.rev), revision) } type mockTxn struct { diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index 1ccc678117d..36b604d44cf 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -29,14 +29,13 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/embed" "go.etcd.io/etcd/pkg/logutil" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" - - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" ) type Captures []*model.CaptureInfo From c3e8be9add2df6ac2f426b8ac35dd650f3dea9d4 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Mon, 14 Feb 2022 10:47:37 +0800 Subject: [PATCH 56/72] checker(dm): support concurrent check (#3975) close pingcap/tiflow#3974 --- dm/checker/check_test.go | 87 ++-- dm/checker/checker.go | 41 +- dm/pkg/checker/mysql_server.go | 9 +- dm/pkg/checker/privilege.go | 20 +- dm/pkg/checker/privilege_test.go | 25 +- dm/pkg/checker/table_structure.go | 523 ++++++++++++++----------- dm/pkg/checker/table_structure_test.go | 167 ++++---- dm/pkg/checker/utils.go | 20 +- dm/pkg/utils/db.go | 20 + dm/pkg/utils/db_test.go | 15 + 10 files changed, 527 insertions(+), 400 deletions(-) diff --git a/dm/checker/check_test.go b/dm/checker/check_test.go index a29f952ea15..45668d2354a 100644 --- a/dm/checker/check_test.go +++ b/dm/checker/check_test.go @@ -255,32 +255,33 @@ func (s *testCheckerSuite) TestTableSchemaChecking(c *tc.C) { } createTable1 := `CREATE TABLE %s ( - id int(11) DEFAULT NULL, - b int(11) DEFAULT NULL - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) DEFAULT NULL, + b int(11) DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` createTable2 := `CREATE TABLE %s ( - id int(11) DEFAULT NULL, - b int(11) DEFAULT NULL, - UNIQUE KEY id (id) - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) DEFAULT NULL, + b int(11) DEFAULT NULL, + UNIQUE KEY id (id) + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` mock := initMockDB(c) - mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) + mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb2))) - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) msg, err := CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(len(msg), tc.Equals, 0) c.Assert(err, tc.ErrorMatches, "(.|\n)*primary/unique key does not exist(.|\n)*") + c.Assert(len(msg), tc.Equals, 0) mock = initMockDB(c) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb1))) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb2))) - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) msg, err = CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(msg, tc.Equals, CheckTaskSuccess) c.Assert(err, tc.IsNil) + c.Assert(msg, tc.Equals, CheckTaskSuccess) } func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { @@ -299,29 +300,37 @@ func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { } createTable1 := `CREATE TABLE %s ( - id int(11) DEFAULT NULL, - b int(11) DEFAULT NULL - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) DEFAULT NULL, + b int(11) DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` createTable2 := `CREATE TABLE %s ( - id int(11) DEFAULT NULL, - c int(11) DEFAULT NULL - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) DEFAULT NULL, + c int(11) DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` mock := initMockDB(c) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) + mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb2))) msg, err := CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(len(msg), tc.Equals, 0) c.Assert(err, tc.ErrorMatches, "(.|\n)*different column definition(.|\n)*") + c.Assert(len(msg), tc.Equals, 0) mock = initMockDB(c) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) + mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb2))) msg, err = CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(msg, tc.Equals, CheckTaskSuccess) c.Assert(err, tc.IsNil) + c.Assert(msg, tc.Equals, CheckTaskSuccess) } func (s *testCheckerSuite) TestShardAutoIncrementIDChecking(c *tc.C) { @@ -340,36 +349,42 @@ func (s *testCheckerSuite) TestShardAutoIncrementIDChecking(c *tc.C) { } createTable1 := `CREATE TABLE %s ( - id int(11) NOT NULL AUTO_INCREMENT, - b int(11) DEFAULT NULL, - PRIMARY KEY (id), - UNIQUE KEY u_b(b) - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) NOT NULL AUTO_INCREMENT, + b int(11) DEFAULT NULL, + PRIMARY KEY (id), + UNIQUE KEY u_b(b) + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` createTable2 := `CREATE TABLE %s ( - id int(11) NOT NULL, - b int(11) DEFAULT NULL, - INDEX (id), - UNIQUE KEY u_b(b) - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + id int(11) NOT NULL, + b int(11) DEFAULT NULL, + INDEX (id), + UNIQUE KEY u_b(b) + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` mock := initMockDB(c) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) + mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb2))) msg, err := CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(len(msg), tc.Equals, 0) - c.Assert(err, tc.ErrorMatches, "(.|\n)*instance table .* of sharding .* have auto-increment key(.|\n)*") + c.Assert(msg, tc.Matches, "(.|\n)*sourceID table .* of sharding .* have auto-increment key(.|\n)*") + c.Assert(err, tc.IsNil) - mock = conn.InitMockDB(c) - mock.ExpectQuery("SHOW DATABASES").WillReturnRows(sqlmock.NewRows([]string{"DATABASE"}).AddRow(schema)) - mock.ExpectQuery("SHOW FULL TABLES").WillReturnRows(sqlmock.NewRows([]string{"Tables_in_" + schema, "Table_type"}).AddRow(tb1, "BASE TABLE").AddRow(tb2, "BASE TABLE")) + mock = initMockDB(c) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) + mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb1))) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb2))) msg, err = CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) - c.Assert(msg, tc.Equals, CheckTaskSuccess) c.Assert(err, tc.IsNil) + c.Assert(msg, tc.Equals, CheckTaskSuccess) } func (s *testCheckerSuite) TestSameTargetTableDetection(c *tc.C) { diff --git a/dm/checker/checker.go b/dm/checker/checker.go index 267aebf9687..0ae60fea15a 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -119,9 +119,11 @@ func (c *Checker) Init(ctx context.Context) (err error) { rollbackHolder.Add(fr.FuncRollback{Name: "close-DBs", Fn: c.closeDBs}) c.tctx = tcontext.NewContext(ctx, log.With(zap.String("unit", "task check"))) - // target name => source => schema => [tables] - sharding := make(map[string]map[string]map[string][]string) + // targetTableID => source => [tables] + sharding := make(map[string]map[string][]*filter.Table) shardingCounter := make(map[string]int) + // sourceID => []table + checkTablesMap := make(map[string][]*filter.Table) dbs := make(map[string]*sql.DB) columnMapping := make(map[string]*column.Mapping) _, checkingShardID := c.checkingItems[config.ShardAutoIncrementIDChecking] @@ -206,29 +208,22 @@ func (c *Checker) Init(ctx context.Context) (err error) { return err } - // checkTables map schema => {table1, table2, ...} - checkTables := make(map[string][]string) + var checkTables []*filter.Table checkSchemas := make(map[string]struct{}, len(mapping)) - for name, tables := range mapping { + for targetTableID, tables := range mapping { + checkTables = append(checkTables, tables...) + if _, ok := sharding[targetTableID]; !ok { + sharding[targetTableID] = make(map[string][]*filter.Table) + } + sharding[targetTableID][instance.cfg.SourceID] = append(sharding[targetTableID][instance.cfg.SourceID], tables...) + shardingCounter[targetTableID] += len(tables) for _, table := range tables { - checkTables[table.Schema] = append(checkTables[table.Schema], table.Name) if _, ok := checkSchemas[table.Schema]; !ok { checkSchemas[table.Schema] = struct{}{} } - if _, ok := sharding[name]; !ok { - sharding[name] = make(map[string]map[string][]string) - } - if _, ok := sharding[name][instance.cfg.SourceID]; !ok { - sharding[name][instance.cfg.SourceID] = make(map[string][]string) - } - if _, ok := sharding[name][instance.cfg.SourceID][table.Schema]; !ok { - sharding[name][instance.cfg.SourceID][table.Schema] = make([]string, 0, 1) - } - - sharding[name][instance.cfg.SourceID][table.Schema] = append(sharding[name][instance.cfg.SourceID][table.Schema], table.Name) - shardingCounter[name]++ } } + checkTablesMap[instance.cfg.SourceID] = checkTables dbs[instance.cfg.SourceID] = instance.sourceDB.DB if _, ok := c.checkingItems[config.DumpPrivilegeChecking]; ok { exportCfg := export.DefaultConfig() @@ -241,9 +236,11 @@ func (c *Checker) Init(ctx context.Context) (err error) { if c.onlineDDL != nil { c.checkList = append(c.checkList, checker.NewOnlineDDLChecker(instance.sourceDB.DB, checkSchemas, c.onlineDDL, bw)) } - if checkSchema { - c.checkList = append(c.checkList, checker.NewTablesChecker(instance.sourceDB.DB, instance.sourceDBinfo, checkTables)) - } + } + + dumpThreads := c.instances[0].cfg.MydumperConfig.Threads + if checkSchema { + c.checkList = append(c.checkList, checker.NewTablesChecker(dbs, checkTablesMap, dumpThreads)) } if checkingShard { @@ -252,7 +249,7 @@ func (c *Checker) Init(ctx context.Context) (err error) { continue } - c.checkList = append(c.checkList, checker.NewShardingTablesChecker(name, dbs, shardingSet, columnMapping, checkingShardID)) + c.checkList = append(c.checkList, checker.NewShardingTablesChecker(name, dbs, shardingSet, columnMapping, checkingShardID, dumpThreads)) } } diff --git a/dm/pkg/checker/mysql_server.go b/dm/pkg/checker/mysql_server.go index 8e3346d70c3..302baa4b73d 100644 --- a/dm/pkg/checker/mysql_server.go +++ b/dm/pkg/checker/mysql_server.go @@ -124,13 +124,12 @@ func (pc *MySQLServerIDChecker) Check(ctx context.Context) *Result { serverID, err := dbutil.ShowServerID(ctx, pc.db) if err != nil { - if utils.OriginError(err) == sql.ErrNoRows { - result.Errors = append(result.Errors, NewError("server_id not set")) - result.Instruction = "please set server_id in your database" - } else { + if utils.OriginError(err) != sql.ErrNoRows { markCheckError(result, err) + return result } - + result.Errors = append(result.Errors, NewError("server_id not set")) + result.Instruction = "please set server_id in your database" return result } diff --git a/dm/pkg/checker/privilege.go b/dm/pkg/checker/privilege.go index 2dab51ff1be..dda6075f178 100644 --- a/dm/pkg/checker/privilege.go +++ b/dm/pkg/checker/privilege.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" @@ -41,12 +42,12 @@ var privNeedGlobal = map[mysql.PrivilegeType]struct{}{ type SourceDumpPrivilegeChecker struct { db *sql.DB dbinfo *dbutil.DBConfig - checkTables map[string][]string // map schema => {table1, table2, ...} + checkTables []*filter.Table consistency string } // NewSourceDumpPrivilegeChecker returns a RealChecker. -func NewSourceDumpPrivilegeChecker(db *sql.DB, dbinfo *dbutil.DBConfig, checkTables map[string][]string, consistency string) RealChecker { +func NewSourceDumpPrivilegeChecker(db *sql.DB, dbinfo *dbutil.DBConfig, checkTables []*filter.Table, consistency string) RealChecker { return &SourceDumpPrivilegeChecker{ db: db, dbinfo: dbinfo, @@ -320,9 +321,8 @@ func verifyPrivileges(result *Result, grants []string, lackPriv map[mysql.Privil return NewError(privileges) } -// checkTables map schema => {table1, table2, ...}. // lackPriv map privilege => schema => table. -func genExpectPriv(privileges map[mysql.PrivilegeType]struct{}, checkTables map[string][]string) map[mysql.PrivilegeType]map[string]map[string]struct{} { +func genExpectPriv(privileges map[mysql.PrivilegeType]struct{}, checkTables []*filter.Table) map[mysql.PrivilegeType]map[string]map[string]struct{} { lackPriv := make(map[mysql.PrivilegeType]map[string]map[string]struct{}, len(privileges)) for p := range privileges { if _, ok := privNeedGlobal[p]; ok { @@ -330,13 +330,11 @@ func genExpectPriv(privileges map[mysql.PrivilegeType]struct{}, checkTables map[ continue } lackPriv[p] = make(map[string]map[string]struct{}, len(checkTables)) - for schema, tables := range checkTables { - if _, ok := lackPriv[p][schema]; !ok { - lackPriv[p][schema] = make(map[string]struct{}, len(tables)) - } - for _, table := range tables { - lackPriv[p][schema][table] = struct{}{} + for _, table := range checkTables { + if _, ok := lackPriv[p][table.Schema]; !ok { + lackPriv[p][table.Schema] = make(map[string]struct{}) } + lackPriv[p][table.Schema][table.Name] = struct{}{} } if p == mysql.SelectPriv { if _, ok := lackPriv[p]["INFORMATION_SCHEMA"]; !ok { @@ -353,7 +351,7 @@ func genReplicPriv(replicationPrivileges map[mysql.PrivilegeType]struct{}) map[m return genExpectPriv(replicationPrivileges, nil) } -func genDumpPriv(dumpPrivileges map[mysql.PrivilegeType]struct{}, checkTables map[string][]string) map[mysql.PrivilegeType]map[string]map[string]struct{} { +func genDumpPriv(dumpPrivileges map[mysql.PrivilegeType]struct{}, checkTables []*filter.Table) map[mysql.PrivilegeType]map[string]map[string]struct{} { // due to dump privilege checker need check db/table level privilege // so we need know the check tables return genExpectPriv(dumpPrivileges, checkTables) diff --git a/dm/pkg/checker/privilege_test.go b/dm/pkg/checker/privilege_test.go index e38da710c93..927a7c4e231 100644 --- a/dm/pkg/checker/privilege_test.go +++ b/dm/pkg/checker/privilege_test.go @@ -17,6 +17,7 @@ import ( "testing" tc "github.com/pingcap/check" + "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/mysql" ) @@ -31,7 +32,7 @@ type testCheckSuite struct{} func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { cases := []struct { grants []string - checkTables map[string][]string + checkTables []*filter.Table dumpState State errMatch string }{ @@ -58,8 +59,8 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { "GRANT EXECUTE ON FUNCTION db1.anomaly_score TO 'user1'@'domain-or-ip-address1'", }, dumpState: StateFailure, - checkTables: map[string][]string{ - "db1": {"anomaly_score"}, + checkTables: []*filter.Table{ + {Schema: "db1", Name: "anomaly_score"}, }, // `db1`.`anomaly_score`; `INFORMATION_SCHEMA` // can't guarantee the order @@ -126,8 +127,8 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { "GRANT ALL PRIVILEGES ON `medz`.* TO `zhangsan`@`10.8.1.9` WITH GRANT OPTION", }, dumpState: StateFailure, - checkTables: map[string][]string{ - "medz": {"medz"}, + checkTables: []*filter.Table{ + {Schema: "medz", Name: "medz"}, }, errMatch: "lack of RELOAD privilege; ", }, @@ -137,8 +138,8 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { "GRANT ALL PRIVILEGES ON `INFORMATION_SCHEMA`.* TO `zhangsan`@`10.8.1.9` WITH GRANT OPTION", }, dumpState: StateFailure, - checkTables: map[string][]string{ - "medz": {"medz"}, + checkTables: []*filter.Table{ + {Schema: "medz", Name: "medz"}, }, errMatch: "lack of RELOAD privilege; ", }, @@ -149,8 +150,8 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { "GRANT SELECT ON `INFORMATION_SCHEMA`.* TO 'user'@'%'", }, dumpState: StateFailure, - checkTables: map[string][]string{ - "lance": {"t"}, + checkTables: []*filter.Table{ + {Schema: "lance", Name: "t"}, }, errMatch: "lack of Select privilege: {`lance`.`t`}; ", }, @@ -162,8 +163,8 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { "GRANT `r1`@`%`,`r2`@`%` TO `u1`@`localhost`", }, dumpState: StateSuccess, - checkTables: map[string][]string{ - "db1": {"t"}, + checkTables: []*filter.Table{ + {Schema: "db1", Name: "t"}, }, }, { @@ -195,7 +196,7 @@ func (t *testCheckSuite) TestVerifyDumpPrivileges(c *tc.C) { func (t *testCheckSuite) TestVerifyReplicationPrivileges(c *tc.C) { cases := []struct { grants []string - checkTables map[string][]string + checkTables []*filter.Table replicationState State errMatch string }{ diff --git a/dm/pkg/checker/table_structure.go b/dm/pkg/checker/table_structure.go index 48b7044078a..e6eb1363ff2 100644 --- a/dm/pkg/checker/table_structure.go +++ b/dm/pkg/checker/table_structure.go @@ -18,23 +18,41 @@ import ( "context" "database/sql" "fmt" + "math" "strings" + "sync" + "time" + + "go.uber.org/zap" + "golang.org/x/sync/errgroup" "github.com/pingcap/errors" column "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) -// AutoIncrementKeyChecking is an identification for auto increment key checking. -const AutoIncrementKeyChecking = "auto-increment key checking" +const ( + // AutoIncrementKeyChecking is an identification for auto increment key checking. + AutoIncrementKeyChecking = "auto-increment key checking" +) + +type checkItem struct { + table *filter.Table + sourceID string +} // hold information of incompatibility option. type incompatibilityOption struct { state State + tableID string instruction string errMessage string } @@ -58,19 +76,28 @@ func (o *incompatibilityOption) String() string { // In generally we need to check definitions of columns, constraints and table options. // Because of the early TiDB engineering design, we did not have a complete list of check items, which are all based on experience now. type TablesChecker struct { - db *sql.DB - dbinfo *dbutil.DBConfig - tables map[string][]string // schema => []table; if []table is empty, query tables from db - + dbs map[string]*sql.DB + tableMap map[string][]*filter.Table // sourceID => {[table1, table2, ...]} + reMu sync.Mutex + inCh chan *checkItem + optCh chan *incompatibilityOption + wg sync.WaitGroup + dumpThreads int } // NewTablesChecker returns a RealChecker. -func NewTablesChecker(db *sql.DB, dbinfo *dbutil.DBConfig, tables map[string][]string) RealChecker { - return &TablesChecker{ - db: db, - dbinfo: dbinfo, - tables: tables, +func NewTablesChecker(dbs map[string]*sql.DB, tableMap map[string][]*filter.Table, dumpThreads int) RealChecker { + if dumpThreads == 0 { + dumpThreads = 1 + } + c := &TablesChecker{ + dbs: dbs, + tableMap: tableMap, + dumpThreads: dumpThreads, } + c.inCh = make(chan *checkItem, dumpThreads) + c.optCh = make(chan *incompatibilityOption, dumpThreads) + return c } // Check implements RealChecker interface. @@ -79,111 +106,119 @@ func (c *TablesChecker) Check(ctx context.Context) *Result { Name: c.Name(), Desc: "check compatibility of table structure", State: StateSuccess, - Extra: fmt.Sprintf("address of db instance - %s:%d", c.dbinfo.Host, c.dbinfo.Port), } - var ( - err error - options = make(map[string][]*incompatibilityOption) - statements = make(map[string]string) - ) - for schema, tables := range c.tables { - if len(tables) == 0 { - tables, err = dbutil.GetTables(ctx, c.db, schema) - if err != nil { - markCheckError(r, err) - return r - } - } - - for _, table := range tables { - tableName := dbutil.TableName(schema, table) - statement, err := dbutil.GetCreateTableSQL(ctx, c.db, schema, table) - if err != nil { - // continue if table was deleted when checking - if isMySQLError(err, mysql.ErrNoSuchTable) { - continue - } - markCheckError(r, err) - return r - } + startTime := time.Now() + concurrency, err := getConcurrency(ctx, c.tableMap, c.dbs, c.dumpThreads) + if err != nil { + markCheckError(r, err) + return r + } + eg, checkCtx := errgroup.WithContext(ctx) + for i := 0; i < concurrency; i++ { + eg.Go(func() error { + return c.checkTable(checkCtx) + }) + } - opts := c.checkCreateSQL(ctx, statement) - if len(opts) > 0 { - options[tableName] = opts - statements[tableName] = statement - } - } + dispatchTableItem(checkCtx, c.tableMap, c.inCh) + c.wg.Add(1) + go c.handleOpts(ctx, r) + if err := eg.Wait(); err != nil { + c.reMu.Lock() + markCheckError(r, err) + c.reMu.Unlock() } + close(c.optCh) + c.wg.Wait() - for name, opts := range options { - if len(opts) == 0 { - continue - } - tableMsg := "table " + name + " " + log.L().Logger.Info("check table structure over", zap.Duration("spend time", time.Since(startTime))) + return r +} - for _, option := range opts { - switch option.state { +// Name implements RealChecker interface. +func (c *TablesChecker) Name() string { + return "table structure compatibility check" +} + +func (c *TablesChecker) handleOpts(ctx context.Context, r *Result) { + defer c.wg.Done() + for { + select { + case <-ctx.Done(): + return + case opt, ok := <-c.optCh: + if !ok { + return + } + tableMsg := "table " + opt.tableID + " " + c.reMu.Lock() + switch opt.state { case StateWarning: if r.State != StateFailure { r.State = StateWarning } - e := NewError(tableMsg + option.errMessage) + e := NewError(tableMsg + opt.errMessage) e.Severity = StateWarning - e.Instruction = option.instruction + e.Instruction = opt.instruction r.Errors = append(r.Errors, e) case StateFailure: r.State = StateFailure - e := NewError(tableMsg + option.errMessage) - e.Instruction = option.instruction + e := NewError(tableMsg + opt.errMessage) + e.Instruction = opt.instruction r.Errors = append(r.Errors, e) } + c.reMu.Unlock() } } - - return r } -// Name implements RealChecker interface. -func (c *TablesChecker) Name() string { - return "table structure compatibility check" -} - -func (c *TablesChecker) checkCreateSQL(ctx context.Context, statement string) []*incompatibilityOption { - parser2, err := dbutil.GetParserForDB(ctx, c.db) - if err != nil { - return []*incompatibilityOption{ - { - state: StateFailure, - errMessage: err.Error(), - }, - } - } +func (c *TablesChecker) checkTable(ctx context.Context) error { + var ( + sourceID string + p *parser.Parser + err error + ) + for { + select { + case <-ctx.Done(): + return context.Canceled + case checkItem, ok := <-c.inCh: + if !ok { + return nil + } + table := checkItem.table + if len(sourceID) == 0 || sourceID != checkItem.sourceID { + sourceID = checkItem.sourceID + p, err = dbutil.GetParserForDB(ctx, c.dbs[sourceID]) + if err != nil { + return err + } + } + db := c.dbs[checkItem.sourceID] + statement, err := dbutil.GetCreateTableSQL(ctx, db, table.Schema, table.Name) + if err != nil { + // continue if table was deleted when checking + if isMySQLError(err, mysql.ErrNoSuchTable) { + continue + } + return err + } - stmt, err := parser2.ParseOneStmt(statement, "", "") - if err != nil { - return []*incompatibilityOption{ - { - state: StateFailure, - errMessage: err.Error(), - }, + ctStmt, err := getCreateTableStmt(p, statement) + if err != nil { + return err + } + opts := c.checkAST(ctStmt) + for _, opt := range opts { + opt.tableID = table.String() + c.optCh <- opt + } } } - // Analyze ast - return c.checkAST(stmt) } -func (c *TablesChecker) checkAST(stmt ast.StmtNode) []*incompatibilityOption { - st, ok := stmt.(*ast.CreateTableStmt) - if !ok { - return []*incompatibilityOption{ - { - state: StateFailure, - errMessage: fmt.Sprintf("Expect CreateTableStmt but got %T", stmt), - }, - } - } - +func (c *TablesChecker) checkAST(st *ast.CreateTableStmt) []*incompatibilityOption { var options []*incompatibilityOption // check columns @@ -271,23 +306,35 @@ func (c *TablesChecker) checkTableOption(opt *ast.TableOption) *incompatibilityO // * check whether they have same column list // * check whether they have auto_increment key. type ShardingTablesChecker struct { - name string - + targetTableID string dbs map[string]*sql.DB - tables map[string]map[string][]string // instance => {schema: [table1, table2, ...]} + tableMap map[string][]*filter.Table // sourceID => {[table1, table2, ...]} mapping map[string]*column.Mapping checkAutoIncrementPrimaryKey bool + firstCreateTableStmtNode *ast.CreateTableStmt + firstTable *filter.Table + firstSourceID string + inCh chan *checkItem + reMu sync.Mutex + dumpThreads int } // NewShardingTablesChecker returns a RealChecker. -func NewShardingTablesChecker(name string, dbs map[string]*sql.DB, tables map[string]map[string][]string, mapping map[string]*column.Mapping, checkAutoIncrementPrimaryKey bool) RealChecker { - return &ShardingTablesChecker{ - name: name, +func NewShardingTablesChecker(targetTableID string, dbs map[string]*sql.DB, tableMap map[string][]*filter.Table, mapping map[string]*column.Mapping, checkAutoIncrementPrimaryKey bool, dumpThreads int) RealChecker { + if dumpThreads == 0 { + dumpThreads = 1 + } + c := &ShardingTablesChecker{ + targetTableID: targetTableID, dbs: dbs, - tables: tables, + tableMap: tableMap, mapping: mapping, checkAutoIncrementPrimaryKey: checkAutoIncrementPrimaryKey, + dumpThreads: dumpThreads, } + c.inCh = make(chan *checkItem, dumpThreads) + + return c } // Check implements RealChecker interface. @@ -296,168 +343,136 @@ func (c *ShardingTablesChecker) Check(ctx context.Context) *Result { Name: c.Name(), Desc: "check consistency of sharding table structures", State: StateSuccess, - Extra: fmt.Sprintf("sharding %s", c.name), + Extra: fmt.Sprintf("sharding %s,", c.targetTableID), } - var ( - stmtNode *ast.CreateTableStmt - firstTable string - firstInstance string - ) - - for instance, schemas := range c.tables { - db, ok := c.dbs[instance] - if !ok { - markCheckError(r, errors.NotFoundf("client for instance %s", instance)) - return r - } - - parser2, err := dbutil.GetParserForDB(ctx, db) - if err != nil { - markCheckError(r, err) - r.Extra = fmt.Sprintf("fail to get parser for instance %s on sharding %s", instance, c.name) - return r - } - - for schema, tables := range schemas { - for _, table := range tables { - statement, err := dbutil.GetCreateTableSQL(ctx, db, schema, table) - if err != nil { - // continue if table was deleted when checking - if isMySQLError(err, mysql.ErrNoSuchTable) { - continue - } - markCheckError(r, err) - r.Extra = fmt.Sprintf("instance %s on sharding %s", instance, c.name) - return r - } + startTime := time.Now() + log.L().Logger.Info("start to check sharding tables") - info, err := dbutil.GetTableInfoBySQL(statement, parser2) - if err != nil { - markCheckError(r, err) - r.Extra = fmt.Sprintf("instance %s on sharding %s", instance, c.name) - return r - } - stmt, err := parser2.ParseOneStmt(statement, "", "") - if err != nil { - markCheckError(r, errors.Annotatef(err, "statement %s", statement)) - r.Extra = fmt.Sprintf("instance %s on sharding %s", instance, c.name) - return r - } + for sourceID, tables := range c.tableMap { + c.firstSourceID = sourceID + c.firstTable = tables[0] + break + } + db, ok := c.dbs[c.firstSourceID] + if !ok { + markCheckError(r, errors.NotFoundf("client for sourceID %s", c.firstSourceID)) + return r + } - ctStmt, ok := stmt.(*ast.CreateTableStmt) - if !ok { - markCheckError(r, errors.Errorf("Expect CreateTableStmt but got %T", stmt)) - r.Extra = fmt.Sprintf("instance %s on sharding %s", instance, c.name) - return r - } + p, err := dbutil.GetParserForDB(ctx, db) + if err != nil { + r.Extra = fmt.Sprintf("fail to get parser for sourceID %s on sharding %s", c.firstSourceID, c.targetTableID) + markCheckError(r, err) + return r + } + r.Extra = fmt.Sprintf("sourceID %s on sharding %s", c.firstSourceID, c.targetTableID) + statement, err := dbutil.GetCreateTableSQL(ctx, db, c.firstTable.Schema, c.firstTable.Name) + if err != nil { + markCheckError(r, err) + return r + } - if c.checkAutoIncrementPrimaryKey { - passed := c.checkAutoIncrementKey(instance, schema, table, ctStmt, info, r) - if !passed { - return r - } - } + c.firstCreateTableStmtNode, err = getCreateTableStmt(p, statement) + if err != nil { + markCheckError(r, err) + return r + } - if stmtNode == nil { - stmtNode = ctStmt - firstTable = dbutil.TableName(schema, table) - firstInstance = instance - continue - } + concurrency, err := getConcurrency(ctx, c.tableMap, c.dbs, c.dumpThreads) + if err != nil { + markCheckError(r, err) + return r + } + eg, checkCtx := errgroup.WithContext(ctx) + for i := 0; i < concurrency; i++ { + eg.Go(func() error { + return c.checkShardingTable(checkCtx, r) + }) + } - checkErr := c.checkConsistency(stmtNode, ctStmt, firstTable, dbutil.TableName(schema, table), firstInstance, instance) - if checkErr != nil { - r.State = StateFailure - r.Errors = append(r.Errors, checkErr) - r.Extra = fmt.Sprintf("error on sharding %s", c.name) - r.Instruction = "please set same table structure for sharding tables" - return r - } - } - } + dispatchTableItem(checkCtx, c.tableMap, c.inCh) + if err := eg.Wait(); err != nil { + markCheckError(r, err) } + log.L().Logger.Info("check sharding table structure over", zap.Duration("spend time", time.Since(startTime))) return r } -func (c *ShardingTablesChecker) checkAutoIncrementKey(instance, schema, table string, ctStmt *ast.CreateTableStmt, info *model.TableInfo, r *Result) bool { - autoIncrementKeys := c.findAutoIncrementKey(ctStmt, info) - for columnName, isBigInt := range autoIncrementKeys { - hasMatchedRule := false - if cm, ok1 := c.mapping[instance]; ok1 { - ruleSet := cm.Selector.Match(schema, table) - for _, rule := range ruleSet { - r, ok2 := rule.(*column.Rule) - if !ok2 { +func (c *ShardingTablesChecker) checkShardingTable(ctx context.Context, r *Result) error { + var ( + sourceID string + p *parser.Parser + err error + ) + for { + select { + case <-ctx.Done(): + return nil + case checkItem, ok := <-c.inCh: + if !ok { + return nil + } + table := checkItem.table + if len(sourceID) == 0 || sourceID != checkItem.sourceID { + sourceID = checkItem.sourceID + p, err = dbutil.GetParserForDB(ctx, c.dbs[sourceID]) + if err != nil { + c.reMu.Lock() + r.Extra = fmt.Sprintf("fail to get parser for sourceID %s on sharding %s", sourceID, c.targetTableID) + c.reMu.Unlock() + return err + } + } + + statement, err := dbutil.GetCreateTableSQL(ctx, c.dbs[sourceID], table.Schema, table.Name) + if err != nil { + // continue if table was deleted when checking + if isMySQLError(err, mysql.ErrNoSuchTable) { continue } + return err + } - if r.Expression == column.PartitionID && r.TargetColumn == columnName { - hasMatchedRule = true - break + ctStmt, err := getCreateTableStmt(p, statement) + if err != nil { + return err + } + + if has := c.hasAutoIncrementKey(ctStmt); has { + c.reMu.Lock() + if r.State == StateSuccess { + r.State = StateWarning + r.Errors = append(r.Errors, NewError("sourceID %s table %v of sharding %s have auto-increment key, please make sure them don't conflict in target table!", sourceID, table, c.targetTableID)) + r.Instruction = "If happen conflict, please handle it by yourself. You can refer to https://docs.pingcap.com/tidb-data-migration/stable/shard-merge-best-practices/#handle-conflicts-between-primary-keys-or-unique-indexes-across-multiple-sharded-tables" + r.Extra = AutoIncrementKeyChecking } + c.reMu.Unlock() } - if hasMatchedRule && !isBigInt { + if checkErr := c.checkConsistency(ctStmt, table.String(), sourceID); checkErr != nil { + c.reMu.Lock() r.State = StateFailure - r.Errors = append(r.Errors, NewError("instance %s table `%s`.`%s` of sharding %s have auto-increment key %s and column mapping, but type of %s should be bigint", instance, schema, table, c.name, columnName, columnName)) - r.Instruction = "please set auto-increment key type to bigint" - r.Extra = AutoIncrementKeyChecking - return false + r.Errors = append(r.Errors, checkErr) + r.Extra = fmt.Sprintf("error on sharding %s", c.targetTableID) + r.Instruction = "please set same table structure for sharding tables" + c.reMu.Unlock() + return nil } } - - if !hasMatchedRule { - r.State = StateFailure - r.Errors = append(r.Errors, NewError("instance %s table `%s`.`%s` of sharding %s have auto-increment key %s and column mapping, but type of %s should be bigint", instance, schema, table, c.name, columnName, columnName)) - r.Instruction = "please handle it by yourself" - r.Extra = AutoIncrementKeyChecking - return false - } } - - return true } -func (c *ShardingTablesChecker) findAutoIncrementKey(stmt *ast.CreateTableStmt, info *model.TableInfo) map[string]bool { - autoIncrementKeys := make(map[string]bool) - autoIncrementCols := make(map[string]bool) - +func (c *ShardingTablesChecker) hasAutoIncrementKey(stmt *ast.CreateTableStmt) bool { for _, col := range stmt.Cols { - var ( - hasAutoIncrementOpt bool - isUnique bool - ) for _, opt := range col.Options { - switch opt.Tp { - case ast.ColumnOptionAutoIncrement: - hasAutoIncrementOpt = true - case ast.ColumnOptionPrimaryKey, ast.ColumnOptionUniqKey: - isUnique = true - } - } - - if hasAutoIncrementOpt { - if isUnique { - autoIncrementKeys[col.Name.Name.O] = col.Tp.Tp == mysql.TypeLonglong - } else { - autoIncrementCols[col.Name.Name.O] = col.Tp.Tp == mysql.TypeLonglong - } - } - } - - for _, index := range info.Indices { - if index.Unique || index.Primary { - if len(index.Columns) == 1 { - if isBigInt, ok := autoIncrementCols[index.Columns[0].Name.O]; ok { - autoIncrementKeys[index.Columns[0].Name.O] = isBigInt - } + if opt.Tp == ast.ColumnOptionAutoIncrement { + return true } } } - - return autoIncrementKeys + return false } type briefColumnInfo struct { @@ -490,8 +505,8 @@ func (cs briefColumnInfos) String() string { return strings.Join(colStrs, "\n") } -func (c *ShardingTablesChecker) checkConsistency(self, other *ast.CreateTableStmt, selfTable, otherTable, selfInstance, otherInstance string) *Error { - selfColumnList := getBriefColumnList(self) +func (c *ShardingTablesChecker) checkConsistency(other *ast.CreateTableStmt, otherTable, othersourceID string) *Error { + selfColumnList := getBriefColumnList(c.firstCreateTableStmtNode) otherColumnList := getBriefColumnList(other) if len(selfColumnList) != len(otherColumnList) { @@ -503,16 +518,16 @@ func (c *ShardingTablesChecker) checkConsistency(self, other *ast.CreateTableStm } return ret } - e.Self = fmt.Sprintf("instance %s table %s columns %v", selfInstance, selfTable, getColumnNames(selfColumnList)) - e.Other = fmt.Sprintf("instance %s table %s columns %v", otherInstance, otherTable, getColumnNames(otherColumnList)) + e.Self = fmt.Sprintf("sourceID %s table %v columns %v", c.firstSourceID, c.firstTable, getColumnNames(selfColumnList)) + e.Other = fmt.Sprintf("sourceID %s table %s columns %v", othersourceID, otherTable, getColumnNames(otherColumnList)) return e } for i := range selfColumnList { if *selfColumnList[i] != *otherColumnList[i] { e := NewError("different column definition") - e.Self = fmt.Sprintf("instance %s table %s column %s", selfInstance, selfTable, selfColumnList[i]) - e.Other = fmt.Sprintf("instance %s table %s column %s", otherInstance, otherTable, otherColumnList[i]) + e.Self = fmt.Sprintf("sourceID %s table %s column %s", c.firstSourceID, c.firstTable, selfColumnList[i]) + e.Other = fmt.Sprintf("sourceID %s table %s column %s", othersourceID, otherTable, otherColumnList[i]) return e } } @@ -546,5 +561,35 @@ func getBriefColumnList(stmt *ast.CreateTableStmt) briefColumnInfos { // Name implements Checker interface. func (c *ShardingTablesChecker) Name() string { - return fmt.Sprintf("sharding table %s consistency checking", c.name) + return fmt.Sprintf("sharding table %s consistency checking", c.targetTableID) +} + +func dispatchTableItem(ctx context.Context, tableMap map[string][]*filter.Table, inCh chan *checkItem) { + for sourceID, tables := range tableMap { + for _, table := range tables { + select { + case <-ctx.Done(): + log.L().Logger.Warn("ctx canceled before input tables completely") + return + case inCh <- &checkItem{table, sourceID}: + } + } + } + close(inCh) +} + +func getConcurrency(ctx context.Context, tableMap map[string][]*filter.Table, dbs map[string]*sql.DB, dumpThreads int) (int, error) { + concurrency := dumpThreads + for sourceID := range tableMap { + db, ok := dbs[sourceID] + if !ok { + return 0, errors.NotFoundf("client for sourceID %s", sourceID) + } + maxConnections, err := utils.GetMaxConnections(ctx, db) + if err != nil { + return 0, err + } + concurrency = int(math.Min(float64(concurrency), float64((maxConnections+1)/2))) + } + return concurrency, nil } diff --git a/dm/pkg/checker/table_structure_test.go b/dm/pkg/checker/table_structure_test.go index cdd48cad524..bb67ab1a6a8 100644 --- a/dm/pkg/checker/table_structure_test.go +++ b/dm/pkg/checker/table_structure_test.go @@ -16,12 +16,10 @@ package checker import ( "context" "database/sql" - "encoding/json" - "fmt" "github.com/DATA-DOG/go-sqlmock" tc "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" ) func (t *testCheckSuite) TestShardingTablesChecker(c *tc.C) { @@ -29,23 +27,8 @@ func (t *testCheckSuite) TestShardingTablesChecker(c *tc.C) { c.Assert(err, tc.IsNil) ctx := context.Background() - printJSON := func(r *Result) { - rawResult, _ := json.MarshalIndent(r, "", "\t") - fmt.Println("\n" + string(rawResult)) - } - // 1. test a success check - - sqlModeRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ANSI_QUOTES") - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) - createTableRow := sqlmock.NewRows([]string{"Table", "Create Table"}). - AddRow("test-table-1", `CREATE TABLE "test-table-1" ( - "c" int(11) NOT NULL, - PRIMARY KEY ("c") -) ENGINE=InnoDB DEFAULT CHARSET=latin1`) - mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) - + mock = initShardingMock(mock) createTableRow2 := sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test-table-2", `CREATE TABLE "test-table-2" ( "c" int(11) NOT NULL, @@ -55,26 +38,28 @@ func (t *testCheckSuite) TestShardingTablesChecker(c *tc.C) { checker := NewShardingTablesChecker("test-name", map[string]*sql.DB{"test-source": db}, - map[string]map[string][]string{"test-source": {"test-db": []string{"test-table-1", "test-table-2"}}}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + {Schema: "test-db", Name: "test-table-2"}, + }}, nil, - false) + false, + 1) result := checker.Check(ctx) - c.Assert(result.State, tc.Equals, StateSuccess) c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) // 2. check different column number - - sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ANSI_QUOTES") - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) - createTableRow = sqlmock.NewRows([]string{"Table", "Create Table"}). - AddRow("test-table-1", `CREATE TABLE "test-table-1" ( - "c" int(11) NOT NULL, - PRIMARY KEY ("c") -) ENGINE=InnoDB DEFAULT CHARSET=latin1`) - mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + checker = NewShardingTablesChecker("test-name", + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + {Schema: "test-db", Name: "test-table-2"}, + }}, + nil, + false, + 1) + mock = initShardingMock(mock) createTableRow2 = sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test-table-2", `CREATE TABLE "test-table-2" ( "c" int(11) NOT NULL, @@ -87,19 +72,18 @@ func (t *testCheckSuite) TestShardingTablesChecker(c *tc.C) { c.Assert(result.State, tc.Equals, StateFailure) c.Assert(result.Errors, tc.HasLen, 1) c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) // 3. check different column def - - sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ANSI_QUOTES") - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) - createTableRow = sqlmock.NewRows([]string{"Table", "Create Table"}). - AddRow("test-table-1", `CREATE TABLE "test-table-1" ( - "c" int(11) NOT NULL, - PRIMARY KEY ("c") -) ENGINE=InnoDB DEFAULT CHARSET=latin1`) - mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + checker = NewShardingTablesChecker("test-name", + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + {Schema: "test-db", Name: "test-table-2"}, + }}, + nil, + false, + 1) + mock = initShardingMock(mock) createTableRow2 = sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test-table-2", `CREATE TABLE "test-table-2" ( "c" varchar(20) NOT NULL, @@ -111,7 +95,6 @@ func (t *testCheckSuite) TestShardingTablesChecker(c *tc.C) { c.Assert(result.State, tc.Equals, StateFailure) c.Assert(result.Errors, tc.HasLen, 1) c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) } func (t *testCheckSuite) TestTablesChecker(c *tc.C) { @@ -119,67 +102,103 @@ func (t *testCheckSuite) TestTablesChecker(c *tc.C) { c.Assert(err, tc.IsNil) ctx := context.Background() - printJSON := func(r *Result) { - rawResult, _ := json.MarshalIndent(r, "", "\t") - fmt.Println("\n" + string(rawResult)) - } - // 1. test a success check - - createTableRow := sqlmock.NewRows([]string{"Table", "Create Table"}). - AddRow("test-table-1", `CREATE TABLE "test-table-1" ( - "c" int(11) NOT NULL, - PRIMARY KEY ("c") -) ENGINE=InnoDB DEFAULT CHARSET=latin1`) - mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + maxConnectionsRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("max_connections", "2") + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(maxConnectionsRow) sqlModeRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). AddRow("sql_mode", "ANSI_QUOTES") mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + createTableRow := sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test-table-1", `CREATE TABLE "test-table-1" ( + "c" int(11) NOT NULL, + PRIMARY KEY ("c") + ) ENGINE=InnoDB DEFAULT CHARSET=latin1`) + mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) - checker := NewTablesChecker(db, - &dbutil.DBConfig{}, - map[string][]string{"test-db": {"test-table-1"}}) + checker := NewTablesChecker( + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + }}, + 1) result := checker.Check(ctx) - c.Assert(result.State, tc.Equals, StateSuccess) c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) // 2. check many errors - + maxConnectionsRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("max_connections", "2") + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(maxConnectionsRow) + sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("sql_mode", "ANSI_QUOTES") + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) createTableRow = sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test-table-1", `CREATE TABLE "test-table-1" ( "c" int(11) NOT NULL, CONSTRAINT "fk" FOREIGN KEY ("c") REFERENCES "t" ("c") ) ENGINE=InnoDB DEFAULT CHARSET=latin1`) mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) - sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ANSI_QUOTES") - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + checker = NewTablesChecker( + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + }}, + 1) result = checker.Check(ctx) - c.Assert(result.State, tc.Equals, StateFailure) c.Assert(result.Errors, tc.HasLen, 2) // no PK/UK + has FK c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) // 3. unsupported charset - + maxConnectionsRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("max_connections", "2") + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(maxConnectionsRow) + sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("sql_mode", "ANSI_QUOTES") + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) createTableRow = sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test-table-1", `CREATE TABLE "test-table-1" ( "c" int(11) NOT NULL, PRIMARY KEY ("c") ) ENGINE=InnoDB DEFAULT CHARSET=ucs2`) mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) - sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "ANSI_QUOTES") - mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + checker = NewTablesChecker( + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + {Schema: "test-db", Name: "test-table-1"}, + }}, + 1) result = checker.Check(ctx) - c.Assert(result.State, tc.Equals, StateFailure) c.Assert(result.Errors, tc.HasLen, 1) c.Assert(mock.ExpectationsWereMet(), tc.IsNil) - printJSON(result) +} + +func initShardingMock(mock sqlmock.Sqlmock) sqlmock.Sqlmock { + sqlModeRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("sql_mode", "ANSI_QUOTES") + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + createTableRow := sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test-table-1", `CREATE TABLE "test-table-1" ( +"c" int(11) NOT NULL, +PRIMARY KEY ("c") +) ENGINE=InnoDB DEFAULT CHARSET=latin1`) + mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + + maxConnecionsRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("max_connections", "2") + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(maxConnecionsRow) + sqlModeRow = sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("sql_mode", "ANSI_QUOTES") + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + createTableRow = sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test-table-1", `CREATE TABLE "test-table-1" ( +"c" int(11) NOT NULL, +PRIMARY KEY ("c") +) ENGINE=InnoDB DEFAULT CHARSET=latin1`) + mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + return mock } diff --git a/dm/pkg/checker/utils.go b/dm/pkg/checker/utils.go index 7d53500feee..689e5627d63 100644 --- a/dm/pkg/checker/utils.go +++ b/dm/pkg/checker/utils.go @@ -23,6 +23,8 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/utils" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" ) // MySQLVersion represents MySQL version number. @@ -136,7 +138,10 @@ func markCheckError(result *Result, err error) { } else { state = StateFailure } - result.State = state + // `StateWarning` can't cover `StateFailure`. + if result.State != StateFailure { + result.State = state + } result.Errors = append(result.Errors, &Error{Severity: state, ShortErr: err.Error()}) } } @@ -146,3 +151,16 @@ func isMySQLError(err error, code uint16) bool { e, ok := err.(*mysql.MySQLError) return ok && e.Number == code } + +func getCreateTableStmt(p *parser.Parser, statement string) (*ast.CreateTableStmt, error) { + stmt, err := p.ParseOneStmt(statement, "", "") + if err != nil { + return nil, errors.Annotatef(err, "statement %s", statement) + } + + ctStmt, ok := stmt.(*ast.CreateTableStmt) + if !ok { + return nil, errors.Errorf("Expect CreateTableStmt but got %T", stmt) + } + return ctStmt, nil +} diff --git a/dm/pkg/utils/db.go b/dm/pkg/utils/db.go index f8761c201f2..89bbbe1bc31 100644 --- a/dm/pkg/utils/db.go +++ b/dm/pkg/utils/db.go @@ -634,3 +634,23 @@ func GetTableCreateSQL(ctx context.Context, conn *sql.Conn, tableID string) (sql } return createStr, nil } + +// GetMaxConnections gets max_connections for sql.DB which is suitable for session variable max_connections. +func GetMaxConnections(ctx context.Context, db *sql.DB) (int, error) { + c, err := db.Conn(ctx) + if err != nil { + return 0, err + } + defer c.Close() + return GetMaxConnectionsForConn(ctx, c) +} + +// GetMaxConnectionsForConn gets max_connections for sql.Conn which is suitable for session variable max_connections. +func GetMaxConnectionsForConn(ctx context.Context, conn *sql.Conn) (int, error) { + maxConnectionsStr, err := GetSessionVariable(ctx, conn, "max_connections") + if err != nil { + return 0, err + } + maxConnections, err := strconv.ParseUint(maxConnectionsStr, 10, 32) + return int(maxConnections), err +} diff --git a/dm/pkg/utils/db_test.go b/dm/pkg/utils/db_test.go index 3687f1eaf82..38d345ac044 100644 --- a/dm/pkg/utils/db_test.go +++ b/dm/pkg/utils/db_test.go @@ -458,3 +458,18 @@ func (t *testDBSuite) TestAddGSetWithPurged(c *C) { c.Assert(originSet, DeepEquals, tc.originGSet) } } + +func (t *testDBSuite) TestGetMaxConnections(c *C) { + ctx, cancel := context.WithTimeout(context.Background(), DefaultDBTimeout) + defer cancel() + + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + + rows := mock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "151") + mock.ExpectQuery(`SHOW VARIABLES LIKE 'max_connections'`).WillReturnRows(rows) + maxConnections, err := GetMaxConnections(ctx, db) + c.Assert(err, IsNil) + c.Assert(maxConnections, Equals, 151) + c.Assert(mock.ExpectationsWereMet(), IsNil) +} From 52f0ade6cbe55e4493c8f7c0dc891b97219366a1 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 14 Feb 2022 15:25:38 +0800 Subject: [PATCH 57/72] validation(dm): manage validator life cycle (#4479) close pingcap/tiflow#4459 --- dm/_utils/terror_gen/errors_release.txt | 2 + dm/dm/common/common.go | 5 +- dm/dm/config/subtask.go | 4 + dm/dm/config/subtask_test.go | 10 + dm/dm/config/task.go | 61 ++++- dm/dm/config/task_converters.go | 32 ++- dm/dm/config/task_test.go | 15 +- dm/dm/master/scheduler/scheduler.go | 50 +++- dm/dm/master/scheduler/scheduler_test.go | 26 ++ dm/dm/pb/dmworker.pb.go | 299 +++++++++++++---------- dm/dm/proto/dmworker.proto | 8 +- dm/dm/worker/server_test.go | 3 +- dm/dm/worker/source_worker.go | 192 +++++++++++++-- dm/dm/worker/source_worker_test.go | 140 ++++++++++- dm/dm/worker/subtask.go | 89 ++++++- dm/dm/worker/subtask_holder.go | 2 +- dm/dm/worker/subtask_test.go | 16 +- dm/dm/worker/task_checker_test.go | 9 +- dm/dm/worker/utils.go | 37 +++ dm/dm/worker/utils_test.go | 63 +++++ dm/errors.toml | 12 + dm/pkg/ha/ops.go | 29 ++- dm/pkg/ha/ops_test.go | 38 ++- dm/pkg/ha/source.go | 4 +- dm/pkg/ha/stage.go | 88 +++++-- dm/pkg/ha/stage_test.go | 9 +- dm/pkg/ha/subtask.go | 22 ++ dm/pkg/ha/subtask_test.go | 6 +- dm/pkg/terror/error_list.go | 4 + dm/syncer/data_validator.go | 200 +++++++++++++++ dm/syncer/streamer_controller.go | 2 +- dm/syncer/syncer.go | 74 +----- dm/syncer/util.go | 57 +++++ dm/tests/dmctl_basic/conf/get_task.yaml | 5 + dm/tests/import_v10x/conf/task.yaml | 5 + 35 files changed, 1307 insertions(+), 311 deletions(-) create mode 100644 dm/dm/worker/utils.go create mode 100644 dm/dm/worker/utils_test.go create mode 100644 dm/syncer/data_validator.go diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 1d732811c0b..fda955674ea 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -184,6 +184,8 @@ ErrOpenAPITaskConfigNotExist,[code=20051:class=config:scope=internal:level=low], ErrConfigCollationCompatibleNotSupport,[code=20052:class=config:scope=internal:level=medium], "Message: collation compatible %s not supported, Workaround: Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." ErrConfigInvalidLoadMode,[code=20053:class=config:scope=internal:level=medium], "Message: invalid load mode '%s', Workaround: Please choose a valid value in ['sql', 'loader']" ErrConfigInvalidDuplicateResolution,[code=20054:class=config:scope=internal:level=medium], "Message: invalid load on-duplicate '%s', Workaround: Please choose a valid value in ['replace', 'error', 'ignore']" +ErrConfigValidationMode,[code=20055:class=config:scope=internal:level=high], "Message: invalid validation mode, Workaround: Please check `validation-mode` config in task configuration file." +ErrContinuousValidatorCfgNotFound,[code=20056:class=config:scope=internal:level=medium], "Message: mysql-instance(%d)'s continuous validator config %s not exist, Workaround: Please check the `continuous-validator-config-name` config in task configuration file." ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index ab4cecfd67a..ecce1e8f4e8 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -60,6 +60,9 @@ var ( // StageSubTaskKeyAdapter is used to store the running stage of the subtask. // k/v: Encode(source-id, task-name) -> the running stage of the subtask. StageSubTaskKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/stage/subtask/") + // StageValidatorKeyAdapter is used to store the running stage of the validator. + // k/v: Encode(source-id, task-name) -> the running stage of the validator. + StageValidatorKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/stage/validator/") // ShardDDLPessimismInfoKeyAdapter is used to store shard DDL info in pessimistic model. // k/v: Encode(task-name, source-id) -> shard DDL info. @@ -112,7 +115,7 @@ func keyAdapterKeysLen(s KeyAdapter) int { WorkerKeepAliveKeyAdapter, StageRelayKeyAdapter, UpstreamLastBoundWorkerKeyAdapter, UpstreamRelayWorkerKeyAdapter, OpenAPITaskTemplateKeyAdapter: return 1 - case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, + case UpstreamSubTaskKeyAdapter, StageSubTaskKeyAdapter, StageValidatorKeyAdapter, ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter, TaskCliArgsKeyAdapter: return 2 diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index 4c63b5d2f33..3bfc2813a5d 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -240,6 +240,7 @@ type SubTaskConfig struct { MydumperConfig // Mydumper configuration LoaderConfig // Loader configuration SyncerConfig // Syncer configuration + ValidatorCfg ValidatorConfig // compatible with standalone dm unit LogLevel string `toml:"log-level" json:"log-level"` @@ -442,6 +443,9 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { if err := c.LoaderConfig.adjust(); err != nil { return err } + if err := c.ValidatorCfg.adjust(); err != nil { + return err + } // TODO: check every member // TODO: since we checked here, we could remove other terror like ErrSyncerUnitGenBAList diff --git a/dm/dm/config/subtask_test.go b/dm/dm/config/subtask_test.go index 37f4ca64eaa..46e68d16e47 100644 --- a/dm/dm/config/subtask_test.go +++ b/dm/dm/config/subtask_test.go @@ -18,6 +18,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" + + "github.com/pingcap/tiflow/dm/pkg/terror" ) func (t *testConfig) TestSubTask(c *C) { @@ -69,6 +71,14 @@ func (t *testConfig) TestSubTask(c *C) { err = cfg.Adjust(true) c.Assert(err, IsNil) + + cfg.ValidatorCfg = ValidatorConfig{Mode: ValidationFast} + err = cfg.Adjust(true) + c.Assert(err, IsNil) + + cfg.ValidatorCfg = ValidatorConfig{Mode: "invalid-mode"} + err = cfg.Adjust(true) + c.Assert(terror.ErrConfigValidationMode.Equal(err), IsTrue) } func (t *testConfig) TestSubTaskAdjustFail(c *C) { diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 0ce58d97d55..0a1601aa153 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -56,6 +56,12 @@ const ( StrictCollationCompatible = "strict" ) +const ( + ValidationNone = "none" + ValidationFast = "fast" + ValidationFull = "full" +) + // default config item values. var ( // TaskConfig. @@ -136,6 +142,9 @@ type MySQLInstance struct { Syncer *SyncerConfig `yaml:"syncer"` // SyncerThread is alias for WorkerCount in SyncerConfig, and its priority is higher than WorkerCount SyncerThread int `yaml:"syncer-thread"` + + ContinuousValidatorConfigName string `yaml:"continuous-validator-config-name"` + ContinuousValidator ValidatorConfig `yaml:"-"` } // VerifyAndAdjust does verification on configs, and adjust some configs. @@ -327,6 +336,26 @@ func (m *SyncerConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { return nil } +type ValidatorConfig struct { + Mode string `yaml:"mode" toml:"mode" json:"mode"` +} + +func (v *ValidatorConfig) adjust() error { + if v.Mode == "" { + v.Mode = ValidationNone + } + if v.Mode != ValidationNone && v.Mode != ValidationFast && v.Mode != ValidationFull { + return terror.ErrConfigValidationMode + } + return nil +} + +func defaultValidatorConfig() ValidatorConfig { + return ValidatorConfig{ + Mode: ValidationNone, + } +} + // TaskConfig is the configuration for Task. type TaskConfig struct { *flag.FlagSet `yaml:"-" toml:"-" json:"-"` @@ -377,9 +406,10 @@ type TaskConfig struct { BWList map[string]*filter.Rules `yaml:"black-white-list" toml:"black-white-list" json:"black-white-list"` BAList map[string]*filter.Rules `yaml:"block-allow-list" toml:"block-allow-list" json:"block-allow-list"` - Mydumpers map[string]*MydumperConfig `yaml:"mydumpers" toml:"mydumpers" json:"mydumpers"` - Loaders map[string]*LoaderConfig `yaml:"loaders" toml:"loaders" json:"loaders"` - Syncers map[string]*SyncerConfig `yaml:"syncers" toml:"syncers" json:"syncers"` + Mydumpers map[string]*MydumperConfig `yaml:"mydumpers" toml:"mydumpers" json:"mydumpers"` + Loaders map[string]*LoaderConfig `yaml:"loaders" toml:"loaders" json:"loaders"` + Syncers map[string]*SyncerConfig `yaml:"syncers" toml:"syncers" json:"syncers"` + Validators map[string]*ValidatorConfig `yaml:"validators" toml:"validators" json:"validators"` CleanDumpFile bool `yaml:"clean-dump-file" toml:"clean-dump-file" json:"clean-dump-file"` // deprecated @@ -413,6 +443,7 @@ func NewTaskConfig() *TaskConfig { Mydumpers: make(map[string]*MydumperConfig), Loaders: make(map[string]*LoaderConfig), Syncers: make(map[string]*SyncerConfig), + Validators: make(map[string]*ValidatorConfig), CleanDumpFile: true, CollationCompatible: defaultCollationCompatible, } @@ -470,7 +501,7 @@ func (c *TaskConfig) RawDecode(data string) error { } // find unused items in config. -var configRefPrefixes = []string{"RouteRules", "FilterRules", "ColumnMappingRules", "Mydumper", "Loader", "Syncer", "ExprFilter"} +var configRefPrefixes = []string{"RouteRules", "FilterRules", "ColumnMappingRules", "Mydumper", "Loader", "Syncer", "ExprFilter", "Validator"} const ( routeRulesIdx = iota @@ -480,6 +511,7 @@ const ( loaderIdx syncerIdx exprFilterIdx + validatorIdx ) // adjust adjusts and verifies config. @@ -562,6 +594,12 @@ func (c *TaskConfig) adjust() error { } } + for _, validatorCfg := range c.Validators { + if err := validatorCfg.adjust(); err != nil { + return err + } + } + instanceIDs := make(map[string]int) // source-id -> instance-index globalConfigReferCount := map[string]int{} duplicateErrorStrings := make([]string, 0) @@ -684,6 +722,16 @@ func (c *TaskConfig) adjust() error { inst.Syncer.WorkerCount = inst.SyncerThread } + inst.ContinuousValidator = defaultValidatorConfig() + if inst.ContinuousValidatorConfigName != "" { + rule, ok := c.Validators[inst.ContinuousValidatorConfigName] + if !ok { + return terror.ErrContinuousValidatorCfgNotFound.Generate(i, inst.ContinuousValidatorConfigName) + } + globalConfigReferCount[configRefPrefixes[validatorIdx]+inst.ContinuousValidatorConfigName]++ + inst.ContinuousValidator = *rule + } + // for backward compatible, set global config `ansi-quotes: true` if any syncer is true if inst.Syncer.EnableANSIQuotes { log.L().Warn("DM could discover proper ANSI_QUOTES, `enable-ansi-quotes` is no longer take effect") @@ -755,6 +803,11 @@ func (c *TaskConfig) adjust() error { unusedConfigs = append(unusedConfigs, exprFilter) } } + for key := range c.Validators { + if globalConfigReferCount[configRefPrefixes[validatorIdx]+key] == 0 { + unusedConfigs = append(unusedConfigs, key) + } + } if len(unusedConfigs) != 0 { sort.Strings(unusedConfigs) diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 99c3c8ef7a9..a5ab3610dea 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -93,6 +93,7 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]DBConfig) ([]* cfg.MydumperConfig = *inst.Mydumper cfg.LoaderConfig = *inst.Loader cfg.SyncerConfig = *inst.Syncer + cfg.ValidatorCfg = inst.ContinuousValidator cfg.CleanDumpFile = c.CleanDumpFile @@ -205,6 +206,7 @@ func OpenAPITaskToSubTaskConfigs(task *openapi.Task, toDBCfg *DBConfig, sourceCf subTaskCfg.SyncerConfig.Batch = *incrCfg.ReplBatch } } + subTaskCfg.ValidatorCfg = defaultValidatorConfig() // set route,blockAllowList,filter config doCnt := len(tableMigrateRuleMap[sourceCfg.SourceName]) doDBs := make([]string, doCnt) @@ -302,6 +304,7 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { c.Syncers = make(map[string]*SyncerConfig) c.ExprFilter = make(map[string]*ExpressionFilter) c.Experimental = stCfg0.Experimental + c.Validators = make(map[string]*ValidatorConfig) baListMap := make(map[string]string, len(stCfgs)) routeMap := make(map[string]string, len(stCfgs)) @@ -311,8 +314,9 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { syncMap := make(map[string]string, len(stCfgs)) cmMap := make(map[string]string, len(stCfgs)) exprFilterMap := make(map[string]string, len(stCfgs)) - var baListIdx, routeIdx, filterIdx, dumpIdx, loadIdx, syncIdx, cmIdx, efIdx int - var baListName, routeName, filterName, dumpName, loadName, syncName, cmName, efName string + validatorMap := make(map[string]string, len(stCfgs)) + var baListIdx, routeIdx, filterIdx, dumpIdx, loadIdx, syncIdx, validateIdx, cmIdx, efIdx int + var baListName, routeName, filterName, dumpName, loadName, syncName, validateName, cmName, efName string // NOTE: // - we choose to ref global configs for instances now. @@ -354,6 +358,9 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { c.ExprFilter[efName] = f } + validateName, validateIdx = getGenerateName(stCfg.ValidatorCfg, validateIdx, "validator", validatorMap) + c.Validators[validateName] = &stCfg.ValidatorCfg + cmNames := make([]string, 0, len(stCfg.ColumnMappingRules)) for _, rule := range stCfg.ColumnMappingRules { cmName, cmIdx = getGenerateName(rule, cmIdx, "cm", cmMap) @@ -362,16 +369,17 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { } c.MySQLInstances = append(c.MySQLInstances, &MySQLInstance{ - SourceID: stCfg.SourceID, - Meta: stCfg.Meta, - FilterRules: filterNames, - ColumnMappingRules: cmNames, - RouteRules: routeNames, - BAListName: baListName, - MydumperConfigName: dumpName, - LoaderConfigName: loadName, - SyncerConfigName: syncName, - ExpressionFilters: exprFilterNames, + SourceID: stCfg.SourceID, + Meta: stCfg.Meta, + FilterRules: filterNames, + ColumnMappingRules: cmNames, + RouteRules: routeNames, + BAListName: baListName, + MydumperConfigName: dumpName, + LoaderConfigName: loadName, + SyncerConfigName: syncName, + ExpressionFilters: exprFilterNames, + ContinuousValidatorConfigName: validateName, }) } if c.CollationCompatible == "" { diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index 71571d8c455..8a1de3f019f 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -623,6 +623,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { Table: "tbl", DeleteValueExpr: "state = 1", } + validatorCfg = ValidatorConfig{Mode: ValidationNone} source1DBCfg = DBConfig{ Host: "127.0.0.1", Port: 3306, @@ -705,6 +706,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { EnableGTID: true, SafeMode: true, }, + ValidatorCfg: validatorCfg, CleanDumpFile: true, EnableANSIQuotes: true, } @@ -723,6 +725,7 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { stCfg2.BAList = &baList2 stCfg2.RouteRules = []*router.TableRule{&routeRule4, &routeRule1, &routeRule2} stCfg2.ExprFilter = []*ExpressionFilter{&exprFilter1} + stCfg2.ValidatorCfg.Mode = ValidationFast cfg := SubTaskConfigsToTaskConfig(stCfg1, stCfg2) @@ -757,6 +760,8 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { SyncerConfigName: "sync-01", Syncer: nil, SyncerThread: 0, + + ContinuousValidatorConfigName: "validator-01", }, { SourceID: source2, @@ -776,6 +781,8 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { Syncer: nil, SyncerThread: 0, ExpressionFilters: []string{"expr-filter-01"}, + + ContinuousValidatorConfigName: "validator-02", }, }, OnlineDDL: onlineDDL, @@ -807,6 +814,12 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { ExprFilter: map[string]*ExpressionFilter{ "expr-filter-01": &exprFilter1, }, + Validators: map[string]*ValidatorConfig{ + "validator-01": &validatorCfg, + "validator-02": { + Mode: ValidationFast, + }, + }, CleanDumpFile: stCfg1.CleanDumpFile, } cfg2.Experimental.AsyncCheckpointFlush = true @@ -1031,7 +1044,7 @@ func (t *testConfig) TestTaskConfigForDowngrade(c *C) { // make sure all new field were added cfgReflect := reflect.Indirect(reflect.ValueOf(cfg)) cfgForDowngradeReflect := reflect.Indirect(reflect.ValueOf(cfgForDowngrade)) - c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+3) // without flag, collation_compatible and experimental + c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+4) // without flag, collation_compatible, experimental, validator // make sure all field were copied cfgForClone := &TaskConfigForDowngrade{} diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index c0784c57392..435b4d1b145 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -175,6 +175,17 @@ type Scheduler struct { // - stop-relay relayWorkers map[string]map[string]struct{} + // expectant validator stages, task name -> source ID -> stage. + // add: + // - on subtask start with validator mode not none + // - start validator manually + // - recover from etcd + // update + // - update stage by user request + // delete: + // - when subtask is removed by user request + expectValidatorStages sync.Map + // workers in load stage // task -> source -> worker loadTasks map[string]map[string]string @@ -897,6 +908,7 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro // 2. construct `Running` stages when adding. newCfgs := make([]config.SubTaskConfig, 0, len(cfgs)-len(existSources)) newStages := make([]ha.Stage, 0, cap(newCfgs)) + validatorStages := make([]ha.Stage, 0, cap(newCfgs)) unbounds := make([]string, 0) for _, cfg := range cfgs { if _, ok := existSourcesM[cfg.SourceID]; ok { @@ -904,6 +916,9 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro } newCfgs = append(newCfgs, cfg) newStages = append(newStages, ha.NewSubTaskStage(pb.Stage_Running, cfg.SourceID, cfg.Name)) + if cfg.ValidatorCfg.Mode != config.ValidationNone { + validatorStages = append(validatorStages, ha.NewValidatorStage(pb.Stage_Running, cfg.SourceID, cfg.Name)) + } if _, ok := s.bounds[cfg.SourceID]; !ok { unbounds = append(unbounds, cfg.SourceID) } @@ -915,7 +930,7 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro } // 4. put the configs and stages into etcd. - _, err := ha.PutSubTaskCfgStage(s.etcdCli, newCfgs, newStages) + _, err := ha.PutSubTaskCfgStage(s.etcdCli, newCfgs, newStages, validatorStages) if err != nil { return err } @@ -931,6 +946,11 @@ func (s *Scheduler) AddSubTasks(latched bool, cfgs ...config.SubTaskConfig) erro m := v.(map[string]ha.Stage) m[stage.Source] = stage } + for _, stage := range validatorStages { + v, _ := s.expectValidatorStages.LoadOrStore(stage.Task, map[string]ha.Stage{}) + m := v.(map[string]ha.Stage) + m[stage.Source] = stage + } return nil } @@ -958,11 +978,17 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { return terror.ErrSchedulerSubTaskOpTaskNotExist.Generate(task) } + var validatorStageM map[string]ha.Stage + if validatorStageV, ok := s.expectValidatorStages.Load(task); ok { + validatorStageM = validatorStageV.(map[string]ha.Stage) + } + var ( stagesM = stagesMapV.(map[string]ha.Stage) cfgsM = cfgsMapV.(map[string]config.SubTaskConfig) notExistSourcesM = make(map[string]struct{}) stages = make([]ha.Stage, 0, len(sources)) + validatorStages = make([]ha.Stage, 0, len(sources)) cfgs = make([]config.SubTaskConfig, 0, len(sources)) ) for _, source := range sources { @@ -971,6 +997,9 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { } else { stages = append(stages, stage) } + if stage, ok := validatorStageM[source]; ok { + validatorStages = append(validatorStages, stage) + } if cfg, ok := cfgsM[source]; ok { cfgs = append(cfgs, cfg) } @@ -982,7 +1011,7 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { } // 2. delete the configs and the stages. - _, err = ha.DeleteSubTaskCfgStage(s.etcdCli, cfgs, stages) + _, err = ha.DeleteSubTaskCfgStage(s.etcdCli, cfgs, stages, validatorStages) if err != nil { return err } @@ -1000,6 +1029,12 @@ func (s *Scheduler) RemoveSubTasks(task string, sources ...string) error { if len(stagesM) == 0 { s.expectSubTaskStages.Delete(task) } + for _, stage := range validatorStages { + delete(validatorStageM, stage.Source) + } + if len(validatorStageM) == 0 { + s.expectValidatorStages.Delete(task) + } return nil } @@ -1661,6 +1696,10 @@ func (s *Scheduler) recoverSubTasks() error { if err != nil { return err } + validatorStageMM, _, err := ha.GetAllValidatorStage(s.etcdCli) + if err != nil { + return err + } // recover in-memory data. for source, cfgM := range cfgMM { @@ -1677,6 +1716,13 @@ func (s *Scheduler) recoverSubTasks() error { m[source] = stage } } + for source, stageM := range validatorStageMM { + for task, stage := range stageM { + v, _ := s.expectValidatorStages.LoadOrStore(task, map[string]ha.Stage{}) + m := v.(map[string]ha.Stage) + m[source] = stage + } + } return nil } diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index e9eb1627e6d..0f775c816b4 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -131,6 +131,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { c.Assert(subtaskCfg21.Adjust(true), IsNil) subtaskCfg22 := subtaskCfg21 subtaskCfg22.SourceID = sourceID2 + subtaskCfg22.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationFast} c.Assert(subtaskCfg22.Adjust(true), IsNil) // not started scheduler can't do anything. @@ -425,6 +426,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskCfgExist(c, s, subtaskCfg22) t.subTaskStageMatch(c, s, taskName2, sourceID1, pb.Stage_Running) t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) + t.validatorStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) // CASE 4.4.2 fail to stop any task. @@ -471,6 +473,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskCfgNotExist(c, s, taskName2, sourceID2) t.subTaskStageMatch(c, s, taskName2, sourceID1, pb.Stage_InvalidStage) t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_InvalidStage) + t.validatorStageMatch(c, s, taskName2, sourceID2, pb.Stage_InvalidStage) rebuildScheduler(ctx) // CASE 4.7: remove source2. @@ -768,6 +771,29 @@ func (t *testScheduler) subTaskStageMatch(c *C, s *Scheduler, task, source strin } } +func (t *testScheduler) validatorStageMatch(c *C, s *Scheduler, task, source string, expectStage pb.Stage) { + stage := ha.NewValidatorStage(expectStage, source, task) + var m map[string]ha.Stage + if v, ok := s.expectValidatorStages.Load(task); ok { + m = v.(map[string]ha.Stage) + } + if expectStage == pb.Stage_InvalidStage { + _, ok := m[source] + c.Assert(ok, IsFalse) + } else { + stageDeepEqualExcludeRev(c, m[source], stage) + } + stageM, _, err := ha.GetValidatorStage(etcdTestCli, source, task, 0) + c.Assert(err, IsNil) + switch expectStage { + case pb.Stage_Running, pb.Stage_Stopped: + c.Assert(stageM, HasLen, 1) + stageDeepEqualExcludeRev(c, stageM[task], stage) + default: + c.Assert(stageM, HasLen, 0) + } +} + func (t *testScheduler) TestRestartScheduler(c *C) { defer clearTestInfoOperation(c) diff --git a/dm/dm/pb/dmworker.pb.go b/dm/dm/pb/dmworker.pb.go index fe9b587c7a4..f52701d0f71 100644 --- a/dm/dm/pb/dmworker.pb.go +++ b/dm/dm/pb/dmworker.pb.go @@ -307,6 +307,34 @@ func (ErrorOp) EnumDescriptor() ([]byte, []int) { return fileDescriptor_51a1b9e17fd67b10, []int{6} } +type ValidatorOp int32 + +const ( + ValidatorOp_InvalidValidatorOp ValidatorOp = 0 + ValidatorOp_StartValidator ValidatorOp = 1 + ValidatorOp_StopValidator ValidatorOp = 2 +) + +var ValidatorOp_name = map[int32]string{ + 0: "InvalidValidatorOp", + 1: "StartValidator", + 2: "StopValidator", +} + +var ValidatorOp_value = map[string]int32{ + "InvalidValidatorOp": 0, + "StartValidator": 1, + "StopValidator": 2, +} + +func (x ValidatorOp) String() string { + return proto.EnumName(ValidatorOp_name, int32(x)) +} + +func (ValidatorOp) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_51a1b9e17fd67b10, []int{7} +} + type QueryStatusRequest struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` } @@ -2567,6 +2595,7 @@ func init() { proto.RegisterEnum("pb.SchemaOp", SchemaOp_name, SchemaOp_value) proto.RegisterEnum("pb.V1MetaOp", V1MetaOp_name, V1MetaOp_value) proto.RegisterEnum("pb.ErrorOp", ErrorOp_name, ErrorOp_value) + proto.RegisterEnum("pb.ValidatorOp", ValidatorOp_name, ValidatorOp_value) proto.RegisterType((*QueryStatusRequest)(nil), "pb.QueryStatusRequest") proto.RegisterType((*CommonWorkerResponse)(nil), "pb.CommonWorkerResponse") proto.RegisterType((*QueryStatusResponse)(nil), "pb.QueryStatusResponse") @@ -2604,140 +2633,142 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2124 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x73, 0xdc, 0x4a, - 0x11, 0x5f, 0xad, 0x76, 0xd7, 0xbb, 0xbd, 0x6b, 0x47, 0x99, 0x24, 0x8f, 0xc5, 0x84, 0xc5, 0xa5, - 0xbc, 0x0a, 0xc6, 0x45, 0xb9, 0x5e, 0xcc, 0xa3, 0x1e, 0xf5, 0xaa, 0x80, 0x47, 0xec, 0x3c, 0x27, - 0xe0, 0xe0, 0x44, 0x76, 0xc2, 0x91, 0x92, 0xa5, 0xf1, 0x5a, 0x58, 0x2b, 0x29, 0x9a, 0x91, 0x5d, - 0x7b, 0xa0, 0xf8, 0x08, 0x70, 0xe1, 0x00, 0xc5, 0x95, 0xeb, 0x3b, 0xf2, 0x11, 0x80, 0x63, 0x8a, - 0x2a, 0xaa, 0x38, 0x52, 0xc9, 0xd7, 0xe0, 0x40, 0x75, 0xcf, 0x48, 0x1a, 0xd9, 0xbb, 0x09, 0x39, - 0x70, 0x53, 0xff, 0xba, 0xa7, 0xbb, 0xa7, 0xa7, 0xff, 0xcc, 0x08, 0xd6, 0xc2, 0xd9, 0x65, 0x9a, - 0x9f, 0xf3, 0x7c, 0x3b, 0xcb, 0x53, 0x99, 0xb2, 0x76, 0x76, 0xe2, 0x6e, 0x02, 0x7b, 0x5e, 0xf0, - 0x7c, 0x7e, 0x24, 0x7d, 0x59, 0x08, 0x8f, 0xbf, 0x2a, 0xb8, 0x90, 0x8c, 0x41, 0x27, 0xf1, 0x67, - 0x7c, 0x6c, 0x6d, 0x58, 0x9b, 0x03, 0x8f, 0xbe, 0xdd, 0x0c, 0x6e, 0xef, 0xa6, 0xb3, 0x59, 0x9a, - 0xfc, 0x82, 0x74, 0x78, 0x5c, 0x64, 0x69, 0x22, 0x38, 0xfb, 0x08, 0x7a, 0x39, 0x17, 0x45, 0x2c, - 0x49, 0xba, 0xef, 0x69, 0x8a, 0x39, 0x60, 0xcf, 0xc4, 0x74, 0xdc, 0x26, 0x15, 0xf8, 0x89, 0x92, - 0x22, 0x2d, 0xf2, 0x80, 0x8f, 0x6d, 0x02, 0x35, 0x85, 0xb8, 0xf2, 0x6b, 0xdc, 0x51, 0xb8, 0xa2, - 0xdc, 0xaf, 0x2c, 0xb8, 0xd5, 0x70, 0xee, 0x83, 0x2d, 0x7e, 0x0a, 0x23, 0x65, 0x43, 0x69, 0x20, - 0xbb, 0xc3, 0x1d, 0x67, 0x3b, 0x3b, 0xd9, 0x3e, 0x32, 0x70, 0xaf, 0x21, 0xc5, 0x3e, 0x83, 0x55, - 0x51, 0x9c, 0x1c, 0xfb, 0xe2, 0x5c, 0x2f, 0xeb, 0x6c, 0xd8, 0x9b, 0xc3, 0x9d, 0x9b, 0xb4, 0xcc, - 0x64, 0x78, 0x4d, 0x39, 0xf7, 0xcf, 0x16, 0x0c, 0x77, 0xcf, 0x78, 0xa0, 0x69, 0x74, 0x34, 0xf3, - 0x85, 0xe0, 0x61, 0xe9, 0xa8, 0xa2, 0xd8, 0x6d, 0xe8, 0xca, 0x54, 0xfa, 0x31, 0xb9, 0xda, 0xf5, - 0x14, 0xc1, 0x26, 0x00, 0xa2, 0x08, 0x02, 0x2e, 0xc4, 0x69, 0x11, 0x93, 0xab, 0x5d, 0xcf, 0x40, - 0x50, 0xdb, 0xa9, 0x1f, 0xc5, 0x3c, 0xa4, 0x30, 0x75, 0x3d, 0x4d, 0xb1, 0x31, 0xac, 0x5c, 0xfa, - 0x79, 0x12, 0x25, 0xd3, 0x71, 0x97, 0x18, 0x25, 0x89, 0x2b, 0x42, 0x2e, 0xfd, 0x28, 0x1e, 0xf7, - 0x36, 0xac, 0xcd, 0x91, 0xa7, 0x29, 0xf7, 0xb5, 0x05, 0xb0, 0x57, 0xcc, 0x32, 0xed, 0xe6, 0x06, - 0x0c, 0xc9, 0x83, 0x63, 0xff, 0x24, 0xe6, 0x82, 0x7c, 0xb5, 0x3d, 0x13, 0x62, 0x9b, 0x70, 0x23, - 0x48, 0x67, 0x59, 0xcc, 0x25, 0x0f, 0xb5, 0x14, 0xba, 0x6e, 0x79, 0x57, 0x61, 0xf6, 0x31, 0xac, - 0x9e, 0x46, 0x49, 0x24, 0xce, 0x78, 0xf8, 0x70, 0x2e, 0xb9, 0x0a, 0xb9, 0xe5, 0x35, 0x41, 0xe6, - 0xc2, 0xa8, 0x04, 0xbc, 0xf4, 0x52, 0xd0, 0x86, 0x2c, 0xaf, 0x81, 0xb1, 0xef, 0xc2, 0x4d, 0x2e, - 0x64, 0x34, 0xf3, 0x25, 0x3f, 0x46, 0x57, 0x48, 0xb0, 0x4b, 0x82, 0xd7, 0x19, 0xee, 0x5f, 0x2c, - 0x80, 0x83, 0xd4, 0x0f, 0xf5, 0x96, 0xae, 0xb9, 0xa1, 0x36, 0x75, 0xc5, 0x8d, 0x09, 0x00, 0xed, - 0x52, 0x89, 0xb4, 0x49, 0xc4, 0x40, 0xd8, 0x3a, 0xf4, 0xb3, 0x3c, 0x9d, 0xe6, 0x5c, 0x08, 0x9d, - 0xb2, 0x15, 0x8d, 0x6b, 0x67, 0x5c, 0xfa, 0x0f, 0xa3, 0x24, 0x4e, 0xa7, 0x3a, 0x71, 0x0d, 0x84, - 0xdd, 0x87, 0xb5, 0x9a, 0xda, 0x3f, 0x7e, 0xb2, 0x47, 0xbe, 0x0f, 0xbc, 0x2b, 0xa8, 0xfb, 0x7b, - 0x0b, 0x56, 0x8f, 0xce, 0xfc, 0x3c, 0x8c, 0x92, 0xe9, 0x7e, 0x9e, 0x16, 0x19, 0x9e, 0x9a, 0xf4, - 0xf3, 0x29, 0x97, 0xba, 0xfc, 0x34, 0x85, 0x45, 0xb9, 0xb7, 0x77, 0x80, 0x7e, 0xda, 0x58, 0x94, - 0xf8, 0xad, 0xf6, 0x99, 0x0b, 0x79, 0x90, 0x06, 0xbe, 0x8c, 0xd2, 0x44, 0xbb, 0xd9, 0x04, 0xa9, - 0xf0, 0xe6, 0x49, 0x40, 0x99, 0x63, 0x53, 0xe1, 0x11, 0x85, 0xfb, 0x2b, 0x12, 0xcd, 0xe9, 0x12, - 0xa7, 0xa2, 0xdd, 0x7f, 0xda, 0x00, 0x47, 0xf3, 0x24, 0xb8, 0x92, 0x23, 0x8f, 0x2e, 0x78, 0x22, - 0x9b, 0x39, 0xa2, 0x20, 0x54, 0xa6, 0x52, 0x26, 0x2b, 0x43, 0x59, 0xd1, 0xec, 0x2e, 0x0c, 0x72, - 0x1e, 0xf0, 0x44, 0x22, 0xd3, 0x26, 0x66, 0x0d, 0x60, 0x36, 0xcc, 0x7c, 0x21, 0x79, 0xde, 0x08, - 0x66, 0x03, 0x63, 0x5b, 0xe0, 0x98, 0xf4, 0xbe, 0x8c, 0x42, 0x1d, 0xd0, 0x6b, 0x38, 0xea, 0xa3, - 0x4d, 0x94, 0xfa, 0x7a, 0x4a, 0x9f, 0x89, 0xa1, 0x3e, 0x93, 0x26, 0x7d, 0x2b, 0x4a, 0xdf, 0x55, - 0x1c, 0xf5, 0x9d, 0xc4, 0x69, 0x70, 0x1e, 0x25, 0x53, 0x3a, 0x80, 0x3e, 0x85, 0xaa, 0x81, 0xb1, - 0x1f, 0x82, 0x53, 0x24, 0x39, 0x17, 0x69, 0x7c, 0xc1, 0x43, 0x3a, 0x47, 0x31, 0x1e, 0x18, 0x6d, - 0xc3, 0x3c, 0x61, 0xef, 0x9a, 0xa8, 0x71, 0x42, 0xa0, 0x3a, 0x85, 0x3e, 0xa1, 0x09, 0xc0, 0x09, - 0x39, 0x72, 0x3c, 0xcf, 0xf8, 0x78, 0xa8, 0xb2, 0xac, 0x46, 0xd8, 0x27, 0x70, 0x4b, 0xf0, 0x20, - 0x4d, 0x42, 0xf1, 0x90, 0x9f, 0x45, 0x49, 0xf8, 0x94, 0x62, 0x31, 0x1e, 0x51, 0x88, 0x17, 0xb1, - 0xdc, 0x3f, 0x59, 0x30, 0x32, 0x7b, 0x9f, 0xd1, 0x95, 0xad, 0x25, 0x5d, 0xb9, 0x6d, 0x76, 0x65, - 0xf6, 0x9d, 0xaa, 0xfb, 0xaa, 0x6e, 0x4a, 0xfb, 0x7b, 0x96, 0xa7, 0xd8, 0xa6, 0x3c, 0x62, 0x54, - 0x0d, 0xf9, 0x01, 0x0c, 0x73, 0x1e, 0xfb, 0xf3, 0xaa, 0x8d, 0xa2, 0xfc, 0x0d, 0x94, 0xf7, 0x6a, - 0xd8, 0x33, 0x65, 0xdc, 0xbf, 0xb5, 0x61, 0x68, 0x30, 0xaf, 0xe5, 0x86, 0xf5, 0x3f, 0xe6, 0x46, - 0x7b, 0x49, 0x6e, 0x6c, 0x94, 0x2e, 0x15, 0x27, 0x7b, 0x51, 0xae, 0xcb, 0xc5, 0x84, 0x2a, 0x89, - 0x46, 0x32, 0x9a, 0x10, 0x76, 0x43, 0x83, 0x34, 0x52, 0xf1, 0x2a, 0xcc, 0xb6, 0x81, 0x11, 0xb4, - 0xeb, 0xcb, 0xe0, 0xec, 0x45, 0xa6, 0x4f, 0xa7, 0x47, 0x47, 0xbc, 0x80, 0xc3, 0xbe, 0x05, 0x5d, - 0x21, 0xfd, 0x29, 0xa7, 0x54, 0x5c, 0xdb, 0x19, 0x50, 0xea, 0x20, 0xe0, 0x29, 0xdc, 0x08, 0x7e, - 0xff, 0x3d, 0xc1, 0x77, 0xff, 0xd3, 0x86, 0xd5, 0xc6, 0xb4, 0x5a, 0x34, 0xd5, 0x6b, 0x8b, 0xed, - 0x25, 0x16, 0x37, 0xa0, 0x53, 0x24, 0x91, 0x3a, 0xec, 0xb5, 0x9d, 0x11, 0xf2, 0x5f, 0x24, 0x91, - 0xc4, 0xec, 0xf3, 0x88, 0x63, 0xf8, 0xd4, 0x79, 0x5f, 0x42, 0x7c, 0x02, 0xb7, 0xea, 0xd4, 0xdf, - 0xdb, 0x3b, 0x38, 0x48, 0x83, 0xf3, 0xaa, 0x33, 0x2e, 0x62, 0x31, 0xa6, 0x66, 0x3a, 0x95, 0xf0, - 0xe3, 0x96, 0x9a, 0xea, 0xdf, 0x86, 0x6e, 0x80, 0x53, 0x96, 0xa2, 0xa4, 0x13, 0xca, 0x18, 0xbb, - 0x8f, 0x5b, 0x9e, 0xe2, 0xb3, 0x8f, 0xa1, 0x13, 0x16, 0xb3, 0x4c, 0xc7, 0x6a, 0x0d, 0xe5, 0xea, - 0xb1, 0xf7, 0xb8, 0xe5, 0x11, 0x17, 0xa5, 0xe2, 0xd4, 0x0f, 0xc7, 0x83, 0x5a, 0xaa, 0x9e, 0x24, - 0x28, 0x85, 0x5c, 0x94, 0xc2, 0x9a, 0xa4, 0xfa, 0xd4, 0x52, 0x75, 0x7b, 0x44, 0x29, 0xe4, 0x3e, - 0xec, 0x43, 0x4f, 0xa8, 0x44, 0xfe, 0x11, 0xdc, 0x6c, 0x44, 0xff, 0x20, 0x12, 0x14, 0x2a, 0xc5, - 0x1e, 0x5b, 0xcb, 0xae, 0x14, 0xe5, 0xfa, 0x09, 0x00, 0xed, 0xe9, 0x51, 0x9e, 0xa7, 0x79, 0x79, - 0xb5, 0xb1, 0xaa, 0xab, 0x8d, 0xfb, 0x4d, 0x18, 0xe0, 0x5e, 0xde, 0xc1, 0xc6, 0x4d, 0x2c, 0x63, - 0x67, 0x30, 0x22, 0xef, 0x9f, 0x1f, 0x2c, 0x91, 0x60, 0x3b, 0x70, 0x5b, 0xdd, 0x2f, 0x54, 0x3a, - 0x3f, 0x4b, 0x45, 0x44, 0x03, 0x46, 0x15, 0xd6, 0x42, 0x1e, 0x8e, 0x00, 0x8e, 0xea, 0x8e, 0x9e, - 0x1f, 0x94, 0xf3, 0xb2, 0xa4, 0xdd, 0xef, 0xc3, 0x00, 0x2d, 0x2a, 0x73, 0x9b, 0xd0, 0x23, 0x46, - 0x19, 0x07, 0xa7, 0x0a, 0xa7, 0x76, 0xc8, 0xd3, 0x7c, 0xf7, 0xb7, 0x16, 0x0c, 0x55, 0xbb, 0x52, - 0x2b, 0x3f, 0xb4, 0x5b, 0x6d, 0x34, 0x96, 0x97, 0xf5, 0x6e, 0x6a, 0xdc, 0x06, 0xa0, 0x86, 0xa3, - 0x04, 0x3a, 0xf5, 0xf1, 0xd6, 0xa8, 0x67, 0x48, 0xe0, 0xc1, 0xd4, 0xd4, 0x82, 0xd0, 0xfe, 0xa1, - 0x0d, 0x23, 0x7d, 0xa4, 0x4a, 0xe4, 0xff, 0x54, 0x76, 0xba, 0x32, 0x3a, 0x66, 0x65, 0xdc, 0x2f, - 0x2b, 0xa3, 0x5b, 0x6f, 0xa3, 0xce, 0xa2, 0xba, 0x30, 0xee, 0xe9, 0xc2, 0xe8, 0x91, 0xd8, 0x6a, - 0x59, 0x18, 0xa5, 0x94, 0xaa, 0x8b, 0x7b, 0xba, 0x2e, 0x56, 0x6a, 0xa1, 0x2a, 0xa5, 0xaa, 0xb2, - 0xb8, 0xa7, 0xcb, 0xa2, 0x5f, 0x0b, 0x55, 0xc7, 0x5c, 0x55, 0xc5, 0x0a, 0x74, 0xe9, 0x38, 0xdd, - 0xcf, 0xc1, 0x31, 0x43, 0x43, 0x35, 0x71, 0x5f, 0x33, 0x1b, 0xa9, 0x60, 0x08, 0x79, 0x7a, 0xed, - 0x2b, 0x58, 0x6d, 0x34, 0x15, 0x9c, 0x8d, 0x91, 0xd8, 0xf5, 0x93, 0x80, 0xc7, 0xd5, 0x0d, 0xdb, - 0x40, 0x8c, 0x24, 0x6b, 0xd7, 0x9a, 0xb5, 0x8a, 0x46, 0x92, 0x19, 0xf7, 0x64, 0xbb, 0x71, 0x4f, - 0xfe, 0x87, 0x05, 0x23, 0x73, 0x01, 0x5e, 0xb5, 0x1f, 0xe5, 0xf9, 0x6e, 0x1a, 0xaa, 0xd3, 0xec, - 0x7a, 0x25, 0x89, 0xa9, 0x8f, 0x9f, 0xb1, 0x2f, 0x84, 0xce, 0xc0, 0x8a, 0xd6, 0xbc, 0xa3, 0x20, - 0xcd, 0xca, 0x97, 0x4f, 0x45, 0x6b, 0xde, 0x01, 0xbf, 0xe0, 0xb1, 0x1e, 0x35, 0x15, 0x8d, 0xd6, - 0x9e, 0x72, 0x21, 0x30, 0x4d, 0x54, 0x87, 0x2c, 0x49, 0x5c, 0xe5, 0xf9, 0x97, 0xbb, 0x7e, 0x21, - 0xb8, 0xbe, 0xdd, 0x54, 0x34, 0x86, 0x05, 0x5f, 0x68, 0x7e, 0x9e, 0x16, 0x49, 0x79, 0xa7, 0x31, - 0x10, 0xf7, 0x12, 0x6e, 0x3e, 0x2b, 0xf2, 0x29, 0xa7, 0x24, 0x2e, 0x1f, 0x7c, 0xeb, 0xd0, 0x8f, - 0x12, 0x3f, 0x90, 0xd1, 0x05, 0xd7, 0x91, 0xac, 0x68, 0xcc, 0x5f, 0x19, 0xcd, 0xb8, 0xbe, 0xd4, - 0xd1, 0x37, 0xca, 0x9f, 0x46, 0x31, 0xa7, 0xbc, 0xd6, 0x5b, 0x2a, 0x69, 0x2a, 0x51, 0x35, 0x5d, - 0xf5, 0x73, 0x4e, 0x51, 0xee, 0x1f, 0xdb, 0xb0, 0x7e, 0x98, 0xf1, 0xdc, 0x97, 0x5c, 0x3d, 0x21, - 0x8f, 0x82, 0x33, 0x3e, 0xf3, 0x4b, 0x17, 0xee, 0x42, 0x3b, 0xcd, 0xc8, 0xb8, 0xce, 0x77, 0xc5, - 0x3e, 0xcc, 0xbc, 0x76, 0x9a, 0x91, 0x13, 0xbe, 0x38, 0xd7, 0xb1, 0xa5, 0xef, 0xa5, 0xef, 0xc9, - 0x75, 0xe8, 0x87, 0xbe, 0xf4, 0x4f, 0x7c, 0xc1, 0xcb, 0x98, 0x96, 0x34, 0x3d, 0xbd, 0xf0, 0xa5, - 0xa2, 0x23, 0xaa, 0x08, 0xd2, 0x44, 0xd6, 0x74, 0x34, 0x35, 0x85, 0xd2, 0xa7, 0x71, 0x21, 0xce, - 0x28, 0x8c, 0x7d, 0x4f, 0x11, 0xe8, 0x4b, 0x95, 0xf3, 0x7d, 0x95, 0xe2, 0x18, 0xf5, 0xd3, 0x3c, - 0x9d, 0xa9, 0xc6, 0x42, 0xa3, 0xa4, 0xef, 0x19, 0x48, 0xc9, 0x3f, 0x56, 0x17, 0x7b, 0xa8, 0xf9, - 0x0a, 0x71, 0x25, 0xac, 0xbe, 0x7c, 0xa0, 0xd3, 0xfe, 0x29, 0x97, 0x3e, 0x5b, 0x37, 0xc2, 0x01, - 0x18, 0x0e, 0xe4, 0xe8, 0x60, 0xbc, 0xb7, 0x7b, 0x94, 0x2d, 0xc7, 0x36, 0x5a, 0x4e, 0x19, 0xc1, - 0x0e, 0xa5, 0x38, 0x7d, 0xbb, 0x9f, 0xc2, 0x6d, 0x7d, 0x22, 0x2f, 0x1f, 0xa0, 0xd5, 0xa5, 0x67, - 0xa1, 0xd8, 0xca, 0xbc, 0xfb, 0x57, 0x0b, 0xee, 0x5c, 0x59, 0xf6, 0xc1, 0x2f, 0xf3, 0xcf, 0xa0, - 0x83, 0x0f, 0xa1, 0xb1, 0x4d, 0xa5, 0x79, 0x0f, 0x6d, 0x2c, 0x54, 0xb9, 0x8d, 0xc4, 0xa3, 0x44, - 0xe6, 0x73, 0x8f, 0x16, 0xac, 0xff, 0x14, 0x06, 0x15, 0x84, 0x7a, 0xcf, 0xf9, 0xbc, 0xec, 0xbe, - 0xe7, 0x7c, 0x8e, 0x77, 0x83, 0x0b, 0x3f, 0x2e, 0x54, 0x68, 0xf4, 0x80, 0x6d, 0x04, 0xd6, 0x53, - 0xfc, 0xcf, 0xdb, 0x3f, 0xb0, 0xdc, 0x5f, 0xc3, 0xf8, 0xb1, 0x9f, 0x84, 0xb1, 0xce, 0x47, 0xd5, - 0x14, 0x74, 0x08, 0xbe, 0x61, 0x84, 0x60, 0x88, 0x5a, 0x88, 0xfb, 0x8e, 0x6c, 0xbc, 0x0b, 0x83, - 0x93, 0x72, 0x1c, 0xea, 0xc0, 0xd7, 0x00, 0xe5, 0xcc, 0xab, 0x58, 0xe8, 0x07, 0x18, 0x7d, 0xbb, - 0x77, 0xe0, 0xd6, 0x3e, 0x97, 0xca, 0xf6, 0xee, 0xe9, 0x54, 0x5b, 0x76, 0x37, 0xe1, 0x76, 0x13, - 0xd6, 0xc1, 0x75, 0xc0, 0x0e, 0x4e, 0xab, 0x51, 0x13, 0x9c, 0x4e, 0xb7, 0x7e, 0x09, 0x3d, 0x95, - 0x15, 0x6c, 0x15, 0x06, 0x4f, 0x92, 0x0b, 0x3f, 0x8e, 0xc2, 0xc3, 0xcc, 0x69, 0xb1, 0x3e, 0x74, - 0x8e, 0x64, 0x9a, 0x39, 0x16, 0x1b, 0x40, 0xf7, 0x19, 0xb6, 0x05, 0xa7, 0xcd, 0x00, 0x7a, 0xd8, - 0x39, 0x67, 0xdc, 0xb1, 0x11, 0x3e, 0x92, 0x7e, 0x2e, 0x9d, 0x0e, 0xc2, 0x2f, 0xb2, 0xd0, 0x97, - 0xdc, 0xe9, 0xb2, 0x35, 0x80, 0x9f, 0x14, 0x32, 0xd5, 0x62, 0xbd, 0xad, 0xdf, 0x90, 0xd8, 0x14, - 0x6d, 0x8f, 0xb4, 0x7e, 0xa2, 0x9d, 0x16, 0x5b, 0x01, 0xfb, 0xe7, 0xfc, 0xd2, 0xb1, 0xd8, 0x10, - 0x56, 0xbc, 0x22, 0x49, 0xa2, 0x64, 0xaa, 0x6c, 0x90, 0xb9, 0xd0, 0xb1, 0x91, 0x81, 0x4e, 0x64, - 0x3c, 0x74, 0x3a, 0x6c, 0x04, 0xfd, 0x2f, 0xf5, 0xdb, 0xdb, 0xe9, 0x22, 0x0b, 0xc5, 0x70, 0x4d, - 0x0f, 0x59, 0x64, 0x10, 0xa9, 0x15, 0xa4, 0x68, 0x15, 0x52, 0xfd, 0xad, 0x43, 0xe8, 0x97, 0x63, - 0x8f, 0xdd, 0x80, 0xa1, 0xf6, 0x01, 0x21, 0xa7, 0x85, 0x9b, 0xa0, 0xe1, 0xe6, 0x58, 0xb8, 0x61, - 0x1c, 0x60, 0x4e, 0x1b, 0xbf, 0x70, 0x4a, 0x39, 0x36, 0x05, 0x61, 0x9e, 0x04, 0x4e, 0x07, 0x05, - 0xa9, 0xdb, 0x39, 0xe1, 0xd6, 0x53, 0x58, 0xa1, 0xcf, 0x43, 0x3c, 0xc4, 0x35, 0xad, 0x4f, 0x23, - 0x4e, 0x0b, 0xe3, 0x88, 0xd6, 0x95, 0xb4, 0x85, 0xf1, 0xa0, 0xed, 0x28, 0xba, 0x8d, 0x2e, 0xa8, - 0xd8, 0x28, 0xc0, 0xde, 0x4a, 0xa0, 0x5f, 0xb6, 0x29, 0x76, 0x0b, 0x6e, 0x94, 0x31, 0xd2, 0x90, - 0x52, 0xb8, 0xcf, 0xa5, 0x02, 0x1c, 0x8b, 0xf4, 0x57, 0x64, 0x1b, 0xc3, 0xea, 0xf1, 0x59, 0x7a, - 0xc1, 0x35, 0x62, 0xa3, 0x45, 0x9c, 0x8a, 0x9a, 0xee, 0xe0, 0x02, 0xa4, 0xe9, 0xef, 0x8a, 0xd3, - 0xdd, 0xfa, 0x02, 0xfa, 0x65, 0x29, 0x1a, 0xf6, 0x4a, 0xa8, 0xb2, 0xa7, 0x00, 0xc7, 0xaa, 0x0d, - 0x68, 0xa4, 0xbd, 0xf5, 0x92, 0x46, 0x18, 0x66, 0xb2, 0x11, 0x00, 0x8d, 0xe8, 0xcc, 0x39, 0x8f, - 0x32, 0x7d, 0xae, 0x3c, 0x8b, 0xfd, 0xa0, 0xca, 0x9d, 0x0b, 0x9e, 0x4b, 0xc7, 0xc6, 0xef, 0x27, - 0xc9, 0xaf, 0x78, 0x80, 0xc9, 0x83, 0xd1, 0x8e, 0x84, 0x74, 0xba, 0x3b, 0x5f, 0xd9, 0xd0, 0x53, - 0x39, 0xcb, 0xbe, 0x80, 0xa1, 0xf1, 0xdb, 0x8e, 0x7d, 0x84, 0xd5, 0x73, 0xfd, 0x27, 0xe3, 0xfa, - 0xd7, 0xae, 0xe1, 0x2a, 0xd1, 0xdd, 0x16, 0xfb, 0x31, 0x40, 0x3d, 0xa3, 0xd8, 0x1d, 0x1a, 0xdc, - 0x57, 0x67, 0xd6, 0xfa, 0x98, 0x6e, 0x37, 0x0b, 0x7e, 0x49, 0xba, 0x2d, 0xf6, 0x33, 0x58, 0xd5, - 0xed, 0x44, 0x45, 0x92, 0x4d, 0x8c, 0x0e, 0xb3, 0x60, 0xfa, 0xbc, 0x53, 0xd9, 0x97, 0x95, 0x32, - 0x15, 0x45, 0x36, 0x5e, 0xd0, 0xae, 0x94, 0x9a, 0xaf, 0x2f, 0x6d, 0x64, 0x6e, 0x8b, 0xed, 0xc3, - 0x50, 0xb5, 0x1b, 0x75, 0x99, 0xb8, 0x8b, 0xb2, 0xcb, 0xfa, 0xcf, 0x3b, 0x1d, 0xda, 0x85, 0x91, - 0xd9, 0x21, 0x18, 0x45, 0x72, 0x41, 0x2b, 0x51, 0x4a, 0x16, 0x35, 0x13, 0xb7, 0xf5, 0x70, 0xfc, - 0xf7, 0x37, 0x13, 0xeb, 0xf5, 0x9b, 0x89, 0xf5, 0xef, 0x37, 0x13, 0xeb, 0x77, 0x6f, 0x27, 0xad, - 0xd7, 0x6f, 0x27, 0xad, 0x7f, 0xbd, 0x9d, 0xb4, 0x4e, 0x7a, 0xf4, 0x7b, 0xf8, 0x7b, 0xff, 0x0d, - 0x00, 0x00, 0xff, 0xff, 0xa7, 0x6e, 0x71, 0xea, 0x30, 0x16, 0x00, 0x00, + // 2154 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1c, 0x4b, + 0x11, 0xdf, 0xd9, 0x7f, 0xde, 0xad, 0x5d, 0x3b, 0x93, 0x4e, 0x5e, 0x58, 0x4c, 0x58, 0xac, 0xc9, + 0x53, 0x30, 0x16, 0xb2, 0x5e, 0xcc, 0x43, 0x0f, 0x3d, 0x09, 0x78, 0xc4, 0xce, 0x73, 0x02, 0x0e, + 0x4e, 0xc6, 0x4e, 0x38, 0xa2, 0xf6, 0x4c, 0x7b, 0x3d, 0x78, 0x76, 0x66, 0x32, 0xdd, 0x63, 0xcb, + 0x07, 0xc4, 0x47, 0x80, 0x0b, 0x07, 0x10, 0x57, 0xae, 0xef, 0xc8, 0x47, 0x00, 0x8e, 0x11, 0x12, + 0x12, 0x47, 0x94, 0x7c, 0x0d, 0x0e, 0xa8, 0xaa, 0x7b, 0x66, 0x7a, 0xec, 0x75, 0x42, 0x0e, 0xdc, + 0xa6, 0x7e, 0x55, 0x5d, 0x55, 0x5d, 0x5d, 0x7f, 0xba, 0x07, 0x56, 0xc2, 0xf9, 0x79, 0x9a, 0x9f, + 0x8a, 0x7c, 0x33, 0xcb, 0x53, 0x95, 0xb2, 0x76, 0x76, 0xe4, 0xad, 0x03, 0x7b, 0x5e, 0x88, 0xfc, + 0xe2, 0x40, 0x71, 0x55, 0x48, 0x5f, 0xbc, 0x2a, 0x84, 0x54, 0x8c, 0x41, 0x37, 0xe1, 0x73, 0x31, + 0x71, 0xd6, 0x9c, 0xf5, 0xa1, 0x4f, 0xdf, 0x5e, 0x06, 0xb7, 0xb7, 0xd3, 0xf9, 0x3c, 0x4d, 0x7e, + 0x41, 0x3a, 0x7c, 0x21, 0xb3, 0x34, 0x91, 0x82, 0xdd, 0x81, 0x7e, 0x2e, 0x64, 0x11, 0x2b, 0x92, + 0x1e, 0xf8, 0x86, 0x62, 0x2e, 0x74, 0xe6, 0x72, 0x36, 0x69, 0x93, 0x0a, 0xfc, 0x44, 0x49, 0x99, + 0x16, 0x79, 0x20, 0x26, 0x1d, 0x02, 0x0d, 0x85, 0xb8, 0xf6, 0x6b, 0xd2, 0xd5, 0xb8, 0xa6, 0xbc, + 0xaf, 0x1c, 0xb8, 0xd5, 0x70, 0xee, 0x83, 0x2d, 0x7e, 0x0a, 0x63, 0x6d, 0x43, 0x6b, 0x20, 0xbb, + 0xa3, 0x2d, 0x77, 0x33, 0x3b, 0xda, 0x3c, 0xb0, 0x70, 0xbf, 0x21, 0xc5, 0x3e, 0x83, 0x65, 0x59, + 0x1c, 0x1d, 0x72, 0x79, 0x6a, 0x96, 0x75, 0xd7, 0x3a, 0xeb, 0xa3, 0xad, 0x9b, 0xb4, 0xcc, 0x66, + 0xf8, 0x4d, 0x39, 0xef, 0xcf, 0x0e, 0x8c, 0xb6, 0x4f, 0x44, 0x60, 0x68, 0x74, 0x34, 0xe3, 0x52, + 0x8a, 0xb0, 0x74, 0x54, 0x53, 0xec, 0x36, 0xf4, 0x54, 0xaa, 0x78, 0x4c, 0xae, 0xf6, 0x7c, 0x4d, + 0xb0, 0x29, 0x80, 0x2c, 0x82, 0x40, 0x48, 0x79, 0x5c, 0xc4, 0xe4, 0x6a, 0xcf, 0xb7, 0x10, 0xd4, + 0x76, 0xcc, 0xa3, 0x58, 0x84, 0x14, 0xa6, 0x9e, 0x6f, 0x28, 0x36, 0x81, 0xa5, 0x73, 0x9e, 0x27, + 0x51, 0x32, 0x9b, 0xf4, 0x88, 0x51, 0x92, 0xb8, 0x22, 0x14, 0x8a, 0x47, 0xf1, 0xa4, 0xbf, 0xe6, + 0xac, 0x8f, 0x7d, 0x43, 0x79, 0xaf, 0x1d, 0x80, 0x9d, 0x62, 0x9e, 0x19, 0x37, 0xd7, 0x60, 0x44, + 0x1e, 0x1c, 0xf2, 0xa3, 0x58, 0x48, 0xf2, 0xb5, 0xe3, 0xdb, 0x10, 0x5b, 0x87, 0x1b, 0x41, 0x3a, + 0xcf, 0x62, 0xa1, 0x44, 0x68, 0xa4, 0xd0, 0x75, 0xc7, 0xbf, 0x0c, 0xb3, 0x8f, 0x61, 0xf9, 0x38, + 0x4a, 0x22, 0x79, 0x22, 0xc2, 0x87, 0x17, 0x4a, 0xe8, 0x90, 0x3b, 0x7e, 0x13, 0x64, 0x1e, 0x8c, + 0x4b, 0xc0, 0x4f, 0xcf, 0x25, 0x6d, 0xc8, 0xf1, 0x1b, 0x18, 0xfb, 0x2e, 0xdc, 0x14, 0x52, 0x45, + 0x73, 0xae, 0xc4, 0x21, 0xba, 0x42, 0x82, 0x3d, 0x12, 0xbc, 0xca, 0xf0, 0xfe, 0xe2, 0x00, 0xec, + 0xa5, 0x3c, 0x34, 0x5b, 0xba, 0xe2, 0x86, 0xde, 0xd4, 0x25, 0x37, 0xa6, 0x00, 0xb4, 0x4b, 0x2d, + 0xd2, 0x26, 0x11, 0x0b, 0x61, 0xab, 0x30, 0xc8, 0xf2, 0x74, 0x96, 0x0b, 0x29, 0x4d, 0xca, 0x56, + 0x34, 0xae, 0x9d, 0x0b, 0xc5, 0x1f, 0x46, 0x49, 0x9c, 0xce, 0x4c, 0xe2, 0x5a, 0x08, 0xbb, 0x0f, + 0x2b, 0x35, 0xb5, 0x7b, 0xf8, 0x64, 0x87, 0x7c, 0x1f, 0xfa, 0x97, 0x50, 0xef, 0xf7, 0x0e, 0x2c, + 0x1f, 0x9c, 0xf0, 0x3c, 0x8c, 0x92, 0xd9, 0x6e, 0x9e, 0x16, 0x19, 0x9e, 0x9a, 0xe2, 0xf9, 0x4c, + 0x28, 0x53, 0x7e, 0x86, 0xc2, 0xa2, 0xdc, 0xd9, 0xd9, 0x43, 0x3f, 0x3b, 0x58, 0x94, 0xf8, 0xad, + 0xf7, 0x99, 0x4b, 0xb5, 0x97, 0x06, 0x5c, 0x45, 0x69, 0x62, 0xdc, 0x6c, 0x82, 0x54, 0x78, 0x17, + 0x49, 0x40, 0x99, 0xd3, 0xa1, 0xc2, 0x23, 0x0a, 0xf7, 0x57, 0x24, 0x86, 0xd3, 0x23, 0x4e, 0x45, + 0x7b, 0xff, 0xec, 0x00, 0x1c, 0x5c, 0x24, 0xc1, 0xa5, 0x1c, 0x79, 0x74, 0x26, 0x12, 0xd5, 0xcc, + 0x11, 0x0d, 0xa1, 0x32, 0x9d, 0x32, 0x59, 0x19, 0xca, 0x8a, 0x66, 0x77, 0x61, 0x98, 0x8b, 0x40, + 0x24, 0x0a, 0x99, 0x1d, 0x62, 0xd6, 0x00, 0x66, 0xc3, 0x9c, 0x4b, 0x25, 0xf2, 0x46, 0x30, 0x1b, + 0x18, 0xdb, 0x00, 0xd7, 0xa6, 0x77, 0x55, 0x14, 0x9a, 0x80, 0x5e, 0xc1, 0x51, 0x1f, 0x6d, 0xa2, + 0xd4, 0xd7, 0xd7, 0xfa, 0x6c, 0x0c, 0xf5, 0xd9, 0x34, 0xe9, 0x5b, 0xd2, 0xfa, 0x2e, 0xe3, 0xa8, + 0xef, 0x28, 0x4e, 0x83, 0xd3, 0x28, 0x99, 0xd1, 0x01, 0x0c, 0x28, 0x54, 0x0d, 0x8c, 0xfd, 0x10, + 0xdc, 0x22, 0xc9, 0x85, 0x4c, 0xe3, 0x33, 0x11, 0xd2, 0x39, 0xca, 0xc9, 0xd0, 0x6a, 0x1b, 0xf6, + 0x09, 0xfb, 0x57, 0x44, 0xad, 0x13, 0x02, 0xdd, 0x29, 0xcc, 0x09, 0x4d, 0x01, 0x8e, 0xc8, 0x91, + 0xc3, 0x8b, 0x4c, 0x4c, 0x46, 0x3a, 0xcb, 0x6a, 0x84, 0x7d, 0x02, 0xb7, 0xa4, 0x08, 0xd2, 0x24, + 0x94, 0x0f, 0xc5, 0x49, 0x94, 0x84, 0x4f, 0x29, 0x16, 0x93, 0x31, 0x85, 0x78, 0x11, 0xcb, 0xfb, + 0x93, 0x03, 0x63, 0xbb, 0xf7, 0x59, 0x5d, 0xd9, 0xb9, 0xa6, 0x2b, 0xb7, 0xed, 0xae, 0xcc, 0xbe, + 0x53, 0x75, 0x5f, 0xdd, 0x4d, 0x69, 0x7f, 0xcf, 0xf2, 0x14, 0xdb, 0x94, 0x4f, 0x8c, 0xaa, 0x21, + 0x3f, 0x80, 0x51, 0x2e, 0x62, 0x7e, 0x51, 0xb5, 0x51, 0x94, 0xbf, 0x81, 0xf2, 0x7e, 0x0d, 0xfb, + 0xb6, 0x8c, 0xf7, 0xb7, 0x36, 0x8c, 0x2c, 0xe6, 0x95, 0xdc, 0x70, 0xfe, 0xc7, 0xdc, 0x68, 0x5f, + 0x93, 0x1b, 0x6b, 0xa5, 0x4b, 0xc5, 0xd1, 0x4e, 0x94, 0x9b, 0x72, 0xb1, 0xa1, 0x4a, 0xa2, 0x91, + 0x8c, 0x36, 0x84, 0xdd, 0xd0, 0x22, 0xad, 0x54, 0xbc, 0x0c, 0xb3, 0x4d, 0x60, 0x04, 0x6d, 0x73, + 0x15, 0x9c, 0xbc, 0xc8, 0xcc, 0xe9, 0xf4, 0xe9, 0x88, 0x17, 0x70, 0xd8, 0xb7, 0xa0, 0x27, 0x15, + 0x9f, 0x09, 0x4a, 0xc5, 0x95, 0xad, 0x21, 0xa5, 0x0e, 0x02, 0xbe, 0xc6, 0xad, 0xe0, 0x0f, 0xde, + 0x13, 0x7c, 0xef, 0x3f, 0x6d, 0x58, 0x6e, 0x4c, 0xab, 0x45, 0x53, 0xbd, 0xb6, 0xd8, 0xbe, 0xc6, + 0xe2, 0x1a, 0x74, 0x8b, 0x24, 0xd2, 0x87, 0xbd, 0xb2, 0x35, 0x46, 0xfe, 0x8b, 0x24, 0x52, 0x98, + 0x7d, 0x3e, 0x71, 0x2c, 0x9f, 0xba, 0xef, 0x4b, 0x88, 0x4f, 0xe0, 0x56, 0x9d, 0xfa, 0x3b, 0x3b, + 0x7b, 0x7b, 0x69, 0x70, 0x5a, 0x75, 0xc6, 0x45, 0x2c, 0xc6, 0xf4, 0x4c, 0xa7, 0x12, 0x7e, 0xdc, + 0xd2, 0x53, 0xfd, 0xdb, 0xd0, 0x0b, 0x70, 0xca, 0x52, 0x94, 0x4c, 0x42, 0x59, 0x63, 0xf7, 0x71, + 0xcb, 0xd7, 0x7c, 0xf6, 0x31, 0x74, 0xc3, 0x62, 0x9e, 0x99, 0x58, 0xad, 0xa0, 0x5c, 0x3d, 0xf6, + 0x1e, 0xb7, 0x7c, 0xe2, 0xa2, 0x54, 0x9c, 0xf2, 0x70, 0x32, 0xac, 0xa5, 0xea, 0x49, 0x82, 0x52, + 0xc8, 0x45, 0x29, 0xac, 0x49, 0xaa, 0x4f, 0x23, 0x55, 0xb7, 0x47, 0x94, 0x42, 0xee, 0xc3, 0x01, + 0xf4, 0xa5, 0x4e, 0xe4, 0x1f, 0xc1, 0xcd, 0x46, 0xf4, 0xf7, 0x22, 0x49, 0xa1, 0xd2, 0xec, 0x89, + 0x73, 0xdd, 0x95, 0xa2, 0x5c, 0x3f, 0x05, 0xa0, 0x3d, 0x3d, 0xca, 0xf3, 0x34, 0x2f, 0xaf, 0x36, + 0x4e, 0x75, 0xb5, 0xf1, 0xbe, 0x09, 0x43, 0xdc, 0xcb, 0x3b, 0xd8, 0xb8, 0x89, 0xeb, 0xd8, 0x19, + 0x8c, 0xc9, 0xfb, 0xe7, 0x7b, 0xd7, 0x48, 0xb0, 0x2d, 0xb8, 0xad, 0xef, 0x17, 0x3a, 0x9d, 0x9f, + 0xa5, 0x32, 0xa2, 0x01, 0xa3, 0x0b, 0x6b, 0x21, 0x0f, 0x47, 0x80, 0x40, 0x75, 0x07, 0xcf, 0xf7, + 0xca, 0x79, 0x59, 0xd2, 0xde, 0xf7, 0x61, 0x88, 0x16, 0xb5, 0xb9, 0x75, 0xe8, 0x13, 0xa3, 0x8c, + 0x83, 0x5b, 0x85, 0xd3, 0x38, 0xe4, 0x1b, 0xbe, 0xf7, 0x5b, 0x07, 0x46, 0xba, 0x5d, 0xe9, 0x95, + 0x1f, 0xda, 0xad, 0xd6, 0x1a, 0xcb, 0xcb, 0x7a, 0xb7, 0x35, 0x6e, 0x02, 0x50, 0xc3, 0xd1, 0x02, + 0xdd, 0xfa, 0x78, 0x6b, 0xd4, 0xb7, 0x24, 0xf0, 0x60, 0x6a, 0x6a, 0x41, 0x68, 0xff, 0xd0, 0x86, + 0xb1, 0x39, 0x52, 0x2d, 0xf2, 0x7f, 0x2a, 0x3b, 0x53, 0x19, 0x5d, 0xbb, 0x32, 0xee, 0x97, 0x95, + 0xd1, 0xab, 0xb7, 0x51, 0x67, 0x51, 0x5d, 0x18, 0xf7, 0x4c, 0x61, 0xf4, 0x49, 0x6c, 0xb9, 0x2c, + 0x8c, 0x52, 0x4a, 0xd7, 0xc5, 0x3d, 0x53, 0x17, 0x4b, 0xb5, 0x50, 0x95, 0x52, 0x55, 0x59, 0xdc, + 0x33, 0x65, 0x31, 0xa8, 0x85, 0xaa, 0x63, 0xae, 0xaa, 0x62, 0x09, 0x7a, 0x74, 0x9c, 0xde, 0xe7, + 0xe0, 0xda, 0xa1, 0xa1, 0x9a, 0xb8, 0x6f, 0x98, 0x8d, 0x54, 0xb0, 0x84, 0x7c, 0xb3, 0xf6, 0x15, + 0x2c, 0x37, 0x9a, 0x0a, 0xce, 0xc6, 0x48, 0x6e, 0xf3, 0x24, 0x10, 0x71, 0x75, 0xc3, 0xb6, 0x10, + 0x2b, 0xc9, 0xda, 0xb5, 0x66, 0xa3, 0xa2, 0x91, 0x64, 0xd6, 0x3d, 0xb9, 0xd3, 0xb8, 0x27, 0xff, + 0xc3, 0x81, 0xb1, 0xbd, 0x00, 0xaf, 0xda, 0x8f, 0xf2, 0x7c, 0x3b, 0x0d, 0xf5, 0x69, 0xf6, 0xfc, + 0x92, 0xc4, 0xd4, 0xc7, 0xcf, 0x98, 0x4b, 0x69, 0x32, 0xb0, 0xa2, 0x0d, 0xef, 0x20, 0x48, 0xb3, + 0xf2, 0xe5, 0x53, 0xd1, 0x86, 0xb7, 0x27, 0xce, 0x44, 0x6c, 0x46, 0x4d, 0x45, 0xa3, 0xb5, 0xa7, + 0x42, 0x4a, 0x4c, 0x13, 0xdd, 0x21, 0x4b, 0x12, 0x57, 0xf9, 0xfc, 0x7c, 0x9b, 0x17, 0x52, 0x98, + 0xdb, 0x4d, 0x45, 0x63, 0x58, 0xf0, 0x85, 0xc6, 0xf3, 0xb4, 0x48, 0xca, 0x3b, 0x8d, 0x85, 0x78, + 0xe7, 0x70, 0xf3, 0x59, 0x91, 0xcf, 0x04, 0x25, 0x71, 0xf9, 0xe0, 0x5b, 0x85, 0x41, 0x94, 0xf0, + 0x40, 0x45, 0x67, 0xc2, 0x44, 0xb2, 0xa2, 0x31, 0x7f, 0x55, 0x34, 0x17, 0xe6, 0x52, 0x47, 0xdf, + 0x28, 0x7f, 0x1c, 0xc5, 0x82, 0xf2, 0xda, 0x6c, 0xa9, 0xa4, 0xa9, 0x44, 0xf5, 0x74, 0x35, 0xcf, + 0x39, 0x4d, 0x79, 0x7f, 0x6c, 0xc3, 0xea, 0x7e, 0x26, 0x72, 0xae, 0x84, 0x7e, 0x42, 0x1e, 0x04, + 0x27, 0x62, 0xce, 0x4b, 0x17, 0xee, 0x42, 0x3b, 0xcd, 0xc8, 0xb8, 0xc9, 0x77, 0xcd, 0xde, 0xcf, + 0xfc, 0x76, 0x9a, 0x91, 0x13, 0x5c, 0x9e, 0x9a, 0xd8, 0xd2, 0xf7, 0xb5, 0xef, 0xc9, 0x55, 0x18, + 0x84, 0x5c, 0xf1, 0x23, 0x2e, 0x45, 0x19, 0xd3, 0x92, 0xa6, 0xa7, 0x17, 0xbe, 0x54, 0x4c, 0x44, + 0x35, 0x41, 0x9a, 0xc8, 0x9a, 0x89, 0xa6, 0xa1, 0x50, 0xfa, 0x38, 0x2e, 0xe4, 0x09, 0x85, 0x71, + 0xe0, 0x6b, 0x02, 0x7d, 0xa9, 0x72, 0x7e, 0xa0, 0x53, 0x1c, 0xa3, 0x7e, 0x9c, 0xa7, 0x73, 0xdd, + 0x58, 0x68, 0x94, 0x0c, 0x7c, 0x0b, 0x29, 0xf9, 0x87, 0xfa, 0x62, 0x0f, 0x35, 0x5f, 0x23, 0x9e, + 0x82, 0xe5, 0x97, 0x0f, 0x4c, 0xda, 0x3f, 0x15, 0x8a, 0xb3, 0x55, 0x2b, 0x1c, 0x80, 0xe1, 0x40, + 0x8e, 0x09, 0xc6, 0x7b, 0xbb, 0x47, 0xd9, 0x72, 0x3a, 0x56, 0xcb, 0x29, 0x23, 0xd8, 0xa5, 0x14, + 0xa7, 0x6f, 0xef, 0x53, 0xb8, 0x6d, 0x4e, 0xe4, 0xe5, 0x03, 0xb4, 0x7a, 0xed, 0x59, 0x68, 0xb6, + 0x36, 0xef, 0xfd, 0xd5, 0x81, 0x8f, 0x2e, 0x2d, 0xfb, 0xe0, 0x97, 0xf9, 0x67, 0xd0, 0xc5, 0x87, + 0xd0, 0xa4, 0x43, 0xa5, 0x79, 0x0f, 0x6d, 0x2c, 0x54, 0xb9, 0x89, 0xc4, 0xa3, 0x44, 0xe5, 0x17, + 0x3e, 0x2d, 0x58, 0xfd, 0x29, 0x0c, 0x2b, 0x08, 0xf5, 0x9e, 0x8a, 0x8b, 0xb2, 0xfb, 0x9e, 0x8a, + 0x0b, 0xbc, 0x1b, 0x9c, 0xf1, 0xb8, 0xd0, 0xa1, 0x31, 0x03, 0xb6, 0x11, 0x58, 0x5f, 0xf3, 0x3f, + 0x6f, 0xff, 0xc0, 0xf1, 0x7e, 0x0d, 0x93, 0xc7, 0x3c, 0x09, 0x63, 0x93, 0x8f, 0xba, 0x29, 0x98, + 0x10, 0x7c, 0xc3, 0x0a, 0xc1, 0x08, 0xb5, 0x10, 0xf7, 0x1d, 0xd9, 0x78, 0x17, 0x86, 0x47, 0xe5, + 0x38, 0x34, 0x81, 0xaf, 0x01, 0xca, 0x99, 0x57, 0xb1, 0x34, 0x0f, 0x30, 0xfa, 0xf6, 0x3e, 0x82, + 0x5b, 0xbb, 0x42, 0x69, 0xdb, 0xdb, 0xc7, 0x33, 0x63, 0xd9, 0x5b, 0x87, 0xdb, 0x4d, 0xd8, 0x04, + 0xd7, 0x85, 0x4e, 0x70, 0x5c, 0x8d, 0x9a, 0xe0, 0x78, 0xb6, 0xf1, 0x4b, 0xe8, 0xeb, 0xac, 0x60, + 0xcb, 0x30, 0x7c, 0x92, 0x9c, 0xf1, 0x38, 0x0a, 0xf7, 0x33, 0xb7, 0xc5, 0x06, 0xd0, 0x3d, 0x50, + 0x69, 0xe6, 0x3a, 0x6c, 0x08, 0xbd, 0x67, 0xd8, 0x16, 0xdc, 0x36, 0x03, 0xe8, 0x63, 0xe7, 0x9c, + 0x0b, 0xb7, 0x83, 0xf0, 0x81, 0xe2, 0xb9, 0x72, 0xbb, 0x08, 0xbf, 0xc8, 0x42, 0xae, 0x84, 0xdb, + 0x63, 0x2b, 0x00, 0x3f, 0x29, 0x54, 0x6a, 0xc4, 0xfa, 0x1b, 0xbf, 0x21, 0xb1, 0x19, 0xda, 0x1e, + 0x1b, 0xfd, 0x44, 0xbb, 0x2d, 0xb6, 0x04, 0x9d, 0x9f, 0x8b, 0x73, 0xd7, 0x61, 0x23, 0x58, 0xf2, + 0x8b, 0x24, 0x89, 0x92, 0x99, 0xb6, 0x41, 0xe6, 0x42, 0xb7, 0x83, 0x0c, 0x74, 0x22, 0x13, 0xa1, + 0xdb, 0x65, 0x63, 0x18, 0x7c, 0x69, 0xde, 0xde, 0x6e, 0x0f, 0x59, 0x28, 0x86, 0x6b, 0xfa, 0xc8, + 0x22, 0x83, 0x48, 0x2d, 0x21, 0x45, 0xab, 0x90, 0x1a, 0x6c, 0xec, 0xc3, 0xa0, 0x1c, 0x7b, 0xec, + 0x06, 0x8c, 0x8c, 0x0f, 0x08, 0xb9, 0x2d, 0xdc, 0x04, 0x0d, 0x37, 0xd7, 0xc1, 0x0d, 0xe3, 0x00, + 0x73, 0xdb, 0xf8, 0x85, 0x53, 0xca, 0xed, 0x50, 0x10, 0x2e, 0x92, 0xc0, 0xed, 0xa2, 0x20, 0x75, + 0x3b, 0x37, 0xdc, 0x78, 0x0a, 0x4b, 0xf4, 0xb9, 0x8f, 0x87, 0xb8, 0x62, 0xf4, 0x19, 0xc4, 0x6d, + 0x61, 0x1c, 0xd1, 0xba, 0x96, 0x76, 0x30, 0x1e, 0xb4, 0x1d, 0x4d, 0xb7, 0xd1, 0x05, 0x1d, 0x1b, + 0x0d, 0x74, 0x36, 0x12, 0x18, 0x94, 0x6d, 0x8a, 0xdd, 0x82, 0x1b, 0x65, 0x8c, 0x0c, 0xa4, 0x15, + 0xee, 0x0a, 0xa5, 0x01, 0xd7, 0x21, 0xfd, 0x15, 0xd9, 0xc6, 0xb0, 0xfa, 0x62, 0x9e, 0x9e, 0x09, + 0x83, 0x74, 0xd0, 0x22, 0x4e, 0x45, 0x43, 0x77, 0x71, 0x01, 0xd2, 0xf4, 0x77, 0xc5, 0xed, 0x6d, + 0x7c, 0x01, 0x83, 0xb2, 0x14, 0x2d, 0x7b, 0x25, 0x54, 0xd9, 0xd3, 0x80, 0xeb, 0xd4, 0x06, 0x0c, + 0xd2, 0xde, 0x78, 0x49, 0x23, 0x0c, 0x33, 0xd9, 0x0a, 0x80, 0x41, 0x4c, 0xe6, 0x9c, 0x46, 0x99, + 0x39, 0x57, 0x91, 0xc5, 0x3c, 0xa8, 0x72, 0xe7, 0x4c, 0xe4, 0xca, 0xed, 0xe0, 0xf7, 0x93, 0xe4, + 0x57, 0x22, 0xc0, 0xe4, 0xc1, 0x68, 0x47, 0x52, 0xb9, 0xbd, 0x8d, 0x3d, 0x18, 0xbd, 0x44, 0x55, + 0x5c, 0x91, 0xee, 0x3b, 0xc0, 0x4a, 0xe7, 0x6a, 0xd4, 0x6d, 0xa1, 0x4d, 0x4a, 0xbc, 0x0a, 0x75, + 0x1d, 0x76, 0x13, 0x96, 0x31, 0xe8, 0x35, 0xd4, 0xde, 0xfa, 0xaa, 0x03, 0x7d, 0x5d, 0x01, 0xec, + 0x0b, 0x18, 0x59, 0x3f, 0x01, 0xd9, 0x1d, 0xac, 0xc5, 0xab, 0xbf, 0x2c, 0x57, 0xbf, 0x76, 0x05, + 0xd7, 0x65, 0xe3, 0xb5, 0xd8, 0x8f, 0x01, 0xea, 0x89, 0xc7, 0x3e, 0xa2, 0x6b, 0xc0, 0xe5, 0x09, + 0xb8, 0x3a, 0xa1, 0xbb, 0xd2, 0x82, 0x1f, 0x9c, 0x5e, 0x8b, 0xfd, 0x0c, 0x96, 0x4d, 0x73, 0xd2, + 0xe7, 0xc2, 0xa6, 0x56, 0xbf, 0x5a, 0x30, 0xcb, 0xde, 0xa9, 0xec, 0xcb, 0x4a, 0x99, 0x3e, 0x13, + 0x36, 0x59, 0xd0, 0xfc, 0xb4, 0x9a, 0xaf, 0x5f, 0xdb, 0x16, 0xbd, 0x16, 0xdb, 0x85, 0x91, 0x6e, + 0x5e, 0xfa, 0x6a, 0x72, 0x17, 0x65, 0xaf, 0xeb, 0x66, 0xef, 0x74, 0x68, 0x1b, 0xc6, 0x76, 0xbf, + 0x61, 0x14, 0xc9, 0x05, 0x8d, 0x49, 0x2b, 0x59, 0xd4, 0x9a, 0xbc, 0xd6, 0xc3, 0xc9, 0xdf, 0xdf, + 0x4c, 0x9d, 0xd7, 0x6f, 0xa6, 0xce, 0xbf, 0xdf, 0x4c, 0x9d, 0xdf, 0xbd, 0x9d, 0xb6, 0x5e, 0xbf, + 0x9d, 0xb6, 0xfe, 0xf5, 0x76, 0xda, 0x3a, 0xea, 0xd3, 0xcf, 0xe6, 0xef, 0xfd, 0x37, 0x00, 0x00, + 0xff, 0xff, 0xf3, 0x6f, 0xde, 0xe1, 0x7e, 0x16, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/dm/dm/proto/dmworker.proto b/dm/dm/proto/dmworker.proto index 99cc56e0ef4..4f54b1d76cf 100644 --- a/dm/dm/proto/dmworker.proto +++ b/dm/dm/proto/dmworker.proto @@ -383,4 +383,10 @@ message GetWorkerCfgRequest { message GetWorkerCfgResponse { string cfg = 1; -} \ No newline at end of file +} + +enum ValidatorOp { + InvalidValidatorOp = 0; + StartValidator = 1; + StopValidator = 2; +} diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index f7c007983da..6220ef3f45c 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -171,8 +171,7 @@ func (t *testServer) TestServer(c *C) { subtaskCfg.MydumperPath = mydumperPath sourceCfg := loadSourceConfigWithoutPassword(c) - _, err = ha.PutSubTaskCfgStage(s.etcdClient, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(s.etcdClient, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index f8dbb5b2c10..b322602bc2e 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -321,7 +321,8 @@ func (w *SourceWorker) EnableRelay(startBySourceCfg bool) (err error) { w.relayCtx, w.relayCancel = context.WithCancel(w.ctx) // 1. adjust relay starting position, to the earliest of subtasks var subTaskCfgs map[string]config.SubTaskConfig - _, subTaskCfgs, _, err = w.fetchSubTasksAndAdjust() + //nolint:dogsled + _, _, subTaskCfgs, _, err = w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -455,7 +456,7 @@ func (w *SourceWorker) EnableHandleSubtasks() error { // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event // because triggering these events is useless now - subTaskStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust() + subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -467,10 +468,14 @@ func (w *SourceWorker) EnableHandleSubtasks() error { if expectStage.IsDeleted { continue } + validatorStage := pb.Stage_InvalidStage + if s, ok := validatorStages[subTaskCfg.Name]; ok { + validatorStage = s.Expect + } w.l.Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) // "for range" of a map will use same value address, so we'd better not pass value address to other function clone := subTaskCfg - if err2 := w.StartSubTask(&clone, expectStage.Expect, false); err2 != nil { + if err2 := w.StartSubTask(&clone, expectStage.Expect, validatorStage, false); err2 != nil { w.subTaskHolder.closeAllSubTasks() return err2 } @@ -483,6 +488,13 @@ func (w *SourceWorker) EnableHandleSubtasks() error { //nolint:errcheck w.observeSubtaskStage(w.subTaskCtx, w.etcdClient, revSubTask) }() + w.subTaskWg.Add(1) + go func() { + defer w.subTaskWg.Done() + // TODO: handle fatal error from observeValidatorStage + //nolint:errcheck + w.observeValidatorStage(w.subTaskCtx, revSubTask) + }() w.subTaskEnabled.Store(true) w.l.Info("handling subtask enabled") @@ -510,23 +522,23 @@ func (w *SourceWorker) DisableHandleSubtasks() { // fetchSubTasksAndAdjust gets source's subtask stages and configs, adjust some values by worker's config and status // source **must not be empty** -// return map{task name -> subtask stage}, map{task name -> subtask config}, revision, error. -func (w *SourceWorker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) { +// return map{task name -> subtask stage}, map{task name -> validator stage}, map{task name -> subtask config}, revision, error. +func (w *SourceWorker) fetchSubTasksAndAdjust() (map[string]ha.Stage, map[string]ha.Stage, map[string]config.SubTaskConfig, int64, error) { // we get the newest subtask stages directly which will omit the subtask stage PUT/DELETE event // because triggering these events is useless now - subTaskStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) + subTaskStages, validatorStages, subTaskCfgM, revSubTask, err := ha.GetSubTaskStageConfig(w.etcdClient, w.cfg.SourceID) if err != nil { - return nil, nil, 0, err + return nil, nil, nil, 0, err } if err = copyConfigFromSourceForEach(subTaskCfgM, w.cfg, w.relayEnabled.Load()); err != nil { - return nil, nil, 0, err + return nil, nil, nil, 0, err } - return subTaskStages, subTaskCfgM, revSubTask, nil + return subTaskStages, validatorStages, subTaskCfgM, revSubTask, nil } // StartSubTask creates a subtask and run it. -func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.Stage, needLock bool) error { +func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage, validatorStage pb.Stage, needLock bool) error { if needLock { w.Lock() defer w.Unlock() @@ -563,7 +575,7 @@ func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.St } w.l.Info("subtask created", zap.Stringer("config", cfg2)) - st.Run(expectStage, w.getRelayWithoutLock()) + st.Run(expectStage, validatorStage, w.getRelayWithoutLock()) return nil } @@ -662,7 +674,7 @@ func (w *SourceWorker) QueryStatus(ctx context.Context, name string) ([]*pb.SubT } func (w *SourceWorker) resetSubtaskStage() (int64, error) { - subTaskStages, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust() + subTaskStages, _, subTaskCfgm, revSubTask, err := w.fetchSubTasksAndAdjust() if err != nil { return 0, err } @@ -787,7 +799,13 @@ func (w *SourceWorker) operateSubTaskStage(stage ha.Stage, subTaskCfg config.Sub if st := w.subTaskHolder.findSubTask(stage.Task); st == nil { // create the subtask for expected running and paused stage. log.L().Info("start to create subtask", zap.String("sourceID", subTaskCfg.SourceID), zap.String("task", subTaskCfg.Name)) - err := w.StartSubTask(&subTaskCfg, stage.Expect, true) + + expectValidatorStage, err := getExpectValidatorStage(subTaskCfg.ValidatorCfg, w.etcdClient, stage.Source, stage.Task, stage.Revision) + if err != nil { + return opErrTypeBeforeOp, terror.Annotate(err, "fail to get validator stage from etcd") + } + + err = w.StartSubTask(&subTaskCfg, stage.Expect, expectValidatorStage, true) return opErrTypeBeforeOp, err } if stage.Expect == pb.Stage_Running { @@ -957,7 +975,7 @@ func (w *SourceWorker) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest uuid := w.relayHolder.Status(nil).RelaySubDir - _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() + _, _, subTaskCfgs, _, err := w.fetchSubTasksAndAdjust() if err != nil { return err } @@ -1097,3 +1115,149 @@ func (w *SourceWorker) HandleError(ctx context.Context, req *pb.HandleWorkerErro return st.HandleError(ctx, req, w.getRelayWithoutLock()) } + +func (w *SourceWorker) observeValidatorStage(ctx context.Context, lastUsedRev int64) error { + var wg sync.WaitGroup + + startRevision := lastUsedRev + 1 + for { + stageCh := make(chan ha.Stage, 10) + errCh := make(chan error, 10) + wg.Add(1) + watchCtx, watchCancel := context.WithCancel(ctx) + go func() { + defer func() { + close(stageCh) + close(errCh) + wg.Done() + }() + ha.WatchValidatorStage(watchCtx, w.etcdClient, w.cfg.SourceID, startRevision, stageCh, errCh) + }() + err := w.handleValidatorStage(watchCtx, stageCh, errCh) + watchCancel() + wg.Wait() + + if etcdutil.IsRetryableError(err) { + startRevision = 0 + retryNum := 1 + for startRevision == 0 { + select { + case <-ctx.Done(): + return nil + case <-time.After(500 * time.Millisecond): + startRevision, err = w.getCurrentValidatorRevision(w.cfg.SourceID) + if err != nil { + log.L().Error("reset validator stage failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum)) + } + } + retryNum++ + } + } else { + if err != nil { + log.L().Error("observe validator stage failed, quit now", zap.Error(err)) + } else { + log.L().Info("observe validator stage will quit now") + } + return err + } + } +} + +func (w *SourceWorker) handleValidatorStage(ctx context.Context, stageCh chan ha.Stage, errCh chan error) error { + closed := false + for { + select { + case <-ctx.Done(): + closed = true + case stage, ok := <-stageCh: + if !ok { + closed = true + break + } + log.L().Info("receive validator stage change", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted)) + err := w.operateValidatorStage(stage) + if err != nil { + opType := w.getValidatorOp(stage) + opErrCounter.WithLabelValues(w.name, opType).Inc() + log.L().Error("fail to operate validator stage", zap.Stringer("stage", stage), zap.Bool("is deleted", stage.IsDeleted), zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + case err, ok := <-errCh: + if !ok { + closed = true + break + } + // TODO: deal with err + log.L().Error("watch validator stage received an error", zap.Error(err)) + if etcdutil.IsRetryableError(err) { + return err + } + } + if closed { + log.L().Info("worker is closed, handle validator stage will quit now") + return nil + } + } +} + +func (w *SourceWorker) getCurrentValidatorRevision(source string) (int64, error) { + _, rev, err := ha.GetValidatorStage(w.etcdClient, source, "", 0) + if err != nil { + return 0, err + } + return rev, nil +} + +func (w *SourceWorker) getValidatorOp(stage ha.Stage) string { + if stage.IsDeleted { + return "validator-delete" + } + if stage.Expect == pb.Stage_Running { + return pb.ValidatorOp_StartValidator.String() + } else if stage.Expect == pb.Stage_Stopped { + return pb.ValidatorOp_StopValidator.String() + } + // should not happen + return "" +} + +func (w *SourceWorker) operateValidatorStage(stage ha.Stage) error { + // if the key it's deleted, the subtask is deleted too, let subtask clean it up. + if stage.IsDeleted { + return nil + } + + subtask := w.subTaskHolder.findSubTask(stage.Task) + if subtask == nil { + // when a new subtask start with validator, both subtask and validator stage observer will observe it, + // if validator observe it first, we may not have the subtask. + log.L().Info("cannot find subtask. maybe it's a new task, let subtask stage observer handles it") + return nil + } + + // stage of validator can only be Running or Stopped + switch stage.Expect { + case pb.Stage_Stopped: + subtask.StopValidator() + case pb.Stage_Running: + // validator's config is stored with subtask config, we need to update subtask config as validator may start + // on the fly. + subTaskCfg, _, err := ha.GetSubTaskCfg(w.etcdClient, stage.Source, stage.Task, stage.Revision) + if err != nil { + return err + } + if _, ok := subTaskCfg[stage.Task]; !ok { + log.L().Error("failed to get subtask config", zap.Reflect("stage", stage)) + return errors.New("failed to get subtask config") + } + + subtask.SetCfg(subTaskCfg[stage.Task]) + subtask.StartValidator(stage.Expect) + default: + // should not happen + log.L().Warn("invalid validator stage", zap.Reflect("stage", stage)) + } + return nil +} diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 5dd29bce6c9..1ae4eaa476a 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/relay" + "github.com/pingcap/tiflow/dm/syncer" ) var emptyWorkerStatusInfoJSONLength = 25 @@ -99,7 +100,7 @@ func (t *testServer) testWorker(c *C) { c.Assert(w.StartSubTask(&config.SubTaskConfig{ Name: "testStartTask", - }, pb.Stage_Running, true), IsNil) + }, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) task := w.subTaskHolder.findSubTask("testStartTask") c.Assert(task, NotNil) c.Assert(task.Result().String(), Matches, ".*worker already closed.*") @@ -197,7 +198,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { c.Assert(err, IsNil) subtaskCfg.Mode = "full" subtaskCfg.Timezone = "UTC" - c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) + c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) // check task in paused state c.Assert(utils.WaitSomething(100, 100*time.Millisecond, func() bool { @@ -396,8 +397,7 @@ func (t *testWorkerFunctionalities) testEnableHandleSubtasks(c *C, w *SourceWork c.Assert(w.EnableHandleSubtasks(), IsNil) c.Assert(w.subTaskEnabled.Load(), IsTrue) - _, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil @@ -480,14 +480,13 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(err, IsNil) subtaskCfg.MydumperPath = mydumperPath - _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) rev, err := ha.DeleteSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)}) + []ha.Stage{ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) // step 2.1: start a subtask manually - c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) + c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) // step 3: trigger etcd compaction and check whether we can receive it through watcher _, err = etcdCli.Compact(ctx, rev) c.Assert(err, IsNil) @@ -515,8 +514,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { return w.subTaskHolder.findSubTask(subtaskCfg.Name) == nil }), IsTrue) // step 4.2: add a new subtask stage, worker should receive and start it - _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, - []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { return w.subTaskHolder.findSubTask(subtaskCfg.Name) != nil @@ -553,6 +551,128 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { wg.Wait() } +func (t *testWorkerEtcdCompact) TestWatchValidatorStageEtcdCompact(c *C) { + var ( + masterAddr = tempurl.Alloc()[len("http://"):] + keepAliveTTL = int64(1) + startRev = int64(1) + ) + + etcdDir := c.MkDir() + ETCD, err := createMockETCD(etcdDir, "http://"+masterAddr) + c.Assert(err, IsNil) + defer ETCD.Close() + cfg := NewConfig() + c.Assert(cfg.Parse([]string{"-config=./dm-worker.toml"}), IsNil) + cfg.Join = masterAddr + cfg.KeepAliveTTL = keepAliveTTL + cfg.RelayKeepAliveTTL = keepAliveTTL + + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: GetJoinURLs(cfg.Join), + DialTimeout: dialTimeout, + DialKeepAliveTime: keepaliveTime, + DialKeepAliveTimeout: keepaliveTimeout, + }) + c.Assert(err, IsNil) + sourceCfg := loadSourceConfigWithoutPassword(c) + sourceCfg.From = config.GetDBConfigForTest() + sourceCfg.EnableRelay = false + + // + // step 1: start worker + w, err := NewSourceWorker(sourceCfg, etcdCli, "", "") + c.Assert(err, IsNil) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + defer w.Close() + go func() { + w.Start() + }() + c.Assert(utils.WaitSomething(50, 100*time.Millisecond, func() bool { + return !w.closed.Load() + }), IsTrue) + + // + // step 2: Put a subtask config and subtask stage to this source, then delete it + subtaskCfg := config.SubTaskConfig{} + err = subtaskCfg.DecodeFile(subtaskSampleFile, true) + c.Assert(err, IsNil) + subtaskCfg.MydumperPath = mydumperPath + subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationNone} + + // increase revision + _, err = etcdCli.Put(context.Background(), "/dummy-key", "value") + c.Assert(err, IsNil) + rev, err := ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, []ha.Stage{ha.NewSubTaskStage(pb.Stage_Running, sourceCfg.SourceID, subtaskCfg.Name)}, nil) + c.Assert(err, IsNil) + + // + // step 2.1: start a subtask manually + c.Assert(w.StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) + + // + // step 3: trigger etcd compaction and check whether we can receive it through watcher + _, err = etcdCli.Compact(ctx, rev) + c.Assert(err, IsNil) + subTaskStageCh := make(chan ha.Stage, 10) + subTaskErrCh := make(chan error, 10) + ctxForWatch, cancelFunc := context.WithCancel(ctx) + ha.WatchValidatorStage(ctxForWatch, etcdCli, sourceCfg.SourceID, startRev, subTaskStageCh, subTaskErrCh) + select { + case err = <-subTaskErrCh: + c.Assert(err, Equals, etcdErrCompacted) + case <-time.After(300 * time.Millisecond): + c.Fatal("fail to get etcd error compacted") + } + cancelFunc() + + // + // step 4: watch subtask stage from startRev + subTask := w.subTaskHolder.findSubTask(subtaskCfg.Name) + getValidator := func() *syncer.DataValidator { + subTask.RLock() + defer subTask.RUnlock() + return subTask.validator + } + c.Assert(subTask, NotNil) + c.Assert(getValidator(), IsNil) + var wg sync.WaitGroup + ctx1, cancel1 := context.WithCancel(ctx) + wg.Add(1) + go func() { + defer wg.Done() + c.Assert(w.observeValidatorStage(ctx1, startRev), IsNil) + }() + time.Sleep(time.Second) + + subtaskCfg.ValidatorCfg = config.ValidatorConfig{Mode: config.ValidationFast} + unitBakup := subTask.units[len(subTask.units)-1] + subTask.units[len(subTask.units)-1] = &syncer.Syncer{} // validator need a Syncer, not a mocked unit + validatorStage := ha.NewValidatorStage(pb.Stage_Running, subtaskCfg.SourceID, subtaskCfg.Name) + _, err = ha.PutSubTaskCfgStage(etcdCli, []config.SubTaskConfig{subtaskCfg}, nil, []ha.Stage{validatorStage}) + c.Assert(err, IsNil) + + // validator created + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return getValidator() != nil + }), IsTrue) + + subTask.units[len(subTask.units)-1] = unitBakup // restore unit + cancel1() + wg.Wait() + + // test operate validator + err = w.operateValidatorStage(ha.Stage{IsDeleted: true}) + c.Assert(err, IsNil) + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Task: "not-exist"}) + c.Assert(err, IsNil) + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Task: subtaskCfg.Name}) + c.Assert(err, ErrorMatches, ".*failed to get subtask config.*") + err = w.operateValidatorStage(ha.Stage{Expect: pb.Stage_Running, Source: subtaskCfg.SourceID, Task: subtaskCfg.Name}) + c.Assert(err, IsNil) +} + func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { var ( masterAddr = tempurl.Alloc()[len("http://"):] diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index ca04012059a..078902ad978 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -117,6 +117,8 @@ type SubTask struct { etcdClient *clientv3.Client workerName string + + validator *syncer.DataValidator } // NewSubTask is subtask initializer @@ -213,7 +215,7 @@ func (st *SubTask) initUnits(relay relay.Process) error { // Run runs the sub task. // TODO: check concurrent problems. -func (st *SubTask) Run(expectStage pb.Stage, relay relay.Process) { +func (st *SubTask) Run(expectStage pb.Stage, expectValidatorStage pb.Stage, relay relay.Process) { if st.Stage() == pb.Stage_Finished || st.Stage() == pb.Stage_Running { st.l.Warn("prepare to run a subtask with invalid stage", zap.Stringer("current stage", st.Stage()), @@ -227,6 +229,8 @@ func (st *SubTask) Run(expectStage pb.Stage, relay relay.Process) { return } + st.StartValidator(expectValidatorStage) + if expectStage == pb.Stage_Running { st.run() } else { @@ -257,6 +261,42 @@ func (st *SubTask) run() { go cu.Process(ctx, pr) } +func (st *SubTask) StartValidator(expect pb.Stage) { + // when validator mode=none + if expect == pb.Stage_InvalidStage { + return + } + st.Lock() + defer st.Unlock() + if st.cfg.ValidatorCfg.Mode != config.ValidationFast && st.cfg.ValidatorCfg.Mode != config.ValidationFull { + return + } + var syncerObj *syncer.Syncer + var ok bool + for _, u := range st.units { + if syncerObj, ok = u.(*syncer.Syncer); ok { + break + } + } + if syncerObj == nil { + st.l.Warn("cannot start validator without syncer") + return + } + + if st.validator == nil { + st.validator = syncer.NewContinuousDataValidator(st.cfg, syncerObj) + } + st.validator.Start(expect) +} + +func (st *SubTask) StopValidator() { + st.Lock() + if st.validator != nil { + st.validator.Stop() + } + st.Unlock() +} + func (st *SubTask) setCurrCtx(ctx context.Context, cancel context.CancelFunc) { st.Lock() // call previous cancel func for safety @@ -492,7 +532,11 @@ func (st *SubTask) Close() { } st.closeUnits() // close all un-closed units - updateTaskMetric(st.cfg.Name, st.cfg.SourceID, pb.Stage_Stopped, st.workerName) + + cfg := st.getCfg() + updateTaskMetric(cfg.Name, cfg.SourceID, pb.Stage_Stopped, st.workerName) + + st.StopValidator() } // Pause pauses a running sub task or a sub task paused by error. @@ -515,7 +559,11 @@ func (st *SubTask) Pause() error { // TODO: similar to Run, refactor later. func (st *SubTask) Resume(relay relay.Process) error { if !st.initialized.Load() { - st.Run(pb.Stage_Running, relay) + expectValidatorStage, err := getExpectValidatorStage(st.cfg.ValidatorCfg, st.etcdClient, st.cfg.SourceID, st.cfg.Name, 0) + if err != nil { + return terror.Annotate(err, "fail to get validator stage from etcd") + } + st.Run(pb.Stage_Running, expectValidatorStage, relay) return nil } @@ -652,8 +700,9 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { loadStatus := pu.Status(nil).(*pb.LoadStatus) - if st.cfg.EnableGTID { - gset1, err = gtid.ParserGTID(st.cfg.Flavor, loadStatus.MetaBinlogGTID) + cfg := st.getCfg() + if cfg.EnableGTID { + gset1, err = gtid.ParserGTID(cfg.Flavor, loadStatus.MetaBinlogGTID) if err != nil { return terror.WithClass(err, terror.ClassDMWorker) } @@ -667,8 +716,8 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { for { relayStatus := hub.w.relayHolder.Status(nil) - if st.cfg.EnableGTID { - gset2, err = gtid.ParserGTID(st.cfg.Flavor, relayStatus.RelayBinlogGtid) + if cfg.EnableGTID { + gset2, err = gtid.ParserGTID(cfg.Flavor, relayStatus.RelayBinlogGtid) if err != nil { return terror.WithClass(err, terror.ClassDMWorker) } @@ -689,11 +738,11 @@ func (st *SubTask) unitTransWaitCondition(subTaskCtx context.Context) error { } } - st.l.Debug("wait relay to catchup", zap.Bool("enableGTID", st.cfg.EnableGTID), zap.Stringer("load end position", pos1), zap.String("load end gtid", loadStatus.MetaBinlogGTID), zap.Stringer("relay position", pos2), zap.String("relay gtid", relayStatus.RelayBinlogGtid)) + st.l.Debug("wait relay to catchup", zap.Bool("enableGTID", cfg.EnableGTID), zap.Stringer("load end position", pos1), zap.String("load end gtid", loadStatus.MetaBinlogGTID), zap.Stringer("relay position", pos2), zap.String("relay gtid", relayStatus.RelayBinlogGtid)) select { case <-ctxWait.Done(): - if st.cfg.EnableGTID { + if cfg.EnableGTID { return terror.ErrWorkerWaitRelayCatchupTimeout.Generate(waitRelayCatchupTimeout, loadStatus.MetaBinlogGTID, relayStatus.RelayBinlogGtid) } return terror.ErrWorkerWaitRelayCatchupTimeout.Generate(waitRelayCatchupTimeout, pos1, pos2) @@ -733,6 +782,28 @@ func (st *SubTask) HandleError(ctx context.Context, req *pb.HandleWorkerErrorReq return msg, err } +func (st *SubTask) getCfg() *config.SubTaskConfig { + st.RLock() + defer st.RUnlock() + return st.cfg +} + +func (st *SubTask) SetCfg(subTaskConfig config.SubTaskConfig) { + st.Lock() + st.cfg = &subTaskConfig + st.Unlock() +} + +func (st *SubTask) getValidatorStage() pb.Stage { + st.RLock() + defer st.RUnlock() + + if st.validator != nil { + return st.validator.Stage() + } + return pb.Stage_InvalidStage +} + func updateTaskMetric(task, sourceID string, stage pb.Stage, workerName string) { if stage == pb.Stage_Stopped || stage == pb.Stage_Finished { taskState.DeleteAllAboutLabels(prometheus.Labels{"task": task, "source_id": sourceID}) diff --git a/dm/dm/worker/subtask_holder.go b/dm/dm/worker/subtask_holder.go index 888b756f894..27f512d62ed 100644 --- a/dm/dm/worker/subtask_holder.go +++ b/dm/dm/worker/subtask_holder.go @@ -59,7 +59,7 @@ func (h *subTaskHolder) resetAllSubTasks(relay relay.Process) { // TODO: make a st.reset st.ctx, st.cancel = context.WithCancel(context.Background()) st.cfg.UseRelay = relay != nil - st.Run(stage, relay) + st.Run(stage, st.getValidatorStage(), relay) } } diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 41e7a5a2152..7e4d142dd43 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -180,7 +180,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, relay relay.Process) []unit.Unit { return nil } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Paused) c.Assert(strings.Contains(st.Result().Errors[0].String(), "has no dm units for mode"), IsTrue) @@ -190,7 +190,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Running) c.Assert(st.CurrUnit(), Equals, mockDumper) c.Assert(st.Result(), IsNil) @@ -301,7 +301,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Running, nil) + st.Run(pb.Stage_Running, pb.Stage_Running, nil) c.Assert(st.Stage(), Equals, pb.Stage_Running) c.Assert(st.CurrUnit(), Equals, mockDumper) c.Assert(st.Result(), IsNil) @@ -401,7 +401,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.Result().Errors, HasLen, 0) - st.Run(pb.Stage_Finished, nil) + st.Run(pb.Stage_Finished, pb.Stage_Stopped, nil) c.Assert(st.CurrUnit(), Equals, mockLoader) c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.Result().Errors, HasLen, 0) @@ -409,9 +409,11 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { func (t *testSubTask) TestSubtaskWithStage(c *C) { cfg := &config.SubTaskConfig{ - Name: "testSubtaskScene", - Mode: config.ModeFull, + SourceID: "source", + Name: "testSubtaskScene", + Mode: config.ModeFull, } + c.Assert(cfg.Adjust(false), IsNil) st := NewSubTaskWithStage(cfg, pb.Stage_Paused, nil, "worker") c.Assert(st.Stage(), DeepEquals, pb.Stage_Paused) @@ -450,7 +452,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { return []unit.Unit{mockDumper, mockLoader} } - st.Run(pb.Stage_Finished, nil) + st.Run(pb.Stage_Finished, pb.Stage_Stopped, nil) c.Assert(st.Stage(), Equals, pb.Stage_Finished) c.Assert(st.CurrUnit(), Equals, nil) c.Assert(st.Result(), IsNil) diff --git a/dm/dm/worker/task_checker_test.go b/dm/dm/worker/task_checker_test.go index 5a58a16a5eb..c15abf97091 100644 --- a/dm/dm/worker/task_checker_test.go +++ b/dm/dm/worker/task_checker_test.go @@ -107,10 +107,11 @@ func (s *testTaskCheckerSuite) TestCheck(c *check.C) { rtsc.w = w st := &SubTask{ - cfg: &config.SubTaskConfig{Name: taskName}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: taskName}, stage: pb.Stage_Running, l: log.With(zap.String("subtask", taskName)), } + c.Assert(st.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st) rtsc.check() bf, ok := rtsc.bc.backoffs[taskName] @@ -241,7 +242,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { // test backoff strategies of different tasks do not affect each other st1 = &SubTask{ - cfg: &config.SubTaskConfig{Name: task1}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: task1}, stage: pb.Stage_Paused, result: &pb.ProcessResult{ IsCanceled: false, @@ -249,9 +250,10 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { }, l: log.With(zap.String("subtask", task1)), } + c.Assert(st1.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st1) st2 = &SubTask{ - cfg: &config.SubTaskConfig{Name: task2}, + cfg: &config.SubTaskConfig{SourceID: "source", Name: task2}, stage: pb.Stage_Paused, result: &pb.ProcessResult{ IsCanceled: false, @@ -259,6 +261,7 @@ func (s *testTaskCheckerSuite) TestCheckTaskIndependent(c *check.C) { }, l: log.With(zap.String("subtask", task2)), } + c.Assert(st2.cfg.Adjust(false), check.IsNil) rtsc.w.subTaskHolder.recordSubTask(st2) task1LatestResumeTime = rtsc.bc.latestResumeTime[task1] diff --git a/dm/dm/worker/utils.go b/dm/dm/worker/utils.go new file mode 100644 index 00000000000..0b4aad256b8 --- /dev/null +++ b/dm/dm/worker/utils.go @@ -0,0 +1,37 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package worker + +import ( + "go.etcd.io/etcd/clientv3" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" +) + +func getExpectValidatorStage(cfg config.ValidatorConfig, etcdClient *clientv3.Client, source, task string, revision int64) (pb.Stage, error) { + // for subtask with validation mode=none, there is no validator stage, set to invalid + expectedValidatorStage := pb.Stage_InvalidStage + if cfg.Mode != config.ValidationNone { + validatorStageM, _, err := ha.GetValidatorStage(etcdClient, source, task, revision) + if err != nil { + return expectedValidatorStage, err + } + if s, ok := validatorStageM[task]; ok { + expectedValidatorStage = s.Expect + } + } + return expectedValidatorStage, nil +} diff --git a/dm/dm/worker/utils_test.go b/dm/dm/worker/utils_test.go new file mode 100644 index 00000000000..0a63f9eb3ff --- /dev/null +++ b/dm/dm/worker/utils_test.go @@ -0,0 +1,63 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package worker + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/integration" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" +) + +func TestGetExpectValidatorStage(t *testing.T) { + mockCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer mockCluster.Terminate(t) + + etcdTestCli := mockCluster.RandClient() + defer func() { + require.Nil(t, ha.ClearTestInfoOperation(etcdTestCli)) + }() + cfg := config.SubTaskConfig{} + require.Nil(t, cfg.DecodeFile("subtask.toml", true)) + source := cfg.SourceID + task := cfg.Name + stage := ha.NewSubTaskStage(pb.Stage_Running, source, task) + + validatorStage, err := getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + cfg.ValidatorCfg.Mode = config.ValidationFast + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + // put subtask config and stage at the same time + rev, err := ha.PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []ha.Stage{stage}, []ha.Stage{stage}) + require.Nil(t, err) + + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task, 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_Running, validatorStage) + validatorStage, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task+"not exist", 0) + require.Nil(t, err) + require.Equal(t, pb.Stage_InvalidStage, validatorStage) + + _, err = getExpectValidatorStage(cfg.ValidatorCfg, etcdTestCli, source, task+"not exist", rev+1) + require.NotNil(t, err) +} diff --git a/dm/errors.toml b/dm/errors.toml index beee203ee50..9e2376a51f7 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1114,6 +1114,18 @@ description = "" workaround = "Please choose a valid value in ['replace', 'error', 'ignore']" tags = ["internal", "medium"] +[error.DM-config-20055] +message = "invalid validation mode" +description = "" +workaround = "Please check `validation-mode` config in task configuration file." +tags = ["internal", "high"] + +[error.DM-config-20056] +message = "mysql-instance(%d)'s continuous validator config %s not exist" +description = "" +workaround = "Please check the `continuous-validator-config-name` config in task configuration file." +tags = ["internal", "medium"] + [error.DM-binlog-op-22001] message = "" description = "" diff --git a/dm/pkg/ha/ops.go b/dm/pkg/ha/ops.go index 07caa515631..b6e2d2eb519 100644 --- a/dm/pkg/ha/ops.go +++ b/dm/pkg/ha/ops.go @@ -85,25 +85,26 @@ func DeleteSourceCfgRelayStageSourceBound(cli *clientv3.Client, source, worker s // - subtask config. // - subtask stage. // NOTE: golang can't use two `...` in the func, so use `[]` instead. -func PutSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { - return opSubTaskCfgStage(cli, mvccpb.PUT, cfgs, stages) +func PutSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage, validatorStages []Stage) (int64, error) { + return operateSubtask(cli, mvccpb.PUT, cfgs, stages, validatorStages) } // DeleteSubTaskCfgStage deletes the following data in one txn. // - subtask config. // - subtask stage. // NOTE: golang can't use two `...` in the func, so use `[]` instead. -func DeleteSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { - return opSubTaskCfgStage(cli, mvccpb.DELETE, cfgs, stages) +func DeleteSubTaskCfgStage(cli *clientv3.Client, cfgs []config.SubTaskConfig, stages []Stage, validatorStages []Stage) (int64, error) { + return operateSubtask(cli, mvccpb.DELETE, cfgs, stages, validatorStages) } -// opSubTaskCfgStage puts/deletes for subtask config and stage in one txn. -func opSubTaskCfgStage(cli *clientv3.Client, evType mvccpb.Event_EventType, - cfgs []config.SubTaskConfig, stages []Stage) (int64, error) { +// operateSubtask puts/deletes KVs for the subtask in one txn. +func operateSubtask(cli *clientv3.Client, evType mvccpb.Event_EventType, cfgs []config.SubTaskConfig, stages []Stage, + validatorStages []Stage) (int64, error) { var ( - ops1 []clientv3.Op - ops2 []clientv3.Op - err error + ops1 []clientv3.Op + ops2 []clientv3.Op + validatorOps []clientv3.Op + err error ) switch evType { case mvccpb.PUT: @@ -115,14 +116,20 @@ func opSubTaskCfgStage(cli *clientv3.Client, evType mvccpb.Event_EventType, if err != nil { return 0, err } + validatorOps, err = putValidatorStageOps(validatorStages...) + if err != nil { + return 0, err + } case mvccpb.DELETE: ops1 = deleteSubTaskCfgOp(cfgs...) ops2 = deleteSubTaskStageOp(stages...) + validatorOps = deleteValidatorStageOps(validatorStages...) } - ops := make([]clientv3.Op, 0, len(ops1)+len(ops2)) + ops := make([]clientv3.Op, 0, 2*len(cfgs)+len(stages)) ops = append(ops, ops1...) ops = append(ops, ops2...) + ops = append(ops, validatorOps...) _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, ops...) return rev, err } diff --git a/dm/pkg/ha/ops_test.go b/dm/pkg/ha/ops_test.go index d0730879f85..a4295a807a4 100644 --- a/dm/pkg/ha/ops_test.go +++ b/dm/pkg/ha/ops_test.go @@ -24,14 +24,15 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { defer clearTestInfoOperation(c) var ( - source = "mysql-replica-1" - worker = "dm-worker-1" - task1 = "task-1" - task2 = "task-2" - relayStage = NewRelayStage(pb.Stage_Running, source) - subtaskStage1 = NewSubTaskStage(pb.Stage_Running, source, task1) - subtaskStage2 = NewSubTaskStage(pb.Stage_Running, source, task2) - bound = NewSourceBound(source, worker) + source = "mysql-replica-1" + worker = "dm-worker-1" + task1 = "task-1" + task2 = "task-2" + relayStage = NewRelayStage(pb.Stage_Running, source) + subtaskStage1 = NewSubTaskStage(pb.Stage_Running, source, task1) + subtaskStage2 = NewSubTaskStage(pb.Stage_Running, source, task2) + validatorStage = NewSubTaskStage(pb.Stage_Running, source, task2) + bound = NewSourceBound(source, worker) emptyStage Stage subtaskCfg1 config.SubTaskConfig @@ -95,7 +96,7 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { c.Assert(scm2, HasLen, 0) // put subtask config and subtask stage. - rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}) + rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}, []Stage{validatorStage}) c.Assert(err, IsNil) c.Assert(rev6, Greater, rev5) @@ -114,9 +115,22 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { subtaskStage2.Revision = rev6 c.Assert(stsm[task1], DeepEquals, subtaskStage1) c.Assert(stsm[task2], DeepEquals, subtaskStage2) + validatorStages, rev7, err := GetValidatorStage(etcdTestCli, source, "", rev6) + c.Assert(err, IsNil) + c.Assert(rev7, Equals, rev6) + c.Assert(validatorStages, HasLen, 1) + validatorStage.Revision = rev6 + c.Assert(validatorStages[task2], DeepEquals, validatorStage) + // get with task name + validatorStages, rev7, err = GetValidatorStage(etcdTestCli, source, task2, rev6) + c.Assert(err, IsNil) + c.Assert(rev7, Equals, rev6) + c.Assert(validatorStages, HasLen, 1) + validatorStage.Revision = rev6 + c.Assert(validatorStages[task2], DeepEquals, validatorStage) // delete them. - rev8, err := DeleteSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}) + rev8, err := DeleteSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{subtaskCfg1, subtaskCfg2}, []Stage{subtaskStage1, subtaskStage2}, []Stage{validatorStage}) c.Assert(err, IsNil) c.Assert(rev8, Greater, rev7) @@ -129,4 +143,8 @@ func (t *testForEtcd) TestOpsEtcd(c *C) { c.Assert(err, IsNil) c.Assert(rev9, Equals, rev8) c.Assert(stsm, HasLen, 0) + validatorStages, rev9, err = GetValidatorStage(etcdTestCli, source, "", 0) + c.Assert(err, IsNil) + c.Assert(rev9, Equals, rev8) + c.Assert(validatorStages, HasLen, 0) } diff --git a/dm/pkg/ha/source.go b/dm/pkg/ha/source.go index 5224e5d04bb..0901362c2c5 100644 --- a/dm/pkg/ha/source.go +++ b/dm/pkg/ha/source.go @@ -134,8 +134,10 @@ func ClearTestInfoOperation(cli *clientv3.Client) error { clearRelayStage := clientv3.OpDelete(common.StageRelayKeyAdapter.Path(), clientv3.WithPrefix()) clearRelayConfig := clientv3.OpDelete(common.UpstreamRelayWorkerKeyAdapter.Path(), clientv3.WithPrefix()) clearSubTaskStage := clientv3.OpDelete(common.StageSubTaskKeyAdapter.Path(), clientv3.WithPrefix()) + clearValidatorStage := clientv3.OpDelete(common.StageValidatorKeyAdapter.Path(), clientv3.WithPrefix()) clearLoadTasks := clientv3.OpDelete(common.LoadTaskKeyAdapter.Path(), clientv3.WithPrefix()) _, _, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clearSource, clearSubTask, clearWorkerInfo, clearBound, - clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearRelayConfig, clearSubTaskStage, clearLoadTasks) + clearLastBound, clearWorkerKeepAlive, clearRelayStage, clearRelayConfig, clearSubTaskStage, clearValidatorStage, + clearLoadTasks) return err } diff --git a/dm/pkg/ha/stage.go b/dm/pkg/ha/stage.go index 781de9667bf..0cee6ce77ce 100644 --- a/dm/pkg/ha/stage.go +++ b/dm/pkg/ha/stage.go @@ -52,6 +52,10 @@ func NewSubTaskStage(expect pb.Stage, source, task string) Stage { return newStage(expect, source, task) } +func NewValidatorStage(expect pb.Stage, source, task string) Stage { + return newStage(expect, source, task) +} + // newStage creates a new Stage instance. func newStage(expect pb.Stage, source, task string) Stage { return Stage{ @@ -172,6 +176,10 @@ func GetAllRelayStage(cli *clientv3.Client) (map[string]Stage, int64, error) { // if task name is "", it will return all subtasks' stage as a map{task-name: stage} for the source. // if task name is given, it will return a map{task-name: stage} whose length is 1. func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stage, int64, error) { + return getStageByKey(cli, common.StageSubTaskKeyAdapter, source, task, 0) +} + +func getStageByKey(cli *clientv3.Client, key common.KeyAdapter, source, task string, revision int64) (map[string]Stage, int64, error) { ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) defer cancel() @@ -179,18 +187,23 @@ func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stag stm = make(map[string]Stage) resp *clientv3.GetResponse err error + opts = make([]clientv3.OpOption, 0) ) + if revision > 0 { + opts = append(opts, clientv3.WithRev(revision)) + } if task != "" { - resp, err = cli.Get(ctx, common.StageSubTaskKeyAdapter.Encode(source, task)) + resp, err = cli.Get(ctx, key.Encode(source, task), opts...) } else { - resp, err = cli.Get(ctx, common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()) + opts = append(opts, clientv3.WithPrefix()) + resp, err = cli.Get(ctx, key.Encode(source), opts...) } if err != nil { return stm, 0, err } - stages, err := subTaskStageFromResp(source, task, resp) + stages, err := getStagesFromResp(source, task, resp) if err != nil { return stm, 0, err } @@ -199,18 +212,26 @@ func GetSubTaskStage(cli *clientv3.Client, source, task string) (map[string]Stag return stm, resp.Header.Revision, nil } +func GetValidatorStage(cli *clientv3.Client, source, task string, revision int64) (map[string]Stage, int64, error) { + return getStageByKey(cli, common.StageValidatorKeyAdapter, source, task, revision) +} + // GetAllSubTaskStage gets all subtask stages. // k/v: source ID -> task name -> subtask stage. func GetAllSubTaskStage(cli *clientv3.Client) (map[string]map[string]Stage, int64, error) { + return getAllStagesInner(cli, common.StageSubTaskKeyAdapter) +} + +func getAllStagesInner(cli *clientv3.Client, key common.KeyAdapter) (map[string]map[string]Stage, int64, error) { ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) defer cancel() - resp, err := cli.Get(ctx, common.StageSubTaskKeyAdapter.Path(), clientv3.WithPrefix()) + resp, err := cli.Get(ctx, key.Path(), clientv3.WithPrefix()) if err != nil { return nil, 0, err } - stages, err := subTaskStageFromResp("", "", resp) + stages, err := getStagesFromResp("", "", resp) if err != nil { return nil, 0, err } @@ -218,34 +239,48 @@ func GetAllSubTaskStage(cli *clientv3.Client) (map[string]map[string]Stage, int6 return stages, resp.Header.Revision, nil } +func GetAllValidatorStage(cli *clientv3.Client) (map[string]map[string]Stage, int64, error) { + return getAllStagesInner(cli, common.StageValidatorKeyAdapter) +} + // GetSubTaskStageConfig gets source's subtask stages and configs at the same time // source **must not be empty** -// return map{task name -> subtask stage}, map{task name -> subtask config}, revision, error. -func GetSubTaskStageConfig(cli *clientv3.Client, source string) (map[string]Stage, map[string]config.SubTaskConfig, int64, error) { +// return map{task name -> subtask stage}, map{task name -> validator stage}, map{task name -> subtask config}, revision, error. +func GetSubTaskStageConfig(cli *clientv3.Client, source string) (map[string]Stage, map[string]Stage, map[string]config.SubTaskConfig, int64, error) { var ( - stm = make(map[string]Stage) - scm = make(map[string]config.SubTaskConfig) + stm = make(map[string]Stage) + validatorStageMap = make(map[string]Stage) + scm = make(map[string]config.SubTaskConfig) ) - txnResp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, clientv3.OpGet(common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()), + txnResp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, + clientv3.OpGet(common.StageSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix()), + clientv3.OpGet(common.StageValidatorKeyAdapter.Encode(source), clientv3.WithPrefix()), clientv3.OpGet(common.UpstreamSubTaskKeyAdapter.Encode(source), clientv3.WithPrefix())) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } stageResp := txnResp.Responses[0].GetResponseRange() - stages, err := subTaskStageFromResp(source, "", (*clientv3.GetResponse)(stageResp)) + stages, err := getStagesFromResp(source, "", (*clientv3.GetResponse)(stageResp)) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } stm = stages[source] - cfgResp := txnResp.Responses[1].GetResponseRange() + validatorStageResp := txnResp.Responses[1].GetResponseRange() + validatorStages, err := getStagesFromResp(source, "", (*clientv3.GetResponse)(validatorStageResp)) + if err != nil { + return stm, validatorStageMap, scm, 0, err + } + validatorStageMap = validatorStages[source] + + cfgResp := txnResp.Responses[2].GetResponseRange() cfgs, err := subTaskCfgFromResp(source, "", (*clientv3.GetResponse)(cfgResp)) if err != nil { - return stm, scm, 0, err + return stm, validatorStageMap, scm, 0, err } scm = cfgs[source] - return stm, scm, rev, err + return stm, validatorStageMap, scm, rev, err } // WatchRelayStage watches PUT & DELETE operations for the relay stage. @@ -268,6 +303,14 @@ func WatchSubTaskStage(ctx context.Context, cli *clientv3.Client, watchStage(ctx, ch, subTaskStageFromKey, outCh, errCh) } +func WatchValidatorStage(ctx context.Context, cli *clientv3.Client, + source string, rev int64, outCh chan<- Stage, errCh chan<- error) { + wCtx, cancel := context.WithCancel(ctx) + defer cancel() + ch := cli.Watch(wCtx, common.StageValidatorKeyAdapter.Encode(source), clientv3.WithPrefix(), clientv3.WithRev(rev)) + watchStage(ctx, ch, validatorStageFromKey, outCh, errCh) +} + // DeleteSubTaskStage deletes the subtask stage. func DeleteSubTaskStage(cli *clientv3.Client, stages ...Stage) (int64, error) { ops := deleteSubTaskStageOp(stages...) @@ -298,7 +341,18 @@ func subTaskStageFromKey(key string) (Stage, error) { return stage, nil } -func subTaskStageFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]Stage, error) { +func validatorStageFromKey(key string) (Stage, error) { + var stage Stage + ks, err := common.StageValidatorKeyAdapter.Decode(key) + if err != nil { + return stage, err + } + stage.Source = ks[0] + stage.Task = ks[1] + return stage, nil +} + +func getStagesFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]Stage, error) { stages := make(map[string]map[string]Stage) if source != "" { stages[source] = make(map[string]Stage) // avoid stages[source] is nil diff --git a/dm/pkg/ha/stage_test.go b/dm/pkg/ha/stage_test.go index a92645bf530..946dea993fb 100644 --- a/dm/pkg/ha/stage_test.go +++ b/dm/pkg/ha/stage_test.go @@ -224,23 +224,26 @@ func (t *testForEtcd) TestGetSubTaskStageConfigEtcd(c *C) { stage := NewSubTaskStage(pb.Stage_Running, source, task) // no subtask stage and config - stm, scm, rev1, err := GetSubTaskStageConfig(etcdTestCli, source) + stm, validatorM, scm, rev1, err := GetSubTaskStageConfig(etcdTestCli, source) c.Assert(err, IsNil) c.Assert(rev1, Greater, int64(0)) c.Assert(stm, HasLen, 0) + c.Assert(validatorM, HasLen, 0) c.Assert(scm, HasLen, 0) // put subtask config and stage at the same time - rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []Stage{stage}) + rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg}, []Stage{stage}, []Stage{stage}) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) // get subtask config and stage at the same time - stm, scm, rev3, err := GetSubTaskStageConfig(etcdTestCli, source) + stm, validatorM, scm, rev3, err := GetSubTaskStageConfig(etcdTestCli, source) c.Assert(err, IsNil) c.Assert(rev3, Equals, rev2) c.Assert(stm, HasLen, 1) + c.Assert(validatorM, HasLen, 1) stage.Revision = rev2 c.Assert(stm[task], DeepEquals, stage) + c.Assert(validatorM[task], DeepEquals, stage) c.Assert(scm[task], DeepEquals, cfg) } diff --git a/dm/pkg/ha/subtask.go b/dm/pkg/ha/subtask.go index 660a487bd87..80b3bdfd60e 100644 --- a/dm/pkg/ha/subtask.go +++ b/dm/pkg/ha/subtask.go @@ -98,6 +98,28 @@ func deleteSubTaskCfgOp(cfgs ...config.SubTaskConfig) []clientv3.Op { return ops } +func putValidatorStageOps(stages ...Stage) ([]clientv3.Op, error) { + ops := make([]clientv3.Op, 0, len(stages)) + for _, stage := range stages { + key := common.StageValidatorKeyAdapter.Encode(stage.Source, stage.Task) + value, err := stage.toJSON() + if err != nil { + return nil, err + } + ops = append(ops, clientv3.OpPut(key, value)) + } + return ops, nil +} + +func deleteValidatorStageOps(stages ...Stage) []clientv3.Op { + ops := make([]clientv3.Op, 0, len(stages)) + for _, stage := range stages { + key := common.StageValidatorKeyAdapter.Encode(stage.Source, stage.Task) + ops = append(ops, clientv3.OpDelete(key)) + } + return ops +} + func subTaskCfgFromResp(source, task string, resp *clientv3.GetResponse) (map[string]map[string]config.SubTaskConfig, error) { cfgs := make(map[string]map[string]config.SubTaskConfig) if source != "" { diff --git a/dm/pkg/ha/subtask_test.go b/dm/pkg/ha/subtask_test.go index eb1e9f567e3..5dc537bc131 100644 --- a/dm/pkg/ha/subtask_test.go +++ b/dm/pkg/ha/subtask_test.go @@ -47,7 +47,7 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm1, HasLen, 0) // put subtask configs. - rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1, cfg2}, []Stage{}) + rev2, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1, cfg2}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev2, Greater, rev1) @@ -92,14 +92,14 @@ func (t *testForEtcd) TestSubTaskEtcd(c *C) { c.Assert(tsm4, HasLen, 0) // put subtask config. - rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1}, []Stage{}) + rev6, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg1}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev6, Greater, int64(0)) // update subtask config. cfg3 := cfg1 cfg3.SourceID = "testForRevision" - rev7, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg3}, []Stage{}) + rev7, err := PutSubTaskCfgStage(etcdTestCli, []config.SubTaskConfig{cfg3}, []Stage{}, nil) c.Assert(err, IsNil) c.Assert(rev7, Greater, rev6) diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 04490f95306..7737bfb9374 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -249,6 +249,8 @@ const ( codeCollationCompatibleNotSupport codeConfigInvalidLoadMode codeConfigInvalidLoadDuplicateResolution + codeConfigValidationMode + codeContinuousValidatorCfgNotFound ) // Binlog operation error code list. @@ -910,6 +912,8 @@ var ( ErrConfigCollationCompatibleNotSupport = New(codeCollationCompatibleNotSupport, ClassConfig, ScopeInternal, LevelMedium, "collation compatible %s not supported", "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`.") ErrConfigInvalidLoadMode = New(codeConfigInvalidLoadMode, ClassConfig, ScopeInternal, LevelMedium, "invalid load mode '%s'", "Please choose a valid value in ['sql', 'loader']") ErrConfigInvalidDuplicateResolution = New(codeConfigInvalidLoadDuplicateResolution, ClassConfig, ScopeInternal, LevelMedium, "invalid load on-duplicate '%s'", "Please choose a valid value in ['replace', 'error', 'ignore']") + ErrConfigValidationMode = New(codeConfigValidationMode, ClassConfig, ScopeInternal, LevelHigh, "invalid validation mode", "Please check `validation-mode` config in task configuration file.") + ErrContinuousValidatorCfgNotFound = New(codeContinuousValidatorCfgNotFound, ClassConfig, ScopeInternal, LevelMedium, "mysql-instance(%d)'s continuous validator config %s not exist", "Please check the `continuous-validator-config-name` config in task configuration file.") // Binlog operation error. ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go new file mode 100644 index 00000000000..1eb368e9bac --- /dev/null +++ b/dm/syncer/data_validator.go @@ -0,0 +1,200 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + "sync" + "time" + + "github.com/go-mysql-org/go-mysql/replication" + "go.uber.org/zap" + + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" +) + +// DataValidator +// validator can be start when there's syncer unit in the subtask and validation mode is not none, +// it's terminated when the subtask is terminated. +// stage of validator is independent of subtask, pause/resume subtask doesn't affect the stage of validator. +// +// validator can be in running or stopped stage +// - in running when it's started with subtask or started later on the fly. +// - in stopped when validation stop is executed. +type DataValidator struct { + sync.RWMutex + cfg *config.SubTaskConfig + syncer *Syncer + + stage pb.Stage + wg sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + L log.Logger + fromDB *conn.BaseDB + timezone *time.Location + syncCfg replication.BinlogSyncerConfig + streamerController *StreamerController + + result pb.ProcessResult +} + +func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { + c := &DataValidator{ + cfg: cfg, + syncer: syncerObj, + stage: pb.Stage_Stopped, + } + c.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) + return c +} + +func (v *DataValidator) initialize() error { + newCtx, cancelFunc := context.WithTimeout(v.ctx, unit.DefaultInitTimeout) + defer cancelFunc() + tctx := tcontext.NewContext(newCtx, v.L) + + var err error + defer func() { + if err != nil && v.fromDB != nil { + v.fromDB.Close() + } + }() + + dbCfg := v.cfg.From + dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) + v.fromDB, err = dbconn.CreateBaseDB(&dbCfg) + if err != nil { + return err + } + + v.timezone, err = str2TimezoneOrFromDB(tctx, v.cfg.Timezone, &v.cfg.To) + if err != nil { + return err + } + + v.syncCfg, err = subtaskCfg2BinlogSyncerCfg(v.cfg, v.timezone) + if err != nil { + return err + } + + v.streamerController = NewStreamerController(v.syncCfg, v.cfg.EnableGTID, &dbconn.UpStreamConn{BaseDB: v.fromDB}, v.cfg.RelayDir, v.timezone, nil) + + return nil +} + +func (v *DataValidator) Start(expect pb.Stage) { + v.Lock() + defer v.Unlock() + + if v.stage == pb.Stage_Running { + v.L.Info("already started") + return + } + + v.ctx, v.cancel = context.WithCancel(context.Background()) + + if err := v.initialize(); err != nil { + v.fillResult(err, false) + return + } + + if expect != pb.Stage_Running { + return + } + + v.wg.Add(1) + go func() { + defer v.wg.Done() + v.doValidate() + }() + + v.stage = pb.Stage_Running +} + +func (v *DataValidator) fillResult(err error, needLock bool) { + if needLock { + v.Lock() + defer v.Unlock() + } + + var errs []*pb.ProcessError + if utils.IsContextCanceledError(err) { + v.L.Info("filter out context cancelled error", log.ShortError(err)) + } else { + errs = append(errs, unit.NewProcessError(err)) + } + + isCanceled := false + select { + case <-v.ctx.Done(): + isCanceled = true + default: + } + + v.result = pb.ProcessResult{ + IsCanceled: isCanceled, + Errors: errs, + } +} + +func (v *DataValidator) doValidate() { + tctx := tcontext.NewContext(v.ctx, v.L) + err := v.streamerController.Start(tctx, lastLocation) + if err != nil { + v.fillResult(terror.Annotate(err, "fail to restart streamer controller"), true) + return + } + + v.L.Info("start continuous validation") +} + +func (v *DataValidator) Stop() { + v.Lock() + defer v.Unlock() + if v.stage != pb.Stage_Running { + v.L.Warn("not started") + return + } + + v.streamerController.Close() + v.fromDB.Close() + + if v.cancel != nil { + v.cancel() + } + v.wg.Wait() + v.stage = pb.Stage_Stopped +} + +func (v *DataValidator) Started() bool { + v.RLock() + defer v.RUnlock() + return v.stage == pb.Stage_Running +} + +func (v *DataValidator) Stage() pb.Stage { + v.RLock() + defer v.RUnlock() + return v.stage +} diff --git a/dm/syncer/streamer_controller.go b/dm/syncer/streamer_controller.go index 4bcaf28244f..6b0f9aa6705 100644 --- a/dm/syncer/streamer_controller.go +++ b/dm/syncer/streamer_controller.go @@ -339,7 +339,7 @@ func (c *StreamerController) ReopenWithRetry(tctx *tcontext.Context, location bi } // Close closes streamer. -func (c *StreamerController) Close(tctx *tcontext.Context) { +func (c *StreamerController) Close() { c.Lock() c.close() c.Unlock() diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 6ba6cf5e487..68eb7ef9ba7 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -16,7 +16,6 @@ package syncer import ( "bytes" "context" - "crypto/tls" "fmt" "math" "os" @@ -36,7 +35,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -51,7 +49,6 @@ import ( "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" "github.com/pingcap/tiflow/dm/pkg/binlog" - "github.com/pingcap/tiflow/dm/pkg/binlog/common" "github.com/pingcap/tiflow/dm/pkg/binlog/event" "github.com/pingcap/tiflow/dm/pkg/binlog/reader" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -318,11 +315,12 @@ func (s *Syncer) Init(ctx context.Context) (err error) { }() tctx := s.tctx.WithContext(ctx) - if err = s.setTimezone(ctx); err != nil { + s.timezone, err = str2TimezoneOrFromDB(tctx, s.cfg.Timezone, &s.cfg.To) + if err != nil { return } - err = s.setSyncCfg() + s.syncCfg, err = subtaskCfg2BinlogSyncerCfg(s.cfg, s.timezone) if err != nil { return err } @@ -560,7 +558,7 @@ func (s *Syncer) IsFreshTask(ctx context.Context) (bool, error) { func (s *Syncer) reset() { if s.streamerController != nil { - s.streamerController.Close(s.tctx) + s.streamerController.Close() } // create new job chans s.newJobChans() @@ -1544,7 +1542,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { failpoint.Inject("AdjustGTIDExit", func() { tctx.L().Warn("exit triggered", zap.String("failpoint", "AdjustGTIDExit")) - s.streamerController.Close(tctx) + s.streamerController.Close() utils.OsExit(1) }) @@ -3392,7 +3390,7 @@ func (s *Syncer) stopSync() { // when resuming, re-create s.syncer if s.streamerController != nil { - s.streamerController.Close(s.tctx) + s.streamerController.Close() } } @@ -3527,7 +3525,8 @@ func (s *Syncer) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // update timezone if s.timezone == nil { - return s.setTimezone(ctx) + s.timezone, err = str2TimezoneOrFromDB(s.tctx.WithContext(ctx), s.cfg.Timezone, &s.cfg.To) + return err } return nil } @@ -3565,7 +3564,7 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { return err } - err = s.setSyncCfg() + s.syncCfg, err = subtaskCfg2BinlogSyncerCfg(s.cfg, s.timezone) if err != nil { return err } @@ -3576,59 +3575,6 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { return nil } -func (s *Syncer) setTimezone(ctx context.Context) error { - tz := s.cfg.Timezone - var err error - if len(tz) == 0 { - tz, err = conn.FetchTimeZoneSetting(ctx, &s.cfg.To) - if err != nil { - return err - } - } - loc, err := utils.ParseTimeZone(tz) - if err != nil { - return err - } - s.tctx.L().Info("use timezone", zap.String("location", loc.String())) - s.timezone = loc - return nil -} - -func (s *Syncer) setSyncCfg() error { - var tlsConfig *tls.Config - var err error - if s.cfg.From.Security != nil { - if loadErr := s.cfg.From.Security.LoadTLSContent(); loadErr != nil { - return terror.ErrCtlLoadTLSCfg.Delegate(loadErr) - } - tlsConfig, err = toolutils.ToTLSConfigWithVerifyByRawbytes(s.cfg.From.Security.SSLCABytes, - s.cfg.From.Security.SSLCertBytes, s.cfg.From.Security.SSLKEYBytes, s.cfg.From.Security.CertAllowedCN) - if err != nil { - return terror.ErrConnInvalidTLSConfig.Delegate(err) - } - if tlsConfig != nil { - tlsConfig.InsecureSkipVerify = true - } - } - - syncCfg := replication.BinlogSyncerConfig{ - ServerID: s.cfg.ServerID, - Flavor: s.cfg.Flavor, - Host: s.cfg.From.Host, - Port: uint16(s.cfg.From.Port), - User: s.cfg.From.User, - Password: s.cfg.From.Password, - TimestampStringLocation: s.timezone, - TLSConfig: tlsConfig, - } - // when retry count > 1, go-mysql will retry sync from the previous GTID set in GTID mode, - // which may get duplicate binlog event after retry success. so just set retry count = 1, and task - // will exit when meet error, and then auto resume by DM itself. - common.SetDefaultReplicationCfg(&syncCfg, 1) - s.syncCfg = syncCfg - return nil -} - // ShardDDLOperation returns the current pending to handle shard DDL lock operation. func (s *Syncer) ShardDDLOperation() *pessimism.Operation { return s.pessimist.PendingOperation() @@ -3708,7 +3654,7 @@ func (s *Syncer) adjustGlobalPointGTID(tctx *tcontext.Context) (bool, error) { if err != nil { return false, err } - defer streamerController.Close(tctx) + defer streamerController.Close() gs, err := reader.GetGTIDsForPosFromStreamer(tctx.Context(), streamerController.streamer, endPos) if err != nil { diff --git a/dm/syncer/util.go b/dm/syncer/util.go index 384dc5d0b7a..7b27b71af40 100644 --- a/dm/syncer/util.go +++ b/dm/syncer/util.go @@ -14,18 +14,25 @@ package syncer import ( + "crypto/tls" "fmt" + "time" + "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/tidb-tools/pkg/filter" + toolutils "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/dumpling/export" dlog "github.com/pingcap/tidb/dumpling/log" "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/relay" ) @@ -118,3 +125,53 @@ func printServerVersion(tctx *tcontext.Context, db *conn.BaseDB, scope string) { } version.ParseServerInfo(versionInfo) } + +func str2TimezoneOrFromDB(tctx *tcontext.Context, tzStr string, dbCfg *config.DBConfig) (*time.Location, error) { + var err error + if len(tzStr) == 0 { + tzStr, err = conn.FetchTimeZoneSetting(tctx.Ctx, dbCfg) + if err != nil { + return nil, err + } + } + loc, err := utils.ParseTimeZone(tzStr) + if err != nil { + return nil, err + } + tctx.L().Info("use timezone", zap.String("location", loc.String())) + return loc, nil +} + +func subtaskCfg2BinlogSyncerCfg(cfg *config.SubTaskConfig, timezone *time.Location) (replication.BinlogSyncerConfig, error) { + var tlsConfig *tls.Config + var err error + if cfg.From.Security != nil { + if loadErr := cfg.From.Security.LoadTLSContent(); loadErr != nil { + return replication.BinlogSyncerConfig{}, terror.ErrCtlLoadTLSCfg.Delegate(loadErr) + } + tlsConfig, err = toolutils.ToTLSConfigWithVerifyByRawbytes(cfg.From.Security.SSLCABytes, + cfg.From.Security.SSLCertBytes, cfg.From.Security.SSLKEYBytes, cfg.From.Security.CertAllowedCN) + if err != nil { + return replication.BinlogSyncerConfig{}, terror.ErrConnInvalidTLSConfig.Delegate(err) + } + if tlsConfig != nil { + tlsConfig.InsecureSkipVerify = true + } + } + + syncCfg := replication.BinlogSyncerConfig{ + ServerID: cfg.ServerID, + Flavor: cfg.Flavor, + Host: cfg.From.Host, + Port: uint16(cfg.From.Port), + User: cfg.From.User, + Password: cfg.From.Password, + TimestampStringLocation: timezone, + TLSConfig: tlsConfig, + } + // when retry count > 1, go-mysql will retry sync from the previous GTID set in GTID mode, + // which may get duplicate binlog event after retry success. so just set retry count = 1, and task + // will exit when meet error, and then auto resume by DM itself. + common.SetDefaultReplicationCfg(&syncCfg, 1) + return syncCfg, nil +} diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index 30aa6e566a5..3a431dbdf37 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -41,6 +41,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 - source-id: mysql-replica-02 meta: null filter-rules: @@ -62,6 +63,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 online-ddl: false shadow-table-rules: [] trash-table-rules: [] @@ -159,6 +161,9 @@ syncers: disable-detect: false safe-mode: false enable-ansi-quotes: false +validators: + validator-01: + mode: none clean-dump-file: true ansi-quotes: false remove-meta: false diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index a8756979ed2..10e8f111563 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -40,6 +40,7 @@ mysql-instances: syncer-config-name: sync-01 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 - source-id: mysql-replica-02 meta: null filter-rules: [] @@ -57,6 +58,7 @@ mysql-instances: syncer-config-name: sync-02 syncer: null syncer-thread: 0 + continuous-validator-config-name: validator-01 online-ddl: false shadow-table-rules: [] trash-table-rules: [] @@ -118,6 +120,9 @@ syncers: disable-detect: false safe-mode: false enable-ansi-quotes: false +validators: + validator-01: + mode: none clean-dump-file: false ansi-quotes: false remove-meta: false From e23016083cdde9568805439df2c244efe92b8e04 Mon Sep 17 00:00:00 2001 From: Ehco Date: Mon, 14 Feb 2022 16:25:38 +0800 Subject: [PATCH 58/72] test(dm): fix unsteable test TestRedirectRequestToLeader (#4556) close pingcap/tiflow#4554 --- dm/dm/master/openapi_test.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dm/dm/master/openapi_test.go b/dm/dm/master/openapi_test.go index 9638136fa99..453a485bdce 100644 --- a/dm/dm/master/openapi_test.go +++ b/dm/dm/master/openapi_test.go @@ -93,10 +93,11 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { s1 := NewServer(cfg1) c.Assert(s1.Start(ctx), check.IsNil) defer s1.Close() + defer cancel() // wait the first one become the leader c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s1.election.IsLeader() + return s1.election.IsLeader() && s1.scheduler.Started() }), check.IsTrue) // join to an existing cluster @@ -113,6 +114,12 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { s2 := NewServer(cfg2) c.Assert(s2.Start(ctx), check.IsNil) defer s2.Close() + defer cancel() // this cancel must call before s.Close() to avoid deadlock + + // wait the second master ready + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return s2.election.IsLeader() + }), check.IsFalse) baseURL := "/api/v1/sources" // list source from leader @@ -128,7 +135,6 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { // list source not from leader will get a redirect result2 := testutil.NewRequest().Get(baseURL).GoWithHTTPHandler(t.testT, s2.openapiHandles) c.Assert(result2.Code(), check.Equals, http.StatusTemporaryRedirect) - cancel() } func (t *openAPISuite) TestOpenAPIWillNotStartInDefaultConfig(c *check.C) { @@ -151,7 +157,7 @@ func (t *openAPISuite) TestOpenAPIWillNotStartInDefaultConfig(c *check.C) { }), check.IsTrue) c.Assert(s1.openapiHandles, check.IsNil) defer s1.Close() - cancel() + defer cancel() } func (t *openAPISuite) TestSourceAPI(c *check.C) { From 7ccbdff7ee36d71b092c13ac386e886cb56daa19 Mon Sep 17 00:00:00 2001 From: Ehco Date: Mon, 14 Feb 2022 17:25:38 +0800 Subject: [PATCH 59/72] test(dm): fix unsteable test TestFailToStartLeader (#4566) close pingcap/tiflow#4565 --- dm/dm/master/election_test.go | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/dm/dm/master/election_test.go b/dm/dm/master/election_test.go index a831726accd..ad13d0d4d9b 100644 --- a/dm/dm/master/election_test.go +++ b/dm/dm/master/election_test.go @@ -33,7 +33,17 @@ type testElectionSuite struct{} func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) - defer cancel() + + var s1, s2 *Server + defer func() { + cancel() + if s1 != nil { + s1.Close() + } + if s2 != nil { + s2.Close() + } + }() // create a new cluster cfg1 := NewConfig() @@ -45,13 +55,11 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { cfg1.AdvertisePeerUrls = cfg1.PeerUrls cfg1.InitialCluster = fmt.Sprintf("%s=%s", cfg1.Name, cfg1.AdvertisePeerUrls) - s1 := NewServer(cfg1) + s1 = NewServer(cfg1) c.Assert(s1.Start(ctx), check.IsNil) - defer s1.Close() - // wait the first one become the leader c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s1.election.IsLeader() + return s1.election.IsLeader() && s1.scheduler.Started() }), check.IsTrue) // join to an existing cluster @@ -64,9 +72,12 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { cfg2.AdvertisePeerUrls = cfg2.PeerUrls cfg2.Join = cfg1.MasterAddr // join to an existing cluster - s2 := NewServer(cfg2) + s2 = NewServer(cfg2) c.Assert(s2.Start(ctx), check.IsNil) - defer s2.Close() + // wait the second master ready + c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return s2.election.IsLeader() + }), check.IsFalse) client, err := etcdutil.CreateClient(strings.Split(cfg1.AdvertisePeerUrls, ","), nil) c.Assert(err, check.IsNil) @@ -103,6 +114,4 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { c.Assert(err, check.IsNil) c.Assert(leaderID, check.Equals, cfg2.Name) c.Assert(clusterID, check.Equals, s2.ClusterID()) - - cancel() } From 803aa9274df8e8ac9f304c7856277c9bf3386eff Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 14 Feb 2022 17:55:38 +0800 Subject: [PATCH 60/72] syncer(dm): implement start-task --start-time (#4485) close pingcap/tiflow#4106 --- dm/_utils/terror_gen/errors_release.txt | 1 + dm/dm/config/task.go | 9 +- dm/dm/config/task_cli_args.go | 11 ++- dm/dm/master/server.go | 12 ++- dm/dm/worker/source_worker.go | 2 +- dm/errors.toml | 6 ++ dm/pkg/binlog/pos_finder.go | 3 + dm/pkg/terror/error_list.go | 2 + dm/syncer/checkpoint.go | 40 ++++++++- dm/syncer/syncer.go | 103 ++++++++++++++++++++-- dm/tests/duplicate_event/run.sh | 1 + dm/tests/start_task/conf/dm-worker2.toml | 2 + dm/tests/start_task/run.sh | 106 +++++++++++++++++++++++ 13 files changed, 282 insertions(+), 16 deletions(-) create mode 100644 dm/tests/start_task/conf/dm-worker2.toml diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index fda955674ea..1a772597b12 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -186,6 +186,7 @@ ErrConfigInvalidLoadMode,[code=20053:class=config:scope=internal:level=medium], ErrConfigInvalidDuplicateResolution,[code=20054:class=config:scope=internal:level=medium], "Message: invalid load on-duplicate '%s', Workaround: Please choose a valid value in ['replace', 'error', 'ignore']" ErrConfigValidationMode,[code=20055:class=config:scope=internal:level=high], "Message: invalid validation mode, Workaround: Please check `validation-mode` config in task configuration file." ErrContinuousValidatorCfgNotFound,[code=20056:class=config:scope=internal:level=medium], "Message: mysql-instance(%d)'s continuous validator config %s not exist, Workaround: Please check the `continuous-validator-config-name` config in task configuration file." +ErrConfigStartTimeTooLate,[code=20057:class=config:scope=internal:level=high], "Message: start-time %s is too late, no binlog location matches it, Workaround: Please check the `--start-time` is expected or try again later." ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 0a1601aa153..69f536236e0 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -514,7 +514,14 @@ const ( validatorIdx ) -// adjust adjusts and verifies config. +// Adjust adjusts and verifies config. +func (c *TaskConfig) Adjust() error { + if c == nil { + return terror.ErrConfigYamlTransform.New("task config is nil") + } + return c.adjust() +} + func (c *TaskConfig) adjust() error { if len(c.Name) == 0 { return terror.ErrConfigNeedUniqueTaskName.Generate() diff --git a/dm/dm/config/task_cli_args.go b/dm/dm/config/task_cli_args.go index 9f1bcedf420..b8c0a09d6ef 100644 --- a/dm/dm/config/task_cli_args.go +++ b/dm/dm/config/task_cli_args.go @@ -20,6 +20,11 @@ import ( "github.com/pingcap/tiflow/dm/pkg/terror" ) +const ( + StartTimeFormat = "2006-01-02 15:04:05" + StartTimeFormat2 = "2006-01-02T15:04:05" +) + // TaskCliArgs is the task command line arguments, these arguments have higher priority than the config file and // downstream checkpoint, but may need to be removed after the first time they take effect. type TaskCliArgs struct { @@ -46,10 +51,10 @@ func (t *TaskCliArgs) Verify() error { if t.StartTime == "" { return nil } - _, err := time.Parse("2006-01-02 15:04:05", t.StartTime) + _, err := time.Parse(StartTimeFormat, t.StartTime) if err == nil { return nil } - _, err = time.Parse("2006-01-02T15:04:05", t.StartTime) - return terror.Annotate(err, "error while parse start-time, expected in the format like '2006-01-02 15:04:05'") + _, err = time.Parse(StartTimeFormat2, t.StartTime) + return terror.Annotate(err, "error while parse start-time, expected in the format like '2006-01-02 15:04:05' or '2006-01-02T15:04:05'") } diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 310a6836d7a..365c1a84964 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tiflow/dm/dm/master/workerrpc" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/cputil" @@ -1545,14 +1546,21 @@ func (s *Server) generateSubTask( task string, cliArgs *config.TaskCliArgs, ) (*config.TaskConfig, []*config.SubTaskConfig, error) { + var err error cfg := config.NewTaskConfig() // bypass the meta check by set any value. If start-time is specified, DM-worker will not use meta field. if cliArgs != nil && cliArgs.StartTime != "" { + err = cfg.RawDecode(task) + if err != nil { + return nil, nil, terror.WithClass(err, terror.ClassDMMaster) + } for _, inst := range cfg.MySQLInstances { - inst.Meta = &config.Meta{BinLogName: cliArgs.StartTime} + inst.Meta = &config.Meta{BinLogName: binlog.FakeBinlogName} } + err = cfg.Adjust() + } else { + err = cfg.Decode(task) } - err := cfg.Decode(task) if err != nil { return nil, nil, terror.WithClass(err, terror.ClassDMMaster) } diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index b322602bc2e..851a3245317 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -551,7 +551,7 @@ func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage, vali } // directly put cfg into subTaskHolder - // the unique of subtask should be assured by etcd + // the uniqueness of subtask should be assured by etcd st := NewSubTask(cfg, w.etcdClient, w.name) w.subTaskHolder.recordSubTask(st) if w.closed.Load() { diff --git a/dm/errors.toml b/dm/errors.toml index 9e2376a51f7..c7cee3f0b32 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1126,6 +1126,12 @@ description = "" workaround = "Please check the `continuous-validator-config-name` config in task configuration file." tags = ["internal", "medium"] +[error.DM-config-20057] +message = "start-time %s is too late, no binlog location matches it" +description = "" +workaround = "Please check the `--start-time` is expected or try again later." +tags = ["internal", "high"] + [error.DM-binlog-op-22001] message = "" description = "" diff --git a/dm/pkg/binlog/pos_finder.go b/dm/pkg/binlog/pos_finder.go index ccd35bccebb..8f911f8253e 100644 --- a/dm/pkg/binlog/pos_finder.go +++ b/dm/pkg/binlog/pos_finder.go @@ -30,6 +30,9 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" ) +// FakeBinlogName is used to bypass the checking of meta in task config when start-task with --start-time. +const FakeBinlogName = "start-task with --start-time" + type binlogPosFinder struct { remote bool tctx *tcontext.Context diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 7737bfb9374..7e8cdf03058 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -251,6 +251,7 @@ const ( codeConfigInvalidLoadDuplicateResolution codeConfigValidationMode codeContinuousValidatorCfgNotFound + codeConfigStartTimeTooLate ) // Binlog operation error code list. @@ -914,6 +915,7 @@ var ( ErrConfigInvalidDuplicateResolution = New(codeConfigInvalidLoadDuplicateResolution, ClassConfig, ScopeInternal, LevelMedium, "invalid load on-duplicate '%s'", "Please choose a valid value in ['replace', 'error', 'ignore']") ErrConfigValidationMode = New(codeConfigValidationMode, ClassConfig, ScopeInternal, LevelHigh, "invalid validation mode", "Please check `validation-mode` config in task configuration file.") ErrContinuousValidatorCfgNotFound = New(codeContinuousValidatorCfgNotFound, ClassConfig, ScopeInternal, LevelMedium, "mysql-instance(%d)'s continuous validator config %s not exist", "Please check the `continuous-validator-config-name` config in task configuration file.") + ErrConfigStartTimeTooLate = New(codeConfigStartTimeTooLate, ClassConfig, ScopeInternal, LevelHigh, "start-time %s is too late, no binlog location matches it", "Please check the `--start-time` is expected or try again later.") // Binlog operation error. ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 833bde9f798..c5b12461287 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -25,6 +25,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" @@ -227,6 +228,9 @@ type CheckPoint interface { // DeleteTablePoint deletes checkpoint for specified table in memory and storage DeleteTablePoint(tctx *tcontext.Context, table *filter.Table) error + // DeleteAllTablePoint deletes all checkpoints for table in memory and storage + DeleteAllTablePoint(tctx *tcontext.Context) error + // DeleteSchemaPoint deletes checkpoint for specified schema DeleteSchemaPoint(tctx *tcontext.Context, sourceSchema string) error @@ -237,10 +241,13 @@ type CheckPoint interface { // corresponding to Meta.Save SaveGlobalPoint(point binlog.Location) + // SaveGlobalPointForcibly saves the global binlog stream's checkpoint forcibly. + SaveGlobalPointForcibly(location binlog.Location) + // Snapshot make a snapshot of current checkpoint Snapshot(isSyncFlush bool) *SnapshotInfo - // FlushGlobalPointsExcept flushes the global checkpoint and tables' + // FlushPointsExcept flushes the global checkpoint and tables' // checkpoints except exceptTables, it also flushes SQLs with Args providing // by extraSQLs and extraArgs. Currently extraSQLs contain shard meta only. // @exceptTables: [[schema, table]... ] @@ -551,6 +558,26 @@ func (cp *RemoteCheckPoint) DeleteTablePoint(tctx *tcontext.Context, table *filt return nil } +// DeleteAllTablePoint implements CheckPoint.DeleteAllTablePoint. +func (cp *RemoteCheckPoint) DeleteAllTablePoint(tctx *tcontext.Context) error { + cp.Lock() + defer cp.Unlock() + + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) + defer cancel() + cp.logCtx.L().Info("delete all table checkpoint") + _, err := cp.dbConn.ExecuteSQL( + tctx2, + []string{`DELETE FROM ` + cp.tableName + ` WHERE id = ? AND is_global = ?`}, + []interface{}{cp.id, false}, + ) + if err != nil { + return err + } + cp.points = make(map[string]map[string]*binlogPoint) + return nil +} + // DeleteSchemaPoint implements CheckPoint.DeleteSchemaPoint. func (cp *RemoteCheckPoint) DeleteSchemaPoint(tctx *tcontext.Context, sourceSchema string) error { cp.Lock() @@ -614,7 +641,16 @@ func (cp *RemoteCheckPoint) SaveGlobalPoint(location binlog.Location) { } } -// FlushPointsExcept implements CheckPoint.FlushSnapshotPointsExcept. +// SaveGlobalPointForcibly implements CheckPoint.SaveGlobalPointForcibly. +func (cp *RemoteCheckPoint) SaveGlobalPointForcibly(location binlog.Location) { + cp.Lock() + defer cp.Unlock() + + cp.logCtx.L().Info("reset global checkpoint", zap.Stringer("location", location)) + cp.globalPoint = newBinlogPoint(location, binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) +} + +// FlushPointsExcept implements CheckPoint.FlushPointsExcept. func (cp *RemoteCheckPoint) FlushPointsExcept( tctx *tcontext.Context, snapshotID int, diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 68eb7ef9ba7..96ebb22d793 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -125,6 +125,7 @@ type Syncer struct { cfg *config.SubTaskConfig syncCfg replication.BinlogSyncerConfig + cliArgs *config.TaskCliArgs sgk *ShardingGroupKeeper // keeper to keep all sharding (sub) group in this syncer pessimist *shardddl.Pessimist // shard DDL pessimist @@ -439,11 +440,13 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } // when Init syncer, set active relay log info - err = s.setInitActiveRelayLog(ctx) - if err != nil { - return err + if s.cfg.Meta == nil || s.cfg.Meta.BinLogName != binlog.FakeBinlogName { + err = s.setInitActiveRelayLog(ctx) + if err != nil { + return err + } + rollbackHolder.Add(fr.FuncRollback{Name: "remove-active-realylog", Fn: s.removeActiveRelayLog}) } - rollbackHolder.Add(fr.FuncRollback{Name: "remove-active-realylog", Fn: s.removeActiveRelayLog}) s.reset() return nil @@ -1257,6 +1260,17 @@ func (s *Syncer) afterFlushCheckpoint(task *checkpointFlushTask) error { s.lastCheckpointFlushedTime = now s.logAndClearFilteredStatistics() + + if s.cliArgs != nil && s.cliArgs.StartTime != "" { + clone := *s.cliArgs + clone.StartTime = "" + err2 := ha.PutTaskCliArgs(s.cli, s.cfg.Name, []string{s.cfg.SourceID}, clone) + if err2 != nil { + s.tctx.L().Error("failed to clean start-time in task cli args", zap.Error(err2)) + } else { + s.cliArgs.StartTime = "" + } + } return nil } @@ -1477,11 +1491,30 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } }() - // some initialization that can't be put in Syncer.Init fresh, err := s.IsFreshTask(runCtx) if err != nil { return err - } else if fresh { + } + + // task command line arguments have the highest priority + // dm-syncer and other usage may not have a etcdCli, so we check it first + skipLoadMeta := false + if s.cli != nil { + s.cliArgs, err = ha.GetTaskCliArgs(s.cli, s.cfg.Name, s.cfg.SourceID) + if err != nil { + s.tctx.L().Error("failed to get task cli args", zap.Error(err)) + } + if s.cliArgs != nil && s.cliArgs.StartTime != "" { + err = s.setGlobalPointByTime(tctx, s.cliArgs.StartTime) + if terror.ErrConfigStartTimeTooLate.Equal(err) { + return err + } + skipLoadMeta = err == nil + } + } + + // some initialization that can't be put in Syncer.Init + if fresh && !skipLoadMeta { // for fresh task, we try to load checkpoints from meta (file or config item) err = s.checkpoint.LoadMeta() if err != nil { @@ -3636,7 +3669,8 @@ func (s *Syncer) adjustGlobalPointGTID(tctx *tcontext.Context) (bool, error) { // 1. GTID is not enabled // 2. location already has GTID position // 3. location is totally new, has no position info - if !s.cfg.EnableGTID || location.GTIDSetStr() != "" || location.Position.Name == "" { + // 4. location is too early thus not a COMMIT location, which happens when it's reset by other logic + if !s.cfg.EnableGTID || location.GTIDSetStr() != "" || location.Position.Name == "" || location.Position.Pos == 4 { return false, nil } // set enableGTID to false for new streamerController @@ -3728,3 +3762,58 @@ func (s *Syncer) flushOptimisticTableInfos(tctx *tcontext.Context) { tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) } } + +func (s *Syncer) setGlobalPointByTime(tctx *tcontext.Context, timeStr string) error { + // we support two layout + t, err := time.ParseInLocation(config.StartTimeFormat, timeStr, s.timezone) + if err != nil { + t, err = time.ParseInLocation(config.StartTimeFormat2, timeStr, s.timezone) + } + if err != nil { + return err + } + + var ( + loc *binlog.Location + posTp binlog.PosType + ) + + if s.relay != nil { + subDir := s.relay.Status(nil).(*pb.RelayStatus).RelaySubDir + relayDir := path.Join(s.cfg.RelayDir, subDir) + finder := binlog.NewLocalBinlogPosFinder(tctx, s.cfg.EnableGTID, s.cfg.Flavor, relayDir) + loc, posTp, err = finder.FindByTimestamp(t.Unix()) + } else { + finder := binlog.NewRemoteBinlogPosFinder(tctx, s.fromDB.BaseDB.DB, s.syncCfg, s.cfg.EnableGTID) + loc, posTp, err = finder.FindByTimestamp(t.Unix()) + } + if err != nil { + s.tctx.L().Error("fail to find binlog position by timestamp", + zap.Time("time", t), + zap.Error(err)) + return err + } + + switch posTp { + case binlog.InRangeBinlogPos: + s.tctx.L().Info("find binlog position by timestamp", + zap.String("time", timeStr), + zap.Stringer("pos", loc)) + case binlog.BelowLowerBoundBinlogPos: + s.tctx.L().Warn("fail to find binlog location by timestamp because the timestamp is too early, will use the earliest binlog location", + zap.String("time", timeStr), + zap.Any("location", loc)) + case binlog.AboveUpperBoundBinlogPos: + return terror.ErrConfigStartTimeTooLate.Generate(timeStr) + } + + err = s.checkpoint.DeleteAllTablePoint(tctx) + if err != nil { + return err + } + s.checkpoint.SaveGlobalPointForcibly(*loc) + s.tctx.L().Info("Will replicate from the specified time, the location recorded in checkpoint and config file will be ignored", + zap.String("time", timeStr), + zap.Any("locationOfTheTime", loc)) + return nil +} diff --git a/dm/tests/duplicate_event/run.sh b/dm/tests/duplicate_event/run.sh index 5d15329d025..85561e9f559 100644 --- a/dm/tests/duplicate_event/run.sh +++ b/dm/tests/duplicate_event/run.sh @@ -76,6 +76,7 @@ function run_with_prepared_source_config() { server_uuid=$(tail -n 1 $WORK_DIR/worker2/relay-dir/server-uuid.index) relay_log_size=$(ls -al $WORK_DIR/worker2/relay-dir/$server_uuid/$binlog_file | awk '{print $5}') + echo "binlog_pos: $binlog_pos relay_log_size: $relay_log_size" [ "$binlog_pos" -eq "$relay_log_size" ] echo "============== run_with_prepared_source_config success ===================" diff --git a/dm/tests/start_task/conf/dm-worker2.toml b/dm/tests/start_task/conf/dm-worker2.toml new file mode 100644 index 00000000000..010e21c73eb --- /dev/null +++ b/dm/tests/start_task/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" diff --git a/dm/tests/start_task/run.sh b/dm/tests/start_task/run.sh index 90e51cb24f7..c725983852f 100644 --- a/dm/tests/start_task/run.sh +++ b/dm/tests/start_task/run.sh @@ -62,7 +62,113 @@ function lazy_init_tracker() { cleanup_process } +function start_task_by_time() { + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(0)' + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + 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 + dmctl_operate_source create $cur/conf/source1.yaml $SOURCE_ID1 + + run_sql_source1 'DROP DATABASE if exists start_task;' + run_sql_source1 'CREATE DATABASE start_task;' + run_sql_source1 'CREATE TABLE start_task.t1 (c INT PRIMARY KEY);' + + sleep 2 + start_time=$(date '+%Y-%m-%d %T') # 2022-01-26 17:32:22 + sleep 2 + + run_sql_source1 'CREATE TABLE start_task.t2 (c INT PRIMARY KEY);' + run_sql_source1 'INSERT INTO start_task.t2 VALUES (1), (2);INSERT INTO start_task.t2 VALUES (3), (4);' + + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml + sed -i "s/task-mode: all/task-mode: incremental/g" $WORK_DIR/dm-task.yaml + + # test with relay + + run_sql_tidb 'DROP DATABASE if exists start_task;CREATE DATABASE start_task;' + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml --start-time '$start_time'" \ + "\"result\": true" 2 + + run_sql_tidb_with_retry "show tables in start_task;" "t2" + run_sql_tidb_with_retry "SELECT count(1) FROM information_schema.tables WHERE table_schema = 'start_task';" "count(1): 1" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 2 + + # test without relay and safe mode + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-relay -s $SOURCE_ID1" \ + "\"result\": true" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"relayStatus\": null" 1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "transfer-source $SOURCE_ID1 worker2" \ + "\"result\": true" 1 + + run_sql_tidb 'DROP DATABASE if exists start_task;CREATE DATABASE start_task;' + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml --start-time '$start_time'" \ + "\"result\": true" 2 + + run_sql_tidb_with_retry "show tables in start_task;" "t2" + run_sql_tidb_with_retry "SELECT count(1) FROM information_schema.tables WHERE table_schema = 'start_task';" "count(1): 1" + + # no duplicate entry error + check_log_contain_with_retry "enable safe-mode for safe mode exit point, will exit at" $WORK_DIR/worker2/log/dm-worker.log + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"result\": true" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 2 + + # test too early + + run_sql_tidb 'DROP DATABASE if exists start_task;CREATE DATABASE start_task;' + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml --start-time '1995-03-07 01:02:03'" \ + "\"result\": true" 2 + + run_sql_tidb_with_retry "show tables in start_task;" "t1" + run_sql_tidb_with_retry "SELECT count(1) FROM information_schema.tables WHERE table_schema = 'start_task';" "count(1): 2" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 2 + + # test too late + + run_sql_tidb 'DROP DATABASE if exists start_task;CREATE DATABASE start_task;' + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml --start-time '2037-12-12 01:02:03'" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Paused\"" 1 \ + "no binlog location matches it" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 2 + + export GO_FAILPOINTS='' + cleanup_process + cleanup_data start_task +} + function run() { + start_task_by_time lazy_init_tracker failpoints=( # 1152 is ErrAbortingConnection From 1fb41f91bb048860c924b4752dee4f22137635d2 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Mon, 14 Feb 2022 19:13:38 +0800 Subject: [PATCH 61/72] checker(dm): support optimistic checker (#4329) close pingcap/tiflow#4328 --- dm/checker/check_test.go | 63 ++++++++-- dm/checker/checker.go | 52 ++++++-- dm/dm/worker/subtask.go | 4 +- dm/pkg/checker/table_structure.go | 127 ++++++++++++++++++++ dm/pkg/checker/table_structure_test.go | 45 +++++++ dm/tests/dmctl_basic/conf/diff_config2.toml | 50 ++++++++ dm/tests/dmctl_basic/conf/dm-task4.yaml | 63 ++++++++++ dm/tests/dmctl_basic/data/db1.prepare.sql | 3 + dm/tests/dmctl_basic/run.sh | 3 + go.mod | 1 + go.sum | 3 +- 11 files changed, 395 insertions(+), 19 deletions(-) create mode 100644 dm/tests/dmctl_basic/conf/diff_config2.toml create mode 100644 dm/tests/dmctl_basic/conf/dm-task4.yaml diff --git a/dm/checker/check_test.go b/dm/checker/check_test.go index 45668d2354a..58f8ae372a8 100644 --- a/dm/checker/check_test.go +++ b/dm/checker/check_test.go @@ -19,11 +19,14 @@ import ( "testing" "github.com/DATA-DOG/go-sqlmock" - + gmysql "github.com/go-sql-driver/mysql" router "github.com/pingcap/tidb-tools/pkg/table-router" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/ctl/common" "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" tc "github.com/pingcap/check" ) @@ -37,9 +40,11 @@ type testCheckerSuite struct{} var _ = tc.Suite(&testCheckerSuite{}) var ( - schema = "db_1" - tb1 = "t_1" - tb2 = "t_2" + schema = "db_1" + tb1 = "t_1" + tb2 = "t_2" + metaSchema = "dm_meta" + taskName = "test" ) func ignoreExcept(itemMap map[string]struct{}) []string { @@ -287,6 +292,9 @@ func (s *testCheckerSuite) TestTableSchemaChecking(c *tc.C) { func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { cfgs := []*config.SubTaskConfig{ { + MetaSchema: metaSchema, + Name: taskName, + ShardMode: config.ShardPessimistic, RouteRules: []*router.TableRule{ { SchemaPattern: schema, @@ -304,11 +312,16 @@ func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { b int(11) DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=latin1` createTable2 := `CREATE TABLE %s ( - id int(11) DEFAULT NULL, - c int(11) DEFAULT NULL - ) ENGINE=InnoDB DEFAULT CHARSET=latin1` - + id int(11) DEFAULT NULL, + c int(11) DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + errNoSuchTable := &gmysql.MySQLError{Number: mysql.ErrNoSuchTable} + createTableSQL := "SHOW CREATE TABLE `%s`.`%s`" + // test different column definition mock := initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LoaderCheckpoint(taskName))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LightningCheckpoint(taskName))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.SyncerCheckpoint(taskName))).WillReturnError(errNoSuchTable) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) @@ -320,7 +333,11 @@ func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { c.Assert(err, tc.ErrorMatches, "(.|\n)*different column definition(.|\n)*") c.Assert(len(msg), tc.Equals, 0) + // test success check mock = initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LoaderCheckpoint(taskName))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LightningCheckpoint(taskName))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.SyncerCheckpoint(taskName))).WillReturnError(errNoSuchTable) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) @@ -331,11 +348,24 @@ func (s *testCheckerSuite) TestShardTableSchemaChecking(c *tc.C) { msg, err = CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) c.Assert(err, tc.IsNil) c.Assert(msg, tc.Equals, CheckTaskSuccess) + + // test exist checkpoint + mock = initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LoaderCheckpoint(taskName))).WillReturnRows(sqlmock. + NewRows([]string{"Table", "Create Table"}).AddRow(cputil.LoaderCheckpoint(taskName), "")) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.LightningCheckpoint(taskName))).WillReturnRows(sqlmock. + NewRows([]string{"Table", "Create Table"}).AddRow(cputil.LightningCheckpoint(taskName), "")) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, metaSchema, cputil.SyncerCheckpoint(taskName))).WillReturnRows(sqlmock. + NewRows([]string{"Table", "Create Table"}).AddRow(cputil.SyncerCheckpoint(taskName), "")) + msg, err = CheckSyncConfig(context.Background(), cfgs, common.DefaultErrorCnt, common.DefaultWarnCnt) + c.Assert(msg, tc.Equals, CheckTaskSuccess) + c.Assert(err, tc.IsNil) } func (s *testCheckerSuite) TestShardAutoIncrementIDChecking(c *tc.C) { cfgs := []*config.SubTaskConfig{ { + ShardMode: config.ShardPessimistic, RouteRules: []*router.TableRule{ { SchemaPattern: schema, @@ -362,7 +392,12 @@ func (s *testCheckerSuite) TestShardAutoIncrementIDChecking(c *tc.C) { UNIQUE KEY u_b(b) ) ENGINE=InnoDB DEFAULT CHARSET=latin1` + errNoSuchTable := &gmysql.MySQLError{Number: mysql.ErrNoSuchTable} + createTableSQL := "SHOW CREATE TABLE `%s`.`%s`" mock := initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LoaderCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LightningCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.SyncerCheckpoint(""))).WillReturnError(errNoSuchTable) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) @@ -375,6 +410,9 @@ func (s *testCheckerSuite) TestShardAutoIncrementIDChecking(c *tc.C) { c.Assert(err, tc.IsNil) mock = initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LoaderCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LightningCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.SyncerCheckpoint(""))).WillReturnError(errNoSuchTable) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable2, tb1))) mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("max_connections", "2")) @@ -413,8 +451,15 @@ func (s *testCheckerSuite) TestSameTargetTableDetection(c *tc.C) { PRIMARY KEY (id), UNIQUE KEY u_b(b) ) ENGINE=InnoDB DEFAULT CHARSET=latin1` - + errNoSuchTable := &gmysql.MySQLError{Number: mysql.ErrNoSuchTable} + createTableSQL := "SHOW CREATE TABLE `%s`.`%s`" mock := initMockDB(c) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LoaderCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LightningCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.SyncerCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LoaderCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.LightningCheckpoint(""))).WillReturnError(errNoSuchTable) + mock.ExpectQuery(fmt.Sprintf(createTableSQL, "", cputil.SyncerCheckpoint(""))).WillReturnError(errNoSuchTable) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", "")) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb1))) mock.ExpectQuery("SHOW CREATE TABLE .*").WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow(tb1, fmt.Sprintf(createTable1, tb2))) diff --git a/dm/checker/checker.go b/dm/checker/checker.go index 0ae60fea15a..dc688ca89ba 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/checker" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/dumpling" fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/log" @@ -44,6 +45,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/dumpling/export" + "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" "go.uber.org/zap" ) @@ -243,13 +245,25 @@ func (c *Checker) Init(ctx context.Context) (err error) { c.checkList = append(c.checkList, checker.NewTablesChecker(dbs, checkTablesMap, dumpThreads)) } - if checkingShard { - for name, shardingSet := range sharding { - if shardingCounter[name] <= 1 { - continue + instance := c.instances[0] + // Not check the sharding tables’ schema when the mode is increment. + // Because the table schema obtained from `show create table` is not the schema at the point of binlog. + if checkingShard && instance.cfg.Mode != config.ModeIncrement { + isFresh, err := c.IsFreshTask() + if err != nil { + return err + } + if isFresh { + for targetTableID, shardingSet := range sharding { + if shardingCounter[targetTableID] <= 1 { + continue + } + if instance.cfg.ShardMode == config.ShardPessimistic { + c.checkList = append(c.checkList, checker.NewShardingTablesChecker(targetTableID, dbs, shardingSet, columnMapping, checkingShardID, dumpThreads)) + } else { + c.checkList = append(c.checkList, checker.NewOptimisticShardingTablesChecker(targetTableID, dbs, shardingSet, dumpThreads)) + } } - - c.checkList = append(c.checkList, checker.NewShardingTablesChecker(name, dbs, shardingSet, columnMapping, checkingShardID, dumpThreads)) } } @@ -448,7 +462,31 @@ func (c *Checker) Type() pb.UnitType { // IsFreshTask implements Unit.IsFreshTask. func (c *Checker) IsFreshTask() (bool, error) { - return true, nil + instance := c.instances[0] + checkpointSQLs := []string{ + fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.LoaderCheckpoint(instance.cfg.Name))), + fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.LightningCheckpoint(instance.cfg.Name))), + fmt.Sprintf("SHOW CREATE TABLE %s", dbutil.TableName(instance.cfg.MetaSchema, cputil.SyncerCheckpoint(instance.cfg.Name))), + } + var existCheckpoint bool + for _, sql := range checkpointSQLs { + c.tctx.Logger.Info("exec query", zap.String("sql", sql)) + rows, err := instance.targetDB.DB.QueryContext(c.tctx.Ctx, sql) + if err != nil { + if utils.IsMySQLError(err, mysql.ErrNoSuchTable) { + continue + } + return false, err + } + defer rows.Close() + if rows.Err() != nil { + return false, rows.Err() + } + existCheckpoint = true + c.tctx.Logger.Info("exist checkpoint, so don't check sharding tables") + break + } + return !existCheckpoint, nil } // Status implements Unit interface. diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index 078902ad978..84621226eb5 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -382,11 +382,11 @@ func (st *SubTask) fetchResultAndUpdateStage(pr chan pb.ProcessResult) { } // setCurrUnit set current dm unit to ut. -func (st *SubTask) setCurrUnit(ut unit.Unit) { +func (st *SubTask) setCurrUnit(cu unit.Unit) { st.Lock() defer st.Unlock() pu := st.currUnit - st.currUnit = ut + st.currUnit = cu st.prevUnit = pu } diff --git a/dm/pkg/checker/table_structure.go b/dm/pkg/checker/table_structure.go index e6eb1363ff2..1bc18de99d8 100644 --- a/dm/pkg/checker/table_structure.go +++ b/dm/pkg/checker/table_structure.go @@ -30,6 +30,7 @@ import ( column "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" @@ -564,6 +565,132 @@ func (c *ShardingTablesChecker) Name() string { return fmt.Sprintf("sharding table %s consistency checking", c.targetTableID) } +// OptimisticShardingTablesChecker checks consistency of table structures of one sharding group in optimistic shard. +// * check whether they have compatible column list. +type OptimisticShardingTablesChecker struct { + targetTableID string + dbs map[string]*sql.DB + tableMap map[string][]*filter.Table // sourceID => [table1, table2, ...] + reMu sync.Mutex + joinedMu sync.Mutex + inCh chan *checkItem + dumpThreads int + joined *schemacmp.Table +} + +// NewOptimisticShardingTablesChecker returns a RealChecker. +func NewOptimisticShardingTablesChecker(targetTableID string, dbs map[string]*sql.DB, tableMap map[string][]*filter.Table, dumpThreads int) RealChecker { + if dumpThreads == 0 { + dumpThreads = 1 + } + c := &OptimisticShardingTablesChecker{ + targetTableID: targetTableID, + dbs: dbs, + tableMap: tableMap, + dumpThreads: dumpThreads, + } + c.inCh = make(chan *checkItem, dumpThreads) + return c +} + +// Name implements Checker interface. +func (c *OptimisticShardingTablesChecker) Name() string { + return fmt.Sprintf("optimistic sharding table %s consistency checking", c.targetTableID) +} + +// Check implements RealChecker interface. +func (c *OptimisticShardingTablesChecker) Check(ctx context.Context) *Result { + r := &Result{ + Name: c.Name(), + Desc: "check consistency of sharding table structures for Optimistic Sharding Merge", + State: StateSuccess, + Extra: fmt.Sprintf("sharding %s", c.targetTableID), + } + + startTime := time.Now() + concurrency, err := getConcurrency(ctx, c.tableMap, c.dbs, c.dumpThreads) + if err != nil { + markCheckError(r, err) + return r + } + eg, checkCtx := errgroup.WithContext(ctx) + for i := 0; i < concurrency; i++ { + eg.Go(func() error { + return c.checkTable(checkCtx, r) + }) + } + + dispatchTableItem(checkCtx, c.tableMap, c.inCh) + if err := eg.Wait(); err != nil { + markCheckError(r, err) + } + + log.L().Logger.Info("check optimistic sharding table structure over", zap.Duration("spend time", time.Since(startTime))) + return r +} + +func (c *OptimisticShardingTablesChecker) checkTable(ctx context.Context, r *Result) error { + var ( + sourceID string + p *parser.Parser + err error + ) + for { + select { + case <-ctx.Done(): + return nil + case checkItem, ok := <-c.inCh: + if !ok { + return nil + } + table := checkItem.table + if len(sourceID) == 0 || sourceID != checkItem.sourceID { + sourceID = checkItem.sourceID + p, err = dbutil.GetParserForDB(ctx, c.dbs[sourceID]) + if err != nil { + c.reMu.Lock() + r.Extra = fmt.Sprintf("fail to get parser for sourceID %s on sharding %s", sourceID, c.targetTableID) + c.reMu.Unlock() + return err + } + } + + statement, err := dbutil.GetCreateTableSQL(ctx, c.dbs[sourceID], table.Schema, table.Name) + if err != nil { + // continue if table was deleted when checking + if isMySQLError(err, mysql.ErrNoSuchTable) { + continue + } + return err + } + + ti, err := dbutil.GetTableInfoBySQL(statement, p) + if err != nil { + return err + } + encodeTi := schemacmp.Encode(ti) + c.joinedMu.Lock() + log.L().Logger.Debug("get schemacmp", zap.Stringer("ti", encodeTi), zap.Stringer("joined", c.joined), zap.Bool("pk is handle", ti.PKIsHandle)) + if c.joined == nil { + c.joined = &encodeTi + c.joinedMu.Unlock() + continue + } + newJoined, err2 := c.joined.Join(encodeTi) + if err2 != nil { + // NOTE: conflict detected. + c.reMu.Lock() + r.Extra = fmt.Sprintf("fail to join table info %s with %s", c.joined, encodeTi) + c.reMu.Unlock() + c.joinedMu.Unlock() + return err2 + } + c.joined = &newJoined + c.joinedMu.Unlock() + } + } +} + func dispatchTableItem(ctx context.Context, tableMap map[string][]*filter.Table, inCh chan *checkItem) { for sourceID, tables := range tableMap { for _, table := range tables { diff --git a/dm/pkg/checker/table_structure_test.go b/dm/pkg/checker/table_structure_test.go index bb67ab1a6a8..cc68f232421 100644 --- a/dm/pkg/checker/table_structure_test.go +++ b/dm/pkg/checker/table_structure_test.go @@ -16,6 +16,8 @@ package checker import ( "context" "database/sql" + "encoding/json" + "fmt" "github.com/DATA-DOG/go-sqlmock" tc "github.com/pingcap/check" @@ -177,6 +179,49 @@ func (t *testCheckSuite) TestTablesChecker(c *tc.C) { c.Assert(mock.ExpectationsWereMet(), tc.IsNil) } +func (t *testCheckSuite) TestOptimisticShardingTablesChecker(c *tc.C) { + db, mock, err := sqlmock.New() + c.Assert(err, tc.IsNil) + ctx := context.Background() + + printJSON := func(r *Result) { + rawResult, _ := json.MarshalIndent(r, "", "\t") + fmt.Println("\n" + string(rawResult)) + } + + // optimistic check different column number + maxConnecionsRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("max_connections", "2") + mock.ExpectQuery("SHOW VARIABLES LIKE 'max_connections'").WillReturnRows(maxConnecionsRow) + sqlModeRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). + AddRow("sql_mode", "ANSI_QUOTES") + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows(sqlModeRow) + createTableRow := sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test-table-1", `CREATE TABLE "test-table-1" ( +"c" int(11) NOT NULL, +PRIMARY KEY ("c") +) ENGINE=InnoDB DEFAULT CHARSET=latin1`) + mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-1`").WillReturnRows(createTableRow) + createTableRow2 := sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test-table-2", `CREATE TABLE "test-table-2" ( +"c" int(11) NOT NULL, +"d" int(11) NOT NULL, +PRIMARY KEY ("c") +) ENGINE=InnoDB DEFAULT CHARSET=latin1`) + mock.ExpectQuery("SHOW CREATE TABLE `test-db`.`test-table-2`").WillReturnRows(createTableRow2) + checker := NewOptimisticShardingTablesChecker("test-name", + map[string]*sql.DB{"test-source": db}, + map[string][]*filter.Table{"test-source": { + &filter.Table{Schema: "test-db", Name: "test-table-1"}, + &filter.Table{Schema: "test-db", Name: "test-table-2"}, + }}, + 0) + result := checker.Check(ctx) + printJSON(result) + c.Assert(result.State, tc.Equals, StateSuccess) + c.Assert(mock.ExpectationsWereMet(), tc.IsNil) +} + func initShardingMock(mock sqlmock.Sqlmock) sqlmock.Sqlmock { sqlModeRow := sqlmock.NewRows([]string{"Variable_name", "Value"}). AddRow("sql_mode", "ANSI_QUOTES") diff --git a/dm/tests/dmctl_basic/conf/diff_config2.toml b/dm/tests/dmctl_basic/conf/diff_config2.toml new file mode 100644 index 00000000000..6b523b50678 --- /dev/null +++ b/dm/tests/dmctl_basic/conf/diff_config2.toml @@ -0,0 +1,50 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + target-check-tables = ["dmctl.precheck_optimistic_tb"] + + target-configs= ["config1"] + +[table-configs] +[table-configs.config1] +target-tables = ["dmctl.precheck_optimistic_tb"] + +[routes.rule1] +schema-pattern = "dmctl" +table-pattern = "precheck_optimistic_tb_?*" +target-schema = "dmctl" +target-table = "precheck_optimistic_tb" + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/dmctl_basic/conf/dm-task4.yaml b/dm/tests/dmctl_basic/conf/dm-task4.yaml new file mode 100644 index 00000000000..ae9355ea391 --- /dev/null +++ b/dm/tests/dmctl_basic/conf/dm-task4.yaml @@ -0,0 +1,63 @@ +--- +name: pre_check_optimistic +task-mode: all +shard-mode: "optimistic" +meta-schema: "dm_meta" +enable-heartbeat: false + +target-database: + host: "127.0.0.1" + port: 4000 + user: "root" + password: "" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-route-rules-table", "sharding-route-rules-schema"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-route-rules-table", "sharding-route-rules-schema"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["dmctl"] + do-tables: + - db-name: "dmctl" + tbl-name: "~^precheck_optimistic_tb_[\\d]+" + +routes: + sharding-route-rules-table: + schema-pattern: dmctl + table-pattern: precheck_optimistic_tb_* + target-schema: dmctl + target-table: precheck_optimistic_tb + + sharding-route-rules-schema: + schema-pattern: dmctl + target-schema: dmctl + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 + checkpoint-flush-interval: 1 diff --git a/dm/tests/dmctl_basic/data/db1.prepare.sql b/dm/tests/dmctl_basic/data/db1.prepare.sql index 289b61d977d..54b469d4aee 100644 --- a/dm/tests/dmctl_basic/data/db1.prepare.sql +++ b/dm/tests/dmctl_basic/data/db1.prepare.sql @@ -26,4 +26,7 @@ INSERT INTO `dmctl`.`t_1` (`b`,`c`,`d`,`id`) VALUES (800180420,'JuUIxUacksp','sX create table tb_1(a INT, b INT); create table tb_2(a INT, c INT); +create table precheck_optimistic_tb_1(a INT, b INT, primary key a(a)); +create table precheck_optimistic_tb_2(a INT, c INT, primary key a(a)); + CREATE TABLE only_warning (id bigint, b int, primary key id(id), FOREIGN KEY (b) REFERENCES t_1(b)); \ No newline at end of file diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index 858713ee737..5437d4f6aa5 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -283,6 +283,9 @@ function run() { check_task_not_pass $cur/conf/dm-task2.yaml check_task_error_count $cur/conf/dm-task3.yaml + echo "check_task_optimistic" + check_task_pass $cur/conf/dm-task4.yaml + echo "check_task_only_warning" check_task_only_warning $cur/conf/only_warning.yaml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/go.mod b/go.mod index c713e718dac..64416e4701d 100644 --- a/go.mod +++ b/go.mod @@ -81,6 +81,7 @@ require ( github.com/uber-go/atomic v1.4.0 github.com/vmihailenco/msgpack/v5 v5.3.5 github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c + github.com/xitongsys/parquet-go v1.6.0 // indirect go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 go.uber.org/atomic v1.9.0 go.uber.org/goleak v1.1.12 diff --git a/go.sum b/go.sum index 1f031ae2e48..232d25cd6d2 100644 --- a/go.sum +++ b/go.sum @@ -1109,8 +1109,9 @@ github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQ 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= -github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs= github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= +github.com/xitongsys/parquet-go v1.6.0 h1:j6YrTVZdQx5yywJLIOklZcKVsCoSD1tqOVRXyTBFSjs= +github.com/xitongsys/parquet-go v1.6.0/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= 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 h1:a742S4V5A15F93smuVxA60LQWsrCnN8bKeWDBARU1/k= github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0/go.mod h1:HYhIKsdns7xz80OgkbgJYrtQY7FjHWHKH6cvN7+czGE= From 67a3029fa40ffccbc0a528821f51b7540ad0fb64 Mon Sep 17 00:00:00 2001 From: maxshuang Date: Tue, 15 Feb 2022 03:07:38 +0800 Subject: [PATCH 62/72] sink(ticdc): fix `Unknown system variable 'transaction_isolation'` for some mysql versions (#4569) close pingcap/tiflow#4504 --- cdc/sink/mysql_params.go | 14 +++++++-- cdc/sink/mysql_params_test.go | 57 +++++++++++++++++++++++++++++++++++ cdc/sink/mysql_test.go | 3 ++ pkg/applier/redo_test.go | 3 ++ 4 files changed, 75 insertions(+), 2 deletions(-) diff --git a/cdc/sink/mysql_params.go b/cdc/sink/mysql_params.go index 9f0fec92a41..320ba69bc30 100644 --- a/cdc/sink/mysql_params.go +++ b/cdc/sink/mysql_params.go @@ -268,8 +268,6 @@ func generateDSNByParams( dsnCfg.Params["readTimeout"] = params.readTimeout dsnCfg.Params["writeTimeout"] = params.writeTimeout dsnCfg.Params["timeout"] = params.dialTimeout - // Since we don't need select, just set default isolation level to read-committed - dsnCfg.Params["transaction_isolation"] = fmt.Sprintf(`"%s"`, defaultTxnIsolationRC) autoRandom, err := checkTiDBVariable(ctx, testDB, "allow_auto_random_explicit_insert", "1") if err != nil { @@ -287,6 +285,18 @@ func generateDSNByParams( dsnCfg.Params["tidb_txn_mode"] = txnMode } + // Since we don't need select, just set default isolation level to read-committed + // transaction_isolation is mysql newly introduced variable and will vary from MySQL5.7/MySQL8.0/Mariadb + isolation, err := checkTiDBVariable(ctx, testDB, "transaction_isolation", defaultTxnIsolationRC) + if err != nil { + return "", err + } + if isolation != "" { + dsnCfg.Params["transaction_isolation"] = fmt.Sprintf(`"%s"`, defaultTxnIsolationRC) + } else { + dsnCfg.Params["tx_isolation"] = fmt.Sprintf(`"%s"`, defaultTxnIsolationRC) + } + dsnClone := dsnCfg.Clone() dsnClone.Passwd = "******" log.Info("sink uri is configured", zap.String("dsn", dsnClone.FormatDSN())) diff --git a/cdc/sink/mysql_params_test.go b/cdc/sink/mysql_params_test.go index 075e5d41d26..80a104ab244 100644 --- a/cdc/sink/mysql_params_test.go +++ b/cdc/sink/mysql_params_test.go @@ -118,9 +118,66 @@ func TestGenerateDSNByParams(t *testing.T) { } } + testIsolationParams := func() { + db, mock, err := sqlmock.New() + require.Nil(t, err) + defer db.Close() // nolint:errcheck + columns := []string{"Variable_name", "Value"} + mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), + ) + mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), + ) + // simulate error + dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") + require.Nil(t, err) + params := defaultParams.Clone() + var dsnStr string + _, err = generateDSNByParams(context.TODO(), dsn, params, db) + require.Error(t, err) + + // simulate no transaction_isolation + mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), + ) + mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), + ) + mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnError(sql.ErrNoRows) + dsnStr, err = generateDSNByParams(context.TODO(), dsn, params, db) + require.Nil(t, err) + expectedParams := []string{ + "tx_isolation=%22READ-COMMITTED%22", + } + for _, param := range expectedParams { + require.True(t, strings.Contains(dsnStr, param)) + } + + // simulate transaction_isolation + mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), + ) + mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), + ) + mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), + ) + dsnStr, err = generateDSNByParams(context.TODO(), dsn, params, db) + require.Nil(t, err) + expectedParams = []string{ + "transaction_isolation=%22READ-COMMITTED%22", + } + for _, param := range expectedParams { + require.True(t, strings.Contains(dsnStr, param)) + } + } + testDefaultParams() testTimezoneParam() testTimeoutParams() + testIsolationParams() } func TestParseSinkURIToParams(t *testing.T) { diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index cc0b119350c..aa69c25da23 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -472,6 +472,9 @@ func mockTestDB(adjustSQLMode bool) (*sql.DB, error) { mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), ) + mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), + ) mock.ExpectClose() return db, nil } diff --git a/pkg/applier/redo_test.go b/pkg/applier/redo_test.go index 5d2466bd27e..8c33f61de86 100644 --- a/pkg/applier/redo_test.go +++ b/pkg/applier/redo_test.go @@ -137,6 +137,9 @@ func TestApplyDMLs(t *testing.T) { mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), ) + mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), + ) mock.ExpectClose() return db, nil } From 28fe713deaba0d0fc8e53b7e038d99e43d0011dd Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 15 Feb 2022 13:35:39 +0800 Subject: [PATCH 63/72] cdc/sink: kafka sink integrate broker level metrics (#4517) close pingcap/tiflow#4588 --- cdc/sink/producer/kafka/kafka.go | 20 +- cdc/sink/producer/kafka/metrics.go | 209 +++- metrics/grafana/ticdc.json | 1655 ++++++++++++++++++++-------- 3 files changed, 1406 insertions(+), 478 deletions(-) diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index dd13cfdfaba..2888587c029 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -54,6 +54,7 @@ type kafkaSaramaProducer struct { // Since we don't close these two clients (which have an input chan) from the // sender routine, data race or send on closed chan could happen. clientLock sync.RWMutex + admin kafka.ClusterAdminClient client sarama.Client asyncProducer sarama.AsyncProducer syncProducer sarama.SyncProducer @@ -262,6 +263,16 @@ func (k *kafkaSaramaProducer) Close() error { zap.String("changefeed", k.id), zap.Any("role", k.role)) } + start = time.Now() + if err := k.admin.Close(); err != nil { + log.Warn("close kafka cluster admin with error", zap.Error(err), + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } else { + log.Info("kafka cluster admin closed", zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } + k.metricsMonitor.Cleanup() return nil } @@ -329,12 +340,6 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - defer func() { - if err := admin.Close(); err != nil { - log.Warn("close kafka cluster admin failed", zap.Error(err), - zap.String("changefeed", changefeedID), zap.Any("role", role)) - } - }() if err := validateAndCreateTopic(admin, topic, config, cfg, opts); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) @@ -361,6 +366,7 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, return nil, err } k := &kafkaSaramaProducer{ + admin: admin, client: client, asyncProducer: asyncProducer, syncProducer: syncProducer, @@ -380,7 +386,7 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, role: role, metricsMonitor: NewSaramaMetricsMonitor(cfg.MetricRegistry, - util.CaptureAddrFromCtx(ctx), changefeedID), + util.CaptureAddrFromCtx(ctx), changefeedID, admin), } go func() { if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { diff --git a/cdc/sink/producer/kafka/metrics.go b/cdc/sink/producer/kafka/metrics.go index 600e2eb3940..844850938cc 100644 --- a/cdc/sink/producer/kafka/metrics.go +++ b/cdc/sink/producer/kafka/metrics.go @@ -14,12 +14,17 @@ package kafka import ( + "strconv" + + "github.com/pingcap/log" + "github.com/pingcap/tiflow/pkg/kafka" "github.com/prometheus/client_golang/prometheus" "github.com/rcrowley/go-metrics" + "go.uber.org/zap" ) var ( - // batch-size + // Histogram update by the `batch-size` batchSizeGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -28,7 +33,7 @@ var ( Help: "the number of bytes sent per partition per request for all topics", }, []string{"capture", "changefeed"}) - // record-send-rate + // meter mark by total records count recordSendRateGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -38,6 +43,7 @@ var ( }, []string{"capture", "changefeed"}) // records-per-request + // histogram update by all records count. recordPerRequestGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -46,7 +52,7 @@ var ( Help: "the number of records sent per request for all topics", }, []string{"capture", "changefeed"}) - // compression-ratio + // histogram update by `compression-ratio`. compressionRatioGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -54,6 +60,80 @@ var ( Name: "kafka_producer_compression_ratio", Help: "the compression ratio times 100 of record batches for all topics", }, []string{"capture", "changefeed"}) + + // metrics for outgoing events + // meter mark for each request's size in bytes + outgoingByteRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_outgoing_byte_rate", + Help: "Bytes/second written off all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by 1 for each request + requestRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_rate", + Help: "Requests/second sent to all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark for each request's size in bytes + requestSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_size", + Help: "the request size in bytes for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // histogram update for each received response, requestLatency := time.Since(response.requestTime) + requestLatencyInMsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_latency", + Help: "the request latency in ms for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // counter inc by 1 once a request send, dec by 1 for a response received. + requestsInFlightGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_in_flight_requests", + Help: "the current number of in-flight requests awaiting a response for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // metrics for incoming events + // meter mark for each received response's size in bytes + incomingByteRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_incoming_byte_rate", + Help: "Bytes/second read off all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by 1 once a response received. + responseRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_response_rate", + Help: "Responses/second received from all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by each read response size + responseSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_response_size", + Help: "the response size in bytes for all brokers", + }, []string{"capture", "changefeed", "broker"}) ) // InitMetrics registers all metrics in this file @@ -62,14 +142,34 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(recordSendRateGauge) registry.MustRegister(recordPerRequestGauge) registry.MustRegister(compressionRatioGauge) + + registry.MustRegister(incomingByteRateGauge) + registry.MustRegister(outgoingByteRateGauge) + registry.MustRegister(requestSizeGauge) + registry.MustRegister(requestRateGauge) + registry.MustRegister(requestLatencyInMsGauge) + registry.MustRegister(requestsInFlightGauge) + registry.MustRegister(responseSizeGauge) + registry.MustRegister(responseRateGauge) } // sarama metrics names, see https://pkg.go.dev/github.com/Shopify/sarama#pkg-overview const ( + // metrics at producer level. batchSizeMetricName = "batch-size" recordSendRateMetricName = "record-send-rate" recordPerRequestMetricName = "records-per-request" compressionRatioMetricName = "compression-ratio" + + // metrics at broker level. + incomingByteRateMetricNamePrefix = "incoming-byte-rate-for-broker-" + outgoingByteRateMetricNamePrefix = "outgoing-byte-rate-for-broker-" + requestRateMetricNamePrefix = "request-rate-for-broker-" + requestSizeMetricNamePrefix = "request-size-for-broker-" + requestLatencyInMsMetricNamePrefix = "request-latency-in-ms-for-broker-" + requestsInFlightMetricNamePrefix = "requests-in-flight-for-broker-" + responseRateMetricNamePrefix = "response-rate-for-broker-" + responseSizeMetricNamePrefix = "response-size-for-broker-" ) type saramaMetricsMonitor struct { @@ -77,42 +177,135 @@ type saramaMetricsMonitor struct { changefeedID string registry metrics.Registry + admin kafka.ClusterAdminClient } // CollectMetrics collect all monitored metrics func (sm *saramaMetricsMonitor) CollectMetrics() { + sm.collectProducerMetrics() + if err := sm.collectBrokerMetrics(); err != nil { + log.Warn("collect broker metrics failed", zap.Error(err)) + } +} + +func (sm *saramaMetricsMonitor) collectProducerMetrics() { batchSizeMetric := sm.registry.Get(batchSizeMetricName) if histogram, ok := batchSizeMetric.(metrics.Histogram); ok { - batchSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + batchSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) } recordSendRateMetric := sm.registry.Get(recordSendRateMetricName) if meter, ok := recordSendRateMetric.(metrics.Meter); ok { - recordSendRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(meter.Rate1()) + recordSendRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(meter.Snapshot().Rate1()) } recordPerRequestMetric := sm.registry.Get(recordPerRequestMetricName) if histogram, ok := recordPerRequestMetric.(metrics.Histogram); ok { - recordPerRequestGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + recordPerRequestGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) } compressionRatioMetric := sm.registry.Get(compressionRatioMetricName) if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { - compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) } } -func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string) *saramaMetricsMonitor { +func getBrokerMetricName(prefix, brokerID string) string { + return prefix + brokerID +} + +func (sm *saramaMetricsMonitor) collectBrokerMetrics() error { + brokers, _, err := sm.admin.DescribeCluster() + if err != nil { + return err + } + + for _, b := range brokers { + brokerID := strconv.Itoa(int(b.ID())) + + incomingByteRateMetric := sm.registry.Get(getBrokerMetricName(incomingByteRateMetricNamePrefix, brokerID)) + if meter, ok := incomingByteRateMetric.(metrics.Meter); ok { + incomingByteRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + outgoingByteRateMetric := sm.registry.Get(getBrokerMetricName(outgoingByteRateMetricNamePrefix, brokerID)) + if meter, ok := outgoingByteRateMetric.(metrics.Meter); ok { + outgoingByteRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + requestRateMetric := sm.registry.Get(getBrokerMetricName(requestRateMetricNamePrefix, brokerID)) + if meter, ok := requestRateMetric.(metrics.Meter); ok { + requestRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + requestSizeMetric := sm.registry.Get(getBrokerMetricName(requestSizeMetricNamePrefix, brokerID)) + if histogram, ok := requestSizeMetric.(metrics.Histogram); ok { + requestSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + + requestLatencyMetric := sm.registry.Get(getBrokerMetricName(requestLatencyInMsMetricNamePrefix, brokerID)) + if histogram, ok := requestLatencyMetric.(metrics.Histogram); ok { + requestLatencyInMsGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + + requestsInFlightMetric := sm.registry.Get(getBrokerMetricName(requestsInFlightMetricNamePrefix, brokerID)) + if counter, ok := requestsInFlightMetric.(metrics.Counter); ok { + requestsInFlightGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(float64(counter.Snapshot().Count())) + } + + responseRateMetric := sm.registry.Get(getBrokerMetricName(responseRateMetricNamePrefix, brokerID)) + if meter, ok := responseRateMetric.(metrics.Meter); ok { + responseRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + responseSizeMetric := sm.registry.Get(getBrokerMetricName(responseSizeMetricNamePrefix, brokerID)) + if histogram, ok := responseSizeMetric.(metrics.Histogram); ok { + responseSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + } + return nil +} + +func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string, admin kafka.ClusterAdminClient) *saramaMetricsMonitor { return &saramaMetricsMonitor{ captureAddr: captureAddr, changefeedID: changefeedID, registry: registry, + admin: admin, } } func (sm *saramaMetricsMonitor) Cleanup() { + sm.cleanUpProducerMetrics() + if err := sm.cleanUpBrokerMetrics(); err != nil { + log.Warn("clean up broker metrics failed", zap.Error(err)) + } +} + +func (sm *saramaMetricsMonitor) cleanUpProducerMetrics() { batchSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) recordSendRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) recordPerRequestGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) compressionRatioGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) } + +func (sm *saramaMetricsMonitor) cleanUpBrokerMetrics() error { + brokers, _, err := sm.admin.DescribeCluster() + if err != nil { + return err + } + + for _, b := range brokers { + brokerID := strconv.Itoa(int(b.ID())) + + incomingByteRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + outgoingByteRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestLatencyInMsGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestsInFlightGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + responseRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + responseSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + } + return nil +} diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index a1f00411376..9a9f8c9a4c2 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -124,8 +124,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": 33, - "iteration": 1644481309661, + "id": 25, + "iteration": 1644826138916, "links": [], "panels": [ { @@ -3503,7 +3503,7 @@ "h": 4, "w": 12, "x": 0, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 218, @@ -3597,7 +3597,7 @@ "h": 4, "w": 12, "x": 12, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 229, @@ -3691,7 +3691,7 @@ "h": 4, "w": 12, "x": 0, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 228, @@ -3785,7 +3785,7 @@ "h": 4, "w": 12, "x": 12, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 220, @@ -3879,7 +3879,7 @@ "h": 4, "w": 12, "x": 0, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 219, @@ -3973,7 +3973,7 @@ "h": 4, "w": 12, "x": 12, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 224, @@ -4067,7 +4067,7 @@ "h": 4, "w": 12, "x": 0, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 223, @@ -4161,7 +4161,7 @@ "h": 4, "w": 12, "x": 12, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 221, @@ -4255,7 +4255,7 @@ "h": 4, "w": 12, "x": 0, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 222, @@ -4349,7 +4349,7 @@ "h": 4, "w": 12, "x": 12, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 226, @@ -4443,7 +4443,7 @@ "h": 4, "w": 12, "x": 0, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 227, @@ -4539,7 +4539,7 @@ "h": 4, "w": 12, "x": 12, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 225, @@ -4635,7 +4635,7 @@ "h": 4, "w": 12, "x": 0, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 108, @@ -4730,7 +4730,7 @@ "h": 4, "w": 12, "x": 12, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 240, @@ -12350,8 +12350,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12536,8 +12536,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [], @@ -12643,8 +12643,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12765,8 +12765,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12863,8 +12863,8 @@ "scopedVars": { "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "title": "Runtime $runtime_instance", @@ -12878,7 +12878,7 @@ "x": 0, "y": 10 }, - "id": 411, + "id": 426, "panels": [ { "aliasColors": {}, @@ -12898,7 +12898,7 @@ "x": 0, "y": 6 }, - "id": 412, + "id": 427, "legend": { "alignAsTable": false, "avg": false, @@ -12925,7 +12925,7 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 157, "repeatedByRow": true, "scopedVars": { @@ -12936,8 +12936,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13087,7 +13087,7 @@ "x": 12, "y": 6 }, - "id": 413, + "id": 428, "legend": { "alignAsTable": false, "avg": false, @@ -13114,7 +13114,7 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 158, "repeatedByRow": true, "scopedVars": { @@ -13125,8 +13125,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [], @@ -13203,7 +13203,7 @@ "x": 0, "y": 13 }, - "id": 414, + "id": 429, "legend": { "alignAsTable": false, "avg": false, @@ -13224,7 +13224,7 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 160, "repeatedByRow": true, "scopedVars": { @@ -13235,8 +13235,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13329,7 +13329,7 @@ "x": 12, "y": 13 }, - "id": 415, + "id": 430, "legend": { "alignAsTable": false, "avg": false, @@ -13349,7 +13349,7 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 161, "repeatedByRow": true, "scopedVars": { @@ -13360,8 +13360,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13455,13 +13455,13 @@ } ], "repeat": null, - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 155, "scopedVars": { "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "title": "Runtime $runtime_instance", @@ -13836,7 +13836,7 @@ "type": "row" }, { - "collapsed": false, + "collapsed": true, "gridPos": { "h": 1, "w": 24, @@ -13844,421 +13844,1150 @@ "y": 12 }, "id": 402, - "panels": [], - "title": "Kafka Sink", - "type": "row" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of bytes sent per partition per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 13 - }, - "hiddenSeries": false, - "id": 410, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "batch size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of records sent per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 13 - }, - "hiddenSeries": false, - "id": 408, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "records send rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "wps", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of records sent per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 20 - }, - "hiddenSeries": false, - "id": 409, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "records per request", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "the compression ratio times 100 of record batches for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 20 - }, - "hiddenSeries": false, - "id": 416, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "compression ratio", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percent", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of bytes send for all topics.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 122 + }, + "hiddenSeries": false, + "id": 410, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "batch size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Records/second sent to all topics\n\nvalue = one-minute moving average of record send rate", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 122 + }, + "hiddenSeries": false, + "id": 408, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records send rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of records sent per request for all topics\n\nvalue = records per request histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 129 + }, + "hiddenSeries": false, + "id": 409, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records per request", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the compression ratio times 100 of record batches for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 129 + }, + "hiddenSeries": false, + "id": 416, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "compression ratio", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bytes/second write to brokers\n\nvalue = one-minute moving average rate of sent bytes", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 136 + }, + "hiddenSeries": false, + "id": 418, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_outgoing_byte_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes/second write to brokers", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bytes/second read from different brokers.\n\nvalue = one-minute moving average rate of all read bytes", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 136 + }, + "hiddenSeries": false, + "id": 417, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_incoming_byte_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes/second read from brokers", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Requests/second sent to all brokers\n\nvalue = one-minute moving average rate of events per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 143 + }, + "hiddenSeries": false, + "id": 419, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "request rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Responses/second received from all brokers\n\nvalue = one-minute moving average of response receive rate", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 143 + }, + "hiddenSeries": false, + "id": 423, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_response_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "response rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the request size in bytes for all brokers\n\nvalue = requests histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 150 + }, + "hiddenSeries": false, + "id": 425, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "request size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the response size in bytes\n\nvalue = response size histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 150 + }, + "hiddenSeries": false, + "id": 424, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_response_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "response size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the request latency in ms for all brokers\n\nvalue = request latency histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 157 + }, + "hiddenSeries": false, + "id": 420, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_latency{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "the request latency", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ms", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the current number of in-flight requests awaiting a response\n\nvalue = the count of inflight requests.\n\nupdated each 1s, in a good network environment, this value should be very small.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 164 + }, + "hiddenSeries": false, + "id": 422, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_in_flight_requests{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "inflight requests", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Kafka Sink", + "type": "row" } ], "refresh": "1m", @@ -14480,5 +15209,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 33 + "version": 34 } From 9ddc713862748b679b3e19caad521a8e492cc24f Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 15 Feb 2022 16:05:40 +0800 Subject: [PATCH 64/72] test(ticdc): migrate test-infra to testify for cdc/kv pkg (#2899) (#4515) close pingcap/tiflow#2899 --- cdc/kv/client_test.go | 731 ++++++++++++++------------------- cdc/kv/grpc_pool_impl_test.go | 48 +-- cdc/kv/main_test.go | 30 ++ cdc/kv/matcher_test.go | 47 +-- cdc/kv/region_worker_test.go | 51 +-- cdc/kv/resolvedts_heap_test.go | 78 ++-- cdc/kv/store_op.go | 3 +- cdc/kv/token_region_test.go | 47 +-- 8 files changed, 475 insertions(+), 560 deletions(-) create mode 100644 cdc/kv/main_test.go diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 181de57e0e9..4abef96e351 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -25,7 +25,6 @@ import ( "time" "github.com/golang/protobuf/proto" // nolint:staticcheck - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/cdcpb" @@ -37,19 +36,16 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/pdtime" "github.com/pingcap/tiflow/pkg/regionspan" "github.com/pingcap/tiflow/pkg/retry" "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/txnutil" "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" - "go.etcd.io/etcd/embed" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -62,31 +58,11 @@ func Test(t *testing.T) { go func() { RunWorkerPool(context.Background()) //nolint:errcheck }() - check.TestingT(t) } -type clientSuite struct { - e *embed.Etcd -} - -var _ = check.Suite(&clientSuite{}) - -func (s *clientSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - var err error - _, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) -} - -func (s *clientSuite) TearDownTest(c *check.C) { - s.e.Close() -} - -func (s *clientSuite) TestNewClient(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestNewClient(t *testing.T) { rpcClient, _, pdClient, err := testutils.NewMockTiKV("", nil) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer pdClient.Close() defer rpcClient.Close() @@ -95,12 +71,10 @@ func (s *clientSuite) TestNewClient(c *check.C) { regionCache := tikv.NewRegionCache(pdClient) defer regionCache.Close() cli := NewCDCClient(context.Background(), pdClient, nil, grpcPool, regionCache, pdtime.NewClock4Test(), "") - c.Assert(cli, check.NotNil) + require.NotNil(t, cli) } -func (s *clientSuite) TestAssembleRowEvent(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestAssembleRowEvent(t *testing.T) { testCases := []struct { regionID uint64 entry *cdcpb.Event_Row @@ -209,24 +183,24 @@ func (s *clientSuite) TestAssembleRowEvent(c *check.C) { for _, tc := range testCases { event, err := assembleRowEvent(tc.regionID, tc.entry, tc.enableOldValue) - c.Assert(event, check.DeepEquals, tc.expected) + require.Equal(t, tc.expected, event) if err != nil { - c.Assert(err.Error(), check.Equals, tc.err) + require.Equal(t, tc.err, err.Error()) } } } type mockChangeDataService struct { - c *check.C + t *testing.T ch chan *cdcpb.ChangeDataEvent recvLoop func(server cdcpb.ChangeData_EventFeedServer) exitNotify sync.Map eventFeedID uint64 } -func newMockChangeDataService(c *check.C, ch chan *cdcpb.ChangeDataEvent) *mockChangeDataService { +func newMockChangeDataService(t *testing.T, ch chan *cdcpb.ChangeDataEvent) *mockChangeDataService { s := &mockChangeDataService{ - c: c, + t: t, ch: ch, } return s @@ -280,23 +254,23 @@ loop: func newMockService( ctx context.Context, - c *check.C, + t *testing.T, srv cdcpb.ChangeDataServer, wg *sync.WaitGroup, ) (grpcServer *grpc.Server, addr string) { - return newMockServiceSpecificAddr(ctx, c, srv, "127.0.0.1:0", wg) + return newMockServiceSpecificAddr(ctx, t, srv, "127.0.0.1:0", wg) } func newMockServiceSpecificAddr( ctx context.Context, - c *check.C, + t *testing.T, srv cdcpb.ChangeDataServer, listenAddr string, wg *sync.WaitGroup, ) (grpcServer *grpc.Server, addr string) { lc := &net.ListenConfig{} lis, err := lc.Listen(ctx, "tcp", listenAddr) - c.Assert(err, check.IsNil) + require.Nil(t, err) addr = lis.Addr().String() kaep := keepalive.EnforcementPolicy{ // force minimum ping interval @@ -318,13 +292,13 @@ func newMockServiceSpecificAddr( go func() { defer wg.Done() err := grpcServer.Serve(lis) - c.Assert(err, check.IsNil) + require.Nil(t, err) }() return } // waitRequestID waits request ID larger than the given allocated ID -func waitRequestID(c *check.C, allocatedID uint64) { +func waitRequestID(t *testing.T, allocatedID uint64) { err := retry.Do(context.Background(), func() error { if currentRequestID() > allocatedID { return nil @@ -332,18 +306,16 @@ func waitRequestID(c *check.C, allocatedID uint64) { return errors.Errorf("request id %d is not larger than %d", currentRequestID(), allocatedID) }, retry.WithBackoffBaseDelay(10), retry.WithMaxTries(20)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestConnectOfflineTiKV(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() wg := &sync.WaitGroup{} ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv := newMockChangeDataService(c, ch2) - server2, addr := newMockService(ctx, c, srv, wg) + srv := newMockChangeDataService(t, ch2) + server2, addr := newMockService(ctx, t, srv, wg) defer func() { close(ch2) server2.Stop() @@ -351,10 +323,10 @@ func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck invalidStore := "localhost:1" @@ -378,11 +350,11 @@ func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // new session, request to store 1, request to store 2 - waitRequestID(c, baseAllocatedID+2) + waitRequestID(t, baseAllocatedID+2) makeEvent := func(ts uint64) *cdcpb.ChangeDataEvent { return &cdcpb.ChangeDataEvent{ @@ -399,7 +371,7 @@ func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { } checkEvent := func(event model.RegionFeedEvent, ts uint64) { - c.Assert(event.Resolved.ResolvedTs, check.Equals, ts) + require.Equal(t, ts, event.Resolved.ResolvedTs) } initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) @@ -409,7 +381,7 @@ func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { ts, err := kvStorage.CurrentTimestamp(oracle.GlobalTxnScope) ver := kv.NewVersion(ts) - c.Assert(err, check.IsNil) + require.Nil(t, err) ch2 <- makeEvent(ver.Ver) var event model.RegionFeedEvent // consume the first resolved ts event, which is sent before region starts @@ -417,35 +389,33 @@ func (s *clientSuite) TestConnectOfflineTiKV(c *check.C) { select { case event = <-eventCh: case <-time.After(time.Second): - c.Fatalf("reconnection not succeed in 1 second") + require.FailNow(t, "reconnection not succeed in 1 second") } checkEvent(event, 1) select { case event = <-eventCh: case <-time.After(time.Second): - c.Fatalf("reconnection not succeed in 1 second") + require.FailNow(t, "reconnection not succeed in 1 second") } checkEvent(event, ver.Ver) // check gRPC connection active counter is updated correctly bucket, ok := grpcPool.bucketConns[invalidStore] - c.Assert(ok, check.IsTrue) + require.True(t, ok) empty := bucket.recycle() - c.Assert(empty, check.IsTrue) + require.True(t, empty) cancel() } // [NOTICE]: I concern this ut may cost too much time when resource limit -func (s *clientSuite) TestRecvLargeMessageSize(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestRecvLargeMessageSize(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv := newMockChangeDataService(c, ch2) - server2, addr := newMockService(ctx, c, srv, wg) + srv := newMockChangeDataService(t, ch2) + server2, addr := newMockService(ctx, t, srv, wg) defer func() { close(ch2) server2.Stop() @@ -453,11 +423,11 @@ func (s *clientSuite) TestRecvLargeMessageSize(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} defer pdClient.Close() //nolint:errcheck kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(2, addr) @@ -476,11 +446,11 @@ func (s *clientSuite) TestRecvLargeMessageSize(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 1, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) ch2 <- initialized @@ -489,9 +459,9 @@ func (s *clientSuite) TestRecvLargeMessageSize(c *check.C) { select { case event = <-eventCh: case <-time.After(time.Second): - c.Fatalf("recving message takes too long") + require.FailNow(t, "recving message takes too long") } - c.Assert(event, check.NotNil) + require.NotNil(t, event) largeValSize := 128*1024*1024 + 1 // 128MB + 1 largeMsg := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ @@ -515,25 +485,23 @@ func (s *clientSuite) TestRecvLargeMessageSize(c *check.C) { select { case event = <-eventCh: case <-time.After(30 * time.Second): // Send 128MB object may costs lots of time. - c.Fatalf("receiving message takes too long") + require.FailNow(t, "receiving message takes too long") } - c.Assert(len(event.Val.Value), check.Equals, largeValSize) + require.Equal(t, largeValSize, len(event.Val.Value)) cancel() } -func (s *clientSuite) TestHandleError(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestHandleError(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv2 := newMockChangeDataService(c, ch2) - server2, addr2 := newMockService(ctx, c, srv2, wg) + srv2 := newMockChangeDataService(t, ch2) + server2, addr2 := newMockService(ctx, t, srv2, wg) defer func() { close(ch1) @@ -544,10 +512,10 @@ func (s *clientSuite) TestHandleError(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck region3 := uint64(3) @@ -574,11 +542,11 @@ func (s *clientSuite) TestHandleError(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("d")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) var event model.RegionFeedEvent notLeader := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ @@ -602,7 +570,7 @@ func (s *clientSuite) TestHandleError(c *check.C) { // wait request id allocated with: // new session, no leader request, epoch not match request - waitRequestID(c, baseAllocatedID+2) + waitRequestID(t, baseAllocatedID+2) epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -616,7 +584,7 @@ func (s *clientSuite) TestHandleError(c *check.C) { }} ch2 <- epochNotMatch - waitRequestID(c, baseAllocatedID+3) + waitRequestID(t, baseAllocatedID+3) regionNotFound := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -630,7 +598,7 @@ func (s *clientSuite) TestHandleError(c *check.C) { }} ch2 <- regionNotFound - waitRequestID(c, baseAllocatedID+4) + waitRequestID(t, baseAllocatedID+4) unknownErr := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -648,8 +616,8 @@ consumePreResolvedTs: for { select { case event = <-eventCh: - c.Assert(event.Resolved, check.NotNil) - c.Assert(event.Resolved.ResolvedTs, check.Equals, uint64(100)) + require.NotNil(t, event.Resolved) + require.Equal(t, uint64(100), event.Resolved.ResolvedTs) case <-time.After(time.Second): break consumePreResolvedTs } @@ -658,7 +626,7 @@ consumePreResolvedTs: // wait request id allocated with: // new session, no leader request, epoch not match request, // region not found request, unknown error request, normal request - waitRequestID(c, baseAllocatedID+5) + waitRequestID(t, baseAllocatedID+5) initialized := mockInitializedEvent(3 /* regionID */, currentRequestID()) ch2 <- initialized @@ -682,10 +650,10 @@ consumePreResolvedTs: select { case event = <-eventCh: case <-time.After(3 * time.Second): - c.Fatalf("reconnection not succeed in 3 seconds") + require.FailNow(t, "reconnection not succeed in 3 seconds") } - c.Assert(event.Resolved, check.NotNil) - c.Assert(event.Resolved.ResolvedTs, check.Equals, uint64(120)) + require.NotNil(t, event.Resolved) + require.Equal(t, uint64(120), event.Resolved.ResolvedTs) cancel() } @@ -693,15 +661,13 @@ consumePreResolvedTs: // TestCompatibilityWithSameConn tests kv client returns an error when TiKV returns // the Compatibility error. This error only happens when the same connection to // TiKV have different versions. -func (s *clientSuite) TestCompatibilityWithSameConn(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestCompatibilityWithSameConn(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) server1.Stop() @@ -709,10 +675,10 @@ func (s *clientSuite) TestCompatibilityWithSameConn(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr1) @@ -732,11 +698,11 @@ func (s *clientSuite) TestCompatibilityWithSameConn(c *check.C) { go func() { defer wg2.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockResolver, isPullInit, eventCh) - c.Assert(cerror.ErrVersionIncompatible.Equal(err), check.IsTrue) + require.True(t, cerror.ErrVersionIncompatible.Equal(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) incompatibility := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -757,16 +723,13 @@ func (s *clientSuite) TestCompatibilityWithSameConn(c *check.C) { // TestClusterIDMismatch tests kv client returns an error when TiKV returns // the cluster ID mismatch error. -func (s *clientSuite) TestClusterIDMismatch(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestClusterIDMismatch(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} changeDataCh := make(chan *cdcpb.ChangeDataEvent, 10) - changeDataService := newMockChangeDataService(c, changeDataCh) - mockService, addr := newMockService(ctx, c, changeDataService, wg) + changeDataService := newMockChangeDataService(t, changeDataCh) + mockService, addr := newMockService(ctx, t, changeDataService, wg) defer func() { close(changeDataCh) mockService.Stop() @@ -774,11 +737,11 @@ func (s *clientSuite) TestClusterIDMismatch(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr) @@ -800,11 +763,11 @@ func (s *clientSuite) TestClusterIDMismatch(c *check.C) { go func() { defer wg2.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockResolver, isPullInit, eventCh) - c.Assert(cerror.ErrClusterIDMismatch.Equal(err), check.IsTrue) + require.True(t, cerror.ErrClusterIDMismatch.Equal(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) clusterIDMismatchEvent := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -826,14 +789,13 @@ func (s *clientSuite) TestClusterIDMismatch(c *check.C) { cancel() } -func (s *clientSuite) testHandleFeedEvent(c *check.C) { - defer s.TearDownTest(c) +func testHandleFeedEvent(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -842,10 +804,10 @@ func (s *clientSuite) testHandleFeedEvent(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr1) @@ -864,11 +826,11 @@ func (s *clientSuite) testHandleFeedEvent(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) eventsBeforeInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ // before initialized, prewrite and commit could be in any sequence, @@ -1222,9 +1184,9 @@ func (s *clientSuite) testHandleFeedEvent(c *check.C) { for _, expectedEv := range expected { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, expectedEv) + require.Equal(t, expectedEv, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", expectedEv) + require.Fail(t, fmt.Sprintf("expected event %v not received", expectedEv)) } } @@ -1232,22 +1194,20 @@ func (s *clientSuite) testHandleFeedEvent(c *check.C) { for i := 0; i < multiSize; i++ { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, multipleExpected) + require.Equal(t, multipleExpected, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", multipleExpected) + require.Fail(t, fmt.Sprintf("expected event %v not received", multipleExpected)) } } cancel() } -func (s *clientSuite) TestHandleFeedEvent(c *check.C) { - defer testleak.AfterTest(c)() - s.testHandleFeedEvent(c) +func TestHandleFeedEvent(t *testing.T) { + testHandleFeedEvent(t) } -func (s *clientSuite) TestHandleFeedEventWithWorkerPool(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleFeedEventWithWorkerPool(t *testing.T) { hwm := regionWorkerHighWatermark lwm := regionWorkerLowWatermark regionWorkerHighWatermark = 8 @@ -1256,15 +1216,13 @@ func (s *clientSuite) TestHandleFeedEventWithWorkerPool(c *check.C) { regionWorkerHighWatermark = hwm regionWorkerLowWatermark = lwm }() - s.testHandleFeedEvent(c) + testHandleFeedEvent(t) } // TestStreamSendWithError mainly tests the scenario that the `Send` call of a gPRC // stream of kv client meets error, and kv client can clean up the broken stream, // establish a new one and recover the normal event feed processing. -func (s *clientSuite) TestStreamSendWithError(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestStreamSendWithError(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} defer wg.Wait() @@ -1273,8 +1231,8 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { var server1StopFlag int32 server1Stopped := make(chan struct{}) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { defer func() { // TiCDC may reestalish stream again, so we need to add failpoint-inject @@ -1296,10 +1254,10 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { } rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID3 := uint64(3) @@ -1320,13 +1278,13 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockerResolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() var requestIds sync.Map <-server1Stopped ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv2 := newMockChangeDataService(c, ch2) + srv2 := newMockChangeDataService(t, ch2) srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -1338,7 +1296,7 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { } } // Reuse the same listen address as server 1 - server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) + server2, _ := newMockServiceSpecificAddr(ctx, t, srv2, addr1, wg) defer func() { close(ch2) server2.Stop() @@ -1356,7 +1314,7 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { return errors.New("waiting for kv client requests received by server") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) reqID1, _ := requestIds.Load(regionID3) reqID2, _ := requestIds.Load(regionID4) initialized1 := mockInitializedEvent(regionID3, reqID1.(uint64)) @@ -1369,31 +1327,30 @@ func (s *clientSuite) TestStreamSendWithError(c *check.C) { for i := 0; i < 2; i++ { select { case event := <-eventCh: - c.Assert(event.Resolved, check.NotNil) + require.NotNil(t, event.Resolved) initRegions[event.RegionID] = struct{}{} case <-time.After(time.Second): - c.Errorf("expected events are not receive, received: %v", initRegions) + require.Fail(t, fmt.Sprintf("expected events are not receive, received: %v", initRegions)) } } expectedInitRegions := map[uint64]struct{}{regionID3: {}, regionID4: {}} - c.Assert(initRegions, check.DeepEquals, expectedInitRegions) + require.Equal(t, expectedInitRegions, initRegions) // a hack way to check the goroutine count of region worker is 1 buf := make([]byte, 1<<20) stackLen := runtime.Stack(buf, true) stack := string(buf[:stackLen]) - c.Assert(strings.Count(stack, "resolveLock"), check.Equals, 1) - c.Assert(strings.Count(stack, "collectWorkpoolError"), check.Equals, 1) + require.Equal(t, 1, strings.Count(stack, "resolveLock")) + require.Equal(t, 1, strings.Count(stack, "collectWorkpoolError")) } -func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { - defer s.TearDownTest(c) +func testStreamRecvWithError(t *testing.T, failpointStr string) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -1402,10 +1359,10 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -1413,7 +1370,7 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", failpointStr) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") }() @@ -1430,11 +1387,11 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized1 := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized1 err = retry.Do(context.Background(), func() error { @@ -1444,7 +1401,7 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { return errors.New("message is not sent") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) // another stream will be established, so we notify and wait the first // EventFeed loop exits. @@ -1452,11 +1409,11 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { select { case <-callback: case <-time.After(time.Second * 3): - c.Error("event feed loop can't exit") + require.Fail(t, "event feed loop can't exit") } // wait request id allocated with: new session, new request*2 - waitRequestID(c, baseAllocatedID+2) + waitRequestID(t, baseAllocatedID+2) initialized2 := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized2 @@ -1499,24 +1456,22 @@ eventLoop: break eventLoop } } - c.Assert(events, check.DeepEquals, expected) + require.Equal(t, expected, events) cancel() } // TestStreamRecvWithErrorAndResolvedGoBack mainly tests the scenario that the `Recv` call of a gPRC // stream in kv client meets error, and kv client reconnection with tikv with the current tso -func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestStreamRecvWithErrorAndResolvedGoBack(t *testing.T) { if !util.FailpointBuild { - c.Skip("skip when this is not a failpoint build") + t.Skip("skip when this is not a failpoint build") } ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} var requestID uint64 ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) + srv1 := newMockChangeDataService(t, ch1) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -1527,7 +1482,7 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { atomic.StoreUint64(&requestID, req.RequestId) } } - server1, addr1 := newMockService(ctx, c, srv1, wg) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -1536,10 +1491,10 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -1560,11 +1515,11 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { defer wg.Done() defer close(eventCh) err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) err = retry.Do(context.Background(), func() error { if atomic.LoadUint64(&requestID) == currentRequestID() { return nil @@ -1573,7 +1528,7 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { atomic.LoadUint64(&requestID), currentRequestID()) }, retry.WithBackoffBaseDelay(50), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) initialized1 := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized1 err = retry.Do(context.Background(), func() error { @@ -1583,7 +1538,7 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { return errors.New("message is not sent") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -1600,9 +1555,9 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { return errors.New("message is not sent") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"\")") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") }() @@ -1614,11 +1569,11 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { select { case <-callback: case <-time.After(time.Second * 3): - c.Error("event feed loop can't exit") + require.Fail(t, "event feed loop can't exit") } // wait request id allocated with: new session, new request*2 - waitRequestID(c, baseAllocatedID+2) + waitRequestID(t, baseAllocatedID+2) err = retry.Do(context.Background(), func() error { if atomic.LoadUint64(&requestID) == currentRequestID() { return nil @@ -1627,7 +1582,7 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { atomic.LoadUint64(&requestID), currentRequestID()) }, retry.WithBackoffBaseDelay(50), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) initialized2 := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized2 err = retry.Do(context.Background(), func() error { @@ -1637,7 +1592,7 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { return errors.New("message is not sent") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) resolved = &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -1662,13 +1617,13 @@ ReceiveLoop: break ReceiveLoop } case <-time.After(time.Second): - c.Errorf("event received timeout") + require.Fail(t, "event received timeout") } } var lastResolvedTs uint64 for _, e := range received { if lastResolvedTs > e.Resolved.ResolvedTs { - c.Errorf("the resolvedTs is back off %#v", resolved) + require.Fail(t, fmt.Sprintf("the resolvedTs is back off %#v", resolved)) } } } @@ -1676,19 +1631,16 @@ ReceiveLoop: // TestStreamRecvWithErrorNormal mainly tests the scenario that the `Recv` call // of a gPRC stream in kv client meets a **logical related** error, and kv client // logs the error and re-establish new request. -func (s *clientSuite) TestStreamRecvWithErrorNormal(c *check.C) { - defer testleak.AfterTest(c)() - s.testStreamRecvWithError(c, "1*return(\"injected stream recv error\")") +func TestStreamRecvWithErrorNormal(t *testing.T) { + testStreamRecvWithError(t, "1*return(\"injected stream recv error\")") } // TestStreamRecvWithErrorIOEOF mainly tests the scenario that the `Recv` call // of a gPRC stream in kv client meets error io.EOF, and kv client logs the error // and re-establish new request -func (s *clientSuite) TestStreamRecvWithErrorIOEOF(c *check.C) { - defer testleak.AfterTest(c)() - - s.testStreamRecvWithError(c, "1*return(\"EOF\")") - s.testStreamRecvWithError(c, "1*return(\"EOF\")") +func TestStreamRecvWithErrorIOEOF(t *testing.T) { + testStreamRecvWithError(t, "1*return(\"EOF\")") + testStreamRecvWithError(t, "1*return(\"EOF\")") } // TestIncompatibleTiKV tests TiCDC new request to TiKV meets `ErrVersionIncompatible` @@ -1696,9 +1648,7 @@ func (s *clientSuite) TestStreamRecvWithErrorIOEOF(c *check.C) { // TiCDC will wait 20s and then retry. This is a common scenario when rolling // upgrade a cluster and the new version is not compatible with the old version // (upgrade TiCDC before TiKV, since upgrade TiKV often takes much longer). -func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestIncompatibleTiKV(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} @@ -1721,7 +1671,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { var requestIds sync.Map ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) + srv1 := newMockChangeDataService(t, ch1) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -1732,7 +1682,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { requestIds.Store(req.RegionId, req.RequestId) } } - server1, addr1 := newMockService(ctx, c, srv1, wg) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -1741,10 +1691,10 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: gen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -1752,7 +1702,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientDelayWhenIncompatible", "return(true)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientDelayWhenIncompatible") }() @@ -1769,7 +1719,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() err = retry.Do(context.Background(), func() error { @@ -1779,7 +1729,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { return errors.Errorf("version generator is not updated in time, call time %d", atomic.LoadInt32(&call)) }, retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = retry.Do(context.Background(), func() error { _, ok := requestIds.Load(regionID) if ok { @@ -1788,16 +1738,16 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { return errors.New("waiting for kv client requests received by server") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) reqID, _ := requestIds.Load(regionID) initialized := mockInitializedEvent(regionID, reqID.(uint64)) ch1 <- initialized select { case event := <-eventCh: - c.Assert(event.Resolved, check.NotNil) - c.Assert(event.RegionID, check.Equals, regionID) + require.NotNil(t, event.Resolved) + require.Equal(t, regionID, event.RegionID) case <-time.After(time.Second): - c.Errorf("expected events are not receive") + require.Fail(t, "expected events are not receive") } cancel() @@ -1806,15 +1756,13 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { // TestPendingRegionError tests kv client should return an error when receiving // a new subscription (the first event of specific region) but the corresponding // region is not found in pending regions. -func (s *clientSuite) TestNoPendingRegionError(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestNoPendingRegionError(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) server1.Stop() @@ -1822,10 +1770,10 @@ func (s *clientSuite) TestNoPendingRegionError(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr1) @@ -1845,11 +1793,11 @@ func (s *clientSuite) TestNoPendingRegionError(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) noPendingRegionEvent := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -1862,8 +1810,8 @@ func (s *clientSuite) TestNoPendingRegionError(c *check.C) { initialized := mockInitializedEvent(3, currentRequestID()) ch1 <- initialized ev := <-eventCh - c.Assert(ev.Resolved, check.NotNil) - c.Assert(ev.Resolved.ResolvedTs, check.Equals, uint64(100)) + require.NotNil(t, ev.Resolved) + require.Equal(t, uint64(100), ev.Resolved.ResolvedTs) resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -1874,22 +1822,20 @@ func (s *clientSuite) TestNoPendingRegionError(c *check.C) { }} ch1 <- resolved ev = <-eventCh - c.Assert(ev.Resolved, check.NotNil) - c.Assert(ev.Resolved.ResolvedTs, check.Equals, uint64(200)) + require.NotNil(t, ev.Resolved) + require.Equal(t, uint64(200), ev.Resolved.ResolvedTs) cancel() } // TestDropStaleRequest tests kv client should drop an event if its request id is outdated. -func (s *clientSuite) TestDropStaleRequest(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestDropStaleRequest(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -1898,10 +1844,10 @@ func (s *clientSuite) TestDropStaleRequest(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -1921,11 +1867,11 @@ func (s *clientSuite) TestDropStaleRequest(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(regionID, currentRequestID()) eventsAfterInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ @@ -1976,24 +1922,22 @@ func (s *clientSuite) TestDropStaleRequest(c *check.C) { for _, expectedEv := range expected { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, expectedEv) + require.Equal(t, expectedEv, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", expectedEv) + require.Fail(t, fmt.Sprintf("expected event %v not received", expectedEv)) } } cancel() } // TestResolveLock tests the resolve lock logic in kv client -func (s *clientSuite) TestResolveLock(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestResolveLock(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -2002,10 +1946,10 @@ func (s *clientSuite) TestResolveLock(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -2013,7 +1957,7 @@ func (s *clientSuite) TestResolveLock(c *check.C) { cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval", "return(3)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval") }() @@ -2030,15 +1974,15 @@ func (s *clientSuite) TestResolveLock(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized physical, logical, err := pdClient.GetTS(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) tso := oracle.ComposeTS(physical, logical) resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -2067,9 +2011,9 @@ func (s *clientSuite) TestResolveLock(c *check.C) { for _, expectedEv := range expected { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, expectedEv) + require.Equal(t, expectedEv, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", expectedEv) + require.Fail(t, fmt.Sprintf("expected event %v not received", expectedEv)) } } @@ -2080,14 +2024,13 @@ func (s *clientSuite) TestResolveLock(c *check.C) { cancel() } -func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.ChangeDataEvent) { - defer s.TearDownTest(c) +func testEventCommitTsFallback(t *testing.T, events []*cdcpb.ChangeDataEvent) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -2096,10 +2039,10 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -2113,7 +2056,7 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan // This inject will make regionWorker exit directly and trigger execution line cancel when meet error err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientErrUnreachable", "return(true)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientErrUnreachable") }() @@ -2131,11 +2074,11 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan go func() { defer clientWg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(err, check.Equals, errUnreachable) + require.Equal(t, errUnreachable, err) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) for _, event := range events { for _, ev := range event.Events { ev.RequestId = currentRequestID() @@ -2147,8 +2090,7 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan } // TestCommittedFallback tests kv client should panic when receiving a fallback committed event -func (s *clientSuite) TestCommittedFallback(c *check.C) { - defer testleak.AfterTest(c)() +func TestCommittedFallback(t *testing.T) { events := []*cdcpb.ChangeDataEvent{ {Events: []*cdcpb.Event{ { @@ -2169,12 +2111,11 @@ func (s *clientSuite) TestCommittedFallback(c *check.C) { }, }}, } - s.testEventCommitTsFallback(c, events) + testEventCommitTsFallback(t, events) } // TestCommitFallback tests kv client should panic when receiving a fallback commit event -func (s *clientSuite) TestCommitFallback(c *check.C) { - defer testleak.AfterTest(c)() +func TestCommitFallback(t *testing.T) { events := []*cdcpb.ChangeDataEvent{ mockInitializedEvent(3, currentRequestID()), {Events: []*cdcpb.Event{ @@ -2195,12 +2136,11 @@ func (s *clientSuite) TestCommitFallback(c *check.C) { }, }}, } - s.testEventCommitTsFallback(c, events) + testEventCommitTsFallback(t, events) } // TestDeuplicateRequest tests kv client should panic when meeting a duplicate error -func (s *clientSuite) TestDuplicateRequest(c *check.C) { - defer testleak.AfterTest(c)() +func TestDuplicateRequest(t *testing.T) { events := []*cdcpb.ChangeDataEvent{ {Events: []*cdcpb.Event{ { @@ -2214,23 +2154,21 @@ func (s *clientSuite) TestDuplicateRequest(c *check.C) { }, }}, } - s.testEventCommitTsFallback(c, events) + testEventCommitTsFallback(t, events) } // testEventAfterFeedStop tests kv client can drop events sent after region feed is stopped // TODO: testEventAfterFeedStop is not stable, re-enable it after it is stable -// nolint:unused -func (s *clientSuite) testEventAfterFeedStop(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +// nolint +func testEventAfterFeedStop(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} var server1StopFlag int32 server1Stopped := make(chan struct{}) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { defer func() { // TiCDC may reestalish stream again, so we need to add failpoint-inject @@ -2253,10 +2191,10 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { } rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -2267,7 +2205,7 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { // before event feed processor is reconstruct, some duplicated events are // sent to event feed processor. err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientSingleFeedProcessDelay", "1*sleep(2000)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientSingleFeedProcessDelay") }() @@ -2284,11 +2222,11 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) // an error event will mark the corresponding region feed as stopped epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -2342,10 +2280,10 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { var requestID uint64 ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv2 := newMockChangeDataService(c, ch2) + srv2 := newMockChangeDataService(t, ch2) // Reuse the same listen addresss as server 1 to simulate TiKV handles the // gRPC stream terminate and reconnect. - server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) + server2, _ := newMockServiceSpecificAddr(ctx, t, srv2, addr1, wg) srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -2370,7 +2308,7 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { }, retry.WithMaxTries(10), retry.WithBackoffBaseDelay(500), retry.WithBackoffMaxDelay(60*1000)) log.Info("retry check request id", zap.Error(err)) - c.Assert(err, check.IsNil) + require.Nil(t, err) // wait request id allocated with: new session, 2 * new request committedClone.Events[0].RequestId = currentRequestID() @@ -2416,23 +2354,21 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { for _, expectedEv := range expected { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, expectedEv) + require.Equal(t, expectedEv, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", expectedEv) + require.Fail(t, fmt.Sprintf("expected event %v not received", expectedEv)) } } cancel() } -func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestOutOfRegionRangeEvent(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -2441,10 +2377,10 @@ func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr1) @@ -2463,11 +2399,11 @@ func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) eventsBeforeInit := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ // will be filtered out @@ -2624,9 +2560,9 @@ func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { for _, expectedEv := range expected { select { case event := <-eventCh: - c.Assert(event, check.DeepEquals, expectedEv) + require.Equal(t, expectedEv, event) case <-time.After(time.Second): - c.Errorf("expected event %v not received", expectedEv) + require.Fail(t, fmt.Sprintf("expected event %v not received", expectedEv)) } } @@ -2635,15 +2571,13 @@ func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { // TestResolveLockNoCandidate tests the resolved ts manager can work normally // when no region exceeds resolve lock interval, that is what candidate means. -func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestResolveLockNoCandidate(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -2652,10 +2586,10 @@ func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -2677,11 +2611,11 @@ func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(regionID, currentRequestID()) ch1 <- initialized @@ -2691,7 +2625,7 @@ func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { defer wg2.Done() for i := 0; i < 6; i++ { physical, logical, err := pdClient.GetTS(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) tso := oracle.ComposeTS(physical, logical) resolved := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -2703,9 +2637,9 @@ func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { ch1 <- resolved select { case event := <-eventCh: - c.Assert(event.Resolved, check.NotNil) + require.NotNil(t, event.Resolved) case <-time.After(time.Second): - c.Error("resolved event not received") + require.Fail(t, "resolved event not received") } // will sleep 6s totally, to ensure resolve lock fired once time.Sleep(time.Second) @@ -2723,15 +2657,13 @@ func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { // 2. We delay the kv client to re-create a new region request by 500ms via failpoint. // 3. Before new region request is fired, simulate kv client `stream.Recv` returns an error, the stream // handler will signal region worker to exit, which will evict all active region states then. -func (s *clientSuite) TestFailRegionReentrant(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestFailRegionReentrant(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -2740,10 +2672,10 @@ func (s *clientSuite) TestFailRegionReentrant(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -2751,9 +2683,9 @@ func (s *clientSuite) TestFailRegionReentrant(c *check.C) { cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantError", "1*return(\"ok\")->1*return(\"error\")") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantErrorDelay", "sleep(500)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantError") _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantErrorDelay") @@ -2771,11 +2703,11 @@ func (s *clientSuite) TestFailRegionReentrant(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) unknownErr := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { RegionId: 3, @@ -2808,22 +2740,19 @@ func (s *clientSuite) TestFailRegionReentrant(c *check.C) { // has been deleted in step-3, so it will create new stream but fails because // of unstable TiKV store, at this point, the kv client should handle with the // pending region correctly. -func (s *clientSuite) TestClientV1UnlockRangeReentrant(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestClientV1UnlockRangeReentrant(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID3 := uint64(3) @@ -2833,9 +2762,9 @@ func (s *clientSuite) TestClientV1UnlockRangeReentrant(c *check.C) { cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"injected stream recv error\")") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->1*sleep(2000)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay") @@ -2852,7 +2781,7 @@ func (s *clientSuite) TestClientV1UnlockRangeReentrant(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait the second region is scheduled @@ -2867,26 +2796,23 @@ func (s *clientSuite) TestClientV1UnlockRangeReentrant(c *check.C) { // TestClientErrNoPendingRegion has the similar procedure with TestClientV1UnlockRangeReentrant // The difference is the delay injected point for region 2 -func (s *clientSuite) TestClientErrNoPendingRegion(c *check.C) { - defer testleak.AfterTest(c)() - s.testClientErrNoPendingRegion(c) +func TestClientErrNoPendingRegion(t *testing.T) { + testClientErrNoPendingRegion(t) } -func (s *clientSuite) testClientErrNoPendingRegion(c *check.C) { - defer s.TearDownTest(c) - +func testClientErrNoPendingRegion(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID3 := uint64(3) @@ -2896,11 +2822,11 @@ func (s *clientSuite) testClientErrNoPendingRegion(c *check.C) { cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"injected error\")") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->2*sleep(1000)") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamCloseDelay", "sleep(2000)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay") @@ -2918,16 +2844,16 @@ func (s *clientSuite) testClientErrNoPendingRegion(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() baseAllocatedID := currentRequestID() // wait the second region is scheduled time.Sleep(time.Millisecond * 500) - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(regionID3, currentRequestID()) ch1 <- initialized - waitRequestID(c, baseAllocatedID+2) + waitRequestID(t, baseAllocatedID+2) initialized = mockInitializedEvent(regionID4, currentRequestID()) ch1 <- initialized // wait the kvClientPendingRegionDelay ends, and the second region is processed @@ -2939,15 +2865,15 @@ func (s *clientSuite) testClientErrNoPendingRegion(c *check.C) { } // TestKVClientForceReconnect force reconnect gRPC stream can work -func (s *clientSuite) testKVClientForceReconnect(c *check.C) { +func testKVClientForceReconnect(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} var server1StopFlag int32 server1Stopped := make(chan struct{}) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { defer func() { // There may be a gap between server.Recv error and ticdc stream reconnect, so we need to add failpoint-inject @@ -2970,10 +2896,10 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { } rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID3 := uint64(3) @@ -2992,11 +2918,11 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() baseAllocatedID := currentRequestID() - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) initialized := mockInitializedEvent(regionID3, currentRequestID()) ch1 <- initialized @@ -3005,7 +2931,7 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { var requestIds sync.Map ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv2 := newMockChangeDataService(c, ch2) + srv2 := newMockChangeDataService(t, ch2) srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -3018,7 +2944,7 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { } // Reuse the same listen addresss as server 1 to simulate TiKV handles the // gRPC stream terminate and reconnect. - server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) + server2, _ := newMockServiceSpecificAddr(ctx, t, srv2, addr1, wg) defer func() { close(ch2) server2.Stop() @@ -3036,7 +2962,7 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { return errors.New("waiting for kv client requests received by server") }, retry.WithBackoffBaseDelay(300), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) requestID, _ := requestIds.Load(regionID3) initialized = mockInitializedEvent(regionID3, requestID.(uint64)) @@ -3066,35 +2992,30 @@ eventLoop: if ev.Resolved != nil && ev.Resolved.ResolvedTs == uint64(100) { continue } - c.Assert(ev, check.DeepEquals, expected) + require.Equal(t, expected, ev) break eventLoop case <-time.After(time.Second): - c.Errorf("expected event %v not received", expected) + require.Fail(t, fmt.Sprintf("expected event %v not received", expected)) } } cancel() } -func (s *clientSuite) TestKVClientForceReconnect(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - - s.testKVClientForceReconnect(c) +func TestKVClientForceReconnect(t *testing.T) { + testKVClientForceReconnect(t) } // TestConcurrentProcessRangeRequest when region range request channel is full, // the kv client can process it correctly without deadlock. This is more likely // to happen when region split and merge frequently and large stale requests exist. -func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestConcurrentProcessRangeRequest(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} requestIDs := new(sync.Map) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) + srv1 := newMockChangeDataService(t, ch1) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -3104,7 +3025,7 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { requestIDs.Store(req.RegionId, req.RequestId) } } - server1, addr1 := newMockService(ctx, c, srv1, wg) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -3113,10 +3034,10 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck storeID := uint64(1) @@ -3126,7 +3047,7 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID) err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientMockRangeLock", "1*return(20)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientMockRangeLock") }() @@ -3142,7 +3063,7 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // the kv client is blocked by failpoint injection, and after region has split @@ -3169,7 +3090,7 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { return errors.Errorf("region number %d is not as expected %d", count, regionNum) }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) - c.Assert(err, check.IsNil) + require.Nil(t, err) // send initialized event and a resolved ts event to each region requestIDs.Range(func(key, value interface{}) bool { @@ -3199,7 +3120,7 @@ checkEvent: break checkEvent } case <-time.After(time.Second): - c.Errorf("no more events received") + require.Fail(t, "no more events received") } } @@ -3209,15 +3130,13 @@ checkEvent: // TestEvTimeUpdate creates a new event feed, send N committed events every 100ms, // use failpoint to set reconnect interval to 1s, the last event time of region // should be updated correctly and no reconnect is triggered -func (s *clientSuite) TestEvTimeUpdate(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestEvTimeUpdate(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer func() { close(ch1) @@ -3226,10 +3145,10 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { }() rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck cluster.AddStore(1, addr1) @@ -3238,7 +3157,7 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { originalReconnectInterval := reconnectInterval reconnectInterval = 1500 * time.Millisecond err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientCheckUnInitRegionInterval", "return(2)") - c.Assert(err, check.IsNil) + require.Nil(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientCheckUnInitRegionInterval") reconnectInterval = originalReconnectInterval @@ -3257,11 +3176,11 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) eventCount := 20 for i := 0; i < eventCount; i++ { @@ -3312,12 +3231,12 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { select { case event := <-eventCh: if i == 0 { - c.Assert(event, check.DeepEquals, expected[0]) + require.Equal(t, expected[0], event) } else { - c.Assert(event, check.DeepEquals, expected[1]) + require.Equal(t, expected[1], event) } case <-time.After(time.Second): - c.Errorf("expected event not received, %d received", i) + require.Fail(t, fmt.Sprintf("expected event not received, %d received", i)) } } @@ -3327,18 +3246,15 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { // TestRegionWorkerExitWhenIsIdle tests region worker can exit, and cancel gRPC // stream automatically when it is idle. // Idle means having no any effective region state -func (s *clientSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestRegionWorkerExitWhenIsIdle(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} server1Stopped := make(chan struct{}) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) defer close(ch1) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) defer server1.Stop() srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { defer func() { @@ -3355,10 +3271,10 @@ func (s *clientSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { } rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck regionID := uint64(3) @@ -3378,11 +3294,11 @@ func (s *clientSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { go func() { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, 100, false, lockresolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // wait request id allocated with: new session, new request - waitRequestID(c, baseAllocatedID+1) + waitRequestID(t, baseAllocatedID+1) // an error event will mark the corresponding region feed as stopped epochNotMatch := &cdcpb.ChangeDataEvent{Events: []*cdcpb.Event{ { @@ -3400,7 +3316,7 @@ func (s *clientSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { select { case <-server1Stopped: case <-time.After(time.Second): - c.Error("stream is not terminated by cdc kv client") + require.Fail(t, "stream is not terminated by cdc kv client") } cancel() } @@ -3410,10 +3326,7 @@ func (s *clientSuite) TestRegionWorkerExitWhenIsIdle(c *check.C) { // TiCDC catches this error and resets the gRPC stream. TiCDC must not send a // new request before closing gRPC stream since currently there is no mechanism // to release an existing region connection. -func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestPrewriteNotMatchError(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) wg := &sync.WaitGroup{} @@ -3421,8 +3334,8 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { var server1Stopped int32 = 0 server1StoppedCh := make(chan struct{}) ch1 := make(chan *cdcpb.ChangeDataEvent, 10) - srv1 := newMockChangeDataService(c, ch1) - server1, addr1 := newMockService(ctx, c, srv1, wg) + srv1 := newMockChangeDataService(t, ch1) + server1, addr1 := newMockService(ctx, t, srv1, wg) srv1.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { if atomic.LoadInt32(&server1Stopped) == int32(1) { return @@ -3444,10 +3357,10 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { } rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer kvStorage.Close() //nolint:errcheck // create two regions to avoid the stream is canceled by no region remained @@ -3471,7 +3384,7 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { go func() { defer wg.Done() err = cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, 100, false, lockResolver, isPullInit, eventCh) - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) }() // The expected request ids are agnostic because the kv client could retry @@ -3486,7 +3399,7 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { return errors.New("waiting for kv client requests received by server") }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) - c.Assert(err, check.IsNil) + require.Nil(t, err) reqID1, _ := requestIds.Load(regionID3) reqID2, _ := requestIds.Load(regionID4) initialized1 := mockInitializedEvent(regionID3, reqID1.(uint64)) @@ -3516,7 +3429,7 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { <-server1StoppedCh ch2 := make(chan *cdcpb.ChangeDataEvent, 10) - srv2 := newMockChangeDataService(c, ch2) + srv2 := newMockChangeDataService(t, ch2) srv2.recvLoop = func(server cdcpb.ChangeData_EventFeedServer) { for { req, err := server.Recv() @@ -3528,7 +3441,7 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { } } // Reuse the same listen address as server 1 - server2, _ := newMockServiceSpecificAddr(ctx, c, srv2, addr1, wg) + server2, _ := newMockServiceSpecificAddr(ctx, t, srv2, addr1, wg) defer func() { close(ch2) server2.Stop() @@ -3537,7 +3450,7 @@ func (s *clientSuite) TestPrewriteNotMatchError(c *check.C) { // After the gRPC stream is canceled, two more reqeusts will be sent, so the // allocated id is increased by 2 from baseAllocatedID+2. - waitRequestID(c, baseAllocatedID+4) + waitRequestID(t, baseAllocatedID+4) cancel() } @@ -3552,10 +3465,7 @@ func createFakeEventFeedSession(ctx context.Context) *eventFeedSession { nil /*eventCh*/) } -func (s *clientSuite) TestCheckRateLimit(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestCheckRateLimit(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -3571,18 +3481,15 @@ func (s *clientSuite) TestCheckRateLimit(c *check.C) { } } if trigger == maxTrigger { - c.Error("get rate limiter too slow") + require.Fail(t, "get rate limiter too slow") } - c.Assert(trigger, check.GreaterEqual, burst) + require.GreaterOrEqual(t, trigger, burst) time.Sleep(100 * time.Millisecond) allowed := session.checkRateLimit(1) - c.Assert(allowed, check.IsTrue) + require.True(t, allowed) } -func (s *clientSuite) TestHandleRateLimit(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func TestHandleRateLimit(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -3590,18 +3497,18 @@ func (s *clientSuite) TestHandleRateLimit(c *check.C) { // empty rate limit item, do nothing session.handleRateLimit(ctx) - c.Assert(session.rateLimitQueue, check.HasLen, 0) - c.Assert(cap(session.rateLimitQueue), check.Equals, defaultRegionRateLimitQueueSize) + require.Len(t, session.rateLimitQueue, 0) + require.Equal(t, defaultRegionRateLimitQueueSize, cap(session.rateLimitQueue)) for i := 0; i < defaultRegionRateLimitQueueSize+1; i++ { session.rateLimitQueue = append(session.rateLimitQueue, regionErrorInfo{}) } session.handleRateLimit(ctx) - c.Assert(session.rateLimitQueue, check.HasLen, 1) - c.Assert(cap(session.rateLimitQueue), check.Equals, 1) + require.Len(t, session.rateLimitQueue, 1) + require.Equal(t, 1, cap(session.rateLimitQueue)) session.handleRateLimit(ctx) - c.Assert(session.rateLimitQueue, check.HasLen, 0) - c.Assert(cap(session.rateLimitQueue), check.Equals, 128) + require.Len(t, session.rateLimitQueue, 0) + require.Equal(t, 128, cap(session.rateLimitQueue)) } func TestRegionErrorInfoLogRateLimitedHint(t *testing.T) { diff --git a/cdc/kv/grpc_pool_impl_test.go b/cdc/kv/grpc_pool_impl_test.go index 70e8b7413bb..4b3e34cb232 100644 --- a/cdc/kv/grpc_pool_impl_test.go +++ b/cdc/kv/grpc_pool_impl_test.go @@ -15,19 +15,19 @@ package kv import ( "context" + "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) // Use clientSuite for some special reasons, the embed etcd uses zap as the only candidate // logger and in the logger initialization it also initializes the grpclog/loggerv2, which // is not a thread-safe operation and it must be called before any gRPC functions // ref: https://github.com/grpc/grpc-go/blob/master/grpclog/loggerv2.go#L67-L72 -func (s *clientSuite) TestConnArray(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestConnArray(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -35,33 +35,33 @@ func (s *clientSuite) TestConnArray(c *check.C) { defer pool.Close() addr := "127.0.0.1:20161" conn, err := pool.GetConn(addr) - c.Assert(err, check.IsNil) - c.Assert(conn.active, check.Equals, int64(1)) + require.Nil(t, err) + require.Equal(t, int64(1), conn.active) pool.ReleaseConn(conn, addr) - c.Assert(conn.active, check.Equals, int64(0)) + require.Equal(t, int64(0), conn.active) lastConn := conn // First grpcConnCapacity*2 connections will use initial two connections. for i := 0; i < grpcConnCapacity*2; i++ { conn, err := pool.GetConn(addr) - c.Assert(err, check.IsNil) - c.Assert(lastConn.ClientConn, check.Not(check.Equals), conn.ClientConn) - c.Assert(conn.active, check.Equals, int64(i)/2+1) + require.Nil(t, err) + require.NotSame(t, conn.ClientConn, lastConn.ClientConn) + require.Equal(t, int64(i)/2+1, conn.active) lastConn = conn } // The following grpcConnCapacity*2 connections will trigger resize of connection array. for i := 0; i < grpcConnCapacity*2; i++ { conn, err := pool.GetConn(addr) - c.Assert(err, check.IsNil) - c.Assert(lastConn.ClientConn, check.Not(check.Equals), conn.ClientConn) - c.Assert(conn.active, check.Equals, int64(i)/2+1) + require.Nil(t, err) + require.NotSame(t, conn.ClientConn, lastConn.ClientConn) + require.Equal(t, int64(i)/2+1, conn.active) lastConn = conn } } -func (s *clientSuite) TestConnArrayRecycle(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestConnArrayRecycle(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -75,7 +75,7 @@ func (s *clientSuite) TestConnArrayRecycle(c *check.C) { // get conn for 6000 times, and grpc pool will create 6 buckets for i := 0; i < grpcConnCapacity*bucket; i++ { conn, err := pool.GetConn(addr) - c.Assert(err, check.IsNil) + require.Nil(t, err) if i%(grpcConnCapacity*resizeBucketStep) == 0 { sharedConns[i/grpcConnCapacity] = conn } @@ -84,22 +84,22 @@ func (s *clientSuite) TestConnArrayRecycle(c *check.C) { } } for i := 2; i < bucket; i++ { - c.Assert(sharedConns[i].active, check.Equals, int64(grpcConnCapacity)) + require.Equal(t, int64(grpcConnCapacity), sharedConns[i].active) for j := 0; j < grpcConnCapacity; j++ { pool.ReleaseConn(sharedConns[i], addr) } } empty := pool.bucketConns[addr].recycle() - c.Assert(empty, check.IsFalse) - c.Assert(pool.bucketConns[addr].conns, check.HasLen, 2) + require.False(t, empty) + require.Len(t, pool.bucketConns[addr].conns, 2) for i := 0; i < 2; i++ { - c.Assert(sharedConns[i].active, check.Equals, int64(grpcConnCapacity)) + require.Equal(t, int64(grpcConnCapacity), sharedConns[i].active) for j := 0; j < grpcConnCapacity; j++ { pool.ReleaseConn(sharedConns[i], addr) } } empty = pool.bucketConns[addr].recycle() - c.Assert(empty, check.IsTrue) - c.Assert(pool.bucketConns[addr].conns, check.HasLen, 0) + require.True(t, empty) + require.Len(t, pool.bucketConns[addr].conns, 0) } diff --git a/cdc/kv/main_test.go b/cdc/kv/main_test.go new file mode 100644 index 00000000000..9fb556a0982 --- /dev/null +++ b/cdc/kv/main_test.go @@ -0,0 +1,30 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/pingcap/tiflow/pkg/workerpool.(*worker).run"), + goleak.IgnoreTopFunction("sync.runtime_Semacquire"), + } + + leakutil.SetUpLeakTest(m, opts...) +} diff --git a/cdc/kv/matcher_test.go b/cdc/kv/matcher_test.go index fd9f745e7aa..8270f314261 100644 --- a/cdc/kv/matcher_test.go +++ b/cdc/kv/matcher_test.go @@ -14,17 +14,14 @@ package kv import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/kvproto/pkg/cdcpb" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type matcherSuite struct{} - -var _ = check.Suite(&matcherSuite{}) - -func (s *matcherSuite) TestMatchRow(c *check.C) { - defer testleak.AfterTest(c)() +func TestMatchRow(t *testing.T) { + t.Parallel() matcher := newMatcher() matcher.putPrewriteRow(&cdcpb.Event_Row{ StartTs: 1, @@ -48,11 +45,11 @@ func (s *matcherSuite) TestMatchRow(c *check.C) { Key: []byte("k1"), } ok := matcher.matchRow(commitRow1) - c.Assert(ok, check.IsFalse) - c.Assert(commitRow1, check.DeepEquals, &cdcpb.Event_Row{ + require.False(t, ok) + require.Equal(t, &cdcpb.Event_Row{ StartTs: 1, Key: []byte("k1"), - }) + }, commitRow1) // test match commit commitRow2 := &cdcpb.Event_Row{ @@ -61,18 +58,18 @@ func (s *matcherSuite) TestMatchRow(c *check.C) { Key: []byte("k1"), } ok = matcher.matchRow(commitRow2) - c.Assert(ok, check.IsTrue) - c.Assert(commitRow2, check.DeepEquals, &cdcpb.Event_Row{ + require.True(t, ok) + require.Equal(t, &cdcpb.Event_Row{ StartTs: 2, CommitTs: 3, Key: []byte("k1"), Value: []byte("v2"), OldValue: []byte("v3"), - }) + }, commitRow2) } -func (s *matcherSuite) TestMatchFakePrewrite(c *check.C) { - defer testleak.AfterTest(c)() +func TestMatchFakePrewrite(t *testing.T) { + t.Parallel() matcher := newMatcher() matcher.putPrewriteRow(&cdcpb.Event_Row{ StartTs: 1, @@ -93,20 +90,20 @@ func (s *matcherSuite) TestMatchFakePrewrite(c *check.C) { Key: []byte("k1"), } ok := matcher.matchRow(commitRow1) - c.Assert(commitRow1, check.DeepEquals, &cdcpb.Event_Row{ + require.Equal(t, &cdcpb.Event_Row{ StartTs: 1, CommitTs: 2, Key: []byte("k1"), Value: []byte("v1"), OldValue: []byte("v3"), - }) - c.Assert(ok, check.IsTrue) + }, commitRow1) + require.True(t, ok) } -func (s *matcherSuite) TestMatchMatchCachedRow(c *check.C) { - defer testleak.AfterTest(c)() +func TestMatchMatchCachedRow(t *testing.T) { + t.Parallel() matcher := newMatcher() - c.Assert(len(matcher.matchCachedRow()), check.Equals, 0) + require.Equal(t, 0, len(matcher.matchCachedRow())) matcher.cacheCommitRow(&cdcpb.Event_Row{ StartTs: 1, CommitTs: 2, @@ -122,7 +119,7 @@ func (s *matcherSuite) TestMatchMatchCachedRow(c *check.C) { CommitTs: 5, Key: []byte("k3"), }) - c.Assert(len(matcher.matchCachedRow()), check.Equals, 0) + require.Equal(t, 0, len(matcher.matchCachedRow())) matcher.cacheCommitRow(&cdcpb.Event_Row{ StartTs: 1, @@ -159,7 +156,7 @@ func (s *matcherSuite) TestMatchMatchCachedRow(c *check.C) { OldValue: []byte("ov3"), }) - c.Assert(matcher.matchCachedRow(), check.DeepEquals, []*cdcpb.Event_Row{{ + require.Equal(t, []*cdcpb.Event_Row{{ StartTs: 1, CommitTs: 2, Key: []byte("k1"), @@ -171,5 +168,5 @@ func (s *matcherSuite) TestMatchMatchCachedRow(c *check.C) { Key: []byte("k2"), Value: []byte("v2"), OldValue: []byte("ov2"), - }}) + }}, matcher.matchCachedRow()) } diff --git a/cdc/kv/region_worker_test.go b/cdc/kv/region_worker_test.go index 13eac0d4ca7..0ec3d27c290 100644 --- a/cdc/kv/region_worker_test.go +++ b/cdc/kv/region_worker_test.go @@ -17,32 +17,26 @@ import ( "math/rand" "runtime" "sync" + "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type regionWorkerSuite struct{} - -var _ = check.Suite(®ionWorkerSuite{}) - -func (s *regionWorkerSuite) TestRegionStateManager(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionStateManager(t *testing.T) { rsm := newRegionStateManager(4) regionID := uint64(1000) _, ok := rsm.getState(regionID) - c.Assert(ok, check.IsFalse) + require.False(t, ok) rsm.setState(regionID, ®ionFeedState{requestID: 2}) state, ok := rsm.getState(regionID) - c.Assert(ok, check.IsTrue) - c.Assert(state.requestID, check.Equals, uint64(2)) + require.True(t, ok) + require.Equal(t, uint64(2), state.requestID) } -func (s *regionWorkerSuite) TestRegionStateManagerThreadSafe(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionStateManagerThreadSafe(t *testing.T) { rsm := newRegionStateManager(4) regionCount := 100 regionIDs := make([]uint64, regionCount) @@ -62,9 +56,9 @@ func (s *regionWorkerSuite) TestRegionStateManagerThreadSafe(c *check.C) { idx := rand.Intn(regionCount) regionID := regionIDs[idx] s, ok := rsm.getState(regionID) - c.Assert(ok, check.IsTrue) + require.True(t, ok) s.lock.RLock() - c.Assert(s.requestID, check.Equals, uint64(idx+1)) + require.Equal(t, uint64(idx+1), s.requestID) s.lock.RUnlock() } }() @@ -79,7 +73,7 @@ func (s *regionWorkerSuite) TestRegionStateManagerThreadSafe(c *check.C) { } regionID := regionIDs[rand.Intn(regionCount)] s, ok := rsm.getState(regionID) - c.Assert(ok, check.IsTrue) + require.True(t, ok) s.lock.Lock() s.lastResolvedTs += 10 s.lock.Unlock() @@ -92,29 +86,26 @@ func (s *regionWorkerSuite) TestRegionStateManagerThreadSafe(c *check.C) { totalResolvedTs := uint64(0) for _, regionID := range regionIDs { s, ok := rsm.getState(regionID) - c.Assert(ok, check.IsTrue) - c.Assert(s.lastResolvedTs, check.Greater, uint64(1000)) + require.True(t, ok) + require.Greater(t, s.lastResolvedTs, uint64(1000)) totalResolvedTs += s.lastResolvedTs } // 100 regions, initial resolved ts 1000; // 2000 * resolved ts forward, increased by 10 each time, routine number is `concurrency`. - c.Assert(totalResolvedTs, check.Equals, uint64(100*1000+2000*10*concurrency)) + require.Equal(t, uint64(100*1000+2000*10*concurrency), totalResolvedTs) } -func (s *regionWorkerSuite) TestRegionStateManagerBucket(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionStateManagerBucket(t *testing.T) { rsm := newRegionStateManager(-1) - c.Assert(rsm.bucket, check.GreaterEqual, minRegionStateBucket) - c.Assert(rsm.bucket, check.LessEqual, maxRegionStateBucket) + require.GreaterOrEqual(t, rsm.bucket, minRegionStateBucket) + require.LessOrEqual(t, rsm.bucket, maxRegionStateBucket) bucket := rsm.bucket * 2 rsm = newRegionStateManager(bucket) - c.Assert(rsm.bucket, check.Equals, bucket) + require.Equal(t, bucket, rsm.bucket) } -func (s *regionWorkerSuite) TestRegionWorkerPoolSize(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRegionWorkerPoolSize(t *testing.T) { conf := config.GetDefaultServerConfig() conf.KVClient.WorkerPoolSize = 0 config.StoreGlobalServerConfig(conf) @@ -125,13 +116,13 @@ func (s *regionWorkerSuite) TestRegionWorkerPoolSize(c *check.C) { } return b } - c.Assert(size, check.Equals, min(runtime.NumCPU()*2, maxWorkerPoolSize)) + require.Equal(t, min(runtime.NumCPU()*2, maxWorkerPoolSize), size) conf.KVClient.WorkerPoolSize = 5 size = getWorkerPoolSize() - c.Assert(size, check.Equals, 5) + require.Equal(t, 5, size) conf.KVClient.WorkerPoolSize = maxWorkerPoolSize + 1 size = getWorkerPoolSize() - c.Assert(size, check.Equals, maxWorkerPoolSize) + require.Equal(t, maxWorkerPoolSize, size) } diff --git a/cdc/kv/resolvedts_heap_test.go b/cdc/kv/resolvedts_heap_test.go index 4fe92e9b5f1..1c4d3f87ff4 100644 --- a/cdc/kv/resolvedts_heap_test.go +++ b/cdc/kv/resolvedts_heap_test.go @@ -14,25 +14,21 @@ package kv import ( + "testing" "time" - "github.com/pingcap/check" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type rtsHeapSuite struct{} - -var _ = check.Suite(&rtsHeapSuite{}) - -func checkRegionTsInfoWithoutEvTime(c *check.C, obtained, expected *regionTsInfo) { - c.Assert(obtained.regionID, check.Equals, expected.regionID) - c.Assert(obtained.index, check.Equals, expected.index) - c.Assert(obtained.ts.resolvedTs, check.Equals, expected.ts.resolvedTs) - c.Assert(obtained.ts.sortByEvTime, check.IsFalse) +func checkRegionTsInfoWithoutEvTime(t *testing.T, obtained, expected *regionTsInfo) { + require.Equal(t, expected.regionID, obtained.regionID) + require.Equal(t, expected.index, obtained.index) + require.Equal(t, expected.ts.resolvedTs, obtained.ts.resolvedTs) + require.False(t, obtained.ts.sortByEvTime) } -func (s *rtsHeapSuite) TestRegionTsManagerResolvedTs(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionTsManagerResolvedTs(t *testing.T) { + t.Parallel() mgr := newRegionTsManager() initRegions := []*regionTsInfo{ {regionID: 102, ts: newResolvedTsItem(1040)}, @@ -42,14 +38,14 @@ func (s *rtsHeapSuite) TestRegionTsManagerResolvedTs(c *check.C) { for _, rts := range initRegions { mgr.Upsert(rts) } - c.Assert(mgr.Len(), check.Equals, 3) + require.Equal(t, 3, mgr.Len()) rts := mgr.Pop() - checkRegionTsInfoWithoutEvTime(c, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1000), index: -1}) + checkRegionTsInfoWithoutEvTime(t, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1000), index: -1}) // resolved ts is not updated mgr.Upsert(rts) rts = mgr.Pop() - checkRegionTsInfoWithoutEvTime(c, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1000), index: -1}) + checkRegionTsInfoWithoutEvTime(t, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1000), index: -1}) // resolved ts updated rts.ts.resolvedTs = 1001 @@ -57,17 +53,17 @@ func (s *rtsHeapSuite) TestRegionTsManagerResolvedTs(c *check.C) { mgr.Upsert(®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1100)}) rts = mgr.Pop() - checkRegionTsInfoWithoutEvTime(c, rts, ®ionTsInfo{regionID: 101, ts: newResolvedTsItem(1020), index: -1}) + checkRegionTsInfoWithoutEvTime(t, rts, ®ionTsInfo{regionID: 101, ts: newResolvedTsItem(1020), index: -1}) rts = mgr.Pop() - checkRegionTsInfoWithoutEvTime(c, rts, ®ionTsInfo{regionID: 102, ts: newResolvedTsItem(1040), index: -1}) + checkRegionTsInfoWithoutEvTime(t, rts, ®ionTsInfo{regionID: 102, ts: newResolvedTsItem(1040), index: -1}) rts = mgr.Pop() - checkRegionTsInfoWithoutEvTime(c, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1100), index: -1}) + checkRegionTsInfoWithoutEvTime(t, rts, ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(1100), index: -1}) rts = mgr.Pop() - c.Assert(rts, check.IsNil) + require.Nil(t, rts) } -func (s *rtsHeapSuite) TestRegionTsManagerPenalty(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionTsManagerPenalty(t *testing.T) { + t.Parallel() mgr := newRegionTsManager() initRegions := []*regionTsInfo{ {regionID: 100, ts: newResolvedTsItem(1000)}, @@ -75,7 +71,7 @@ func (s *rtsHeapSuite) TestRegionTsManagerPenalty(c *check.C) { for _, rts := range initRegions { mgr.Upsert(rts) } - c.Assert(mgr.Len(), check.Equals, 1) + require.Equal(t, 1, mgr.Len()) // test penalty increases if resolved ts keeps unchanged for i := 0; i < 6; i++ { @@ -83,20 +79,20 @@ func (s *rtsHeapSuite) TestRegionTsManagerPenalty(c *check.C) { mgr.Upsert(rts) } rts := mgr.Pop() - c.Assert(rts.ts.resolvedTs, check.Equals, uint64(1000)) - c.Assert(rts.ts.penalty, check.Equals, 6) + require.Equal(t, uint64(1000), rts.ts.resolvedTs) + require.Equal(t, 6, rts.ts.penalty) // test penalty is cleared to zero if resolved ts is advanced mgr.Upsert(rts) rtsNew := ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(2000)} mgr.Upsert(rtsNew) rts = mgr.Pop() - c.Assert(rts.ts.penalty, check.DeepEquals, 0) - c.Assert(rts.ts.resolvedTs, check.DeepEquals, uint64(2000)) + require.Equal(t, 0, rts.ts.penalty) + require.Equal(t, uint64(2000), rts.ts.resolvedTs) } -func (s *rtsHeapSuite) TestRegionTsManagerPenaltyForFallBackEvent(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionTsManagerPenaltyForFallBackEvent(t *testing.T) { + t.Parallel() mgr := newRegionTsManager() initRegions := []*regionTsInfo{ {regionID: 100, ts: newResolvedTsItem(1000)}, @@ -104,7 +100,7 @@ func (s *rtsHeapSuite) TestRegionTsManagerPenaltyForFallBackEvent(c *check.C) { for _, rts := range initRegions { mgr.Upsert(rts) } - c.Assert(mgr.Len(), check.Equals, 1) + require.Equal(t, 1, mgr.Len()) // test penalty increases if we meet a fallback event for i := 0; i < 6; i++ { @@ -113,20 +109,20 @@ func (s *rtsHeapSuite) TestRegionTsManagerPenaltyForFallBackEvent(c *check.C) { } rts := mgr.Pop() // original resolvedTs will remain unchanged - c.Assert(rts.ts.resolvedTs, check.Equals, uint64(1000)) - c.Assert(rts.ts.penalty, check.Equals, 6) + require.Equal(t, uint64(1000), rts.ts.resolvedTs) + require.Equal(t, 6, rts.ts.penalty) // test penalty is cleared to zero if resolved ts is advanced mgr.Upsert(rts) rtsNew := ®ionTsInfo{regionID: 100, ts: newResolvedTsItem(2000)} mgr.Upsert(rtsNew) rts = mgr.Pop() - c.Assert(rts.ts.penalty, check.DeepEquals, 0) - c.Assert(rts.ts.resolvedTs, check.DeepEquals, uint64(2000)) + require.Equal(t, 0, rts.ts.penalty) + require.Equal(t, uint64(2000), rts.ts.resolvedTs) } -func (s *rtsHeapSuite) TestRegionTsManagerEvTime(c *check.C) { - defer testleak.AfterTest(c)() +func TestRegionTsManagerEvTime(t *testing.T) { + t.Parallel() mgr := newRegionTsManager() initRegions := []*regionTsInfo{ {regionID: 100, ts: newEventTimeItem()}, @@ -136,14 +132,14 @@ func (s *rtsHeapSuite) TestRegionTsManagerEvTime(c *check.C) { mgr.Upsert(item) } info := mgr.Remove(101) - c.Assert(info.regionID, check.Equals, uint64(101)) + require.Equal(t, uint64(101), info.regionID) ts := time.Now() mgr.Upsert(®ionTsInfo{regionID: 100, ts: newEventTimeItem()}) info = mgr.Pop() - c.Assert(info.regionID, check.Equals, uint64(100)) - c.Assert(ts.Before(info.ts.eventTime), check.IsTrue) - c.Assert(time.Now().After(info.ts.eventTime), check.IsTrue) + require.Equal(t, uint64(100), info.regionID) + require.True(t, ts.Before(info.ts.eventTime)) + require.True(t, time.Now().After(info.ts.eventTime)) info = mgr.Pop() - c.Assert(info, check.IsNil) + require.Nil(t, info) } diff --git a/cdc/kv/store_op.go b/cdc/kv/store_op.go index cce8c6c40dd..22771feff10 100644 --- a/cdc/kv/store_op.go +++ b/cdc/kv/store_op.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/errors" tidbconfig "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/kv" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/store" @@ -36,7 +35,7 @@ func GetSnapshotMeta(tiStore tidbkv.Storage, ts uint64) (*meta.Meta, error) { } // CreateTiStore creates a new tikv storage client -func CreateTiStore(urls string, credential *security.Credential) (kv.Storage, error) { +func CreateTiStore(urls string, credential *security.Credential) (tidbkv.Storage, error) { urlv, err := flags.NewURLsValue(urls) if err != nil { return nil, errors.Trace(err) diff --git a/cdc/kv/token_region_test.go b/cdc/kv/token_region_test.go index a5ad7d5279e..4e99c985727 100644 --- a/cdc/kv/token_region_test.go +++ b/cdc/kv/token_region_test.go @@ -17,21 +17,17 @@ import ( "context" "fmt" "sync/atomic" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" "golang.org/x/sync/errgroup" ) -type tokenRegionSuite struct{} - -var _ = check.Suite(&tokenRegionSuite{}) - -func (s *tokenRegionSuite) TestRouter(c *check.C) { - defer testleak.AfterTest(c)() +func TestRouter(t *testing.T) { + t.Parallel() store := "store-1" limit := 10 r := NewSizedRegionRouter(context.Background(), limit) @@ -43,31 +39,31 @@ func (s *tokenRegionSuite) TestRouter(c *check.C) { for i := 0; i < limit; i++ { select { case sri := <-r.Chan(): - c.Assert(sri.ts, check.Equals, uint64(i)) + require.Equal(t, uint64(i), sri.ts) r.Acquire(store) regions = append(regions, sri) default: - c.Error("expect region info from router") + t.Error("expect region info from router") } } - c.Assert(r.tokens[store], check.Equals, limit) + require.Equal(t, limit, r.tokens[store]) for range regions { r.Release(store) } - c.Assert(r.tokens[store], check.Equals, 0) + require.Equal(t, 0, r.tokens[store]) } -func (s *tokenRegionSuite) TestRouterWithFastConsumer(c *check.C) { - defer testleak.AfterTest(c)() - s.testRouterWithConsumer(c, func() {}) +func TestRouterWithFastConsumer(t *testing.T) { + t.Parallel() + testRouterWithConsumer(t, func() {}) } -func (s *tokenRegionSuite) TestRouterWithSlowConsumer(c *check.C) { - defer testleak.AfterTest(c)() - s.testRouterWithConsumer(c, func() { time.Sleep(time.Millisecond * 15) }) +func TestRouterWithSlowConsumer(t *testing.T) { + t.Parallel() + testRouterWithConsumer(t, func() { time.Sleep(time.Millisecond * 15) }) } -func (s *tokenRegionSuite) testRouterWithConsumer(c *check.C, funcDoSth func()) { +func testRouterWithConsumer(t *testing.T, funcDoSth func()) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -118,13 +114,12 @@ func (s *tokenRegionSuite) testRouterWithConsumer(c *check.C, funcDoSth func()) } err := wg.Wait() - c.Assert(errors.Cause(err), check.Equals, context.Canceled) - c.Assert(r.tokens[store], check.Equals, 0) + require.Equal(t, context.Canceled, errors.Cause(err)) + require.Equal(t, 0, r.tokens[store]) } -func (s *tokenRegionSuite) TestRouterWithMultiStores(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRouterWithMultiStores(t *testing.T) { + t.Parallel() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -173,8 +168,8 @@ func (s *tokenRegionSuite) TestRouterWithMultiStores(c *check.C) { } err := wg.Wait() - c.Assert(errors.Cause(err), check.Equals, context.Canceled) + require.Equal(t, context.Canceled, errors.Cause(err)) for _, store := range stores { - c.Assert(r.tokens[store], check.Equals, 0) + require.Equal(t, 0, r.tokens[store]) } } From 87cfd448649a68464f7c52aa252e70c3f27e3bcf Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Tue, 15 Feb 2022 16:55:38 +0800 Subject: [PATCH 65/72] pkg/actor(ticdc): reduce metrics overhead (#4585) close pingcap/tiflow#4584 --- metrics/grafana/ticdc.json | 1096 ++++++++++++++++++++++++++++++++---- pkg/actor/metrics.go | 36 +- pkg/actor/system.go | 80 ++- 3 files changed, 1058 insertions(+), 154 deletions(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 9a9f8c9a4c2..9e08e9a549c 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -624,7 +624,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -2297,7 +2297,7 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "total", + "legendFormat": "{{changefeed}}", "refId": "B" } ], @@ -12295,49 +12295,47 @@ "x": 0, "y": 9 }, - "id": 155, + "id": 403, "panels": [ { "aliasColors": {}, - "bars": true, - "cacheTimeout": null, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "CPU usage of TiCDC actors", + "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 6 + "y": 10 }, - "id": 157, + "hiddenSeries": false, + "id": 418, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "hideEmpty": true, - "hideZero": true, - "max": false, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sideWidth": null, "total": false, - "values": false + "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], - "maxPerRow": 3, "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "repeat": null, @@ -12381,73 +12379,23 @@ } ], "spaceLength": 10, - "stack": true, + "stack": false, "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "expr": "sum(rate(ticdc_actor_worker_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}[1m])) by (instance, name)", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "alloc-from-os", + "legendFormat": "{{instance}}-{{name}}", "refId": "A" - }, - { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "estimate-inuse", - "refId": "H" - }, - { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "estimate-garbage", - "refId": "C" - }, - { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "reserved-by-go", - "refId": "B" - }, - { - "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "used-by-go", - "refId": "D" - }, - { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "gc-threshold", - "refId": "E" - }, - { - "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "gc", - "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory Usage", + "title": "CPU usage", "tooltip": { - "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -12462,19 +12410,19 @@ }, "yaxes": [ { - "format": "bytes", - "label": "", + "format": "percentunit", + "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { "format": "short", - "label": "", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -12486,44 +12434,42 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Count of live objects.", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The number of workers for actor systems", + "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 6 + "y": 10 }, - "id": 158, + "hiddenSeries": false, + "id": 421, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "hideEmpty": true, - "hideZero": true, - "max": false, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sideWidth": null, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "maxPerRow": 3, "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "repeat": null, @@ -12546,11 +12492,10 @@ "steppedLine": false, "targets": [ { - "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "expr": "ticdc_actor_number_of_workers{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "objects", + "legendFormat": "{{instance}}-{{name}}", "refId": "A" } ], @@ -12558,9 +12503,8 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Estimated Live Objects", + "title": "Worker count", "tooltip": { - "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -12576,18 +12520,18 @@ "yaxes": [ { "format": "short", - "label": "", + "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { "format": "short", - "label": "", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -12596,32 +12540,114 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time of slow polling actors (> 100 ms)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 17 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 414, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_actor_slow_poll_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Slow poll duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC process Go garbage collection STW pause duration", - "editable": true, - "error": false, + "description": "Percentiles of slow polling actors duration (> 100 ms)", "fill": 1, - "grid": {}, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 13 + "x": 12, + "y": 17 }, - "id": 160, + "hiddenSeries": false, + "id": 420, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -12629,7 +12655,815 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/slow.*/", + "bars": true, + "fill": 0, + "transform": "negative-Y", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.80, sum(rate(ticdc_actor_slow_poll_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}[1m])) by (le,name)) < 5", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}-p80", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_actor_slow_poll_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}[1m])) by (le,name)) < 5", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_actor_slow_poll_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\"}[1m])) by (le,name)) >= 5", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "slow-{{name}}-p99", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Slow poll duration percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average of message batch size. ", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 24 + }, + "hiddenSeries": false, + "id": 428, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_actor_batch_size_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=\"msg\"}) by (name) / sum(ticdc_actor_poll_loop_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=~\"actor\"}) by (name)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Avg message batch size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average of actor batch size. ", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 24 + }, + "hiddenSeries": false, + "id": 434, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_actor_batch_size_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=\"proc\"}) by (name) / sum(ticdc_actor_poll_loop_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=\"system\"}) by (name)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Avg actor batch size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of actor poll per second.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 31 + }, + "hiddenSeries": false, + "id": 441, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_actor_poll_loop_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=\"actor\"}[1m])) by (name)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Actor poll/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of system poll loop per second.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 31 + }, + "hiddenSeries": false, + "id": 440, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_actor_poll_loop_total{tidb_cluster=\"$tidb_cluster\", name=~\"$actor_name\", type=\"system\"}[1m])) by (name)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{name}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "System poll/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "repeat": null, + "title": "Actor", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 155, + "panels": [ + { + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 6 + }, + "id": 157, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, + "seriesOverrides": [ + { + "alias": "alloc-from-os", + "fill": 3, + "lines": true, + "stack": false + }, + { + "alias": "gc-threshold", + "bars": false, + "color": "#C4162A", + "lines": true, + "linewidth": 2, + "stack": false + }, + { + "alias": "gc", + "bars": false, + "color": "#C4162A", + "hideTooltip": true, + "legend": false, + "pointradius": 3, + "points": true, + "stack": false + } + ], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "alloc-from-os", + "refId": "A" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-inuse", + "refId": "H" + }, + { + "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-garbage", + "refId": "C" + }, + { + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "reserved-by-go", + "refId": "B" + }, + { + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "used-by-go", + "refId": "D" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc-threshold", + "refId": "E" + }, + { + "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc", + "refId": "F" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory Usage", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Count of live objects.", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 6 + }, + "id": 158, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "objects", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated Live Objects", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process Go garbage collection STW pause duration", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 160, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", "percentage": false, "pluginVersion": "6.1.6", "pointradius": 5, @@ -15174,6 +16008,28 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "allValue": null, + "current": {}, + "datasource": "${DS_TEST-CLUSTER}", + "definition": "label_values(ticdc_actor_number_of_workers{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, name)", + "hide": 0, + "includeAll": true, + "label": "Actor", + "multi": true, + "name": "actor_name", + "options": [], + "query": "label_values(ticdc_actor_number_of_workers{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, name)", + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 0, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false } ] }, diff --git a/pkg/actor/metrics.go b/pkg/actor/metrics.go index ea5f4f535c1..c2b76e84797 100644 --- a/pkg/actor/metrics.go +++ b/pkg/actor/metrics.go @@ -14,9 +14,18 @@ package actor import ( + "time" + "github.com/prometheus/client_golang/prometheus" ) +const ( + slowPollThreshold = 100 * time.Millisecond + // Prometheus collects metrics every 15 seconds, we use a smaller interval + // to improve accuracy. + metricsInterval = 5 * time.Second +) + var ( totalWorkers = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -39,21 +48,27 @@ var ( Name: "worker_cpu_seconds_total", Help: "Total user and system CPU time spent by workers in seconds.", }, []string{"name", "id"}) - batchSizeHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ + batchSizeCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "actor", + Name: "batch_size_total", + Help: "Total number of batch size of an actor system.", + }, []string{"name", "type"}) + pollCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ Namespace: "ticdc", Subsystem: "actor", - Name: "batch", - Help: "Bucketed histogram of batch size of an actor system.", - Buckets: prometheus.ExponentialBuckets(1, 2, 10), + Name: "poll_loop_total", + Help: "Total number of poll loop count.", }, []string{"name", "type"}) - pollActorDuration = prometheus.NewHistogramVec( + slowPollActorDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "ticdc", Subsystem: "actor", - Name: "poll_duration_seconds", + Name: "slow_poll_duration_seconds", Help: "Bucketed histogram of actor poll time (s).", - Buckets: prometheus.ExponentialBuckets(0.01, 2, 16), + Buckets: prometheus.ExponentialBuckets(slowPollThreshold.Seconds(), 2, 16), }, []string{"name"}) dropMsgCount = prometheus.NewCounterVec( prometheus.CounterOpts{ @@ -69,7 +84,8 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(totalWorkers) registry.MustRegister(workingWorkers) registry.MustRegister(workingDuration) - registry.MustRegister(batchSizeHistogram) - registry.MustRegister(pollActorDuration) + registry.MustRegister(batchSizeCounter) + registry.MustRegister(pollCounter) + registry.MustRegister(slowPollActorDuration) registry.MustRegister(dropMsgCount) } diff --git a/pkg/actor/system.go b/pkg/actor/system.go index 51a19901aa7..aae25aa0b8c 100644 --- a/pkg/actor/system.go +++ b/pkg/actor/system.go @@ -34,9 +34,9 @@ import ( const ( // The max number of workers of a system. maxWorkerNum = 64 - // The default size of polled actor batch. + // defaultActorBatchSize is the default size of polled actor batch. defaultActorBatchSize = 1 - // The default size of receive message batch. + // defaultMsgBatchSizePerActor is the default size of receive message batch. defaultMsgBatchSizePerActor = 64 ) @@ -346,9 +346,11 @@ func (b *SystemBuilder) Build() (*System, *Router) { metricTotalWorkers: totalWorkers.WithLabelValues(b.name), metricWorkingWorkers: workingWorkers.WithLabelValues(b.name), metricWorkingDurations: metricWorkingDurations, - metricPollDuration: pollActorDuration.WithLabelValues(b.name), - metricProcBatch: batchSizeHistogram.WithLabelValues(b.name, "proc"), - metricMsgBatch: batchSizeHistogram.WithLabelValues(b.name, "msg"), + metricSystemPollLoop: pollCounter.WithLabelValues(b.name, "system"), + metricActorPollLoop: pollCounter.WithLabelValues(b.name, "actor"), + metricSlowPollDuration: slowPollActorDuration.WithLabelValues(b.name), + metricProcBatch: batchSizeCounter.WithLabelValues(b.name, "proc"), + metricMsgBatch: batchSizeCounter.WithLabelValues(b.name, "msg"), }, router } @@ -370,9 +372,11 @@ type System struct { metricTotalWorkers prometheus.Gauge metricWorkingWorkers prometheus.Gauge metricWorkingDurations []prometheus.Counter - metricPollDuration prometheus.Observer - metricProcBatch prometheus.Observer - metricMsgBatch prometheus.Observer + metricSystemPollLoop prometheus.Counter + metricActorPollLoop prometheus.Counter + metricSlowPollDuration prometheus.Observer + metricProcBatch prometheus.Counter + metricMsgBatch prometheus.Counter } // Start the system. Cancelling the context to stop the system. @@ -414,8 +418,6 @@ func (s *System) Spawn(mb Mailbox, actor Actor) error { return s.router.insert(id, p) } -const slowReceiveThreshold = time.Second - // The main poll of actor system. func (s *System) poll(ctx context.Context, id int) { batchPBuf := make([]*proc, s.actorBatchSize) @@ -423,9 +425,32 @@ func (s *System) poll(ctx context.Context, id int) { rd := s.rd rd.Lock() - startTime := time.Now() + // Approximate current time. It is updated when calling `now`. + var approximateCurrentTime time.Time + now := func() time.Time { + approximateCurrentTime = time.Now() + return approximateCurrentTime + } + // Start time of polling procs. + systemPollStartTime := now() + // The last time of recording metrics. + lastRecordMetricTime := systemPollStartTime + procBatchCnt, systemPollLoopCnt := 0, 0 + msgBatchCnt, actorPollLoopCnt := 0, 0 s.metricWorkingWorkers.Inc() for { + // Recording batch and loop metrics. + // We update metrics every `metricsInterval` to reduce overhead. + if approximateCurrentTime.Sub(lastRecordMetricTime) > metricsInterval { + lastRecordMetricTime = approximateCurrentTime + s.metricProcBatch.Add(float64(procBatchCnt)) + s.metricSystemPollLoop.Add(float64(systemPollLoopCnt)) + procBatchCnt, systemPollLoopCnt = 0, 0 + s.metricMsgBatch.Add(float64(msgBatchCnt)) + s.metricActorPollLoop.Add(float64(actorPollLoopCnt)) + msgBatchCnt, actorPollLoopCnt = 0, 0 + } + var batchP []*proc for { if rd.stopped { @@ -436,19 +461,23 @@ func (s *System) poll(ctx context.Context, id int) { n := rd.batchReceiveProcs(batchPBuf) if n != 0 { batchP = batchPBuf[:n] - s.metricProcBatch.Observe(float64(n)) + procBatchCnt += n + systemPollLoopCnt++ break } - // Recording metrics. - s.metricWorkingDurations[id].Add(time.Since(startTime).Seconds()) + // Recording working metrics. + systemPollDuration := now().Sub(systemPollStartTime) + s.metricWorkingDurations[id].Add(systemPollDuration.Seconds()) s.metricWorkingWorkers.Dec() // Park the poll until it is awakened. rd.cond.Wait() - startTime = time.Now() + systemPollStartTime = now() s.metricWorkingWorkers.Inc() } rd.Unlock() + actorPollLoopCnt += len(batchP) + actorPollStartTime := now() for _, p := range batchP { closed := p.isClosed() if closed { @@ -463,23 +492,26 @@ func (s *System) poll(ctx context.Context, id int) { continue } batchMsg := batchMsgBuf[:n] - s.metricMsgBatch.Observe(float64(n)) + msgBatchCnt += n // Poll actor. - pollStartTime := time.Now() running := p.actor.Poll(ctx, batchMsg) if !running { // Close the actor. p.close() } - receiveDuration := time.Since(pollStartTime) - if receiveDuration > slowReceiveThreshold { - log.Warn("actor handle received messages too slow", - zap.Duration("duration", receiveDuration), - zap.Uint64("id", uint64(p.mb.ID())), - zap.String("name", s.name)) + actorPollDuration := now().Sub(actorPollStartTime) + actorPollStartTime = approximateCurrentTime + if actorPollDuration > slowPollThreshold { + // Prometheus histogram is expensive, we only record slow poll. + s.metricSlowPollDuration.Observe(actorPollDuration.Seconds()) + if actorPollDuration > 10*slowPollThreshold { // 1s + log.Warn("actor poll received messages too slow", + zap.Duration("duration", actorPollDuration), + zap.Uint64("id", uint64(p.mb.ID())), + zap.String("name", s.name)) + } } - s.metricPollDuration.Observe(receiveDuration.Seconds()) } rd.Lock() From 2293dc2c1b9317cf6c2fb9c9da238d441afd43b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Tue, 15 Feb 2022 18:51:38 +0800 Subject: [PATCH 66/72] ticdc(sink): remove useless AppendResolvedEvent (#4596) ref pingcap/tiflow#4423 --- cdc/sink/codec/avro.go | 5 ----- cdc/sink/codec/canal.go | 6 ------ cdc/sink/codec/canal_flat.go | 36 +++++++------------------------ cdc/sink/codec/canal_flat_test.go | 26 ++++------------------ cdc/sink/codec/craft.go | 5 ----- cdc/sink/codec/interface.go | 4 ---- cdc/sink/codec/json.go | 5 ----- cdc/sink/codec/maxwell.go | 5 ----- cdc/sink/mq.go | 11 ++++------ 9 files changed, 16 insertions(+), 87 deletions(-) diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index 49e4b45090f..f9cab6aabc0 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -129,11 +129,6 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) return EncoderNeedAsyncWrite, nil } -// AppendResolvedEvent is no-op for Avro -func (a *AvroEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - return EncoderNoOperation, nil -} - // EncodeCheckpointEvent is no-op for now func (a *AvroEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { return nil, nil diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 20fdfa363ae..51e25f6d144 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -396,12 +396,6 @@ type CanalEventBatchEncoder struct { entryBuilder *canalEntryBuilder } -// AppendResolvedEvent appends a resolved event to the encoder -// TODO TXN support -func (d *CanalEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - return EncoderNoOperation, nil -} - // EncodeCheckpointEvent implements the EventBatchEncoder interface func (d *CanalEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { // For canal now, there is no such a corresponding type to ResolvedEvent so far. diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 98bcff8fa55..6feec68d120 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -35,9 +35,8 @@ const tidbWaterMarkType = "TIDB_WATERMARK" // CanalFlatEventBatchEncoder encodes Canal flat messages in JSON format type CanalFlatEventBatchEncoder struct { - builder *canalEntryBuilder - unresolvedBuf []canalFlatMessageInterface - resolvedBuf []canalFlatMessageInterface + builder *canalEntryBuilder + messageBuf []canalFlatMessageInterface // When it is true, canal-json would generate TiDB extension information // which, at the moment, only includes `tidbWaterMarkType` and `_tidb` fields. enableTiDBExtension bool @@ -47,8 +46,7 @@ type CanalFlatEventBatchEncoder struct { func NewCanalFlatEventBatchEncoder() EventBatchEncoder { return &CanalFlatEventBatchEncoder{ builder: NewCanalEntryBuilder(), - unresolvedBuf: make([]canalFlatMessageInterface, 0), - resolvedBuf: make([]canalFlatMessageInterface, 0), + messageBuf: make([]canalFlatMessageInterface, 0), enableTiDBExtension: false, } } @@ -322,25 +320,7 @@ func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEv if err != nil { return EncoderNoOperation, errors.Trace(err) } - c.unresolvedBuf = append(c.unresolvedBuf, message) - return EncoderNoOperation, nil -} - -// AppendResolvedEvent receives the latest resolvedTs -func (c *CanalFlatEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - nextIdx := 0 - for _, msg := range c.unresolvedBuf { - if msg.getTikvTs() <= ts { - c.resolvedBuf = append(c.resolvedBuf, msg) - } else { - break - } - nextIdx++ - } - c.unresolvedBuf = c.unresolvedBuf[nextIdx:] - if len(c.resolvedBuf) > 0 { - return EncoderNeedAsyncWrite, nil - } + c.messageBuf = append(c.messageBuf, message) return EncoderNoOperation, nil } @@ -356,11 +336,11 @@ func (c *CanalFlatEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessa // Build implements the EventBatchEncoder interface func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { - if len(c.resolvedBuf) == 0 { + if len(c.messageBuf) == 0 { return nil } - ret := make([]*MQMessage, len(c.resolvedBuf)) - for i, msg := range c.resolvedBuf { + ret := make([]*MQMessage, len(c.messageBuf)) + for i, msg := range c.messageBuf { value, err := json.Marshal(msg) if err != nil { log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) @@ -370,7 +350,7 @@ func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { m.IncRowsCount() ret[i] = m } - c.resolvedBuf = c.resolvedBuf[0:0] + c.messageBuf = make([]canalFlatMessageInterface, 0) return ret } diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index fb968f2b2ac..567560a435e 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -163,10 +163,6 @@ func (s *canalFlatSuite) TestNewCanalFlatEventBatchDecoder4RowMessage(c *check.C c.Assert(err, check.IsNil) c.Assert(result, check.Equals, EncoderNoOperation) - result, err = encoder.AppendResolvedEvent(417318403368288260) - c.Assert(err, check.IsNil) - c.Assert(result, check.Equals, EncoderNeedAsyncWrite) - mqMessages := encoder.Build() c.Assert(len(mqMessages), check.Equals, 1) @@ -295,27 +291,17 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { c.Assert(encoder, check.NotNil) updateCase := *testCaseUpdate - lastResolved := uint64(0) - for i := 1; i < 1000; i++ { + for i := 1; i <= 1000; i++ { ts := uint64(i) updateCase.CommitTs = ts result, err := encoder.AppendRowChangedEvent(&updateCase) c.Assert(err, check.IsNil) c.Assert(result, check.Equals, EncoderNoOperation) - if i >= 100 && (i%100 == 0 || i == 999) { - resolvedTs := uint64(i - 50) - if i == 999 { - resolvedTs = 999 - } - result, err := encoder.AppendResolvedEvent(resolvedTs) - - c.Assert(err, check.IsNil) - c.Assert(result, check.Equals, EncoderNeedAsyncWrite) - + if i%100 == 0 { msgs := encoder.Build() c.Assert(msgs, check.NotNil) - c.Assert(msgs, check.HasLen, int(resolvedTs-lastResolved)) + c.Assert(msgs, check.HasLen, 100) for j := range msgs { c.Assert(msgs[j].GetRowsCount(), check.Equals, 1) @@ -324,15 +310,11 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { err := json.Unmarshal(msgs[j].Value, &msg) c.Assert(err, check.IsNil) c.Assert(msg.EventType, check.Equals, "UPDATE") - c.Assert(msg.ExecutionTime, check.Equals, convertToCanalTs(lastResolved+uint64(i))) } - - lastResolved = resolvedTs } } - c.Assert(encoder.unresolvedBuf, check.HasLen, 0) - c.Assert(encoder.resolvedBuf, check.HasLen, 0) + c.Assert(encoder.messageBuf, check.HasLen, 0) } func (s *canalFlatSuite) TestEncodeCheckpointEvent(c *check.C) { diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index b3f35187a29..b390595b761 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -62,11 +62,6 @@ func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent return EncoderNoOperation, nil } -// AppendResolvedEvent is no-op -func (e *CraftEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - return EncoderNoOperation, nil -} - // EncodeDDLEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) EncodeDDLEvent(ev *model.DDLEvent) (*MQMessage, error) { return newDDLMQMessage(config.ProtocolCraft, nil, craft.NewDDLEventEncoder(e.allocator, ev).Encode(), ev), nil diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 0dafac1d16b..847e447c8df 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -33,9 +33,6 @@ type EventBatchEncoder interface { EncodeCheckpointEvent(ts uint64) (*MQMessage, error) // AppendRowChangedEvent appends a row changed event into the batch AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) - // AppendResolvedEvent appends a resolved event into the batch. - // This event is used to tell the encoder that no event prior to ts will be sent. - AppendResolvedEvent(ts uint64) (EncoderResult, error) // EncodeDDLEvent appends a DDL event into the batch EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) // Build builds the batch and returns the bytes of key and value. @@ -47,7 +44,6 @@ type EventBatchEncoder interface { // TODO decouple it out MixedBuild(withVersion bool) []byte // Size returns the size of the batch(bytes) - // Deprecated: Size is deprecated Size() int // Reset reset the kv buffer Reset() diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 979dacbfa37..b96da852c92 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -400,11 +400,6 @@ func (d *JSONEventBatchEncoder) SetMixedBuildSupport(enabled bool) { d.supportMixedBuild = enabled } -// AppendResolvedEvent is no-op -func (d *JSONEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - return EncoderNoOperation, nil -} - // EncodeCheckpointEvent implements the EventBatchEncoder interface func (d *JSONEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { keyMsg := newResolvedMessage(ts) diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 42f54aafc10..03ab4eda75e 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -85,11 +85,6 @@ func (d *MaxwellEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, return nil, nil } -// AppendResolvedEvent implements the EventBatchEncoder interface -func (d *MaxwellEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { - return EncoderNoOperation, nil -} - func rowEventToMaxwellMessage(e *model.RowChangedEvent) (*mqMessageKey, *maxwellMessage) { var partition *int64 if e.Table.IsPartition { diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index cf69bdf70e4..2071097ad1a 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -399,15 +399,12 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { continue case e = <-input: } - // flush resolvedTs event if e.row == nil { + // When receiving resolved ts events, we need to write all events to the producer. + // We don't need to flush it immediately, we wait until all partitions have received + // this event before we flush it uniformly. if e.resolvedTs != 0 { - op, err := encoder.AppendResolvedEvent(e.resolvedTs) - if err != nil { - return errors.Trace(err) - } - - if err := flushToProducer(op); err != nil { + if err := flushToProducer(codec.EncoderNoOperation); err != nil { return errors.Trace(err) } From 536d8ae9a5f0032991f2addcb46491770d4f45f2 Mon Sep 17 00:00:00 2001 From: Obliviate <756541536@qq.com> Date: Wed, 16 Feb 2022 11:59:39 +0800 Subject: [PATCH 67/72] dm: new router compatible with regular expression (#4358) close pingcap/tiflow#4256 --- dm/_utils/terror_gen/errors_release.txt | 1 + dm/checker/checker.go | 4 +- dm/dm/config/subtask.go | 4 +- dm/errors.toml | 6 + dm/loader/convert_data.go | 4 +- dm/loader/convert_data_test.go | 10 +- dm/loader/loader.go | 12 +- dm/pkg/router/router.go | 247 +++++++++++ dm/pkg/router/router_test.go | 387 ++++++++++++++++++ dm/pkg/terror/error_list.go | 3 +- dm/pkg/utils/common.go | 4 +- dm/pkg/utils/common_test.go | 6 +- dm/syncer/ddl_test.go | 4 +- dm/syncer/syncer.go | 12 +- dm/tests/all_mode/conf/regexpr-task.yaml | 47 +++ .../all_mode/conf/regexpr_diff_config.toml | 49 +++ dm/tests/all_mode/data/db1.regexpr.sql | 13 + dm/tests/all_mode/data/db2.regexpr.sql | 14 + dm/tests/all_mode/run.sh | 56 +++ 19 files changed, 852 insertions(+), 31 deletions(-) create mode 100644 dm/pkg/router/router.go create mode 100644 dm/pkg/router/router_test.go create mode 100644 dm/tests/all_mode/conf/regexpr-task.yaml create mode 100644 dm/tests/all_mode/conf/regexpr_diff_config.toml create mode 100644 dm/tests/all_mode/data/db1.regexpr.sql create mode 100644 dm/tests/all_mode/data/db2.regexpr.sql diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 1a772597b12..383b6e2f268 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -479,6 +479,7 @@ ErrWorkerDDLLockOpNotFound,[code=40075:class=dm-worker:scope=internal:level=high ErrWorkerTLSConfigNotValid,[code=40076:class=dm-worker:scope=internal:level=high], "Message: TLS config not valid, Workaround: Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in worker configuration file." ErrWorkerFailConnectMaster,[code=40077:class=dm-worker:scope=internal:level=high], "Message: cannot join with master endpoints: %v, error: %v, Workaround: Please check network connection of worker and check worker name is unique." ErrWorkerRelayConfigChanging,[code=40079:class=dm-worker:scope=internal:level=low], "Message: relay config of worker %s is changed too frequently, last relay source %s:, new relay source %s, Workaround: Please try again later" +ErrWorkerRouteTableDupMatch,[code=40080:class=dm-worker:scope=internal:level=high], "Message: table %s.%s matches more than one rule, Workaround: please check the route rules in the task config" ErrTracerParseFlagSet,[code=42001:class=dm-tracer:scope=internal:level=medium], "Message: parse dm-tracer config flag set" ErrTracerConfigTomlTransform,[code=42002:class=dm-tracer:scope=internal:level=medium], "Message: config toml transform, Workaround: Please check the configuration file has correct TOML format." ErrTracerConfigInvalidFlag,[code=42003:class=dm-tracer:scope=internal:level=medium], "Message: '%s' is an invalid flag" diff --git a/dm/checker/checker.go b/dm/checker/checker.go index dc688ca89ba..18315083ff8 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/dumpling" fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" @@ -43,7 +44,6 @@ import ( column "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/dumpling/export" "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" @@ -137,7 +137,7 @@ func (c *Checker) Init(ctx context.Context) (err error) { if err != nil { return terror.ErrTaskCheckGenBAList.Delegate(err) } - r, err := router.NewTableRouter(instance.cfg.CaseSensitive, instance.cfg.RouteRules) + r, err := router.NewRouter(instance.cfg.CaseSensitive, instance.cfg.RouteRules) if err != nil { return terror.ErrTaskCheckGenTableRouter.Delegate(err) } diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index 3bfc2813a5d..b4a3d94d9a8 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -25,11 +25,11 @@ import ( bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/dumpling" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" ) @@ -426,7 +426,7 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { if _, err := filter.New(c.CaseSensitive, c.BAList); err != nil { return terror.ErrConfigGenBAList.Delegate(err) } - if _, err := router.NewTableRouter(c.CaseSensitive, c.RouteRules); err != nil { + if _, err := router.NewRouter(c.CaseSensitive, c.RouteRules); err != nil { return terror.ErrConfigGenTableRouter.Delegate(err) } // NewMapping will fill arguments with the default values. diff --git a/dm/errors.toml b/dm/errors.toml index c7cee3f0b32..252d383e171 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2884,6 +2884,12 @@ description = "" workaround = "Please try again later" tags = ["internal", "low"] +[error.DM-dm-worker-40080] +message = "table %s.%s matches more than one rule" +description = "" +workaround = "please check the route rules in the task config" +tags = ["internal", "high"] + [error.DM-dm-tracer-42001] message = "parse dm-tracer config flag set" description = "" diff --git a/dm/loader/convert_data.go b/dm/loader/convert_data.go index 72e6bdd598b..65bb0a002a9 100644 --- a/dm/loader/convert_data.go +++ b/dm/loader/convert_data.go @@ -24,12 +24,12 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/errors" cm "github.com/pingcap/tidb-tools/pkg/column-mapping" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/parser/ast" ) @@ -234,7 +234,7 @@ func tableName(schema, table string) string { return fmt.Sprintf("`%s`.`%s`", schema, table) } -func parseTable(ctx *tcontext.Context, r *router.Table, schema, table, file, sqlMode, sourceID string) (*tableInfo, error) { +func parseTable(ctx *tcontext.Context, r *router.RouteTable, schema, table, file, sqlMode, sourceID string) (*tableInfo, error) { statement, err := exportStatement(file) if err != nil { return nil, err diff --git a/dm/loader/convert_data_test.go b/dm/loader/convert_data_test.go index de29efa45ed..6a6977184a6 100644 --- a/dm/loader/convert_data_test.go +++ b/dm/loader/convert_data_test.go @@ -15,9 +15,9 @@ package loader import ( cm "github.com/pingcap/tidb-tools/pkg/column-mapping" - router "github.com/pingcap/tidb-tools/pkg/table-router" tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/router" . "github.com/pingcap/check" ) @@ -165,7 +165,7 @@ func (t *testConvertDataSuite) TestParseTable(c *C) { insertHeadStmt: "INSERT INTO `t` VALUES", } - r, err := router.NewTableRouter(false, rules) + r, err := router.NewRouter(false, rules) c.Assert(err, IsNil) tableInfo, err := parseTable(tcontext.Background(), r, "test1", "t2", "./dumpfile/test1.t2-schema.sql", "ANSI_QUOTES", "source-mysql-01") @@ -193,7 +193,7 @@ func (t *testConvertDataSuite) TestParseTableWithGeneratedColumn(c *C) { insertHeadStmt: "INSERT INTO `t` (`id`,`t_json`) VALUES", } - r, err := router.NewTableRouter(false, rules) + r, err := router.NewRouter(false, rules) c.Assert(err, IsNil) tableInfo, err := parseTable(tcontext.Background(), r, "test1", "t3", "./dumpfile/test1.t3-schema.sql", "", "source-mysql-01") @@ -411,7 +411,7 @@ func (t *testConvertDataSuite) TestParseTableWithExtendColumn(c *C) { extendVal: []string{"t2", "test1", "source1"}, } - r, err := router.NewTableRouter(false, rules) + r, err := router.NewRouter(false, rules) c.Assert(err, IsNil) tableInfo, err := parseTable(tcontext.Background(), r, "test1", "t2", "./dumpfile/test1.t2-schema.sql", "ANSI_QUOTES", "source1") @@ -456,7 +456,7 @@ func (t *testConvertDataSuite) TestParseTableWithGeneratedColumnExtendColumn(c * extendVal: []string{"t3", "test1", "source1"}, } - r, err := router.NewTableRouter(false, rules) + r, err := router.NewRouter(false, rules) c.Assert(err, IsNil) tableInfo, err := parseTable(tcontext.Background(), r, "test1", "t3", "./dumpfile/test1.t3-schema.sql", "", "source1") diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 728e3352862..3ecc6492ce8 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -35,6 +35,7 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -42,7 +43,6 @@ import ( "github.com/pingcap/failpoint" cm "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "go.uber.org/atomic" "go.uber.org/zap" ) @@ -430,7 +430,7 @@ type Loader struct { fileJobQueue chan *fileJob - tableRouter *router.Table + tableRouter *router.RouteTable baList *filter.Filter columnMapping *cm.Mapping @@ -876,7 +876,7 @@ func (l *Loader) Update(ctx context.Context, cfg *config.SubTaskConfig) error { var ( err error oldBaList *filter.Filter - oldTableRouter *router.Table + oldTableRouter *router.RouteTable oldColumnMapping *cm.Mapping ) @@ -904,7 +904,7 @@ func (l *Loader) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // update route, for loader, this almost useless, because schemas often have been restored oldTableRouter = l.tableRouter - l.tableRouter, err = router.NewTableRouter(cfg.CaseSensitive, cfg.RouteRules) + l.tableRouter, err = router.NewRouter(cfg.CaseSensitive, cfg.RouteRules) if err != nil { return terror.ErrLoadUnitGenTableRouter.Delegate(err) } @@ -924,7 +924,7 @@ func (l *Loader) Update(ctx context.Context, cfg *config.SubTaskConfig) error { } func (l *Loader) genRouter(rules []*router.TableRule) error { - l.tableRouter, _ = router.NewTableRouter(l.cfg.CaseSensitive, []*router.TableRule{}) + l.tableRouter, _ = router.NewRouter(l.cfg.CaseSensitive, []*router.TableRule{}) for _, rule := range rules { err := l.tableRouter.AddRule(rule) if err != nil { @@ -1233,7 +1233,7 @@ func renameShardingSchema(query, srcSchema, dstSchema string, ansiquote bool) st return SQLReplace(query, srcSchema, dstSchema, ansiquote) } -func fetchMatchedLiteral(ctx *tcontext.Context, router *router.Table, schema, table string) (targetSchema string, targetTable string) { +func fetchMatchedLiteral(ctx *tcontext.Context, router *router.RouteTable, schema, table string) (targetSchema string, targetTable string) { if schema == "" { // nothing change return schema, table diff --git a/dm/pkg/router/router.go b/dm/pkg/router/router.go new file mode 100644 index 00000000000..10ea7f697b0 --- /dev/null +++ b/dm/pkg/router/router.go @@ -0,0 +1,247 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package router + +import ( + "regexp" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/filter" + tablerouter "github.com/pingcap/tidb-tools/pkg/table-router" + + "github.com/pingcap/tiflow/dm/pkg/terror" +) + +type ( + TableRule = tablerouter.TableRule + Table = filter.Table + FilterRule = filter.Rules + TableExtractor = tablerouter.TableExtractor + SchemaExtractor = tablerouter.SchemaExtractor + SourceExtractor = tablerouter.SourceExtractor +) + +type FilterType = int32 + +const ( + TblFilter FilterType = iota + 1 + SchmFilter +) + +type filterWrapper struct { + filter *filter.Filter + typ FilterType + target Table + + rawRule *TableRule +} + +type RouteTable struct { + filters []*filterWrapper + caseSensitive bool +} + +func NewRouter(caseSensitive bool, rules []*TableRule) (*RouteTable, error) { + r := &RouteTable{ + filters: make([]*filterWrapper, 0), + caseSensitive: caseSensitive, + } + for _, rule := range rules { + if err := r.AddRule(rule); err != nil { + return nil, err + } + } + return r, nil +} + +func (r *RouteTable) AddRule(rule *TableRule) error { + err := rule.Valid() + if err != nil { + return errors.Trace(err) + } + if !r.caseSensitive { + rule.ToLower() + } + newFilter := &filterWrapper{ + rawRule: rule, + } + newFilter.target = Table{ + Schema: rule.TargetSchema, + Name: rule.TargetTable, + } + if len(rule.TablePattern) == 0 { + // raw schema rule + newFilter.typ = SchmFilter + rawFilter, err := filter.New(r.caseSensitive, &FilterRule{ + DoDBs: []string{rule.SchemaPattern}, + }) + if err != nil { + return errors.Annotatef(err, "add rule %+v into table router", rule) + } + newFilter.filter = rawFilter + } else { + newFilter.typ = TblFilter + rawFilter, err := filter.New(r.caseSensitive, &FilterRule{ + DoTables: []*Table{ + { + Schema: rule.SchemaPattern, + Name: rule.TablePattern, + }, + }, + DoDBs: []string{rule.SchemaPattern}, + }) + if err != nil { + return errors.Annotatef(err, "add rule %+v into table router", rule) + } + newFilter.filter = rawFilter + } + r.filters = append(r.filters, newFilter) + return nil +} + +func (r *RouteTable) Route(schema, table string) (string, string, error) { + curTable := &Table{ + Schema: schema, + Name: table, + } + tblRules := make([]*filterWrapper, 0) + schmRules := make([]*filterWrapper, 0) + for _, filterWrapper := range r.filters { + if filterWrapper.filter.Match(curTable) { + if filterWrapper.typ == TblFilter { + tblRules = append(tblRules, filterWrapper) + } else { + schmRules = append(schmRules, filterWrapper) + } + } + } + var ( + targetSchema string + targetTable string + ) + if table == "" || len(tblRules) == 0 { + // 1. no need to match table or + // 2. match no table + if len(schmRules) > 1 { + return "", "", terror.ErrWorkerRouteTableDupMatch.Generate(schema, table) + } + if len(schmRules) == 1 { + targetSchema, targetTable = schmRules[0].target.Schema, schmRules[0].target.Name + } + } else { + if len(tblRules) > 1 { + return "", "", terror.ErrWorkerRouteTableDupMatch.Generate(schema, table) + } + targetSchema, targetTable = tblRules[0].target.Schema, tblRules[0].target.Name + } + if len(targetSchema) == 0 { + targetSchema = schema + } + if len(targetTable) == 0 { + targetTable = table + } + return targetSchema, targetTable, nil +} + +func (r *RouteTable) AllRules() ([]TableRule, []TableRule) { + var ( + schmRouteRules []TableRule + tableRouteRules []TableRule + ) + for _, filter := range r.filters { + if filter.typ == SchmFilter { + schmRouteRules = append(schmRouteRules, *filter.rawRule) + } else { + tableRouteRules = append(tableRouteRules, *filter.rawRule) + } + } + return schmRouteRules, tableRouteRules +} + +func (r *RouteTable) FetchExtendColumn(schema, table, source string) ([]string, []string) { + var cols []string + var vals []string + rules := []*filterWrapper{} + curTable := &Table{ + Schema: schema, + Name: table, + } + for _, filter := range r.filters { + if filter.filter.Match(curTable) { + rules = append(rules, filter) + } + } + var ( + schemaRules = make([]*TableRule, 0, len(rules)) + tableRules = make([]*TableRule, 0, len(rules)) + ) + for i := range rules { + rule := rules[i].rawRule + if rule.TablePattern == "" { + schemaRules = append(schemaRules, rule) + } else { + tableRules = append(tableRules, rule) + } + } + if len(tableRules) == 0 && len(schemaRules) == 0 { + return cols, vals + } + var rule *TableRule + if len(tableRules) == 0 { + rule = schemaRules[0] + } else { + rule = tableRules[0] + } + if rule.TableExtractor != nil { + cols = append(cols, rule.TableExtractor.TargetColumn) + vals = append(vals, extractVal(table, rule.TableExtractor)) + } + + if rule.SchemaExtractor != nil { + cols = append(cols, rule.SchemaExtractor.TargetColumn) + vals = append(vals, extractVal(schema, rule.SchemaExtractor)) + } + + if rule.SourceExtractor != nil { + cols = append(cols, rule.SourceExtractor.TargetColumn) + vals = append(vals, extractVal(source, rule.SourceExtractor)) + } + return cols, vals +} + +func extractVal(s string, ext interface{}) string { + var params []string + switch e := ext.(type) { + case *tablerouter.TableExtractor: + if regExpr, err := regexp.Compile(e.TableRegexp); err == nil { + params = regExpr.FindStringSubmatch(s) + } + case *tablerouter.SchemaExtractor: + if regExpr, err := regexp.Compile(e.SchemaRegexp); err == nil { + params = regExpr.FindStringSubmatch(s) + } + case *tablerouter.SourceExtractor: + if regExpr, err := regexp.Compile(e.SourceRegexp); err == nil { + params = regExpr.FindStringSubmatch(s) + } + } + var val strings.Builder + for idx, param := range params { + if idx > 0 { + val.WriteString(param) + } + } + return val.String() +} diff --git a/dm/pkg/router/router_test.go b/dm/pkg/router/router_test.go new file mode 100644 index 00000000000..f0b3ce9d4de --- /dev/null +++ b/dm/pkg/router/router_test.go @@ -0,0 +1,387 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package router + +import ( + "fmt" + "testing" + + . "github.com/pingcap/check" + "github.com/pingcap/tiflow/dm/pkg/terror" + + oldrouter "github.com/pingcap/tidb-tools/pkg/table-router" +) + +var _ = Suite(&testRouteSuite{}) + +type testRouteSuite struct{} + +func TestRoute(t *testing.T) { + TestingT(t) +} + +func (s *testRouteSuite) TestCreateRouter(c *C) { + _, err := NewRouter(true, []*TableRule{}) + c.Assert(err, Equals, nil) + _, err = NewRouter(false, []*TableRule{}) + c.Assert(err, Equals, nil) +} + +func (s *testRouteSuite) TestAddRule(c *C) { + r, err := NewRouter(true, []*TableRule{}) + c.Assert(err, Equals, nil) + rules := []*TableRule{ + { + SchemaPattern: "test1", + TargetSchema: "dtest1", + }, + { + SchemaPattern: "test2", + TablePattern: "table2", + TargetSchema: "dtest2", + TargetTable: "dtable2", + }, + } + for _, rule := range rules { + err = r.AddRule(rule) + c.Assert(err, Equals, nil) // successfully insert + } + r, err = NewRouter(false, []*TableRule{}) + c.Assert(err, Equals, nil) + for _, rule := range rules { + err := r.AddRule(rule) + c.Assert(err, Equals, nil) // successfully insert + } +} + +func (s *testRouteSuite) TestSchemaRoute(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "test1", + TargetSchema: "dtest1", + }, + { + SchemaPattern: "gtest*", + TargetSchema: "dtest", + }, + } + oldRouter, err := oldrouter.NewTableRouter(true, rules) + c.Assert(err, Equals, nil) + newRouter, err := NewRouter(true, rules) + c.Assert(err, Equals, nil) + inputTables := []Table{ + { + Schema: "test1", // match rule 1 + Name: "table1", + }, + { + Schema: "gtesttest", // match rule 2 + Name: "atable", + }, + { + Schema: "ptest", // match neither + Name: "atableg", + }, + } + expectedResult := []Table{ + { + Schema: "dtest1", + Name: "table1", + }, + { + Schema: "dtest", + Name: "atable", + }, + { + Schema: "ptest", + Name: "atableg", + }, + } + for idx := range inputTables { + schema, table := inputTables[idx].Schema, inputTables[idx].Name + expSchema, expTable := expectedResult[idx].Schema, expectedResult[idx].Name + oldSchema, oldTable, err := oldRouter.Route(schema, table) + c.Assert(err, Equals, nil) + newSchema, newTable, err := newRouter.Route(schema, table) + c.Assert(err, Equals, nil) + c.Assert(oldSchema, Equals, expSchema) + c.Assert(oldTable, Equals, expTable) + c.Assert(newSchema, Equals, expSchema) + c.Assert(newTable, Equals, expTable) + } +} + +func (s *testRouteSuite) TestTableRoute(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "test1", + TablePattern: "table1", + TargetSchema: "dtest1", + TargetTable: "dtable1", + }, + { + SchemaPattern: "test*", + TablePattern: "table2", + TargetSchema: "dtest2", + TargetTable: "dtable2", + }, + { + SchemaPattern: "test3", + TablePattern: "table*", + TargetSchema: "dtest3", + TargetTable: "dtable3", + }, + } + inputTables := []*Table{} + expTables := []*Table{} + for i := 1; i <= 3; i++ { + inputTables = append(inputTables, &Table{ + Schema: fmt.Sprintf("test%d", i), + Name: fmt.Sprintf("table%d", i), + }) + expTables = append(expTables, &Table{ + Schema: fmt.Sprintf("dtest%d", i), + Name: fmt.Sprintf("dtable%d", i), + }) + } + oldRouter, err := oldrouter.NewTableRouter(true, rules) + c.Assert(err, Equals, nil) + newRouter, err := NewRouter(true, rules) + c.Assert(err, Equals, nil) + for i := range inputTables { + schema, table := inputTables[i].Schema, inputTables[i].Name + expSchema, expTable := expTables[i].Schema, expTables[i].Name + oldSch, oldTbl, _ := oldRouter.Route(schema, table) + newSch, newTbl, _ := newRouter.Route(schema, table) + c.Assert(newSch, Equals, expSchema) + c.Assert(newTbl, Equals, expTable) + c.Assert(oldSch, Equals, expSchema) + c.Assert(oldTbl, Equals, expTable) + } +} + +func (s *testRouteSuite) TestRegExprRoute(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "~test.[0-9]+", + TargetSchema: "dtest1", + }, + { + SchemaPattern: "~test2?[animal|human]", + TablePattern: "~tbl.*[cat|dog]+", + TargetSchema: "dtest2", + TargetTable: "dtable2", + }, + { + SchemaPattern: "~test3_(schema)?.*", + TablePattern: "test3_*", + TargetSchema: "dtest3", + TargetTable: "dtable3", + }, + { + SchemaPattern: "test4s_*", + TablePattern: "~testtable_[donot_delete]?", + TargetSchema: "dtest4", + TargetTable: "dtable4", + }, + } + inputTable := []Table{ + { + Schema: "tests100", + Name: "table1", // match rule 1 + }, + { + Schema: "test2animal", + Name: "tbl_animal_dogcat", // match rule 2 + }, + { + Schema: "test3_schema_meta", + Name: "test3_tail", // match rule 3 + }, + { + Schema: "test4s_2022", + Name: "testtable_donot_delete", // match rule 4 + }, + { + Schema: "mytst5566", + Name: "gtable", // match nothing + }, + } + expectedOutput := []Table{ + { + Schema: "dtest1", + Name: "table1", + }, + { + Schema: "dtest2", + Name: "dtable2", + }, + { + Schema: "dtest3", + Name: "dtable3", + }, + { + Schema: "dtest4", + Name: "dtable4", + }, + { + Schema: "mytst5566", + Name: "gtable", + }, + } + newRouter, err := NewRouter(true, rules) + c.Assert(err, Equals, nil) + for idx := range inputTable { + s, n := inputTable[idx].Schema, inputTable[idx].Name + expSchm, expName := expectedOutput[idx].Schema, expectedOutput[idx].Name + newSchm, newName, err := newRouter.Route(s, n) + c.Assert(err, Equals, nil) + c.Assert(newSchm, Equals, expSchm) + c.Assert(newName, Equals, expName) + } +} + +func (s *testRouteSuite) TestFetchExtendColumn(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "schema*", + TablePattern: "t*", + TargetSchema: "test", + TargetTable: "t", + TableExtractor: &TableExtractor{ + TargetColumn: "table_name", + TableRegexp: "table_(.*)", + }, + SchemaExtractor: &SchemaExtractor{ + TargetColumn: "schema_name", + SchemaRegexp: "schema_(.*)", + }, + SourceExtractor: &SourceExtractor{ + TargetColumn: "source_name", + SourceRegexp: "source_(.*)_(.*)", + }, + }, + { + SchemaPattern: "~s?chema.*", + TargetSchema: "test", + TargetTable: "t2", + SchemaExtractor: &SchemaExtractor{ + TargetColumn: "schema_name", + SchemaRegexp: "(.*)", + }, + SourceExtractor: &SourceExtractor{ + TargetColumn: "source_name", + SourceRegexp: "(.*)", + }, + }, + } + r, err := NewRouter(false, rules) + c.Assert(err, IsNil) + expected := [][]string{ + {"table_name", "schema_name", "source_name"}, + {"t1", "s1", "s1s1"}, + + {"schema_name", "source_name"}, + {"schema_s2", "source_s2"}, + } + + // table level rules have highest priority + extendCol, extendVal := r.FetchExtendColumn("schema_s1", "table_t1", "source_s1_s1") + c.Assert(expected[0], DeepEquals, extendCol) + c.Assert(expected[1], DeepEquals, extendVal) + + // only schema rules + extendCol2, extendVal2 := r.FetchExtendColumn("schema_s2", "a_table_t2", "source_s2") + c.Assert(expected[2], DeepEquals, extendCol2) + c.Assert(expected[3], DeepEquals, extendVal2) +} + +func (s *testRouteSuite) TestAllRule(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "~test.[0-9]+", + TargetSchema: "dtest1", + }, + { + SchemaPattern: "~test2?[animal|human]", + TablePattern: "~tbl.*[cat|dog]+", + TargetSchema: "dtest2", + TargetTable: "dtable2", + }, + { + SchemaPattern: "~test3_(schema)?.*", + TablePattern: "test3_*", + TargetSchema: "dtest3", + TargetTable: "dtable3", + }, + { + SchemaPattern: "test4s_*", + TablePattern: "~testtable_[donot_delete]?", + TargetSchema: "dtest4", + TargetTable: "dtable4", + }, + } + r, err := NewRouter(true, rules) + c.Assert(err, Equals, nil) + schemaRules, tableRules := r.AllRules() + c.Assert(len(schemaRules), Equals, 1) + c.Assert(len(tableRules), Equals, 3) + c.Assert(schemaRules[0].SchemaPattern, Equals, rules[0].SchemaPattern) + for i := 0; i < 3; i++ { + c.Assert(tableRules[i].SchemaPattern, Equals, rules[i+1].SchemaPattern) + c.Assert(tableRules[i].TablePattern, Equals, rules[i+1].TablePattern) + } +} + +func (s *testRouteSuite) TestDupMatch(c *C) { + rules := []*TableRule{ + { + SchemaPattern: "~test[0-9]+.*", + TablePattern: "~.*", + TargetSchema: "dtest1", + }, + { + SchemaPattern: "~test2?[a|b]", + TablePattern: "~tbl2", + TargetSchema: "dtest2", + TargetTable: "dtable2", + }, + { + SchemaPattern: "mytest*", + TargetSchema: "mytest", + }, + { + SchemaPattern: "~mytest(_meta)?_schema", + TargetSchema: "test", + }, + } + inputTables := []Table{ + { + Schema: "test2a", // match rule1 and rule2 + Name: "tbl2", + }, + { + Schema: "mytest_meta_schema", // match rule3 and rule4 + Name: "", + }, + } + r, err := NewRouter(true, rules) + c.Assert(err, Equals, nil) + for i := range inputTables { + targetSchm, targetTbl, err := r.Route(inputTables[i].Schema, inputTables[i].Name) + c.Assert(targetSchm, Equals, "") + c.Assert(targetTbl, Equals, "") + c.Assert(terror.ErrWorkerRouteTableDupMatch.Equal(err), Equals, true) + } +} diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 7e8cdf03058..e86fb47ba00 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -584,6 +584,7 @@ const ( codeWorkerFailConnectMaster codeWorkerWaitRelayCatchupGTID codeWorkerRelayConfigChanging + codeWorkerRouteTableDupMatch ) // DM-tracer error code. @@ -1235,7 +1236,7 @@ var ( ErrWorkerTLSConfigNotValid = New(codeWorkerTLSConfigNotValid, ClassDMWorker, ScopeInternal, LevelHigh, "TLS config not valid", "Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in worker configuration file.") ErrWorkerFailConnectMaster = New(codeWorkerFailConnectMaster, ClassDMWorker, ScopeInternal, LevelHigh, "cannot join with master endpoints: %v, error: %v", "Please check network connection of worker and check worker name is unique.") ErrWorkerRelayConfigChanging = New(codeWorkerRelayConfigChanging, ClassDMWorker, ScopeInternal, LevelLow, "relay config of worker %s is changed too frequently, last relay source %s:, new relay source %s", "Please try again later") - + ErrWorkerRouteTableDupMatch = New(codeWorkerRouteTableDupMatch, ClassDMWorker, ScopeInternal, LevelHigh, "table %s.%s matches more than one rule", "please check the route rules in the task config") // DM-tracer error. ErrTracerParseFlagSet = New(codeTracerParseFlagSet, ClassDMTracer, ScopeInternal, LevelMedium, "parse dm-tracer config flag set", "") ErrTracerConfigTomlTransform = New(codeTracerConfigTomlTransform, ClassDMTracer, ScopeInternal, LevelMedium, "config toml transform", "Please check the configuration file has correct TOML format.") diff --git a/dm/pkg/utils/common.go b/dm/pkg/utils/common.go index 25c4ecb348a..e873374a0ed 100644 --- a/dm/pkg/utils/common.go +++ b/dm/pkg/utils/common.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/parser/model" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" @@ -34,6 +33,7 @@ import ( "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -123,7 +123,7 @@ func FetchAllDoTables(ctx context.Context, db *sql.DB, bw *filter.Filter) (map[s } // FetchTargetDoTables returns all need to do tables after filtered and routed (fetches from upstream MySQL). -func FetchTargetDoTables(ctx context.Context, db *sql.DB, bw *filter.Filter, router *router.Table) (map[string][]*filter.Table, error) { +func FetchTargetDoTables(ctx context.Context, db *sql.DB, bw *filter.Filter, router *router.RouteTable) (map[string][]*filter.Table, error) { // fetch tables from source and filter them sourceTables, err := FetchAllDoTables(ctx, db, bw) diff --git a/dm/pkg/utils/common_test.go b/dm/pkg/utils/common_test.go index 71787336517..c678deadcc0 100644 --- a/dm/pkg/utils/common_test.go +++ b/dm/pkg/utils/common_test.go @@ -21,8 +21,8 @@ import ( "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tiflow/dm/pkg/router" ) var _ = Suite(&testCommonSuite{}) @@ -151,7 +151,7 @@ func (s *testCommonSuite) TestFetchTargetDoTables(c *C) { // empty filter and router, just as upstream. ba, err := filter.New(false, nil) c.Assert(err, IsNil) - r, err := router.NewTableRouter(false, nil) + r, err := router.NewRouter(false, nil) c.Assert(err, IsNil) schemas := []string{"shard1"} @@ -178,7 +178,7 @@ func (s *testCommonSuite) TestFetchTargetDoTables(c *C) { c.Assert(mock.ExpectationsWereMet(), IsNil) // route to the same downstream. - r, err = router.NewTableRouter(false, []*router.TableRule{ + r, err = router.NewRouter(false, []*router.TableRule{ {SchemaPattern: "shard*", TablePattern: "tbl*", TargetSchema: "shard", TargetTable: "tbl"}, }) c.Assert(err, IsNil) diff --git a/dm/syncer/ddl_test.go b/dm/syncer/ddl_test.go index 3b32ddc0f21..365cd0009b9 100644 --- a/dm/syncer/ddl_test.go +++ b/dm/syncer/ddl_test.go @@ -22,7 +22,6 @@ import ( "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" @@ -32,6 +31,7 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" @@ -220,7 +220,7 @@ func (s *testDDLSuite) TestResolveDDLSQL(c *C) { syncer.baList, err = filter.New(syncer.cfg.CaseSensitive, syncer.cfg.BAList) c.Assert(err, IsNil) - syncer.tableRouter, err = router.NewTableRouter(false, []*router.TableRule{ + syncer.tableRouter, err = router.NewRouter(false, []*router.TableRule{ { SchemaPattern: "s1", TargetSchema: "xs1", diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 96ebb22d793..41291504414 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -34,7 +34,6 @@ import ( cm "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" - router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -57,6 +56,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/ha" "github.com/pingcap/tiflow/dm/pkg/log" parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/router" "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" "github.com/pingcap/tiflow/dm/pkg/streamer" @@ -157,7 +157,7 @@ type Syncer struct { isTransactionEnd bool waitTransactionLock sync.Mutex - tableRouter *router.Table + tableRouter *router.RouteTable binlogFilter *bf.BinlogEvent columnMapping *cm.Mapping baList *filter.Filter @@ -2435,7 +2435,7 @@ func (qec *queryEventContext) String() string { } // generateExtendColumn generate extended columns by extractor. -func generateExtendColumn(data [][]interface{}, r *router.Table, table *filter.Table, sourceID string) [][]interface{} { +func generateExtendColumn(data [][]interface{}, r *router.RouteTable, table *filter.Table, sourceID string) [][]interface{} { extendCol, extendVal := r.FetchExtendColumn(table.Schema, table.Name, sourceID) if len(extendCol) == 0 { return nil @@ -3168,7 +3168,7 @@ func (s *Syncer) trackOriginDDL(ev *replication.QueryEvent, ec eventContext) (ma } func (s *Syncer) genRouter() error { - s.tableRouter, _ = router.NewTableRouter(s.cfg.CaseSensitive, []*router.TableRule{}) + s.tableRouter, _ = router.NewRouter(s.cfg.CaseSensitive, []*router.TableRule{}) for _, rule := range s.cfg.RouteRules { err := s.tableRouter.AddRule(rule) if err != nil { @@ -3480,7 +3480,7 @@ func (s *Syncer) Update(ctx context.Context, cfg *config.SubTaskConfig) error { var ( err error oldBaList *filter.Filter - oldTableRouter *router.Table + oldTableRouter *router.RouteTable oldBinlogFilter *bf.BinlogEvent oldColumnMapping *cm.Mapping ) @@ -3512,7 +3512,7 @@ func (s *Syncer) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // update route oldTableRouter = s.tableRouter - s.tableRouter, err = router.NewTableRouter(cfg.CaseSensitive, cfg.RouteRules) + s.tableRouter, err = router.NewRouter(cfg.CaseSensitive, cfg.RouteRules) if err != nil { return terror.ErrSyncerUnitGenTableRouter.Delegate(err) } diff --git a/dm/tests/all_mode/conf/regexpr-task.yaml b/dm/tests/all_mode/conf/regexpr-task.yaml new file mode 100644 index 00000000000..557129beb40 --- /dev/null +++ b/dm/tests/all_mode/conf/regexpr-task.yaml @@ -0,0 +1,47 @@ +--- +name: regexprtest +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +timezone: "+04:00" +# enable-heartbeat: true +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "root" + password: "" + session: + tidb_skip_utf8_check: 1 + tidb_disable_txn_auto_retry: off + tidb_retry_limit: "10" + +mysql-instances: + - source-id: "mysql-replica-01" + route-rules: ["rule1"] + block-allow-list: "balist1" + loader: + import-mode: "loader" + - source-id: "mysql-replica-02" + route-rules: ["rule2"] + block-allow-list: "balist1" + loader: + import-mode: "loader" + +routes: + rule1: + schema-pattern: "~test2?[animal|human]" + table-pattern: "~tbl.*[cat|dog]+" + target-schema: "dtest2" + target-table: "dtable2" + rule2: + schema-pattern: "test4s_*" + table-pattern: "~testtable_[donot_delete]?" + target-schema: "dtest4" + target-table: "dtable4" + +block-allow-list: + balist1: + do-dbs: ["~test2?[animal|human]", "test4s_*"] diff --git a/dm/tests/all_mode/conf/regexpr_diff_config.toml b/dm/tests/all_mode/conf/regexpr_diff_config.toml new file mode 100644 index 00000000000..88635ba8680 --- /dev/null +++ b/dm/tests/all_mode/conf/regexpr_diff_config.toml @@ -0,0 +1,49 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + target-check-tables = ["dtest2.dtable2", "dtest4.dtable4"] + + +[routes.rule1] + schema-pattern = "test2animal" + table-pattern = "tbl_animal_dogcat" + target-schema = "dtest2" + target-table = "dtable2" +[routes.rule2] + schema-pattern = "test4s_*" + table-pattern = "testtable_donot_delete" + target-schema = "dtest4" + target-table = "dtable4" + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +route-rules = ["rule2"] + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" \ No newline at end of file diff --git a/dm/tests/all_mode/data/db1.regexpr.sql b/dm/tests/all_mode/data/db1.regexpr.sql new file mode 100644 index 00000000000..ff0c641a604 --- /dev/null +++ b/dm/tests/all_mode/data/db1.regexpr.sql @@ -0,0 +1,13 @@ +-- test regular expression router +drop database if exists `test2animal`; +create database `test2animal`; +use `test2animal`; +create table tbl_animal_dogcat ( + a int, + b int, + primary key(a) +); +insert into tbl_animal_dogcat values +(1, 1), +(2, 2), +(3, 3); \ No newline at end of file diff --git a/dm/tests/all_mode/data/db2.regexpr.sql b/dm/tests/all_mode/data/db2.regexpr.sql new file mode 100644 index 00000000000..789c9b021ac --- /dev/null +++ b/dm/tests/all_mode/data/db2.regexpr.sql @@ -0,0 +1,14 @@ +-- test regular expression router +drop database if exists `test4s_2022`; +create database `test4s_2022`; +use `test4s_2022`; +create table testtable_donot_delete ( + a int, + b int, + primary key(a) +); +insert into testtable_donot_delete values +(1, 1), +(2, 2), +(3, 3), +(4, 4); \ No newline at end of file diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index d933b763d1b..3d43bc83262 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -322,6 +322,61 @@ function test_expression_filter() { echo "[$(date)] <<<<<< finish test_expression_filter >>>>>>" } +function test_regexpr_router() { + echo "[$(date)] <<<<<< start test_regexpr_router >>>>>>" + cleanup_process + cleanup_data all_mode + cleanup_data test2animal + cleanup_data test4s_2022 + cleanup_data_upstream test2animal + cleanup_data_upstream test4s_2022 + run_sql_file $cur/data/db1.regexpr.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $cur/data/db2.regexpr.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + run_sql_tidb 'drop database if exists dtest2;' + run_sql_tidb 'create database dtest2;' + run_sql_tidb 'drop database if exists dtest4;' + run_sql_tidb 'create database dtest4;' + run_sql_tidb 'create table if not exists dtest2.dtable2(a int, b int);' + run_sql_tidb 'create table if not exists dtest4.dtable4(a int, b int);' + # start DM worker and master + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 + 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 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + # operate mysql config to worker + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + cp $cur/conf/regexpr-task.yaml $WORK_DIR/regexpr-task.yaml + sed -i "s/name: test/name: $ILLEGAL_CHAR_NAME/g" $WORK_DIR/regexpr-task.yaml + + # error config + # there should be a error message like "Incorrect argument type to variable 'tidb_retry_limit'" + # but different TiDB version output different message. so we only roughly match here + sed -i 's/tidb_retry_limit: "10"/tidb_retry_limit: "fjs"/g' $WORK_DIR/regexpr-task.yaml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/regexpr-task.yaml --remove-meta" \ + "tidb_retry_limit" 1 + + sed -i 's/tidb_retry_limit: "fjs"/tidb_retry_limit: "10"/g' $WORK_DIR/regexpr-task.yaml + dmctl_start_task "$WORK_DIR/regexpr-task.yaml" "--remove-meta" + + check_sync_diff $WORK_DIR $cur/conf/regexpr_diff_config.toml + + cleanup_process + cleanup_data test2animal + cleanup_data test4s_2022 + cleanup_data_upstream test2animal + cleanup_data_upstream test4s_2022 + echo "[$(date)] <<<<<< finish test_regexpr_router >>>>>>" +} + function run() { run_sql_both_source "SET @@GLOBAL.SQL_MODE='ANSI_QUOTES,NO_AUTO_VALUE_ON_ZERO'" run_sql_source1 "SET @@global.time_zone = '+01:00';" @@ -331,6 +386,7 @@ function run() { test_session_config test_query_timeout test_stop_task_before_checkpoint + test_regexpr_router inject_points=( "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" From 744a26e0b588b5dd82fb505c4b78a4daa5201d1c Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 16 Feb 2022 14:43:39 +0800 Subject: [PATCH 68/72] loader(dm): skip feature for nil etcd client (#4599) ref pingcap/tiflow#4287 --- dm/loader/util.go | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/dm/loader/util.go b/dm/loader/util.go index 9abd144d0ed..c963b4fb58c 100644 --- a/dm/loader/util.go +++ b/dm/loader/util.go @@ -117,7 +117,7 @@ func getMydumpMetadata(cli *clientv3.Client, cfg *config.SubTaskConfig, workerNa loc, _, err := dumpling.ParseMetaData(metafile, cfg.Flavor) if err != nil { if os.IsNotExist(err) { - worker, _, err2 := ha.GetLoadTask(cli, cfg.Name, cfg.SourceID) + worker, err2 := getLoadTask(cli, cfg.Name, cfg.SourceID) if err2 != nil { log.L().Warn("get load task", log.ShortError(err2)) } @@ -171,7 +171,12 @@ func cleanDumpFiles(cfg *config.SubTaskConfig) { } // putLoadTask is called when start restoring data, to put load worker in etcd. +// This is no-op when the `cli` argument is nil. func putLoadTask(cli *clientv3.Client, cfg *config.SubTaskConfig, workerName string) error { + // some usage like DM as a library, we don't support this feature + if cli == nil { + return nil + } _, err := ha.PutLoadTask(cli, cfg.Name, cfg.SourceID, workerName) if err != nil { return err @@ -181,7 +186,12 @@ func putLoadTask(cli *clientv3.Client, cfg *config.SubTaskConfig, workerName str } // delLoadTask is called when finish restoring data, to delete load worker in etcd. +// This is no-op when the `cli` argument is nil. func delLoadTask(cli *clientv3.Client, cfg *config.SubTaskConfig, workerName string) error { + // some usage like DM as a library, we don't support this feature + if cli == nil { + return nil + } _, _, err := ha.DelLoadTask(cli, cfg.Name, cfg.SourceID) if err != nil { return err @@ -189,3 +199,13 @@ func delLoadTask(cli *clientv3.Client, cfg *config.SubTaskConfig, workerName str log.L().Info("delete load worker in etcd for full mode", zap.String("task", cfg.Name), zap.String("source", cfg.SourceID), zap.String("worker", workerName)) return nil } + +// getLoadTask gets the worker which in load stage for the source of the subtask. +// It will return "" and no error when the `cli` argument is nil. +func getLoadTask(cli *clientv3.Client, task, sourceID string) (string, error) { + if cli == nil { + return "", nil + } + name, _, err := ha.GetLoadTask(cli, task, sourceID) + return name, err +} From e7b0aae47c52cf72313b1123df7665cabb37d3c5 Mon Sep 17 00:00:00 2001 From: Ehco Date: Wed, 16 Feb 2022 15:11:39 +0800 Subject: [PATCH 69/72] unit(dm): add Kill func for unit (#4035) close pingcap/tiflow#3737 --- dm/dm/unit/unit.go | 2 + dm/dm/worker/hub_test.go | 2 +- dm/dm/worker/join.go | 3 +- dm/dm/worker/server.go | 57 +-- dm/dm/worker/server_test.go | 30 +- dm/dm/worker/source_worker.go | 10 +- dm/dm/worker/source_worker_test.go | 24 +- dm/dm/worker/subtask.go | 21 + dm/dm/worker/subtask_holder.go | 10 + dm/dm/worker/subtask_test.go | 2 + dm/dumpling/dumpling.go | 6 + dm/loader/lightning.go | 6 + dm/loader/loader.go | 6 + dm/syncer/dml_worker.go | 7 +- dm/syncer/syncer.go | 403 +++++++++--------- dm/syncer/syncer_test.go | 129 ++++-- dm/tests/_utils/ha_cases_lib.sh | 2 +- dm/tests/all_mode/run.sh | 8 +- .../conf/dm-worker1.toml | 4 +- dm/tests/checkpoint_transaction/run.sh | 78 +++- dm/tests/expression_filter/run.sh | 2 +- dm/tests/full_mode/run.sh | 6 +- dm/tests/load_interrupt/run.sh | 2 +- dm/tests/new_relay/run.sh | 190 +++++---- dm/tests/safe_mode/run.sh | 4 +- dm/tests/start_task/run.sh | 2 +- 26 files changed, 614 insertions(+), 402 deletions(-) diff --git a/dm/dm/unit/unit.go b/dm/dm/unit/unit.go index 4451a652e86..363d17bf2cc 100644 --- a/dm/dm/unit/unit.go +++ b/dm/dm/unit/unit.go @@ -48,6 +48,8 @@ type Unit interface { // Close shuts down the process and closes the unit, after that can not call Process to resume // The implementation should not block for a long time. Close() + // Kill shuts down the process and closes the unit without graceful. + Kill() // Pause does some cleanups and the unit can be resumed later. The caller will make sure Process has returned. // The implementation should not block for a long time. Pause() diff --git a/dm/dm/worker/hub_test.go b/dm/dm/worker/hub_test.go index d76697b1e6a..ca20fd437c3 100644 --- a/dm/dm/worker/hub_test.go +++ b/dm/dm/worker/hub_test.go @@ -20,5 +20,5 @@ import ( func (t *testServer) testConidtionHub(c *C, s *Server) { // test condition hub c.Assert(GetConditionHub(), NotNil) - c.Assert(GetConditionHub().w, DeepEquals, s.getWorker(true)) + c.Assert(GetConditionHub().w, DeepEquals, s.getSourceWorker(true)) } diff --git a/dm/dm/worker/join.go b/dm/dm/worker/join.go index ece56ec1a4d..a50c47d44fe 100644 --- a/dm/dm/worker/join.go +++ b/dm/dm/worker/join.go @@ -108,7 +108,8 @@ func (s *Server) KeepAlive() { failpoint.Label("bypass") // TODO: report the error. - err := s.stopWorker("", true) + // when lost keepalive, stop the worker without graceful. this is to fix https://github.com/pingcap/tiflow/issues/3737 + err := s.stopSourceWorker("", true, false) if err != nil { log.L().Error("fail to stop worker", zap.Error(err)) return // return if failed to stop the worker. diff --git a/dm/dm/worker/server.go b/dm/dm/worker/server.go index 3ce68d01b18..6cfcbd481bc 100644 --- a/dm/dm/worker/server.go +++ b/dm/dm/worker/server.go @@ -300,7 +300,7 @@ func (s *Server) observeRelayConfig(ctx context.Context, rev int64) error { } rev = rev1 if relaySource == nil { - if w := s.getWorker(true); w != nil && w.startedRelayBySourceCfg { + if w := s.getSourceWorker(true); w != nil && w.startedRelayBySourceCfg { break } log.L().Info("didn't found relay config after etcd retryable error. Will stop relay now") @@ -314,7 +314,7 @@ func (s *Server) observeRelayConfig(ctx context.Context, rev int64) error { s.Lock() defer s.Unlock() - if w := s.getWorker(false); w != nil && w.cfg.SourceID == relaySource.SourceID { + if w := s.getSourceWorker(false); w != nil && w.cfg.SourceID == relaySource.SourceID { // we may face both relay config and subtask bound changed in a compaction error, so here // we check if observeSourceBound has started a worker // TODO: add a test for this situation @@ -325,7 +325,7 @@ func (s *Server) observeRelayConfig(ctx context.Context, rev int64) error { } return nil } - err = s.stopWorker("", false) + err = s.stopSourceWorker("", false, true) if err != nil { log.L().Error("fail to stop worker", zap.Error(err)) return err // return if failed to stop the worker. @@ -403,7 +403,7 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { s.Lock() defer s.Unlock() - if w := s.getWorker(false); w != nil && w.cfg.SourceID == bound.Source { + if w := s.getSourceWorker(false); w != nil && w.cfg.SourceID == bound.Source { // we may face both relay config and subtask bound changed in a compaction error, so here // we check if observeRelayConfig has started a worker // TODO: add a test for this situation @@ -414,7 +414,7 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { } return nil } - err = s.stopWorker("", false) + err = s.stopSourceWorker("", false, true) if err != nil { log.L().Error("fail to stop worker", zap.Error(err)) return err // return if failed to stop the worker. @@ -441,30 +441,33 @@ func (s *Server) observeSourceBound(ctx context.Context, rev int64) error { } func (s *Server) doClose() { - s.cancel() - // close server in advance, stop receiving source bound and relay bound - s.wg.Wait() - s.Lock() defer s.Unlock() + if s.closed.Load() { return } - // close worker and wait for return - if w := s.getWorker(false); w != nil { - w.Close() + // stop server in advance, stop receiving source bound and relay bound + s.cancel() + s.wg.Wait() + // stop worker and wait for return(we already lock the whole Sever, so no need use lock to get source worker) + if w := s.getSourceWorker(false); w != nil { + w.Stop(true) } s.closed.Store(true) } // Close close the RPC server, this function can be called multiple times. func (s *Server) Close() { + s.doClose() // we should stop current sync first, otherwise master may schedule task on new worker while we are closing s.stopKeepAlive() - s.doClose() + if s.etcdClient != nil { + s.etcdClient.Close() + } } // if needLock is false, we should make sure Server has been locked in caller. -func (s *Server) getWorker(needLock bool) *SourceWorker { +func (s *Server) getSourceWorker(needLock bool) *SourceWorker { if needLock { s.Lock() defer s.Unlock() @@ -497,7 +500,7 @@ func (s *Server) setSourceStatus(source string, err error, needLock bool) { defer s.Unlock() } // now setSourceStatus will be concurrently called. skip setting a source status if worker has been closed - if s.getWorker(false) == nil && source != "" { + if s.getSourceWorker(false) == nil && source != "" { return } s.sourceStatus = pb.SourceStatus{ @@ -515,12 +518,12 @@ func (s *Server) setSourceStatus(source string, err error, needLock bool) { // if sourceID is set to "", worker will be closed directly // if sourceID is not "", we will check sourceID with w.cfg.SourceID. -func (s *Server) stopWorker(sourceID string, needLock bool) error { +func (s *Server) stopSourceWorker(sourceID string, needLock, graceful bool) error { if needLock { s.Lock() defer s.Unlock() } - w := s.getWorker(false) + w := s.getSourceWorker(false) if w == nil { log.L().Warn("worker has not been started, no need to stop", zap.String("source", sourceID)) return nil // no need to stop because not started yet @@ -531,7 +534,7 @@ func (s *Server) stopWorker(sourceID string, needLock bool) error { s.UpdateKeepAliveTTL(s.cfg.KeepAliveTTL) s.setWorker(nil, false) s.setSourceStatus("", nil, false) - w.Close() + w.Stop(graceful) return nil } @@ -659,7 +662,7 @@ func (s *Server) enableHandleSubtasks(sourceCfg *config.SourceConfig, needLock b func (s *Server) disableHandleSubtasks(source string) error { s.Lock() defer s.Unlock() - w := s.getWorker(false) + w := s.getSourceWorker(false) if w == nil { log.L().Warn("worker has already stopped before DisableHandleSubtasks", zap.String("source", source)) return nil @@ -676,7 +679,7 @@ func (s *Server) disableHandleSubtasks(source string) error { var err error if !w.relayEnabled.Load() { log.L().Info("relay is not enabled after disabling subtask, so stop worker") - err = s.stopWorker(source, false) + err = s.stopSourceWorker(source, false, true) } return err } @@ -721,7 +724,7 @@ func (s *Server) enableRelay(sourceCfg *config.SourceConfig, needLock bool) erro func (s *Server) disableRelay(source string) error { s.Lock() defer s.Unlock() - w := s.getWorker(false) + w := s.getSourceWorker(false) if w == nil { log.L().Warn("worker has already stopped before DisableRelay", zap.Any("relaySource", source)) return nil @@ -731,7 +734,7 @@ func (s *Server) disableRelay(source string) error { var err error if !w.subTaskEnabled.Load() { log.L().Info("subtask is not enabled after disabling relay, so stop worker") - err = s.stopWorker(source, false) + err = s.stopSourceWorker(source, false, true) } return err } @@ -747,7 +750,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* SourceStatus: &sourceStatus, } - w := s.getWorker(true) + w := s.getSourceWorker(true) if w == nil { log.L().Warn("fail to call QueryStatus, because no mysql source is being handled in the worker") resp.Result = false @@ -769,7 +772,7 @@ func (s *Server) QueryStatus(ctx context.Context, req *pb.QueryStatusRequest) (* // PurgeRelay implements WorkerServer.PurgeRelay. func (s *Server) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "PurgeRelay"), zap.Stringer("payload", req)) - w := s.getWorker(true) + w := s.getSourceWorker(true) if w == nil { log.L().Warn("fail to call StartSubTask, because no mysql source is being handled in the worker") return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil @@ -786,7 +789,7 @@ func (s *Server) PurgeRelay(ctx context.Context, req *pb.PurgeRelayRequest) (*pb func (s *Server) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "OperateSchema"), zap.Stringer("payload", req)) - w := s.getWorker(true) + w := s.getSourceWorker(true) if w == nil { log.L().Warn("fail to call OperateSchema, because no mysql source is being handled in the worker") return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil @@ -813,7 +816,7 @@ func (s *Server) getOrStartWorker(cfg *config.SourceConfig, needLock bool) (*Sou defer s.Unlock() } - if w := s.getWorker(false); w != nil { + if w := s.getSourceWorker(false); w != nil { if w.cfg.SourceID == cfg.SourceID { log.L().Info("mysql source is being handled", zap.String("sourceID", s.worker.cfg.SourceID)) return w, nil @@ -904,7 +907,7 @@ func getMinLocForSubTask(ctx context.Context, subTaskCfg config.SubTaskConfig) ( func (s *Server) HandleError(ctx context.Context, req *pb.HandleWorkerErrorRequest) (*pb.CommonWorkerResponse, error) { log.L().Info("", zap.String("request", "HandleError"), zap.Stringer("payload", req)) - w := s.getWorker(true) + w := s.getSourceWorker(true) if w == nil { log.L().Warn("fail to call HandleError, because no mysql source is being handled in the worker") return makeCommonWorkerResponse(terror.ErrWorkerNoStart.Generate()), nil diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index 6220ef3f45c..984240f3846 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -208,7 +208,7 @@ func (t *testServer) TestServer(c *C) { _, err = ha.DeleteSubTaskStage(s.etcdClient, ha.NewSubTaskStage(pb.Stage_Stopped, sourceCfg.SourceID, subtaskCfg.Name)) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s.getWorker(true).subTaskHolder.findSubTask(subtaskCfg.Name) == nil + return s.getSourceWorker(true).subTaskHolder.findSubTask(subtaskCfg.Name) == nil }), IsTrue) dupServer := NewServer(cfg) @@ -337,13 +337,13 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { _, err = ha.PutSourceCfg(etcdCli, sourceCfg) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s.getWorker(true) != nil + return s.getSourceWorker(true) != nil }), IsTrue) _, err = ha.DeleteSourceBound(etcdCli, s.cfg.Name) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s.getWorker(true) == nil + return s.getSourceWorker(true) == nil }), IsTrue) } @@ -413,19 +413,19 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { }() // step 4.1: should stop the running worker, source bound has been deleted, should stop this worker c.Assert(utils.WaitSomething(20, 100*time.Millisecond, func() bool { - return s.getWorker(true) == nil + return s.getSourceWorker(true) == nil }), IsTrue) // step 4.2: put a new source bound, source should be started _, err = ha.PutSourceBound(etcdCli, sourceBound) c.Assert(err, IsNil) c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s.getWorker(true) != nil + return s.getSourceWorker(true) != nil }), IsTrue) - cfg2 := s.getWorker(true).cfg + cfg2 := s.getSourceWorker(true).cfg c.Assert(cfg2, DeepEquals, sourceCfg) cancel1() wg.Wait() - c.Assert(s.stopWorker(sourceCfg.SourceID, true), IsNil) + c.Assert(s.stopSourceWorker(sourceCfg.SourceID, true, true), IsNil) // step 5: start observeSourceBound from compacted revision again, should start worker ctx2, cancel2 := context.WithCancel(ctx) wg.Add(1) @@ -434,9 +434,9 @@ func (t *testServer) TestWatchSourceBoundEtcdCompact(c *C) { c.Assert(s.observeSourceBound(ctx2, startRev), IsNil) }() c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return s.getWorker(true) != nil + return s.getSourceWorker(true) != nil }), IsTrue) - cfg2 = s.getWorker(true).cfg + cfg2 = s.getSourceWorker(true).cfg c.Assert(cfg2, DeepEquals, sourceCfg) cancel2() wg.Wait() @@ -480,13 +480,13 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) c.Assert(err, IsNil) // worker should be started and without error c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - w := s.getWorker(true) + w := s.getSourceWorker(true) return w != nil && !w.closed.Load() }), IsTrue) c.Assert(s.getSourceStatus(true).Result, IsNil) } else { // worker should be started before stopped - w := s.getWorker(true) + w := s.getSourceWorker(true) c.Assert(w, NotNil) c.Assert(w.closed.Load(), IsFalse) _, err := ha.DeleteRelayConfig(s.etcdClient, w.name) @@ -495,7 +495,7 @@ func (t *testServer) testOperateWorker(c *C, s *Server, dir string, start bool) c.Assert(err, IsNil) // worker should be closed and without error c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - currentWorker := s.getWorker(true) + currentWorker := s.getSourceWorker(true) return currentWorker == nil && w.closed.Load() }), IsTrue) c.Assert(s.getSourceStatus(true).Result, IsNil) @@ -506,7 +506,7 @@ func (t *testServer) testRetryConnectMaster(c *C, s *Server, etcd *embed.Etcd, d etcd.Close() time.Sleep(6 * time.Second) // When worker server fail to keepalive with etcd, server should close its worker - c.Assert(s.getWorker(true), IsNil) + c.Assert(s.getSourceWorker(true), IsNil) c.Assert(s.getSourceStatus(true).Result, IsNil) ETCD, err := createMockETCD(dir, "http://"+hostName) c.Assert(err, IsNil) @@ -551,9 +551,9 @@ func (t *testServer) testSubTaskRecover(c *C, s *Server, dir string) { func (t *testServer) testStopWorkerWhenLostConnect(c *C, s *Server, etcd *embed.Etcd) { etcd.Close() c.Assert(utils.WaitSomething(int(defaultKeepAliveTTL+3), time.Second, func() bool { - return s.getWorker(true) == nil + return s.getSourceWorker(true) == nil }), IsTrue) - c.Assert(s.getWorker(true), IsNil) + c.Assert(s.getSourceWorker(true), IsNil) } func (t *testServer) TestGetMinLocInAllSubTasks(c *C) { diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 851a3245317..5c85300590f 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -203,7 +203,7 @@ func (w *SourceWorker) Start() { } // Close stops working and releases resources. -func (w *SourceWorker) Close() { +func (w *SourceWorker) Stop(graceful bool) { if w.closed.Load() { w.l.Warn("already closed") return @@ -218,8 +218,12 @@ func (w *SourceWorker) Close() { w.Lock() defer w.Unlock() - // close all sub tasks - w.subTaskHolder.closeAllSubTasks() + // close or kill all subtasks + if graceful { + w.subTaskHolder.closeAllSubTasks() + } else { + w.subTaskHolder.killAllSubTasks() + } if w.relayHolder != nil { w.relayHolder.Close() diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 1ae4eaa476a..a9ec7be9691 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -89,11 +89,11 @@ func (t *testServer) testWorker(c *C) { c.Assert(err, IsNil) c.Assert(w.GetUnitAndSourceStatusJSON("", nil), HasLen, emptyWorkerStatusInfoJSONLength) - // close twice - w.Close() + // stop twice + w.Stop(true) c.Assert(w.closed.Load(), IsTrue) c.Assert(w.subTaskHolder.getAllSubTasks(), HasLen, 0) - w.Close() + w.Stop(true) c.Assert(w.closed.Load(), IsTrue) c.Assert(w.subTaskHolder.getAllSubTasks(), HasLen, 0) c.Assert(w.closed.Load(), IsTrue) @@ -198,11 +198,11 @@ func (t *testServer2) TestTaskAutoResume(c *C) { c.Assert(err, IsNil) subtaskCfg.Mode = "full" subtaskCfg.Timezone = "UTC" - c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) + c.Assert(s.getSourceWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, pb.Stage_Stopped, true), IsNil) // check task in paused state c.Assert(utils.WaitSomething(100, 100*time.Millisecond, func() bool { - subtaskStatus, _, _ := s.getWorker(true).QueryStatus(context.Background(), taskName) + subtaskStatus, _, _ := s.getSourceWorker(true).QueryStatus(context.Background(), taskName) for _, st := range subtaskStatus { if st.Name == taskName && st.Stage == pb.Stage_Paused { return true @@ -213,7 +213,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { //nolint:errcheck failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError") - rtsc, ok := s.getWorker(true).taskStatusChecker.(*realTaskStatusChecker) + rtsc, ok := s.getSourceWorker(true).taskStatusChecker.(*realTaskStatusChecker) c.Assert(ok, IsTrue) defer func() { // close multiple time @@ -223,7 +223,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { // check task will be auto resumed c.Assert(utils.WaitSomething(10, 100*time.Millisecond, func() bool { - sts, _, _ := s.getWorker(true).QueryStatus(context.Background(), taskName) + sts, _, _ := s.getSourceWorker(true).QueryStatus(context.Background(), taskName) for _, st := range sts { if st.Name == taskName && st.Stage == pb.Stage_Running { return true @@ -295,7 +295,7 @@ func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { // start worker w, err := NewSourceWorker(sourceCfg, etcdCli, "", "") c.Assert(err, IsNil) - defer w.Close() + defer w.Stop(true) go func() { w.Start() }() @@ -467,7 +467,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - defer w.Close() + defer w.Stop(true) go func() { w.Start() }() @@ -546,7 +546,7 @@ func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { c.Assert(status, HasLen, 1) c.Assert(status[0].Name, Equals, subtaskCfg.Name) c.Assert(status[0].Stage, Equals, pb.Stage_Running) - w.Close() + w.Stop(true) cancel2() wg.Wait() } @@ -585,7 +585,7 @@ func (t *testWorkerEtcdCompact) TestWatchValidatorStageEtcdCompact(c *C) { c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - defer w.Close() + defer w.Stop(true) go func() { w.Start() }() @@ -706,7 +706,7 @@ func (t *testWorkerEtcdCompact) TestWatchRelayStageEtcdCompact(c *C) { c.Assert(err, IsNil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - defer w.Close() + defer w.Stop(true) go func() { c.Assert(w.EnableRelay(false), IsNil) w.Start() diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index 84621226eb5..772723ff3f9 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -428,6 +428,16 @@ func (st *SubTask) closeUnits() { u := st.units[i] st.l.Info("closing unit process", zap.Stringer("unit", cu.Type())) u.Close() + st.l.Info("closing unit done", zap.Stringer("unit", cu.Type())) + } +} + +func (st *SubTask) killCurrentUnit() { + if st.CurrUnit() != nil { + ut := st.CurrUnit().Type() + st.l.Info("kill unit", zap.String("task", st.cfg.Name), zap.Stringer("unit", ut)) + st.CurrUnit().Kill() + st.l.Info("kill unit done", zap.String("task", st.cfg.Name), zap.Stringer("unit", ut)) } } @@ -530,7 +540,18 @@ func (st *SubTask) Close() { st.l.Info("subTask is already closed, no need to close") return } + st.closeUnits() // close all un-closed units + updateTaskMetric(st.cfg.Name, st.cfg.SourceID, pb.Stage_Stopped, st.workerName) +} +// Kill kill running unit and stop the sub task. +func (st *SubTask) Kill() { + st.l.Info("killing") + if !st.setStageIfNotIn([]pb.Stage{pb.Stage_Stopped, pb.Stage_Stopping, pb.Stage_Finished}, pb.Stage_Stopping) { + st.l.Info("subTask is already closed, no need to close") + return + } + st.killCurrentUnit() st.closeUnits() // close all un-closed units cfg := st.getCfg() diff --git a/dm/dm/worker/subtask_holder.go b/dm/dm/worker/subtask_holder.go index 27f512d62ed..a723bf5b41c 100644 --- a/dm/dm/worker/subtask_holder.go +++ b/dm/dm/worker/subtask_holder.go @@ -73,6 +73,16 @@ func (h *subTaskHolder) closeAllSubTasks() { h.subTasks = make(map[string]*SubTask) } +// killAllSubTasks kill and stop all subtask instances. +func (h *subTaskHolder) killAllSubTasks() { + h.mu.Lock() + defer h.mu.Unlock() + for _, st := range h.subTasks { + st.Kill() + } + h.subTasks = make(map[string]*SubTask) +} + // findSubTask finds subtask instance by name. func (h *subTaskHolder) findSubTask(name string) *SubTask { h.mu.RLock() diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 7e4d142dd43..8fd1b6eb166 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -118,6 +118,8 @@ func (m *MockUnit) Process(ctx context.Context, pr chan pb.ProcessResult) { func (m *MockUnit) Close() {} +func (m *MockUnit) Kill() {} + func (m MockUnit) Pause() {} func (m *MockUnit) Resume(ctx context.Context, pr chan pb.ProcessResult) { m.Process(ctx, pr) } diff --git a/dm/dumpling/dumpling.go b/dm/dumpling/dumpling.go index 51fb5d7bd88..b50f9139b39 100644 --- a/dm/dumpling/dumpling.go +++ b/dm/dumpling/dumpling.go @@ -175,6 +175,12 @@ func (m *Dumpling) Close() { m.closed.Store(true) } +// Kill implements Unit.Kill. +func (m *Dumpling) Kill() { + // TODO: implement kill + m.Close() +} + // Pause implements Unit.Pause. func (m *Dumpling) Pause() { if m.closed.Load() { diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index 08cc1bc5df1..3ed0004ec3d 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -334,6 +334,12 @@ func (l *LightningLoader) Close() { l.closed.Store(true) } +// Kill does ungraceful shutdown. +func (l *LightningLoader) Kill() { + // TODO: implement kill + l.Close() +} + // Pause pauses the process, and it can be resumed later // should cancel context from external. func (l *LightningLoader) Pause() { diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 3ecc6492ce8..ebbc5416b53 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -804,6 +804,12 @@ func (l *Loader) Close() { l.closed.Store(true) } +// Kill kill the loader without graceful. +func (l *Loader) Kill() { + // TODO: implement kill + l.Close() +} + // stopLoad stops loading, now it used by Close and Pause // maybe we can refine the workflow more clear. func (l *Loader) stopLoad() { diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index 016a418969a..381cd55b19b 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -36,7 +36,7 @@ type DMLWorker struct { chanSize int multipleRows bool toDBConns []*dbconn.DBConn - tctx *tcontext.Context + syncCtx *tcontext.Context logger log.Logger // for metrics @@ -75,7 +75,7 @@ func dmlWorkerWrap(inCh chan *job, syncer *Syncer) chan *job { fatalFunc: syncer.fatalFunc, lagFunc: syncer.updateReplicationJobTS, updateJobMetricsFunc: syncer.updateJobMetrics, - tctx: syncer.tctx, + syncCtx: syncer.syncCtx, // this ctx can be used to cancel all the workers toDBConns: syncer.toDBConns, inCh: inCh, flushCh: make(chan *job), @@ -112,7 +112,6 @@ func (w *DMLWorker) run() { for i := 0; i < w.workerCount; i++ { queueBucketMapping[i] = queueBucketName(i) } - for j := range w.inCh { metrics.QueueSizeGauge.WithLabelValues(w.task, "dml_worker_input", w.source).Set(float64(len(w.inCh))) switch j.tp { @@ -244,7 +243,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { time.Sleep(time.Duration(t) * time.Second) }) // use background context to execute sqls as much as possible - ctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) + ctx, cancel := w.syncCtx.WithTimeout(maxDMLExecutionDuration) defer cancel() affect, err = db.ExecuteSQL(ctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 41291504414..de05b2132a8 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -121,7 +121,17 @@ const ( type Syncer struct { sync.RWMutex - tctx *tcontext.Context + tctx *tcontext.Context // this ctx only used for logger. + + // this ctx derives from a background ctx and was initialized in s.Run, it is used for some background tasks in s.Run + // when this ctx cancelled, syncer will shutdown all background running jobs (except the syncDML and syncDDL) and not wait transaction end. + runCtx *tcontext.Context + runCancel context.CancelFunc + // this ctx only used for syncDML and syncDDL and only cancelled when ungraceful stop. + syncCtx *tcontext.Context + syncCancel context.CancelFunc + // control all goroutines that started in S.Run + runWg sync.WaitGroup cfg *config.SubTaskConfig syncCfg replication.BinlogSyncerConfig @@ -135,7 +145,6 @@ type Syncer struct { binlogType BinlogType streamerController *StreamerController - wg sync.WaitGroup // counts goroutines jobWg sync.WaitGroup // counts ddl/flush/asyncFlush job in-flight in s.dmlJobCh and s.ddlJobCh schemaTracker *schema.Tracker @@ -188,8 +197,6 @@ type Syncer struct { filteredUpdate atomic.Int64 filteredDelete atomic.Int64 - done chan struct{} - checkpoint CheckPoint checkpointFlushWorker *checkpointFlushWorker onlineDDL onlineddl.OnlinePlugin @@ -253,7 +260,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, relay rel syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) syncer.count.Store(0) - syncer.done = nil syncer.handleJobFunc = syncer.handleJob syncer.cli = etcdClient @@ -279,11 +285,9 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, relay rel } func (s *Syncer) newJobChans() { - s.closeJobChans() chanSize := calculateChanSize(s.cfg.QueueSize, s.cfg.WorkerCount, s.cfg.Compact) s.dmlJobCh = make(chan *job, chanSize) s.ddlJobCh = make(chan *job, s.cfg.QueueSize) - s.checkpointFlushWorker.input = make(chan *checkpointFlushTask, 16) s.jobsClosed.Store(false) } @@ -295,7 +299,6 @@ func (s *Syncer) closeJobChans() { } close(s.dmlJobCh) close(s.ddlJobCh) - close(s.checkpointFlushWorker.input) s.jobsClosed.Store(true) } @@ -431,13 +434,6 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } } } - s.checkpointFlushWorker = &checkpointFlushWorker{ - input: nil, // will be created in s.reset() - cp: s.checkpoint, - execError: &s.execError, - afterFlushFn: s.afterFlushCheckpoint, - updateJobMetricsFn: s.updateJobMetrics, - } // when Init syncer, set active relay log info if s.cfg.Meta == nil || s.cfg.Meta.BinLogName != binlog.FakeBinlogName { @@ -447,7 +443,6 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } rollbackHolder.Add(fr.FuncRollback{Name: "remove-active-realylog", Fn: s.removeActiveRelayLog}) } - s.reset() return nil } @@ -565,6 +560,13 @@ func (s *Syncer) reset() { } // create new job chans s.newJobChans() + s.checkpointFlushWorker = &checkpointFlushWorker{ + input: make(chan *checkpointFlushTask, 16), + cp: s.checkpoint, + execError: &s.execError, + afterFlushFn: s.afterFlushCheckpoint, + updateJobMetricsFn: s.updateJobMetrics, + } s.execError.Store(nil) s.setErrLocation(nil, nil, false) @@ -572,7 +574,6 @@ func (s *Syncer) reset() { s.waitXIDJob.Store(int64(noWait)) s.isTransactionEnd = true s.flushSeq = 0 - switch s.cfg.ShardMode { case config.ShardPessimistic: // every time start to re-sync from resume, we reset status to make it like a fresh syncing @@ -639,7 +640,6 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { s.Unlock() return } - s.done = make(chan struct{}) s.Unlock() runFatalChan := make(chan *pb.ProcessError, s.cfg.WorkerCount+1) @@ -678,8 +678,6 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { // cancel goroutines created in s.Run cancel() } - s.closeJobChans() // Run returned, all jobs sent, we can close s.jobs - s.wg.Wait() // wait for sync goroutine to return close(runFatalChan) // Run returned, all potential fatal sent to s.runFatalChan wg.Wait() // wait for receive all fatal from s.runFatalChan @@ -701,14 +699,6 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { default: } - // try to rollback checkpoints, if they already flushed, no effect - prePos := s.checkpoint.GlobalPoint() - s.checkpoint.Rollback(s.schemaTracker) - currPos := s.checkpoint.GlobalPoint() - if binlog.CompareLocation(prePos, currPos, s.cfg.EnableGTID) != 0 { - s.tctx.L().Warn("something wrong with rollback global checkpoint", zap.Stringer("previous position", prePos), zap.Stringer("current position", currPos)) - } - pr <- pb.ProcessResult{ IsCanceled: isCanceled, Errors: errs, @@ -978,7 +968,7 @@ func (s *Syncer) addJob(job *job) { s.dmlJobCh <- job failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { s.tctx.L().Info("receive dml job", zap.Any("dml job", job)) - time.Sleep(100 * time.Millisecond) + time.Sleep(500 * time.Millisecond) }) case gc: s.dmlJobCh <- job @@ -1159,7 +1149,7 @@ func (s *Syncer) flushCheckPoints() error { snapshotInfo, exceptTables, shardMetaSQLs, shardMetaArgs := s.createCheckpointSnapshot(true) if snapshotInfo == nil { - log.L().Info("checkpoint has no change, skip sync flush checkpoint") + s.tctx.L().Info("checkpoint has no change, skip sync flush checkpoint") return nil } @@ -1195,7 +1185,7 @@ func (s *Syncer) flushCheckPointsAsync(asyncFlushJob *job) { snapshotInfo, exceptTables, shardMetaSQLs, shardMetaArgs := s.createCheckpointSnapshot(false) if snapshotInfo == nil { - log.L().Info("checkpoint has no change, skip async flush checkpoint", zap.Int64("job seq", asyncFlushJob.flushSeq)) + s.tctx.L().Info("checkpoint has no change, skip async flush checkpoint", zap.Int64("job seq", asyncFlushJob.flushSeq)) return } @@ -1287,8 +1277,8 @@ func (s *Syncer) logAndClearFilteredStatistics() { } // DDL synced one by one, so we only need to process one DDL at a time. -func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn.DBConn, ddlJobChan chan *job) { - defer s.wg.Done() +func (s *Syncer) syncDDL(queueBucket string, db *dbconn.DBConn, ddlJobChan chan *job) { + defer s.runWg.Done() var err error for { @@ -1312,12 +1302,12 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. shardPessimistOp = s.pessimist.PendingOperation() if shardPessimistOp != nil && !shardPessimistOp.Exec { ignore = true - tctx.L().Info("ignore shard DDLs in pessimistic shard mode", zap.Strings("ddls", ddlJob.ddls)) + s.tctx.L().Info("ignore shard DDLs in pessimistic shard mode", zap.Strings("ddls", ddlJob.ddls)) } case config.ShardOptimistic: if len(ddlJob.ddls) == 0 { ignore = true - tctx.L().Info("ignore shard DDLs in optimistic mode", zap.Stringer("info", s.optimist.PendingInfo())) + s.tctx.L().Info("ignore shard DDLs in optimistic mode", zap.Stringer("info", s.optimist.PendingInfo())) } } @@ -1329,9 +1319,9 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. if !ignore { var affected int - affected, err = db.ExecuteSQLWithIgnore(tctx, errorutil.IsIgnorableMySQLDDLError, ddlJob.ddls) + affected, err = db.ExecuteSQLWithIgnore(s.syncCtx, errorutil.IsIgnorableMySQLDDLError, ddlJob.ddls) if err != nil { - err = s.handleSpecialDDLError(tctx, err, ddlJob.ddls, affected, db) + err = s.handleSpecialDDLError(s.syncCtx, err, ddlJob.ddls, affected, db) err = terror.WithScope(err, terror.ScopeDownstream) } } @@ -1365,7 +1355,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. switch { case shardInfo == nil: // no need to do the shard DDL handle for `CREATE DATABASE/TABLE` now. - tctx.L().Warn("skip shard DDL handle in pessimistic shard mode", zap.Strings("ddl", ddlJob.ddls)) + s.tctx.L().Warn("skip shard DDL handle in pessimistic shard mode", zap.Strings("ddl", ddlJob.ddls)) case shardPessimistOp == nil: err = terror.ErrWorkerDDLLockOpNotFound.Generate(shardInfo) default: @@ -1378,7 +1368,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. // no need to do the shard DDL handle for `DROP DATABASE/TABLE` now. // but for `CREATE DATABASE` and `ALTER DATABASE` we execute it to the downstream directly without `shardInfo`. if ignore { // actually ignored. - tctx.L().Warn("skip shard DDL handle in optimistic shard mode", zap.Strings("ddl", ddlJob.ddls)) + s.tctx.L().Warn("skip shard DDL handle in optimistic shard mode", zap.Strings("ddl", ddlJob.ddls)) } case s.optimist.PendingOperation() == nil: err = terror.ErrWorkerDDLLockOpNotFound.Generate(shardInfo) @@ -1437,7 +1427,7 @@ func (s *Syncer) fatalFunc(job *job, err error) { // DML synced with causality. func (s *Syncer) syncDML() { - defer s.wg.Done() + defer s.runWg.Done() dmlJobCh := s.dmlJobCh if s.cfg.Compact { @@ -1451,47 +1441,109 @@ func (s *Syncer) syncDML() { } } -// Run starts running for sync, we should guarantee it can rerun when paused. -func (s *Syncer) Run(ctx context.Context) (err error) { - runCtx, runCancel := context.WithCancel(context.Background()) - defer runCancel() - tctx := s.tctx.WithContext(runCtx) +func (s *Syncer) waitBeforeRunExit(ctx context.Context) { + defer s.runWg.Done() + failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { + s.tctx.L().Info("incr maxPauseOrStopWaitTime time ") + maxPauseOrStopWaitTime = time.Minute * 10 + }) - defer func() { - if s.done != nil { - close(s.done) + select { + case <-ctx.Done(): // hijack the root context from s.Run to wait for the transaction to end. + s.tctx.L().Info("received subtask's done, try graceful stop") + s.waitTransactionLock.Lock() + if s.isTransactionEnd { + s.waitXIDJob.Store(int64(waitComplete)) + s.waitTransactionLock.Unlock() + s.tctx.L().Info("the last job is transaction end, done directly") + s.runCancel() + return } - }() + s.waitXIDJob.Store(int64(waiting)) + s.waitTransactionLock.Unlock() + select { + case <-s.runCtx.Ctx.Done(): + s.tctx.L().Info("syncer run exit so runCtx done") + case <-time.After(maxPauseOrStopWaitTime): + // TODO: maxPauseOrStopWaitTime should also count the time of waiting waitTransactionLock + s.tctx.L().Info("wait transaction end timeout, exit now") + s.runCancel() + } + case <-s.runCtx.Ctx.Done(): // when no graceful stop, run ctx will canceled first. + s.tctx.L().Info("received ungraceful exit ctx, exit now") + } +} - go func() { - <-ctx.Done() +func (s *Syncer) updateTSOffsetCronJob(ctx context.Context) { + defer s.runWg.Done() + // temporarily hard code there. if this metrics works well add this to config file. + ticker := time.NewTicker(time.Minute * 10) + defer ticker.Stop() + for { select { - case <-runCtx.Done(): - default: - tctx.L().Info("received subtask's done") - - s.waitTransactionLock.Lock() - if s.isTransactionEnd { - s.waitXIDJob.Store(int64(waitComplete)) - tctx.L().Info("the last job is transaction end, done directly") - runCancel() - s.waitTransactionLock.Unlock() - return + case <-ticker.C: + if utErr := s.updateTSOffset(ctx); utErr != nil { + s.tctx.L().Error("get server unix ts err", zap.Error(utErr)) } - s.waitXIDJob.Store(int64(waiting)) - s.waitTransactionLock.Unlock() + case <-ctx.Done(): + return + } + } +} - select { - case <-runCtx.Done(): - tctx.L().Info("received syncer's done") - case <-time.After(maxPauseOrStopWaitTime): - tctx.L().Info("wait transaction end timeout") - runCancel() - } +func (s *Syncer) updateLagCronJob(ctx context.Context) { + defer s.runWg.Done() + // temporarily hard code there. if this metrics works well add this to config file. + ticker := time.NewTicker(time.Millisecond * 100) + defer ticker.Stop() + for { + select { + case <-ticker.C: + s.updateReplicationLagMetric() + case <-ctx.Done(): + return } + } +} + +func (s *Syncer) updateTSOffset(ctx context.Context) error { + t1 := time.Now() + ts, tsErr := s.fromDB.GetServerUnixTS(ctx) + rtt := time.Since(t1).Seconds() + if tsErr == nil { + s.tsOffset.Store(time.Now().Unix() - ts - int64(rtt/2)) + } + return tsErr +} + +// Run starts running for sync, we should guarantee it can rerun when paused. +func (s *Syncer) Run(ctx context.Context) (err error) { + runCtx, runCancel := context.WithCancel(context.Background()) + s.runCtx, s.runCancel = tcontext.NewContext(runCtx, s.tctx.L()), runCancel + syncCtx, syncCancel := context.WithCancel(context.Background()) + s.syncCtx, s.syncCancel = tcontext.NewContext(syncCtx, s.tctx.L()), syncCancel + defer func() { + s.runCancel() + s.closeJobChans() + s.checkpointFlushWorker.Close() + s.runWg.Wait() + // s.syncCancel won't be called when normal exit, this call just to follow the best practice of use context. + s.syncCancel() + }() + + // we should start this goroutine as soon as possible, because it's the only goroutine that cancel syncer.Run + s.runWg.Add(1) + go func() { + s.waitBeforeRunExit(ctx) }() - fresh, err := s.IsFreshTask(runCtx) + // before sync run, we get the ts offset from upstream first + if utErr := s.updateTSOffset(ctx); utErr != nil { + return utErr + } + + // some initialization that can't be put in Syncer.Init + fresh, err := s.IsFreshTask(s.runCtx.Ctx) if err != nil { return err } @@ -1505,7 +1557,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { s.tctx.L().Error("failed to get task cli args", zap.Error(err)) } if s.cliArgs != nil && s.cliArgs.StartTime != "" { - err = s.setGlobalPointByTime(tctx, s.cliArgs.StartTime) + err = s.setGlobalPointByTime(s.runCtx, s.cliArgs.StartTime) if terror.ErrConfigStartTimeTooLate.Equal(err) { return err } @@ -1522,19 +1574,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } } - // start flush checkpoints worker. - s.wg.Add(1) - go func() { - defer s.wg.Done() - s.checkpointFlushWorker.Run(s.tctx) - }() - var ( flushCheckpoint bool delLoadTask bool cleanDumpFile = s.cfg.CleanDumpFile ) - flushCheckpoint, err = s.adjustGlobalPointGTID(tctx) + flushCheckpoint, err = s.adjustGlobalPointGTID(s.runCtx) if err != nil { return err } @@ -1543,11 +1588,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { flushCheckpoint = true err = s.loadTableStructureFromDump(ctx) if err != nil { - tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) + s.tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) cleanDumpFile = false } if s.cfg.ShardMode == config.ShardOptimistic { - s.flushOptimisticTableInfos(tctx) + s.flushOptimisticTableInfos(s.runCtx) } } @@ -1555,62 +1600,44 @@ func (s *Syncer) Run(ctx context.Context) (err error) { cleanDumpFile = false } + s.runWg.Add(1) + go s.syncDML() + s.runWg.Add(1) + go func() { + defer s.runWg.Done() + // also need to use a different ctx. checkpointFlushWorker worker will be closed in the first defer + s.checkpointFlushWorker.Run(s.tctx) + }() + s.runWg.Add(1) + go s.syncDDL(adminQueueName, s.ddlDBConn, s.ddlJobCh) + s.runWg.Add(1) + go s.updateLagCronJob(s.runCtx.Ctx) + s.runWg.Add(1) + go s.updateTSOffsetCronJob(s.runCtx.Ctx) if flushCheckpoint { if err = s.flushCheckPoints(); err != nil { - tctx.L().Warn("fail to flush checkpoints when starting task", zap.Error(err)) + s.tctx.L().Warn("fail to flush checkpoints when starting task", zap.Error(err)) return err } } if delLoadTask { if err = s.delLoadTask(); err != nil { - tctx.L().Warn("error when del load task in etcd", zap.Error(err)) + s.tctx.L().Warn("error when del load task in etcd", zap.Error(err)) } } if cleanDumpFile { - tctx.L().Info("try to remove all dump files") + s.tctx.L().Info("try to remove all dump files") if err = os.RemoveAll(s.cfg.Dir); err != nil { - tctx.L().Warn("error when remove loaded dump folder", zap.String("data folder", s.cfg.Dir), zap.Error(err)) + s.tctx.L().Warn("error when remove loaded dump folder", zap.String("data folder", s.cfg.Dir), zap.Error(err)) } } failpoint.Inject("AdjustGTIDExit", func() { - tctx.L().Warn("exit triggered", zap.String("failpoint", "AdjustGTIDExit")) + s.tctx.L().Warn("exit triggered", zap.String("failpoint", "AdjustGTIDExit")) s.streamerController.Close() utils.OsExit(1) }) - updateTSOffset := func() error { - t1 := time.Now() - ts, tsErr := s.fromDB.GetServerUnixTS(runCtx) - rtt := time.Since(t1).Seconds() - if tsErr == nil { - s.tsOffset.Store(time.Now().Unix() - ts - int64(rtt/2)) - } - return tsErr - } - // before sync run, we get the tsoffset from upstream first - if utErr := updateTSOffset(); utErr != nil { - return utErr - } - // start background task to get/update current ts offset between dm and upstream - s.wg.Add(1) - go func() { - defer s.wg.Done() - // temporarily hard code there. if this metrics works well add this to config file. - updateTicker := time.NewTicker(time.Minute * 10) - defer updateTicker.Stop() - for { - select { - case <-updateTicker.C: - if utErr := updateTSOffset(); utErr != nil { - s.tctx.L().Error("get server unix ts err", zap.Error(utErr)) - } - case <-runCtx.Done(): - return - } - } - }() - // startLocation is the start location for current received event // currentLocation is the end location for current received event (End_log_pos in `show binlog events` for mysql) // lastLocation is the end location for last received (ROTATE / QUERY / XID) event @@ -1624,37 +1651,15 @@ func (s *Syncer) Run(ctx context.Context) (err error) { currentGTID string ) - tctx.L().Info("replicate binlog from checkpoint", zap.Stringer("checkpoint", lastLocation)) + s.tctx.L().Info("replicate binlog from checkpoint", zap.Stringer("checkpoint", lastLocation)) if s.streamerController.IsClosed() { s.locations.reset(lastLocation) - err = s.streamerController.Start(tctx, lastLocation) + err = s.streamerController.Start(s.runCtx, lastLocation) if err != nil { return terror.Annotate(err, "fail to restart streamer controller") } } - - s.wg.Add(1) - go s.syncDML() - - s.wg.Add(1) - go s.syncDDL(tctx, adminQueueName, s.ddlDBConn, s.ddlJobCh) - - s.wg.Add(1) - go func() { - defer s.wg.Done() - updateLagTicker := time.NewTicker(time.Millisecond * 100) - defer updateLagTicker.Stop() - for { - select { - case <-updateLagTicker.C: - s.updateReplicationLagMetric() - case <-runCtx.Done(): - return - } - } - }() - // syncing progress with sharding DDL group // 1. use the global streamer to sync regular binlog events // 2. sharding DDL synced for some sharding groups @@ -1672,13 +1677,14 @@ func (s *Syncer) Run(ctx context.Context) (err error) { traceSource = fmt.Sprintf("%s.syncer.%s", s.cfg.SourceID, s.cfg.Name) ) + // this is second defer func in syncer.Run so in this time checkpointFlushWorker are still running defer func() { if err1 := recover(); err1 != nil { failpoint.Inject("ExitAfterSaveOnlineDDL", func() { - tctx.L().Info("force panic") + s.tctx.L().Info("force panic") panic("ExitAfterSaveOnlineDDL") }) - tctx.L().Error("panic log", zap.Reflect("error message", err1), zap.Stack("stack")) + s.tctx.L().Error("panic log", zap.Reflect("error message", err1), zap.Stack("stack")) err = terror.ErrSyncerUnitPanic.Generate(err1) } @@ -1695,13 +1701,13 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // flush all jobs before exit if err2 = s.flushJobs(); err2 != nil { - tctx.L().Warn("failed to flush jobs when exit task", zap.Error(err2)) + s.tctx.L().Warn("failed to flush jobs when exit task", zap.Error(err2)) } // 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)) + s.tctx.L().Warn("failed to flush safe mode checkpoints when exit task", zap.Error(err2)) } } }() @@ -1720,7 +1726,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // if we start syncer at an early position, database must bear a period of inconsistent state, // it's eventual consistency. s.safeMode = sm.NewSafeMode() - s.enableSafeModeInitializationPhase(tctx) + s.enableSafeModeInitializationPhase(s.runCtx) closeShardingResync := func() error { if shardingReSync == nil { @@ -1744,7 +1750,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { s.isReplacingOrInjectingErr = currentLocation.Suffix != 0 s.locations.reset(currentLocation) - err3 := s.streamerController.RedirectStreamer(tctx, currentLocation) + err3 := s.streamerController.RedirectStreamer(s.tctx, currentLocation) if err3 != nil { return err3 } @@ -1759,7 +1765,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } for i := 0; i < n; { - e, err1 := s.getEvent(tctx, currentLocation) + e, err1 := s.getEvent(s.runCtx, currentLocation) if err1 != nil { return err } @@ -1780,7 +1786,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { i++ } } - log.L().Info("discard event already consumed", zap.Int("count", n), + s.tctx.L().Info("discard event already consumed", zap.Int("count", n), zap.Any("cur_loc", currentLocation)) return nil } @@ -1810,13 +1816,13 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // if suffix>0, we are replacing error s.isReplacingOrInjectingErr = currentLocation.Suffix != 0 s.locations.reset(shardingReSync.currLocation) - err = s.streamerController.RedirectStreamer(tctx, shardingReSync.currLocation) + err = s.streamerController.RedirectStreamer(s.runCtx, shardingReSync.currLocation) if err != nil { return err } failpoint.Inject("ReSyncExit", func() { - tctx.L().Warn("exit triggered", zap.String("failpoint", "ReSyncExit")) + s.tctx.L().Warn("exit triggered", zap.String("failpoint", "ReSyncExit")) utils.OsExit(1) }) } @@ -1824,7 +1830,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { var e *replication.BinlogEvent startTime := time.Now() - e, err = s.getEvent(tctx, currentLocation) + e, err = s.getEvent(s.runCtx, currentLocation) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { if intVal, ok := val.(int); ok && intVal == 1 { @@ -1842,30 +1848,30 @@ func (s *Syncer) Run(ctx context.Context) (err error) { }) switch { case err == context.Canceled: - tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) + s.tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) return nil case err == context.DeadlineExceeded: - tctx.L().Info("deadline exceeded when fetching binlog event") + s.tctx.L().Info("deadline exceeded when fetching binlog event") continue case isDuplicateServerIDError(err): // if the server id is already used, need to use a new server id - tctx.L().Info("server id is already used by another slave, will change to a new server id and get event again") - err1 := s.streamerController.UpdateServerIDAndResetReplication(tctx, lastLocation) + s.tctx.L().Info("server id is already used by another slave, will change to a new server id and get event again") + err1 := s.streamerController.UpdateServerIDAndResetReplication(s.tctx, lastLocation) if err1 != nil { return err1 } continue case err == relay.ErrorMaybeDuplicateEvent: - tctx.L().Warn("read binlog met a truncated file, will skip events that has been consumed") + s.tctx.L().Warn("read binlog met a truncated file, will skip events that has been consumed") err = maybeSkipNRowsEvent(eventIndex) if err == nil { continue } - log.L().Warn("skip duplicate rows event failed", zap.Error(err)) + s.tctx.L().Warn("skip duplicate rows event failed", zap.Error(err)) } if err != nil { - tctx.L().Error("fail to fetch binlog", log.ShortError(err)) + s.tctx.L().Error("fail to fetch binlog", log.ShortError(err)) if isConnectionRefusedError(err) { return err @@ -1873,11 +1879,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.streamerController.CanRetry(err) { // GlobalPoint is the last finished transaction location - err = s.streamerController.ResetReplicationSyncer(tctx, s.checkpoint.GlobalPoint()) + err = s.streamerController.ResetReplicationSyncer(s.tctx, s.checkpoint.GlobalPoint()) if err != nil { return err } - log.L().Info("reset replication binlog puller", zap.Any("pos", s.checkpoint.GlobalPoint())) + s.tctx.L().Info("reset replication binlog puller", zap.Any("pos", s.checkpoint.GlobalPoint())) if err = maybeSkipNRowsEvent(eventIndex); err != nil { return err } @@ -1887,7 +1893,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // try to re-sync in gtid mode if tryReSync && s.cfg.EnableGTID && utils.IsErrBinlogPurged(err) && s.cfg.AutoFixGTID { time.Sleep(retryTimeout) - err = s.reSyncBinlog(*tctx, lastLocation) + err = s.reSyncBinlog(*s.runCtx, lastLocation) if err != nil { return err } @@ -1900,7 +1906,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { failpoint.Inject("IgnoreSomeTypeEvent", func(val failpoint.Value) { if e.Header.EventType.String() == val.(string) { - tctx.L().Debug("IgnoreSomeTypeEvent", zap.Reflect("event", e)) + s.tctx.L().Debug("IgnoreSomeTypeEvent", zap.Reflect("event", e)) failpoint.Continue() } }) @@ -1914,7 +1920,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { metrics.BinlogPosGauge.WithLabelValues("syncer", s.cfg.Name, s.cfg.SourceID).Set(float64(e.Header.LogPos)) index, err := binlog.GetFilenameIndex(lastLocation.Position.Name) if err != nil { - tctx.L().Warn("fail to get index number of binlog file, may because only specify GTID and hasn't saved according binlog position", log.ShortError(err)) + s.tctx.L().Warn("fail to get index number of binlog file, may because only specify GTID and hasn't saved according binlog position", log.ShortError(err)) } else { metrics.BinlogFileGauge.WithLabelValues("syncer", s.cfg.Name, s.cfg.SourceID).Set(float64(index)) } @@ -1923,7 +1929,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { failpoint.Inject("ProcessBinlogSlowDown", nil) - tctx.L().Debug("receive binlog event", zap.Reflect("header", e.Header)) + s.tctx.L().Debug("receive binlog event", zap.Reflect("header", e.Header)) // support QueryEvent and RowsEvent // we calculate startLocation and endLocation(currentLocation) for Query event here @@ -1970,7 +1976,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } else if op == pb.ErrorOp_Skip { queryEvent := ev.(*replication.QueryEvent) ec := eventContext{ - tctx: tctx, + tctx: s.tctx, header: e.Header, startLocation: &startLocation, currentLocation: ¤tLocation, @@ -1979,7 +1985,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { var sourceTbls map[string]map[string]struct{} sourceTbls, err = s.trackOriginDDL(queryEvent, ec) if err != nil { - tctx.L().Warn("failed to track query when handle-error skip", zap.Error(err), zap.ByteString("sql", queryEvent.Query)) + s.tctx.L().Warn("failed to track query when handle-error skip", zap.Error(err), zap.ByteString("sql", queryEvent.Query)) } s.saveGlobalPoint(currentLocation) @@ -1993,9 +1999,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } err = s.flushJobs() if err != nil { - tctx.L().Warn("failed to flush jobs when handle-error skip", zap.Error(err)) + s.tctx.L().Warn("failed to flush jobs when handle-error skip", zap.Error(err)) } else { - tctx.L().Info("flush jobs when handle-error skip") + s.tctx.L().Info("flush jobs when handle-error skip") } } // skip the current event @@ -2009,7 +2015,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { s.locations.reset(currentLocation) if !s.errOperatorHolder.IsInject(startLocation) { // replace operator need redirect to currentLocation - if err = s.streamerController.RedirectStreamer(tctx, currentLocation); err != nil { + if err = s.streamerController.RedirectStreamer(s.runCtx, currentLocation); err != nil { return err } } @@ -2028,17 +2034,17 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // 2. push forward and replicate some sqls after safeModeExitPoint to downstream // 3. quit because of network error, fail to flush global checkpoint and new safeModeExitPoint to downstream // 4. restart again, quit safe mode at safeModeExitPoint, but some sqls after this location have already been replicated to the downstream - if err = s.checkpoint.FlushSafeModeExitPoint(s.tctx); err != nil { + if err = s.checkpoint.FlushSafeModeExitPoint(s.runCtx); err != nil { return err } - if err = s.safeMode.Add(tctx, -1); err != nil { + if err = s.safeMode.Add(s.runCtx, -1); err != nil { return err } } } ec := eventContext{ - tctx: tctx, + tctx: s.runCtx, header: e.Header, startLocation: &startLocation, currentLocation: ¤tLocation, @@ -2079,7 +2085,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // only need compare binlog position? lastLocation = shardingReSync.currLocation if binlog.CompareLocation(shardingReSync.currLocation, shardingReSync.latestLocation, s.cfg.EnableGTID) >= 0 { - tctx.L().Info("re-replicate shard group was completed", zap.String("event", "XID"), zap.Stringer("re-shard", shardingReSync)) + s.tctx.L().Info("re-replicate shard group was completed", zap.String("event", "XID"), zap.Stringer("re-shard", shardingReSync)) err = closeShardingResync() if err != nil { return terror.Annotatef(err, "shard group current location %s", shardingReSync.currLocation) @@ -2094,7 +2100,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return terror.Annotatef(err, "fail to record GTID %v", ev.GSet) } - tctx.L().Debug("", zap.String("event", "XID"), zap.Stringer("last location", lastLocation), log.WrapStringerField("location", currentLocation)) + s.tctx.L().Debug("", zap.String("event", "XID"), zap.Stringer("last location", lastLocation), log.WrapStringerField("location", currentLocation)) lastLocation.Position.Pos = e.Header.LogPos // update lastPos err = lastLocation.SetGTID(ev.GSet) if err != nil { @@ -2107,7 +2113,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if e.Header.EventType == replication.HEARTBEAT_EVENT { // flush checkpoint even if there are no real binlog events if s.checkpoint.CheckGlobalPoint() { - tctx.L().Info("meet heartbeat event and then flush jobs") + s.tctx.L().Info("meet heartbeat event and then flush jobs") err2 = s.flushJobs() } } @@ -2123,6 +2129,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } } if waitXIDStatus(s.waitXIDJob.Load()) == waitComplete { + // already wait until XID event, we can stop sync now, s.runcancel will be called in defer func return nil } } @@ -2508,7 +2515,7 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext, o qec.p, err = event.GetParserForStatusVars(ev.StatusVars) if err != nil { - log.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) + s.tctx.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) } stmt, err := parseOneStmt(qec) @@ -3119,7 +3126,7 @@ func (s *Syncer) trackOriginDDL(ev *replication.QueryEvent, ec eventContext) (ma } qec.p, err = event.GetParserForStatusVars(ev.StatusVars) if err != nil { - log.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) + s.tctx.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) } stmt, err := parseOneStmt(qec) if err != nil { @@ -3386,45 +3393,47 @@ func (s *Syncer) Close() { if s.isClosed() { return } - s.stopSync() s.closeDBs() - s.checkpoint.Close() - if err := s.schemaTracker.Close(); err != nil { s.tctx.L().Error("fail to close schema tracker", log.ShortError(err)) } - if s.sgk != nil { s.sgk.Close() } - s.closeOnlineDDL() - // when closing syncer by `stop-task`, remove active relay log from hub s.removeActiveRelayLog() - metrics.RemoveLabelValuesWithTaskInMetrics(s.cfg.Name) - + s.runWg.Wait() s.closed.Store(true) } -// stopSync stops syncing, now it used by Close and Pause -// maybe we can refine the workflow more clear. -func (s *Syncer) stopSync() { - if s.done != nil { - <-s.done // wait Run to return - } - s.closeJobChans() - s.wg.Wait() // wait job workers to return +// Kill kill syncer without graceful. +func (s *Syncer) Kill() { + s.tctx.L().Warn("kill syncer without graceful") + s.runCancel() + s.syncCancel() + s.Close() +} +// stopSync stops stream and rollbacks checkpoint now it used by Close() and Pause(). +func (s *Syncer) stopSync() { // before re-write workflow for s.syncer, simply close it // when resuming, re-create s.syncer if s.streamerController != nil { s.streamerController.Close() } + + // try to rollback checkpoints, if they already flushed, no effect, this operation should call before close schemaTracker + prePos := s.checkpoint.GlobalPoint() + s.checkpoint.Rollback(s.schemaTracker) + currPos := s.checkpoint.GlobalPoint() + if binlog.CompareLocation(prePos, currPos, s.cfg.EnableGTID) != 0 { + s.tctx.L().Warn("something wrong with rollback global checkpoint", zap.Stringer("previous position", prePos), zap.Stringer("current position", currPos)) + } } func (s *Syncer) closeOnlineDDL() { diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index c555fb766c6..34b29e94266 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/sqlmodel" + "github.com/stretchr/testify/require" sqlmock "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" @@ -103,12 +104,16 @@ type testSyncerSuite struct { } type MockStreamer struct { - events []*replication.BinlogEvent - idx uint32 + events []*replication.BinlogEvent + idx uint32 + pending bool } func (m *MockStreamer) GetEvent(ctx context.Context) (*replication.BinlogEvent, error) { if int(m.idx) >= len(m.events) { + if m.pending { + <-ctx.Done() + } return nil, context.Canceled } e := m.events[m.idx] @@ -122,7 +127,7 @@ type MockStreamProducer struct { func (mp *MockStreamProducer) generateStreamer(location binlog.Location) (reader.Streamer, error) { if location.Position.Pos == 4 { - return &MockStreamer{mp.events, 0}, nil + return &MockStreamer{mp.events, 0, false}, nil } bytesLen := 0 idx := uint32(0) @@ -133,32 +138,11 @@ func (mp *MockStreamProducer) generateStreamer(location binlog.Location) (reader break } } - return &MockStreamer{mp.events, idx}, nil + return &MockStreamer{mp.events, idx, false}, nil } func (s *testSyncerSuite) SetUpSuite(c *C) { - loaderDir, err := os.MkdirTemp("", "loader") - c.Assert(err, IsNil) - loaderCfg := config.LoaderConfig{ - Dir: loaderDir, - } - s.cfg = &config.SubTaskConfig{ - From: config.GetDBConfigForTest(), - To: config.GetDBConfigForTest(), - ServerID: 101, - MetaSchema: "test", - Name: "syncer_ut", - ShadowTableRules: []string{config.DefaultShadowTableRules}, - TrashTableRules: []string{config.DefaultTrashTableRules}, - Mode: config.ModeIncrement, - Flavor: "mysql", - LoaderConfig: loaderCfg, - } - s.cfg.Experimental.AsyncCheckpointFlush = true - s.cfg.From.Adjust() - s.cfg.To.Adjust() - - s.cfg.UseRelay = false + s.cfg = genDefaultSubTaskConfig4Test() s.resetEventsGenerator(c) c.Assert(log.InitLogger(&log.Config{}), IsNil) } @@ -237,7 +221,7 @@ func (s *testSyncerSuite) TearDownSuite(c *C) { os.RemoveAll(s.cfg.Dir) } -func (s *testSyncerSuite) mockGetServerUnixTS(mock sqlmock.Sqlmock) { +func mockGetServerUnixTS(mock sqlmock.Sqlmock) { ts := time.Now().Unix() rows := sqlmock.NewRows([]string{"UNIX_TIMESTAMP()"}).AddRow(strconv.FormatInt(ts, 10)) mock.ExpectQuery("SELECT UNIX_TIMESTAMP()").WillReturnRows(rows) @@ -742,14 +726,14 @@ func (s *testSyncerSuite) TestcheckpointID(c *C) { func (s *testSyncerSuite) TestRun(c *C) { // 1. run syncer with column mapping - // 2. execute some sqls which will trigger casuality + // 2. execute some sqls which will trigger causality // 3. check the generated jobs // 4. update config, add route rules, and update syncer - // 5. execute somes sqls and then check jobs generated + // 5. execute some sqls and then check jobs generated db, mock, err := sqlmock.New() c.Assert(err, IsNil) - s.mockGetServerUnixTS(mock) + mockGetServerUnixTS(mock) dbConn, err := db.Conn(context.Background()) c.Assert(err, IsNil) checkPointDB, checkPointMock, err := sqlmock.New() @@ -803,7 +787,7 @@ func (s *testSyncerSuite) TestRun(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("t_1", "create table t_1(id int primary key, name varchar(24), KEY `index1` (`name`))")) - s.mockGetServerUnixTS(mock) + mockGetServerUnixTS(mock) mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_2`").WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("t_2", "create table t_2(id int primary key, name varchar(24))")) @@ -1041,7 +1025,7 @@ func (s *testSyncerSuite) TestRun(c *C) { func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { db, mock, err := sqlmock.New() c.Assert(err, IsNil) - s.mockGetServerUnixTS(mock) + mockGetServerUnixTS(mock) dbConn, err := db.Conn(context.Background()) c.Assert(err, IsNil) @@ -1744,3 +1728,84 @@ func (s *testSyncerSuite) TestExecuteSQLSWithIgnore(c *C) { c.Assert(mock.ExpectationsWereMet(), IsNil) } + +func genDefaultSubTaskConfig4Test() *config.SubTaskConfig { + loaderDir, err := os.MkdirTemp("", "loader") + if err != nil { + panic(err) // no happen + } + + loaderCfg := config.LoaderConfig{ + Dir: loaderDir, + } + cfg := &config.SubTaskConfig{ + From: config.GetDBConfigForTest(), + To: config.GetDBConfigForTest(), + ServerID: 101, + MetaSchema: "test", + Name: "syncer_ut", + ShadowTableRules: []string{config.DefaultShadowTableRules}, + TrashTableRules: []string{config.DefaultTrashTableRules}, + Mode: config.ModeIncrement, + Flavor: "mysql", + LoaderConfig: loaderCfg, + UseRelay: false, + } + cfg.Experimental.AsyncCheckpointFlush = true + cfg.From.Adjust() + cfg.To.Adjust() + return cfg +} + +func TestWaitBeforeRunExit(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + cfg := genDefaultSubTaskConfig4Test() + cfg.WorkerCount = 0 + syncer := NewSyncer(cfg, nil, nil) + + db, mock, err := sqlmock.New() + require.NoError(t, err) + mockGetServerUnixTS(mock) + + syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db)} + syncer.reset() + require.NoError(t, syncer.genRouter()) + + mockStreamerProducer := &MockStreamProducer{} + mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) + require.NoError(t, err) + // let getEvent pending until ctx.Done() + mockStreamer.(*MockStreamer).pending = true + syncer.streamerController = &StreamerController{ + streamerProducer: mockStreamerProducer, streamer: mockStreamer, closed: false, + } + + wg := &sync.WaitGroup{} + errCh := make(chan error, 1) + wg.Add(1) + go func() { + defer wg.Done() + errCh <- syncer.Run(ctx) + }() + time.Sleep(time.Second) // wait s.Run start + + // test s.Run will not exit unit caller cancel ctx or call s.runCancel + cancel() // this will make s.Run exit + wg.Wait() + require.Nil(t, <-errCh) + require.Equal(t, 0, len(errCh)) + require.NotNil(t, syncer.runCtx) + require.NotNil(t, syncer.runCancel) + + // test syncer wait time not more than maxPauseOrStopWaitTime + oldMaxPauseOrStopWaitTime := maxPauseOrStopWaitTime + maxPauseOrStopWaitTime = time.Second + ctx2, cancel := context.WithCancel(context.Background()) + cancel() + runCtx, runCancel := context.WithCancel(context.Background()) + syncer.runCtx, syncer.runCancel = tcontext.NewContext(runCtx, syncer.tctx.L()), runCancel + syncer.runWg.Add(1) + syncer.waitBeforeRunExit(ctx2) + require.Equal(t, context.Canceled, syncer.runCtx.Ctx.Err()) + maxPauseOrStopWaitTime = oldMaxPauseOrStopWaitTime +} diff --git a/dm/tests/_utils/ha_cases_lib.sh b/dm/tests/_utils/ha_cases_lib.sh index d9fb774a90b..6a0a2896ed8 100644 --- a/dm/tests/_utils/ha_cases_lib.sh +++ b/dm/tests/_utils/ha_cases_lib.sh @@ -145,6 +145,7 @@ function start_multi_tasks_cluster() { } function cleanup() { + cleanup_process $* cleanup_data $ha_test cleanup_data $ha_test2 echo "clean source table" @@ -154,7 +155,6 @@ function cleanup() { $(mysql -h127.0.0.1 -p123456 -P${i} -uroot -e "drop database if exists ha_test2;") sleep 1 done - cleanup_process $* } function isolate_master() { diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 3d43bc83262..e3f23df4231 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -142,8 +142,8 @@ function test_query_timeout() { run_sql_tidb 'SHOW PROCESSLIST;' check_rows_equal 1 - cleanup_data all_mode cleanup_process + cleanup_data all_mode export GO_FAILPOINTS='' echo "[$(date)] <<<<<< finish test_query_timeout >>>>>>" @@ -210,8 +210,8 @@ function test_stop_task_before_checkpoint() { "stop-task test" \ "\"result\": true" 3 - cleanup_data all_mode cleanup_process + cleanup_data all_mode export GO_FAILPOINTS='' echo "[$(date)] <<<<<< finish test_stop_task_before_checkpoint >>>>>>" @@ -270,8 +270,8 @@ function test_fail_job_between_event() { "\"result\": true" 3 check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - cleanup_data all_mode cleanup_process + cleanup_data all_mode export GO_FAILPOINTS='' echo "[$(date)] <<<<<< finish test_fail_job_between_event >>>>>>" @@ -317,8 +317,8 @@ function test_expression_filter() { "query-status test" \ "\"result\": true" 3 - cleanup_data all_mode cleanup_process + cleanup_data all_mode echo "[$(date)] <<<<<< finish test_expression_filter >>>>>>" } diff --git a/dm/tests/checkpoint_transaction/conf/dm-worker1.toml b/dm/tests/checkpoint_transaction/conf/dm-worker1.toml index 7a72ea72bf8..3d99321d632 100644 --- a/dm/tests/checkpoint_transaction/conf/dm-worker1.toml +++ b/dm/tests/checkpoint_transaction/conf/dm-worker1.toml @@ -1,2 +1,4 @@ -name = "worker1" join = "127.0.0.1:8261" +keepalive-ttl = 1 +name = "worker1" +worker-addr = "127.0.0.1:8262" diff --git a/dm/tests/checkpoint_transaction/run.sh b/dm/tests/checkpoint_transaction/run.sh index 5771945384a..e3834469245 100755 --- a/dm/tests/checkpoint_transaction/run.sh +++ b/dm/tests/checkpoint_transaction/run.sh @@ -6,6 +6,31 @@ cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME +function check_worker_ungraceful_stop_with_retry() { + for ((k = 0; k < 10; k++)); do + sleep 1 + echo "start check_worker_ungraceful_stop_with_retry times: $k" + + num=$(grep "kill unit" $WORK_DIR/worker1/log/dm-worker.log | wc -l) + if [ $num -lt 1 ]; then + continue + fi + num=$(grep "kill syncer without graceful" $WORK_DIR/worker1/log/dm-worker.log | wc -l) + if [ $num -lt 1 ]; then + continue + fi + num=$(grep "received ungraceful exit ctx, exit now" $WORK_DIR/worker1/log/dm-worker.log | wc -l) + if [ $num -lt 1 ]; then + continue + fi + echo "check_worker_ungraceful_stop_with_retry success after retry: $k" + return 0 + done + + echo "check_worker_ungraceful_stop_with_retry failed after retry" + exit 1 +} + function run() { export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/checkCheckpointInMiddleOfTransaction=return" @@ -28,6 +53,33 @@ function run() { # check diff check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + # test ungraceful stop, worker will not wait transaction finish + run_sql_file $cur/data/db1.increment1.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + sleep 2 + # kill dm-master 1 to make worker lost keep alive while a transaction is not finished + echo "kill dm-master1" + kill_dm_master + check_master_port_offline 1 + sleep 1 # wait worker lost keep alive ttl is 1 second + + # check dm-worker will exit quickly without waiting for the transaction to finish + check_worker_ungraceful_stop_with_retry + + # test data in tidb less than source + dataCountSource=$(mysql -uroot -h$MYSQL_HOST1 -P$MYSQL_PORT1 -p$MYSQL_PASSWORD1 -se "select count(1) from checkpoint_transaction.t1") + dataCountInTiDB=$(mysql -uroot -h127.0.0.1 -P4000 -se "select count(1) from checkpoint_transaction.t1") + echo "after ungraceful exit data in source count: $dataCountSource data in tidb count: $dataCountInTiDB" + if [ "$dataCountInTiDB" -lt "$dataCountSource" ]; then + echo "ungraceful stop test success" + else + echo "ungraceful stop test failed" + exit 1 + fi + + # start dm-master again task will be resume, and data will be synced + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_sql_file $cur/data/db1.increment1.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 # wait transaction start # you can see why sleep in https://github.com/pingcap/dm/pull/1928#issuecomment-895820239 @@ -41,8 +93,13 @@ function run() { "\"stage\": \"Paused\"" 1 # check the point is the middle of checkpoint num=$(grep "not receive xid job yet" $WORK_DIR/worker1/log/dm-worker.log | wc -l) - [[ $num -gt 0 ]] - sed -e '/not receive xid job yet/d' $WORK_DIR/worker1/log/dm-worker.log >$WORK_DIR/worker1/log/dm-worker.log + + if [ "$num" -gt 0 ]; then + echo "graceful pause test success" + else + echo "graceful pause test failed" + exit 1 + fi echo "start check pause diff" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml @@ -55,6 +112,16 @@ function run() { "query-status test" \ "\"stage\": \"Running\"" 1 + echo "kill dm-worker1" + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + rm -rf $WORK_DIR/worker1 + 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 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"stage\": \"Running\"" 1 + run_sql_file $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 # wait transaction start # you can see why sleep in https://github.com/pingcap/dm/pull/1928#issuecomment-895820239 @@ -65,7 +132,12 @@ function run() { "\"result\": true" 2 # check the point is the middle of checkpoint num=$(grep "not receive xid job yet" $WORK_DIR/worker1/log/dm-worker.log | wc -l) - [[ $num -gt 0 ]] + if [ "$num" -gt 0 ]; then + echo "graceful stop test success" + else + echo "graceful stop test failed" + exit 1 + fi echo "start check stop diff" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml diff --git a/dm/tests/expression_filter/run.sh b/dm/tests/expression_filter/run.sh index 01b559ccba9..faa7d1d43db 100755 --- a/dm/tests/expression_filter/run.sh +++ b/dm/tests/expression_filter/run.sh @@ -53,8 +53,8 @@ function complex_behaviour() { update_num=$(grep -o '"number of filtered update"=[0-9]\+' $WORK_DIR/worker1/log/dm-worker.log | grep -o '[0-9]\+' | awk '{n += $1}; END{print n}') [ $update_num -eq 3 ] - cleanup_data expr_filter cleanup_process $* + cleanup_data expr_filter } function run() { diff --git a/dm/tests/full_mode/run.sh b/dm/tests/full_mode/run.sh index 80fd8975c52..b74e7599a03 100755 --- a/dm/tests/full_mode/run.sh +++ b/dm/tests/full_mode/run.sh @@ -55,8 +55,8 @@ function fail_acquire_global_lock() { "\"stage\": \"Paused\"" 2 \ "you need (at least one of) the RELOAD privilege(s) for this operation" 2 - cleanup_data full_mode cleanup_process $* + cleanup_data full_mode } function escape_schema() { @@ -113,8 +113,8 @@ function escape_schema() { check_metric $WORKER1_PORT 'dumpling_dump_finished_tables' 3 0 3 check_metric $WORKER2_PORT 'dumpling_dump_finished_tables' 3 0 3 - cleanup_data full/mode cleanup_process $* + cleanup_data full/mode } function empty_data() { @@ -138,8 +138,8 @@ function empty_data() { check_log_contains $WORK_DIR/worker1/log/dm-worker.log "progress=\"100.00 %\"" check_log_contains $WORK_DIR/worker2/log/dm-worker.log "progress=\"100.00 %\"" - cleanup_data full_mode cleanup_process $* + cleanup_data full_mode } function run() { diff --git a/dm/tests/load_interrupt/run.sh b/dm/tests/load_interrupt/run.sh index 79f8c79c44c..e0e96aebb9c 100755 --- a/dm/tests/load_interrupt/run.sh +++ b/dm/tests/load_interrupt/run.sh @@ -60,8 +60,8 @@ function test_save_checkpoint_failed() { ls $WORK_DIR/worker1/dumped_data.test echo "test_save_checkpoint_failed SUCCESS!" - cleanup_data load_interrupt cleanup_process $* + cleanup_data load_interrupt } function run() { diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index f9beb3a4c7f..b359bd0c606 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -10,82 +10,10 @@ SQL_RESULT_FILE="$TEST_DIR/sql_res.$TEST_NAME.txt" API_VERSION="v1alpha1" -function test_cant_dail_upstream() { - cleanup_data $TEST_NAME - cleanup_process - - run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml - check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT - 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 - - cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml - dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 - - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-relay -s $SOURCE_ID1 worker1" \ - "\"result\": true" 2 - - kill_dm_worker - - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/conn/failDBPing=return()" - 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 - - # make sure DM-worker doesn't exit - sleep 2 - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status -s $SOURCE_ID1" \ - "injected error" 1 - - export GO_FAILPOINTS="" - cleanup_process - cleanup_data $TEST_NAME -} - -function test_cant_dail_downstream() { - cleanup_data $TEST_NAME - cleanup_process - - run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml - check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT - 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 - - cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml - dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 - - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-relay -s $SOURCE_ID1 worker1" \ - "\"result\": true" 2 - dmctl_start_task_standalone $cur/conf/dm-task.yaml "--remove-meta" - - kill_dm_worker - # kill tidb - pkill -hup tidb-server 2>/dev/null || true - wait_process_exit tidb-server - - 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 - - # make sure DM-worker doesn't exit - sleep 2 - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status -s $SOURCE_ID1" \ - "\"relayCatchUpMaster\": true" 1 \ - "dial tcp 127.0.0.1:4000: connect: connection refused" 1 - - # restart tidb - run_tidb_server 4000 $TIDB_PASSWORD - sleep 2 - - cleanup_process - cleanup_data $TEST_NAME -} - function test_restart_relay_status() { - cleanup_data $TEST_NAME cleanup_process + cleanup_data $TEST_NAME + export GO_FAILPOINTS="" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT @@ -157,11 +85,83 @@ function test_restart_relay_status() { "list-member --worker" \ "relay" 1 \ "bound" 2 + + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>test test_restart_relay_status passed" } -function test_kill_dump_connection() { +function test_cant_dail_upstream() { + cleanup_process cleanup_data $TEST_NAME + export GO_FAILPOINTS="" + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + 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 + + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 2 + + echo "kill dm-worker1" + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/conn/failDBPing=return()" + 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 + + # make sure DM-worker doesn't exit + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "injected error" 1 + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>test test_cant_dail_upstream passed" +} + +function test_cant_dail_downstream() { + cleanup_process + cleanup_data $TEST_NAME + export GO_FAILPOINTS="" + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + 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 + + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 2 + dmctl_start_task_standalone $cur/conf/dm-task.yaml "--remove-meta" + + echo "kill dm-worker1" + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + # kill tidb + pkill -hup tidb-server 2>/dev/null || true + wait_process_exit tidb-server + + 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 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"relayCatchUpMaster\": true" 1 \ + "dial tcp 127.0.0.1:4000: connect: connection refused" 1 + + # restart tidb + run_tidb_server 4000 $TIDB_PASSWORD + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>test test_cant_dail_downstream passed" +} + +function test_kill_dump_connection() { cleanup_process + cleanup_data $TEST_NAME run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' @@ -193,16 +193,12 @@ function test_kill_dump_connection() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ "\"relayCatchUpMaster\": true" 1 + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>test test_kill_dump_connection passed" +} +function test_relay_operations() { cleanup_process cleanup_data $TEST_NAME -} - -function run() { - test_restart_relay_status - test_cant_dail_downstream - test_cant_dail_upstream - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/relay/ReportRelayLogSpaceInBackground=return(1)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 @@ -234,7 +230,7 @@ function run() { "\"worker\": \"worker1\"" 1 \ "\"worker\": \"worker2\"" 1 - # worker1 and worker2 has one realy job and worker3 have none. + # worker1 and worker2 has one relay job and worker3 have none. check_metric $WORKER1_PORT "dm_relay_binlog_file{node=\"relay\"}" 3 0 2 check_metric $WORKER1_PORT "dm_relay_exit_with_error_count" 3 -1 1 check_metric $WORKER2_PORT "dm_relay_binlog_file{node=\"relay\"}" 3 0 2 @@ -246,7 +242,7 @@ function run() { run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 - # relay task tranfer to worker1 with no error. + # relay task transfer to worker1 with no error. check_metric $WORKER1_PORT "dm_relay_data_corruption" 3 -1 1 check_metric $WORKER1_PORT "dm_relay_read_error_count" 3 -1 1 check_metric $WORKER1_PORT "dm_relay_write_error_count" 3 -1 1 @@ -254,8 +250,9 @@ function run() { check_metric $WORKER1_PORT 'dm_relay_space{type="available"}' 5 0 9223372036854775807 # subtask is preferred to scheduled to another relay worker - pkill -hup -f dm-worker1.toml 2>/dev/null || true - wait_pattern_exit dm-worker1.toml + echo "kill dm-worker1" + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 # worker1 is down, worker2 has running relay and sync unit run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ @@ -296,11 +293,12 @@ function run() { [ "$new_relay_log_count_1" -eq 1 ] [ "$new_relay_log_count_2" -eq 1 ] - pkill -hup -f dm-worker1.toml 2>/dev/null || true - wait_pattern_exit dm-worker1.toml - pkill -hup -f dm-worker2.toml 2>/dev/null || true - wait_pattern_exit dm-worker2.toml - + echo "kill dm-worker1" + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + echo "kill dm-worker2" + ps aux | grep dm-worker2 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 # if all relay workers are offline, relay-not-enabled worker should continue to sync run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status -s $SOURCE_ID1" \ @@ -322,8 +320,7 @@ function run() { # destroy cluster cleanup_process $* - rm -rf $WORK_DIR - mkdir $WORK_DIR + cleanup_data $TEST_NAME # insert new data run_sql_file $cur/data/db1.increment5.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 @@ -352,7 +349,14 @@ function run() { "\"result\": true" 2 check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>test test_relay_operations passed" +} +function run() { + test_relay_operations + test_cant_dail_upstream + test_restart_relay_status + test_cant_dail_downstream test_kill_dump_connection } diff --git a/dm/tests/safe_mode/run.sh b/dm/tests/safe_mode/run.sh index 644967cd306..ed1d8ff74ff 100755 --- a/dm/tests/safe_mode/run.sh +++ b/dm/tests/safe_mode/run.sh @@ -55,8 +55,8 @@ function consistency_none() { check_log_contain_with_retry "\[\"enable safe-mode for safe mode exit point, will exit at\"\] \[task=test\] \[unit=\"binlog replication\"\] \[location=\"position: ($name2, $pos2), gtid-set: $gtid2\"\]" $WORK_DIR/worker2/log/dm-worker.log run_sql_source2 "SET @@GLOBAL.SQL_MODE='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'" - cleanup_data safe_mode_target cleanup_process $* + cleanup_data safe_mode_target } function check_exit_safe_binlog() { @@ -182,8 +182,8 @@ function safe_mode_recover() { echo "finish running run safe mode recover case $i" ((i += 1)) - cleanup_data safe_mode_target cleanup_process $* + cleanup_data safe_mode_target done } diff --git a/dm/tests/start_task/run.sh b/dm/tests/start_task/run.sh index c725983852f..49b3a5519f2 100644 --- a/dm/tests/start_task/run.sh +++ b/dm/tests/start_task/run.sh @@ -58,8 +58,8 @@ function lazy_init_tracker() { check_log_contains $WORK_DIR/worker1/log/dm-worker.log 'lazy init table info.*t50' 1 check_log_not_contains $WORK_DIR/worker1/log/dm-worker.log 'lazy init table info.*t51' - cleanup_data start_task cleanup_process + cleanup_data start_task } function start_task_by_time() { From 1c3bf688f62ba1c8b017b510ec84fd6aeed688de Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Wed, 16 Feb 2022 20:39:39 +0800 Subject: [PATCH 70/72] cdc/sink: decouple opt out of statistics (#4606) close pingcap/tiflow#4607 --- cdc/sink/black_hole.go | 4 ++-- cdc/sink/buffer_sink_test.go | 4 ++-- cdc/sink/mq.go | 2 +- cdc/sink/mysql.go | 2 +- cdc/sink/mysql_test.go | 2 +- cdc/sink/producer/kafka/kafka.go | 4 +++- cdc/sink/sink.go | 2 +- cdc/sink/statistics.go | 13 ++++++------- 8 files changed, 17 insertions(+), 16 deletions(-) diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index e4ba39ba7a1..7d17caaca23 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -23,9 +23,9 @@ import ( ) // newBlackHoleSink creates a black hole sink -func newBlackHoleSink(ctx context.Context, opts map[string]string) *blackHoleSink { +func newBlackHoleSink(ctx context.Context) *blackHoleSink { return &blackHoleSink{ - statistics: NewStatistics(ctx, "blackhole", opts), + statistics: NewStatistics(ctx, "blackhole"), } } diff --git a/cdc/sink/buffer_sink_test.go b/cdc/sink/buffer_sink_test.go index 2b9f9c2d86f..043b4d78101 100644 --- a/cdc/sink/buffer_sink_test.go +++ b/cdc/sink/buffer_sink_test.go @@ -38,7 +38,7 @@ func TestFlushTable(t *testing.T) { ctx, cancel := context.WithCancel(context.TODO()) defer cancel() - b := newBufferSink(newBlackHoleSink(ctx, make(map[string]string)), 5, make(chan drawbackMsg)) + b := newBufferSink(newBlackHoleSink(ctx), 5, make(chan drawbackMsg)) go b.run(ctx, make(chan error)) require.Equal(t, uint64(5), b.getTableCheckpointTs(2)) @@ -82,7 +82,7 @@ func TestFlushFailed(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.TODO()) - b := newBufferSink(newBlackHoleSink(ctx, make(map[string]string)), 5, make(chan drawbackMsg)) + b := newBufferSink(newBlackHoleSink(ctx), 5, make(chan drawbackMsg)) go b.run(ctx, make(chan error)) checkpoint, err := b.FlushRowChangedEvents(ctx, 3, 8) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 2071097ad1a..fca2772aa8c 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -135,7 +135,7 @@ func newMqSink( resolvedNotifier: notifier, resolvedReceiver: resolvedReceiver, - statistics: NewStatistics(ctx, "MQ", opts), + statistics: NewStatistics(ctx, "MQ"), role: role, id: changefeedID, diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 7b81d0a24b0..29c8367902b 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -185,7 +185,7 @@ func newMySQLSink( filter: filter, cyclic: sinkCyclic, txnCache: common.NewUnresolvedTxnCache(), - statistics: NewStatistics(ctx, "mysql", opts), + statistics: NewStatistics(ctx, "mysql"), metricConflictDetectDurationHis: metricConflictDetectDurationHis, metricBucketSizeCounters: metricBucketSizeCounters, errCh: make(chan error, 1), diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index aa69c25da23..1374710220c 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -49,7 +49,7 @@ func newMySQLSink4Test(ctx context.Context, t *testing.T) *mysqlSink { return &mysqlSink{ txnCache: common.NewUnresolvedTxnCache(), filter: f, - statistics: NewStatistics(ctx, "test", make(map[string]string)), + statistics: NewStatistics(ctx, "test"), params: params, } } diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 2888587c029..f0498e66c46 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -263,6 +263,9 @@ func (k *kafkaSaramaProducer) Close() error { zap.String("changefeed", k.id), zap.Any("role", k.role)) } + k.metricsMonitor.Cleanup() + + // adminClient should be closed last, since `metricsMonitor` would use it when `Cleanup`. start = time.Now() if err := k.admin.Close(); err != nil { log.Warn("close kafka cluster admin with error", zap.Error(err), @@ -273,7 +276,6 @@ func (k *kafkaSaramaProducer) Close() error { zap.String("changefeed", k.id), zap.Any("role", k.role)) } - k.metricsMonitor.Cleanup() return nil } diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index 39e81767fb2..ae28c2cd2b5 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -89,7 +89,7 @@ func init() { // register blackhole sink sinkIniterMap["blackhole"] = func(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI *url.URL, filter *filter.Filter, config *config.ReplicaConfig, opts map[string]string, errCh chan error) (Sink, error) { - return newBlackHoleSink(ctx, opts), nil + return newBlackHoleSink(ctx), nil } // register mysql sink diff --git a/cdc/sink/statistics.go b/cdc/sink/statistics.go index 26c1bad7fb4..c12931e714e 100644 --- a/cdc/sink/statistics.go +++ b/cdc/sink/statistics.go @@ -30,13 +30,12 @@ const ( ) // NewStatistics creates a statistics -func NewStatistics(ctx context.Context, name string, opts map[string]string) *Statistics { - statistics := &Statistics{name: name, lastPrintStatusTime: time.Now()} - if cid, ok := opts[OptChangefeedID]; ok { - statistics.changefeedID = cid - } - if cid, ok := opts[OptCaptureAddr]; ok { - statistics.captureAddr = cid +func NewStatistics(ctx context.Context, name string) *Statistics { + statistics := &Statistics{ + name: name, + captureAddr: util.CaptureAddrFromCtx(ctx), + changefeedID: util.ChangefeedIDFromCtx(ctx), + lastPrintStatusTime: time.Now(), } statistics.metricExecTxnHis = execTxnHistogram.WithLabelValues(statistics.captureAddr, statistics.changefeedID) statistics.metricExecDDLHis = execDDLHistogram.WithLabelValues(statistics.captureAddr, statistics.changefeedID) From 2f4b70681bc3730c5e6b199a10a2a3e2728cf144 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B7=A5=E4=B8=9A=E5=BA=9F=E6=B0=B4?= Date: Wed, 16 Feb 2022 23:37:39 +0800 Subject: [PATCH 71/72] ticdc(mounter): remove unnecessary delete event decode logic (#4602) close pingcap/tiflow#4582 --- cdc/entry/mounter.go | 9 --------- 1 file changed, 9 deletions(-) diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 5418d4f8d01..0a19e20b09f 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -261,15 +261,6 @@ func (m *mounterImpl) unmarshalRowKVEntry(tableInfo *model.TableInfo, rawKey []b return nil, errors.Trace(err) } - if base.Delete && !m.enableOldValue && (tableInfo.PKIsHandle || tableInfo.IsCommonHandle) { - handleColIDs, fieldTps, _ := tableInfo.GetRowColInfos() - preRow, err = tablecodec.DecodeHandleToDatumMap(recordID, handleColIDs, fieldTps, m.tz, nil) - if err != nil { - return nil, errors.Trace(err) - } - preRowExist = true - } - base.RecordID = recordID return &rowKVEntry{ baseKVEntry: base, From 2592e989a0da88425c127cd218cf47f2ed4e0ef8 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Thu, 17 Feb 2022 11:56:42 +0800 Subject: [PATCH 72/72] revert client --- cdc/kv/client.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 8169fdcf4fa..96449cf83a0 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -513,8 +513,6 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { zap.Stringer("span", s.totalSpan), zap.Uint64("ts", ts), zap.String("changefeed", s.client.changefeed)) - ctx, cancel := context.WithCancel(ctx) - defer cancel() g, ctx := errgroup.WithContext(ctx) g.Go(func() error { @@ -528,14 +526,15 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { tableID, tableName := util.TableIDFromCtx(ctx) cfID := util.ChangefeedIDFromCtx(ctx) g.Go(func() error { - checkRegionRateLimitTicker := time.NewTicker(defaultCheckRegionRateLimitInterval) - defer checkRegionRateLimitTicker.Stop() + timer := time.NewTimer(defaultCheckRegionRateLimitInterval) + defer timer.Stop() for { select { case <-ctx.Done(): return ctx.Err() - case <-checkRegionRateLimitTicker.C: + case <-timer.C: s.handleRateLimit(ctx) + timer.Reset(defaultCheckRegionRateLimitInterval) case task := <-s.requestRangeCh: s.rangeChSizeGauge.Dec() // divideAndSendEventFeedToRegions could be block for some time, @@ -581,12 +580,13 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { } }) + g.Go(func() error { + return s.regionRouter.Run(ctx) + }) + s.requestRangeCh <- rangeRequestTask{span: s.totalSpan, ts: ts} s.rangeChSizeGauge.Inc() - err := s.regionRouter.Run(ctx) - if err != nil { - return err - } + return g.Wait() }