From ce420139388c7094f0049d2192f5d51b1d91b6df Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 12 Nov 2021 18:07:06 +0800 Subject: [PATCH 01/16] schema_storage: fix schema GC threshold & improve memory management (#3172) (#3273) --- cdc/entry/schema_storage.go | 24 ++++-- cdc/entry/schema_storage_test.go | 8 +- cdc/processor/manager.go | 12 --- cdc/processor/manager_test.go | 14 +++- cdc/processor/metrics.go | 8 ++ cdc/processor/processor.go | 74 ++++++++++--------- cdc/processor/processor_test.go | 81 ++++++++++++++++++++- metrics/grafana/ticdc.json | 121 +++++++++++++++++++++++++++++++ 8 files changed, 285 insertions(+), 57 deletions(-) diff --git a/cdc/entry/schema_storage.go b/cdc/entry/schema_storage.go index 28a49d60a03..6031a1a859b 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -652,7 +652,7 @@ func (s *schemaSnapshot) Tables() map[model.TableID]*model.TableInfo { // SchemaStorage stores the schema information with multi-version type SchemaStorage interface { // GetSnapshot returns the snapshot which of ts is specified - GetSnapshot(ctx context.Context, ts uint64) (*schemaSnapshot, error) + GetSnapshot(ctx context.Context, ts uint64) (*SingleSchemaSnapshot, error) // GetLastSnapshot returns the last snapshot GetLastSnapshot() *schemaSnapshot // HandleDDLJob creates a new snapshot in storage and handles the ddl job @@ -661,8 +661,9 @@ type SchemaStorage interface { AdvanceResolvedTs(ts uint64) // ResolvedTs returns the resolved ts of the schema storage ResolvedTs() uint64 - // DoGC removes snaps which of ts less than this specified ts - DoGC(ts uint64) + // DoGC removes snaps that are no longer needed at the specified TS. + // It returns the TS from which the oldest maintained snapshot is valid. + DoGC(ts uint64) (lastSchemaTs uint64) } type schemaStorageImpl struct { @@ -796,7 +797,7 @@ func (s *schemaStorageImpl) ResolvedTs() uint64 { } // DoGC removes snaps which of ts less than this specified ts -func (s *schemaStorageImpl) DoGC(ts uint64) { +func (s *schemaStorageImpl) DoGC(ts uint64) (lastSchemaTs uint64) { s.snapsMu.Lock() defer s.snapsMu.Unlock() var startIdx int @@ -807,7 +808,7 @@ func (s *schemaStorageImpl) DoGC(ts uint64) { startIdx = i } if startIdx == 0 { - return + return s.snaps[0].currentTs } if log.GetLevel() == zapcore.DebugLevel { log.Debug("Do GC in schema storage") @@ -815,9 +816,16 @@ func (s *schemaStorageImpl) DoGC(ts uint64) { s.snaps[i].PrintStatus(log.Debug) } } - s.snaps = s.snaps[startIdx:] - atomic.StoreUint64(&s.gcTs, s.snaps[0].currentTs) - log.Info("finished gc in schema storage", zap.Uint64("gcTs", s.snaps[0].currentTs)) + + // copy the part of the slice that is needed instead of re-slicing it + // to maximize efficiency of Go runtime GC. + newSnaps := make([]*schemaSnapshot, len(s.snaps)-startIdx) + copy(newSnaps, s.snaps[startIdx:]) + s.snaps = newSnaps + + lastSchemaTs = s.snaps[0].currentTs + atomic.StoreUint64(&s.gcTs, lastSchemaTs) + return } // SkipJob skip the job should not be executed diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index 7f9f13b44be..15db66ebd82 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -623,7 +623,9 @@ func (t *schemaSuite) TestMultiVersionStorage(c *check.C) { _, exist = snap.TableByID(3) c.Assert(exist, check.IsFalse) - storage.DoGC(0) + lastSchemaTs := storage.DoGC(0) + c.Check(lastSchemaTs, check.Equals, uint64(0)) + snap, err = storage.GetSnapshot(ctx, 100) c.Assert(err, check.IsNil) _, exist = snap.SchemaByID(1) @@ -644,7 +646,9 @@ func (t *schemaSuite) TestMultiVersionStorage(c *check.C) { _, exist = snap.TableByID(3) c.Assert(exist, check.IsFalse) - storage.DoGC(155) + lastSchemaTs = storage.DoGC(155) + c.Check(lastSchemaTs, check.Equals, uint64(140)) + storage.AdvanceResolvedTs(185) snap, err = storage.GetSnapshot(ctx, 180) diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 3d14e4c19c4..d1bf39e7854 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" - tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" cdcContext "github.com/pingcap/ticdc/pkg/context" cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/orchestrator" @@ -62,17 +61,6 @@ func NewManager() *Manager { } } -// NewManager4Test creates a new processor manager for test -func NewManager4Test( - createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), -) *Manager { - m := NewManager() - m.newProcessor = func(ctx cdcContext.Context) *processor { - return newProcessor4Test(ctx, createTablePipeline) - } - return m -} - // Tick implements the `orchestrator.State` interface // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // the Tick function of Manager create or remove processor instances according to the specified `state`, or pass the `state` to processor instances diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index 78bbc7df086..33ae08d3f24 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -38,8 +38,20 @@ type managerSuite struct { var _ = check.Suite(&managerSuite{}) +// NewManager4Test creates a new processor manager for test +func NewManager4Test( + c *check.C, + createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), +) *Manager { + m := NewManager() + m.newProcessor = func(ctx cdcContext.Context) *processor { + return newProcessor4Test(ctx, c, createTablePipeline) + } + return m +} + func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { - s.manager = NewManager4Test(func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { + s.manager = NewManager4Test(c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), diff --git a/cdc/processor/metrics.go b/cdc/processor/metrics.go index 651f0afa94b..b63d60fef05 100644 --- a/cdc/processor/metrics.go +++ b/cdc/processor/metrics.go @@ -60,6 +60,13 @@ var ( Name: "exit_with_error_count", Help: "counter for processor exits with error", }, []string{"changefeed", "capture"}) + processorSchemaStorageGcTsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "processor", + Name: "schema_storage_gc_ts", + Help: "the TS of the currently maintained oldest snapshot in SchemaStorage", + }, []string{"changefeed", "capture"}) ) // InitMetrics registers all metrics used in processor @@ -70,4 +77,5 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(checkpointTsLagGauge) registry.MustRegister(syncTableNumGauge) registry.MustRegister(processorErrorCounter) + registry.MustRegister(processorSchemaStorageGcTsGauge) } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index b3344b51bce..949d03e193c 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -45,8 +45,6 @@ import ( ) const ( - schemaStorageGCLag = time.Minute * 20 - backoffBaseDelayInMs = 5 maxTries = 3 ) @@ -59,9 +57,11 @@ type processor struct { tables map[model.TableID]tablepipeline.TablePipeline schemaStorage entry.SchemaStorage - filter *filter.Filter - mounter entry.Mounter - sinkManager *sink.Manager + lastSchemaTs model.Ts + + filter *filter.Filter + mounter entry.Mounter + sinkManager *sink.Manager initialized bool errCh chan error @@ -71,12 +71,13 @@ type processor struct { lazyInit func(ctx cdcContext.Context) error createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) - metricResolvedTsGauge prometheus.Gauge - metricResolvedTsLagGauge prometheus.Gauge - metricCheckpointTsGauge prometheus.Gauge - metricCheckpointTsLagGauge prometheus.Gauge - metricSyncTableNumGauge prometheus.Gauge - metricProcessorErrorCounter prometheus.Counter + metricResolvedTsGauge prometheus.Gauge + metricResolvedTsLagGauge prometheus.Gauge + metricCheckpointTsGauge prometheus.Gauge + metricCheckpointTsLagGauge prometheus.Gauge + metricSyncTableNumGauge prometheus.Gauge + metricSchemaStorageGcTsGauge prometheus.Gauge + metricProcessorErrorCounter prometheus.Counter } // newProcessor creates a new processor @@ -90,27 +91,19 @@ func newProcessor(ctx cdcContext.Context) *processor { captureInfo: ctx.GlobalVars().CaptureInfo, cancel: func() {}, - metricResolvedTsGauge: resolvedTsGauge.WithLabelValues(changefeedID, advertiseAddr), - metricResolvedTsLagGauge: resolvedTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), - metricCheckpointTsGauge: checkpointTsGauge.WithLabelValues(changefeedID, advertiseAddr), - metricCheckpointTsLagGauge: checkpointTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), - metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, advertiseAddr), - metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, advertiseAddr), + metricResolvedTsGauge: resolvedTsGauge.WithLabelValues(changefeedID, advertiseAddr), + metricResolvedTsLagGauge: resolvedTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), + metricCheckpointTsGauge: checkpointTsGauge.WithLabelValues(changefeedID, advertiseAddr), + metricCheckpointTsLagGauge: checkpointTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), + metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, advertiseAddr), + metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, advertiseAddr), + metricSchemaStorageGcTsGauge: processorSchemaStorageGcTsGauge.WithLabelValues(changefeedID, advertiseAddr), } p.createTablePipeline = p.createTablePipelineImpl p.lazyInit = p.lazyInitImpl return p } -func newProcessor4Test(ctx cdcContext.Context, - createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), -) *processor { - p := newProcessor(ctx) - p.lazyInit = func(ctx cdcContext.Context) error { return nil } - p.createTablePipeline = createTablePipeline - return p -} - // Tick implements the `orchestrator.State` interface // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // The main logic of processor is in this function, including the calculation of many kinds of ts, maintain table pipeline, error handling, etc. @@ -179,7 +172,7 @@ func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorS p.handlePosition() p.pushResolvedTs2Table() p.handleWorkload() - p.doGCSchemaStorage() + p.doGCSchemaStorage(ctx) return p.changefeed, nil } @@ -750,16 +743,30 @@ func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID mode } // doGCSchemaStorage trigger the schema storage GC -func (p *processor) doGCSchemaStorage() { +func (p *processor) doGCSchemaStorage(ctx cdcContext.Context) { if p.schemaStorage == nil { // schemaStorage is nil only in test return } - // Delay GC to accommodate pullers starting from a startTs that's too small - // TODO fix startTs problem and remove GC delay, or use other mechanism that prevents the problem deterministically - gcTime := oracle.GetTimeFromTS(p.changefeed.Status.CheckpointTs).Add(-schemaStorageGCLag) - gcTs := oracle.ComposeTS(gcTime.Unix(), 0) - p.schemaStorage.DoGC(gcTs) + + if p.changefeed.Status == nil { + // This could happen if Etcd data is not complete. + return + } + + // Please refer to `unmarshalAndMountRowChanged` in cdc/entry/mounter.go + // for why we need -1. + lastSchemaTs := p.schemaStorage.DoGC(p.changefeed.Status.CheckpointTs - 1) + if p.lastSchemaTs == lastSchemaTs { + return + } + p.lastSchemaTs = lastSchemaTs + + log.Debug("finished gc in schema storage", + zap.Uint64("gcTs", lastSchemaTs), + cdcContext.ZapFieldChangefeed(ctx)) + lastSchemaPhysicalTs := oracle.ExtractPhysical(lastSchemaTs) + p.metricSchemaStorageGcTsGauge.Set(float64(lastSchemaPhysicalTs)) } func (p *processor) Close() error { @@ -779,6 +786,7 @@ func (p *processor) Close() error { checkpointTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) + processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) if p.sinkManager != nil { // pass a canceled context is ok here, since we don't need to wait Close ctx, cancel := context.WithCancel(context.Background()) diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index d1fa9558835..1786d99c1a8 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -15,16 +15,21 @@ package processor import ( "context" + "encoding/json" "fmt" + "math" + "sync/atomic" "testing" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/entry" "github.com/pingcap/ticdc/cdc/model" tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/orchestrator" "github.com/pingcap/ticdc/pkg/util/testleak" ) @@ -35,8 +40,20 @@ type processorSuite struct{} var _ = check.Suite(&processorSuite{}) +func newProcessor4Test( + ctx cdcContext.Context, + c *check.C, + createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), +) *processor { + p := newProcessor(ctx) + p.lazyInit = func(ctx cdcContext.Context) error { return nil } + p.createTablePipeline = createTablePipeline + p.schemaStorage = &mockSchemaStorage{c: c} + return p +} + func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orchestrator.ReactorStateTester) { - p := newProcessor4Test(ctx, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { + p := newProcessor4Test(ctx, c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), @@ -109,6 +126,25 @@ func (m *mockTablePipeline) Wait() { // do nothing } +type mockSchemaStorage struct { + // dummy to provide default versions of unimplemented interface methods, + // as we only need ResolvedTs() and DoGC() in unit tests. + entry.SchemaStorage + + c *check.C + lastGcTs uint64 +} + +func (s *mockSchemaStorage) ResolvedTs() uint64 { + return math.MaxUint64 +} + +func (s *mockSchemaStorage) DoGC(ts uint64) uint64 { + s.c.Assert(s.lastGcTs, check.LessEqual, ts) + atomic.StoreUint64(&s.lastGcTs, ts) + return ts +} + func (s *processorSuite) TestCheckTablesNum(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(true) @@ -664,6 +700,32 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { }) } +func (s *processorSuite) TestSchemaGC(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(true) + p, tester := initProcessor4Test(ctx, c) + p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} + status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} + return status, true, nil + }) + + var err error + // init tick + _, err = p.Tick(ctx, p.changefeed) + c.Assert(err, check.IsNil) + tester.MustApplyPatches() + + updateChangeFeedPosition(c, tester, "changefeed-id-test", 50, 50) + _, err = p.Tick(ctx, p.changefeed) + c.Assert(err, check.IsNil) + tester.MustApplyPatches() + + // GC Ts should be (checkpoint - 1). + c.Assert(p.schemaStorage.(*mockSchemaStorage).lastGcTs, check.Equals, uint64(49)) + c.Assert(p.lastSchemaTs, check.Equals, uint64(49)) +} + func cleanUpFinishedOpOperation(state *model.ChangefeedReactorState, captureID model.CaptureID, tester *orchestrator.ReactorStateTester) { state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { if status == nil || status.Operation == nil { @@ -678,3 +740,20 @@ func cleanUpFinishedOpOperation(state *model.ChangefeedReactorState, captureID m }) tester.MustApplyPatches() } + +func updateChangeFeedPosition(c *check.C, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, resolvedTs, checkpointTs model.Ts) { + key := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangeFeedStatus, + ChangefeedID: cfID, + } + keyStr := key.String() + + cfStatus := &model.ChangeFeedStatus{ + ResolvedTs: resolvedTs, + CheckpointTs: checkpointTs, + } + valueBytes, err := json.Marshal(cfStatus) + c.Assert(err, check.IsNil) + + tester.MustUpdate(keyStr, valueBytes) +} diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 9bc72e7c33d..33cbc23f028 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -3089,6 +3089,127 @@ "align": false, "alignLevel": null } + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "left", + "filterable": true + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "dateTimeAsSystem" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "capture" + }, + "properties": [ + { + "id": "custom.width", + "value": 191 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "changefeed" + }, + "properties": [ + { + "id": "custom.width", + "value": 209 + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 69 + }, + "id": 200, + "options": { + "showHeader": true, + "sortBy": [ + { + "desc": true, + "displayName": "last-snap-ts" + } + ] + }, + "pluginVersion": "6.1.6", + "targets": [ + { + "exemplar": true, + "expr": "ticdc_processor_schema_storage_gc_ts{changefeed=~\"$changefeed\", capture=~\"$capture\"}", + "format": "table", + "instant": true, + "interval": "", + "legendFormat": "", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "Schema Storage GC progress", + "transformations": [ + { + "id": "filterFieldsByName", + "options": { + "include": { + "names": [ + "capture", + "changefeed", + "Value" + ] + } + } + }, + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "Value #A": "last-snap-ts" + } + } + }, + { + "id": "sortBy", + "options": { + "fields": {}, + "sort": [ + { + "field": "last-snap-ts" + } + ] + } + } + ], + "type": "table" } ], "title": "Changefeed", From c8a6a87f937be171f12bb27790ec34143c21ebea Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 12 Nov 2021 19:13:06 +0800 Subject: [PATCH 02/16] ticdc/processor: don't close error channel in receiver (#3414) (#3427) --- cdc/processor/processor.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 949d03e193c..a69bb16502f 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -217,6 +217,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() { @@ -228,7 +231,6 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { for { select { case <-ctx.Done(): - close(errCh) return case err := <-errCh: if err == nil { From 6c75da7dc2985d994c182fd1a638ea7fe77bf6a0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 12 Nov 2021 19:41:06 +0800 Subject: [PATCH 03/16] kv/client: fix region loss in single region handler (#3281) (#3293) --- cdc/kv/region_worker.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index c498c726cb0..b92d06dff07 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -235,7 +235,7 @@ func (w *regionWorker) checkShouldExit() error { return nil } -func (w *regionWorker) handleSingleRegionError(ctx context.Context, err error, state *regionFeedState) error { +func (w *regionWorker) handleSingleRegionError(err error, state *regionFeedState) error { if state.lastResolvedTs > state.sri.ts { state.sri.ts = state.lastResolvedTs } @@ -273,7 +273,9 @@ func (w *regionWorker) handleSingleRegionError(ctx context.Context, err error, s } revokeToken := !state.initialized - err2 := w.session.onRegionFail(ctx, regionErrorInfo{ + // since the context used in region worker will be cancelled after region + // worker exits, we must use the parent context to prevent regionErrorInfo loss. + err2 := w.session.onRegionFail(w.parentCtx, regionErrorInfo{ singleRegionInfo: state.sri, err: err, }, revokeToken) @@ -388,26 +390,25 @@ func (w *regionWorker) processEvent(ctx context.Context, event *regionStatefulEv case *cdcpb.Event_Entries_: err = w.handleEventEntry(ctx, x, event.state) if err != nil { - err = w.handleSingleRegionError(ctx, err, event.state) + err = w.handleSingleRegionError(err, event.state) } case *cdcpb.Event_Admin_: log.Info("receive admin event", zap.Stringer("event", event.changeEvent)) case *cdcpb.Event_Error: err = w.handleSingleRegionError( - ctx, cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}), event.state, ) case *cdcpb.Event_ResolvedTs: if err = w.handleResolvedTs(ctx, x.ResolvedTs, event.state); err != nil { - err = w.handleSingleRegionError(ctx, err, event.state) + err = w.handleSingleRegionError(err, event.state) } } } if event.resolvedTs != nil { if err = w.handleResolvedTs(ctx, event.resolvedTs.Ts, event.state); err != nil { - err = w.handleSingleRegionError(ctx, err, event.state) + err = w.handleSingleRegionError(err, event.state) } } event.state.lock.Unlock() From 445bfe79816ffe4470a13a008855cd0304a43d03 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 12 Nov 2021 19:59:05 +0800 Subject: [PATCH 04/16] tests: Enhanced old value test for more strict checking of output values (#3127) (#3374) --- tests/multi_changefeed/run.sh | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/tests/multi_changefeed/run.sh b/tests/multi_changefeed/run.sh index 62dadc5aed7..8027cbb5bd7 100755 --- a/tests/multi_changefeed/run.sh +++ b/tests/multi_changefeed/run.sh @@ -11,35 +11,43 @@ SINK_TYPE=$1 MAX_RETRIES=10 function check_old_value_enabled() { - row_logs=$(grep "EmitRowChangedEvents" "$1/cdc.log" || true) + row_logs=$(grep "EmitRowChangedEvents" "$1/cdc.log") echo $row_logs # check update rows # check if exist a row include `column` and `pre-column` - update_with_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep '\"columns\\\":\[' | wc -l || true) - if [[ "$update_with_old_value_count" -eq 0 ]]; then + # When old value is turned on, we will have both column and pre-column in the update. + # So here we have 2 (pre val) and 3 (new val). + update_with_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep '\"columns\\\":\[' | grep 'value\\\":2' | grep -c 'value\\\":3') + if [[ "$update_with_old_value_count" -ne 1 ]]; then echo "can't found update row with old value" exit 1 fi # check if exist a update row without `pre-column` - update_without_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep -v 'pre\-columns\\\":\[' | grep 'value\\\":2' | wc -l || true) - if [[ "$update_without_old_value_count" -eq 0 ]]; then + # When old value is turned off, we only have the column in the update. + # So here we only have 3 (new val). + update_without_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":null' | grep -c 'value\\\":3') + if [[ "$update_without_old_value_count" -ne 1 ]]; then echo "can't found update row without old value" exit 1 fi # check delete rows # check if exist a delete row with a complete `pre-column` - delete_with_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep 'columns\\\":null' | grep 'value\\\":2' | wc -l || true) - if [[ "$delete_with_old_value_count" -eq 0 ]]; then + # When old value is turned on, the pre-column in our delete will include all the columns. + # So here we have 1 (id) and 3 (val). + delete_with_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep 'columns\\\":null' | grep 'value\\\":1' | grep -c 'value\\\":3') + if [[ "$delete_with_old_value_count" -ne 1 ]]; then echo "can't found delete row with old value" exit 1 fi # check if exist a delete row without a complete `pre-column` - delete_without_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep 'columns\\\":null' | grep 'value\\\":1},null' | wc -l || true) - if [[ "$delete_without_old_value_count" -eq 0 ]]; then + # When old value is turned off, the pre-column in our delete will only include the handle columns. + # So here we only have 1 (id). + delete_without_old_value_count=$(grep "EmitRowChangedEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep 'columns\\\":null' | grep -c 'value\\\":1},null') + if [[ "$delete_without_old_value_count" -ne 1 ]]; then echo "can't found delete row without old value" exit 1 fi @@ -65,8 +73,8 @@ function run() { cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" -c "no-old-value-cf" --config="$CUR/conf/changefeed2.toml" run_sql "CREATE DATABASE multi_changefeed;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table multi_changefeed.t1 (id int primary key, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_changefeed.t1 VALUES (1,1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "UPDATE multi_changefeed.t1 SET val = 2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_changefeed.t1 VALUES (1,2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_changefeed.t1 SET val = 3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "DELETE FROM multi_changefeed.t1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} ensure $MAX_RETRIES check_old_value_enabled $WORK_DIR From 206c20ca5aeee2614f639936d48575ca0008a356 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Sat, 13 Nov 2021 15:41:05 +0800 Subject: [PATCH 05/16] sink: add log when skip the empty row event (#3357) (#3385) --- cdc/processor/pipeline/sink.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 } From 9afef7b1f95a6e830141c74a0d8337cf998ed408 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 16 Nov 2021 13:45:07 +0800 Subject: [PATCH 06/16] cdc/sorter: make unified sorter cgroup aware (#3436) (#3438) --- cdc/puller/sorter/backend_pool.go | 21 +++++++++++---------- cdc/puller/sorter/backend_pool_test.go | 11 +++++++---- go.mod | 1 - go.sum | 3 --- 4 files changed, 18 insertions(+), 18 deletions(-) diff --git a/cdc/puller/sorter/backend_pool.go b/cdc/puller/sorter/backend_pool.go index a6aeed51173..dcdfd522d93 100644 --- a/cdc/puller/sorter/backend_pool.go +++ b/cdc/puller/sorter/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" @@ -32,6 +31,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" ) @@ -96,6 +96,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: @@ -109,14 +116,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)) }) @@ -127,7 +128,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/puller/sorter/backend_pool_test.go b/cdc/puller/sorter/backend_pool_test.go index b7c94b4378d..81bef2a4c90 100644 --- a/cdc/puller/sorter/backend_pool_test.go +++ b/cdc/puller/sorter/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/puller/sorter/getMemoryPressureFails", "return(true)") - c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/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/go.mod b/go.mod index 874dd48cc53..09f1a414b56 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,6 @@ require ( github.com/jmoiron/sqlx v1.2.0 github.com/lib/pq v1.3.0 // indirect github.com/linkedin/goavro/v2 v2.9.7 - github.com/mackerelio/go-osstat v0.1.0 github.com/mattn/go-shellwords v1.0.3 github.com/mattn/go-sqlite3 v2.0.2+incompatible // indirect github.com/philhofer/fwd v1.0.0 // indirect diff --git a/go.sum b/go.sum index 8f9b0f995ef..8aa2ba09cb5 100644 --- a/go.sum +++ b/go.sum @@ -526,8 +526,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.7 h1:Vd++Rb/RKcmNJjM0HP/JJFMEWa21eUBVKPYlKehOGrM= github.com/linkedin/goavro/v2 v2.9.7/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= @@ -1044,7 +1042,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= From ca987bc329710892fbf53b956506dbf54bed8766 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 23 Nov 2021 14:27:50 +0800 Subject: [PATCH 07/16] test(ticdc): separate statistics for unit tests and integration tests (#3570) (#3580) --- Makefile | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 91a98a33382..344872f8e9a 100644 --- a/Makefile +++ b/Makefile @@ -186,16 +186,21 @@ tidy: check: check-copyright fmt lint check-static tidy errdoc check-leaktest-added check-merge-conflicts -coverage: +integration_test_coverage: GO111MODULE=off go get github.com/wadey/gocovmerge 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 @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 From 96a119c3bf10d62b83e4bf32a9ab253a349fea67 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 23 Nov 2021 18:05:52 +0800 Subject: [PATCH 08/16] ticdc/metrics: fix processor checkpoint-ts lag expr (#3543) (#3555) --- metrics/grafana/ticdc.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 33cbc23f028..9a039e8537d 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -1585,7 +1585,7 @@ "refId": "A" }, { - "expr": "sum(ticdc_processor_resolved_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "expr": "sum(ticdc_processor_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", "format": "time_series", "interval": "", "intervalFactor": 1, From 46d2487697653a3aa5ecf18aa8d5b061308d9ac1 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 30 Nov 2021 18:15:53 +0800 Subject: [PATCH 09/16] test: add new ci collation without old value test (#3114) (#3370) --- tests/_utils/start_tidb_cluster_impl | 21 ++++++-- tests/_utils/start_tls_tidb_cluster_impl | 1 + .../conf/changefeed.toml | 0 .../conf/diff_config.toml | 2 +- .../conf/tidb_config.toml | 0 .../data/test1.sql | 6 +-- .../data/test2.sql | 2 +- .../run.sh | 4 +- .../conf/changefeed.toml | 1 + .../conf/diff_config.toml | 29 +++++++++++ .../conf/tidb_config.toml | 2 + .../data/test1.sql | 49 +++++++++++++++++++ .../data/test2.sql | 6 +++ .../new_ci_collation_without_old_value/run.sh | 48 ++++++++++++++++++ 14 files changed, 161 insertions(+), 10 deletions(-) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/conf/changefeed.toml (100%) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/conf/diff_config.toml (85%) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/conf/tidb_config.toml (100%) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/data/test1.sql (90%) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/data/test2.sql (64%) rename tests/{new_ci_collation => new_ci_collation_with_old_value}/run.sh (92%) create mode 100644 tests/new_ci_collation_without_old_value/conf/changefeed.toml create mode 100644 tests/new_ci_collation_without_old_value/conf/diff_config.toml create mode 100644 tests/new_ci_collation_without_old_value/conf/tidb_config.toml create mode 100644 tests/new_ci_collation_without_old_value/data/test1.sql create mode 100644 tests/new_ci_collation_without_old_value/data/test2.sql create mode 100755 tests/new_ci_collation_without_old_value/run.sh diff --git a/tests/_utils/start_tidb_cluster_impl b/tests/_utils/start_tidb_cluster_impl index d259fc03e5f..0afb3952fef 100755 --- a/tests/_utils/start_tidb_cluster_impl +++ b/tests/_utils/start_tidb_cluster_impl @@ -10,6 +10,18 @@ OUT_DIR= tidb_config= pd_config= multiple_upstream_pd= +random_file_name= + +# Random generate the sockets config. +# Make sure we dont use the same sock. +randomGenSocketsConf() { + date=$(date '+%s%N') + + random_file_name="$OUT_DIR/tidb-config-$date.toml" + + cat "$OUT_DIR/tidb-config.toml" >"$random_file_name" + echo "socket = \"/tmp/tidb-$date.sock\"" >>"$random_file_name" +} while [[ ${1} ]]; do case "${1}" in @@ -196,26 +208,29 @@ tikv-server \ sleep 2 echo "Starting Upstream TiDB..." +randomGenSocketsConf tidb-server \ -P ${UP_TIDB_PORT} \ - -config "$OUT_DIR/tidb-config.toml" \ + -config "$random_file_name" \ --store tikv \ --path ${UP_PD_HOST_1}:${UP_PD_PORT_1} \ --status=${UP_TIDB_STATUS} \ --log-file "$OUT_DIR/tidb.log" & +randomGenSocketsConf tidb-server \ -P ${UP_TIDB_OTHER_PORT} \ - -config "$OUT_DIR/tidb-config.toml" \ + -config "$random_file_name" \ --store tikv \ --path ${UP_PD_HOST_1}:${UP_PD_PORT_1} \ --status=${UP_TIDB_OTHER_STATUS} \ --log-file "$OUT_DIR/tidb_other.log" & echo "Starting Downstream TiDB..." +randomGenSocketsConf tidb-server \ -P ${DOWN_TIDB_PORT} \ - -config "$OUT_DIR/tidb-config.toml" \ + -config "$random_file_name" \ --store tikv \ --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ --status=${DOWN_TIDB_STATUS} \ diff --git a/tests/_utils/start_tls_tidb_cluster_impl b/tests/_utils/start_tls_tidb_cluster_impl index d0c07b342b4..37dbf784829 100755 --- a/tests/_utils/start_tls_tidb_cluster_impl +++ b/tests/_utils/start_tls_tidb_cluster_impl @@ -88,6 +88,7 @@ EOF # tidb server config file cat - >"$OUT_DIR/tidb-config-tls.toml" < 'A'; +drop index `primary` on t4; +update t5 set b = 12; diff --git a/tests/new_ci_collation_without_old_value/data/test2.sql b/tests/new_ci_collation_without_old_value/data/test2.sql new file mode 100644 index 00000000000..dba8001ba91 --- /dev/null +++ b/tests/new_ci_collation_without_old_value/data/test2.sql @@ -0,0 +1,6 @@ +use `new_ci_collation_without_old_value_test`; +delete from t1; +delete from t2; +delete from t3; +delete from t4; +delete from t5; diff --git a/tests/new_ci_collation_without_old_value/run.sh b/tests/new_ci_collation_without_old_value/run.sh new file mode 100755 index 00000000000..9458d56a19a --- /dev/null +++ b/tests/new_ci_collation_without_old_value/run.sh @@ -0,0 +1,48 @@ +#!/bin/bash + +set -e + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function run() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR --tidb-config $CUR/conf/tidb_config.toml + + cd $WORK_DIR + + # record tso before we create tables to skip the system table DDLs + start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + TOPIC_NAME="ticdc-new_ci_collation_without_old_value-test-$RANDOM" + case $SINK_TYPE in + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}" ;; + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true" ;; + esac + cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml + if [ "$SINK_TYPE" == "kafka" ]; then + run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}" + fi + + run_sql_file $CUR/data/test1.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + for i in $(seq 1 5); do + table="new_ci_collation_without_old_value_test.t$i" + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + done + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + run_sql_file $CUR/data/test2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 3169b6e98fbd96b5d6ee0fb2aa1106b792844aa8 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 30 Nov 2021 18:41:53 +0800 Subject: [PATCH 10/16] processor,sink(cdc): let sink report resolved ts and do not skip buffer sink flush (#3540) (#3564) --- cdc/sink/manager.go | 33 ++++++++++++++++++++++++++++----- 1 file changed, 28 insertions(+), 5 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 0e8d5df9617..ae80bc7df9e 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() @@ -174,12 +177,27 @@ func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error } 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 }) if i == 0 { atomic.StoreUint64(&t.emittedTs, resolvedTs) - 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:]...) @@ -189,7 +207,12 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 return t.manager.getCheckpointTs(), errors.Trace(err) } atomic.StoreUint64(&t.emittedTs, resolvedTs) - 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) getEmittedTs() uint64 { From a4d46ee009ce6f01d8069e4fdb35f1f47af39203 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 2 Dec 2021 19:07:54 +0800 Subject: [PATCH 11/16] integration_tests(ticdc): move integration tests into integration_tests dir (#3646) (#3691) --- Makefile | 4 ++-- pkg/security/credential_test.go | 8 ++++---- tests/{ => integration_tests}/README.md | 7 ++++--- tests/{ => integration_tests}/_certificates/ca.pem | 0 .../_certificates/client-key.pem | 0 .../_certificates/client.pem | 0 .../_certificates/kafka.server.keystore.jks | Bin .../_certificates/kafka.server.truststore.jks | Bin .../_certificates/server-key.pem | 0 .../_certificates/server.pem | 0 tests/{ => integration_tests}/_utils/check_contains | 0 tests/{ => integration_tests}/_utils/check_logs | 0 .../_utils/check_logs_contains | 0 .../_utils/check_not_contains | 0 .../{ => integration_tests}/_utils/check_sync_diff | 0 .../_utils/check_table_exists | 0 .../{ => integration_tests}/_utils/cleanup_process | 0 tests/{ => integration_tests}/_utils/ensure | 0 .../_utils/random_kill_process | 0 tests/{ => integration_tests}/_utils/run_cdc_cli | 0 .../_utils/run_cdc_cli_tso_query | 0 tests/{ => integration_tests}/_utils/run_cdc_server | 0 .../_utils/run_kafka_consumer | 0 tests/{ => integration_tests}/_utils/run_sql | 0 tests/{ => integration_tests}/_utils/run_sql_file | 0 .../_utils/start_tidb_cluster | 0 .../_utils/start_tidb_cluster_impl | 2 +- .../_utils/start_tls_tidb_cluster | 0 .../_utils/start_tls_tidb_cluster_impl | 2 +- .../_utils/stop_tidb_cluster | 0 tests/{ => integration_tests}/_utils/test_prepare | 0 .../_utils/tiflash-users.toml | 0 .../autorandom/conf/diff_config.toml | 0 .../autorandom/conf/tidb_config.toml | 0 .../autorandom/data/test.sql | 0 tests/{ => integration_tests}/autorandom/run.sh | 0 .../{ => integration_tests}/availability/capture.sh | 0 tests/{ => integration_tests}/availability/owner.sh | 0 .../availability/processor.sh | 0 tests/{ => integration_tests}/availability/run.sh | 0 tests/{ => integration_tests}/bank/bank.go | 0 tests/{ => integration_tests}/bank/case.go | 0 tests/{ => integration_tests}/bank/run.sh | 0 .../batch_add_table/conf/diff_config.toml | 0 .../batch_add_table/data/prepare.sql | 0 .../batch_add_table/data/test.sql | 0 .../{ => integration_tests}/batch_add_table/run.sh | 0 .../conf/diff_config.toml | 0 .../capture_session_done_during_task/run.sh | 0 .../conf/diff_config.toml | 0 .../capture_suicide_while_balance_table/run.sh | 0 tests/{ => integration_tests}/cdc/cdc.go | 4 ++-- tests/{ => integration_tests}/cdc/config.toml | 0 tests/{ => integration_tests}/cdc/run.sh | 0 tests/{ => integration_tests}/cdclog_file/run.sh | 0 tests/{ => integration_tests}/cdclog_s3/run.sh | 0 .../changefeed_auto_stop/conf/diff_config.toml | 0 .../changefeed_auto_stop/conf/workload | 0 .../changefeed_auto_stop/run.sh | 0 .../changefeed_error/conf/diff_config.toml | 0 .../changefeed_error/conf/workload | 0 .../{ => integration_tests}/changefeed_error/run.sh | 0 .../changefeed_fast_fail/run.sh | 0 .../changefeed_finish/conf/diff_config.toml | 0 .../changefeed_finish/run.sh | 0 .../changefeed_pause_resume/conf/diff_config.toml | 0 .../changefeed_pause_resume/run.sh | 0 .../changefeed_reconstruct/conf/diff_config.toml | 0 .../changefeed_reconstruct/conf/workload | 0 .../changefeed_reconstruct/run.sh | 0 tests/{ => integration_tests}/cli/run.sh | 0 .../clustered_index/conf/diff_config.toml | 0 .../clustered_index/data/test.sql | 0 .../{ => integration_tests}/clustered_index/run.sh | 0 .../common_1/conf/diff_config.toml | 0 .../{ => integration_tests}/common_1/data/test.sql | 0 .../common_1/data/test_finish.sql | 0 .../common_1/data/test_v5.sql | 0 tests/{ => integration_tests}/common_1/run.sh | 0 .../cyclic_ab/conf/diff_config.toml | 0 .../cyclic_ab/conf/only_test_simple.toml | 0 tests/{ => integration_tests}/cyclic_ab/run.sh | 0 .../cyclic_abc/conf/changefeed.toml | 0 .../cyclic_abc/conf/diff_config_down_tls.toml | 0 .../cyclic_abc/conf/diff_config_up_down.toml | 0 tests/{ => integration_tests}/cyclic_abc/run.sh | 0 tests/{ => integration_tests}/dailytest/case.go | 0 .../{ => integration_tests}/dailytest/dailytest.go | 0 tests/{ => integration_tests}/dailytest/data.go | 0 tests/{ => integration_tests}/dailytest/db.go | 2 +- tests/{ => integration_tests}/dailytest/exector.go | 0 tests/{ => integration_tests}/dailytest/job.go | 0 tests/{ => integration_tests}/dailytest/parser.go | 0 tests/{ => integration_tests}/dailytest/rand.go | 0 tests/{ => integration_tests}/ddl_puller_lag/run.sh | 0 .../ddl_reentrant/conf/diff_config.toml | 0 .../ddl_reentrant/conf/tidb_config.toml | 0 tests/{ => integration_tests}/ddl_reentrant/run.sh | 0 .../ddl_sequence/conf/diff_config.toml | 0 .../ddl_sequence/data/prepare.sql | 0 tests/{ => integration_tests}/ddl_sequence/run.sh | 0 .../drop_many_tables/conf/diff_config.toml | 0 .../drop_many_tables/data/prepare.sql | 0 .../{ => integration_tests}/drop_many_tables/run.sh | 0 .../force_replicate_table/conf/changefeed.toml | 0 .../force_replicate_table/conf/tidb_config.toml | 0 .../force_replicate_table/data/test.sql | 0 .../force_replicate_table/run.sh | 0 .../gc_safepoint/conf/diff_config.toml | 0 tests/{ => integration_tests}/gc_safepoint/run.sh | 0 .../generate_column/conf/diff_config.toml | 0 .../generate_column/data/prepare.sql | 0 .../{ => integration_tests}/generate_column/run.sh | 0 tests/{ => integration_tests}/http_api/run.sh | 0 .../http_api/util/test_case.py | 0 .../http_proxies/run-proxy.go | 0 tests/{ => integration_tests}/http_proxies/run.sh | 0 .../kafka_messages/conf/diff_config.toml | 0 .../kafka_messages/conf/workload | 0 tests/{ => integration_tests}/kafka_messages/run.sh | 0 .../kafka_sink_error_resume/conf/diff_config.toml | 0 .../kafka_sink_error_resume/run.sh | 0 .../kill_owner_with_ddl/conf/diff_config.toml | 0 .../kill_owner_with_ddl/run.sh | 0 .../conf/diff_config.toml | 0 .../kv_client_stream_reconnect/run.sh | 0 .../many_pk_or_uk/config.toml | 0 .../many_pk_or_uk/diff_config.toml | 0 tests/{ => integration_tests}/many_pk_or_uk/main.go | 2 +- tests/{ => integration_tests}/many_pk_or_uk/run.sh | 0 .../move_table/conf/diff_config.toml | 0 .../move_table/conf/workload | 0 tests/{ => integration_tests}/move_table/main.go | 0 tests/{ => integration_tests}/move_table/run.sh | 0 .../multi_capture/conf/diff_config.toml | 0 .../multi_capture/conf/workload1 | 0 .../multi_capture/conf/workload2 | 0 tests/{ => integration_tests}/multi_capture/run.sh | 0 .../multi_changefeed/conf/changefeed1.toml | 0 .../multi_changefeed/conf/changefeed2.toml | 0 .../{ => integration_tests}/multi_changefeed/run.sh | 0 .../multi_source/config.toml | 0 .../multi_source/diff_config.toml | 0 tests/{ => integration_tests}/multi_source/main.go | 2 +- tests/{ => integration_tests}/multi_source/run.sh | 0 .../conf/changefeed.toml | 0 .../conf/diff_config.toml | 0 .../conf/tidb_config.toml | 0 .../new_ci_collation_with_old_value/data/test1.sql | 0 .../new_ci_collation_with_old_value/data/test2.sql | 0 .../new_ci_collation_with_old_value/run.sh | 0 .../conf/changefeed.toml | 0 .../conf/diff_config.toml | 0 .../conf/tidb_config.toml | 0 .../data/test1.sql | 0 .../data/test2.sql | 0 .../new_ci_collation_without_old_value/run.sh | 0 .../owner_remove_table_error/conf/diff_config.toml | 0 .../owner_remove_table_error/run.sh | 0 .../partition_table/conf/diff_config.toml | 0 .../partition_table/data/prepare.sql | 0 .../{ => integration_tests}/partition_table/run.sh | 0 .../processor_err_chan/conf/diff_config.toml | 0 .../processor_err_chan/run.sh | 0 .../processor_panic/config.toml | 0 .../processor_panic/diff_config.toml | 0 .../{ => integration_tests}/processor_panic/main.go | 2 +- .../{ => integration_tests}/processor_panic/run.sh | 0 .../conf/diff_config.toml | 0 .../processor_resolved_ts_fallback/run.sh | 0 .../processor_stop_delay/conf/diff_config.toml | 0 .../processor_stop_delay/run.sh | 0 .../region_merge/conf/diff_config.toml | 0 .../region_merge/conf/pd_config.toml | 0 tests/{ => integration_tests}/region_merge/run.sh | 0 .../resolve_lock/config.toml | 0 .../resolve_lock/diff_config.toml | 0 tests/{ => integration_tests}/resolve_lock/main.go | 2 +- tests/{ => integration_tests}/resolve_lock/run.sh | 0 .../row_format/conf/diff_config.toml | 0 .../row_format/data/step1.sql | 0 .../row_format/data/step2.sql | 0 .../row_format/data/step3.sql | 0 .../row_format/data/step4.sql | 0 tests/{ => integration_tests}/row_format/run.sh | 0 tests/{ => integration_tests}/run.sh | 12 ++++++------ tests/{ => integration_tests}/simple/run.sh | 0 .../sink_hang/conf/diff_config.toml | 0 tests/{ => integration_tests}/sink_hang/run.sh | 0 .../sink_retry/conf/diff_config.toml | 0 .../sink_retry/conf/workload | 0 tests/{ => integration_tests}/sink_retry/run.sh | 0 .../split_region/conf/diff_config.toml | 0 .../split_region/data/increment.sql | 0 .../split_region/data/prepare.sql | 0 tests/{ => integration_tests}/split_region/run.sh | 0 .../syncpoint/conf/diff_config_final.toml | 0 .../syncpoint/conf/diff_config_part1.toml | 0 .../syncpoint/conf/diff_config_part2.toml | 0 .../{ => integration_tests}/syncpoint/conf/workload | 0 tests/{ => integration_tests}/syncpoint/run.sh | 0 .../tiflash/conf/diff_config.toml | 0 .../tiflash/data/prepare.sql | 0 tests/{ => integration_tests}/tiflash/run.sh | 0 .../unified_sorter/conf/diff_config.toml | 0 .../unified_sorter/conf/workload | 0 tests/{ => integration_tests}/unified_sorter/run.sh | 0 tests/{ => integration_tests}/util/config.go | 0 tests/{ => integration_tests}/util/db.go | 2 +- 209 files changed, 26 insertions(+), 25 deletions(-) rename tests/{ => integration_tests}/README.md (93%) rename tests/{ => integration_tests}/_certificates/ca.pem (100%) rename tests/{ => integration_tests}/_certificates/client-key.pem (100%) rename tests/{ => integration_tests}/_certificates/client.pem (100%) rename tests/{ => integration_tests}/_certificates/kafka.server.keystore.jks (100%) rename tests/{ => integration_tests}/_certificates/kafka.server.truststore.jks (100%) rename tests/{ => integration_tests}/_certificates/server-key.pem (100%) rename tests/{ => integration_tests}/_certificates/server.pem (100%) rename tests/{ => integration_tests}/_utils/check_contains (100%) rename tests/{ => integration_tests}/_utils/check_logs (100%) rename tests/{ => integration_tests}/_utils/check_logs_contains (100%) rename tests/{ => integration_tests}/_utils/check_not_contains (100%) rename tests/{ => integration_tests}/_utils/check_sync_diff (100%) rename tests/{ => integration_tests}/_utils/check_table_exists (100%) rename tests/{ => integration_tests}/_utils/cleanup_process (100%) rename tests/{ => integration_tests}/_utils/ensure (100%) rename tests/{ => integration_tests}/_utils/random_kill_process (100%) rename tests/{ => integration_tests}/_utils/run_cdc_cli (100%) rename tests/{ => integration_tests}/_utils/run_cdc_cli_tso_query (100%) rename tests/{ => integration_tests}/_utils/run_cdc_server (100%) rename tests/{ => integration_tests}/_utils/run_kafka_consumer (100%) rename tests/{ => integration_tests}/_utils/run_sql (100%) rename tests/{ => integration_tests}/_utils/run_sql_file (100%) rename tests/{ => integration_tests}/_utils/start_tidb_cluster (100%) rename tests/{ => integration_tests}/_utils/start_tidb_cluster_impl (99%) rename tests/{ => integration_tests}/_utils/start_tls_tidb_cluster (100%) rename tests/{ => integration_tests}/_utils/start_tls_tidb_cluster_impl (98%) rename tests/{ => integration_tests}/_utils/stop_tidb_cluster (100%) rename tests/{ => integration_tests}/_utils/test_prepare (100%) rename tests/{ => integration_tests}/_utils/tiflash-users.toml (100%) rename tests/{ => integration_tests}/autorandom/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/autorandom/conf/tidb_config.toml (100%) rename tests/{ => integration_tests}/autorandom/data/test.sql (100%) rename tests/{ => integration_tests}/autorandom/run.sh (100%) rename tests/{ => integration_tests}/availability/capture.sh (100%) rename tests/{ => integration_tests}/availability/owner.sh (100%) rename tests/{ => integration_tests}/availability/processor.sh (100%) rename tests/{ => integration_tests}/availability/run.sh (100%) rename tests/{ => integration_tests}/bank/bank.go (100%) rename tests/{ => integration_tests}/bank/case.go (100%) rename tests/{ => integration_tests}/bank/run.sh (100%) rename tests/{ => integration_tests}/batch_add_table/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/batch_add_table/data/prepare.sql (100%) rename tests/{ => integration_tests}/batch_add_table/data/test.sql (100%) rename tests/{ => integration_tests}/batch_add_table/run.sh (100%) rename tests/{ => integration_tests}/capture_session_done_during_task/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/capture_session_done_during_task/run.sh (100%) rename tests/{ => integration_tests}/capture_suicide_while_balance_table/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/capture_suicide_while_balance_table/run.sh (100%) rename tests/{ => integration_tests}/cdc/cdc.go (93%) rename tests/{ => integration_tests}/cdc/config.toml (100%) rename tests/{ => integration_tests}/cdc/run.sh (100%) rename tests/{ => integration_tests}/cdclog_file/run.sh (100%) rename tests/{ => integration_tests}/cdclog_s3/run.sh (100%) rename tests/{ => integration_tests}/changefeed_auto_stop/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/changefeed_auto_stop/conf/workload (100%) rename tests/{ => integration_tests}/changefeed_auto_stop/run.sh (100%) rename tests/{ => integration_tests}/changefeed_error/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/changefeed_error/conf/workload (100%) rename tests/{ => integration_tests}/changefeed_error/run.sh (100%) rename tests/{ => integration_tests}/changefeed_fast_fail/run.sh (100%) rename tests/{ => integration_tests}/changefeed_finish/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/changefeed_finish/run.sh (100%) rename tests/{ => integration_tests}/changefeed_pause_resume/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/changefeed_pause_resume/run.sh (100%) rename tests/{ => integration_tests}/changefeed_reconstruct/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/changefeed_reconstruct/conf/workload (100%) rename tests/{ => integration_tests}/changefeed_reconstruct/run.sh (100%) rename tests/{ => integration_tests}/cli/run.sh (100%) rename tests/{ => integration_tests}/clustered_index/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/clustered_index/data/test.sql (100%) rename tests/{ => integration_tests}/clustered_index/run.sh (100%) rename tests/{ => integration_tests}/common_1/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/common_1/data/test.sql (100%) rename tests/{ => integration_tests}/common_1/data/test_finish.sql (100%) rename tests/{ => integration_tests}/common_1/data/test_v5.sql (100%) rename tests/{ => integration_tests}/common_1/run.sh (100%) rename tests/{ => integration_tests}/cyclic_ab/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/cyclic_ab/conf/only_test_simple.toml (100%) rename tests/{ => integration_tests}/cyclic_ab/run.sh (100%) rename tests/{ => integration_tests}/cyclic_abc/conf/changefeed.toml (100%) rename tests/{ => integration_tests}/cyclic_abc/conf/diff_config_down_tls.toml (100%) rename tests/{ => integration_tests}/cyclic_abc/conf/diff_config_up_down.toml (100%) rename tests/{ => integration_tests}/cyclic_abc/run.sh (100%) rename tests/{ => integration_tests}/dailytest/case.go (100%) rename tests/{ => integration_tests}/dailytest/dailytest.go (100%) rename tests/{ => integration_tests}/dailytest/data.go (100%) rename tests/{ => integration_tests}/dailytest/db.go (99%) rename tests/{ => integration_tests}/dailytest/exector.go (100%) rename tests/{ => integration_tests}/dailytest/job.go (100%) rename tests/{ => integration_tests}/dailytest/parser.go (100%) rename tests/{ => integration_tests}/dailytest/rand.go (100%) rename tests/{ => integration_tests}/ddl_puller_lag/run.sh (100%) rename tests/{ => integration_tests}/ddl_reentrant/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/ddl_reentrant/conf/tidb_config.toml (100%) rename tests/{ => integration_tests}/ddl_reentrant/run.sh (100%) rename tests/{ => integration_tests}/ddl_sequence/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/ddl_sequence/data/prepare.sql (100%) rename tests/{ => integration_tests}/ddl_sequence/run.sh (100%) rename tests/{ => integration_tests}/drop_many_tables/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/drop_many_tables/data/prepare.sql (100%) rename tests/{ => integration_tests}/drop_many_tables/run.sh (100%) rename tests/{ => integration_tests}/force_replicate_table/conf/changefeed.toml (100%) rename tests/{ => integration_tests}/force_replicate_table/conf/tidb_config.toml (100%) rename tests/{ => integration_tests}/force_replicate_table/data/test.sql (100%) rename tests/{ => integration_tests}/force_replicate_table/run.sh (100%) rename tests/{ => integration_tests}/gc_safepoint/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/gc_safepoint/run.sh (100%) rename tests/{ => integration_tests}/generate_column/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/generate_column/data/prepare.sql (100%) rename tests/{ => integration_tests}/generate_column/run.sh (100%) rename tests/{ => integration_tests}/http_api/run.sh (100%) rename tests/{ => integration_tests}/http_api/util/test_case.py (100%) rename tests/{ => integration_tests}/http_proxies/run-proxy.go (100%) rename tests/{ => integration_tests}/http_proxies/run.sh (100%) rename tests/{ => integration_tests}/kafka_messages/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/kafka_messages/conf/workload (100%) rename tests/{ => integration_tests}/kafka_messages/run.sh (100%) rename tests/{ => integration_tests}/kafka_sink_error_resume/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/kafka_sink_error_resume/run.sh (100%) rename tests/{ => integration_tests}/kill_owner_with_ddl/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/kill_owner_with_ddl/run.sh (100%) rename tests/{ => integration_tests}/kv_client_stream_reconnect/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/kv_client_stream_reconnect/run.sh (100%) rename tests/{ => integration_tests}/many_pk_or_uk/config.toml (100%) rename tests/{ => integration_tests}/many_pk_or_uk/diff_config.toml (100%) rename tests/{ => integration_tests}/many_pk_or_uk/main.go (98%) rename tests/{ => integration_tests}/many_pk_or_uk/run.sh (100%) rename tests/{ => integration_tests}/move_table/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/move_table/conf/workload (100%) rename tests/{ => integration_tests}/move_table/main.go (100%) rename tests/{ => integration_tests}/move_table/run.sh (100%) rename tests/{ => integration_tests}/multi_capture/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/multi_capture/conf/workload1 (100%) rename tests/{ => integration_tests}/multi_capture/conf/workload2 (100%) rename tests/{ => integration_tests}/multi_capture/run.sh (100%) rename tests/{ => integration_tests}/multi_changefeed/conf/changefeed1.toml (100%) rename tests/{ => integration_tests}/multi_changefeed/conf/changefeed2.toml (100%) rename tests/{ => integration_tests}/multi_changefeed/run.sh (100%) rename tests/{ => integration_tests}/multi_source/config.toml (100%) rename tests/{ => integration_tests}/multi_source/diff_config.toml (100%) rename tests/{ => integration_tests}/multi_source/main.go (99%) rename tests/{ => integration_tests}/multi_source/run.sh (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/conf/changefeed.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/conf/tidb_config.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/data/test1.sql (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/data/test2.sql (100%) rename tests/{ => integration_tests}/new_ci_collation_with_old_value/run.sh (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/conf/changefeed.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/conf/tidb_config.toml (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/data/test1.sql (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/data/test2.sql (100%) rename tests/{ => integration_tests}/new_ci_collation_without_old_value/run.sh (100%) rename tests/{ => integration_tests}/owner_remove_table_error/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/owner_remove_table_error/run.sh (100%) rename tests/{ => integration_tests}/partition_table/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/partition_table/data/prepare.sql (100%) rename tests/{ => integration_tests}/partition_table/run.sh (100%) rename tests/{ => integration_tests}/processor_err_chan/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/processor_err_chan/run.sh (100%) rename tests/{ => integration_tests}/processor_panic/config.toml (100%) rename tests/{ => integration_tests}/processor_panic/diff_config.toml (100%) rename tests/{ => integration_tests}/processor_panic/main.go (97%) rename tests/{ => integration_tests}/processor_panic/run.sh (100%) rename tests/{ => integration_tests}/processor_resolved_ts_fallback/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/processor_resolved_ts_fallback/run.sh (100%) rename tests/{ => integration_tests}/processor_stop_delay/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/processor_stop_delay/run.sh (100%) rename tests/{ => integration_tests}/region_merge/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/region_merge/conf/pd_config.toml (100%) rename tests/{ => integration_tests}/region_merge/run.sh (100%) rename tests/{ => integration_tests}/resolve_lock/config.toml (100%) rename tests/{ => integration_tests}/resolve_lock/diff_config.toml (100%) rename tests/{ => integration_tests}/resolve_lock/main.go (99%) rename tests/{ => integration_tests}/resolve_lock/run.sh (100%) rename tests/{ => integration_tests}/row_format/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/row_format/data/step1.sql (100%) rename tests/{ => integration_tests}/row_format/data/step2.sql (100%) rename tests/{ => integration_tests}/row_format/data/step3.sql (100%) rename tests/{ => integration_tests}/row_format/data/step4.sql (100%) rename tests/{ => integration_tests}/row_format/run.sh (100%) rename tests/{ => integration_tests}/run.sh (85%) rename tests/{ => integration_tests}/simple/run.sh (100%) rename tests/{ => integration_tests}/sink_hang/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/sink_hang/run.sh (100%) rename tests/{ => integration_tests}/sink_retry/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/sink_retry/conf/workload (100%) rename tests/{ => integration_tests}/sink_retry/run.sh (100%) rename tests/{ => integration_tests}/split_region/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/split_region/data/increment.sql (100%) rename tests/{ => integration_tests}/split_region/data/prepare.sql (100%) rename tests/{ => integration_tests}/split_region/run.sh (100%) rename tests/{ => integration_tests}/syncpoint/conf/diff_config_final.toml (100%) rename tests/{ => integration_tests}/syncpoint/conf/diff_config_part1.toml (100%) rename tests/{ => integration_tests}/syncpoint/conf/diff_config_part2.toml (100%) rename tests/{ => integration_tests}/syncpoint/conf/workload (100%) rename tests/{ => integration_tests}/syncpoint/run.sh (100%) rename tests/{ => integration_tests}/tiflash/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/tiflash/data/prepare.sql (100%) rename tests/{ => integration_tests}/tiflash/run.sh (100%) rename tests/{ => integration_tests}/unified_sorter/conf/diff_config.toml (100%) rename tests/{ => integration_tests}/unified_sorter/conf/workload (100%) rename tests/{ => integration_tests}/unified_sorter/run.sh (100%) rename tests/{ => integration_tests}/util/config.go (100%) rename tests/{ => integration_tests}/util/db.go (98%) diff --git a/Makefile b/Makefile index 344872f8e9a..7412bf401cb 100644 --- a/Makefile +++ b/Makefile @@ -145,10 +145,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)" 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/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 0acdfbda8d4..601e4c42e73 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 100% rename from tests/_utils/check_sync_diff rename to tests/integration_tests/_utils/check_sync_diff 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 100% rename from tests/autorandom/conf/diff_config.toml rename to tests/integration_tests/autorandom/conf/diff_config.toml 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 100% rename from tests/batch_add_table/conf/diff_config.toml rename to tests/integration_tests/batch_add_table/conf/diff_config.toml 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 100% 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 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 100% 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 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/cdclog_file/run.sh b/tests/integration_tests/cdclog_file/run.sh similarity index 100% rename from tests/cdclog_file/run.sh rename to tests/integration_tests/cdclog_file/run.sh diff --git a/tests/cdclog_s3/run.sh b/tests/integration_tests/cdclog_s3/run.sh similarity index 100% rename from tests/cdclog_s3/run.sh rename to tests/integration_tests/cdclog_s3/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 100% rename from tests/changefeed_auto_stop/conf/diff_config.toml rename to tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml 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 100% rename from tests/changefeed_error/conf/diff_config.toml rename to tests/integration_tests/changefeed_error/conf/diff_config.toml 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 100% rename from tests/changefeed_finish/conf/diff_config.toml rename to tests/integration_tests/changefeed_finish/conf/diff_config.toml 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 100% rename from tests/changefeed_pause_resume/conf/diff_config.toml rename to tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml 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 100% rename from tests/changefeed_reconstruct/conf/diff_config.toml rename to tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml 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 100% rename from tests/clustered_index/conf/diff_config.toml rename to tests/integration_tests/clustered_index/conf/diff_config.toml 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 100% rename from tests/common_1/conf/diff_config.toml rename to tests/integration_tests/common_1/conf/diff_config.toml 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/cyclic_ab/conf/diff_config.toml b/tests/integration_tests/cyclic_ab/conf/diff_config.toml similarity index 100% rename from tests/cyclic_ab/conf/diff_config.toml rename to tests/integration_tests/cyclic_ab/conf/diff_config.toml 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 100% rename from tests/cyclic_abc/conf/diff_config_down_tls.toml rename to tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml 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 100% rename from tests/cyclic_abc/conf/diff_config_up_down.toml rename to tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml 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 15095a11f4f..76f4f90c7be 100644 --- a/tests/dailytest/db.go +++ b/tests/integration_tests/dailytest/db.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "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_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 100% rename from tests/ddl_reentrant/conf/diff_config.toml rename to tests/integration_tests/ddl_reentrant/conf/diff_config.toml 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 100% rename from tests/ddl_sequence/conf/diff_config.toml rename to tests/integration_tests/ddl_sequence/conf/diff_config.toml 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 100% rename from tests/drop_many_tables/conf/diff_config.toml rename to tests/integration_tests/drop_many_tables/conf/diff_config.toml 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 100% rename from tests/gc_safepoint/conf/diff_config.toml rename to tests/integration_tests/gc_safepoint/conf/diff_config.toml 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 100% rename from tests/generate_column/conf/diff_config.toml rename to tests/integration_tests/generate_column/conf/diff_config.toml 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 100% rename from tests/kafka_messages/conf/diff_config.toml rename to tests/integration_tests/kafka_messages/conf/diff_config.toml 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 100% rename from tests/kafka_sink_error_resume/conf/diff_config.toml rename to tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml 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 100% rename from tests/kill_owner_with_ddl/conf/diff_config.toml rename to tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml 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 100% rename from tests/kv_client_stream_reconnect/conf/diff_config.toml rename to tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml 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/many_pk_or_uk/diff_config.toml b/tests/integration_tests/many_pk_or_uk/diff_config.toml similarity index 100% rename from tests/many_pk_or_uk/diff_config.toml rename to tests/integration_tests/many_pk_or_uk/diff_config.toml 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 100% rename from tests/move_table/conf/diff_config.toml rename to tests/integration_tests/move_table/conf/diff_config.toml 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 100% rename from tests/multi_capture/conf/diff_config.toml rename to tests/integration_tests/multi_capture/conf/diff_config.toml 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/multi_source/diff_config.toml b/tests/integration_tests/multi_source/diff_config.toml similarity index 100% rename from tests/multi_source/diff_config.toml rename to tests/integration_tests/multi_source/diff_config.toml 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 100% 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 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 100% 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 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 100% rename from tests/owner_remove_table_error/conf/diff_config.toml rename to tests/integration_tests/owner_remove_table_error/conf/diff_config.toml 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 100% rename from tests/partition_table/conf/diff_config.toml rename to tests/integration_tests/partition_table/conf/diff_config.toml 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 100% rename from tests/processor_err_chan/conf/diff_config.toml rename to tests/integration_tests/processor_err_chan/conf/diff_config.toml 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/processor_panic/diff_config.toml b/tests/integration_tests/processor_panic/diff_config.toml similarity index 100% rename from tests/processor_panic/diff_config.toml rename to tests/integration_tests/processor_panic/diff_config.toml 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 100% rename from tests/processor_resolved_ts_fallback/conf/diff_config.toml rename to tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml 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 100% rename from tests/processor_stop_delay/conf/diff_config.toml rename to tests/integration_tests/processor_stop_delay/conf/diff_config.toml 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 100% rename from tests/region_merge/conf/diff_config.toml rename to tests/integration_tests/region_merge/conf/diff_config.toml 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/resolve_lock/diff_config.toml b/tests/integration_tests/resolve_lock/diff_config.toml similarity index 100% rename from tests/resolve_lock/diff_config.toml rename to tests/integration_tests/resolve_lock/diff_config.toml 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 3586aaa933a..db4dff8b24f 100644 --- a/tests/resolve_lock/main.go +++ b/tests/integration_tests/resolve_lock/main.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/tests/util" + "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/tablecodec" 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 100% rename from tests/row_format/conf/diff_config.toml rename to tests/integration_tests/row_format/conf/diff_config.toml 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 85% rename from tests/run.sh rename to tests/integration_tests/run.sh index b42b8ea9a36..e90c2832801 100755 --- a/tests/run.sh +++ b/tests/integration_tests/run.sh @@ -4,8 +4,8 @@ set -eu OUT_DIR=/tmp/tidb_cdc_test CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -export PATH=$PATH:$CUR/_utils:$CUR/../bin - +echo $PATH +export PATH=$PATH:$CUR/_utils:$CUR/../../bin mkdir -p $OUT_DIR || true if [ "${1-}" = '--debug' ]; then @@ -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 100% rename from tests/sink_hang/conf/diff_config.toml rename to tests/integration_tests/sink_hang/conf/diff_config.toml 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 100% rename from tests/sink_retry/conf/diff_config.toml rename to tests/integration_tests/sink_retry/conf/diff_config.toml 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 100% rename from tests/split_region/conf/diff_config.toml rename to tests/integration_tests/split_region/conf/diff_config.toml 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 100% rename from tests/syncpoint/conf/diff_config_final.toml rename to tests/integration_tests/syncpoint/conf/diff_config_final.toml diff --git a/tests/syncpoint/conf/diff_config_part1.toml b/tests/integration_tests/syncpoint/conf/diff_config_part1.toml similarity index 100% rename from tests/syncpoint/conf/diff_config_part1.toml rename to tests/integration_tests/syncpoint/conf/diff_config_part1.toml 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 100% rename from tests/tiflash/conf/diff_config.toml rename to tests/integration_tests/tiflash/conf/diff_config.toml 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 100% rename from tests/unified_sorter/conf/diff_config.toml rename to tests/integration_tests/unified_sorter/conf/diff_config.toml 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", From 71a74d978e350bb22b9cd09d5f07bcb5d2e8a28c Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 7 Dec 2021 17:29:56 +0800 Subject: [PATCH 12/16] sink(ticdc): Set config.Metadata.Timeout correctly (#3665) (#3672) --- cdc/sink/producer/kafka/kafka.go | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) 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 From 07f19656b9e7d05633e80f4f3b6ff91600f33f58 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 8 Dec 2021 14:49:57 +0800 Subject: [PATCH 13/16] ticdc.rules: fix alertmanager rules (#3422) (#3596) --- metrics/alertmanager/ticdc.rules.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 }}' From 632f53e74870fd60aa7efb7bc28f7968c167b744 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 8 Dec 2021 15:25:56 +0800 Subject: [PATCH 14/16] Fix handling of JSON columns (#3643) (#3653) --- cdc/sink/codec/avro.go | 3 +-- cdc/sink/codec/avro_test.go | 1 + 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index 1d6452975ff..7a7b2b3feef 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/ticdc/cdc/model" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/types" - tijson "github.com/pingcap/tidb/types/json" "go.uber.org/zap" ) @@ -474,7 +473,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 97074664fa8..779be79c451 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) From 636c0f565210577605d7a9bc96a808385a99c983 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 8 Dec 2021 15:55:56 +0800 Subject: [PATCH 15/16] *: fix changefeed checkpoint lag negative value error (#3013) (#3535) --- cdc/capture/capture.go | 36 +++++--- cdc/owner/changefeed.go | 13 ++- cdc/owner/changefeed_test.go | 2 + cdc/processor/manager.go | 1 + cdc/processor/processor.go | 29 ++++--- pkg/context/context.go | 14 +-- pkg/orchestrator/etcd_worker_test.go | 6 +- pkg/pdtime/acquirer.go | 125 +++++++++++++++++++++++++++ pkg/pdtime/acquirer_test.go | 60 +++++++++++++ pkg/txnutil/gc/gc_manager.go | 33 +++---- pkg/txnutil/gc/gc_manager_test.go | 45 ++++------ 11 files changed, 271 insertions(+), 93 deletions(-) create mode 100644 pkg/pdtime/acquirer.go create mode 100644 pkg/pdtime/acquirer_test.go diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index c6a174ee6c6..46e580c62e2 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -32,6 +32,7 @@ import ( cdcContext "github.com/pingcap/ticdc/pkg/context" cerror "github.com/pingcap/ticdc/pkg/errors" "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" @@ -54,12 +55,12 @@ type Capture struct { session *concurrency.Session election *concurrency.Election - pdClient pd.Client - kvStorage tidbkv.Storage - etcdClient *kv.CDCEtcdClient - grpcPool kv.GrpcPool - - cancel context.CancelFunc + pdClient pd.Client + kvStorage tidbkv.Storage + etcdClient *kv.CDCEtcdClient + grpcPool kv.GrpcPool + TimeAcquirer pdtime.TimeAcquirer + cancel context.CancelFunc newProcessorManager func() *processor.Manager newOwner func(pd.Client) *owner.Owner @@ -99,6 +100,12 @@ func (c *Capture) reset(ctx context.Context) error { } c.session = sess c.election = concurrency.NewElection(sess, kv.CaptureOwnerKey) + + if c.TimeAcquirer != nil { + c.TimeAcquirer.Stop() + } + c.TimeAcquirer = pdtime.NewTimeAcquirer(c.pdClient) + if c.grpcPool != nil { c.grpcPool.Close() } @@ -147,11 +154,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 { @@ -165,7 +173,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() @@ -187,6 +195,10 @@ func (c *Capture) run(stdCtx context.Context) error { processorErr = c.runEtcdWorker(ctx, c.processorManager, model.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 c6670f5098c..a4c1355e0ab 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" @@ -178,7 +177,9 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *model.ChangefeedReactor return errors.Trace(err) } if shouldUpdateState { - c.updateStatus(barrierTs) + pdTime, _ := ctx.GlobalVars().TimeAcquirer.CurrentTimeFromCached() + currentTs := oracle.GetPhysical(pdTime) + c.updateStatus(currentTs, barrierTs) } return nil } @@ -438,7 +439,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 { @@ -470,12 +471,10 @@ 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() { diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 4d3e1cd3733..ec255aabd0f 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -26,6 +26,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" @@ -217,6 +218,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", diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index d1bf39e7854..29c4f6e737d 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/processor.go b/cdc/processor/processor.go index a69bb16502f..06fdfa2a16c 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -20,7 +20,6 @@ import ( "math" "strconv" "sync" - "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -154,7 +153,8 @@ func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorS 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 { @@ -169,7 +169,12 @@ func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorS if err := p.checkTablesNum(ctx); err != nil { return nil, errors.Trace(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) @@ -187,10 +192,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 { @@ -552,7 +557,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() @@ -573,15 +578,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/pkg/context/context.go b/pkg/context/context.go index 21f99ffbeba..9449638e4a0 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -18,6 +18,8 @@ import ( "log" "time" + "github.com/pingcap/ticdc/pkg/pdtime" + "github.com/pingcap/ticdc/pkg/version" "github.com/pingcap/ticdc/cdc/kv" @@ -33,11 +35,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 *kv.CDCEtcdClient - GrpcPool kv.GrpcPool + PDClient pd.Client + KVStorage tidbkv.Storage + CaptureInfo *model.CaptureInfo + EtcdClient *kv.CDCEtcdClient + GrpcPool kv.GrpcPool + TimeAcquirer pdtime.TimeAcquirer } // ChangefeedVars contains some vars which can be used anywhere in a pipeline @@ -184,6 +187,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/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index db032e1e2ce..661f93751bf 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" @@ -224,7 +225,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) @@ -273,7 +273,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/pdtime/acquirer.go b/pkg/pdtime/acquirer.go new file mode 100644 index 00000000000..f232b56b027 --- /dev/null +++ b/pkg/pdtime/acquirer.go @@ -0,0 +1,125 @@ +// 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 + +// TimeAcquirer cache time get from PD periodically +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) +} + +// Stop stop TimeAcquirer +func (c *TimeAcquirerImpl) Stop() { + c.cancel() +} + +// TimeAcquirer4Test only for test +type TimeAcquirer4Test struct{} + +// NewTimeAcquirer4Test return a TimeAcquirer for test +func NewTimeAcquirer4Test() TimeAcquirer { + return &TimeAcquirer4Test{} +} + +// CurrentTimeFromCached return current time +func (c *TimeAcquirer4Test) CurrentTimeFromCached() (time.Time, error) { + return time.Now(), nil +} + +// Run implements TimeAcquirer +func (c *TimeAcquirer4Test) Run(ctx context.Context) { +} + +// Stop implements TimeAcquirer +func (c *TimeAcquirer4Test) Stop() { +} diff --git a/pkg/pdtime/acquirer_test.go b/pkg/pdtime/acquirer_test.go new file mode 100644 index 00000000000..92ab915b78e --- /dev/null +++ b/pkg/pdtime/acquirer_test.go @@ -0,0 +1,60 @@ +// 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/pingcap/check" + "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" +) + +func TestSuite(t *testing.T) { check.TestingT(t) } + +type pdTimeSuite struct{} + +var _ = check.Suite(&pdTimeSuite{}) + +// 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 (s *pdTimeSuite) TestTimeFromPD(c *check.C) { + defer testleak.AfterTest(c)() + mockPDClient := &MockPDClient{} + TimeAcquirer := NewTimeAcquirer(mockPDClient) + go TimeAcquirer.Run(context.Background()) + defer TimeAcquirer.Stop() + time.Sleep(1 * time.Second) + + t1, err := TimeAcquirer.CurrentTimeFromCached() + c.Assert(err, check.IsNil) + + time.Sleep(400 * time.Millisecond) + // assume that the gc safe point updated one hour ago + t2, err := TimeAcquirer.CurrentTimeFromCached() + c.Assert(err, check.IsNil) + // should return new time + c.Assert(t1, check.Less, t2) +} 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 9ac9465b490..314631c3868 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" @@ -88,49 +90,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) } From 036ff3427699fc877632b0a160a2014922fc7a9c Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 10 Dec 2021 23:09:57 +0800 Subject: [PATCH 16/16] workerpool: limit the rate to output deadlock warning (#3775) (#3799) --- pkg/workerpool/pool_impl.go | 20 +++++++++++++++++--- pkg/workerpool/pool_test.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index 966e6ea35c3..d98d4ffc1a8 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/ticdc/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) const ( @@ -248,6 +249,9 @@ type worker struct { isRunning int32 // notifies exits of run() stopNotifier notify.Notifier + + slowSynchronizeThreshold time.Duration + slowSynchronizeLimiter *rate.Limiter } func newWorker() *worker { @@ -255,6 +259,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), } } @@ -340,13 +347,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 dc5a2bff61a..c34378fd625 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -25,8 +25,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) func TestSuite(t *testing.T) { check.TestingT(t) } @@ -425,6 +427,38 @@ func (s *workerPoolSuite) TestCancelByAddEventContext(c *check.C) { c.Assert(err, check.IsNil) } +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) {