diff --git a/.golangci.yml b/.golangci.yml index f33f66e680c..861c668b3bc 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -3,6 +3,7 @@ linters: - unconvert - unparam - revive + - depguard linters-settings: revive: @@ -28,3 +29,14 @@ linters-settings: - name: superfluous-else - name: modifies-parameter - name: unreachable-code + + depguard: + list-type: blacklist + include-go-root: false + packages: + - log + - github.com/juju/errors + packages-with-error-message: + # specify an error message to output when a blacklisted package is used + - log: "logging is allowed only by pingcap/log" + - github.com/juju/errors: "error handling is allowed only by pingcap/errors" diff --git a/Makefile b/Makefile index 5c3bd946860..0ead040af7c 100644 --- a/Makefile +++ b/Makefile @@ -151,10 +151,10 @@ integration_test_build: check_failpoint_ctl integration_test: integration_test_mysql integration_test_mysql: - tests/run.sh mysql "$(CASE)" + tests/integration_tests/run.sh mysql "$(CASE)" integration_test_kafka: check_third_party_binary - tests/run.sh kafka "$(CASE)" + tests/integration_tests/run.sh kafka "$(CASE)" fmt: tools/bin/gofumports tools/bin/shfmt @echo "gofmt (simplify)" @@ -201,15 +201,20 @@ check-static: tools/bin/golangci-lint check: check-copyright fmt check-static tidy terror_check errdoc check-leaktest-added check-merge-conflicts -coverage: tools/bin/gocovmerge tools/bin/goveralls +integration_test_coverage: tools/bin/gocovmerge tools/bin/goveralls tools/bin/gocovmerge "$(TEST_DIR)"/cov.* | grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/entry/schema_test_helper.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" > "$(TEST_DIR)/all_cov.out" - grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" "$(TEST_DIR)/cov.unit.out" > "$(TEST_DIR)/unit_cov.out" ifeq ("$(JenkinsCI)", "1") GO111MODULE=off go get github.com/mattn/goveralls tools/bin/goveralls -coverprofile=$(TEST_DIR)/all_cov.out -service=jenkins-ci -repotoken $(COVERALLS_TOKEN) - @bash <(curl -s https://codecov.io/bash) -f $(TEST_DIR)/unit_cov.out -t $(CODECOV_TOKEN) else go tool cover -html "$(TEST_DIR)/all_cov.out" -o "$(TEST_DIR)/all_cov.html" +endif + +unit_test_coverage: + grep -vE ".*.pb.go|$(CDC_PKG)/testing_utils/.*|$(CDC_PKG)/cdc/kv/testing.go|$(CDC_PKG)/cdc/sink/simple_mysql_tester.go|.*.__failpoint_binding__.go" "$(TEST_DIR)/cov.unit.out" > "$(TEST_DIR)/unit_cov.out" +ifeq ("$(JenkinsCI)", "1") + @bash <(curl -s https://codecov.io/bash) -f $(TEST_DIR)/unit_cov.out -t $(CODECOV_TOKEN) +else go tool cover -html "$(TEST_DIR)/unit_cov.out" -o "$(TEST_DIR)/unit_cov.html" go tool cover -func="$(TEST_DIR)/unit_cov.out" endif diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 3d09c741779..7518ee0d7e1 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -33,6 +33,7 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/pdtime" "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" pd "github.com/tikv/pd/client" @@ -55,10 +56,11 @@ type Capture struct { session *concurrency.Session election *concurrency.Election - pdClient pd.Client - kvStorage tidbkv.Storage - etcdClient *etcd.CDCEtcdClient - grpcPool kv.GrpcPool + pdClient pd.Client + kvStorage tidbkv.Storage + etcdClient *etcd.CDCEtcdClient + grpcPool kv.GrpcPool + TimeAcquirer pdtime.TimeAcquirer cancel context.CancelFunc @@ -100,6 +102,12 @@ func (c *Capture) reset(ctx context.Context) error { } c.session = sess c.election = concurrency.NewElection(sess, etcd.CaptureOwnerKey) + + if c.TimeAcquirer != nil { + c.TimeAcquirer.Stop() + } + c.TimeAcquirer = pdtime.NewTimeAcquirer(c.pdClient) + if c.grpcPool != nil { c.grpcPool.Close() } @@ -148,11 +156,12 @@ func (c *Capture) Run(ctx context.Context) error { func (c *Capture) run(stdCtx context.Context) error { ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ - PDClient: c.pdClient, - KVStorage: c.kvStorage, - CaptureInfo: c.info, - EtcdClient: c.etcdClient, - GrpcPool: c.grpcPool, + PDClient: c.pdClient, + KVStorage: c.kvStorage, + CaptureInfo: c.info, + EtcdClient: c.etcdClient, + GrpcPool: c.grpcPool, + TimeAcquirer: c.TimeAcquirer, }) err := c.register(ctx) if err != nil { @@ -166,7 +175,7 @@ func (c *Capture) run(stdCtx context.Context) error { cancel() }() wg := new(sync.WaitGroup) - wg.Add(3) + wg.Add(4) var ownerErr, processorErr error go func() { defer wg.Done() @@ -188,6 +197,10 @@ func (c *Capture) run(stdCtx context.Context) error { processorErr = c.runEtcdWorker(ctx, c.processorManager, orchestrator.NewGlobalState(), processorFlushInterval) log.Info("the processor routine has exited", zap.Error(processorErr)) }() + go func() { + defer wg.Done() + c.TimeAcquirer.Run(ctx) + }() go func() { defer wg.Done() c.grpcPool.RecycleConn(ctx) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 2eaaffd016d..3bb3dfbb826 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -16,7 +16,6 @@ package owner import ( "context" "sync" - "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -49,6 +48,8 @@ type changefeed struct { sink AsyncSink ddlPuller DDLPuller initialized bool + // isRemoved is true if the changefeed is removed + isRemoved bool // only used for asyncExecDDL function // ddlEventCache is not nil when the changefeed is executing a DDL event asynchronously @@ -119,7 +120,7 @@ func (c *changefeed) Tick(ctx cdcContext.Context, state *orchestrator.Changefeed Code: code, Message: err.Error(), }) - c.releaseResources() + c.releaseResources(ctx) } } @@ -148,7 +149,8 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *orchestrator.Changefeed } if !c.feedStateManager.ShouldRunning() { - c.releaseResources() + c.isRemoved = c.feedStateManager.ShouldRemoved() + c.releaseResources(ctx) return nil } @@ -181,7 +183,9 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *orchestrator.Changefeed return errors.Trace(err) } if shouldUpdateState { - c.updateStatus(barrierTs) + pdTime, _ := ctx.GlobalVars().TimeAcquirer.CurrentTimeFromCached() + currentTs := oracle.GetPhysical(pdTime) + c.updateStatus(currentTs, barrierTs) } return nil } @@ -282,16 +286,22 @@ LOOP: return nil } -func (c *changefeed) releaseResources() { +func (c *changefeed) releaseResources(ctx context.Context) { if !c.initialized { return } log.Info("close changefeed", zap.String("changefeed", c.state.ID), - zap.Stringer("info", c.state.Info)) + zap.Stringer("info", c.state.Info), zap.Bool("isRemoved", c.isRemoved)) c.cancel() c.cancel = func() {} c.ddlPuller.Close() c.schema = nil + if c.isRemoved && c.redoManager.Enabled() { + err := c.redoManager.Cleanup(ctx) + if err != nil { + log.Error("cleanup redo logs failed", zap.String("changefeed", c.id), zap.Error(err)) + } + } ctx, cancel := context.WithCancel(context.Background()) cancel() // We don't need to wait sink Close, pass a canceled context is ok @@ -455,7 +465,7 @@ func (c *changefeed) asyncExecDDL(ctx cdcContext.Context, job *timodel.Job) (don return done, nil } -func (c *changefeed) updateStatus(barrierTs model.Ts) { +func (c *changefeed) updateStatus(currentTs int64, barrierTs model.Ts) { resolvedTs := barrierTs for _, position := range c.state.TaskPositions { if resolvedTs > position.ResolvedTs { @@ -487,14 +497,12 @@ func (c *changefeed) updateStatus(barrierTs model.Ts) { } return status, changed, nil }) - phyTs := oracle.ExtractPhysical(checkpointTs) + c.metricsChangefeedCheckpointTsGauge.Set(float64(phyTs)) - // It is more accurate to get tso from PD, but in most cases since we have - // deployed NTP service, a little bias is acceptable here. - c.metricsChangefeedCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) + c.metricsChangefeedCheckpointTsLagGauge.Set(float64(currentTs-phyTs) / 1e3) } -func (c *changefeed) Close() { - c.releaseResources() +func (c *changefeed) Close(ctx context.Context) { + c.releaseResources(ctx) } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 917918a7bc7..e3a630a4340 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/ticdc/pkg/config" cdcContext "github.com/pingcap/ticdc/pkg/context" "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/ticdc/pkg/pdtime" "github.com/pingcap/ticdc/pkg/txnutil/gc" "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/ticdc/pkg/version" @@ -166,7 +167,7 @@ func (s *changefeedSuite) TestInitialize(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(true) cf, state, captures, tester := createChangefeed4Test(ctx, c) - defer cf.Close() + defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) tester.MustApplyPatches() @@ -181,7 +182,7 @@ func (s *changefeedSuite) TestHandleError(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(true) cf, state, captures, tester := createChangefeed4Test(ctx, c) - defer cf.Close() + defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) tester.MustApplyPatches() @@ -216,6 +217,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { AdvertiseAddr: "127.0.0.1:0000", Version: version.ReleaseVersion, }, + TimeAcquirer: pdtime.NewTimeAcquirer4Test(), }) ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ ID: "changefeed-id-test", @@ -226,7 +228,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { }) cf, state, captures, tester := createChangefeed4Test(ctx, c) - defer cf.Close() + defer cf.Close(ctx) tickThreeTime := func() { cf.Tick(ctx, state, captures) tester.MustApplyPatches() @@ -298,7 +300,7 @@ func (s *changefeedSuite) TestSyncPoint(c *check.C) { ctx.ChangefeedVars().Info.SyncPointEnabled = true ctx.ChangefeedVars().Info.SyncPointInterval = 1 * time.Second cf, state, captures, tester := createChangefeed4Test(ctx, c) - defer cf.Close() + defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) @@ -329,7 +331,7 @@ func (s *changefeedSuite) TestFinished(c *check.C) { ctx := cdcContext.NewBackendContext4Test(true) ctx.ChangefeedVars().Info.TargetTs = ctx.ChangefeedVars().Info.StartTs + 1000 cf, state, captures, tester := createChangefeed4Test(ctx, c) - defer cf.Close() + defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) diff --git a/cdc/owner/feed_state_manager.go b/cdc/owner/feed_state_manager.go index a8f82d8ed2c..909cc9a2f80 100644 --- a/cdc/owner/feed_state_manager.go +++ b/cdc/owner/feed_state_manager.go @@ -29,6 +29,10 @@ import ( type feedStateManager struct { state *orchestrator.ChangefeedReactorState shouldBeRunning bool + // Based on shouldBeRunning = false + // shouldBeRemoved = true means the changefeed is removed + // shouldBeRemoved = false means the changefeed is paused + shouldBeRemoved bool adminJobQueue []*model.AdminJob } @@ -49,7 +53,11 @@ func (m *feedStateManager) Tick(state *orchestrator.ChangefeedReactorState) { return } switch m.state.Info.State { - case model.StateStopped, model.StateFailed, model.StateRemoved, model.StateFinished: + case model.StateRemoved: + m.shouldBeRunning = false + m.shouldBeRemoved = true + return + case model.StateStopped, model.StateFailed, model.StateFinished: m.shouldBeRunning = false return } @@ -61,6 +69,10 @@ func (m *feedStateManager) ShouldRunning() bool { return m.shouldBeRunning } +func (m *feedStateManager) ShouldRemoved() bool { + return m.shouldBeRemoved +} + func (m *feedStateManager) MarkFinished() { if m.state == nil { // when state is nil, it means that Tick has never been called @@ -111,6 +123,7 @@ func (m *feedStateManager) handleAdminJob() (jobsPending bool) { return } m.shouldBeRunning = false + m.shouldBeRemoved = true jobsPending = true // remove changefeedInfo @@ -123,7 +136,6 @@ func (m *feedStateManager) handleAdminJob() (jobsPending bool) { }) checkpointTs := m.state.Info.GetCheckpointTs(m.state.Status) log.Info("the changefeed is removed", zap.String("changefeed-id", m.state.ID), zap.Uint64("checkpoint-ts", checkpointTs)) - case model.AdminResume: switch m.state.Info.State { case model.StateFailed, model.StateError, model.StateStopped, model.StateFinished: diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index 54aa23c43e6..66d8c951a46 100644 --- a/cdc/owner/feed_state_manager_test.go +++ b/cdc/owner/feed_state_manager_test.go @@ -71,6 +71,7 @@ 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) @@ -83,6 +84,7 @@ 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) @@ -95,6 +97,7 @@ 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) } @@ -217,6 +220,7 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { tester.MustApplyPatches() } c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(manager.ShouldRemoved(), check.IsFalse) c.Assert(state.Info.State, check.Equals, model.StateError) c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) @@ -234,6 +238,7 @@ func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { }) manager.Tick(state) c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(manager.ShouldRemoved(), check.IsFalse) tester.MustApplyPatches() manager.PushAdminJob(&model.AdminJob{ @@ -243,6 +248,7 @@ func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { }) manager.Tick(state) c.Assert(manager.ShouldRunning(), check.IsFalse) + c.Assert(manager.ShouldRemoved(), check.IsTrue) tester.MustApplyPatches() c.Assert(state.Info, check.IsNil) c.Assert(state.Exist(), check.IsFalse) diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 96a37bbbcd4..358bfee893c 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -137,6 +137,9 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) for changefeedID, changefeedState := range state.Changefeeds { if changefeedState.Info == nil { o.cleanUpChangefeed(changefeedState) + if cfReactor, ok := o.changefeeds[changefeedID]; ok { + cfReactor.isRemoved = true + } continue } ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ @@ -155,13 +158,13 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) if _, exist := state.Changefeeds[changefeedID]; exist { continue } - cfReactor.Close() + cfReactor.Close(stdCtx) delete(o.changefeeds, changefeedID) } } if atomic.LoadInt32(&o.closed) != 0 { for _, cfReactor := range o.changefeeds { - cfReactor.Close() + cfReactor.Close(stdCtx) } return state, cerror.ErrReactorFinished.GenWithStackByArgs() } diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 6b3782fbd99..968a65ebca4 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -70,6 +70,7 @@ func (m *Manager) Tick(stdCtx context.Context, state orchestrator.ReactorState) if err := m.handleCommand(); err != nil { return state, err } + captureID := ctx.GlobalVars().CaptureInfo.ID var inactiveChangefeedCount int for changefeedID, changefeedState := range globalState.Changefeeds { diff --git a/cdc/processor/pipeline/mounter.go b/cdc/processor/pipeline/mounter.go index 1bfc94b43c4..2e156b9cdbd 100644 --- a/cdc/processor/pipeline/mounter.go +++ b/cdc/processor/pipeline/mounter.go @@ -15,12 +15,12 @@ package pipeline import ( "context" - "log" "sync" "time" "github.com/edwingeng/deque" "github.com/pingcap/failpoint" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/notify" "github.com/pingcap/ticdc/pkg/pipeline" diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index 15b28cb94ed..651a250a340 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -151,7 +151,7 @@ func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err func (n *sinkNode) emitEvent(ctx pipeline.NodeContext, event *model.PolymorphicEvent) error { if event == nil || event.Row == nil { - log.Warn("skip emit empty rows", zap.Any("event", event)) + log.Warn("skip emit nil event", zap.Any("event", event)) return nil } @@ -161,6 +161,7 @@ func (n *sinkNode) emitEvent(ctx pipeline.NodeContext, event *model.PolymorphicE // begin; insert into t (id) values (1); delete from t where id=1; commit; // Just ignore these row changed events if colLen == 0 && preColLen == 0 { + log.Warn("skip emit empty row event", zap.Any("event", event)) return nil } diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index be4ba66f5c1..7ee785b90dc 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/entry" "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" "github.com/pingcap/ticdc/cdc/sink" "github.com/pingcap/ticdc/cdc/sink/common" serverConfig "github.com/pingcap/ticdc/pkg/config" @@ -89,10 +88,15 @@ func (t *tablePipelineImpl) ResolvedTs() model.Ts { // will be able to cooperate replication status directly. Then we will add // another replication barrier for consistent replication instead of reusing // the global resolved-ts. - if redo.IsConsistentEnabled(t.replConfig.Consistent.Level) { - return t.sinkNode.ResolvedTs() - } - return t.sorterNode.ResolvedTs() + + // Always report resolved ts from sink for resolving #3503. + // TODO uncomment the following lines. + // if redo.IsConsistentEnabled(t.replConfig.Consistent.Level) { + // return t.sinkNode.ResolvedTs() + // } + // return t.sorterNode.ResolvedTs() + + return t.sinkNode.ResolvedTs() } // CheckpointTs returns the checkpoint ts in this table pipeline diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 8ab353b7997..36787e3d70b 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -159,7 +159,8 @@ func (p *processor) tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR if !p.checkChangefeedNormal() { return nil, cerror.ErrAdminStopProcessor.GenWithStackByArgs() } - if skip := p.checkPosition(); skip { + // we should skip this tick after create a task position + if p.createTaskPosition() { return p.changefeed, nil } if err := p.handleErrorCh(ctx); err != nil { @@ -177,7 +178,11 @@ func (p *processor) tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR if err := p.flushRedoLogMeta(ctx); err != nil { return nil, err } - p.handlePosition() + // it is no need to check the err here, because we will use + // local time when an error return, which is acceptable + pdTime, _ := ctx.GlobalVars().TimeAcquirer.CurrentTimeFromCached() + + p.handlePosition(oracle.GetPhysical(pdTime)) p.pushResolvedTs2Table() p.handleWorkload() p.doGCSchemaStorage(ctx) @@ -195,10 +200,10 @@ func (p *processor) checkChangefeedNormal() bool { return true } -// checkPosition create a new task position, and put it into the etcd state. -// task position maybe be not exist only when the processor is running first time. -func (p *processor) checkPosition() (skipThisTick bool) { - if p.changefeed.TaskPositions[p.captureInfo.ID] != nil { +// createTaskPosition will create a new task position if a task position does not exist. +// task position not exist only when the processor is running first in the first tick. +func (p *processor) createTaskPosition() (skipThisTick bool) { + if _, exist := p.changefeed.TaskPositions[p.captureInfo.ID]; exist { return false } if p.initialized { @@ -225,6 +230,9 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { ctx, cancel := cdcContext.WithCancel(ctx) p.cancel = cancel + // We don't close this error channel, since it is only safe to close channel + // in sender, and this channel will be used in many modules including sink, + // redo log manager, etc. Let runtime GC to recycle it. errCh := make(chan error, 16) p.wg.Add(1) go func() { @@ -236,7 +244,6 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { for { select { case <-ctx.Done(): - close(errCh) return case err := <-errCh: if err == nil { @@ -557,7 +564,7 @@ func (p *processor) checkTablesNum(ctx cdcContext.Context) error { } // handlePosition calculates the local resolved ts and local checkpoint ts -func (p *processor) handlePosition() { +func (p *processor) handlePosition(currentTs int64) { minResolvedTs := uint64(math.MaxUint64) if p.schemaStorage != nil { minResolvedTs = p.schemaStorage.ResolvedTs() @@ -578,15 +585,11 @@ func (p *processor) handlePosition() { } resolvedPhyTs := oracle.ExtractPhysical(minResolvedTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - p.metricResolvedTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-resolvedPhyTs) / 1e3) + p.metricResolvedTsLagGauge.Set(float64(currentTs-resolvedPhyTs) / 1e3) p.metricResolvedTsGauge.Set(float64(resolvedPhyTs)) checkpointPhyTs := oracle.ExtractPhysical(minCheckpointTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - p.metricCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-checkpointPhyTs) / 1e3) + p.metricCheckpointTsLagGauge.Set(float64(currentTs-checkpointPhyTs) / 1e3) p.metricCheckpointTsGauge.Set(float64(checkpointPhyTs)) // minResolvedTs and minCheckpointTs may less than global resolved ts and global checkpoint ts when a new table added, the startTs of the new table is less than global checkpoint ts. diff --git a/cdc/puller/frontier/list.go b/cdc/puller/frontier/list.go index 2a14aeafff8..249127d6bfb 100644 --- a/cdc/puller/frontier/list.go +++ b/cdc/puller/frontier/list.go @@ -16,11 +16,12 @@ package frontier import ( "bytes" "fmt" - "log" "math" "strings" _ "unsafe" // required by go:linkname + + "github.com/pingcap/log" ) const ( diff --git a/cdc/redo/common/util.go b/cdc/redo/common/util.go index 9e1e577f78b..106fe1b2ba8 100644 --- a/cdc/redo/common/util.go +++ b/cdc/redo/common/util.go @@ -28,7 +28,7 @@ import ( // InitS3storage init a storage used for s3, // s3URI should be like s3URI="s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" -func InitS3storage(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { +var InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { if len(uri.Host) == 0 { return nil, cerror.WrapError(cerror.ErrS3StorageInitialize, errors.Errorf("please specify the bucket for s3 in %v", uri)) } diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index ff3792b91ff..94553c9db8b 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -102,6 +102,9 @@ type LogManager interface { // EmitDDLEvent and FlushResolvedAndCheckpointTs are called from owner only EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error FlushResolvedAndCheckpointTs(ctx context.Context, resolvedTs, checkpointTs uint64) (err error) + + // Cleanup removes all redo logs + Cleanup(ctx context.Context) error } // ManagerOptions defines options for redo log manager @@ -302,7 +305,11 @@ func (m *ManagerImpl) RemoveTable(tableID model.TableID) { } else { log.Warn("remove a table not maintained in redo log manager", zap.Int64("table-id", tableID)) } - // TODO: send remove table command to redo log writer +} + +// Cleanup removes all redo logs of this manager, it is called when changefeed is removed +func (m *ManagerImpl) Cleanup(ctx context.Context) error { + return m.writer.DeleteAllLogs(ctx) } // updatertsMap reads rtsMap from redo log writer and calculate the minimum diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index d70bd51a8ea..18a5f9fb48e 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -193,4 +193,7 @@ func TestLogManagerInOwner(t *testing.T) { ddl := &model.DDLEvent{StartTs: 100, CommitTs: 120, Query: "CREATE TABLE `TEST.T1`"} err = logMgr.EmitDDLEvent(ctx, ddl) require.Nil(t, err) + + err = logMgr.writer.DeleteAllLogs(ctx) + require.Nil(t, err) } diff --git a/cdc/redo/reader/file_test.go b/cdc/redo/reader/file_test.go index 4acbb59981d..0de8bc77295 100644 --- a/cdc/redo/reader/file_test.go +++ b/cdc/redo/reader/file_test.go @@ -71,8 +71,10 @@ func TestReaderRead(t *testing.T) { require.Nil(t, err) w.AdvanceTs(11) _, err = w.Write(data) - w.Close() require.Nil(t, err) + err = w.Close() + require.Nil(t, err) + require.True(t, !w.IsRunning()) fileName := fmt.Sprintf("%s_%s_%d_%s_%d%s", cfg.CaptureID, cfg.ChangeFeedID, cfg.CreateTime.Unix(), cfg.FileType, 11, common.LogEXT) path := filepath.Join(cfg.Dir, fileName) info, err := os.Stat(path) @@ -92,6 +94,7 @@ func TestReaderRead(t *testing.T) { err = r[0].Read(log) require.Nil(t, err) require.EqualValues(t, 1123, log.RedoRow.Row.CommitTs) + time.Sleep(1001 * time.Millisecond) } func TestReaderOpenSelectedFiles(t *testing.T) { @@ -123,7 +126,8 @@ func TestReaderOpenSelectedFiles(t *testing.T) { data, err = log.MarshalMsg(nil) require.Nil(t, err) _, err = w.Write(data) - w.Close() + require.Nil(t, err) + err = w.Close() require.Nil(t, err) path := filepath.Join(cfg.Dir, fileName) f, err := os.Open(path) @@ -245,4 +249,5 @@ func TestReaderOpenSelectedFiles(t *testing.T) { require.Regexp(t, tt.wantErr, err.Error(), tt.name) } } + time.Sleep(1001 * time.Millisecond) } diff --git a/cdc/redo/reader/reader.go b/cdc/redo/reader/reader.go index af5ab8ebc52..339c5184956 100644 --- a/cdc/redo/reader/reader.go +++ b/cdc/redo/reader/reader.go @@ -95,6 +95,11 @@ func NewLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) if err != nil { return nil, err } + // remove logs in local dir first, if have logs left belongs to previous changefeed with the same name may have error when apply logs + err = os.RemoveAll(cfg.Dir) + if err != nil { + return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) + } err = downLoadToLocal(ctx, cfg.Dir, s3storage, common.DefaultMetaFileType) if err != nil { return nil, cerror.WrapError(cerror.ErrRedoDownloadFailed, err) diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index 88ad069cbb5..2d89bafe5ab 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -18,15 +18,19 @@ import ( "fmt" "io" "io/ioutil" + "net/url" "os" "path/filepath" "testing" "time" + "github.com/golang/mock/gomock" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/redo/common" "github.com/pingcap/ticdc/cdc/redo/writer" + mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/multierr" @@ -38,6 +42,35 @@ func TestNewLogReader(t *testing.T) { _, err = NewLogReader(context.Background(), &LogReaderConfig{}) require.Nil(t, err) + + dir, err := ioutil.TempDir("", "redo-NewLogReader") + require.Nil(t, err) + defer os.RemoveAll(dir) + + s3URI, err := url.Parse("s3://logbucket/test-changefeed?endpoint=http://111/") + require.Nil(t, err) + + origin := common.InitS3storage + defer func() { + common.InitS3storage = origin + }() + controller := gomock.NewController(t) + mockStorage := mockstorage.NewMockExternalStorage(controller) + // no file to download + mockStorage.EXPECT().WalkDir(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + common.InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { + return mockStorage, nil + } + + // after init should rm the dir + _, err = NewLogReader(context.Background(), &LogReaderConfig{ + S3Storage: true, + Dir: dir, + S3URI: *s3URI, + }) + require.Nil(t, err) + _, err = os.Stat(dir) + require.True(t, os.IsNotExist(err)) } func TestLogReaderResetReader(t *testing.T) { @@ -63,7 +96,8 @@ func TestLogReaderResetReader(t *testing.T) { require.Nil(t, err) _, err = w.Write(data) require.Nil(t, err) - w.Close() + err = w.Close() + require.Nil(t, err) path := filepath.Join(dir, fileName) f, err := os.Open(path) @@ -81,7 +115,8 @@ func TestLogReaderResetReader(t *testing.T) { require.Nil(t, err) _, err = w.Write(data) require.Nil(t, err) - w.Close() + err = w.Close() + require.Nil(t, err) path = filepath.Join(dir, fileName) f1, err := os.Open(path) require.Nil(t, err) @@ -193,6 +228,7 @@ func TestLogReaderResetReader(t *testing.T) { } } + time.Sleep(1001 * time.Millisecond) } func TestLogReaderReadMeta(t *testing.T) { diff --git a/cdc/redo/writer/blackhole_writer.go b/cdc/redo/writer/blackhole_writer.go index aadef99e162..9accc98086b 100644 --- a/cdc/redo/writer/blackhole_writer.go +++ b/cdc/redo/writer/blackhole_writer.go @@ -31,6 +31,10 @@ type blackHoleWriter struct { checkpointTs uint64 } +func (bs *blackHoleWriter) DeleteAllLogs(ctx context.Context) error { + return nil +} + // NewBlackHoleWriter creates a blackHole writer func NewBlackHoleWriter() *blackHoleWriter { return &blackHoleWriter{ diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index 23015b836de..d307f98f45c 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -49,13 +49,6 @@ const ( defaultS3Timeout = 3 * time.Second ) -const ( - // stopped defines the state value of a Writer which has been stopped - stopped bool = false - // started defines the state value of a Writer which is currently started - started bool = true -) - var ( // for easy testing, not set to const megabyte int64 = 1024 * 1024 @@ -119,7 +112,7 @@ type Writer struct { commitTS atomic.Uint64 // the ts send with the event eventCommitTS atomic.Uint64 - state atomic.Bool + running atomic.Bool gcRunning atomic.Bool size int64 file *os.File @@ -162,7 +155,7 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri storage: s3storage, } - w.state.Store(started) + w.running.Store(true) go w.runFlushToDisk(ctx, cfg.FlushIntervalInMs) return w, nil @@ -179,12 +172,14 @@ func (w *Writer) runFlushToDisk(ctx context.Context, flushIntervalInMs int64) { select { case <-ctx.Done(): - log.Info("runFlushToDisk got canceled", zap.Error(ctx.Err())) - return + err := w.Close() + if err != nil { + log.Error("runFlushToDisk close fail", zap.String("changefeedID", w.cfg.ChangeFeedID), zap.Error(err)) + } case <-ticker.C: err := w.Flush() if err != nil { - log.Error("redo log flush error", zap.Error(err)) + log.Error("redo log flush fail", zap.String("changefeedID", w.cfg.ChangeFeedID), zap.Error(err)) } } } @@ -260,19 +255,20 @@ func (w *Writer) Close() error { } w.Lock() defer w.Unlock() + // always set to false when closed, since if having err may not get fixed just by retry + defer w.running.Store(false) err := w.close() if err != nil { return err } - w.state.Store(stopped) return nil } // IsRunning implement IsRunning interface func (w *Writer) IsRunning() bool { - return w.state.Load() + return w.running.Load() } func (w *Writer) isGCRunning() bool { @@ -523,15 +519,11 @@ func (w *Writer) flush() error { } func (w *Writer) writeToS3(ctx context.Context, name string) error { - log.Debug("writeToS3", zap.String("name", name)) - fileData, err := os.ReadFile(name) if err != nil { return cerror.WrapError(cerror.ErrRedoFileOp, err) } - log.Debug("storage.WriteFile", zap.String("name", filepath.Base(name)), zap.Int("data size", len(fileData))) - // Key in s3: aws.String(rs.options.Prefix + name), prefix should be changefeed name return cerror.WrapError(cerror.ErrS3StorageAPI, w.storage.WriteFile(ctx, filepath.Base(name), fileData)) } diff --git a/cdc/redo/writer/file_test.go b/cdc/redo/writer/file_test.go index 5c06e8f2c60..ad3a7f7bdb7 100644 --- a/cdc/redo/writer/file_test.go +++ b/cdc/redo/writer/file_test.go @@ -57,7 +57,7 @@ func TestWriterWrite(t *testing.T) { CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), }, uint64buf: make([]byte, 8), - state: *atomic.NewBool(started), + running: *atomic.NewBool(true), } w.eventCommitTS.Store(1) @@ -89,13 +89,45 @@ func TestWriterWrite(t *testing.T) { info, err = os.Stat(path) require.Nil(t, err) require.Equal(t, fileName, info.Name()) - w.Close() + err = w.Close() + require.Nil(t, err) + require.False(t, w.IsRunning()) // safe close, rename to .log with max eventCommitTS as name fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, 22, common.LogEXT) path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) require.Nil(t, err) require.Equal(t, fileName, info.Name()) + + w1 := &Writer{ + cfg: &FileWriterConfig{ + MaxLogSize: 10, + Dir: dir, + ChangeFeedID: "test-cf11", + CaptureID: "cp", + FileType: common.DefaultRowLogFileType, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + }, + uint64buf: make([]byte, 8), + running: *atomic.NewBool(true), + } + + w1.eventCommitTS.Store(1) + _, err = w1.Write([]byte("tes1t11111")) + require.Nil(t, err) + // create a .tmp file + fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w1.cfg.CaptureID, w1.cfg.ChangeFeedID, w1.cfg.CreateTime.Unix(), w1.cfg.FileType, 1, common.LogEXT) + common.TmpEXT + path = filepath.Join(w1.cfg.Dir, fileName) + info, err = os.Stat(path) + require.Nil(t, err) + require.Equal(t, fileName, info.Name()) + // change the file name, should cause CLose err + err = os.Rename(path, path+"new") + require.Nil(t, err) + err = w1.Close() + require.NotNil(t, err) + // closed anyway + require.False(t, w1.IsRunning()) } func TestWriterGC(t *testing.T) { @@ -136,7 +168,7 @@ func TestWriterGC(t *testing.T) { uint64buf: make([]byte, 8), storage: mockStorage, } - w.state.Store(started) + w.running.Store(true) w.eventCommitTS.Store(1) _, err = w.Write([]byte("t1111")) require.Nil(t, err) @@ -156,7 +188,7 @@ func TestWriterGC(t *testing.T) { err = w.Close() require.Nil(t, err) - + require.False(t, w.IsRunning()) files, err = ioutil.ReadDir(w.cfg.Dir) require.Nil(t, err) require.Equal(t, 1, len(files), "should have 1 log left after GC") @@ -179,8 +211,6 @@ func TestNewWriter(t *testing.T) { _, err := NewWriter(context.Background(), nil) require.NotNil(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() s3URI, err := url.Parse("s3://logbucket/test-changefeed?endpoint=http://111/") require.Nil(t, err) @@ -188,13 +218,16 @@ func TestNewWriter(t *testing.T) { require.Nil(t, err) defer os.RemoveAll(dir) - _, err = NewWriter(ctx, &FileWriterConfig{ + w, err := NewWriter(context.Background(), &FileWriterConfig{ Dir: "sdfsf", S3Storage: true, S3URI: *s3URI, }) require.Nil(t, err) - time.Sleep(time.Duration(defaultFlushIntervalInMs+1000) * time.Millisecond) + time.Sleep(time.Duration(defaultFlushIntervalInMs+1) * time.Millisecond) + err = w.Close() + require.Nil(t, err) + require.False(t, w.IsRunning()) controller := gomock.NewController(t) mockStorage := mockstorage.NewMockExternalStorage(controller) @@ -202,7 +235,7 @@ func TestNewWriter(t *testing.T) { mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_ddl_0.log", gomock.Any()).Return(nil).Times(1) mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_ddl_0.log.tmp").Return(nil).Times(1) - w := &Writer{ + w = &Writer{ cfg: &FileWriterConfig{ Dir: dir, CaptureID: "cp", @@ -215,7 +248,7 @@ func TestNewWriter(t *testing.T) { uint64buf: make([]byte, 8), storage: mockStorage, } - w.state.Store(started) + w.running.Store(true) _, err = w.Write([]byte("test")) require.Nil(t, err) // @@ -224,5 +257,6 @@ func TestNewWriter(t *testing.T) { err = w.Close() require.Nil(t, err) - require.Equal(t, w.state.Load(), stopped) + require.Equal(t, w.running.Load(), false) + time.Sleep(time.Duration(defaultFlushIntervalInMs+1) * time.Millisecond) } diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index 2422f9769e8..37b019c0742 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -24,6 +24,8 @@ import ( "sync" "time" + "github.com/aws/aws-sdk-go/aws/awserr" + "github.com/aws/aws-sdk-go/service/s3" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" @@ -32,6 +34,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "go.uber.org/multierr" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) //go:generate mockery --name=RedoLogWriter --inpackage @@ -59,13 +62,16 @@ type RedoLogWriter interface { // GetCurrentResolvedTs return all the ResolvedTs list for given tableIDs GetCurrentResolvedTs(ctx context.Context, tableIDs []int64) (resolvedTsList map[int64]uint64, err error) + + // DeleteAllLogs delete all log files related to the changefeed, called from owner only when delete changefeed + DeleteAllLogs(ctx context.Context) error } var defaultGCIntervalInMs = 5000 var ( - initOnce sync.Once - logWriter *LogWriter + logWriters = map[string]*LogWriter{} + initLock sync.Mutex ) var redoLogPool = sync.Pool{ @@ -99,72 +105,79 @@ type LogWriter struct { } // NewLogWriter creates a LogWriter instance. It is guaranteed only one LogWriter per changefeed -// TODO: delete log files when changefeed removed, metric func NewLogWriter(ctx context.Context, cfg *LogWriterConfig) (*LogWriter, error) { - var errInit error - // currently, caller do not have the ability of self-healing, like try to fix if on some error, - // so initOnce just literary init once, do not support re-init if fail on some condition - initOnce.Do(func() { - if cfg == nil { - errInit = cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("LogWriterConfig can not be nil")) - return - } + if cfg == nil { + return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("LogWriterConfig can not be nil")) + } - rowCfg := &FileWriterConfig{ - Dir: cfg.Dir, - ChangeFeedID: cfg.ChangeFeedID, - CaptureID: cfg.CaptureID, - FileType: common.DefaultRowLogFileType, - CreateTime: cfg.CreateTime, - MaxLogSize: cfg.MaxLogSize, - FlushIntervalInMs: cfg.FlushIntervalInMs, - S3Storage: cfg.S3Storage, - S3URI: cfg.S3URI, - } - ddlCfg := &FileWriterConfig{ - Dir: cfg.Dir, - ChangeFeedID: cfg.ChangeFeedID, - CaptureID: cfg.CaptureID, - FileType: common.DefaultDDLLogFileType, - CreateTime: cfg.CreateTime, - MaxLogSize: cfg.MaxLogSize, - FlushIntervalInMs: cfg.FlushIntervalInMs, - S3Storage: cfg.S3Storage, - S3URI: cfg.S3URI, - } - logWriter = &LogWriter{ - cfg: cfg, - } - logWriter.rowWriter, errInit = NewWriter(ctx, rowCfg) - if errInit != nil { - return - } - logWriter.ddlWriter, errInit = NewWriter(ctx, ddlCfg) - if errInit != nil { - return + initLock.Lock() + defer initLock.Unlock() + + if v, ok := logWriters[cfg.ChangeFeedID]; ok { + // if cfg changed or already closed need create a new LogWriter + if cfg.String() == v.cfg.String() && !v.isStopped() { + return v, nil } + } - // since the error will not block write log, so keep go to the next init process - err := logWriter.initMeta(ctx) + var err error + var logWriter *LogWriter + rowCfg := &FileWriterConfig{ + Dir: cfg.Dir, + ChangeFeedID: cfg.ChangeFeedID, + CaptureID: cfg.CaptureID, + FileType: common.DefaultRowLogFileType, + CreateTime: cfg.CreateTime, + MaxLogSize: cfg.MaxLogSize, + FlushIntervalInMs: cfg.FlushIntervalInMs, + S3Storage: cfg.S3Storage, + S3URI: cfg.S3URI, + } + ddlCfg := &FileWriterConfig{ + Dir: cfg.Dir, + ChangeFeedID: cfg.ChangeFeedID, + CaptureID: cfg.CaptureID, + FileType: common.DefaultDDLLogFileType, + CreateTime: cfg.CreateTime, + MaxLogSize: cfg.MaxLogSize, + FlushIntervalInMs: cfg.FlushIntervalInMs, + S3Storage: cfg.S3Storage, + S3URI: cfg.S3URI, + } + logWriter = &LogWriter{ + cfg: cfg, + } + logWriter.rowWriter, err = NewWriter(ctx, rowCfg) + if err != nil { + return nil, err + } + logWriter.ddlWriter, err = NewWriter(ctx, ddlCfg) + if err != nil { + return nil, err + } + + // since the error will not block write log, so keep go to the next init process + err = logWriter.initMeta(ctx) + if err != nil { + log.Warn("init redo meta fail", + zap.String("change feed", cfg.ChangeFeedID), + zap.Error(err)) + } + if cfg.S3Storage { + logWriter.storage, err = common.InitS3storage(ctx, cfg.S3URI) if err != nil { - log.Warn("init redo meta fail", - zap.String("change feed", cfg.ChangeFeedID), - zap.Error(err)) + return nil, err } - if cfg.S3Storage { - var s3storage storage.ExternalStorage - s3storage, errInit = common.InitS3storage(ctx, cfg.S3URI) - if errInit != nil { - return - } - logWriter.storage = s3storage + } + // close previous writer + if v, ok := logWriters[cfg.ChangeFeedID]; ok { + err = v.Close() + if err != nil { + return nil, err } - - go logWriter.runGC(ctx) - }) - if errInit != nil { - return nil, errInit } + logWriters[cfg.ChangeFeedID] = logWriter + go logWriter.runGC(ctx) return logWriter, nil } @@ -214,12 +227,14 @@ func (l *LogWriter) runGC(ctx context.Context) { select { case <-ctx.Done(): - log.Info("runGC got canceled", zap.Error(ctx.Err())) - return + err := l.Close() + if err != nil { + log.Error("runGC close fail", zap.String("changefeedID", l.cfg.ChangeFeedID), zap.Error(err)) + } case <-ticker.C: err := l.gc() if err != nil { - log.Error("redo log GC error", zap.Error(err)) + log.Error("redo log GC fail", zap.String("changefeedID", l.cfg.ChangeFeedID), zap.Error(err)) } } } @@ -390,6 +405,86 @@ func (l *LogWriter) GetCurrentResolvedTs(ctx context.Context, tableIDs []int64) return ret, nil } +// DeleteAllLogs implement DeleteAllLogs api +func (l *LogWriter) DeleteAllLogs(ctx context.Context) error { + err := l.Close() + if err != nil { + return err + } + + if !l.cfg.S3Storage { + err = os.RemoveAll(l.cfg.Dir) + if err != nil { + return cerror.WrapError(cerror.ErrRedoFileOp, err) + } + // after delete logs, rm the LogWriter since it is already closed + l.cleanUpLogWriter() + return nil + } + + files, err := getAllFilesInS3(ctx, l) + if err != nil { + return err + } + + err = l.deleteFilesInS3(ctx, files) + if err != nil { + return err + } + // after delete logs, rm the LogWriter since it is already closed + l.cleanUpLogWriter() + return nil +} + +func (l *LogWriter) cleanUpLogWriter() { + initLock.Lock() + defer initLock.Unlock() + delete(logWriters, l.cfg.ChangeFeedID) +} + +func (l *LogWriter) deleteFilesInS3(ctx context.Context, files []string) error { + eg, eCtx := errgroup.WithContext(ctx) + for _, f := range files { + name := f + eg.Go(func() error { + err := l.storage.DeleteFile(eCtx, name) + if err != nil { + // if fail then retry, may end up with notExit err, ignore the error + if !isNotExistInS3(err) { + return cerror.WrapError(cerror.ErrS3StorageAPI, err) + } + } + return nil + }) + } + return eg.Wait() +} + +func isNotExistInS3(err error) bool { + if err != nil { + if aerr, ok := errors.Cause(err).(awserr.Error); ok { // nolint:errorlint + switch aerr.Code() { + case s3.ErrCodeNoSuchKey: + return true + } + } + } + return false +} + +var getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { + files := []string{} + err := l.storage.WalkDir(ctx, &storage.WalkOption{}, func(path string, _ int64) error { + files = append(files, path) + return nil + }) + if err != nil { + return nil, cerror.WrapError(cerror.ErrS3StorageAPI, err) + } + + return files, nil +} + // Close implements RedoLogWriter.Close. func (l *LogWriter) Close() error { var err error @@ -498,3 +593,7 @@ func (l *LogWriter) writeMetaToS3(ctx context.Context) error { func (l *LogWriter) filePath() string { return filepath.Join(l.cfg.Dir, l.getMetafileName()) } + +func (cfg LogWriterConfig) String() string { + return fmt.Sprintf("%s:%s:%s:%d:%d:%s:%t", cfg.ChangeFeedID, cfg.CaptureID, cfg.Dir, cfg.MaxLogSize, cfg.FlushIntervalInMs, cfg.S3URI.String(), cfg.S3Storage) +} diff --git a/cdc/redo/writer/writer_test.go b/cdc/redo/writer/writer_test.go index ad1e4337516..bd7432ca49a 100644 --- a/cdc/redo/writer/writer_test.go +++ b/cdc/redo/writer/writer_test.go @@ -17,12 +17,15 @@ import ( "context" "fmt" "io/ioutil" + "math" "os" "path/filepath" - "sync" + "strings" "testing" "time" + "github.com/aws/aws-sdk-go/aws/awserr" + "github.com/aws/aws-sdk-go/service/s3" "github.com/golang/mock/gomock" "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" @@ -612,17 +615,30 @@ func TestNewLogWriter(t *testing.T) { CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), FlushIntervalInMs: 5, } - var ll *LogWriter - initOnce = sync.Once{} - ll, err = NewLogWriter(ctx, cfg) + ll, err := NewLogWriter(ctx, cfg) require.Nil(t, err) time.Sleep(time.Duration(defaultGCIntervalInMs+1) * time.Millisecond) require.Equal(t, map[int64]uint64{}, ll.meta.ResolvedTsList) - cfg.Dir += "ttt" - ll1, err := NewLogWriter(ctx, cfg) + ll2, err := NewLogWriter(ctx, cfg) require.Nil(t, err) - require.Same(t, ll, ll1) + require.Same(t, ll, ll2) + + cfg1 := &LogWriterConfig{ + Dir: "dirt111", + ChangeFeedID: "test-cf", + CaptureID: "cp", + MaxLogSize: 10, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FlushIntervalInMs: 5, + } + ll1, err := NewLogWriter(ctx, cfg1) + require.Nil(t, err) + require.NotSame(t, ll, ll1) + + ll2, err = NewLogWriter(ctx, cfg) + require.Nil(t, err) + require.NotSame(t, ll, ll2) dir, err := ioutil.TempDir("", "redo-NewLogWriter") require.Nil(t, err) @@ -641,14 +657,24 @@ func TestNewLogWriter(t *testing.T) { _, err = f.Write(data) require.Nil(t, err) - cfg.Dir = dir - initOnce = sync.Once{} + cfg = &LogWriterConfig{ + Dir: dir, + ChangeFeedID: "test-cf", + CaptureID: "cp", + MaxLogSize: 10, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FlushIntervalInMs: 5, + } l, err := NewLogWriter(ctx, cfg) require.Nil(t, err) + err = l.Close() + require.Nil(t, err) + require.True(t, l.isStopped()) require.Equal(t, cfg.Dir, l.cfg.Dir) require.Equal(t, meta.CheckPointTs, l.meta.CheckPointTs) require.Equal(t, meta.ResolvedTs, l.meta.ResolvedTs) require.Equal(t, map[int64]uint64{}, l.meta.ResolvedTsList) + time.Sleep(time.Millisecond * time.Duration(math.Max(float64(defaultFlushIntervalInMs), float64(defaultGCIntervalInMs))+1)) } func TestWriterRedoGC(t *testing.T) { @@ -661,9 +687,6 @@ func TestWriterRedoGC(t *testing.T) { FlushIntervalInMs: 5, } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - type args struct { isRunning bool } @@ -686,8 +709,10 @@ func TestWriterRedoGC(t *testing.T) { } for _, tt := range tests { mockWriter := &mockFileWriter{} - mockWriter.On("IsRunning").Return(tt.args.isRunning) + mockWriter.On("IsRunning").Return(tt.args.isRunning).Twice() mockWriter.On("Close").Return(nil) + mockWriter.On("IsRunning").Return(false) + if tt.args.isRunning { mockWriter.On("GC", mock.Anything).Return(nil) } @@ -697,7 +722,7 @@ func TestWriterRedoGC(t *testing.T) { meta: &common.LogMeta{ResolvedTsList: map[int64]uint64{}}, cfg: cfg, } - go writer.runGC(ctx) + go writer.runGC(context.Background()) time.Sleep(time.Duration(defaultGCIntervalInMs+1) * time.Millisecond) writer.Close() @@ -710,3 +735,107 @@ func TestWriterRedoGC(t *testing.T) { } } } + +func TestDeleteAllLogs(t *testing.T) { + fileName := "1" + fileName1 := "11" + + type args struct { + enableS3 bool + } + + tests := []struct { + name string + args args + closeErr error + getAllFilesInS3Err error + deleteFileErr error + wantErr string + }{ + { + name: "happy local", + args: args{enableS3: false}, + }, + { + name: "happy s3", + args: args{enableS3: true}, + }, + { + name: "close err", + args: args{enableS3: true}, + closeErr: errors.New("xx"), + wantErr: ".*xx*.", + }, + { + name: "getAllFilesInS3 err", + args: args{enableS3: true}, + getAllFilesInS3Err: errors.New("xx"), + wantErr: ".*xx*.", + }, + { + name: "deleteFile normal err", + args: args{enableS3: true}, + deleteFileErr: errors.New("xx"), + wantErr: ".*ErrS3StorageAPI*.", + }, + { + name: "deleteFile notExist err", + args: args{enableS3: true}, + deleteFileErr: awserr.New(s3.ErrCodeNoSuchKey, "no such key", nil), + }, + } + + for _, tt := range tests { + dir, err := ioutil.TempDir("", "redo-DeleteAllLogs") + require.Nil(t, err) + path := filepath.Join(dir, fileName) + _, err = os.Create(path) + require.Nil(t, err) + path = filepath.Join(dir, fileName1) + _, err = os.Create(path) + require.Nil(t, err) + + origin := getAllFilesInS3 + getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { + return []string{fileName, fileName1}, tt.getAllFilesInS3Err + } + controller := gomock.NewController(t) + mockStorage := mockstorage.NewMockExternalStorage(controller) + + mockStorage.EXPECT().DeleteFile(gomock.Any(), gomock.Any()).Return(tt.deleteFileErr).MaxTimes(2) + mockWriter := &mockFileWriter{} + mockWriter.On("Close").Return(tt.closeErr) + cfg := &LogWriterConfig{ + Dir: dir, + ChangeFeedID: "test-cf", + CaptureID: "cp", + MaxLogSize: 10, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FlushIntervalInMs: 5, + S3Storage: tt.args.enableS3, + } + writer := LogWriter{ + rowWriter: mockWriter, + ddlWriter: mockWriter, + meta: &common.LogMeta{ResolvedTsList: map[int64]uint64{}}, + cfg: cfg, + storage: mockStorage, + } + if strings.Contains(tt.name, "happy") { + logWriters[writer.cfg.ChangeFeedID] = &writer + } + ret := writer.DeleteAllLogs(context.Background()) + if tt.wantErr != "" { + require.Regexp(t, tt.wantErr, ret.Error(), tt.name) + } else { + require.Nil(t, ret, tt.name) + require.Equal(t, 0, len(logWriters), tt.name) + if !tt.args.enableS3 { + _, err := os.Stat(dir) + require.True(t, os.IsNotExist(err), tt.name) + } + } + os.RemoveAll(dir) + getAllFilesInS3 = origin + } +} diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index c64bc828921..eed118c3ce8 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" - tijson "github.com/pingcap/tidb/types/json" "go.uber.org/zap" ) @@ -476,7 +475,7 @@ func columnToAvroNativeData(col *model.Column, tz *time.Location) (interface{}, case mysql.TypeYear: return col.Value.(int64), "long", nil case mysql.TypeJSON: - return col.Value.(tijson.BinaryJSON).String(), "string", nil + return col.Value.(string), "string", nil case mysql.TypeNewDecimal: return col.Value.(string), "string", nil case mysql.TypeEnum: diff --git a/cdc/sink/codec/avro_test.go b/cdc/sink/codec/avro_test.go index c1ce55c2c48..4b0290fa726 100644 --- a/cdc/sink/codec/avro_test.go +++ b/cdc/sink/codec/avro_test.go @@ -88,6 +88,7 @@ func (s *avroBatchEncoderSuite) TestAvroEncodeOnly(c *check.C) { {Name: "myfloat", Value: float64(3.14), Type: mysql.TypeFloat}, {Name: "mybytes", Value: []byte("Hello World"), Type: mysql.TypeBlob}, {Name: "ts", Value: time.Now().Format(types.TimeFSPFormat), Type: mysql.TypeTimestamp}, + {Name: "myjson", Value: "{\"foo\": \"bar\"}", Type: mysql.TypeJSON}, }, time.Local) c.Assert(err, check.IsNil) diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index cde3bba4f79..91427c9bb8d 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -16,12 +16,12 @@ package codec import ( "context" "fmt" - "log" "strconv" "strings" "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" cerror "github.com/pingcap/ticdc/pkg/errors" canal "github.com/pingcap/ticdc/proto/canal" diff --git a/cdc/sink/common/flow_control.go b/cdc/sink/common/flow_control.go index 47ad19fc6b7..a008d58b25a 100644 --- a/cdc/sink/common/flow_control.go +++ b/cdc/sink/common/flow_control.go @@ -14,12 +14,12 @@ package common import ( - "log" "sync" "sync/atomic" "github.com/edwingeng/deque" "github.com/pingcap/errors" + "github.com/pingcap/log" cerrors "github.com/pingcap/ticdc/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 80bf0c265ea..8df65c0d9b0 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -110,9 +110,12 @@ func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { // NOTICE: Because all table sinks will try to flush backend sink, // which will cause a lot of lock contention and blocking in high concurrency cases. // So here we use flushing as a lightweight lock to improve the lock competition problem. - if !atomic.CompareAndSwapInt64(&m.flushing, 0, 1) { - return m.getCheckpointTs(), nil - } + // + // Do not skip flushing for resolving #3503. + // TODO uncomment the following return. + // if !atomic.CompareAndSwapInt64(&m.flushing, 0, 1) { + // return m.getCheckpointTs(), nil + // } m.flushMu.Lock() defer func() { m.flushMu.Unlock() diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 57efe3c30a2..e718f543dcc 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -537,17 +537,27 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { } 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(120 * 500ms). + // 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/ticdc/issues/3352. + config.Metadata.Timeout = 1 * time.Minute config.Producer.Partitioner = sarama.NewManualPartitioner config.Producer.MaxMessageBytes = c.MaxMessageBytes config.Producer.Return.Successes = true config.Producer.Return.Errors = true config.Producer.RequiredAcks = sarama.WaitForAll - switch strings.ToLower(strings.TrimSpace(c.Compression)) { case "none": config.Producer.Compression = sarama.CompressionNone @@ -580,7 +590,6 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { return nil, errors.Trace(err) } } - if c.SaslScram != nil && len(c.SaslScram.SaslUser) != 0 { config.Net.SASL.Enable = true config.Net.SASL.User = c.SaslScram.SaslUser diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 098566b6499..1e9679c7bc4 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -19,8 +19,10 @@ import ( "sync/atomic" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/redo" + "go.uber.org/zap" ) type tableSink struct { @@ -55,6 +57,16 @@ func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error // is required to be no more than global resolvedTs, table barrierTs and table // redo log watermarkTs. func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + // Log abnormal checkpoint that is large than resolved ts. + logAbnormalCheckpoint := func(ckpt uint64) { + if ckpt > resolvedTs { + log.L().WithOptions(zap.AddCallerSkip(1)). + Warn("checkpoint ts > resolved ts, flushed more than emitted", + zap.Int64("tableID", t.tableID), + zap.Uint64("resolvedTs", resolvedTs), + zap.Uint64("checkpointTs", ckpt)) + } + } i := sort.Search(len(t.buffer), func(i int) bool { return t.buffer[i].CommitTs > resolvedTs }) @@ -64,7 +76,12 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 if err != nil { return ckpt, err } - return t.manager.flushBackendSink(ctx) + ckpt, err = t.manager.flushBackendSink(ctx) + if err != nil { + return ckpt, err + } + logAbnormalCheckpoint(ckpt) + return ckpt, err } resolvedRows := t.buffer[:i] t.buffer = append(make([]*model.RowChangedEvent, 0, len(t.buffer[i:])), t.buffer[i:]...) @@ -78,7 +95,12 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 if err != nil { return ckpt, err } - return t.manager.flushBackendSink(ctx) + ckpt, err = t.manager.flushBackendSink(ctx) + if err != nil { + return ckpt, err + } + logAbnormalCheckpoint(ckpt) + return ckpt, err } func (t *tableSink) flushRedoLogs(ctx context.Context, resolvedTs uint64) (uint64, error) { diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go index 3f5add5f70b..60375a12600 100644 --- a/cdc/sorter/unified/backend_pool.go +++ b/cdc/sorter/unified/backend_pool.go @@ -24,7 +24,6 @@ import ( "time" "unsafe" - "github.com/mackerelio/go-osstat/memory" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" @@ -33,6 +32,7 @@ import ( cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/filelock" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" ) @@ -97,6 +97,13 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { metricSorterOnDiskDataSizeGauge := sorterOnDiskDataSizeGauge.WithLabelValues(captureAddr) metricSorterOpenFileCountGauge := sorterOpenFileCountGauge.WithLabelValues(captureAddr) + // TODO: The underlaying implementation only recognizes cgroups set by + // containers, we need to support cgroups set by systemd or manually. + // See https://github.com/pingcap/tidb/issues/22132 + totalMemory, err := memory.MemTotal() + if err != nil { + log.Panic("read memory stat failed", zap.Error(err)) + } for { select { case <-ret.cancelCh: @@ -110,14 +117,8 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { metricSorterOpenFileCountGauge.Set(float64(atomic.LoadInt64(&openFDCount))) // update memPressure - m, err := memory.Get() - - failpoint.Inject("getMemoryPressureFails", func() { - m = nil - err = errors.New("injected get memory pressure failure") - }) - - if err != nil { + usedMemory, err := memory.MemUsed() + if err != nil || totalMemory == 0 { failpoint.Inject("sorterDebug", func() { log.Panic("unified sorter: getting system memory usage failed", zap.Error(err)) }) @@ -128,7 +129,7 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { // encountered, we can fail gracefully. atomic.StoreInt32(&ret.memPressure, 100) } else { - memPressure := m.Used * 100 / m.Total + memPressure := usedMemory * 100 / totalMemory atomic.StoreInt32(&ret.memPressure, int32(memPressure)) } diff --git a/cdc/sorter/unified/backend_pool_test.go b/cdc/sorter/unified/backend_pool_test.go index 3998eb7c456..e616a23f4a2 100644 --- a/cdc/sorter/unified/backend_pool_test.go +++ b/cdc/sorter/unified/backend_pool_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filelock" "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/util/memory" ) type backendPoolSuite struct{} @@ -325,9 +326,11 @@ func (s *backendPoolSuite) TestCleanUpStaleLockNoPermission(c *check.C) { func (s *backendPoolSuite) TestGetMemoryPressureFailure(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/getMemoryPressureFails", "return(true)") - c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/getMemoryPressureFails") //nolint:errcheck + origin := memory.MemTotal + defer func() { + memory.MemTotal = origin + }() + memory.MemTotal = func() (uint64, error) { return 0, nil } dir := c.MkDir() backEndPool, err := newBackEndPool(dir, "") @@ -336,7 +339,7 @@ func (s *backendPoolSuite) TestGetMemoryPressureFailure(c *check.C) { defer backEndPool.terminate() after := time.After(time.Second * 20) - tick := time.Tick(time.Second * 1) + tick := time.Tick(time.Millisecond * 100) for { select { case <-after: diff --git a/dm/chaos/cases/instance.go b/dm/chaos/cases/instance.go index 7fe6cabceec..12142190aec 100644 --- a/dm/chaos/cases/instance.go +++ b/dm/chaos/cases/instance.go @@ -26,7 +26,7 @@ var mustExecSQLs = []string{ } // setInstancesState sets the state (like global sql_mode) for upstream and downstream DB instances. -func setInstancesState(ctx context.Context, targetCfg config2.DBConfig, sourcesCfg ...config2.DBConfig) error { +func setInstancesState(ctx context.Context, targetCfg *config2.DBConfig, sourcesCfg ...*config2.DBConfig) error { targetDB, err := conn.DefaultDBProvider.Apply(targetCfg) if err != nil { return err diff --git a/dm/chaos/cases/main.go b/dm/chaos/cases/main.go index db9c0165f38..92d23cde904 100644 --- a/dm/chaos/cases/main.go +++ b/dm/chaos/cases/main.go @@ -106,7 +106,7 @@ func main() { } // set upstream and downstream instances state. - err = setInstancesState(ctx, cfg.Target, cfg.Source1, cfg.Source2, cfg.Source3) + err = setInstancesState(ctx, &cfg.Target, &cfg.Source1, &cfg.Source2, &cfg.Source3) if err != nil { log.L().Error("fail to set instances state", zap.Error(err)) code = 2 diff --git a/dm/chaos/cases/task.go b/dm/chaos/cases/task.go index ec869d62681..ab69874fa65 100644 --- a/dm/chaos/cases/task.go +++ b/dm/chaos/cases/task.go @@ -86,7 +86,7 @@ func newTask(ctx context.Context, cli pb.MasterClient, taskFile string, schema s } cfg := sourcesCfg[i] - db, err2 := conn.DefaultDBProvider.Apply(cfg) + db, err2 := conn.DefaultDBProvider.Apply(&cfg) if err2 != nil { return nil, err2 } @@ -110,7 +110,7 @@ func newTask(ctx context.Context, cli pb.MasterClient, taskFile string, schema s res = append(res, singleResult{}) } - targetDB, err := conn.DefaultDBProvider.Apply(targetCfg) + targetDB, err := conn.DefaultDBProvider.Apply(&targetCfg) if err != nil { return nil, err } diff --git a/dm/checker/checker.go b/dm/checker/checker.go index 009ebd925af..03242f767c8 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -144,7 +144,7 @@ func (c *Checker) Init(ctx context.Context) (err error) { } dbCfg := instance.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(readTimeout) - instance.sourceDB, err = conn.DefaultDBProvider.Apply(dbCfg) + instance.sourceDB, err = conn.DefaultDBProvider.Apply(&dbCfg) if err != nil { return terror.WithScope(terror.ErrTaskCheckFailedOpenDB.Delegate(err, instance.cfg.From.User, instance.cfg.From.Host, instance.cfg.From.Port), terror.ScopeUpstream) } @@ -157,7 +157,7 @@ func (c *Checker) Init(ctx context.Context) (err error) { } dbCfg = instance.cfg.To dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(readTimeout) - instance.targetDB, err = conn.DefaultDBProvider.Apply(dbCfg) + instance.targetDB, err = conn.DefaultDBProvider.Apply(&dbCfg) if err != nil { return terror.WithScope(terror.ErrTaskCheckFailedOpenDB.Delegate(err, instance.cfg.To.User, instance.cfg.To.Host, instance.cfg.To.Port), terror.ScopeDownstream) } @@ -405,7 +405,7 @@ func (c *Checker) Resume(ctx context.Context, pr chan pb.ProcessResult) { } // Update implements Unit.Update. -func (c *Checker) Update(cfg *config.SubTaskConfig) error { +func (c *Checker) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // not support update configuration now return nil } diff --git a/dm/dm/config/source_config.go b/dm/dm/config/source_config.go index 64d1bde0a0b..b200577cece 100644 --- a/dm/dm/config/source_config.go +++ b/dm/dm/config/source_config.go @@ -390,9 +390,6 @@ func (c *SourceConfig) YamlForDowngrade() (string, error) { } s.From.Password = cipher - // omit default values, so we can ignore them for later marshal - s.omitDefaultVals() - // not write this field when exporting s.EnableRelay = false return s.Yaml() @@ -448,17 +445,6 @@ func NewSourceConfigForDowngrade(sourceCfg *SourceConfig) *SourceConfigForDowngr } } -// omitDefaultVals change default value to empty value for new config item. -// If any default value for new config item is not empty(0 or false or nil), -// we should change it to empty. -func (c *SourceConfigForDowngrade) omitDefaultVals() { - if len(c.From.Session) > 0 { - if timeZone, ok := c.From.Session["time_zone"]; ok && timeZone == defaultTimeZone { - delete(c.From.Session, "time_zone") - } - } -} - // Yaml returns YAML format representation of the config. func (c *SourceConfigForDowngrade) Yaml() (string, error) { b, err := yaml.Marshal(c) diff --git a/dm/dm/config/source_config_test.go b/dm/dm/config/source_config_test.go index 9c48ec36a68..9f6bfd3f5b9 100644 --- a/dm/dm/config/source_config_test.go +++ b/dm/dm/config/source_config_test.go @@ -115,6 +115,7 @@ func (t *testConfig) TestConfig(c *C) { clone6, err := ParseYaml(clone4yaml) c.Assert(err, IsNil) + clone6.From.Session = nil c.Assert(clone6, DeepEquals, clone4) // test invalid config diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index 89c9b912005..bdc66bc5fa2 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -127,13 +127,18 @@ func (db *DBConfig) Decode(data string) error { // Adjust adjusts the config. func (db *DBConfig) Adjust() { - // force set session time zone to UTC here. - AdjustTargetDBTimeZone(db) if len(db.Password) > 0 { db.Password = utils.DecryptOrPlaintext(db.Password) } } +func (db *DBConfig) AdjustWithTimeZone(timeZone string) { + if timeZone != "" { + AdjustDBTimeZone(db, timeZone) + } + db.Adjust() +} + // Clone returns a deep copy of DBConfig. This function only fixes data race when adjusting Session. func (db *DBConfig) Clone() *DBConfig { if db == nil { @@ -238,9 +243,8 @@ type SubTaskConfig struct { // deprecated HeartbeatReportInterval int `toml:"heartbeat-report-interval" json:"heartbeat-report-interval"` // deprecated - EnableHeartbeat bool `toml:"enable-heartbeat" json:"enable-heartbeat"` - // deprecated - Timezone string `toml:"timezone" json:"timezone"` + EnableHeartbeat bool `toml:"enable-heartbeat" json:"enable-heartbeat"` + Timezone string `toml:"timezone" json:"timezone"` Meta *Meta `toml:"meta" json:"meta"` @@ -415,11 +419,6 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { c.MetaSchema = defaultMetaSchema } - if c.Timezone != "" { - log.L().Warn("'timezone' is deprecated, please remove this field.") - c.Timezone = "" - } - dirSuffix := "." + c.Name if !strings.HasSuffix(c.LoaderConfig.Dir, dirSuffix) { // check to support multiple times calling // if not ends with the task name, we append the task name to the tail @@ -433,8 +432,8 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { c.SyncerConfig.CheckpointFlushInterval = defaultCheckpointFlushInterval } - c.From.Adjust() - c.To.Adjust() + c.From.AdjustWithTimeZone(c.Timezone) + c.To.AdjustWithTimeZone(c.Timezone) if verifyDecryptPassword { _, err1 := c.DecryptPassword() diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index af917cdfcbe..48dd784b746 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -71,8 +71,6 @@ var ( defaultBatch = 100 defaultQueueSize = 1024 // do not give too large default value to avoid OOM defaultCheckpointFlushInterval = 30 // in seconds - // force use UTC time_zone. - defaultTimeZone = "+00:00" // TargetDBConfig. defaultSessionCfg = []struct { @@ -296,9 +294,8 @@ type TaskConfig struct { // deprecated HeartbeatUpdateInterval int `yaml:"heartbeat-update-interval" toml:"heartbeat-update-interval" json:"heartbeat-update-interval"` // deprecated - HeartbeatReportInterval int `yaml:"heartbeat-report-interval" toml:"heartbeat-report-interval" json:"heartbeat-report-interval"` - // deprecated - Timezone string `yaml:"timezone" toml:"timezone" json:"timezone"` + HeartbeatReportInterval int `yaml:"heartbeat-report-interval" toml:"heartbeat-report-interval" json:"heartbeat-report-interval"` + Timezone string `yaml:"timezone" toml:"timezone" json:"timezone"` // handle schema/table name mode, and only for schema/table name // if case insensitive, we would convert schema/table name to lower case @@ -696,9 +693,11 @@ func (c *TaskConfig) adjust() error { sort.Strings(unusedConfigs) return terror.ErrConfigGlobalConfigsUnused.Generate(unusedConfigs) } + // we postpone default time_zone init in each unit so we won't change the config value in task/sub_task config if c.Timezone != "" { - log.L().Warn("`timezone` is deprecated and useless anymore, please remove it.") - c.Timezone = "" + if _, err := utils.ParseTimeZone(c.Timezone); err != nil { + return err + } } if c.RemoveMeta { log.L().Warn("`remove-meta` in task config is deprecated, please use `start-task ... --remove-meta` instead") @@ -760,29 +759,25 @@ func AdjustTargetDBSessionCfg(dbConfig *DBConfig, version *semver.Version) { lowerMap[cfg.key] = cfg.val } } - // force set time zone to UTC - if tz, ok := lowerMap["time_zone"]; ok { - log.L().Warn("session variable 'time_zone' is overwritten with UTC timezone.", - zap.String("time_zone", tz)) - } - lowerMap["time_zone"] = defaultTimeZone dbConfig.Session = lowerMap } -// AdjustTargetDBTimeZone force adjust session `time_zone` to UTC. -func AdjustTargetDBTimeZone(config *DBConfig) { - for k := range config.Session { +// AdjustDBTimeZone force adjust session `time_zone`. +func AdjustDBTimeZone(config *DBConfig, timeZone string) { + for k, v := range config.Session { if strings.ToLower(k) == "time_zone" { - log.L().Warn("session variable 'time_zone' is overwritten by default UTC timezone.", - zap.String("time_zone", config.Session[k])) - config.Session[k] = defaultTimeZone + if v != timeZone { + log.L().Warn("session variable 'time_zone' is overwritten by task config's timezone", + zap.String("time_zone", config.Session[k])) + config.Session[k] = timeZone + } return } } if config.Session == nil { config.Session = make(map[string]string, 1) } - config.Session["time_zone"] = defaultTimeZone + config.Session["time_zone"] = timeZone } var defaultParser = parser.New() @@ -979,11 +974,6 @@ func NewTaskConfigForDowngrade(taskConfig *TaskConfig) *TaskConfigForDowngrade { // If any default value for new config item is not empty(0 or false or nil), // we should change it to empty. func (c *TaskConfigForDowngrade) omitDefaultVals() { - if len(c.TargetDB.Session) > 0 { - if timeZone, ok := c.TargetDB.Session["time_zone"]; ok && timeZone == defaultTimeZone { - delete(c.TargetDB.Session, "time_zone") - } - } if len(c.ShadowTableRules) == 1 && c.ShadowTableRules[0] == DefaultShadowTableRules { c.ShadowTableRules = nil } diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 4eebf42e024..92d5a91eb1c 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -50,6 +50,7 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]DBConfig) ([]* cfg.EnableHeartbeat = false cfg.HeartbeatUpdateInterval = c.HeartbeatUpdateInterval cfg.HeartbeatReportInterval = c.HeartbeatReportInterval + cfg.Timezone = c.Timezone cfg.Meta = inst.Meta fromClone := dbCfg.Clone() @@ -285,6 +286,7 @@ func SubTaskConfigsToTaskConfig(stCfgs ...*SubTaskConfig) *TaskConfig { c.EnableHeartbeat = stCfg0.EnableHeartbeat c.HeartbeatUpdateInterval = stCfg0.HeartbeatUpdateInterval c.HeartbeatReportInterval = stCfg0.HeartbeatReportInterval + c.Timezone = stCfg0.Timezone c.CaseSensitive = stCfg0.CaseSensitive c.TargetDB = &stCfg0.To // just ref c.OnlineDDL = stCfg0.OnlineDDL diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index 25149376df8..29b8e0b320d 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -910,22 +910,22 @@ func (t *testConfig) TestAdjustTargetDBConfig(c *C) { }{ { DBConfig{}, - DBConfig{Session: map[string]string{"time_zone": "+00:00"}}, + DBConfig{Session: map[string]string{}}, semver.New("0.0.0"), }, { DBConfig{Session: map[string]string{"SQL_MODE": "ANSI_QUOTES"}}, - DBConfig{Session: map[string]string{"sql_mode": "ANSI_QUOTES", "time_zone": "+00:00"}}, + DBConfig{Session: map[string]string{"sql_mode": "ANSI_QUOTES"}}, semver.New("2.0.7"), }, { DBConfig{}, - DBConfig{Session: map[string]string{tidbTxnMode: tidbTxnOptimistic, "time_zone": "+00:00"}}, + DBConfig{Session: map[string]string{tidbTxnMode: tidbTxnOptimistic}}, semver.New("3.0.1"), }, { DBConfig{Session: map[string]string{"SQL_MODE": "", tidbTxnMode: "pessimistic"}}, - DBConfig{Session: map[string]string{"sql_mode": "", tidbTxnMode: "pessimistic", "time_zone": "+00:00"}}, + DBConfig{Session: map[string]string{"sql_mode": "", tidbTxnMode: "pessimistic"}}, semver.New("4.0.0-beta.2"), }, } diff --git a/dm/dm/master/bootstrap.go b/dm/dm/master/bootstrap.go index acb0f8042c3..1b3609bc0f0 100644 --- a/dm/dm/master/bootstrap.go +++ b/dm/dm/master/bootstrap.go @@ -333,7 +333,7 @@ func (s *Server) upgradeDBSchemaV1Import(tctx *tcontext.Context, cfgs map[string return err } if targetDB == nil { - targetDB, err = conn.DefaultDBProvider.Apply(cfg2.To) + targetDB, err = conn.DefaultDBProvider.Apply(&cfg2.To) if err != nil { return err } diff --git a/dm/dm/master/bootstrap_test.go b/dm/dm/master/bootstrap_test.go index ab08fce15a9..e6df89f539a 100644 --- a/dm/dm/master/bootstrap_test.go +++ b/dm/dm/master/bootstrap_test.go @@ -93,6 +93,9 @@ func (t *testMaster) TestCollectSourceConfigFilesV1Import(c *C) { // collect again, two configs exist. cfgs, err = s.collectSourceConfigFilesV1Import(tctx) c.Assert(err, IsNil) + for _, cfg := range cfgs { + cfg.From.Session = nil + } c.Assert(cfgs, HasLen, 2) c.Assert(cfgs[cfg1.SourceID], DeepEquals, cfg1) c.Assert(cfgs[cfg2.SourceID], DeepEquals, cfg2) diff --git a/dm/dm/master/openapi.go b/dm/dm/master/openapi.go index 4cbb3290a55..2c598ca46ef 100644 --- a/dm/dm/master/openapi.go +++ b/dm/dm/master/openapi.go @@ -278,7 +278,7 @@ func (s *Server) getBaseDBBySourceName(sourceName string) (*conn.BaseDB, error) return nil, terror.ErrSchedulerSourceCfgNotExist.Generate(sourceName) } dbCfg := sourceCfg.GenerateDBConfig() - return conn.DefaultDBProvider.Apply(*dbCfg) + return conn.DefaultDBProvider.Apply(dbCfg) } // DMAPIGetSourceSchemaList get source schema list url is: (GET /api/v1/sources/{source-name}/schemas). diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index f29e14fb080..b11dac80510 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -646,16 +646,13 @@ func (s *Scheduler) TransferSource(source, worker string) error { // 2. check new worker is free and not started relay for another source switch w.Stage() { - case WorkerOffline: + case WorkerOffline, WorkerBound: return terror.ErrSchedulerWorkerInvalidTrans.Generate(worker, w.Stage(), WorkerBound) case WorkerFree: case WorkerRelay: if relaySource := w.RelaySourceID(); relaySource != source { return terror.ErrSchedulerBoundDiffWithStartedRelay.Generate(worker, source, relaySource) } - case WorkerBound: - s.logger.DPanic("worker should not be bound because we have checked it") - return terror.ErrSchedulerWorkerInvalidTrans.Generate(worker, w.Stage(), WorkerBound) } // 3. if no old worker, bound it directly diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index c4306158f89..fd9ebdb197b 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -1129,7 +1129,7 @@ func parseAndAdjustSourceConfig(ctx context.Context, contents []string) ([]*conf func checkAndAdjustSourceConfig(ctx context.Context, cfg *config.SourceConfig) error { dbConfig := cfg.GenerateDBConfig() - fromDB, err := conn.DefaultDBProvider.Apply(*dbConfig) + fromDB, err := conn.DefaultDBProvider.Apply(dbConfig) if err != nil { return err } @@ -1165,7 +1165,7 @@ func adjustTargetDB(ctx context.Context, dbConfig *config.DBConfig) error { failpoint.Return(nil) }) - toDB, err := conn.DefaultDBProvider.Apply(cfg) + toDB, err := conn.DefaultDBProvider.Apply(&cfg) if err != nil { return err } @@ -1182,7 +1182,6 @@ func adjustTargetDB(ctx context.Context, dbConfig *config.DBConfig) error { config.AdjustTargetDBSessionCfg(dbConfig, version) } else { log.L().Warn("get tidb version", log.ShortError(err)) - config.AdjustTargetDBTimeZone(dbConfig) } return nil } @@ -1439,7 +1438,7 @@ func (s *Server) removeMetaData(ctx context.Context, taskName, metaSchema string } // set up db and clear meta data in downstream db - baseDB, err := conn.DefaultDBProvider.Apply(*toDBCfg) + baseDB, err := conn.DefaultDBProvider.Apply(toDBCfg) if err != nil { return terror.WithScope(err, terror.ScopeDownstream) } diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index 2c1ca3c568b..eda57ea0332 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -698,7 +698,7 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk func (o *Optimist) removeLock(lock *optimism.Lock) (bool, error) { failpoint.Inject("SleepWhenRemoveLock", func(val failpoint.Value) { t := val.(int) - log.L().Info("wait new ddl info putted into etcd", + log.L().Info("wait new ddl info putted into etcd in optimistic", zap.String("failpoint", "SleepWhenRemoveLock"), zap.Int("max wait second", t)) diff --git a/dm/dm/master/shardddl/pessimist.go b/dm/dm/master/shardddl/pessimist.go index e20d228097c..e1f44df1e08 100644 --- a/dm/dm/master/shardddl/pessimist.go +++ b/dm/dm/master/shardddl/pessimist.go @@ -645,7 +645,7 @@ func (p *Pessimist) removeLock(lock *pessimism.Lock) error { failpoint.Inject("SleepWhenRemoveLock", func(val failpoint.Value) { t := val.(int) - log.L().Info("wait new ddl info putted into etcd", + log.L().Info("wait new ddl info putted into etcd in pessimistic", zap.String("failpoint", "SleepWhenRemoveLock"), zap.Int("max wait second", t)) diff --git a/dm/dm/unit/unit.go b/dm/dm/unit/unit.go index 788eea6f710..3b851d40d64 100644 --- a/dm/dm/unit/unit.go +++ b/dm/dm/unit/unit.go @@ -54,7 +54,7 @@ type Unit interface { // Resume resumes the paused process and its returning must send a result to pr channel. Resume(ctx context.Context, pr chan pb.ProcessResult) // Update updates the configuration - Update(cfg *config.SubTaskConfig) error + Update(ctx context.Context, cfg *config.SubTaskConfig) error // Status returns the unit's current status. The result may need calculation with source status, like estimated time // to catch up. If sourceStatus is nil, the calculation should be skipped. diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 9ff7097830d..11e626fda7d 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -134,7 +134,7 @@ func (w *SourceWorker) Start() { } var err error - w.sourceDB, err = conn.DefaultDBProvider.Apply(w.cfg.DecryptPassword().From) + w.sourceDB, err = conn.DefaultDBProvider.Apply(&w.cfg.DecryptPassword().From) if err != nil { w.l.Error("can't connected to upstream", zap.Error(err)) } @@ -230,7 +230,7 @@ func (w *SourceWorker) updateSourceStatus(ctx context.Context) error { w.sourceDBMu.Lock() if w.sourceDB == nil { var err error - w.sourceDB, err = conn.DefaultDBProvider.Apply(w.cfg.DecryptPassword().From) + w.sourceDB, err = conn.DefaultDBProvider.Apply(&w.cfg.DecryptPassword().From) if err != nil { w.sourceDBMu.Unlock() return err @@ -524,7 +524,7 @@ func (w *SourceWorker) StartSubTask(cfg *config.SubTaskConfig, expectStage pb.St } // UpdateSubTask update config for a sub task. -func (w *SourceWorker) UpdateSubTask(cfg *config.SubTaskConfig) error { +func (w *SourceWorker) UpdateSubTask(ctx context.Context, cfg *config.SubTaskConfig) error { w.Lock() defer w.Unlock() @@ -538,7 +538,7 @@ func (w *SourceWorker) UpdateSubTask(cfg *config.SubTaskConfig) error { } w.l.Info("update sub task", zap.String("task", cfg.Name)) - return st.Update(cfg) + return st.Update(ctx, cfg) } // OperateSubTask stop/resume/pause sub task. diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 8fff70dfcca..0d65f19284f 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -104,7 +104,7 @@ func (t *testServer) testWorker(c *C) { c.Assert(task, NotNil) c.Assert(task.Result().String(), Matches, ".*worker already closed.*") - err = w.UpdateSubTask(&config.SubTaskConfig{ + err = w.UpdateSubTask(context.Background(), &config.SubTaskConfig{ Name: "testStartTask", }) c.Assert(err, ErrorMatches, ".*worker already closed.*") @@ -196,6 +196,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { c.Assert(subtaskCfg.DecodeFile("./subtask.toml", true), IsNil) c.Assert(err, IsNil) subtaskCfg.Mode = "full" + subtaskCfg.Timezone = "UTC" c.Assert(s.getWorker(true).StartSubTask(&subtaskCfg, pb.Stage_Running, true), IsNil) // check task in paused state diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index c8205cfcfc7..81ef19296bb 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -549,14 +549,14 @@ func (st *SubTask) Resume() error { } // Update update the sub task's config. -func (st *SubTask) Update(cfg *config.SubTaskConfig) error { +func (st *SubTask) Update(ctx context.Context, cfg *config.SubTaskConfig) error { if !st.stageCAS(pb.Stage_Paused, pb.Stage_Paused) { // only test for Paused return terror.ErrWorkerUpdateTaskStage.Generate(st.Stage().String()) } // update all units' configuration, if SubTask itself has configuration need to update, do it later for _, u := range st.units { - err := u.Update(cfg) + err := u.Update(ctx, cfg) if err != nil { return err } diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 06b77b2a9eb..d53cdaac6b0 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -122,7 +122,7 @@ func (m MockUnit) Pause() {} func (m *MockUnit) Resume(ctx context.Context, pr chan pb.ProcessResult) { m.Process(ctx, pr) } -func (m *MockUnit) Update(_ *config.SubTaskConfig) error { +func (m *MockUnit) Update(context.Context, *config.SubTaskConfig) error { return m.errUpdate } @@ -229,7 +229,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { c.Assert(st.Stage(), Equals, pb.Stage_Running) // update in running - c.Assert(st.Update(nil), NotNil) + c.Assert(st.Update(context.Background(), nil), NotNil) c.Assert(st.CurrUnit(), Equals, mockLoader) c.Assert(st.Result(), IsNil) c.Assert(st.Stage(), Equals, pb.Stage_Running) @@ -243,7 +243,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { } // update again - c.Assert(st.Update(&config.SubTaskConfig{Name: "updateSubtask"}), IsNil) + c.Assert(st.Update(context.Background(), &config.SubTaskConfig{Name: "updateSubtask"}), IsNil) c.Assert(st.Stage(), Equals, pb.Stage_Paused) c.Assert(st.CurrUnit(), Equals, mockLoader) if st.Result() != nil && (!st.Result().IsCanceled || len(st.Result().Errors) > 0) { diff --git a/dm/dumpling/dumpling.go b/dm/dumpling/dumpling.go index f010ab472a0..05cf5280cc7 100644 --- a/dm/dumpling/dumpling.go +++ b/dm/dumpling/dumpling.go @@ -60,11 +60,9 @@ func NewDumpling(cfg *config.SubTaskConfig) *Dumpling { // Init implements Unit.Init. func (m *Dumpling) Init(ctx context.Context) error { var err error - if m.dumpConfig, err = m.constructArgs(); err != nil { + if m.dumpConfig, err = m.constructArgs(ctx); err != nil { return err } - m.detectSQLMode(ctx) - m.dumpConfig.SessionParams["time_zone"] = "+00:00" m.logger.Info("create dumpling", zap.Stringer("config", m.dumpConfig)) return nil } @@ -190,7 +188,7 @@ func (m *Dumpling) Resume(ctx context.Context, pr chan pb.ProcessResult) { } // Update implements Unit.Update. -func (m *Dumpling) Update(cfg *config.SubTaskConfig) error { +func (m *Dumpling) Update(context.Context, *config.SubTaskConfig) error { // not support update configuration now return nil } @@ -212,7 +210,7 @@ func (m *Dumpling) IsFreshTask(ctx context.Context) (bool, error) { } // constructArgs constructs arguments for exec.Command. -func (m *Dumpling) constructArgs() (*export.Config, error) { +func (m *Dumpling) constructArgs(ctx context.Context) (*export.Config, error) { cfg := m.cfg db := cfg.From @@ -233,9 +231,18 @@ func (m *Dumpling) constructArgs() (*export.Config, error) { dumpConfig.TableFilter = tableFilter dumpConfig.CompleteInsert = true // always keep column name in `INSERT INTO` statements. dumpConfig.Logger = m.logger.Logger - // force using UTC timezone + + tz := m.cfg.Timezone + if len(tz) == 0 { + // use target db time_zone as default + var err1 error + tz, err1 = conn.FetchTimeZoneSetting(ctx, &m.cfg.To) + if err1 != nil { + return nil, err1 + } + } dumpConfig.SessionParams = map[string]interface{}{ - "time_zone": "+00:00", + "time_zone": tz, } if cfg.Threads > 0 { @@ -293,15 +300,19 @@ func (m *Dumpling) constructArgs() (*export.Config, error) { } dumpConfig.Labels = prometheus.Labels{"task": m.cfg.Name, "source_id": m.cfg.SourceID} + // update sql_mode if needed + m.detectSQLMode(ctx, dumpConfig) return dumpConfig, nil } // detectSQLMode tries to detect SQL mode from upstream. If success, write it to LoaderConfig. // Because loader will use this SQL mode, we need to treat disable `EscapeBackslash` when NO_BACKSLASH_ESCAPES. -func (m *Dumpling) detectSQLMode(ctx context.Context) { - baseDB, err := conn.DefaultDBProvider.Apply(m.cfg.From) +func (m *Dumpling) detectSQLMode(ctx context.Context, dumpCfg *export.Config) { + baseDB, err := conn.DefaultDBProvider.Apply(&m.cfg.From) if err != nil { + log.L().Warn("set up db connect failed", zap.Any("db", m.cfg.From), + zap.Error(err)) return } defer baseDB.Close() @@ -309,13 +320,14 @@ func (m *Dumpling) detectSQLMode(ctx context.Context) { sqlMode, err := utils.GetGlobalVariable(ctx, db, "sql_mode") if err != nil { + log.L().Warn("get global sql_mode from upstream failed", zap.Any("db", m.cfg.From), zap.Error(err)) return } m.logger.Info("found upstream SQL mode", zap.String("SQL mode", sqlMode)) m.cfg.LoaderConfig.SQLMode = sqlMode if strings.Contains(sqlMode, "NO_BACKSLASH_ESCAPES") { - m.dumpConfig.EscapeBackslash = false + dumpCfg.EscapeBackslash = false } else { - m.dumpConfig.EscapeBackslash = true + dumpCfg.EscapeBackslash = true } } diff --git a/dm/dumpling/dumpling_test.go b/dm/dumpling/dumpling_test.go index c2db8fa65b4..6427353214d 100644 --- a/dm/dumpling/dumpling_test.go +++ b/dm/dumpling/dumpling_test.go @@ -47,8 +47,9 @@ type testDumplingSuite struct { func (d *testDumplingSuite) SetUpSuite(c *C) { dir := c.MkDir() d.cfg = &config.SubTaskConfig{ - Name: "dumpling_ut", - From: config.GetDBConfigForTest(), + Name: "dumpling_ut", + Timezone: "UTC", + From: config.GetDBConfigForTest(), LoaderConfig: config.LoaderConfig{ Dir: dir, }, diff --git a/dm/dumpling/util_test.go b/dm/dumpling/util_test.go index 62414bcc500..67e90123c98 100644 --- a/dm/dumpling/util_test.go +++ b/dm/dumpling/util_test.go @@ -14,29 +14,39 @@ package dumpling import ( + "context" "strings" + "github.com/DATA-DOG/go-sqlmock" "github.com/docker/go-units" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" tfilter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/ticdc/dm/pkg/conn" "github.com/pingcap/ticdc/dm/pkg/log" ) func (d *testDumplingSuite) TestParseArgs(c *C) { logger := log.L() + ctx := context.Background() + + mock := conn.InitMockDB(c) + mock.ExpectQuery("SELECT cast\\(TIMEDIFF\\(NOW\\(6\\), UTC_TIMESTAMP\\(6\\)\\) as time\\);"). + WillReturnRows(sqlmock.NewRows([]string{""}).AddRow("01:00:00")) cfg := &config.SubTaskConfig{} cfg.ExtraArgs = `--statement-size=100 --where "t>10" --threads 8 -F 50B` dumpling := NewDumpling(cfg) - exportCfg, err := dumpling.constructArgs() + exportCfg, err := dumpling.constructArgs(ctx) c.Assert(err, IsNil) c.Assert(exportCfg.StatementSize, Equals, uint64(100)) c.Assert(exportCfg.Where, Equals, "t>10") c.Assert(exportCfg.Threads, Equals, 8) c.Assert(exportCfg.FileSize, Equals, uint64(50)) + c.Assert(exportCfg.SessionParams, NotNil) + c.Assert(exportCfg.SessionParams["time_zone"], Equals, "+01:00") extraArgs := `--threads 16 --skip-tz-utc` err = parseExtraArgs(&logger, exportCfg, strings.Fields(extraArgs)) @@ -91,7 +101,9 @@ func (d *testDumplingSuite) TestParseArgs(c *C) { } func (d *testDumplingSuite) TestParseArgsWontOverwrite(c *C) { - cfg := &config.SubTaskConfig{} + cfg := &config.SubTaskConfig{ + Timezone: "UTC", + } cfg.ChunkFilesize = "1" rules := &filter.Rules{ DoDBs: []string{"unit_test"}, @@ -101,7 +113,7 @@ func (d *testDumplingSuite) TestParseArgsWontOverwrite(c *C) { cfg.ExtraArgs = "-s=4000 --consistency lock" dumpling := NewDumpling(cfg) - exportCfg, err := dumpling.constructArgs() + exportCfg, err := dumpling.constructArgs(context.Background()) c.Assert(err, IsNil) c.Assert(exportCfg.StatementSize, Equals, uint64(4000)) diff --git a/dm/loader/db.go b/dm/loader/db.go index eeeb2891bcc..d588c6a3626 100644 --- a/dm/loader/db.go +++ b/dm/loader/db.go @@ -197,7 +197,7 @@ func (conn *DBConn) resetConn(tctx *tcontext.Context) error { } func createConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, workerCount int) (*conn.BaseDB, []*DBConn, error) { - baseDB, err := conn.DefaultDBProvider.Apply(cfg.To) + baseDB, err := conn.DefaultDBProvider.Apply(&cfg.To) if err != nil { return nil, nil, terror.WithScope(err, terror.ScopeDownstream) } diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index df7a66788b6..e71fd4038b5 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -58,6 +58,7 @@ type LightningLoader struct { toDB *conn.BaseDB toDBConns []*DBConn lightningConfig *lcfg.GlobalConfig + timeZone string finish atomic.Bool closed atomic.Bool @@ -119,6 +120,15 @@ func (l *LightningLoader) Init(ctx context.Context) (err error) { }) l.checkPoint = checkpoint l.toDB, l.toDBConns, err = createConns(tctx, l.cfg, 1) + timeZone := l.cfg.Timezone + if len(timeZone) == 0 { + var err1 error + timeZone, err1 = conn.FetchTimeZoneSetting(ctx, &l.cfg.To) + if err1 != nil { + return err1 + } + } + l.timeZone = timeZone return err } @@ -162,6 +172,9 @@ func (l *LightningLoader) restore(ctx context.Context) error { } cfg.Checkpoint.DSN = param.ToDSN() cfg.TiDB.StrSQLMode = l.cfg.LoaderConfig.SQLMode + cfg.TiDB.Vars = map[string]string{ + "time_zone": l.timeZone, + } if err = cfg.Adjust(ctx); err != nil { return err } @@ -273,7 +286,7 @@ func (l *LightningLoader) Resume(ctx context.Context, pr chan pb.ProcessResult) // now, only support to update config for routes, filters, column-mappings, block-allow-list // now no config diff implemented, so simply re-init use new config // no binlog filter for loader need to update. -func (l *LightningLoader) Update(cfg *config.SubTaskConfig) error { +func (l *LightningLoader) Update(ctx context.Context, cfg *config.SubTaskConfig) error { // update l.cfg l.cfg.BAList = cfg.BAList l.cfg.RouteRules = cfg.RouteRules diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 35239edf0b4..2b80fba6ac7 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -441,8 +441,8 @@ type Loader struct { // to calculate remainingTimeGauge metric, map will be init in `l.prepare.prepareDataFiles` dbTableDataTotalSize map[string]map[string]*atomic.Int64 dbTableDataFinishedSize map[string]map[string]*atomic.Int64 - dbTableDataLastFinishedSize map[string]map[string]int64 - dbTableDataLastUpdatedTime time.Time + dbTableDataLastFinishedSize map[string]map[string]*atomic.Int64 + dbTableDataLastUpdatedTime atomic.Time metaBinlog atomic.String metaBinlogGTID atomic.String @@ -534,7 +534,15 @@ func (l *Loader) Init(ctx context.Context) (err error) { if lcfg.To.Session == nil { lcfg.To.Session = make(map[string]string) } - lcfg.To.Session["time_zone"] = "+00:00" + timeZone := l.cfg.Timezone + if len(timeZone) == 0 { + var err1 error + timeZone, err1 = conn.FetchTimeZoneSetting(ctx, &lcfg.To) + if err1 != nil { + return err1 + } + } + lcfg.To.Session["time_zone"] = timeZone hasSQLMode := false for k := range l.cfg.To.Session { @@ -861,7 +869,7 @@ func (l *Loader) resetDBs(ctx context.Context) error { // now, only support to update config for routes, filters, column-mappings, block-allow-list // now no config diff implemented, so simply re-init use new config // no binlog filter for loader need to update. -func (l *Loader) Update(cfg *config.SubTaskConfig) error { +func (l *Loader) Update(ctx context.Context, cfg *config.SubTaskConfig) error { var ( err error oldBaList *filter.Filter @@ -1045,12 +1053,12 @@ func (l *Loader) prepareDataFiles(files map[string]struct{}) error { if _, ok := l.dbTableDataTotalSize[db]; !ok { l.dbTableDataTotalSize[db] = make(map[string]*atomic.Int64) l.dbTableDataFinishedSize[db] = make(map[string]*atomic.Int64) - l.dbTableDataLastFinishedSize[db] = make(map[string]int64) + l.dbTableDataLastFinishedSize[db] = make(map[string]*atomic.Int64) } if _, ok := l.dbTableDataTotalSize[db][table]; !ok { l.dbTableDataTotalSize[db][table] = atomic.NewInt64(0) l.dbTableDataFinishedSize[db][table] = atomic.NewInt64(0) - l.dbTableDataLastFinishedSize[db][table] = 0 + l.dbTableDataLastFinishedSize[db][table] = atomic.NewInt64(0) } l.dbTableDataTotalSize[db][table].Add(size) @@ -1075,7 +1083,7 @@ func (l *Loader) prepare() error { l.finishedDataSize.Store(0) // reset before load from checkpoint l.dbTableDataTotalSize = make(map[string]map[string]*atomic.Int64) l.dbTableDataFinishedSize = make(map[string]map[string]*atomic.Int64) - l.dbTableDataLastFinishedSize = make(map[string]map[string]int64) + l.dbTableDataLastFinishedSize = make(map[string]map[string]*atomic.Int64) // check if mydumper dir data exists. if !utils.IsDirExists(l.cfg.Dir) { diff --git a/dm/loader/status.go b/dm/loader/status.go index fe525c6022d..33a619ea30f 100644 --- a/dm/loader/status.go +++ b/dm/loader/status.go @@ -44,19 +44,25 @@ func (l *Loader) printStatus() { totalSize := l.totalDataSize.Load() totalFileCount := l.totalFileCount.Load() - interval := time.Since(l.dbTableDataLastUpdatedTime) + interval := time.Since(l.dbTableDataLastUpdatedTime.Load()) + intervalSecond := interval.Seconds() + if intervalSecond == 0 { + return + } + for db, tables := range l.dbTableDataFinishedSize { for table, size := range tables { curFinished := size.Load() - speed := float64(curFinished-l.dbTableDataLastFinishedSize[db][table]) / interval.Seconds() - l.dbTableDataLastFinishedSize[db][table] = curFinished + lastFinished := l.dbTableDataFinishedSize[db][table].Load() + speed := float64(curFinished-lastFinished) / intervalSecond + l.dbTableDataLastFinishedSize[db][table].Store(curFinished) if speed > 0 { remainingSeconds := float64(l.dbTableDataTotalSize[db][table].Load()-curFinished) / speed remainingTimeGauge.WithLabelValues(l.cfg.Name, l.cfg.WorkerName, l.cfg.SourceID, db, table).Set(remainingSeconds) } } } - l.dbTableDataLastUpdatedTime = time.Now() + l.dbTableDataLastUpdatedTime.Store(time.Now()) l.logger.Info("progress status of load", zap.Int64("finished_bytes", finishedSize), diff --git a/dm/loader/status_test.go b/dm/loader/status_test.go new file mode 100644 index 00000000000..f756258dcc4 --- /dev/null +++ b/dm/loader/status_test.go @@ -0,0 +1,56 @@ +// 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 loader + +import ( + "sync" + + . "github.com/pingcap/check" + "go.uber.org/atomic" + + "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/ticdc/dm/pkg/log" +) + +func (*testLoaderSuite) TestConcurrentStatus(c *C) { + l := &Loader{} + l.cfg = &config.SubTaskConfig{} + l.logger = log.L() + l.finishedDataSize.Store(100) + l.totalDataSize.Store(200) + l.totalFileCount.Store(10) + l.dbTableDataFinishedSize = map[string]map[string]*atomic.Int64{ + "db1": { + "table1": atomic.NewInt64(10), + "table2": atomic.NewInt64(20), + }, + } + l.dbTableDataLastFinishedSize = map[string]map[string]*atomic.Int64{ + "db1": { + "table1": atomic.NewInt64(0), + "table2": atomic.NewInt64(0), + }, + } + + // test won't race or panic + wg := sync.WaitGroup{} + wg.Add(20) + for i := 0; i < 20; i++ { + go func() { + l.Status(nil) + wg.Done() + }() + } + wg.Wait() +} diff --git a/dm/pkg/conn/basedb.go b/dm/pkg/conn/basedb.go index 453cbb2aba2..de3566afe7b 100644 --- a/dm/pkg/conn/basedb.go +++ b/dm/pkg/conn/basedb.go @@ -39,7 +39,7 @@ var customID int64 // DBProvider providers BaseDB instance. type DBProvider interface { - Apply(config config.DBConfig) (*BaseDB, error) + Apply(config *config.DBConfig) (*BaseDB, error) } // DefaultDBProviderImpl is default DBProvider implement. @@ -56,7 +56,7 @@ func init() { var mockDB sqlmock.Sqlmock // Apply will build BaseDB with DBConfig. -func (d *DefaultDBProviderImpl) Apply(config config.DBConfig) (*BaseDB, error) { +func (d *DefaultDBProviderImpl) Apply(config *config.DBConfig) (*BaseDB, error) { // maxAllowedPacket=0 can be used to automatically fetch the max_allowed_packet variable from server on every connection. // https://github.com/go-sql-driver/mysql#maxallowedpacket dsn := fmt.Sprintf("%s:%s@tcp(%s:%d)/?charset=utf8mb4&interpolateParams=true&maxAllowedPacket=0", @@ -148,11 +148,11 @@ type BaseDB struct { Retry retry.Strategy // this function will do when close the BaseDB - doFuncInClose func() + doFuncInClose []func() } // NewBaseDB returns *BaseDB object. -func NewBaseDB(db *sql.DB, doFuncInClose func()) *BaseDB { +func NewBaseDB(db *sql.DB, doFuncInClose ...func()) *BaseDB { conns := make(map[*BaseConn]struct{}) return &BaseDB{DB: db, conns: conns, Retry: &retry.FiniteRetryStrategy{}, doFuncInClose: doFuncInClose} } @@ -197,7 +197,9 @@ func (d *BaseDB) Close() error { } } terr := d.DB.Close() - d.doFuncInClose() + for _, f := range d.doFuncInClose { + f() + } if err == nil { return terr diff --git a/dm/pkg/conn/basedb_test.go b/dm/pkg/conn/basedb_test.go index 614f6033f28..c6053a0bcc3 100644 --- a/dm/pkg/conn/basedb_test.go +++ b/dm/pkg/conn/basedb_test.go @@ -31,7 +31,7 @@ func (t *testBaseDBSuite) TestGetBaseConn(c *C) { db, mock, err := sqlmock.New() c.Assert(err, IsNil) - baseDB := NewBaseDB(db, func() {}) + baseDB := NewBaseDB(db) tctx := tcontext.Background() @@ -67,7 +67,7 @@ func (t *testBaseDBSuite) TestFailDBPing(c *C) { //nolint:errcheck defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/conn/failDBPing") - cfg := config.DBConfig{User: "root", Host: "127.0.0.1", Port: 3306} + cfg := &config.DBConfig{User: "root", Host: "127.0.0.1", Port: 3306} cfg.Adjust() db, err := DefaultDBProvider.Apply(cfg) c.Assert(db, IsNil) diff --git a/dm/pkg/conn/mockdb.go b/dm/pkg/conn/mockdb.go index b7160571138..7c1e95af018 100644 --- a/dm/pkg/conn/mockdb.go +++ b/dm/pkg/conn/mockdb.go @@ -28,14 +28,14 @@ type mockDBProvider struct { } // Apply will build BaseDB with DBConfig. -func (d *mockDBProvider) Apply(config config.DBConfig) (*BaseDB, error) { +func (d *mockDBProvider) Apply(config *config.DBConfig) (*BaseDB, error) { if d.verDB != nil { if err := d.verDB.Ping(); err == nil { // nolint:nilerr - return NewBaseDB(d.verDB, func() {}), nil + return NewBaseDB(d.verDB), nil } } - return NewBaseDB(d.db, func() {}), nil + return NewBaseDB(d.db), nil } // InitMockDB return a mocked db for unit test. diff --git a/dm/pkg/conn/utils.go b/dm/pkg/conn/utils.go new file mode 100644 index 00000000000..277fe573a00 --- /dev/null +++ b/dm/pkg/conn/utils.go @@ -0,0 +1,37 @@ +// 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 conn + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/dbutil" + + "github.com/pingcap/ticdc/dm/dm/config" +) + +// FetchTimeZoneSetting fetch target db global time_zone setting. +func FetchTimeZoneSetting(ctx context.Context, cfg *config.DBConfig) (string, error) { + db, err := DefaultDBProvider.Apply(cfg) + if err != nil { + return "", err + } + defer db.Close() + dur, err := dbutil.GetTimeZoneOffset(ctx, db.DB) + if err != nil { + return "", errors.Trace(err) + } + return dbutil.FormatTimeZoneOffset(dur), nil +} diff --git a/dm/pkg/conn/utils_test.go b/dm/pkg/conn/utils_test.go new file mode 100644 index 00000000000..93c208593f7 --- /dev/null +++ b/dm/pkg/conn/utils_test.go @@ -0,0 +1,36 @@ +// 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 conn + +import ( + "context" + + . "github.com/pingcap/check" + + "github.com/pingcap/ticdc/dm/dm/config" +) + +var _ = Suite(testUtilSuite{}) + +type testUtilSuite struct{} + +func (s testUtilSuite) TestFetchTZSetting(c *C) { + m := InitMockDB(c) + + m.ExpectQuery("SELECT cast\\(TIMEDIFF\\(NOW\\(6\\), UTC_TIMESTAMP\\(6\\)\\) as time\\);"). + WillReturnRows(m.NewRows([]string{""}).AddRow("01:00:00")) + tz, err := FetchTimeZoneSetting(context.Background(), &config.DBConfig{}) + c.Assert(err, IsNil) + c.Assert(tz, Equals, "+01:00") +} diff --git a/dm/pkg/streamer/reader.go b/dm/pkg/streamer/reader.go index 4d01ea0cd8c..f078f9e0d24 100644 --- a/dm/pkg/streamer/reader.go +++ b/dm/pkg/streamer/reader.go @@ -39,10 +39,6 @@ import ( "github.com/pingcap/ticdc/dm/pkg/utils" ) -// ErrorMaybeDuplicateEvent indicates that there may be duplicate event in next binlog file -// this is mainly happened when upstream master changed when relay log not finish reading a transaction. -var ErrorMaybeDuplicateEvent = errors.New("truncate binlog file found, event may be duplicated") - // Meta represents binlog meta information in relay.meta. type Meta struct { BinLogName string `toml:"binlog-name" json:"binlog-name"` @@ -560,15 +556,6 @@ func (r *BinlogReader) parseFile( if err != nil { if possibleLast && isIgnorableParseError(err) { r.tctx.L().Warn("fail to parse relay log file, meet some ignorable error", zap.String("file", fullPath), zap.Int64("offset", offset), zap.Error(err)) - // the file is truncated, we send a mock event with `IGNORABLE_EVENT` to notify the the consumer - // TODO: should add a integration test for this - e := &replication.BinlogEvent{ - RawData: []byte(ErrorMaybeDuplicateEvent.Error()), - Header: &replication.EventHeader{ - EventType: replication.IGNORABLE_EVENT, - }, - } - s.ch <- e } else { r.tctx.L().Error("parse relay log file", zap.String("file", fullPath), zap.Int64("offset", offset), zap.Error(err)) return false, false, 0, "", "", false, terror.ErrParserParseRelayLog.Delegate(err, fullPath) diff --git a/dm/pkg/streamer/streamer.go b/dm/pkg/streamer/streamer.go index a7e4b67db06..1398508300e 100644 --- a/dm/pkg/streamer/streamer.go +++ b/dm/pkg/streamer/streamer.go @@ -14,7 +14,6 @@ package streamer import ( - "bytes" "context" "time" @@ -67,12 +66,6 @@ func (s *LocalStreamer) GetEvent(ctx context.Context) (*replication.BinlogEvent, heartbeatHeader := &replication.EventHeader{} return event.GenHeartbeatEvent(heartbeatHeader), nil case c := <-s.ch: - // special check for maybe truncated relay log - if c.Header.EventType == replication.IGNORABLE_EVENT { - if bytes.Equal(c.RawData, []byte(ErrorMaybeDuplicateEvent.Error())) { - return nil, ErrorMaybeDuplicateEvent - } - } return c, nil case s.err = <-s.ech: return nil, s.err diff --git a/dm/pkg/upgrade/upgrade.go b/dm/pkg/upgrade/upgrade.go index a6e76239798..b4e9ce336a3 100644 --- a/dm/pkg/upgrade/upgrade.go +++ b/dm/pkg/upgrade/upgrade.go @@ -197,7 +197,7 @@ func upgradeToVer2(cli *clientv3.Client, uctx Context) error { defer cancel() for tableName, cfg := range dbConfigs { - targetDB, err := conn.DefaultDBProvider.Apply(cfg) + targetDB, err := conn.DefaultDBProvider.Apply(&cfg) if err != nil { logger.Error("target DB error when upgrading", zap.String("table name", tableName)) return err diff --git a/dm/pkg/utils/common.go b/dm/pkg/utils/common.go index 323a713c470..b7cd414cc85 100644 --- a/dm/pkg/utils/common.go +++ b/dm/pkg/utils/common.go @@ -20,7 +20,6 @@ import ( "regexp" "strings" "sync" - "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" @@ -324,20 +323,26 @@ func (se *session) GetBuiltinFunctionUsage() map[string]uint32 { return se.builtinFunctionUsage } -// UTCSession can be used as a sessionctx.Context, with UTC timezone. -var UTCSession *session +// NewSessionCtx return a session context with specified session variables. +func NewSessionCtx(vars map[string]string) sessionctx.Context { + variables := variable.NewSessionVars() + for k, v := range vars { + _ = variables.SetSystemVar(k, v) + if strings.EqualFold(k, "time_zone") { + loc, _ := ParseTimeZone(v) + variables.StmtCtx.TimeZone = loc + } + } -func init() { - UTCSession = &session{} - vars := variable.NewSessionVars() - vars.StmtCtx.TimeZone = time.UTC - UTCSession.vars = vars - UTCSession.values = make(map[fmt.Stringer]interface{}, 1) - UTCSession.builtinFunctionUsage = make(map[string]uint32) + return &session{ + vars: variables, + values: make(map[fmt.Stringer]interface{}, 1), + builtinFunctionUsage: make(map[string]uint32), + } } // AdjustBinaryProtocolForDatum converts the data in binlog to TiDB datum. -func AdjustBinaryProtocolForDatum(data []interface{}, cols []*model.ColumnInfo) ([]types.Datum, error) { +func AdjustBinaryProtocolForDatum(ctx sessionctx.Context, data []interface{}, cols []*model.ColumnInfo) ([]types.Datum, error) { log.L().Debug("AdjustBinaryProtocolForChunk", zap.Any("data", data), zap.Any("columns", cols)) @@ -362,9 +367,7 @@ func AdjustBinaryProtocolForDatum(data []interface{}, cols []*model.ColumnInfo) d = v.String() } datum := types.NewDatum(d) - - // TODO: should we use timezone of upstream? - castDatum, err := table.CastValue(UTCSession, datum, cols[i], false, false) + castDatum, err := table.CastValue(ctx, datum, cols[i], false, false) if err != nil { return nil, err } diff --git a/dm/pkg/utils/time.go b/dm/pkg/utils/time.go new file mode 100644 index 00000000000..7991f1e6f37 --- /dev/null +++ b/dm/pkg/utils/time.go @@ -0,0 +1,65 @@ +// 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 utils + +import ( + "strings" + "time" + + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/types" + + "github.com/pingcap/ticdc/dm/pkg/terror" +) + +// ParseTimeZone parse the time zone location by name or offset +// +// NOTE: we don't support the "SYSTEM" or "Local" time_zone. +func ParseTimeZone(s string) (*time.Location, error) { + if s == "SYSTEM" || s == "Local" { + return nil, terror.ErrConfigInvalidTimezone.New("'SYSTEM' or 'Local' time_zone is not supported") + } + + loc, err := time.LoadLocation(s) + if err == nil { + return loc, nil + } + + // The value can be given as a string indicating an offset from UTC, such as '+10:00' or '-6:00'. + // The time zone's value should in [-12:59,+14:00]. + // See: https://dev.mysql.com/doc/refman/8.0/en/time-zone-support.html#time-zone-variables + if strings.HasPrefix(s, "+") || strings.HasPrefix(s, "-") { + d, err := types.ParseDuration(nil, s[1:], 0) + if err == nil { + if s[0] == '-' { + if d.Duration > 12*time.Hour+59*time.Minute { + return nil, terror.ErrConfigInvalidTimezone.Generate(s) + } + } else { + if d.Duration > 14*time.Hour { + return nil, terror.ErrConfigInvalidTimezone.Generate(s) + } + } + + ofst := int(d.Duration / time.Second) + if s[0] == '-' { + ofst = -ofst + } + name := dbutil.FormatTimeZoneOffset(d.Duration) + return time.FixedZone(name, ofst), nil + } + } + + return nil, terror.ErrConfigInvalidTimezone.Generate(s) +} diff --git a/dm/pkg/v1dbschema/schema_test.go b/dm/pkg/v1dbschema/schema_test.go index ba4a4e213d5..f4a10e857c5 100644 --- a/dm/pkg/v1dbschema/schema_test.go +++ b/dm/pkg/v1dbschema/schema_test.go @@ -68,7 +68,7 @@ func (t *testSchema) setUpDBConn(c *C) { } t.password = os.Getenv("MYSQL_PSWD") - cfg := config.DBConfig{ + cfg := &config.DBConfig{ Host: t.host, Port: t.port, User: t.user, diff --git a/dm/relay/relay.go b/dm/relay/relay.go index 3db05c27de1..0a9bfa5da32 100644 --- a/dm/relay/relay.go +++ b/dm/relay/relay.go @@ -176,7 +176,7 @@ func (r *Relay) process(ctx context.Context) error { return err } - db, err := conn.DefaultDBProvider.Apply(r.cfg.From) + db, err := conn.DefaultDBProvider.Apply(&r.cfg.From) if err != nil { return terror.WithScope(err, terror.ScopeUpstream) } @@ -240,7 +240,7 @@ func (r *Relay) process(ctx context.Context) error { } } else { _, metaPos := r.meta.Pos() - if neededBinlogName > metaPos.Name { + if mysql.CompareBinlogFileName(neededBinlogName, metaPos.Name) > 0 { isRelayMetaOutdated = true } } @@ -307,8 +307,7 @@ func (r *Relay) process(ctx context.Context) error { // handles binlog events with retry mechanism. // it only do the retry for some binlog reader error now. for { - eventIdx, err := r.handleEvents(ctx, reader2, transformer2, writer2) - checkError: + err := r.handleEvents(ctx, reader2, transformer2, writer2) if err == nil { return nil } else if !readerRetry.Check(ctx, err) { @@ -325,33 +324,6 @@ func (r *Relay) process(ctx context.Context) error { return err } r.logger.Info("retrying to read binlog") - if r.cfg.EnableGTID && eventIdx > 0 { - // check if server has switched - isNew, err2 := isNewServer(ctx, r.meta.UUID(), r.db.DB, r.cfg.Flavor) - // should start from the transaction beginning when switch to a new server - if err2 != nil { - r.logger.Warn("check new server failed, continue outer loop", log.ShortError(err2)) - err = err2 - goto checkError - } - if !isNew { - for i := 0; i < eventIdx; { - res, err2 := reader2.GetEvent(ctx) - if err2 != nil { - err = err2 - goto checkError - } - tResult := transformer2.Transform(res.Event) - // do not count skip event - if !tResult.Ignore { - i++ - } - } - if eventIdx > 0 { - r.logger.Info("discard duplicate event", zap.Int("count", eventIdx)) - } - } - } } } @@ -465,16 +437,15 @@ func (r *Relay) handleEvents( reader2 reader.Reader, transformer2 transformer.Transformer, writer2 writer.Writer, -) (int, error) { +) error { var ( _, lastPos = r.meta.Pos() _, lastGTID = r.meta.GTID() err error - eventIndex int ) if lastGTID == nil { if lastGTID, err = gtid.ParserGTID(r.cfg.Flavor, ""); err != nil { - return 0, err + return err } } @@ -483,20 +454,10 @@ func (r *Relay) handleEvents( // 1. read events from upstream server readTimer := time.Now() rResult, err := reader2.GetEvent(ctx) - failpoint.Inject("RelayGetEventFailed", func(v failpoint.Value) { - if intVal, ok := v.(int); ok && intVal == eventIndex { - err = errors.New("fail point triggered") - _, gtid := r.meta.GTID() - r.logger.Warn("failed to get event", zap.Int("event_index", eventIndex), - zap.Any("gtid", gtid), log.ShortError(err)) - // wait backoff retry interval - time.Sleep(1 * time.Second) - } - }) if err != nil { switch errors.Cause(err) { case context.Canceled: - return 0, nil + return nil case replication.ErrChecksumMismatch: relayLogDataCorruptionCounter.Inc() case replication.ErrSyncClosed, replication.ErrNeedSyncAgain: @@ -515,7 +476,7 @@ func (r *Relay) handleEvents( } binlogReadErrorCounter.Inc() } - return eventIndex, err + return err } binlogReadDurationHistogram.Observe(time.Since(readTimer).Seconds()) @@ -547,15 +508,13 @@ func (r *Relay) handleEvents( if _, ok := e.Event.(*replication.RotateEvent); ok && utils.IsFakeRotateEvent(e.Header) { isNew, err2 := isNewServer(ctx, r.meta.UUID(), r.db.DB, r.cfg.Flavor) - // should start from the transaction beginning when switch to a new server if err2 != nil { - return 0, err2 + return err2 } // upstream database switch // report an error, let outer logic handle it - // should start from the transaction beginning when switch to a new server if isNew { - return 0, terror.ErrRotateEventWithDifferentServerID.Generate() + return terror.ErrRotateEventWithDifferentServerID.Generate() } } @@ -566,7 +525,7 @@ func (r *Relay) handleEvents( // and meta file is not created when relay resumed. firstEvent = false if err2 := r.saveAndFlushMeta(lastPos, lastGTID); err2 != nil { - return 0, err2 + return err2 } } @@ -576,7 +535,7 @@ func (r *Relay) handleEvents( wResult, err := writer2.WriteEvent(e) if err != nil { relayLogWriteErrorCounter.Inc() - return eventIndex, err + return err } else if wResult.Ignore { r.logger.Info("ignore event by writer", zap.Reflect("header", e.Header), @@ -595,7 +554,7 @@ func (r *Relay) handleEvents( lastPos.Pos = tResult.LogPos err = lastGTID.Set(tResult.GTIDSet) if err != nil { - return 0, terror.ErrRelayUpdateGTID.Delegate(err, lastGTID, tResult.GTIDSet) + return terror.ErrRelayUpdateGTID.Delegate(err, lastGTID, tResult.GTIDSet) } if !r.cfg.EnableGTID { // if go-mysql set RawModeEnabled to true @@ -620,17 +579,14 @@ func (r *Relay) handleEvents( if needSavePos { err = r.SaveMeta(lastPos, lastGTID) if err != nil { - return 0, terror.Annotatef(err, "save position %s, GTID sets %v into meta", lastPos, lastGTID) + return terror.Annotatef(err, "save position %s, GTID sets %v into meta", lastPos, lastGTID) } - eventIndex = 0 - } else { - eventIndex++ } if tResult.NextLogName != "" && !utils.IsFakeRotateEvent(e.Header) { // if the binlog is rotated, we need to save and flush the next binlog filename to meta lastPos.Name = tResult.NextLogName if err := r.saveAndFlushMeta(lastPos, lastGTID); err != nil { - return 0, err + return err } } } @@ -1007,7 +963,7 @@ func (r *Relay) Reload(newCfg *Config) error { r.cfg.From.RawDBCfg = config.DefaultRawDBConfig() } r.cfg.From.RawDBCfg.ReadTimeout = showStatusConnectionTimeout - db, err := conn.DefaultDBProvider.Apply(r.cfg.From) + db, err := conn.DefaultDBProvider.Apply(&r.cfg.From) if err != nil { return terror.WithScope(terror.DBErrorAdapt(err, terror.ErrDBDriverError), terror.ScopeUpstream) } diff --git a/dm/relay/relay_test.go b/dm/relay/relay_test.go index 3cb01ec80be..6e782e58e56 100644 --- a/dm/relay/relay_test.go +++ b/dm/relay/relay_test.go @@ -79,7 +79,7 @@ func newRelayCfg(c *C, flavor string) *Config { } } -func getDBConfigForTest() config.DBConfig { +func getDBConfigForTest() *config.DBConfig { host := os.Getenv("MYSQL_HOST") if host == "" { host = "127.0.0.1" @@ -93,7 +93,7 @@ func getDBConfigForTest() config.DBConfig { user = "root" } password := os.Getenv("MYSQL_PSWD") - return config.DBConfig{ + return &config.DBConfig{ Host: host, Port: port, User: user, @@ -441,9 +441,6 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { queryEv2 := queryEv.Event.(*replication.QueryEvent) queryEv2.GSet, _ = gmysql.ParseGTIDSet(relayCfg.Flavor, "1-2-3") - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) - defer cancel() - // reader return with an error for _, reader2.err = range []error{ errors.New("reader error for testing"), @@ -451,7 +448,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { replication.ErrSyncClosed, replication.ErrNeedSyncAgain, } { - _, handleErr := r.handleEvents(ctx, reader2, transformer2, writer2) + handleErr := r.handleEvents(context.Background(), reader2, transformer2, writer2) c.Assert(errors.Cause(handleErr), Equals, reader2.err) } @@ -461,7 +458,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { // writer return error to force handleEvents return writer2.err = errors.New("writer error for testing") // return with the annotated writer error - _, err = r.handleEvents(ctx, reader2, transformer2, writer2) + err = r.handleEvents(context.Background(), reader2, transformer2, writer2) c.Assert(errors.Cause(err), Equals, writer2.err) // after handle rotate event, we save and flush the meta immediately c.Assert(r.meta.Dirty(), Equals, false) @@ -480,7 +477,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { lm := r.meta.(*LocalMeta) backupUUID := lm.currentUUID lm.currentUUID = "not exist" - _, err = r.handleEvents(ctx, reader2, transformer2, writer2) + err = r.handleEvents(context.Background(), reader2, transformer2, writer2) c.Assert(os.IsNotExist(errors.Cause(err)), Equals, true) lm.currentUUID = backupUUID } @@ -492,14 +489,15 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { // writer return error writer2.err = errors.New("writer error for testing") // return with the annotated writer error - _, err = r.handleEvents(context.Background(), reader2, transformer2, writer2) + err = r.handleEvents(context.Background(), reader2, transformer2, writer2) c.Assert(errors.Cause(err), Equals, writer2.err) - // after handle rotate event, we save and flush the meta immediately c.Assert(r.meta.Dirty(), Equals, false) // writer without error + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + defer cancel() writer2.err = nil - _, err = r.handleEvents(ctx, reader2, transformer2, writer2) // returned when ctx timeout + err = r.handleEvents(ctx, reader2, transformer2, writer2) // returned when ctx timeout c.Assert(errors.Cause(err), Equals, ctx.Err()) // check written event c.Assert(writer2.latestEvent, Equals, reader2.result.Event) @@ -514,7 +512,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { // write a QueryEvent with GTID sets reader2.result.Event = queryEv - _, err = r.handleEvents(ctx2, reader2, transformer2, writer2) + err = r.handleEvents(ctx2, reader2, transformer2, writer2) c.Assert(errors.Cause(err), Equals, ctx.Err()) // check written event c.Assert(writer2.latestEvent, Equals, reader2.result.Event) @@ -533,7 +531,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { } ctx4, cancel4 := context.WithTimeout(context.Background(), 10*time.Millisecond) defer cancel4() - _, err = r.handleEvents(ctx4, reader2, transformer2, writer2) + err = r.handleEvents(ctx4, reader2, transformer2, writer2) c.Assert(errors.Cause(err), Equals, ctx.Err()) select { case <-ctx4.Done(): @@ -546,7 +544,7 @@ func (t *testRelaySuite) TestHandleEvent(c *C) { writer2.result.Ignore = true ctx5, cancel5 := context.WithTimeout(context.Background(), 10*time.Millisecond) defer cancel5() - _, err = r.handleEvents(ctx5, reader2, transformer2, writer2) + err = r.handleEvents(ctx5, reader2, transformer2, writer2) c.Assert(errors.Cause(err), Equals, ctx.Err()) select { case <-ctx5.Done(): diff --git a/dm/relay/retry/reader.go b/dm/relay/retry/reader.go index c9e37e00c95..b155aaf8b14 100644 --- a/dm/relay/retry/reader.go +++ b/dm/relay/retry/reader.go @@ -17,8 +17,6 @@ import ( "context" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/pkg/backoff" "github.com/pingcap/ticdc/dm/pkg/retry" "github.com/pingcap/ticdc/dm/pkg/terror" @@ -58,9 +56,6 @@ func NewReaderRetry(cfg ReaderRetryConfig) (*ReaderRetry, error) { // Check checks whether should retry for the error. func (rr *ReaderRetry) Check(ctx context.Context, err error) bool { - failpoint.Inject("RelayAllowRetry", func() { - failpoint.Return(true) - }) if !retry.IsConnectionError(err) { return false } diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 6778140556d..560fe80f778 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -313,7 +313,7 @@ func NewRemoteCheckPoint(tctx *tcontext.Context, cfg *config.SubTaskConfig, id s func (cp *RemoteCheckPoint) Init(tctx *tcontext.Context) error { checkPointDB := cp.cfg.To checkPointDB.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) - db, dbConns, err := dbconn.CreateConns(tctx, cp.cfg, checkPointDB, 1) + db, dbConns, err := dbconn.CreateConns(tctx, cp.cfg, &checkPointDB, 1) if err != nil { return err } diff --git a/dm/syncer/dbconn/db.go b/dm/syncer/dbconn/db.go index fdebaaa5614..1e5ff3bed2b 100644 --- a/dm/syncer/dbconn/db.go +++ b/dm/syncer/dbconn/db.go @@ -35,7 +35,7 @@ import ( var retryTimeout = 3 * time.Second // CreateBaseDB creates a db from config. -func CreateBaseDB(dbCfg config.DBConfig) (*conn.BaseDB, error) { +func CreateBaseDB(dbCfg *config.DBConfig) (*conn.BaseDB, error) { db, err := conn.DefaultDBProvider.Apply(dbCfg) if err != nil { return nil, terror.WithScope(err, terror.ScopeDownstream) @@ -233,7 +233,7 @@ func (conn *DBConn) ExecuteSQL(tctx *tcontext.Context, queries []string, args .. } // CreateConns returns a opened DB from dbCfg and number of `count` connections of that DB. -func CreateConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, dbCfg config.DBConfig, count int) (*conn.BaseDB, []*DBConn, error) { +func CreateConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, dbCfg *config.DBConfig, count int) (*conn.BaseDB, []*DBConn, error) { conns := make([]*DBConn, 0, count) baseDB, err := CreateBaseDB(dbCfg) if err != nil { diff --git a/dm/syncer/dbconn/upstream_db.go b/dm/syncer/dbconn/upstream_db.go index 4c85c3acab5..4303d198775 100644 --- a/dm/syncer/dbconn/upstream_db.go +++ b/dm/syncer/dbconn/upstream_db.go @@ -41,7 +41,7 @@ type UpStreamConn struct { } // NewUpStreamConn creates an UpStreamConn from config. -func NewUpStreamConn(dbCfg config.DBConfig) (*UpStreamConn, error) { +func NewUpStreamConn(dbCfg *config.DBConfig) (*UpStreamConn, error) { baseDB, err := CreateBaseDB(dbCfg) if err != nil { return nil, terror.WithScope(terror.DBErrorAdapt(err, terror.ErrDBDriverError), terror.ScopeUpstream) diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index 50771a99c9e..9b96f5c0f7d 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -96,7 +96,7 @@ RowLoop: } for _, expr := range filterExprs { - skip, err := SkipDMLByExpression(originalValue, expr, ti.Columns) + skip, err := SkipDMLByExpression(s.sessCtx, originalValue, expr, ti.Columns) if err != nil { return nil, err } @@ -155,11 +155,11 @@ RowLoop: for j := range oldValueFilters { // AND logic oldExpr, newExpr := oldValueFilters[j], newValueFilters[j] - skip1, err := SkipDMLByExpression(oriOldValues, oldExpr, ti.Columns) + skip1, err := SkipDMLByExpression(s.sessCtx, oriOldValues, oldExpr, ti.Columns) if err != nil { return nil, err } - skip2, err := SkipDMLByExpression(oriChangedValues, newExpr, ti.Columns) + skip2, err := SkipDMLByExpression(s.sessCtx, oriChangedValues, newExpr, ti.Columns) if err != nil { return nil, err } @@ -192,7 +192,7 @@ RowLoop: value := extractValueFromData(data, ti.Columns) for _, expr := range filterExprs { - skip, err := SkipDMLByExpression(value, expr, ti.Columns) + skip, err := SkipDMLByExpression(s.sessCtx, value, expr, ti.Columns) if err != nil { return nil, err } diff --git a/dm/syncer/expr_filter_group.go b/dm/syncer/expr_filter_group.go index 2df14135328..ee0904beb62 100644 --- a/dm/syncer/expr_filter_group.go +++ b/dm/syncer/expr_filter_group.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" "go.uber.org/zap" @@ -39,10 +40,12 @@ type ExprFilterGroup struct { hasUpdateOldFilter map[string]struct{} // set(tableName) hasUpdateNewFilter map[string]struct{} // set(tableName) hasDeleteFilter map[string]struct{} // set(tableName) + + ctx sessionctx.Context } // NewExprFilterGroup creates an ExprFilterGroup. -func NewExprFilterGroup(exprConfig []*config.ExpressionFilter) *ExprFilterGroup { +func NewExprFilterGroup(ctx sessionctx.Context, exprConfig []*config.ExpressionFilter) *ExprFilterGroup { ret := &ExprFilterGroup{ configs: map[string][]*config.ExpressionFilter{}, insertExprs: map[string][]expression.Expression{}, @@ -53,6 +56,7 @@ func NewExprFilterGroup(exprConfig []*config.ExpressionFilter) *ExprFilterGroup hasUpdateOldFilter: map[string]struct{}{}, hasUpdateNewFilter: map[string]struct{}{}, hasDeleteFilter: map[string]struct{}{}, + ctx: ctx, } for _, c := range exprConfig { tableName := dbutil.TableName(c.Schema, c.Table) @@ -88,7 +92,7 @@ func (g *ExprFilterGroup) GetInsertExprs(table *filter.Table, ti *model.TableInf for _, c := range g.configs[tableID] { if c.InsertValueExpr != "" { - expr, err2 := getSimpleExprOfTable(c.InsertValueExpr, ti) + expr, err2 := getSimpleExprOfTable(g.ctx, c.InsertValueExpr, ti) if err2 != nil { // TODO: terror return nil, err2 @@ -114,7 +118,7 @@ func (g *ExprFilterGroup) GetUpdateExprs(table *filter.Table, ti *model.TableInf if _, ok := g.hasUpdateOldFilter[tableID]; ok { for _, c := range g.configs[tableID] { if c.UpdateOldValueExpr != "" { - expr, err := getSimpleExprOfTable(c.UpdateOldValueExpr, ti) + expr, err := getSimpleExprOfTable(g.ctx, c.UpdateOldValueExpr, ti) if err != nil { // TODO: terror return nil, nil, err @@ -129,7 +133,7 @@ func (g *ExprFilterGroup) GetUpdateExprs(table *filter.Table, ti *model.TableInf if _, ok := g.hasUpdateNewFilter[tableID]; ok { for _, c := range g.configs[tableID] { if c.UpdateNewValueExpr != "" { - expr, err := getSimpleExprOfTable(c.UpdateNewValueExpr, ti) + expr, err := getSimpleExprOfTable(g.ctx, c.UpdateNewValueExpr, ti) if err != nil { // TODO: terror return nil, nil, err @@ -158,7 +162,7 @@ func (g *ExprFilterGroup) GetDeleteExprs(table *filter.Table, ti *model.TableInf for _, c := range g.configs[tableID] { if c.DeleteValueExpr != "" { - expr, err2 := getSimpleExprOfTable(c.DeleteValueExpr, ti) + expr, err2 := getSimpleExprOfTable(g.ctx, c.DeleteValueExpr, ti) if err2 != nil { // TODO: terror return nil, err2 @@ -179,10 +183,10 @@ func (g *ExprFilterGroup) ResetExprs(table *filter.Table) { } // SkipDMLByExpression returns true when given row matches the expr, which means this row should be skipped. -func SkipDMLByExpression(row []interface{}, expr expression.Expression, upstreamCols []*model.ColumnInfo) (bool, error) { +func SkipDMLByExpression(ctx sessionctx.Context, row []interface{}, expr expression.Expression, upstreamCols []*model.ColumnInfo) (bool, error) { // TODO: add metrics log.L().Debug("will evaluate the expression", zap.Stringer("expression", expr), zap.Any("raw row", row)) - data, err := utils.AdjustBinaryProtocolForDatum(row, upstreamCols) + data, err := utils.AdjustBinaryProtocolForDatum(ctx, row, upstreamCols) if err != nil { return false, err } @@ -196,9 +200,9 @@ func SkipDMLByExpression(row []interface{}, expr expression.Expression, upstream } // getSimpleExprOfTable returns an expression of given `expr` string, using the table structure that is tracked before. -func getSimpleExprOfTable(expr string, ti *model.TableInfo) (expression.Expression, error) { +func getSimpleExprOfTable(ctx sessionctx.Context, expr string, ti *model.TableInfo) (expression.Expression, error) { // TODO: use upstream timezone? - e, err := expression.ParseSimpleExprWithTableInfo(utils.UTCSession, expr, ti) + e, err := expression.ParseSimpleExprWithTableInfo(ctx, expr, ti) if err != nil { // if expression contains an unknown column, we return an expression that skips nothing if core.ErrUnknownColumn.Equal(err) { diff --git a/dm/syncer/expr_filter_group_test.go b/dm/syncer/expr_filter_group_test.go index 204859b255b..c86ab6cbb95 100644 --- a/dm/syncer/expr_filter_group_test.go +++ b/dm/syncer/expr_filter_group_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/ticdc/dm/dm/config" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/schema" + "github.com/pingcap/ticdc/dm/pkg/utils" ) func (s *testFilterSuite) TestSkipDMLByExpression(c *C) { @@ -107,17 +108,18 @@ create table t ( InsertValueExpr: ca.exprStr, }, } - g := NewExprFilterGroup(exprConfig) + sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) + g := NewExprFilterGroup(sessCtx, exprConfig) exprs, err := g.GetInsertExprs(table, ti) c.Assert(err, IsNil) c.Assert(exprs, HasLen, 1) expr := exprs[0] - skip, err := SkipDMLByExpression(ca.skippedRow, expr, ti.Columns) + skip, err := SkipDMLByExpression(sessCtx, ca.skippedRow, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, true) - skip, err = SkipDMLByExpression(ca.passedRow, expr, ti.Columns) + skip, err = SkipDMLByExpression(sessCtx, ca.passedRow, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, false) @@ -365,17 +367,18 @@ create table t ( InsertValueExpr: ca.exprStr, }, } - g := NewExprFilterGroup(exprConfig) + sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) + g := NewExprFilterGroup(sessCtx, exprConfig) exprs, err := g.GetInsertExprs(table, ti) c.Assert(err, IsNil) c.Assert(exprs, HasLen, 1) expr := exprs[0] - skip, err := SkipDMLByExpression(ca.skippedRow, expr, ti.Columns) + skip, err := SkipDMLByExpression(sessCtx, ca.skippedRow, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, true) - skip, err = SkipDMLByExpression(ca.passedRow, expr, ti.Columns) + skip, err = SkipDMLByExpression(sessCtx, ca.passedRow, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, false) @@ -413,7 +416,8 @@ create table t ( InsertValueExpr: exprStr, }, } - g := NewExprFilterGroup(exprConfig) + sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) + g := NewExprFilterGroup(sessCtx, exprConfig) exprs, err := g.GetInsertExprs(table, ti) c.Assert(err, IsNil) c.Assert(exprs, HasLen, 1) @@ -421,10 +425,10 @@ create table t ( c.Assert(expr.String(), Equals, "0") // skip nothing - skip, err := SkipDMLByExpression([]interface{}{0}, expr, ti.Columns) + skip, err := SkipDMLByExpression(sessCtx, []interface{}{0}, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, false) - skip, err = SkipDMLByExpression([]interface{}{2}, expr, ti.Columns) + skip, err = SkipDMLByExpression(sessCtx, []interface{}{2}, expr, ti.Columns) c.Assert(err, IsNil) c.Assert(skip, Equals, false) } diff --git a/dm/syncer/filter_test.go b/dm/syncer/filter_test.go index 7888a58609a..b927b6b3d6a 100644 --- a/dm/syncer/filter_test.go +++ b/dm/syncer/filter_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/ticdc/dm/pkg/conn" tcontext "github.com/pingcap/ticdc/dm/pkg/context" "github.com/pingcap/ticdc/dm/pkg/schema" + "github.com/pingcap/ticdc/dm/pkg/utils" "github.com/pingcap/ticdc/dm/syncer/dbconn" ) @@ -68,7 +69,7 @@ func (s *testFilterSuite) TestSkipQueryEvent(c *C) { syncer.ddlDBConn = &dbconn.DBConn{Cfg: syncer.cfg, BaseConn: s.baseConn} syncer.schemaTracker, err = schema.NewTracker(context.Background(), syncer.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) c.Assert(err, IsNil) - syncer.exprFilterGroup = NewExprFilterGroup(nil) + syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) // test binlog filter filterRules := []*bf.BinlogEventRule{ diff --git a/dm/syncer/handle_error_test.go b/dm/syncer/handle_error_test.go index b55bf5e9656..99430545bb9 100644 --- a/dm/syncer/handle_error_test.go +++ b/dm/syncer/handle_error_test.go @@ -65,7 +65,7 @@ func (s *testSyncerSuite) TestHandleError(c *C) { ) mockDB := conn.InitMockDB(c) var err error - syncer.fromDB, err = dbconn.NewUpStreamConn(s.cfg.From) // used to get parser + syncer.fromDB, err = dbconn.NewUpStreamConn(&s.cfg.From) // used to get parser c.Assert(err, IsNil) for _, cs := range cases { diff --git a/dm/syncer/online-ddl-tools/online_ddl.go b/dm/syncer/online-ddl-tools/online_ddl.go index a46f32e29f9..d485d6f6805 100644 --- a/dm/syncer/online-ddl-tools/online_ddl.go +++ b/dm/syncer/online-ddl-tools/online_ddl.go @@ -129,7 +129,7 @@ func NewOnlineDDLStorage(logCtx *tcontext.Context, cfg *config.SubTaskConfig) *S func (s *Storage) Init(tctx *tcontext.Context) error { onlineDB := s.cfg.To onlineDB.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) - db, dbConns, err := dbconn.CreateConns(tctx, s.cfg, onlineDB, 1) + db, dbConns, err := dbconn.CreateConns(tctx, s.cfg, &onlineDB, 1) if err != nil { return terror.WithScope(err, terror.ScopeDownstream) } diff --git a/dm/syncer/sharding_group.go b/dm/syncer/sharding_group.go index 72c3e97fb8c..447f94efb7b 100644 --- a/dm/syncer/sharding_group.go +++ b/dm/syncer/sharding_group.go @@ -431,7 +431,7 @@ func (k *ShardingGroupKeeper) Init() error { k.clear() sgkDB := k.cfg.To sgkDB.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) - db, dbConns, err := dbconn.CreateConns(k.tctx, k.cfg, sgkDB, 1) + db, dbConns, err := dbconn.CreateConns(k.tctx, k.cfg, &sgkDB, 1) if err != nil { return err } diff --git a/dm/syncer/sharding_group_test.go b/dm/syncer/sharding_group_test.go index db128fb96b7..96d5b8b1028 100644 --- a/dm/syncer/sharding_group_test.go +++ b/dm/syncer/sharding_group_test.go @@ -241,7 +241,7 @@ func (t *testShardingGroupSuite) TestKeeper(c *C) { c.Assert(err, IsNil) dbConn, err := db.Conn(context.Background()) c.Assert(err, IsNil) - k.db = conn.NewBaseDB(db, func() {}) + k.db = conn.NewBaseDB(db) k.dbConn = &dbconn.DBConn{Cfg: t.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} mock.ExpectBegin() diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 51bd6767c9e..96437d3bf67 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" "go.uber.org/zap" @@ -158,6 +159,7 @@ type Syncer struct { columnMapping *cm.Mapping baList *filter.Filter exprFilterGroup *ExprFilterGroup + sessCtx sessionctx.Context closed atomic.Bool @@ -243,7 +245,6 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, notifier syncer.lastCount.Store(0) syncer.count.Store(0) syncer.done = nil - syncer.setTimezone() syncer.addJobFunc = syncer.addJob syncer.enableRelay = cfg.UseRelay syncer.cli = etcdClient @@ -304,6 +305,9 @@ func (s *Syncer) Init(ctx context.Context) (err error) { }() tctx := s.tctx.WithContext(ctx) + if err = s.setTimezone(ctx); err != nil { + return + } err = s.setSyncCfg() if err != nil { @@ -333,7 +337,11 @@ func (s *Syncer) Init(ctx context.Context) (err error) { return terror.ErrSyncerUnitGenBinlogEventFilter.Delegate(err) } - s.exprFilterGroup = NewExprFilterGroup(s.cfg.ExprFilter) + vars := map[string]string{ + "time_zone": s.timezone.String(), + } + s.sessCtx = utils.NewSessionCtx(vars) + s.exprFilterGroup = NewExprFilterGroup(s.sessCtx, s.cfg.ExprFilter) if len(s.cfg.ColumnMappingRules) > 0 { s.columnMapping, err = cm.NewMapping(s.cfg.CaseSensitive, s.cfg.ColumnMappingRules) @@ -1546,28 +1554,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return nil } - maybeSkipNRowsEvent := func(n int) error { - if s.cfg.EnableGTID && n > 0 { - for i := 0; i < n; { - e, err1 := s.getEvent(tctx, currentLocation) - if err1 != nil { - return err - } - if _, ok := e.Event.(*replication.RowsEvent); ok { - i++ - } - } - log.L().Info("discard event already consumed", zap.Int("count", n), - zap.Any("cur_loc", currentLocation)) - } - return nil - } - - // eventIndex is the rows event index in this transaction, it's used to avoiding read duplicate event in gtid mode - eventIndex := 0 - // the relay log file may be truncated(not end with an RotateEvent), in this situation, we may read some rows events - // and then read from the gtid again, so we force enter safe-mode for one more transaction to avoid failure due to - // conflict for { if s.execError.Load() != nil { return nil @@ -1609,14 +1595,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { err = errors.New("connect: connection refused") } }) - failpoint.Inject("GetEventErrorInTxn", func(val failpoint.Value) { - if intVal, ok := val.(int); ok && intVal == eventIndex { - err = errors.New("failpoint triggered") - s.tctx.L().Warn("failed to get event", zap.Int("event_index", eventIndex), - zap.Any("cur_pos", currentLocation), zap.Any("las_pos", lastLocation), - zap.Any("pos", e.Header.LogPos), log.ShortError(err)) - } - }) switch { case err == context.Canceled: tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) @@ -1632,13 +1610,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return err1 } continue - case err == streamer.ErrorMaybeDuplicateEvent: - tctx.L().Warn("read binlog met a truncated file, need to open safe-mode until the next transaction") - err = maybeSkipNRowsEvent(eventIndex) - if err == nil { - continue - } - log.L().Warn("skip duplicate rows event failed", zap.Error(err)) } if err != nil { @@ -1649,15 +1620,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } if s.streamerController.CanRetry(err) { - // GlobalPoint is the last finished GTID - err = s.streamerController.ResetReplicationSyncer(tctx, s.checkpoint.GlobalPoint()) + // lastLocation is the last finished GTID + err = s.streamerController.ResetReplicationSyncer(tctx, lastLocation) if err != nil { return err } log.L().Info("reset replication binlog puller", zap.Any("pos", s.checkpoint.GlobalPoint())) - if err = maybeSkipNRowsEvent(eventIndex); err != nil { - return err - } continue } @@ -1809,15 +1777,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { case *replication.RotateEvent: err2 = s.handleRotateEvent(ev, ec) case *replication.RowsEvent: - eventIndex++ metrics.BinlogEventRowHistogram.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID).Observe(float64(len(ev.Rows))) err2 = s.handleRowsEvent(ev, ec) case *replication.QueryEvent: originSQL = strings.TrimSpace(string(ev.Query)) err2 = s.handleQueryEvent(ev, ec, originSQL) case *replication.XIDEvent: - // reset eventIndex and force safeMode flag here. - eventIndex = 0 if shardingReSync != nil { shardingReSync.currLocation.Position.Pos = e.Header.LogPos shardingReSync.currLocation.Suffix = currentLocation.Suffix @@ -2852,7 +2817,7 @@ func (s *Syncer) createDBs(ctx context.Context) error { var err error dbCfg := s.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) - s.fromDB, err = dbconn.NewUpStreamConn(dbCfg) + s.fromDB, err = dbconn.NewUpStreamConn(&dbCfg) if err != nil { return err } @@ -2895,7 +2860,7 @@ func (s *Syncer) createDBs(ctx context.Context) error { SetReadTimeout(maxDMLConnectionTimeout). SetMaxIdleConns(s.cfg.WorkerCount) - s.toDB, s.toDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, dbCfg, s.cfg.WorkerCount) + s.toDB, s.toDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, &dbCfg, s.cfg.WorkerCount) if err != nil { dbconn.CloseUpstreamConn(s.tctx, s.fromDB) // release resources acquired before return with error return err @@ -2905,7 +2870,7 @@ func (s *Syncer) createDBs(ctx context.Context) error { dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDDLConnectionTimeout) var ddlDBConns []*dbconn.DBConn - s.ddlDB, ddlDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, dbCfg, 1) + s.ddlDB, ddlDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, &dbCfg, 1) if err != nil { dbconn.CloseUpstreamConn(s.tctx, s.fromDB) dbconn.CloseBaseDB(s.tctx, s.toDB) @@ -3060,7 +3025,7 @@ func (s *Syncer) Resume(ctx context.Context, pr chan pb.ProcessResult) { // Update implements Unit.Update // now, only support to update config for routes, filters, column-mappings, block-allow-list // now no config diff implemented, so simply re-init use new config. -func (s *Syncer) Update(cfg *config.SubTaskConfig) error { +func (s *Syncer) Update(ctx context.Context, cfg *config.SubTaskConfig) error { if s.cfg.ShardMode == config.ShardPessimistic { _, tables := s.sgk.UnresolvedTables() if len(tables) > 0 { @@ -3148,8 +3113,9 @@ func (s *Syncer) Update(cfg *config.SubTaskConfig) error { s.cfg.ColumnMappingRules = cfg.ColumnMappingRules // update timezone - s.setTimezone() - + if s.timezone == nil { + return s.setTimezone(ctx) + } return nil } @@ -3180,7 +3146,7 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { var err error s.cfg.From.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) - s.fromDB, err = dbconn.NewUpStreamConn(s.cfg.From) + s.fromDB, err = dbconn.NewUpStreamConn(&s.cfg.From) if err != nil { s.tctx.L().Error("fail to create baseConn connection", log.ShortError(err)) return err @@ -3197,9 +3163,22 @@ func (s *Syncer) UpdateFromConfig(cfg *config.SubTaskConfig) error { return nil } -func (s *Syncer) setTimezone() { - s.tctx.L().Info("use timezone", log.WrapStringerField("location", time.UTC)) - s.timezone = time.UTC +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 { diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index fdcdd868d6e..6bfb8564b68 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/ticdc/dm/pkg/retry" "github.com/pingcap/ticdc/dm/pkg/schema" streamer2 "github.com/pingcap/ticdc/dm/pkg/streamer" + "github.com/pingcap/ticdc/dm/pkg/utils" "github.com/pingcap/ticdc/dm/syncer/dbconn" sqlmock "github.com/DATA-DOG/go-sqlmock" @@ -772,14 +773,14 @@ func (s *testSyncerSuite) TestRun(c *C) { c.Assert(err, IsNil) syncer := NewSyncer(cfg, nil, nil) syncer.cfg.CheckpointFlushInterval = 30 - syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db, func() {})} + syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db)} syncer.toDBConns = []*dbconn.DBConn{ {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, } syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) - syncer.exprFilterGroup = NewExprFilterGroup(nil) + syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) c.Assert(err, IsNil) c.Assert(syncer.Type(), Equals, pb.UnitType_Sync) @@ -926,7 +927,12 @@ func (s *testSyncerSuite) TestRun(c *C) { cancel() // when syncer exit Run(), will flush job syncer.Pause() - c.Assert(syncer.Update(s.cfg), IsNil) + + mockDBProvider := conn.InitMockDB(c) + mockDBProvider.ExpectQuery("SELECT cast\\(TIMEDIFF\\(NOW\\(6\\), UTC_TIMESTAMP\\(6\\)\\) as time\\);"). + WillReturnRows(sqlmock.NewRows([]string{""}).AddRow("01:00:00")) + c.Assert(syncer.Update(context.Background(), s.cfg), IsNil) + c.Assert(syncer.timezone.String(), Equals, "+01:00") events2 := mockBinlogEvents{ mockBinlogEvent{typ: Write, args: []interface{}{uint64(8), "test_1", "t_1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), "c"}}}}, @@ -1014,14 +1020,14 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { cfg, err := s.cfg.Clone() c.Assert(err, IsNil) syncer := NewSyncer(cfg, nil, nil) - syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db, func() {})} + syncer.fromDB = &dbconn.UpStreamConn{BaseDB: conn.NewBaseDB(db)} syncer.toDBConns = []*dbconn.DBConn{ {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, } syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) - syncer.exprFilterGroup = NewExprFilterGroup(nil) + syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) c.Assert(err, IsNil) c.Assert(syncer.Type(), Equals, pb.UnitType_Sync) @@ -1210,7 +1216,7 @@ func (s *testSyncerSuite) TestTrackDDL(c *C) { syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} syncer.checkpoint.(*RemoteCheckPoint).dbConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(checkPointDBConn, &retry.FiniteRetryStrategy{})} syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) - syncer.exprFilterGroup = NewExprFilterGroup(nil) + syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) c.Assert(syncer.genRouter(), IsNil) c.Assert(err, IsNil) diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index de838513889..04e3f298fcd 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -10,6 +10,13 @@ function cleanup_data() { run_sql "drop database if exists dm_meta" $TIDB_PORT $TIDB_PASSWORD } +function cleanup_data_upstream() { + for target_db in "$@"; do + run_sql "drop database if exists \`${target_db}\`" $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql "drop database if exists \`${target_db}\`" $MYSQL_PORT2 $MYSQL_PASSWORD2 + done +} + function cleanup_process() { dm_master_num=$(ps aux >temp && grep "dm-master.test" temp | wc -l && rm temp) echo "$dm_master_num dm-master alive" diff --git a/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml b/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml new file mode 100644 index 00000000000..455ed32451f --- /dev/null +++ b/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml @@ -0,0 +1,31 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql1" + + target-check-tables = ["adjust_gtid.t?*"] + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml b/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml new file mode 100644 index 00000000000..6d38ad7fb17 --- /dev/null +++ b/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml @@ -0,0 +1,31 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql2" + + target-check-tables = ["adjust_gtid.t?*"] + + +[data-sources] +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/adjust_gtid/run.sh b/dm/tests/adjust_gtid/run.sh index 751f2fca8a3..f51a28a6e9d 100755 --- a/dm/tests/adjust_gtid/run.sh +++ b/dm/tests/adjust_gtid/run.sh @@ -84,10 +84,13 @@ function run() { # avoid cannot unmarshal !!str `binlog-...` into uint32 error sed -i "s/binlog-pos-placeholder-1/4/g" $WORK_DIR/dm-task.yaml sed -i "s/binlog-pos-placeholder-2/4/g" $WORK_DIR/dm-task.yaml - dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" + # start DM task. don't check error because it will meet injected error soon + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml --remove-meta" # use sync_diff_inspector to check full dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml name1=$(grep "Log: " $WORK_DIR/worker1/dumped_data.$TASK_NAME/metadata | awk -F: '{print $2}' | tr -d ' ') pos1=$(grep "Pos: " $WORK_DIR/worker1/dumped_data.$TASK_NAME/metadata | awk -F: '{print $2}' | tr -d ' ') @@ -129,7 +132,8 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # use sync_diff_inspector to check incremental dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml run_sql_both_source "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'" run_sql_both_source "SET @@global.time_zone = 'SYSTEM';" diff --git a/dm/tests/all_mode/conf/dm-task.yaml b/dm/tests/all_mode/conf/dm-task.yaml index ddc192017f7..17553c1b0f7 100644 --- a/dm/tests/all_mode/conf/dm-task.yaml +++ b/dm/tests/all_mode/conf/dm-task.yaml @@ -3,6 +3,7 @@ name: test task-mode: all is-sharding: false meta-schema: "dm_meta" +timezone: "+04:00" # enable-heartbeat: true heartbeat-update-interval: 1 heartbeat-report-interval: 1 diff --git a/dm/tests/all_mode/data/db1.increment0.sql b/dm/tests/all_mode/data/db1.increment0.sql index fa406496ae0..8ed3117814a 100644 --- a/dm/tests/all_mode/data/db1.increment0.sql +++ b/dm/tests/all_mode/data/db1.increment0.sql @@ -1,3 +1,4 @@ SET @@SESSION.TIME_ZONE = "+03:00"; use all_mode; +insert into no_diff (id) values (5); insert into t1 (id, name, ts) values (100, 'Eddard Stark', '2021-05-11 12:01:05'); diff --git a/dm/tests/all_mode/data/db1.prepare.sql b/dm/tests/all_mode/data/db1.prepare.sql index eef8893e84c..4bf15276eba 100644 --- a/dm/tests/all_mode/data/db1.prepare.sql +++ b/dm/tests/all_mode/data/db1.prepare.sql @@ -13,8 +13,13 @@ insert into t1 (id, name, dt, ts) values (1, 'ar"ya', now(), now()), (2, 'cately -- test sql_mode=NO_AUTO_VALUE_ON_ZERO insert into t1 (id, name) values (0, 'lalala'); +-- test downstream schema contains extra datetime/timestamp column +-- do not use sync-diff to check this table +create table no_diff(id int NOT NULL PRIMARY KEY); +insert into no_diff (id) values (1), (2), (3); + -- test block-allow-list drop database if exists `ignore_db`; create database `ignore_db`; use `ignore_db`; -create table `ignore_table`(id int); \ No newline at end of file +create table `ignore_table`(id int); diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index c9f4c82c63e..3d57c1fef24 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -335,6 +335,12 @@ function run() { ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" + # manually create target table with two extra field + run_sql_tidb "drop database if exists all_mode;" + run_sql_tidb "create database all_mode;" + run_sql_tidb "drop table if exists all_mode.no_diff;" + run_sql_tidb "create table all_mode.no_diff(id int NOT NULL PRIMARY KEY, dt datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, ts timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP);" + run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 @@ -374,10 +380,22 @@ function run() { # use sync_diff_inspector to check full dump loader check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + run_sql_tidb "set time_zone = '+04:00';SELECT count(*) from all_mode.no_diff where dt = ts;" + check_contains "count(*): 3" + # check default session config check_log_contain_with_retry '\\"tidb_txn_mode\\":\\"optimistic\\"' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry '\\"tidb_txn_mode\\":\\"optimistic\\"' $WORK_DIR/worker2/log/dm-worker.log + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "pause-task $ILLEGAL_CHAR_NAME" \ + "\"result\": true" 3 + echo 'create table all_mode.no_diff(id int NOT NULL PRIMARY KEY);' >${WORK_DIR}/schema.sql + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 $ILLEGAL_CHAR_NAME -d all_mode -t no_diff ${WORK_DIR}/schema.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" "resume-task $ILLEGAL_CHAR_NAME" "\"result\": true" 3 + # restart dm-worker1 pkill -hup -f dm-worker1.toml 2>/dev/null || true wait_pattern_exit dm-worker1.toml @@ -427,6 +445,10 @@ function run() { check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + # check compatibility after incremental sync + run_sql_tidb "set time_zone = '+04:00';SELECT count(*) from all_mode.no_diff where dt = ts;" + check_contains "count(*): 4" + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "pause-relay -s mysql-replica-01" \ "\"result\": true" 2 diff --git a/dm/tests/dm_syncer/run.sh b/dm/tests/dm_syncer/run.sh index 1ed98da1d8a..11994fdb95c 100755 --- a/dm/tests/dm_syncer/run.sh +++ b/dm/tests/dm_syncer/run.sh @@ -71,7 +71,7 @@ function run() { run_dm_syncer $WORK_DIR/syncer2 $WORK_DIR/dm-syncer-2.toml $meta_file --syncer-config-format syncer2 # wait for dm_syncer to init and start - sleep 5 + sleep 10 check_sync_diff $WORK_DIR $cur/conf/diff_config.toml check_sync_diff $WORK_DIR $cur/conf/diff_config_blalist.toml check_sync_diff $WORK_DIR $cur/conf/diff_config_route_rules.toml diff --git a/dm/tests/dmctl_basic/conf/diff_config.toml b/dm/tests/dmctl_basic/conf/diff_config.toml index 55d568baf4b..7f3079908c9 100644 --- a/dm/tests/dmctl_basic/conf/diff_config.toml +++ b/dm/tests/dmctl_basic/conf/diff_config.toml @@ -19,8 +19,7 @@ check-struct-only = false [table-configs] [table-configs.config1] -schema = "dmctl" -table = "t_target" +target-tables = ["dmctl.t_target"] ignore-columns = ["id"] [routes.rule1] diff --git a/dm/tests/dmctl_basic/conf/get_source1.yaml b/dm/tests/dmctl_basic/conf/get_source1.yaml index da750e6c0b6..ac79c9d01e0 100644 --- a/dm/tests/dmctl_basic/conf/get_source1.yaml +++ b/dm/tests/dmctl_basic/conf/get_source1.yaml @@ -14,8 +14,7 @@ from: user: root password: '******' max-allowed-packet: null - session: - time_zone: "+00:00" + session: {} security: null purge: interval: 3600 diff --git a/dm/tests/dmctl_basic/conf/get_source2.yaml b/dm/tests/dmctl_basic/conf/get_source2.yaml index 114d04acc6e..2948f2f878c 100644 --- a/dm/tests/dmctl_basic/conf/get_source2.yaml +++ b/dm/tests/dmctl_basic/conf/get_source2.yaml @@ -14,8 +14,7 @@ from: user: root password: '******' max-allowed-packet: null - session: - time_zone: "+00:00" + session: {} security: null purge: interval: 3600 diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index cac039ec9e9..e26a00dae17 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -17,7 +17,6 @@ target-database: max-allowed-packet: null session: tidb_txn_mode: optimistic - time_zone: "+00:00" security: null mysql-instances: - source-id: mysql-replica-01 diff --git a/dm/tests/full_mode/conf/diff_config_revert_1.toml b/dm/tests/full_mode/conf/diff_config_revert_1.toml new file mode 100644 index 00000000000..f690ce6e804 --- /dev/null +++ b/dm/tests/full_mode/conf/diff_config_revert_1.toml @@ -0,0 +1,30 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql1" + + target-check-tables = ["full_mode.t?*"] + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/full_mode/conf/diff_config_revert_2.toml b/dm/tests/full_mode/conf/diff_config_revert_2.toml new file mode 100644 index 00000000000..84a3ecd2d43 --- /dev/null +++ b/dm/tests/full_mode/conf/diff_config_revert_2.toml @@ -0,0 +1,30 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql2" + + target-check-tables = ["full_mode.t?*"] + + +[data-sources] +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/full_mode/run.sh b/dm/tests/full_mode/run.sh index 714a9e8a610..30a8344eadc 100755 --- a/dm/tests/full_mode/run.sh +++ b/dm/tests/full_mode/run.sh @@ -60,6 +60,7 @@ function fail_acquire_global_lock() { "you need (at least one of) the RELOAD privilege(s) for this operation" 2 cleanup_data full_mode + cleanup_data_upstream full_mode cleanup_process $* } @@ -104,7 +105,6 @@ function escape_schema() { # start DM task only dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" - check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml check_log_contain_with_retry 'clean dump files' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'clean dump files' $WORK_DIR/worker2/log/dm-worker.log @@ -118,6 +118,7 @@ function escape_schema() { check_metric $WORKER2_PORT 'dumpling_dump_finished_tables' 3 0 3 cleanup_data full/mode + cleanup_data_upstream full/mode cleanup_process $* } @@ -130,7 +131,8 @@ function empty_data() { init_cluster dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ @@ -142,6 +144,7 @@ function empty_data() { check_log_contains $WORK_DIR/worker2/log/dm-worker.log "progress=\"100.00 %\"" cleanup_data full_mode + cleanup_data_upstream full_mode cleanup_process $* } @@ -190,7 +193,8 @@ function run() { dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" # use sync_diff_inspector to check full dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml echo "check dump files have been cleaned" ls $WORK_DIR/worker1/dumped_data.test && exit 1 || echo "worker1 auto removed dump files" diff --git a/dm/tests/ha_cases2/run.sh b/dm/tests/ha_cases2/run.sh index 12a62f60826..5d3c478f155 100755 --- a/dm/tests/ha_cases2/run.sh +++ b/dm/tests/ha_cases2/run.sh @@ -92,12 +92,12 @@ function test_pause_task() { for name in ${task_name[@]}; do echo "resume tasks $name" - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "resume-task $name" \ "\"result\": true" 3 # resume twice, just used to test resume by the way - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "resume-task $name" \ "\"result\": true" 3 diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index ec35bde56a1..b39ea4bd7a0 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -7,7 +7,7 @@ meta-schema: dm_meta enable-heartbeat: false heartbeat-update-interval: 1 heartbeat-report-interval: 1 -timezone: "" +timezone: Asia/Shanghai case-sensitive: false target-database: host: 127.0.0.1 @@ -19,7 +19,7 @@ target-database: tidb_disable_txn_auto_retry: "off" tidb_retry_limit: "10" tidb_skip_utf8_check: "1" - time_zone: "+00:00" + time_zone: Asia/Shanghai security: null mysql-instances: - source-id: mysql-replica-01 diff --git a/dm/tests/incremental_mode/conf/diff_config_revert_1.toml b/dm/tests/incremental_mode/conf/diff_config_revert_1.toml new file mode 100644 index 00000000000..45328c12217 --- /dev/null +++ b/dm/tests/incremental_mode/conf/diff_config_revert_1.toml @@ -0,0 +1,30 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql1" + + target-check-tables = ["incremental_mode.t?*"] + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/incremental_mode/conf/diff_config_revert_2.toml b/dm/tests/incremental_mode/conf/diff_config_revert_2.toml new file mode 100644 index 00000000000..16ed402a93f --- /dev/null +++ b/dm/tests/incremental_mode/conf/diff_config_revert_2.toml @@ -0,0 +1,30 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["tidb0"] + + target-instance = "mysql2" + + target-check-tables = ["incremental_mode.t?*"] + + +[data-sources] +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/incremental_mode/run.sh b/dm/tests/incremental_mode/run.sh index 40774b7a469..b8b7dbbf171 100755 --- a/dm/tests/incremental_mode/run.sh +++ b/dm/tests/incremental_mode/run.sh @@ -76,7 +76,8 @@ function run() { sed -i "s/binlog-pos-placeholder-2/4/g" $WORK_DIR/dm-task.yaml dmctl_start_task $WORK_DIR/dm-task.yaml - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml dmctl_stop_task $TASK_NAME @@ -218,7 +219,9 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT sleep 3 - dmctl_start_task $WORK_DIR/dm-task.yaml + # start DM task. don't check error because it will meet injected error soon + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/dm-task.yaml" # the task should paused by `FlushCheckpointStage` failpont before flush old checkpoint. # `db2.increment.sql` has no DDL, so we check count of content as `1`. @@ -263,7 +266,8 @@ function run() { "resume-task test" \ "\"result\": true" 3 - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml # test rotate binlog, after rotate and ddl, master binlog should be equal to sync binlog run_sql "flush logs;" $MYSQL_PORT1 $MYSQL_PASSWORD1 diff --git a/dm/tests/online_ddl/conf/diff_config.toml b/dm/tests/online_ddl/conf/diff_config.toml index f576c586860..bce21d71a01 100644 --- a/dm/tests/online_ddl/conf/diff_config.toml +++ b/dm/tests/online_ddl/conf/diff_config.toml @@ -15,7 +15,7 @@ check-struct-only = false target-check-tables = ["online_ddl.gho_t_target", "online_ddl.pt_t_target"] - target-configs= ["config1", "config2"] + target-configs= ["config1"] [routes.rule1] schema-pattern = "online_ddl" @@ -31,13 +31,7 @@ target-table = "pt_t_target" [table-configs] [table-configs.config1] -schema = "online_ddl" -table = "gho_t_target" -ignore-columns = ["id"] - -[table-configs.config2] -schema = "online_ddl" -table = "pt_t_target" +target-tables = ["online_ddl.*"] ignore-columns = ["id"] [data-sources] diff --git a/dm/tests/others_integration_2.txt b/dm/tests/others_integration_2.txt index c8959ae28bb..eb7c5d37db7 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -7,5 +7,4 @@ case_sensitive sql_mode http_proxies openapi -duplicate_event tracker_ignored_ddl diff --git a/dm/tests/print_status/conf/diff_config.toml b/dm/tests/print_status/conf/diff_config.toml index 30f2a1dda89..8b147560c6d 100644 --- a/dm/tests/print_status/conf/diff_config.toml +++ b/dm/tests/print_status/conf/diff_config.toml @@ -19,8 +19,7 @@ check-struct-only = false [table-configs] [table-configs.config1] -schema = "print_status" -table = "t_1" +target-tables = ["print_status.t_1"] # currently ignore check float and timestamp field ignore-columns = ["c5", "c9", "c11", "c15"] diff --git a/dm/tests/safe_mode/conf/diff_config.toml b/dm/tests/safe_mode/conf/diff_config.toml index 1f25e7fdbab..91af4e81d93 100644 --- a/dm/tests/safe_mode/conf/diff_config.toml +++ b/dm/tests/safe_mode/conf/diff_config.toml @@ -19,8 +19,7 @@ check-struct-only = false [table-configs] [table-configs.config1] -schema = "safe_mode_target" -table = "t_target" +target-tables = ["safe_mode_target.t_target"] ignore-columns = ["id"] [routes.rule1] diff --git a/dm/tests/sequence_safe_mode/conf/diff_config.toml b/dm/tests/sequence_safe_mode/conf/diff_config.toml index 46cd1690c47..5cdbd0c2ef9 100644 --- a/dm/tests/sequence_safe_mode/conf/diff_config.toml +++ b/dm/tests/sequence_safe_mode/conf/diff_config.toml @@ -25,8 +25,7 @@ target-table = "t_target" [table-configs] [table-configs.config1] -schema = "sequence_safe_mode_target" -table = "t_target" +target-tables = ["sequence_safe_mode_target.t_target"] ignore-columns = ["id"] [data-sources] diff --git a/dm/tests/sequence_sharding/conf/diff_config.toml b/dm/tests/sequence_sharding/conf/diff_config.toml index 0fbf2736182..0e8cc34b7b5 100644 --- a/dm/tests/sequence_sharding/conf/diff_config.toml +++ b/dm/tests/sequence_sharding/conf/diff_config.toml @@ -25,10 +25,9 @@ target-table = "t_target" [table-configs] [table-configs.config1] -schema = "sharding_target2" -table = "t_target" +target-tables = ["sharding_target2.t_target"] ignore-columns = ["id"] -index-fields = "uid" +index-fields = ["uid"] # range-placeholder [data-sources] diff --git a/dm/tests/sequence_sharding_optimistic/conf/diff_config.toml b/dm/tests/sequence_sharding_optimistic/conf/diff_config.toml index 937e7694b30..d0c6a4bccbf 100644 --- a/dm/tests/sequence_sharding_optimistic/conf/diff_config.toml +++ b/dm/tests/sequence_sharding_optimistic/conf/diff_config.toml @@ -25,9 +25,8 @@ target-table = "t_target" [table-configs] [table-configs.config1] -schema = "sharding_target_opt" -table = "t_target" -index-fields = "id" +target-tables = ["sharding_target_opt.t_target"] +index-fields = ["id"] # range-placeholder [data-sources] diff --git a/dm/tests/sequence_sharding_removemeta/conf/diff_config.toml b/dm/tests/sequence_sharding_removemeta/conf/diff_config.toml index 7b5453ba06c..10f10147c35 100644 --- a/dm/tests/sequence_sharding_removemeta/conf/diff_config.toml +++ b/dm/tests/sequence_sharding_removemeta/conf/diff_config.toml @@ -26,10 +26,9 @@ target-table = "t_target" [table-configs] [table-configs.config1] -schema = "sharding_target3" -table = "t_target" +target-tables = ["sharding_target3.t_target"] ignore-columns = ["id"] -index-fields = "uid" +index-fields = ["uid"] # range-placeholder [data-sources] diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 82483679b4b..4b63801670d 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -239,7 +239,7 @@ function DM_RemoveLock_CASE() { run_sql_source1 "alter table ${shardddl1}.${tb1} add column c double;" run_sql_source2 "alter table ${shardddl1}.${tb1} add column c double;" run_sql_source2 "alter table ${shardddl1}.${tb2} add column c double;" - check_log_contain_with_retry "wait new ddl info putted into etcd" $WORK_DIR/master/log/dm-master.log + check_log_contain_with_retry "wait new ddl info putted into etcd in ${1}" $WORK_DIR/master/log/dm-master.log check_metric_not_contains $MASTER_PORT "dm_master_shard_ddl_error" 3 run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" diff --git a/dm/tests/sharding/conf/diff_config.toml b/dm/tests/sharding/conf/diff_config.toml index a3066bfabb8..f69fc0306ff 100644 --- a/dm/tests/sharding/conf/diff_config.toml +++ b/dm/tests/sharding/conf/diff_config.toml @@ -32,11 +32,10 @@ target-table = "t_target" [table-configs] [table-configs.config1] -schema = "db_target" -table = "t_target" +target-tables = ["db_target.t_target"] # currently sync_diff does not support json fields well ignore-columns = ["id", "info_json"] -index-fields = "uid" +index-fields = ["uid"] # range-placeholder diff --git a/dm/tests/sql_mode/data/db1.prepare.sql b/dm/tests/sql_mode/data/db1.prepare.sql index 6b17c8d1f7f..bfa2c27be28 100644 --- a/dm/tests/sql_mode/data/db1.prepare.sql +++ b/dm/tests/sql_mode/data/db1.prepare.sql @@ -44,6 +44,6 @@ grant select on *.* to 'no_auto_create_user'; -- test different timezone create table if not exists `sql_mode`.`timezone` (`id` int, `a` timestamp, PRIMARY KEY (id)); set @@session.time_zone = "Asia/Shanghai"; -insert into `sql_mode`.`timezone`(`id`, `a`) values (1, '1990-04-15 01:30:12'); +insert into `sql_mode`.`timezone`(`id`, `a`) values (1, '2001-04-15 01:30:12'); set @@session.time_zone = "America/Phoenix"; -insert into `sql_mode`.`timezone`(`id`, `a`) values (4, '1990-04-15 01:30:12'); +insert into `sql_mode`.`timezone`(`id`, `a`) values (4, '2001-04-15 01:30:12'); diff --git a/dm/tests/sql_mode/data/timezone.America-Phoenix.sql b/dm/tests/sql_mode/data/timezone.America-Phoenix.sql index a137d3707de..bf2e6f1a9d5 100644 --- a/dm/tests/sql_mode/data/timezone.America-Phoenix.sql +++ b/dm/tests/sql_mode/data/timezone.America-Phoenix.sql @@ -6,9 +6,9 @@ set @@session.time_zone = "America/Phoenix"; insert into `sql_mode`.`timezone`(`id`, `a`) values - (5, '1990-04-15 02:30:12'); + (5, '2001-04-15 02:30:12'); insert into `sql_mode`.`timezone`(`id`, `a`) values - (6, '1990-04-15 03:30:12'); \ No newline at end of file + (6, '2001-04-15 03:30:12'); diff --git a/dm/tests/sql_mode/data/timezone.Asia-Shanghai.sql b/dm/tests/sql_mode/data/timezone.Asia-Shanghai.sql index 3edbf41bd83..b8773b4ec5a 100644 --- a/dm/tests/sql_mode/data/timezone.Asia-Shanghai.sql +++ b/dm/tests/sql_mode/data/timezone.Asia-Shanghai.sql @@ -6,9 +6,9 @@ set @@session.time_zone = "Asia/Shanghai"; insert into `sql_mode`.`timezone`(`id`, `a`) values - (2, '1990-04-15 02:30:12'); + (2, '2001-04-15 02:30:12'); insert into `sql_mode`.`timezone`(`id`, `a`) values - (3, '1990-04-15 03:30:12'); \ No newline at end of file + (3, '2001-04-15 03:30:12'); diff --git a/go.mod b/go.mod index 11577d097d0..90b9c296732 100644 --- a/go.mod +++ b/go.mod @@ -8,6 +8,7 @@ require ( github.com/Shopify/sarama v1.27.2 github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 github.com/apache/pulsar-client-go v0.6.0 + github.com/aws/aws-sdk-go v1.35.3 github.com/bradleyjkemp/grpc-tools v0.2.5 github.com/cenkalti/backoff v2.2.1+incompatible github.com/chaos-mesh/go-sqlsmith v0.0.0-20211025024535-03ae33408684 @@ -42,20 +43,19 @@ require ( github.com/labstack/echo/v4 v4.6.1 github.com/lib/pq v1.3.0 // indirect github.com/linkedin/goavro/v2 v2.9.8 - github.com/mackerelio/go-osstat v0.1.0 github.com/mattn/go-colorable v0.1.11 // indirect - github.com/mattn/go-shellwords v1.0.3 + github.com/mattn/go-shellwords v1.0.12 github.com/mattn/go-sqlite3 v2.0.2+incompatible // indirect github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 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.20210513014640-40f9a1999b3b github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd - github.com/pingcap/kvproto v0.0.0-20211011042309-a4518fcacbc8 + github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551 github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 - github.com/pingcap/tidb v1.1.0-beta.0.20211026030648-c497d5c06348 + github.com/pingcap/tidb v1.1.0-beta.0.20211111080905-76b00f3ec11e github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible - github.com/pingcap/tidb/parser v0.0.0-20211026030648-c497d5c06348 + github.com/pingcap/tidb/parser v0.0.0-20211111080905-76b00f3ec11e github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 @@ -72,8 +72,8 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 github.com/tidwall/gjson v1.9.1 github.com/tidwall/sjson v1.2.2 - github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211011083157-49c8dd23f1f0 - github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211029104011-2fd3841894de + github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae github.com/tinylib/msgp v1.1.0 github.com/uber-go/atomic v1.4.0 github.com/unrolled/render v1.0.1 diff --git a/go.sum b/go.sum index 9b3d0f125d8..e333ecd701f 100644 --- a/go.sum +++ b/go.sum @@ -517,6 +517,7 @@ github.com/improbable-eng/grpc-web v0.12.0 h1:GlCS+lMZzIkfouf7CNqY+qqpowdKuJLSLL github.com/improbable-eng/grpc-web v0.12.0/go.mod h1:6hRR09jOEG81ADP5wCQju1z71g6OL4eEvELdran/3cs= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc h1:4IZpk3M4m6ypx0IlRoEyEyY1gAdicWLMQ0NcG/gBnnA= github.com/integralist/go-findroot v0.0.0-20160518114804-ac90681525dc/go.mod h1:UlaC6ndby46IJz9m/03cZPKKkR9ykeIVBBDE3UDBdJk= github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= @@ -619,8 +620,6 @@ 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/mackerelio/go-osstat v0.1.0 h1:e57QHeHob8kKJ5FhcXGdzx5O6Ktuc5RHMDIkeqhgkFA= -github.com/mackerelio/go-osstat v0.1.0/go.mod h1:1K3NeYLhMHPvzUu+ePYXtoB58wkaRpxZsGClZBJyIFw= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -647,8 +646,9 @@ github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.12 h1:Y41i/hVW3Pgwr8gV+J23B9YEY0zxjptBuCWEaxmAOow= github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= -github.com/mattn/go-shellwords v1.0.3 h1:K/VxK7SZ+cvuPgFSLKi5QPI9Vr/ipOf4C1gN+ntueUk= github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= +github.com/mattn/go-shellwords v1.0.12 h1:M2zGm7EW6UQJvDeQxo4T51eKPurbeFbe8WtebGE2xrk= +github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= github.com/mattn/go-sqlite3 v1.14.5/go.mod h1:WVKg1VTActs4Qso6iwGbiFih2UIHo0ENGwNd0Lj+XmI= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= @@ -759,9 +759,10 @@ github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210819164333-bd5706b9d9f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211011042309-a4518fcacbc8 h1:IBNpJoFfn8BHhqyeKuAuVnYpViQE6/LOXFtDxuQPbqA= -github.com/pingcap/kvproto v0.0.0-20211011042309-a4518fcacbc8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551 h1:aRx2l2TAeYNPPUc+lk5dEFCXfUGxR/C2fbt/YA5nqiQ= +github.com/pingcap/kvproto v0.0.0-20211011060348-d957056f1551/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -776,19 +777,21 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041 github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 h1:7rvAtZe/ZUzOKzgriNPQoBNvleJXBk4z7L3Z47+tS98= github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= github.com/pingcap/tidb v1.1.0-beta.0.20211023132847-efa94595c071/go.mod h1:Ci7ABF58a4jn6YtaHi7655jP409edqC2JxWWFRqOubg= -github.com/pingcap/tidb v1.1.0-beta.0.20211026030648-c497d5c06348 h1:DVcIszrlW/71/VA1Vpu0Sl6kY1oJiGRRHEpP+LqNrw0= -github.com/pingcap/tidb v1.1.0-beta.0.20211026030648-c497d5c06348/go.mod h1:0x9V4RDM9rVr1FduK7jZLG3E6gp4ggqB896y1l2HFUI= +github.com/pingcap/tidb v1.1.0-beta.0.20211111080905-76b00f3ec11e h1:ZCT++ksXnkoGV14159vfEKcfrGJunfvjtAf+3JhlYTQ= +github.com/pingcap/tidb v1.1.0-beta.0.20211111080905-76b00f3ec11e/go.mod h1:MGT/AUgZBwr8sJ7KUZIsHaxUme9nzAfQYMkUclWr26s= 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= github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+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-20211026030648-c497d5c06348 h1:Dt7Lsp9i6pMF/eKp/wMG2lO9ZsTM/gLSwRfeLUI3SDk= -github.com/pingcap/tidb/parser v0.0.0-20211026030648-c497d5c06348/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= -github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e h1:fZY5T65QWiPc9noQJ1UkdwejZyBZjNfxzSyTcBjKrEU= +github.com/pingcap/tidb/parser v0.0.0-20211111080905-76b00f3ec11e h1:rqfPYqKz7NlJ7fLEIOG4lkVcpcuwzuE6n5V1vVdBr9g= +github.com/pingcap/tidb/parser v0.0.0-20211111080905-76b00f3ec11e/go.mod h1:MAa22tagoj7nv5b1NBcxPkc5CiUNhqj1wuSQnw4f9WE= github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20211105090418-71142a4d40e3 h1:xnp/Qkk5gELlB8TaY6oro0JNXMBXTafNVxU/vbrNU8I= +github.com/pingcap/tipb v0.0.0-20211105090418-71142a4d40e3/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -937,11 +940,13 @@ github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= github.com/tidwall/sjson v1.2.2 h1:H1Llj/C9G+BoUN2DsybLHjWvr9dx4Uazavf0sXQ+rOs= github.com/tidwall/sjson v1.2.2/go.mod h1:jmW2RZpbKuExPFUHeFSBMiovT9ZyOziEHDRkbsdp0B0= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211011083157-49c8dd23f1f0 h1:DZdVqFVe+r7M0tSZ2LdqeY+UNplR1DSLCdGtH1AKi5M= github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211011083157-49c8dd23f1f0/go.mod h1:00plYwQsQ5kBUmafHO+JkjznGgFaBokMZl82TZIbsQk= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211029104011-2fd3841894de h1:DKo2grkDpP9hQHuYbkAz4yxMS1742qBkUd4kwyZK2As= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211029104011-2fd3841894de/go.mod h1:gdd4S4uS3/apOF9iet/DIYUdr6J4WzGLWyDgn6SMtg0= github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw= -github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d h1:AFm1Dzw+QRUevWRfrFp45CPPkuK/zdSWcfxI10z+WVE= github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA= +github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae h1:PmnkhiOopgMZYDQ7Htj1kt/zwW4MEOUL+Dem6WLZISY= +github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= 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= @@ -1220,7 +1225,6 @@ golang.org/x/sys v0.0.0-20181228144115-9a3f9b0469bb/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190410235845-0ad05ae3009d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1283,6 +1287,7 @@ 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-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210910150752-751e447fb3d0/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-20211025201205-69cdffdb9359 h1:2B5p2L5IfGiD7+b9BOoRMC6DgObAVZV+Fsp050NqXik= @@ -1388,7 +1393,9 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= +gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= diff --git a/integration/tests/case_date_time.go b/integration/tests/case_date_time.go index 8d5f622a398..ac060b94036 100644 --- a/integration/tests/case_date_time.go +++ b/integration/tests/case_date_time.go @@ -15,9 +15,9 @@ package tests import ( "errors" - "log" "time" + "github.com/pingcap/log" "github.com/pingcap/ticdc/integration/framework" "github.com/pingcap/ticdc/integration/framework/avro" "github.com/pingcap/ticdc/integration/framework/canal" diff --git a/metrics/alertmanager/ticdc.rules.yml b/metrics/alertmanager/ticdc.rules.yml index f101f2a8c92..5290867501d 100644 --- a/metrics/alertmanager/ticdc.rules.yml +++ b/metrics/alertmanager/ticdc.rules.yml @@ -14,24 +14,24 @@ groups: summary: cdc cluster has multiple owners - alert: cdc_checkpoint_high_delay - expr: (time() - ticdc_processor_checkpoint_ts / 1000) > 600 + expr: ticdc_processor_checkpoint_ts_lag > 600 for: 1m labels: env: ENV_LABELS_ENV level: critical - expr: (time() - ticdc_processor_checkpoint_ts / 1000) > 600 + expr: ticdc_processor_checkpoint_ts_lag > 600 annotations: description: 'cluster: ENV_LABELS_ENV, instance: {{ $labels.instance }}, values: {{ $value }}' value: '{{ $value }}' summary: cdc processor checkpoint delay more than 10 minutes - alert: cdc_resolvedts_high_delay - expr: (time() - ticdc_processor_resolved_ts / 1000) > 300 + expr: ticdc_processor_resolved_ts_lag > 300 for: 1m labels: env: ENV_LABELS_ENV level: critical - expr: (time() - ticdc_processor_resolved_ts / 1000) > 300 + expr: ticdc_processor_resolved_ts_lag > 300 annotations: description: 'cluster: ENV_LABELS_ENV, instance: {{ $labels.instance }}, values: {{ $value }}' value: '{{ $value }}' diff --git a/pkg/context/context.go b/pkg/context/context.go index 9c2445de027..1930d3f5db4 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -15,13 +15,14 @@ package context import ( "context" - "log" "time" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/ticdc/pkg/pdtime" "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" "github.com/tikv/client-go/v2/oracle" @@ -33,11 +34,12 @@ import ( // the lifecycle of vars in the GlobalVars should be aligned with the ticdc server process. // All field in Vars should be READ-ONLY and THREAD-SAFE type GlobalVars struct { - PDClient pd.Client - KVStorage tidbkv.Storage - CaptureInfo *model.CaptureInfo - EtcdClient *etcd.CDCEtcdClient - GrpcPool kv.GrpcPool + PDClient pd.Client + KVStorage tidbkv.Storage + CaptureInfo *model.CaptureInfo + EtcdClient *etcd.CDCEtcdClient + GrpcPool kv.GrpcPool + TimeAcquirer pdtime.TimeAcquirer } // ChangefeedVars contains some vars which can be used anywhere in a pipeline @@ -184,6 +186,7 @@ func NewBackendContext4Test(withChangefeedVars bool) Context { AdvertiseAddr: "127.0.0.1:0000", Version: version.ReleaseVersion, }, + TimeAcquirer: pdtime.NewTimeAcquirer4Test(), }) if withChangefeedVars { ctx = WithChangefeedVars(ctx, &ChangefeedVars{ diff --git a/pkg/context/context_test.go b/pkg/context/context_test.go index f274e4fad05..184459e5b18 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -129,7 +129,7 @@ func TestThrowPanic(t *testing.T) { t.Parallel() defer func() { panicMsg := recover() - require.Equal(t, panicMsg, "an error has escaped, please report a bug{error 26 0 mock error}") + require.Equal(t, panicMsg, "an error has escaped, please report a bug") }() stdCtx := context.Background() ctx := NewContext(stdCtx, &GlobalVars{}) diff --git a/pkg/etcd/etcdkey.go b/pkg/etcd/etcdkey.go index bf47a210a30..8b91dc36921 100644 --- a/pkg/etcd/etcdkey.go +++ b/pkg/etcd/etcdkey.go @@ -14,9 +14,9 @@ package etcd import ( - "log" "strings" + "github.com/pingcap/log" cerror "github.com/pingcap/ticdc/pkg/errors" ) diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index 4b1075cdc79..65bfc26a8d8 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -18,6 +18,7 @@ import ( "encoding/json" "regexp" "strconv" + "strings" "sync" "testing" "time" @@ -223,7 +224,6 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { defer func() { _ = cli.Unwrap().Close() }() - _, err := cli.Put(ctx, testEtcdKeyPrefix+"/sum", "0") c.Check(err, check.IsNil) @@ -272,7 +272,9 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { } err = errg.Wait() - if err != nil && (errors.Cause(err) == context.DeadlineExceeded || errors.Cause(err) == context.Canceled) { + if err != nil && (errors.Cause(err) == context.DeadlineExceeded || + errors.Cause(err) == context.Canceled || + strings.Contains(err.Error(), "etcdserver: request timeout")) { return } c.Check(err, check.IsNil) diff --git a/pkg/orchestrator/util/key_utils.go b/pkg/orchestrator/util/key_utils.go index e9b68ca491c..008c8ae9705 100644 --- a/pkg/orchestrator/util/key_utils.go +++ b/pkg/orchestrator/util/key_utils.go @@ -14,8 +14,9 @@ package util import ( - "log" "strings" + + "github.com/pingcap/log" ) // EtcdKey represents a complete key in Etcd. diff --git a/pkg/pdtime/acquirer.go b/pkg/pdtime/acquirer.go new file mode 100644 index 00000000000..3fae739fa9d --- /dev/null +++ b/pkg/pdtime/acquirer.go @@ -0,0 +1,118 @@ +// 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 pdtime + +import ( + "context" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/retry" + "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" + "go.uber.org/zap" +) + +const pdTimeUpdateInterval = 200 * time.Millisecond + +type TimeAcquirer interface { + // Run run the TimeAcquirer + Run(ctx context.Context) + // CurrentTimeFromCached returns current time from cache + CurrentTimeFromCached() (time.Time, error) + // Stop stops the TimeAcquirer + Stop() +} + +// TimeAcquirerImpl cache time get from PD periodically and cache it +type TimeAcquirerImpl struct { + pdClient pd.Client + timeCache time.Time + mu sync.RWMutex + cancel context.CancelFunc + err error +} + +// NewTimeAcquirer return a new TimeAcquirer +func NewTimeAcquirer(pdClient pd.Client) TimeAcquirer { + return &TimeAcquirerImpl{ + pdClient: pdClient, + } +} + +// Run will get time from pd periodically to cache in pdPhysicalTimeCache +func (c *TimeAcquirerImpl) Run(ctx context.Context) { + ctx, cancel := context.WithCancel(ctx) + c.cancel = cancel + ticker := time.NewTicker(pdTimeUpdateInterval) + for { + select { + // c.Stop() was called or parent ctx was canceled + case <-ctx.Done(): + log.Info("TimeAcquirer exit") + return + case <-ticker.C: + err := retry.Do(ctx, func() error { + physical, _, err := c.pdClient.GetTS(ctx) + if err != nil { + log.Info("get time from pd failed, retry later", zap.Error(err)) + return err + } + c.mu.Lock() + c.timeCache = oracle.GetTimeFromTS(oracle.ComposeTS(physical, 0)) + c.err = nil + c.mu.Unlock() + return nil + }, retry.WithBackoffBaseDelay(200), retry.WithMaxTries(10)) + if err != nil { + log.Warn("get time from pd failed, will use local time as pd time") + c.mu.Lock() + c.timeCache = time.Now() + c.err = err + c.mu.Unlock() + } + } + } +} + +// CurrentTimeFromCached return current time from pd cache +func (c *TimeAcquirerImpl) CurrentTimeFromCached() (time.Time, error) { + c.mu.RLock() + err := c.err + cacheTime := c.timeCache + c.mu.RUnlock() + return cacheTime, errors.Trace(err) +} + +func (c *TimeAcquirerImpl) Stop() { + c.cancel() +} + +type TimeAcquirer4Test struct{} + +func NewTimeAcquirer4Test() TimeAcquirer { + return &TimeAcquirer4Test{} +} + +func (c *TimeAcquirer4Test) CurrentTimeFromCached() (time.Time, error) { + return time.Now(), nil +} + +func (c *TimeAcquirer4Test) Run(ctx context.Context) { +} + +func (c *TimeAcquirer4Test) Stop() { +} diff --git a/pkg/pdtime/acquirer_test.go b/pkg/pdtime/acquirer_test.go new file mode 100644 index 00000000000..55b2950192e --- /dev/null +++ b/pkg/pdtime/acquirer_test.go @@ -0,0 +1,53 @@ +// 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 pdtime + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" +) + +// MockPDClient mocks pd.Client to facilitate unit testing. +type MockPDClient struct { + pd.Client +} + +// GetTS implements pd.Client.GetTS. +func (m *MockPDClient) GetTS(ctx context.Context) (int64, int64, error) { + return oracle.GetPhysical(time.Now()), 0, nil +} + +func TestTimeFromPD(t *testing.T) { + t.Parallel() + mockPDClient := &MockPDClient{} + TimeAcquirer := NewTimeAcquirer(mockPDClient) + go TimeAcquirer.Run(context.Background()) + defer TimeAcquirer.Stop() + time.Sleep(1 * time.Second) + + t1, err := TimeAcquirer.CurrentTimeFromCached() + require.Nil(t, err) + + time.Sleep(400 * time.Millisecond) + // assume that the gc safe point updated one hour ago + t2, err := TimeAcquirer.CurrentTimeFromCached() + require.Nil(t, err) + // should return new time + require.NotEqual(t, t1, t2) +} diff --git a/pkg/pdtime/main_test.go b/pkg/pdtime/main_test.go new file mode 100644 index 00000000000..229d3e567f5 --- /dev/null +++ b/pkg/pdtime/main_test.go @@ -0,0 +1,24 @@ +// 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 pdtime + +import ( + "testing" + + "github.com/pingcap/ticdc/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/pkg/security/credential_test.go b/pkg/security/credential_test.go index 7a3b12f594f..3ffd279a3bb 100644 --- a/pkg/security/credential_test.go +++ b/pkg/security/credential_test.go @@ -21,15 +21,15 @@ import ( func TestGetCommonName(t *testing.T) { cd := &Credential{ - CAPath: "../../tests/_certificates/ca.pem", - CertPath: "../../tests/_certificates/server.pem", - KeyPath: "../../tests/_certificates/server-key.pem", + CAPath: "../../tests/integration_tests/_certificates/ca.pem", + CertPath: "../../tests/integration_tests/_certificates/server.pem", + KeyPath: "../../tests/integration_tests/_certificates/server-key.pem", } cn, err := cd.getSelfCommonName() require.Nil(t, err) require.Equal(t, "tidb-server", cn) - cd.CertPath = "../../tests/_certificates/server-key.pem" + cd.CertPath = "../../tests/integration_tests/_certificates/server-key.pem" _, err = cd.getSelfCommonName() require.NotNil(t, err) require.Contains(t, err.Error(), "failed to decode PEM block to certificate") diff --git a/pkg/tcpserver/tcp_server_test.go b/pkg/tcpserver/tcp_server_test.go index ab329b525e3..654fade8489 100644 --- a/pkg/tcpserver/tcp_server_test.go +++ b/pkg/tcpserver/tcp_server_test.go @@ -163,7 +163,7 @@ func makeCredential4Testing(t *testing.T) *security.Credential { stat, err := find.Repo() require.NoError(t, err) - tlsPath := fmt.Sprintf("%s/tests/_certificates/", stat.Path) + tlsPath := fmt.Sprintf("%s/tests/integration_tests/_certificates/", stat.Path) return &security.Credential{ CAPath: path.Join(tlsPath, "ca.pem"), CertPath: path.Join(tlsPath, "server.pem"), diff --git a/pkg/txnutil/gc/gc_manager.go b/pkg/txnutil/gc/gc_manager.go index df997487dbf..1a158fa6f9d 100644 --- a/pkg/txnutil/gc/gc_manager.go +++ b/pkg/txnutil/gc/gc_manager.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" + cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" @@ -31,7 +32,6 @@ import ( const ( // CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. CDCServiceSafePointID = "ticdc" - pdTimeUpdateInterval = 10 * time.Minute ) // gcSafepointUpdateInterval is the minimum interval that CDC can update gc safepoint @@ -43,32 +43,27 @@ type Manager interface { // Manager may skip update when it thinks it is too frequent. // Set `forceUpdate` to force Manager update. TryUpdateGCSafePoint(ctx context.Context, checkpointTs model.Ts, forceUpdate bool) error - CurrentTimeFromPDCached(ctx context.Context) (time.Time, error) CheckStaleCheckpointTs(ctx context.Context, changefeedID model.ChangeFeedID, checkpointTs model.Ts) error } type gcManager struct { pdClient pd.Client - - gcTTL int64 + gcTTL int64 lastUpdatedTime time.Time lastSucceededTime time.Time lastSafePointTs uint64 isTiCDCBlockGC bool - - pdPhysicalTimeCache time.Time - lastUpdatedPdTime time.Time } // NewManager creates a new Manager. -func NewManager(pdClint pd.Client) Manager { +func NewManager(pdClient pd.Client) Manager { serverConfig := config.GetGlobalServerConfig() failpoint.Inject("InjectGcSafepointUpdateInterval", func(val failpoint.Value) { gcSafepointUpdateInterval = time.Duration(val.(int) * int(time.Millisecond)) }) return &gcManager{ - pdClient: pdClint, + pdClient: pdClient, lastSucceededTime: time.Now(), gcTTL: serverConfig.GcTTL, } @@ -111,25 +106,17 @@ func (m *gcManager) TryUpdateGCSafePoint( return nil } -func (m *gcManager) CurrentTimeFromPDCached(ctx context.Context) (time.Time, error) { - if time.Since(m.lastUpdatedPdTime) <= pdTimeUpdateInterval { - return m.pdPhysicalTimeCache, nil - } - physical, logical, err := m.pdClient.GetTS(ctx) - if err != nil { - return time.Now(), errors.Trace(err) - } - m.pdPhysicalTimeCache = oracle.GetTimeFromTS(oracle.ComposeTS(physical, logical)) - m.lastUpdatedPdTime = time.Now() - return m.pdPhysicalTimeCache, nil -} - func (m *gcManager) CheckStaleCheckpointTs( ctx context.Context, changefeedID model.ChangeFeedID, checkpointTs model.Ts, ) error { gcSafepointUpperBound := checkpointTs - 1 if m.isTiCDCBlockGC { - pdTime, err := m.CurrentTimeFromPDCached(ctx) + cctx, ok := ctx.(cdcContext.Context) + if !ok { + return cerror.ErrOwnerUnknown.GenWithStack("ctx not an cdcContext.Context, it should be") + } + pdTime, err := cctx.GlobalVars().TimeAcquirer.CurrentTimeFromCached() + // TODO: should we return err here, or just log it? if err != nil { return errors.Trace(err) } diff --git a/pkg/txnutil/gc/gc_manager_test.go b/pkg/txnutil/gc/gc_manager_test.go index 1f535f8cc94..d650aa72443 100644 --- a/pkg/txnutil/gc/gc_manager_test.go +++ b/pkg/txnutil/gc/gc_manager_test.go @@ -18,6 +18,8 @@ import ( "testing" "time" + "github.com/pingcap/ticdc/pkg/pdtime" + "github.com/pingcap/check" "github.com/pingcap/errors" cdcContext "github.com/pingcap/ticdc/pkg/context" @@ -87,49 +89,32 @@ func (s *gcManagerSuite) TestUpdateGCSafePoint(c *check.C) { } } -func (s *gcManagerSuite) TestTimeFromPD(c *check.C) { - defer testleak.AfterTest(c)() - mockPDClient := &MockPDClient{} - gcManager := NewManager(mockPDClient).(*gcManager) - ctx := cdcContext.NewBackendContext4Test(true) - ctx.GlobalVars().PDClient = mockPDClient - t1, err := gcManager.CurrentTimeFromPDCached(ctx) - c.Assert(err, check.IsNil) - c.Assert(t1, check.Equals, gcManager.pdPhysicalTimeCache) - - time.Sleep(50 * time.Millisecond) - // should return cached time - t2, err := gcManager.CurrentTimeFromPDCached(ctx) - c.Assert(err, check.IsNil) - c.Assert(t2, check.Equals, gcManager.pdPhysicalTimeCache) - c.Assert(t2, check.Equals, t1) - - time.Sleep(50 * time.Millisecond) - // assume that the gc safe point updated one hour ago - gcManager.lastUpdatedPdTime = time.Now().Add(-time.Hour) - t3, err := gcManager.CurrentTimeFromPDCached(ctx) - c.Assert(err, check.IsNil) - c.Assert(t3, check.Equals, gcManager.pdPhysicalTimeCache) - // should return new time - c.Assert(t3, check.Not(check.Equals), t2) -} - func (s *gcManagerSuite) TestCheckStaleCheckpointTs(c *check.C) { defer testleak.AfterTest(c)() mockPDClient := &MockPDClient{} gcManager := NewManager(mockPDClient).(*gcManager) gcManager.isTiCDCBlockGC = true ctx := context.Background() - err := gcManager.CheckStaleCheckpointTs(ctx, "cfID", 10) + + TimeAcquirer := pdtime.NewTimeAcquirer(mockPDClient) + go TimeAcquirer.Run(ctx) + time.Sleep(1 * time.Second) + defer TimeAcquirer.Stop() + + cCtx := cdcContext.NewContext(ctx, &cdcContext.GlobalVars{ + TimeAcquirer: TimeAcquirer, + }) + + err := gcManager.CheckStaleCheckpointTs(cCtx, "cfID", 10) c.Assert(cerror.ErrGCTTLExceeded.Equal(errors.Cause(err)), check.IsTrue) c.Assert(cerror.ChangefeedFastFailError(err), check.IsTrue) - err = gcManager.CheckStaleCheckpointTs(ctx, "cfID", oracle.GoTimeToTS(time.Now())) + err = gcManager.CheckStaleCheckpointTs(cCtx, "cfID", oracle.GoTimeToTS(time.Now())) c.Assert(err, check.IsNil) gcManager.isTiCDCBlockGC = false gcManager.lastSafePointTs = 20 - err = gcManager.CheckStaleCheckpointTs(ctx, "cfID", 10) + err = gcManager.CheckStaleCheckpointTs(cCtx, "cfID", 10) c.Assert(cerror.ErrSnapshotLostByGC.Equal(errors.Cause(err)), check.IsTrue) c.Assert(cerror.ChangefeedFastFailError(err), check.IsTrue) } diff --git a/pkg/util/tz.go b/pkg/util/tz.go index 632e9810860..b1e18043542 100644 --- a/pkg/util/tz.go +++ b/pkg/util/tz.go @@ -14,12 +14,12 @@ package util import ( - "os" "path/filepath" "strings" "time" cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tidb/util/timeutil" ) // GetTimezone returns the timezone specified by the name @@ -57,9 +57,6 @@ func GetLocalTimezone() (*time.Location, error) { if time.Local.String() != "Local" { return time.Local, nil } - str, err := os.Readlink("/etc/localtime") - if err != nil { - return nil, cerror.WrapError(cerror.ErrLoadTimezone, err) - } + str := timeutil.InferSystemTZ() return getTimezoneFromZonefile(str) } diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index e3f754450b1..5927c12069d 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/ticdc/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) const ( @@ -298,6 +299,9 @@ type worker struct { isRunning int32 // notifies exits of run() stopNotifier notify.Notifier + + slowSynchronizeThreshold time.Duration + slowSynchronizeLimiter *rate.Limiter } func newWorker() *worker { @@ -305,6 +309,9 @@ func newWorker() *worker { taskCh: make(chan task, 128), handles: make(map[*defaultEventHandle]struct{}), handleCancelCh: make(chan struct{}), // this channel must be unbuffered, i.e. blocking + + slowSynchronizeThreshold: 10 * time.Second, + slowSynchronizeLimiter: rate.NewLimiter(rate.Every(time.Second*5), 1), } } @@ -398,13 +405,20 @@ func (w *worker) synchronize() { break } - if time.Since(startTime) > time.Second*10 { - // likely the workerpool has deadlocked, or there is a bug in the event handlers. - log.Warn("synchronize is taking too long, report a bug", zap.Duration("elapsed", time.Since(startTime))) + if time.Since(startTime) > w.slowSynchronizeThreshold && + w.slowSynchronizeLimiter.Allow() { + // likely the workerpool has deadlocked, or there is a bug + // in the event handlers. + logWarn("synchronize is taking too long, report a bug", + zap.Duration("elapsed", time.Since(startTime)), + zap.Stack("stacktrace")) } } } +// A delegate to log.Warn. It exists only for testing. +var logWarn = log.Warn + func (w *worker) addHandle(handle *defaultEventHandle) { w.handleRWLock.Lock() defer w.handleRWLock.Unlock() diff --git a/pkg/workerpool/pool_test.go b/pkg/workerpool/pool_test.go index 76cf1701ec5..3aad1b0b1f7 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -27,6 +27,7 @@ import ( "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) func TestTaskError(t *testing.T) { @@ -507,6 +508,38 @@ func TestGracefulUnregisterTimeout(t *testing.T) { require.Truef(t, cerror.ErrWorkerPoolGracefulUnregisterTimedOut.Equal(err), "%s", err.Error()) } +func TestSynchronizeLog(t *testing.T) { + w := newWorker() + w.isRunning = 1 + // Always report "synchronize is taking too long". + w.slowSynchronizeThreshold = time.Duration(0) + w.slowSynchronizeLimiter = rate.NewLimiter(rate.Every(100*time.Minute), 1) + + counter := int32(0) + logWarn = func(msg string, fields ...zap.Field) { + atomic.AddInt32(&counter, 1) + } + defer func() { logWarn = log.Warn }() + + doneCh := make(chan struct{}) + go func() { + w.synchronize() + close(doneCh) + }() + + time.Sleep(300 * time.Millisecond) + w.stopNotifier.Notify() + time.Sleep(300 * time.Millisecond) + w.stopNotifier.Notify() + + // Close worker. + atomic.StoreInt32(&w.isRunning, 0) + w.stopNotifier.Close() + <-doneCh + + require.EqualValues(t, 1, atomic.LoadInt32(&counter)) +} + // Benchmark workerpool with ping-pong workflow. // go test -benchmem -run='^$' -bench '^(BenchmarkWorkerpool)$' github.com/pingcap/ticdc/pkg/workerpool func BenchmarkWorkerpool(b *testing.B) { diff --git a/tests/README.md b/tests/integration_tests/README.md similarity index 93% rename from tests/README.md rename to tests/integration_tests/README.md index 097902c1295..0bcb28ee6c2 100644 --- a/tests/README.md +++ b/tests/integration_tests/README.md @@ -36,8 +36,8 @@ 2. Run `make integration_test` to execute the integration tests. This command will - 1. Check that all required executables exist. - 2. Execute `tests/run.sh` + 1. Check that all required executables exist. + 2. Execute `tests/integration_tests/run.sh` > If want to run one integration test case only, just pass the CASE parameter, such as `make integration_test CASE=simple`. @@ -50,4 +50,5 @@ ## Writing new tests -New integration tests can be written as shell scripts in `tests/TEST_NAME/run.sh`. The script should exit with a nonzero error code on failure. +New integration tests can be written as shell scripts in `tests/integration_tests/TEST_NAME/run.sh`. The script should +exit with a nonzero error code on failure. diff --git a/tests/_certificates/ca.pem b/tests/integration_tests/_certificates/ca.pem similarity index 100% rename from tests/_certificates/ca.pem rename to tests/integration_tests/_certificates/ca.pem diff --git a/tests/_certificates/client-key.pem b/tests/integration_tests/_certificates/client-key.pem similarity index 100% rename from tests/_certificates/client-key.pem rename to tests/integration_tests/_certificates/client-key.pem diff --git a/tests/_certificates/client.pem b/tests/integration_tests/_certificates/client.pem similarity index 100% rename from tests/_certificates/client.pem rename to tests/integration_tests/_certificates/client.pem diff --git a/tests/_certificates/kafka.server.keystore.jks b/tests/integration_tests/_certificates/kafka.server.keystore.jks similarity index 100% rename from tests/_certificates/kafka.server.keystore.jks rename to tests/integration_tests/_certificates/kafka.server.keystore.jks diff --git a/tests/_certificates/kafka.server.truststore.jks b/tests/integration_tests/_certificates/kafka.server.truststore.jks similarity index 100% rename from tests/_certificates/kafka.server.truststore.jks rename to tests/integration_tests/_certificates/kafka.server.truststore.jks diff --git a/tests/_certificates/server-key.pem b/tests/integration_tests/_certificates/server-key.pem similarity index 100% rename from tests/_certificates/server-key.pem rename to tests/integration_tests/_certificates/server-key.pem diff --git a/tests/_certificates/server.pem b/tests/integration_tests/_certificates/server.pem similarity index 100% rename from tests/_certificates/server.pem rename to tests/integration_tests/_certificates/server.pem diff --git a/tests/_utils/check_contains b/tests/integration_tests/_utils/check_contains similarity index 100% rename from tests/_utils/check_contains rename to tests/integration_tests/_utils/check_contains diff --git a/tests/_utils/check_logs b/tests/integration_tests/_utils/check_logs similarity index 100% rename from tests/_utils/check_logs rename to tests/integration_tests/_utils/check_logs diff --git a/tests/_utils/check_logs_contains b/tests/integration_tests/_utils/check_logs_contains similarity index 100% rename from tests/_utils/check_logs_contains rename to tests/integration_tests/_utils/check_logs_contains diff --git a/tests/_utils/check_not_contains b/tests/integration_tests/_utils/check_not_contains similarity index 100% rename from tests/_utils/check_not_contains rename to tests/integration_tests/_utils/check_not_contains diff --git a/tests/_utils/check_sync_diff b/tests/integration_tests/_utils/check_sync_diff similarity index 88% rename from tests/_utils/check_sync_diff rename to tests/integration_tests/_utils/check_sync_diff index 42a8c5b5f6c..7f1b72e3f7c 100755 --- a/tests/_utils/check_sync_diff +++ b/tests/integration_tests/_utils/check_sync_diff @@ -14,14 +14,13 @@ binary=sync_diff_inspector PWD=$(pwd) LOG=$workdir/sync_diff_inspector.log -OUTPUT_DIR=/tmp/ticdc_dm_test/output/ mkdir -p /tmp/ticdc_dm_test cd $workdir i=0 while [ $i -lt $check_time ]; do - rm -rf $OUTPUT_DIR + rm -rf $workdir/sync_diff/ $binary --config=$conf >>$LOG 2>&1 ret=$? if [ "$ret" == 0 ]; then @@ -38,7 +37,7 @@ if [ $i -ge $check_time ]; then # show \n and other blanks cat $LOG printf "\n" - cat $OUTPUT_DIR/sync_diff.log + cat $workdir/sync_diff/output/sync_diff.log printf "\n" exit 1 fi diff --git a/tests/_utils/check_table_exists b/tests/integration_tests/_utils/check_table_exists similarity index 100% rename from tests/_utils/check_table_exists rename to tests/integration_tests/_utils/check_table_exists diff --git a/tests/_utils/cleanup_process b/tests/integration_tests/_utils/cleanup_process similarity index 100% rename from tests/_utils/cleanup_process rename to tests/integration_tests/_utils/cleanup_process diff --git a/tests/_utils/ensure b/tests/integration_tests/_utils/ensure similarity index 100% rename from tests/_utils/ensure rename to tests/integration_tests/_utils/ensure diff --git a/tests/_utils/random_kill_process b/tests/integration_tests/_utils/random_kill_process similarity index 100% rename from tests/_utils/random_kill_process rename to tests/integration_tests/_utils/random_kill_process diff --git a/tests/_utils/run_cdc_cli b/tests/integration_tests/_utils/run_cdc_cli similarity index 100% rename from tests/_utils/run_cdc_cli rename to tests/integration_tests/_utils/run_cdc_cli diff --git a/tests/_utils/run_cdc_cli_tso_query b/tests/integration_tests/_utils/run_cdc_cli_tso_query similarity index 100% rename from tests/_utils/run_cdc_cli_tso_query rename to tests/integration_tests/_utils/run_cdc_cli_tso_query diff --git a/tests/_utils/run_cdc_server b/tests/integration_tests/_utils/run_cdc_server similarity index 100% rename from tests/_utils/run_cdc_server rename to tests/integration_tests/_utils/run_cdc_server diff --git a/tests/_utils/run_kafka_consumer b/tests/integration_tests/_utils/run_kafka_consumer similarity index 100% rename from tests/_utils/run_kafka_consumer rename to tests/integration_tests/_utils/run_kafka_consumer diff --git a/tests/_utils/run_sql b/tests/integration_tests/_utils/run_sql similarity index 100% rename from tests/_utils/run_sql rename to tests/integration_tests/_utils/run_sql diff --git a/tests/_utils/run_sql_file b/tests/integration_tests/_utils/run_sql_file similarity index 100% rename from tests/_utils/run_sql_file rename to tests/integration_tests/_utils/run_sql_file diff --git a/tests/_utils/start_tidb_cluster b/tests/integration_tests/_utils/start_tidb_cluster similarity index 100% rename from tests/_utils/start_tidb_cluster rename to tests/integration_tests/_utils/start_tidb_cluster diff --git a/tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl similarity index 99% rename from tests/_utils/start_tidb_cluster_impl rename to tests/integration_tests/_utils/start_tidb_cluster_impl index 0afb3952fef..911fb2c5e25 100755 --- a/tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -372,7 +372,7 @@ while ! curl -o /dev/null -sf http://127.0.0.1:17000/metrics 1>/dev/null 2>&1; d done echo "Starting CDC state checker..." -cd $CUR/../../testing_utils/cdc_state_checker +cd $CUR/../../../testing_utils/cdc_state_checker if [ ! -f ./cdc_state_checker ]; then GO111MODULE=on go build fi diff --git a/tests/_utils/start_tls_tidb_cluster b/tests/integration_tests/_utils/start_tls_tidb_cluster similarity index 100% rename from tests/_utils/start_tls_tidb_cluster rename to tests/integration_tests/_utils/start_tls_tidb_cluster diff --git a/tests/_utils/start_tls_tidb_cluster_impl b/tests/integration_tests/_utils/start_tls_tidb_cluster_impl similarity index 98% rename from tests/_utils/start_tls_tidb_cluster_impl rename to tests/integration_tests/_utils/start_tls_tidb_cluster_impl index 37dbf784829..044b2bf3630 100755 --- a/tests/_utils/start_tls_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tls_tidb_cluster_impl @@ -142,7 +142,7 @@ run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc --ssl-key=$TLS_DIR/server-key.pem echo "Starting CDC state checker..." -cd $CUR/../../testing_utils/cdc_state_checker +cd $CUR/../../../testing_utils/cdc_state_checker if [ ! -f ./cdc_state_checker ]; then GO111MODULE=on go build fi diff --git a/tests/_utils/stop_tidb_cluster b/tests/integration_tests/_utils/stop_tidb_cluster similarity index 100% rename from tests/_utils/stop_tidb_cluster rename to tests/integration_tests/_utils/stop_tidb_cluster diff --git a/tests/_utils/test_prepare b/tests/integration_tests/_utils/test_prepare similarity index 100% rename from tests/_utils/test_prepare rename to tests/integration_tests/_utils/test_prepare diff --git a/tests/_utils/tiflash-users.toml b/tests/integration_tests/_utils/tiflash-users.toml similarity index 100% rename from tests/_utils/tiflash-users.toml rename to tests/integration_tests/_utils/tiflash-users.toml diff --git a/tests/autorandom/conf/diff_config.toml b/tests/integration_tests/autorandom/conf/diff_config.toml similarity index 86% rename from tests/autorandom/conf/diff_config.toml rename to tests/integration_tests/autorandom/conf/diff_config.toml index 6b27666eaf6..0c7c54811ee 100644 --- a/tests/autorandom/conf/diff_config.toml +++ b/tests/integration_tests/autorandom/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/autorandom/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/autorandom/conf/tidb_config.toml b/tests/integration_tests/autorandom/conf/tidb_config.toml similarity index 100% rename from tests/autorandom/conf/tidb_config.toml rename to tests/integration_tests/autorandom/conf/tidb_config.toml diff --git a/tests/autorandom/data/test.sql b/tests/integration_tests/autorandom/data/test.sql similarity index 100% rename from tests/autorandom/data/test.sql rename to tests/integration_tests/autorandom/data/test.sql diff --git a/tests/autorandom/run.sh b/tests/integration_tests/autorandom/run.sh similarity index 100% rename from tests/autorandom/run.sh rename to tests/integration_tests/autorandom/run.sh diff --git a/tests/availability/capture.sh b/tests/integration_tests/availability/capture.sh similarity index 100% rename from tests/availability/capture.sh rename to tests/integration_tests/availability/capture.sh diff --git a/tests/availability/owner.sh b/tests/integration_tests/availability/owner.sh similarity index 100% rename from tests/availability/owner.sh rename to tests/integration_tests/availability/owner.sh diff --git a/tests/availability/processor.sh b/tests/integration_tests/availability/processor.sh similarity index 100% rename from tests/availability/processor.sh rename to tests/integration_tests/availability/processor.sh diff --git a/tests/availability/run.sh b/tests/integration_tests/availability/run.sh similarity index 100% rename from tests/availability/run.sh rename to tests/integration_tests/availability/run.sh diff --git a/tests/bank/bank.go b/tests/integration_tests/bank/bank.go similarity index 100% rename from tests/bank/bank.go rename to tests/integration_tests/bank/bank.go diff --git a/tests/bank/case.go b/tests/integration_tests/bank/case.go similarity index 100% rename from tests/bank/case.go rename to tests/integration_tests/bank/case.go diff --git a/tests/bank/run.sh b/tests/integration_tests/bank/run.sh similarity index 100% rename from tests/bank/run.sh rename to tests/integration_tests/bank/run.sh diff --git a/tests/batch_add_table/conf/diff_config.toml b/tests/integration_tests/batch_add_table/conf/diff_config.toml similarity index 85% rename from tests/batch_add_table/conf/diff_config.toml rename to tests/integration_tests/batch_add_table/conf/diff_config.toml index 48fc5dabeae..fb2e5b0d977 100644 --- a/tests/batch_add_table/conf/diff_config.toml +++ b/tests/integration_tests/batch_add_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/batch_add_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/batch_add_table/data/prepare.sql b/tests/integration_tests/batch_add_table/data/prepare.sql similarity index 100% rename from tests/batch_add_table/data/prepare.sql rename to tests/integration_tests/batch_add_table/data/prepare.sql diff --git a/tests/batch_add_table/data/test.sql b/tests/integration_tests/batch_add_table/data/test.sql similarity index 100% rename from tests/batch_add_table/data/test.sql rename to tests/integration_tests/batch_add_table/data/test.sql diff --git a/tests/batch_add_table/run.sh b/tests/integration_tests/batch_add_table/run.sh similarity index 100% rename from tests/batch_add_table/run.sh rename to tests/integration_tests/batch_add_table/run.sh diff --git a/tests/capture_session_done_during_task/conf/diff_config.toml b/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml similarity index 83% rename from tests/capture_session_done_during_task/conf/diff_config.toml rename to tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml index d5bbea9452c..190cee78e06 100644 --- a/tests/capture_session_done_during_task/conf/diff_config.toml +++ b/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/capture_session_done_during_task/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/capture_session_done_during_task/run.sh b/tests/integration_tests/capture_session_done_during_task/run.sh similarity index 100% rename from tests/capture_session_done_during_task/run.sh rename to tests/integration_tests/capture_session_done_during_task/run.sh diff --git a/tests/capture_suicide_while_balance_table/conf/diff_config.toml b/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml similarity index 83% rename from tests/capture_suicide_while_balance_table/conf/diff_config.toml rename to tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml index 7a52b92d91c..381543403c4 100644 --- a/tests/capture_suicide_while_balance_table/conf/diff_config.toml +++ b/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/capture_suicide_while_balance_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/capture_suicide_while_balance_table/run.sh b/tests/integration_tests/capture_suicide_while_balance_table/run.sh similarity index 100% rename from tests/capture_suicide_while_balance_table/run.sh rename to tests/integration_tests/capture_suicide_while_balance_table/run.sh diff --git a/tests/cdc/cdc.go b/tests/integration_tests/cdc/cdc.go similarity index 93% rename from tests/cdc/cdc.go rename to tests/integration_tests/cdc/cdc.go index a12fbf519f6..a62dcad8b98 100644 --- a/tests/cdc/cdc.go +++ b/tests/integration_tests/cdc/cdc.go @@ -20,8 +20,8 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/dailytest" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/dailytest" + "github.com/pingcap/ticdc/tests/integration_tests/util" ) func main() { diff --git a/tests/cdc/config.toml b/tests/integration_tests/cdc/config.toml similarity index 100% rename from tests/cdc/config.toml rename to tests/integration_tests/cdc/config.toml diff --git a/tests/cdc/run.sh b/tests/integration_tests/cdc/run.sh similarity index 100% rename from tests/cdc/run.sh rename to tests/integration_tests/cdc/run.sh diff --git a/tests/changefeed_auto_stop/conf/diff_config.toml b/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml similarity index 87% rename from tests/changefeed_auto_stop/conf/diff_config.toml rename to tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml index 5946907b719..a50bddac095 100644 --- a/tests/changefeed_auto_stop/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_auto_stop/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/changefeed_auto_stop/conf/workload b/tests/integration_tests/changefeed_auto_stop/conf/workload similarity index 100% rename from tests/changefeed_auto_stop/conf/workload rename to tests/integration_tests/changefeed_auto_stop/conf/workload diff --git a/tests/changefeed_auto_stop/run.sh b/tests/integration_tests/changefeed_auto_stop/run.sh similarity index 100% rename from tests/changefeed_auto_stop/run.sh rename to tests/integration_tests/changefeed_auto_stop/run.sh diff --git a/tests/changefeed_error/conf/diff_config.toml b/tests/integration_tests/changefeed_error/conf/diff_config.toml similarity index 85% rename from tests/changefeed_error/conf/diff_config.toml rename to tests/integration_tests/changefeed_error/conf/diff_config.toml index 54e11c9d242..394f3b00b70 100644 --- a/tests/changefeed_error/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_error/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_error/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/changefeed_error/conf/workload b/tests/integration_tests/changefeed_error/conf/workload similarity index 100% rename from tests/changefeed_error/conf/workload rename to tests/integration_tests/changefeed_error/conf/workload diff --git a/tests/changefeed_error/run.sh b/tests/integration_tests/changefeed_error/run.sh similarity index 100% rename from tests/changefeed_error/run.sh rename to tests/integration_tests/changefeed_error/run.sh diff --git a/tests/changefeed_fast_fail/run.sh b/tests/integration_tests/changefeed_fast_fail/run.sh similarity index 100% rename from tests/changefeed_fast_fail/run.sh rename to tests/integration_tests/changefeed_fast_fail/run.sh diff --git a/tests/changefeed_finish/conf/diff_config.toml b/tests/integration_tests/changefeed_finish/conf/diff_config.toml similarity index 85% rename from tests/changefeed_finish/conf/diff_config.toml rename to tests/integration_tests/changefeed_finish/conf/diff_config.toml index b191d2973d8..4d551f7d5a5 100644 --- a/tests/changefeed_finish/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_finish/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_finish/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/changefeed_finish/run.sh b/tests/integration_tests/changefeed_finish/run.sh similarity index 100% rename from tests/changefeed_finish/run.sh rename to tests/integration_tests/changefeed_finish/run.sh diff --git a/tests/changefeed_pause_resume/conf/diff_config.toml b/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml similarity index 84% rename from tests/changefeed_pause_resume/conf/diff_config.toml rename to tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml index b8eeb0b3a3d..1bb531f7004 100644 --- a/tests/changefeed_pause_resume/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_pause_resume/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/changefeed_pause_resume/run.sh b/tests/integration_tests/changefeed_pause_resume/run.sh similarity index 100% rename from tests/changefeed_pause_resume/run.sh rename to tests/integration_tests/changefeed_pause_resume/run.sh diff --git a/tests/changefeed_reconstruct/conf/diff_config.toml b/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml similarity index 85% rename from tests/changefeed_reconstruct/conf/diff_config.toml rename to tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml index cf34a8c82fe..1181abbc6c2 100644 --- a/tests/changefeed_reconstruct/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_reconstruct/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/changefeed_reconstruct/conf/workload b/tests/integration_tests/changefeed_reconstruct/conf/workload similarity index 100% rename from tests/changefeed_reconstruct/conf/workload rename to tests/integration_tests/changefeed_reconstruct/conf/workload diff --git a/tests/changefeed_reconstruct/run.sh b/tests/integration_tests/changefeed_reconstruct/run.sh similarity index 100% rename from tests/changefeed_reconstruct/run.sh rename to tests/integration_tests/changefeed_reconstruct/run.sh diff --git a/tests/cli/run.sh b/tests/integration_tests/cli/run.sh similarity index 100% rename from tests/cli/run.sh rename to tests/integration_tests/cli/run.sh diff --git a/tests/clustered_index/conf/diff_config.toml b/tests/integration_tests/clustered_index/conf/diff_config.toml similarity index 85% rename from tests/clustered_index/conf/diff_config.toml rename to tests/integration_tests/clustered_index/conf/diff_config.toml index 06604de2bfb..cc84ddc3606 100644 --- a/tests/clustered_index/conf/diff_config.toml +++ b/tests/integration_tests/clustered_index/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/clustered_index/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/clustered_index/data/test.sql b/tests/integration_tests/clustered_index/data/test.sql similarity index 100% rename from tests/clustered_index/data/test.sql rename to tests/integration_tests/clustered_index/data/test.sql diff --git a/tests/clustered_index/run.sh b/tests/integration_tests/clustered_index/run.sh similarity index 100% rename from tests/clustered_index/run.sh rename to tests/integration_tests/clustered_index/run.sh diff --git a/tests/common_1/conf/diff_config.toml b/tests/integration_tests/common_1/conf/diff_config.toml similarity index 86% rename from tests/common_1/conf/diff_config.toml rename to tests/integration_tests/common_1/conf/diff_config.toml index 6df50ab03bd..36b4eb0a9fe 100644 --- a/tests/common_1/conf/diff_config.toml +++ b/tests/integration_tests/common_1/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/common_1/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/common_1/data/test.sql b/tests/integration_tests/common_1/data/test.sql similarity index 100% rename from tests/common_1/data/test.sql rename to tests/integration_tests/common_1/data/test.sql diff --git a/tests/common_1/data/test_finish.sql b/tests/integration_tests/common_1/data/test_finish.sql similarity index 100% rename from tests/common_1/data/test_finish.sql rename to tests/integration_tests/common_1/data/test_finish.sql diff --git a/tests/common_1/data/test_v5.sql b/tests/integration_tests/common_1/data/test_v5.sql similarity index 100% rename from tests/common_1/data/test_v5.sql rename to tests/integration_tests/common_1/data/test_v5.sql diff --git a/tests/common_1/run.sh b/tests/integration_tests/common_1/run.sh similarity index 100% rename from tests/common_1/run.sh rename to tests/integration_tests/common_1/run.sh diff --git a/tests/consistent_replicate_nfs/conf/changefeed.toml b/tests/integration_tests/consistent_replicate_nfs/conf/changefeed.toml similarity index 100% rename from tests/consistent_replicate_nfs/conf/changefeed.toml rename to tests/integration_tests/consistent_replicate_nfs/conf/changefeed.toml diff --git a/tests/consistent_replicate_nfs/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml similarity index 84% rename from tests/consistent_replicate_nfs/conf/diff_config.toml rename to tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml index 908fdb26da8..9677f26b0e4 100644 --- a/tests/consistent_replicate_nfs/conf/diff_config.toml +++ b/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_nfs/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/consistent_replicate_nfs/conf/workload b/tests/integration_tests/consistent_replicate_nfs/conf/workload similarity index 100% rename from tests/consistent_replicate_nfs/conf/workload rename to tests/integration_tests/consistent_replicate_nfs/conf/workload diff --git a/tests/consistent_replicate_nfs/run.sh b/tests/integration_tests/consistent_replicate_nfs/run.sh similarity index 100% rename from tests/consistent_replicate_nfs/run.sh rename to tests/integration_tests/consistent_replicate_nfs/run.sh diff --git a/tests/consistent_replicate_s3/conf/changefeed.toml b/tests/integration_tests/consistent_replicate_s3/conf/changefeed.toml similarity index 100% rename from tests/consistent_replicate_s3/conf/changefeed.toml rename to tests/integration_tests/consistent_replicate_s3/conf/changefeed.toml diff --git a/tests/consistent_replicate_s3/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml similarity index 84% rename from tests/consistent_replicate_s3/conf/diff_config.toml rename to tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml index ef672ccb24c..610eab9dc1e 100644 --- a/tests/consistent_replicate_s3/conf/diff_config.toml +++ b/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_s3/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/consistent_replicate_s3/conf/workload b/tests/integration_tests/consistent_replicate_s3/conf/workload similarity index 100% rename from tests/consistent_replicate_s3/conf/workload rename to tests/integration_tests/consistent_replicate_s3/conf/workload diff --git a/tests/consistent_replicate_s3/run.sh b/tests/integration_tests/consistent_replicate_s3/run.sh similarity index 100% rename from tests/consistent_replicate_s3/run.sh rename to tests/integration_tests/consistent_replicate_s3/run.sh diff --git a/tests/cyclic_ab/conf/diff_config.toml b/tests/integration_tests/cyclic_ab/conf/diff_config.toml similarity index 86% rename from tests/cyclic_ab/conf/diff_config.toml rename to tests/integration_tests/cyclic_ab/conf/diff_config.toml index 26c78959203..122b642fc77 100644 --- a/tests/cyclic_ab/conf/diff_config.toml +++ b/tests/integration_tests/cyclic_ab/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_ab/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/cyclic_ab/conf/only_test_simple.toml b/tests/integration_tests/cyclic_ab/conf/only_test_simple.toml similarity index 100% rename from tests/cyclic_ab/conf/only_test_simple.toml rename to tests/integration_tests/cyclic_ab/conf/only_test_simple.toml diff --git a/tests/cyclic_ab/run.sh b/tests/integration_tests/cyclic_ab/run.sh similarity index 100% rename from tests/cyclic_ab/run.sh rename to tests/integration_tests/cyclic_ab/run.sh diff --git a/tests/cyclic_abc/conf/changefeed.toml b/tests/integration_tests/cyclic_abc/conf/changefeed.toml similarity index 100% rename from tests/cyclic_abc/conf/changefeed.toml rename to tests/integration_tests/cyclic_abc/conf/changefeed.toml diff --git a/tests/cyclic_abc/conf/diff_config_down_tls.toml b/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml similarity index 86% rename from tests/cyclic_abc/conf/diff_config_down_tls.toml rename to tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml index a38a536fe50..1973a48a3ae 100644 --- a/tests/cyclic_abc/conf/diff_config_down_tls.toml +++ b/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_abc/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/cyclic_abc/conf/diff_config_up_down.toml b/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml similarity index 86% rename from tests/cyclic_abc/conf/diff_config_up_down.toml rename to tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml index 26c78959203..bb38ca4729c 100644 --- a/tests/cyclic_abc/conf/diff_config_up_down.toml +++ b/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_abc/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/cyclic_abc/run.sh b/tests/integration_tests/cyclic_abc/run.sh similarity index 100% rename from tests/cyclic_abc/run.sh rename to tests/integration_tests/cyclic_abc/run.sh diff --git a/tests/dailytest/case.go b/tests/integration_tests/dailytest/case.go similarity index 100% rename from tests/dailytest/case.go rename to tests/integration_tests/dailytest/case.go diff --git a/tests/dailytest/dailytest.go b/tests/integration_tests/dailytest/dailytest.go similarity index 100% rename from tests/dailytest/dailytest.go rename to tests/integration_tests/dailytest/dailytest.go diff --git a/tests/dailytest/data.go b/tests/integration_tests/dailytest/data.go similarity index 100% rename from tests/dailytest/data.go rename to tests/integration_tests/dailytest/data.go diff --git a/tests/dailytest/db.go b/tests/integration_tests/dailytest/db.go similarity index 99% rename from tests/dailytest/db.go rename to tests/integration_tests/dailytest/db.go index 56b68b7f557..c0c7694bc79 100644 --- a/tests/dailytest/db.go +++ b/tests/integration_tests/dailytest/db.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap/zapcore" ) diff --git a/tests/dailytest/exector.go b/tests/integration_tests/dailytest/exector.go similarity index 100% rename from tests/dailytest/exector.go rename to tests/integration_tests/dailytest/exector.go diff --git a/tests/dailytest/job.go b/tests/integration_tests/dailytest/job.go similarity index 100% rename from tests/dailytest/job.go rename to tests/integration_tests/dailytest/job.go diff --git a/tests/dailytest/parser.go b/tests/integration_tests/dailytest/parser.go similarity index 100% rename from tests/dailytest/parser.go rename to tests/integration_tests/dailytest/parser.go diff --git a/tests/dailytest/rand.go b/tests/integration_tests/dailytest/rand.go similarity index 100% rename from tests/dailytest/rand.go rename to tests/integration_tests/dailytest/rand.go diff --git a/tests/ddl_attributes/conf/diff_config.toml b/tests/integration_tests/ddl_attributes/conf/diff_config.toml similarity index 85% rename from tests/ddl_attributes/conf/diff_config.toml rename to tests/integration_tests/ddl_attributes/conf/diff_config.toml index 5994a8fad64..81f746e9a9c 100644 --- a/tests/ddl_attributes/conf/diff_config.toml +++ b/tests/integration_tests/ddl_attributes/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_attributes/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/ddl_attributes/data/prepare.sql b/tests/integration_tests/ddl_attributes/data/prepare.sql similarity index 100% rename from tests/ddl_attributes/data/prepare.sql rename to tests/integration_tests/ddl_attributes/data/prepare.sql diff --git a/tests/ddl_attributes/run.sh b/tests/integration_tests/ddl_attributes/run.sh similarity index 100% rename from tests/ddl_attributes/run.sh rename to tests/integration_tests/ddl_attributes/run.sh diff --git a/tests/ddl_puller_lag/run.sh b/tests/integration_tests/ddl_puller_lag/run.sh similarity index 100% rename from tests/ddl_puller_lag/run.sh rename to tests/integration_tests/ddl_puller_lag/run.sh diff --git a/tests/ddl_reentrant/conf/diff_config.toml b/tests/integration_tests/ddl_reentrant/conf/diff_config.toml similarity index 86% rename from tests/ddl_reentrant/conf/diff_config.toml rename to tests/integration_tests/ddl_reentrant/conf/diff_config.toml index 32c241bd5d1..afc30f86a57 100644 --- a/tests/ddl_reentrant/conf/diff_config.toml +++ b/tests/integration_tests/ddl_reentrant/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_reentrant/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/ddl_reentrant/conf/tidb_config.toml b/tests/integration_tests/ddl_reentrant/conf/tidb_config.toml similarity index 100% rename from tests/ddl_reentrant/conf/tidb_config.toml rename to tests/integration_tests/ddl_reentrant/conf/tidb_config.toml diff --git a/tests/ddl_reentrant/run.sh b/tests/integration_tests/ddl_reentrant/run.sh similarity index 100% rename from tests/ddl_reentrant/run.sh rename to tests/integration_tests/ddl_reentrant/run.sh diff --git a/tests/ddl_sequence/conf/diff_config.toml b/tests/integration_tests/ddl_sequence/conf/diff_config.toml similarity index 86% rename from tests/ddl_sequence/conf/diff_config.toml rename to tests/integration_tests/ddl_sequence/conf/diff_config.toml index c06d75a429a..6ff0b3b6c7d 100644 --- a/tests/ddl_sequence/conf/diff_config.toml +++ b/tests/integration_tests/ddl_sequence/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_sequence/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/ddl_sequence/data/prepare.sql b/tests/integration_tests/ddl_sequence/data/prepare.sql similarity index 100% rename from tests/ddl_sequence/data/prepare.sql rename to tests/integration_tests/ddl_sequence/data/prepare.sql diff --git a/tests/ddl_sequence/run.sh b/tests/integration_tests/ddl_sequence/run.sh similarity index 100% rename from tests/ddl_sequence/run.sh rename to tests/integration_tests/ddl_sequence/run.sh diff --git a/tests/drop_many_tables/conf/diff_config.toml b/tests/integration_tests/drop_many_tables/conf/diff_config.toml similarity index 85% rename from tests/drop_many_tables/conf/diff_config.toml rename to tests/integration_tests/drop_many_tables/conf/diff_config.toml index e85c5294595..203c0607219 100644 --- a/tests/drop_many_tables/conf/diff_config.toml +++ b/tests/integration_tests/drop_many_tables/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/drop_many_tables/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/drop_many_tables/data/prepare.sql b/tests/integration_tests/drop_many_tables/data/prepare.sql similarity index 100% rename from tests/drop_many_tables/data/prepare.sql rename to tests/integration_tests/drop_many_tables/data/prepare.sql diff --git a/tests/drop_many_tables/run.sh b/tests/integration_tests/drop_many_tables/run.sh similarity index 100% rename from tests/drop_many_tables/run.sh rename to tests/integration_tests/drop_many_tables/run.sh diff --git a/tests/force_replicate_table/conf/changefeed.toml b/tests/integration_tests/force_replicate_table/conf/changefeed.toml similarity index 100% rename from tests/force_replicate_table/conf/changefeed.toml rename to tests/integration_tests/force_replicate_table/conf/changefeed.toml diff --git a/tests/force_replicate_table/conf/tidb_config.toml b/tests/integration_tests/force_replicate_table/conf/tidb_config.toml similarity index 100% rename from tests/force_replicate_table/conf/tidb_config.toml rename to tests/integration_tests/force_replicate_table/conf/tidb_config.toml diff --git a/tests/force_replicate_table/data/test.sql b/tests/integration_tests/force_replicate_table/data/test.sql similarity index 100% rename from tests/force_replicate_table/data/test.sql rename to tests/integration_tests/force_replicate_table/data/test.sql diff --git a/tests/force_replicate_table/run.sh b/tests/integration_tests/force_replicate_table/run.sh similarity index 100% rename from tests/force_replicate_table/run.sh rename to tests/integration_tests/force_replicate_table/run.sh diff --git a/tests/gc_safepoint/conf/diff_config.toml b/tests/integration_tests/gc_safepoint/conf/diff_config.toml similarity index 86% rename from tests/gc_safepoint/conf/diff_config.toml rename to tests/integration_tests/gc_safepoint/conf/diff_config.toml index 5d290c6ba58..f4e7a89971b 100644 --- a/tests/gc_safepoint/conf/diff_config.toml +++ b/tests/integration_tests/gc_safepoint/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/gc_safepoint/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/gc_safepoint/run.sh b/tests/integration_tests/gc_safepoint/run.sh similarity index 100% rename from tests/gc_safepoint/run.sh rename to tests/integration_tests/gc_safepoint/run.sh diff --git a/tests/generate_column/conf/diff_config.toml b/tests/integration_tests/generate_column/conf/diff_config.toml similarity index 85% rename from tests/generate_column/conf/diff_config.toml rename to tests/integration_tests/generate_column/conf/diff_config.toml index d3c05685f52..c3705f732b2 100644 --- a/tests/generate_column/conf/diff_config.toml +++ b/tests/integration_tests/generate_column/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/generate_column/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/generate_column/data/prepare.sql b/tests/integration_tests/generate_column/data/prepare.sql similarity index 100% rename from tests/generate_column/data/prepare.sql rename to tests/integration_tests/generate_column/data/prepare.sql diff --git a/tests/generate_column/run.sh b/tests/integration_tests/generate_column/run.sh similarity index 100% rename from tests/generate_column/run.sh rename to tests/integration_tests/generate_column/run.sh diff --git a/tests/http_api/run.sh b/tests/integration_tests/http_api/run.sh similarity index 100% rename from tests/http_api/run.sh rename to tests/integration_tests/http_api/run.sh diff --git a/tests/http_api/util/test_case.py b/tests/integration_tests/http_api/util/test_case.py similarity index 100% rename from tests/http_api/util/test_case.py rename to tests/integration_tests/http_api/util/test_case.py diff --git a/tests/http_proxies/run-proxy.go b/tests/integration_tests/http_proxies/run-proxy.go similarity index 100% rename from tests/http_proxies/run-proxy.go rename to tests/integration_tests/http_proxies/run-proxy.go diff --git a/tests/http_proxies/run.sh b/tests/integration_tests/http_proxies/run.sh similarity index 100% rename from tests/http_proxies/run.sh rename to tests/integration_tests/http_proxies/run.sh diff --git a/tests/kafka_messages/conf/diff_config.toml b/tests/integration_tests/kafka_messages/conf/diff_config.toml similarity index 86% rename from tests/kafka_messages/conf/diff_config.toml rename to tests/integration_tests/kafka_messages/conf/diff_config.toml index 527f0835f00..f4a6d29c149 100644 --- a/tests/kafka_messages/conf/diff_config.toml +++ b/tests/integration_tests/kafka_messages/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kafka_message/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/kafka_messages/conf/workload b/tests/integration_tests/kafka_messages/conf/workload similarity index 100% rename from tests/kafka_messages/conf/workload rename to tests/integration_tests/kafka_messages/conf/workload diff --git a/tests/kafka_messages/run.sh b/tests/integration_tests/kafka_messages/run.sh similarity index 100% rename from tests/kafka_messages/run.sh rename to tests/integration_tests/kafka_messages/run.sh diff --git a/tests/kafka_sink_error_resume/conf/diff_config.toml b/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml similarity index 84% rename from tests/kafka_sink_error_resume/conf/diff_config.toml rename to tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml index 15774d89159..db689a10112 100644 --- a/tests/kafka_sink_error_resume/conf/diff_config.toml +++ b/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kafka_sink_error_resume/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/kafka_sink_error_resume/run.sh b/tests/integration_tests/kafka_sink_error_resume/run.sh similarity index 100% rename from tests/kafka_sink_error_resume/run.sh rename to tests/integration_tests/kafka_sink_error_resume/run.sh diff --git a/tests/kill_owner_with_ddl/conf/diff_config.toml b/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml similarity index 85% rename from tests/kill_owner_with_ddl/conf/diff_config.toml rename to tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml index b5e8b1d7914..040865d28be 100644 --- a/tests/kill_owner_with_ddl/conf/diff_config.toml +++ b/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kill_owner_with_ddl/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/kill_owner_with_ddl/run.sh b/tests/integration_tests/kill_owner_with_ddl/run.sh similarity index 100% rename from tests/kill_owner_with_ddl/run.sh rename to tests/integration_tests/kill_owner_with_ddl/run.sh diff --git a/tests/kv_client_stream_reconnect/conf/diff_config.toml b/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml similarity index 84% rename from tests/kv_client_stream_reconnect/conf/diff_config.toml rename to tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml index 8b63ddae343..197ca066b01 100644 --- a/tests/kv_client_stream_reconnect/conf/diff_config.toml +++ b/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kv_client_stream_reconnect/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/kv_client_stream_reconnect/run.sh b/tests/integration_tests/kv_client_stream_reconnect/run.sh similarity index 100% rename from tests/kv_client_stream_reconnect/run.sh rename to tests/integration_tests/kv_client_stream_reconnect/run.sh diff --git a/tests/many_pk_or_uk/config.toml b/tests/integration_tests/many_pk_or_uk/config.toml similarity index 100% rename from tests/many_pk_or_uk/config.toml rename to tests/integration_tests/many_pk_or_uk/config.toml diff --git a/tests/processor_panic/diff_config.toml b/tests/integration_tests/many_pk_or_uk/diff_config.toml similarity index 85% rename from tests/processor_panic/diff_config.toml rename to tests/integration_tests/many_pk_or_uk/diff_config.toml index 26c78959203..5247e336450 100644 --- a/tests/processor_panic/diff_config.toml +++ b/tests/integration_tests/many_pk_or_uk/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/many_pk_or_uk/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/many_pk_or_uk/main.go b/tests/integration_tests/many_pk_or_uk/main.go similarity index 98% rename from tests/many_pk_or_uk/main.go rename to tests/integration_tests/many_pk_or_uk/main.go index 080a2d69328..152e40acaba 100644 --- a/tests/many_pk_or_uk/main.go +++ b/tests/integration_tests/many_pk_or_uk/main.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" ) func main() { diff --git a/tests/many_pk_or_uk/run.sh b/tests/integration_tests/many_pk_or_uk/run.sh similarity index 100% rename from tests/many_pk_or_uk/run.sh rename to tests/integration_tests/many_pk_or_uk/run.sh diff --git a/tests/move_table/conf/diff_config.toml b/tests/integration_tests/move_table/conf/diff_config.toml similarity index 86% rename from tests/move_table/conf/diff_config.toml rename to tests/integration_tests/move_table/conf/diff_config.toml index 6d8a575cb8a..c6aeddb77d9 100644 --- a/tests/move_table/conf/diff_config.toml +++ b/tests/integration_tests/move_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/move_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/move_table/conf/workload b/tests/integration_tests/move_table/conf/workload similarity index 100% rename from tests/move_table/conf/workload rename to tests/integration_tests/move_table/conf/workload diff --git a/tests/move_table/main.go b/tests/integration_tests/move_table/main.go similarity index 100% rename from tests/move_table/main.go rename to tests/integration_tests/move_table/main.go diff --git a/tests/move_table/run.sh b/tests/integration_tests/move_table/run.sh similarity index 100% rename from tests/move_table/run.sh rename to tests/integration_tests/move_table/run.sh diff --git a/tests/multi_capture/conf/diff_config.toml b/tests/integration_tests/multi_capture/conf/diff_config.toml similarity index 88% rename from tests/multi_capture/conf/diff_config.toml rename to tests/integration_tests/multi_capture/conf/diff_config.toml index 89dd57024a3..f6cd453010a 100644 --- a/tests/multi_capture/conf/diff_config.toml +++ b/tests/integration_tests/multi_capture/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/multi_capture/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/multi_capture/conf/workload1 b/tests/integration_tests/multi_capture/conf/workload1 similarity index 100% rename from tests/multi_capture/conf/workload1 rename to tests/integration_tests/multi_capture/conf/workload1 diff --git a/tests/multi_capture/conf/workload2 b/tests/integration_tests/multi_capture/conf/workload2 similarity index 100% rename from tests/multi_capture/conf/workload2 rename to tests/integration_tests/multi_capture/conf/workload2 diff --git a/tests/multi_capture/run.sh b/tests/integration_tests/multi_capture/run.sh similarity index 100% rename from tests/multi_capture/run.sh rename to tests/integration_tests/multi_capture/run.sh diff --git a/tests/multi_changefeed/conf/changefeed1.toml b/tests/integration_tests/multi_changefeed/conf/changefeed1.toml similarity index 100% rename from tests/multi_changefeed/conf/changefeed1.toml rename to tests/integration_tests/multi_changefeed/conf/changefeed1.toml diff --git a/tests/multi_changefeed/conf/changefeed2.toml b/tests/integration_tests/multi_changefeed/conf/changefeed2.toml similarity index 100% rename from tests/multi_changefeed/conf/changefeed2.toml rename to tests/integration_tests/multi_changefeed/conf/changefeed2.toml diff --git a/tests/multi_changefeed/run.sh b/tests/integration_tests/multi_changefeed/run.sh similarity index 100% rename from tests/multi_changefeed/run.sh rename to tests/integration_tests/multi_changefeed/run.sh diff --git a/tests/multi_source/config.toml b/tests/integration_tests/multi_source/config.toml similarity index 100% rename from tests/multi_source/config.toml rename to tests/integration_tests/multi_source/config.toml diff --git a/tests/many_pk_or_uk/diff_config.toml b/tests/integration_tests/multi_source/diff_config.toml similarity index 86% rename from tests/many_pk_or_uk/diff_config.toml rename to tests/integration_tests/multi_source/diff_config.toml index 26c78959203..103c77690c9 100644 --- a/tests/many_pk_or_uk/diff_config.toml +++ b/tests/integration_tests/multi_source/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/multi_source/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/multi_source/main.go b/tests/integration_tests/multi_source/main.go similarity index 99% rename from tests/multi_source/main.go rename to tests/integration_tests/multi_source/main.go index e6208efa94c..3324d5ee5ba 100644 --- a/tests/multi_source/main.go +++ b/tests/integration_tests/multi_source/main.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "go.uber.org/zap" ) diff --git a/tests/multi_source/run.sh b/tests/integration_tests/multi_source/run.sh similarity index 100% rename from tests/multi_source/run.sh rename to tests/integration_tests/multi_source/run.sh diff --git a/tests/new_ci_collation_with_old_value/conf/changefeed.toml b/tests/integration_tests/new_ci_collation_with_old_value/conf/changefeed.toml similarity index 100% rename from tests/new_ci_collation_with_old_value/conf/changefeed.toml rename to tests/integration_tests/new_ci_collation_with_old_value/conf/changefeed.toml diff --git a/tests/new_ci_collation_with_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml similarity index 83% rename from tests/new_ci_collation_with_old_value/conf/diff_config.toml rename to tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml index ed282c8a8ad..32b8417e91f 100644 --- a/tests/new_ci_collation_with_old_value/conf/diff_config.toml +++ b/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/new_ci_collation_with_old_value/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/new_ci_collation_with_old_value/conf/tidb_config.toml b/tests/integration_tests/new_ci_collation_with_old_value/conf/tidb_config.toml similarity index 100% rename from tests/new_ci_collation_with_old_value/conf/tidb_config.toml rename to tests/integration_tests/new_ci_collation_with_old_value/conf/tidb_config.toml diff --git a/tests/new_ci_collation_with_old_value/data/test1.sql b/tests/integration_tests/new_ci_collation_with_old_value/data/test1.sql similarity index 100% rename from tests/new_ci_collation_with_old_value/data/test1.sql rename to tests/integration_tests/new_ci_collation_with_old_value/data/test1.sql diff --git a/tests/new_ci_collation_with_old_value/data/test2.sql b/tests/integration_tests/new_ci_collation_with_old_value/data/test2.sql similarity index 100% rename from tests/new_ci_collation_with_old_value/data/test2.sql rename to tests/integration_tests/new_ci_collation_with_old_value/data/test2.sql diff --git a/tests/new_ci_collation_with_old_value/run.sh b/tests/integration_tests/new_ci_collation_with_old_value/run.sh similarity index 100% rename from tests/new_ci_collation_with_old_value/run.sh rename to tests/integration_tests/new_ci_collation_with_old_value/run.sh diff --git a/tests/new_ci_collation_without_old_value/conf/changefeed.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/changefeed.toml similarity index 100% rename from tests/new_ci_collation_without_old_value/conf/changefeed.toml rename to tests/integration_tests/new_ci_collation_without_old_value/conf/changefeed.toml diff --git a/tests/new_ci_collation_without_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml similarity index 83% rename from tests/new_ci_collation_without_old_value/conf/diff_config.toml rename to tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml index 19607ddf9ad..412de80d2a2 100644 --- a/tests/new_ci_collation_without_old_value/conf/diff_config.toml +++ b/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/new_ci_collation_without_old_value/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/new_ci_collation_without_old_value/conf/tidb_config.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/tidb_config.toml similarity index 100% rename from tests/new_ci_collation_without_old_value/conf/tidb_config.toml rename to tests/integration_tests/new_ci_collation_without_old_value/conf/tidb_config.toml diff --git a/tests/new_ci_collation_without_old_value/data/test1.sql b/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql similarity index 100% rename from tests/new_ci_collation_without_old_value/data/test1.sql rename to tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql diff --git a/tests/new_ci_collation_without_old_value/data/test2.sql b/tests/integration_tests/new_ci_collation_without_old_value/data/test2.sql similarity index 100% rename from tests/new_ci_collation_without_old_value/data/test2.sql rename to tests/integration_tests/new_ci_collation_without_old_value/data/test2.sql diff --git a/tests/new_ci_collation_without_old_value/run.sh b/tests/integration_tests/new_ci_collation_without_old_value/run.sh similarity index 100% rename from tests/new_ci_collation_without_old_value/run.sh rename to tests/integration_tests/new_ci_collation_without_old_value/run.sh diff --git a/tests/owner_remove_table_error/conf/diff_config.toml b/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml similarity index 84% rename from tests/owner_remove_table_error/conf/diff_config.toml rename to tests/integration_tests/owner_remove_table_error/conf/diff_config.toml index ea5a5674744..750741652fa 100644 --- a/tests/owner_remove_table_error/conf/diff_config.toml +++ b/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/owner_remove_table_error/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/owner_remove_table_error/run.sh b/tests/integration_tests/owner_remove_table_error/run.sh similarity index 100% rename from tests/owner_remove_table_error/run.sh rename to tests/integration_tests/owner_remove_table_error/run.sh diff --git a/tests/partition_table/conf/diff_config.toml b/tests/integration_tests/partition_table/conf/diff_config.toml similarity index 85% rename from tests/partition_table/conf/diff_config.toml rename to tests/integration_tests/partition_table/conf/diff_config.toml index 8a03301f160..24e98abed31 100644 --- a/tests/partition_table/conf/diff_config.toml +++ b/tests/integration_tests/partition_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/partition_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/partition_table/data/prepare.sql b/tests/integration_tests/partition_table/data/prepare.sql similarity index 100% rename from tests/partition_table/data/prepare.sql rename to tests/integration_tests/partition_table/data/prepare.sql diff --git a/tests/partition_table/run.sh b/tests/integration_tests/partition_table/run.sh similarity index 100% rename from tests/partition_table/run.sh rename to tests/integration_tests/partition_table/run.sh diff --git a/tests/processor_err_chan/conf/diff_config.toml b/tests/integration_tests/processor_err_chan/conf/diff_config.toml similarity index 85% rename from tests/processor_err_chan/conf/diff_config.toml rename to tests/integration_tests/processor_err_chan/conf/diff_config.toml index 0ba4707fd81..6bfc3058a1a 100644 --- a/tests/processor_err_chan/conf/diff_config.toml +++ b/tests/integration_tests/processor_err_chan/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_err_chan/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/processor_err_chan/run.sh b/tests/integration_tests/processor_err_chan/run.sh similarity index 100% rename from tests/processor_err_chan/run.sh rename to tests/integration_tests/processor_err_chan/run.sh diff --git a/tests/processor_panic/config.toml b/tests/integration_tests/processor_panic/config.toml similarity index 100% rename from tests/processor_panic/config.toml rename to tests/integration_tests/processor_panic/config.toml diff --git a/tests/integration_tests/processor_panic/diff_config.toml b/tests/integration_tests/processor_panic/diff_config.toml new file mode 100644 index 00000000000..9485bb2c819 --- /dev/null +++ b/tests/integration_tests/processor_panic/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/processor_panic/sync_diff/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["test.?*"] + +[data-sources] +[data-sources.mysql1] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.tidb0] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/processor_panic/main.go b/tests/integration_tests/processor_panic/main.go similarity index 97% rename from tests/processor_panic/main.go rename to tests/integration_tests/processor_panic/main.go index cae764ac4cb..c6aef27aa22 100644 --- a/tests/processor_panic/main.go +++ b/tests/integration_tests/processor_panic/main.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/tests/processor_panic/run.sh b/tests/integration_tests/processor_panic/run.sh similarity index 100% rename from tests/processor_panic/run.sh rename to tests/integration_tests/processor_panic/run.sh diff --git a/tests/processor_resolved_ts_fallback/conf/diff_config.toml b/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml similarity index 83% rename from tests/processor_resolved_ts_fallback/conf/diff_config.toml rename to tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml index 9dca856dd84..1796c36ecb3 100644 --- a/tests/processor_resolved_ts_fallback/conf/diff_config.toml +++ b/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_resolved_ts_fallback/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/processor_resolved_ts_fallback/run.sh b/tests/integration_tests/processor_resolved_ts_fallback/run.sh similarity index 100% rename from tests/processor_resolved_ts_fallback/run.sh rename to tests/integration_tests/processor_resolved_ts_fallback/run.sh diff --git a/tests/processor_stop_delay/conf/diff_config.toml b/tests/integration_tests/processor_stop_delay/conf/diff_config.toml similarity index 85% rename from tests/processor_stop_delay/conf/diff_config.toml rename to tests/integration_tests/processor_stop_delay/conf/diff_config.toml index cb55b724d49..83d857c924c 100644 --- a/tests/processor_stop_delay/conf/diff_config.toml +++ b/tests/integration_tests/processor_stop_delay/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_stop_delay/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/processor_stop_delay/run.sh b/tests/integration_tests/processor_stop_delay/run.sh similarity index 100% rename from tests/processor_stop_delay/run.sh rename to tests/integration_tests/processor_stop_delay/run.sh diff --git a/tests/region_merge/conf/diff_config.toml b/tests/integration_tests/region_merge/conf/diff_config.toml similarity index 86% rename from tests/region_merge/conf/diff_config.toml rename to tests/integration_tests/region_merge/conf/diff_config.toml index 8dbd018c17d..0a2b43d5a2f 100644 --- a/tests/region_merge/conf/diff_config.toml +++ b/tests/integration_tests/region_merge/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/region_merge/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/region_merge/conf/pd_config.toml b/tests/integration_tests/region_merge/conf/pd_config.toml similarity index 100% rename from tests/region_merge/conf/pd_config.toml rename to tests/integration_tests/region_merge/conf/pd_config.toml diff --git a/tests/region_merge/run.sh b/tests/integration_tests/region_merge/run.sh similarity index 100% rename from tests/region_merge/run.sh rename to tests/integration_tests/region_merge/run.sh diff --git a/tests/resolve_lock/config.toml b/tests/integration_tests/resolve_lock/config.toml similarity index 100% rename from tests/resolve_lock/config.toml rename to tests/integration_tests/resolve_lock/config.toml diff --git a/tests/multi_source/diff_config.toml b/tests/integration_tests/resolve_lock/diff_config.toml similarity index 86% rename from tests/multi_source/diff_config.toml rename to tests/integration_tests/resolve_lock/diff_config.toml index 26c78959203..1af712185a9 100644 --- a/tests/multi_source/diff_config.toml +++ b/tests/integration_tests/resolve_lock/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/resolve_lock/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/resolve_lock/main.go b/tests/integration_tests/resolve_lock/main.go similarity index 99% rename from tests/resolve_lock/main.go rename to tests/integration_tests/resolve_lock/main.go index a8f237f933f..f02ed745270 100644 --- a/tests/resolve_lock/main.go +++ b/tests/integration_tests/resolve_lock/main.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/driver" diff --git a/tests/resolve_lock/run.sh b/tests/integration_tests/resolve_lock/run.sh similarity index 100% rename from tests/resolve_lock/run.sh rename to tests/integration_tests/resolve_lock/run.sh diff --git a/tests/row_format/conf/diff_config.toml b/tests/integration_tests/row_format/conf/diff_config.toml similarity index 86% rename from tests/row_format/conf/diff_config.toml rename to tests/integration_tests/row_format/conf/diff_config.toml index d5f470cfec6..28572faf4f5 100644 --- a/tests/row_format/conf/diff_config.toml +++ b/tests/integration_tests/row_format/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/row_format/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/row_format/data/step1.sql b/tests/integration_tests/row_format/data/step1.sql similarity index 100% rename from tests/row_format/data/step1.sql rename to tests/integration_tests/row_format/data/step1.sql diff --git a/tests/row_format/data/step2.sql b/tests/integration_tests/row_format/data/step2.sql similarity index 100% rename from tests/row_format/data/step2.sql rename to tests/integration_tests/row_format/data/step2.sql diff --git a/tests/row_format/data/step3.sql b/tests/integration_tests/row_format/data/step3.sql similarity index 100% rename from tests/row_format/data/step3.sql rename to tests/integration_tests/row_format/data/step3.sql diff --git a/tests/row_format/data/step4.sql b/tests/integration_tests/row_format/data/step4.sql similarity index 100% rename from tests/row_format/data/step4.sql rename to tests/integration_tests/row_format/data/step4.sql diff --git a/tests/row_format/run.sh b/tests/integration_tests/row_format/run.sh similarity index 100% rename from tests/row_format/run.sh rename to tests/integration_tests/row_format/run.sh diff --git a/tests/run.sh b/tests/integration_tests/run.sh similarity index 86% rename from tests/run.sh rename to tests/integration_tests/run.sh index b42b8ea9a36..9be18d27f5e 100755 --- a/tests/run.sh +++ b/tests/integration_tests/run.sh @@ -4,7 +4,7 @@ set -eu OUT_DIR=/tmp/tidb_cdc_test CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -export PATH=$PATH:$CUR/_utils:$CUR/../bin +export PATH=$PATH:$CUR/_utils:$CUR/../../bin mkdir -p $OUT_DIR || true @@ -14,8 +14,8 @@ if [ "${1-}" = '--debug' ]; then rm -rf $WORK_DIR && mkdir -p $WORK_DIR - PATH="$CUR/../bin:$CUR/_utils:$PATH" \ - LD_LIBRARY_PATH="$CUR/../bin:$CUR/_utils:$PATH" \ + PATH="$CUR/../../bin:$CUR/_utils:$PATH" \ + LD_LIBRARY_PATH="$CUR/../../bin:$CUR/_utils:$PATH" \ OUT_DIR=$OUT_DIR \ TEST_NAME="debug" \ start_tidb_cluster --workdir $WORK_DIR @@ -37,8 +37,8 @@ run_case() { local script=$2 local sink_type=$3 echo "=================>> Running test $script using Sink-Type: $sink_type... <<=================" - PATH="$CUR/../bin:$CUR/_utils:$PATH" \ - LD_LIBRARY_PATH="$CUR/../bin:$CUR/_utils:$PATH" \ + PATH="$CUR/../../bin:$CUR/_utils:$PATH" \ + LD_LIBRARY_PATH="$CUR/../../bin:$CUR/_utils:$PATH" \ OUT_DIR=$OUT_DIR \ TEST_NAME=$case \ bash "$script" "$sink_type" diff --git a/tests/simple/run.sh b/tests/integration_tests/simple/run.sh similarity index 100% rename from tests/simple/run.sh rename to tests/integration_tests/simple/run.sh diff --git a/tests/sink_hang/conf/diff_config.toml b/tests/integration_tests/sink_hang/conf/diff_config.toml similarity index 86% rename from tests/sink_hang/conf/diff_config.toml rename to tests/integration_tests/sink_hang/conf/diff_config.toml index 894edbf2bcc..0d1605c60ae 100644 --- a/tests/sink_hang/conf/diff_config.toml +++ b/tests/integration_tests/sink_hang/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/sink_hang/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/sink_hang/run.sh b/tests/integration_tests/sink_hang/run.sh similarity index 100% rename from tests/sink_hang/run.sh rename to tests/integration_tests/sink_hang/run.sh diff --git a/tests/sink_retry/conf/diff_config.toml b/tests/integration_tests/sink_retry/conf/diff_config.toml similarity index 86% rename from tests/sink_retry/conf/diff_config.toml rename to tests/integration_tests/sink_retry/conf/diff_config.toml index 9d4155398ce..2d31480fbe4 100644 --- a/tests/sink_retry/conf/diff_config.toml +++ b/tests/integration_tests/sink_retry/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/sink_retry/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/sink_retry/conf/workload b/tests/integration_tests/sink_retry/conf/workload similarity index 100% rename from tests/sink_retry/conf/workload rename to tests/integration_tests/sink_retry/conf/workload diff --git a/tests/sink_retry/run.sh b/tests/integration_tests/sink_retry/run.sh similarity index 100% rename from tests/sink_retry/run.sh rename to tests/integration_tests/sink_retry/run.sh diff --git a/tests/split_region/conf/diff_config.toml b/tests/integration_tests/split_region/conf/diff_config.toml similarity index 86% rename from tests/split_region/conf/diff_config.toml rename to tests/integration_tests/split_region/conf/diff_config.toml index 8433fad74e2..8e624a5c525 100644 --- a/tests/split_region/conf/diff_config.toml +++ b/tests/integration_tests/split_region/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/split_region/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/split_region/data/increment.sql b/tests/integration_tests/split_region/data/increment.sql similarity index 100% rename from tests/split_region/data/increment.sql rename to tests/integration_tests/split_region/data/increment.sql diff --git a/tests/split_region/data/prepare.sql b/tests/integration_tests/split_region/data/prepare.sql similarity index 100% rename from tests/split_region/data/prepare.sql rename to tests/integration_tests/split_region/data/prepare.sql diff --git a/tests/split_region/run.sh b/tests/integration_tests/split_region/run.sh similarity index 100% rename from tests/split_region/run.sh rename to tests/integration_tests/split_region/run.sh diff --git a/tests/syncpoint/conf/diff_config_final.toml b/tests/integration_tests/syncpoint/conf/diff_config_final.toml similarity index 84% rename from tests/syncpoint/conf/diff_config_final.toml rename to tests/integration_tests/syncpoint/conf/diff_config_final.toml index 635eced0c7c..5e960cbac03 100644 --- a/tests/syncpoint/conf/diff_config_final.toml +++ b/tests/integration_tests/syncpoint/conf/diff_config_final.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/syncpoint/sync_diff/output" source-instances = ["mysql1"] @@ -26,4 +26,4 @@ check-struct-only = false host = "127.0.0.1" port = 3306 user = "root" - password = "" \ No newline at end of file + password = "" diff --git a/tests/syncpoint/conf/diff_config_part1.toml b/tests/integration_tests/syncpoint/conf/diff_config_part1.toml similarity index 86% rename from tests/syncpoint/conf/diff_config_part1.toml rename to tests/integration_tests/syncpoint/conf/diff_config_part1.toml index 578f58ccfdc..40c3af5f1ec 100644 --- a/tests/syncpoint/conf/diff_config_part1.toml +++ b/tests/integration_tests/syncpoint/conf/diff_config_part1.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/syncpoint/sync_diff/output" source-instances = ["mysql1"] @@ -23,4 +23,3 @@ check-struct-only = false port = 4000 user = "root" password = "" - \ No newline at end of file diff --git a/tests/syncpoint/conf/diff_config_part2.toml b/tests/integration_tests/syncpoint/conf/diff_config_part2.toml similarity index 100% rename from tests/syncpoint/conf/diff_config_part2.toml rename to tests/integration_tests/syncpoint/conf/diff_config_part2.toml diff --git a/tests/syncpoint/conf/workload b/tests/integration_tests/syncpoint/conf/workload similarity index 100% rename from tests/syncpoint/conf/workload rename to tests/integration_tests/syncpoint/conf/workload diff --git a/tests/syncpoint/run.sh b/tests/integration_tests/syncpoint/run.sh similarity index 100% rename from tests/syncpoint/run.sh rename to tests/integration_tests/syncpoint/run.sh diff --git a/tests/tiflash/conf/diff_config.toml b/tests/integration_tests/tiflash/conf/diff_config.toml similarity index 87% rename from tests/tiflash/conf/diff_config.toml rename to tests/integration_tests/tiflash/conf/diff_config.toml index 9794baad8ea..4740f1d014d 100644 --- a/tests/tiflash/conf/diff_config.toml +++ b/tests/integration_tests/tiflash/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/tiflash/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/tiflash/data/prepare.sql b/tests/integration_tests/tiflash/data/prepare.sql similarity index 100% rename from tests/tiflash/data/prepare.sql rename to tests/integration_tests/tiflash/data/prepare.sql diff --git a/tests/tiflash/run.sh b/tests/integration_tests/tiflash/run.sh similarity index 100% rename from tests/tiflash/run.sh rename to tests/integration_tests/tiflash/run.sh diff --git a/tests/unified_sorter/conf/diff_config.toml b/tests/integration_tests/unified_sorter/conf/diff_config.toml similarity index 86% rename from tests/unified_sorter/conf/diff_config.toml rename to tests/integration_tests/unified_sorter/conf/diff_config.toml index 99a31dc902b..54b2eb79fdf 100644 --- a/tests/unified_sorter/conf/diff_config.toml +++ b/tests/integration_tests/unified_sorter/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/unified_sort/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/unified_sorter/conf/workload b/tests/integration_tests/unified_sorter/conf/workload similarity index 100% rename from tests/unified_sorter/conf/workload rename to tests/integration_tests/unified_sorter/conf/workload diff --git a/tests/unified_sorter/run.sh b/tests/integration_tests/unified_sorter/run.sh similarity index 100% rename from tests/unified_sorter/run.sh rename to tests/integration_tests/unified_sorter/run.sh diff --git a/tests/util/config.go b/tests/integration_tests/util/config.go similarity index 100% rename from tests/util/config.go rename to tests/integration_tests/util/config.go diff --git a/tests/util/db.go b/tests/integration_tests/util/db.go similarity index 98% rename from tests/util/db.go rename to tests/integration_tests/util/db.go index cce902f3220..1dbfdeb7e0c 100644 --- a/tests/util/db.go +++ b/tests/integration_tests/util/db.go @@ -158,7 +158,7 @@ func MustExecWithConn(ctx context.Context, conn *sql.Conn, sql string, args ...i } // CreateSourceDBs return source sql.DB for test -// we create two TiDB instance now in tests/run.sh, change it if needed +// we create two TiDB instance now in tests/integration_tests/run.sh, change it if needed func CreateSourceDBs() (dbs []*sql.DB, err error) { cfg := DBConfig{ Host: "127.0.0.1", diff --git a/tests/resolve_lock/diff_config.toml b/tests/resolve_lock/diff_config.toml deleted file mode 100644 index 26c78959203..00000000000 --- a/tests/resolve_lock/diff_config.toml +++ /dev/null @@ -1,29 +0,0 @@ -# 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"] - - target-instance = "tidb0" - - target-check-tables = ["test.?*"] - -[data-sources] -[data-sources.mysql1] - host = "127.0.0.1" - port = 4000 - user = "root" - password = "" - -[data-sources.tidb0] - host = "127.0.0.1" - port = 3306 - user = "root" - password = ""