diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 1cd2f6f736d..348bbccbd9a 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -244,6 +244,7 @@ func (o *Owner) WriteDebugInfo(w io.Writer) { // AsyncStop stops the owner asynchronously func (o *Owner) AsyncStop() { atomic.StoreInt32(&o.closed, 1) + o.cleanStaleMetrics() } func (o *Owner) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { @@ -273,6 +274,7 @@ func (o *Owner) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { // Bootstrap checks if the state contains incompatible or incorrect information and tries to fix it. func (o *Owner) Bootstrap(state *orchestrator.GlobalReactorState) { log.Info("Start bootstrapping", zap.Any("state", state)) + o.cleanStaleMetrics() fixChangefeedInfos(state) } @@ -281,6 +283,9 @@ func fixChangefeedInfos(state *orchestrator.GlobalReactorState) { for _, changefeedState := range state.Changefeeds { if changefeedState != nil { changefeedState.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + if info == nil { + return nil, false, nil + } info.FixIncompatible() return info, true, nil }) @@ -288,6 +293,18 @@ func fixChangefeedInfos(state *orchestrator.GlobalReactorState) { } } +func (o *Owner) cleanStaleMetrics() { + // The gauge metrics of the Owner should be reset + // each time a new owner is launched, in case the previous owner + // has crashed and has not cleaned up the stale metrics values. + changefeedCheckpointTsGauge.Reset() + changefeedCheckpointTsLagGauge.Reset() + changefeedResolvedTsGauge.Reset() + changefeedResolvedTsLagGauge.Reset() + ownerMaintainTableNumGauge.Reset() + changefeedStatusGauge.Reset() +} + func (o *Owner) updateMetrics(state *orchestrator.GlobalReactorState) { // Keep the value of prometheus expression `rate(counter)` = 1 // Please also change alert rule in ticdc.rules.yml when change the expression value. @@ -297,6 +314,7 @@ func (o *Owner) updateMetrics(state *orchestrator.GlobalReactorState) { ownerMaintainTableNumGauge.Reset() changefeedStatusGauge.Reset() + for changefeedID, changefeedState := range state.Changefeeds { for captureID, captureInfo := range state.Captures { taskStatus, exist := changefeedState.TaskStatuses[captureID] diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index 75b12a5ba4c..6f07b527329 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -58,6 +58,10 @@ func (c *mockFlowController) GetConsumption() uint64 { return 0 } +func (s *mockSink) Init(tableID model.TableID) error { + return nil +} + func (s *mockSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { for _, row := range rows { s.received = append(s.received, struct { diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index f9c1c0f5572..32dcf68c3e1 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -763,7 +763,11 @@ func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID mode tableNameStr = tableName.QuoteString() } - sink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs, p.redoManager) + sink, err := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs, p.redoManager) + if err != nil { + return nil, errors.Trace(err) + } + table := tablepipeline.NewTablePipeline( ctx, p.mounter, diff --git a/cdc/redo/common/util.go b/cdc/redo/common/util.go index 5ba7321583a..fdfd4478104 100644 --- a/cdc/redo/common/util.go +++ b/cdc/redo/common/util.go @@ -26,6 +26,15 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" ) +const ( + // RedoLogFileFormatV1 was used before v6.1.0, which doesn't contain namespace information + // layout: captureID_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName + RedoLogFileFormatV1 = "%s_%s_%s_%d_%s%s" + // RedoLogFileFormatV2 is available since v6.1.0, which contains namespace information + // layout: captureID_namespace_changefeedID_fileType_maxEventCommitTs_uuid.fileExtName + RedoLogFileFormatV2 = "%s_%s_%s_%s_%d_%s%s" +) + // InitS3storage init a storage used for s3, // s3URI should be like s3URI="s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/" var InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { @@ -58,6 +67,13 @@ var InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStor return s3storage, nil } +// logFormat2ParseFormat converts redo log file name format to the space separated +// format, which can be read and parsed by sscanf. Besides remove the suffix `%s` +// which is used as file name extension, since we will parse extension first. +func logFormat2ParseFormat(fmtStr string) string { + return strings.TrimSuffix(strings.ReplaceAll(fmtStr, "_", " "), "%s") +} + // ParseLogFileName extract the commitTs, fileType from log fileName func ParseLogFileName(name string) (uint64, string, error) { ext := filepath.Ext(name) @@ -66,7 +82,9 @@ func ParseLogFileName(name string) (uint64, string, error) { } // if .sort, the name should be like - // fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.captureID, w.cfg.changeFeedID, w.cfg.createTime.Unix(), w.cfg.fileType, w.commitTS.Load(), LogEXT)+SortLogEXT + // fmt.Sprintf("%s_%s_%s_%d_%s_%d%s", w.cfg.captureID, + // w.cfg.changeFeedID.Namespace,w.cfg.changeFeedID.ID, + // w.cfg.fileType, w.commitTS.Load(), uuid, LogEXT)+SortLogEXT if ext == SortLogEXT { name = strings.TrimSuffix(name, SortLogEXT) ext = filepath.Ext(name) @@ -75,15 +93,12 @@ func ParseLogFileName(name string) (uint64, string, error) { return 0, "", nil } - var commitTs, d1 uint64 - var s1, s2, fileType string + var commitTs uint64 + var s1, s2, fileType, uid string // the log looks like: fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.captureID, w.cfg.changeFeedID, w.cfg.createTime.Unix(), w.cfg.fileType, w.commitTS.Load(), redo.LogEXT) - formatStr := "%s %s %d %s %d" + LogEXT - if ext == TmpEXT { - formatStr += TmpEXT - } + formatStr := logFormat2ParseFormat(RedoLogFileFormatV1) name = strings.ReplaceAll(name, "_", " ") - _, err := fmt.Sscanf(name, formatStr, &s1, &s2, &d1, &fileType, &commitTs) + _, err := fmt.Sscanf(name, formatStr, &s1, &s2, &fileType, &commitTs, &uid) if err != nil { return 0, "", errors.Annotatef(err, "bad log name: %s", name) } diff --git a/cdc/redo/common/util_test.go b/cdc/redo/common/util_test.go index acd9d92b4b8..d1aaadc98d5 100644 --- a/cdc/redo/common/util_test.go +++ b/cdc/redo/common/util_test.go @@ -16,8 +16,8 @@ package common import ( "fmt" "testing" - "time" + "github.com/google/uuid" "github.com/stretchr/testify/require" ) @@ -25,7 +25,6 @@ func TestParseLogFileName(t *testing.T) { type arg struct { name string } - // the log looks like: fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.captureID, w.cfg.changeFeedID, w.cfg.createTime.Unix(), w.cfg.fileType, w.commitTS.Load(), redo.LogEXT) tests := []struct { name string args arg @@ -36,7 +35,29 @@ func TestParseLogFileName(t *testing.T) { { name: "happy row .log", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test", time.Now().Unix(), DefaultRowLogFileType, 1, LogEXT), + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultRowLogFileType, 1, uuid.New().String(), LogEXT), + }, + wantTs: 1, + wantFileType: DefaultRowLogFileType, + }, + { + name: "happy row .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultRowLogFileType, 1, uuid.New().String(), LogEXT), + }, + wantTs: 1, + wantFileType: DefaultRowLogFileType, + }, + { + name: "happy row .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultRowLogFileType, 1, uuid.New().String(), LogEXT) + TmpEXT, }, wantTs: 1, wantFileType: DefaultRowLogFileType, @@ -44,7 +65,9 @@ func TestParseLogFileName(t *testing.T) { { name: "happy row .tmp", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test", time.Now().Unix(), DefaultRowLogFileType, 1, LogEXT) + TmpEXT, + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultRowLogFileType, 1, uuid.New().String(), LogEXT) + TmpEXT, }, wantTs: 1, wantFileType: DefaultRowLogFileType, @@ -52,7 +75,19 @@ func TestParseLogFileName(t *testing.T) { { name: "happy ddl .log", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test", time.Now().Unix(), DefaultDDLLogFileType, 1, LogEXT), + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT), + }, + wantTs: 1, + wantFileType: DefaultDDLLogFileType, + }, + { + name: "happy ddl .log", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT), }, wantTs: 1, wantFileType: DefaultDDLLogFileType, @@ -60,7 +95,29 @@ func TestParseLogFileName(t *testing.T) { { name: "happy ddl .sort", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test", time.Now().Unix(), DefaultDDLLogFileType, 1, LogEXT) + SortLogEXT, + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT) + SortLogEXT, + }, + wantTs: 1, + wantFileType: DefaultDDLLogFileType, + }, + { + name: "happy ddl .sort", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT) + SortLogEXT, + }, + wantTs: 1, + wantFileType: DefaultDDLLogFileType, + }, + { + name: "happy ddl .tmp", + args: arg{ + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT) + TmpEXT, }, wantTs: 1, wantFileType: DefaultDDLLogFileType, @@ -68,7 +125,9 @@ func TestParseLogFileName(t *testing.T) { { name: "happy ddl .tmp", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test", time.Now().Unix(), DefaultDDLLogFileType, 1, LogEXT) + TmpEXT, + name: fmt.Sprintf(RedoLogFileFormatV1, "cp", + "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT) + TmpEXT, }, wantTs: 1, wantFileType: DefaultDDLLogFileType, @@ -90,7 +149,9 @@ func TestParseLogFileName(t *testing.T) { { name: "err wrong format ddl .tmp", args: arg{ - name: fmt.Sprintf("%s_%s_%d_%s%d%s", "cp", "test", time.Now().Unix(), DefaultDDLLogFileType, 1, LogEXT) + TmpEXT, + name: fmt.Sprintf("%s_%s_%s_%d%s%s", /* a wrong format */ + "cp", "test", + DefaultDDLLogFileType, 1, uuid.New().String(), LogEXT) + TmpEXT, }, wantErr: ".*bad log name*.", }, diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 886cd781f0f..2eba8af7bf3 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -22,6 +22,7 @@ import ( "sync/atomic" "time" + "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" @@ -117,6 +118,11 @@ type ManagerOptions struct { type cacheRows struct { tableID model.TableID rows []*model.RowChangedEvent + // When calling FlushLog for a table, we must ensure that all data of this + // table has been written to underlying writer. Since the EmitRowChangedEvents + // and FlushLog of the same table can't be executed concurrently, we can + // insert a simple barrier data into data stream to achieve this goal. + flushCallback chan struct{} } // ManagerImpl manages redo log writer, buffers un-persistent redo logs, calculates @@ -214,6 +220,9 @@ func (m *ManagerImpl) Enabled() bool { // error ErrBufferLogTimeout will be returned. // TODO: if the API is truly non-blocking, we should return an error immediatel // when the log buffer channel is full. +// TODO: After buffer sink in sink node is removed, there is no batch mechanism +// before sending row changed events to redo manager, the original log buffer +// design may have performance issue. func (m *ManagerImpl) EmitRowChangedEvents( ctx context.Context, tableID model.TableID, @@ -248,6 +257,20 @@ func (m *ManagerImpl) FlushLog( return nil } defer atomic.StoreInt64(&m.flushing, 0) + + // Adding a barrier to data stream, to ensure all logs of this table has been + // written to underlying writer. + flushCallbackCh := make(chan struct{}) + m.logBuffer <- cacheRows{ + tableID: tableID, + flushCallback: flushCallbackCh, + } + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case <-flushCallbackCh: + } + return m.writer.FlushLog(ctx, tableID, resolvedTs) } @@ -322,8 +345,11 @@ func (m *ManagerImpl) updateTableResolvedTs(ctx context.Context) error { return err } minResolvedTs := uint64(math.MaxUint64) - for tableID, rts := range rtsMap { - m.rtsMap[tableID] = rts + for tableID := range m.rtsMap { + if rts, ok := rtsMap[tableID]; ok { + m.rtsMap[tableID] = rts + } + rts := m.rtsMap[tableID] if rts < minResolvedTs { minResolvedTs = rts } @@ -359,6 +385,10 @@ func (m *ManagerImpl) bgWriteLog(ctx context.Context, errCh chan<- error) { case <-ctx.Done(): return case cache := <-m.logBuffer: + if cache.flushCallback != nil { + close(cache.flushCallback) + continue + } logs := make([]*model.RedoRowChangedEvent, 0, len(cache.rows)) for _, row := range cache.rows { logs = append(logs, RowToRedo(row)) diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index 72b51731d5d..23422bd8c32 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -15,6 +15,8 @@ package redo import ( "context" + "sync" + "sync/atomic" "testing" "time" @@ -173,6 +175,81 @@ func TestLogManagerInProcessor(t *testing.T) { require.Nil(t, err) } +// TestUpdateResolvedTsWithDelayedTable tests redo manager doesn't move resolved +// ts forward if one or more tables resolved ts are not returned from underlying +// writer, this secenario happens when there is no data or resolved ts of this +// table sent to redo log writer yet. +func TestUpdateResolvedTsWithDelayedTable(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + cfg := &config.ConsistentConfig{ + Level: string(consistentLevelEventual), + Storage: "blackhole://", + } + errCh := make(chan error, 1) + opts := &ManagerOptions{ + EnableBgRunner: true, + ErrCh: errCh, + } + logMgr, err := NewManager(ctx, cfg, opts) + require.Nil(t, err) + + var ( + table53 = int64(53) + table55 = int64(55) + table57 = int64(57) + + startTs = uint64(100) + table53Ts = uint64(125) + table55Ts = uint64(120) + table57Ts = uint64(110) + ) + tables := []model.TableID{table53, table55, table57} + for _, tableID := range tables { + logMgr.AddTable(tableID, startTs) + } + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + logMgr.bgWriteLog(ctx, errCh) + }() + + // table 53 has new data, resolved-ts moves forward to 125 + rows := []*model.RowChangedEvent{ + {CommitTs: table53Ts, Table: &model.TableName{TableID: table53}}, + {CommitTs: table53Ts, Table: &model.TableName{TableID: table53}}, + } + err = logMgr.EmitRowChangedEvents(ctx, table53, rows...) + require.Nil(t, err) + require.Eventually(t, func() bool { + tsMap, err := logMgr.writer.GetCurrentResolvedTs(ctx, []int64{table53}) + require.Nil(t, err) + ts, ok := tsMap[table53] + return ok && ts == table53Ts + }, time.Second, time.Millisecond*10) + + // table 55 has no data, but receives resolved-ts event and moves forward to 120 + err = logMgr.FlushLog(ctx, table55, table55Ts) + require.Nil(t, err) + + // get min resolved ts should take each table into consideration + err = logMgr.updateTableResolvedTs(ctx) + require.Nil(t, err) + require.Equal(t, startTs, logMgr.GetMinResolvedTs()) + + // table 57 moves forward, update table resolved ts and check again + err = logMgr.FlushLog(ctx, table57, table57Ts) + require.Nil(t, err) + err = logMgr.updateTableResolvedTs(ctx) + require.Nil(t, err) + require.Equal(t, table57Ts, logMgr.GetMinResolvedTs()) + + cancel() + wg.Wait() +} + // TestLogManagerInOwner tests how redo log manager is used in owner, // where the redo log manager needs to handle DDL event only. func TestLogManagerInOwner(t *testing.T) { @@ -197,3 +274,73 @@ func TestLogManagerInOwner(t *testing.T) { err = logMgr.writer.DeleteAllLogs(ctx) require.Nil(t, err) } + +// TestWriteLogFlushLogSequence tests flush log must be executed after table's +// log has been written to writer. +func TestWriteLogFlushLogSequence(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + cfg := &config.ConsistentConfig{ + Level: string(consistentLevelEventual), + Storage: "blackhole://", + } + errCh := make(chan error, 1) + opts := &ManagerOptions{ + EnableBgRunner: false, + ErrCh: errCh, + } + logMgr, err := NewManager(ctx, cfg, opts) + require.Nil(t, err) + + var ( + wg sync.WaitGroup + + tableID = int64(53) + startTs = uint64(100) + resolvedTs = uint64(150) + ) + logMgr.AddTable(tableID, startTs) + + wg.Add(1) + go func() { + defer wg.Done() + select { + case <-ctx.Done(): + return + case err := <-errCh: + require.Nil(t, err) + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + // FlushLog blocks until bgWriteLog consumes data and close callback chan. + err := logMgr.FlushLog(ctx, tableID, resolvedTs) + require.Nil(t, err) + }() + + // Sleep a short time to ensure `logMgr.FlushLog` is called + time.Sleep(time.Millisecond * 100) + // FlushLog is still ongoing + require.Equal(t, int64(1), atomic.LoadInt64(&logMgr.flushing)) + err = logMgr.updateTableResolvedTs(ctx) + require.Nil(t, err) + require.Equal(t, startTs, logMgr.GetMinResolvedTs()) + + wg.Add(1) + go func() { + defer wg.Done() + logMgr.bgWriteLog(ctx, errCh) + }() + + require.Eventually(t, func() bool { + err = logMgr.updateTableResolvedTs(ctx) + require.Nil(t, err) + return logMgr.GetMinResolvedTs() == resolvedTs + }, time.Second, time.Millisecond*20) + + cancel() + wg.Wait() +} diff --git a/cdc/redo/reader/file_test.go b/cdc/redo/reader/file_test.go index 47c55dbf303..dd04b1f7e7a 100644 --- a/cdc/redo/reader/file_test.go +++ b/cdc/redo/reader/file_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" "golang.org/x/net/context" ) @@ -62,7 +63,10 @@ func TestReaderRead(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - w, err := writer.NewWriter(ctx, cfg) + uuidGen := uuid.NewConstGenerator("const-uuid") + w, err := writer.NewWriter(ctx, cfg, + writer.WithUUIDGenerator(func() uuid.Generator { return uuidGen }), + ) require.Nil(t, err) log := &model.RedoLog{ RedoRow: &model.RedoRowChangedEvent{Row: &model.RowChangedEvent{CommitTs: 1123}}, @@ -75,7 +79,9 @@ func TestReaderRead(t *testing.T) { err = w.Close() require.Nil(t, err) require.True(t, !w.IsRunning()) - fileName := fmt.Sprintf("%s_%s_%d_%s_%d%s", cfg.CaptureID, cfg.ChangeFeedID, cfg.CreateTime.Unix(), cfg.FileType, 11, common.LogEXT) + fileName := fmt.Sprintf(common.RedoLogFileFormatV1, cfg.CaptureID, + cfg.ChangeFeedID, + cfg.FileType, 11, uuidGen.NewString(), common.LogEXT) path := filepath.Join(cfg.Dir, fileName) info, err := os.Stat(path) require.Nil(t, err) @@ -108,7 +114,10 @@ func TestReaderOpenSelectedFiles(t *testing.T) { MaxLogSize: 100000, Dir: dir, } - fileName := fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf", time.Now().Unix(), common.DefaultDDLLogFileType, 11, common.LogEXT+common.TmpEXT) + uuidGen := uuid.NewGenerator() + fileName := fmt.Sprintf(common.RedoLogFileFormatV1, "cp", + "test-cf", common.DefaultDDLLogFileType, 11, + uuidGen.NewString(), common.LogEXT+common.TmpEXT) w, err := writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) @@ -134,13 +143,17 @@ func TestReaderOpenSelectedFiles(t *testing.T) { require.Nil(t, err) // no data, wil not open - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf11", time.Now().Unix(), common.DefaultDDLLogFileType, 10, common.LogEXT) + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, "cp", + "test-cf11", common.DefaultDDLLogFileType, 10, + uuidGen.NewString(), common.LogEXT) path = filepath.Join(dir, fileName) _, err = os.Create(path) require.Nil(t, err) // SortLogEXT, wil open - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf111", time.Now().Unix(), common.DefaultDDLLogFileType, 10, common.LogEXT) + common.SortLogEXT + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, "cp", + "test-cf111", common.DefaultDDLLogFileType, 10, uuidGen.NewString(), + common.LogEXT) + common.SortLogEXT path = filepath.Join(dir, fileName) f1, err := os.Create(path) require.Nil(t, err) @@ -148,7 +161,8 @@ func TestReaderOpenSelectedFiles(t *testing.T) { dir1, err := ioutil.TempDir("", "redo-openSelectedFiles1") require.Nil(t, err) defer os.RemoveAll(dir1) //nolint:errcheck - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf", time.Now().Unix(), common.DefaultDDLLogFileType, 11, common.LogEXT+"test") + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, "cp", "test-cf", + common.DefaultDDLLogFileType, 11, uuidGen.NewString(), common.LogEXT+"test") path = filepath.Join(dir1, fileName) _, err = os.Create(path) require.Nil(t, err) diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index dd963d7cf55..0c637197c92 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -25,6 +25,7 @@ import ( "time" "github.com/golang/mock/gomock" + "github.com/google/uuid" "github.com/pingcap/errors" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" "github.com/pingcap/tidb/br/pkg/storage" @@ -84,7 +85,9 @@ func TestLogReaderResetReader(t *testing.T) { MaxLogSize: 100000, Dir: dir, } - fileName := fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf100", time.Now().Unix(), common.DefaultDDLLogFileType, 100, common.LogEXT) + fileName := fmt.Sprintf(common.RedoLogFileFormatV1, "cp", + "test-cf100", + common.DefaultDDLLogFileType, 100, uuid.New().String(), common.LogEXT) w, err := writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) @@ -103,7 +106,9 @@ func TestLogReaderResetReader(t *testing.T) { f, err := os.Open(path) require.Nil(t, err) - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", "cp", "test-cf10", time.Now().Unix(), common.DefaultRowLogFileType, 10, common.LogEXT) + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, "cp", + "test-cf10", + common.DefaultRowLogFileType, 10, uuid.New().String(), common.LogEXT) w, err = writer.NewWriter(ctx, cfg, writer.WithLogFileName(func() string { return fileName })) diff --git a/cdc/redo/writer/blackhole_writer.go b/cdc/redo/writer/blackhole_writer.go index 6682edef6f3..c2022be3e4f 100644 --- a/cdc/redo/writer/blackhole_writer.go +++ b/cdc/redo/writer/blackhole_writer.go @@ -83,7 +83,9 @@ func (bs *blackHoleWriter) GetCurrentResolvedTs(_ context.Context, tableIDs []in defer bs.tableRtsMu.RUnlock() rtsMap := make(map[int64]uint64, len(bs.tableRtsMap)) for _, tableID := range tableIDs { - rtsMap[tableID] = bs.tableRtsMap[tableID] + if rts, ok := bs.tableRtsMap[tableID]; ok { + rtsMap[tableID] = rts + } } return rtsMap, nil } diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index 0e55d7d89a3..eb846c3c161 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -29,13 +29,15 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" - "github.com/pingcap/tiflow/cdc/redo/common" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/uber-go/atomic" pioutil "go.etcd.io/etcd/pkg/ioutil" "go.uber.org/multierr" "go.uber.org/zap" + + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/uuid" ) const ( @@ -91,7 +93,8 @@ type FileWriterConfig struct { type Option func(writer *writerOptions) type writerOptions struct { - getLogFileName func() string + getLogFileName func() string + getUUIDGenerator func() uuid.Generator } // WithLogFileName provide the Option for fileName @@ -103,6 +106,15 @@ func WithLogFileName(f func() string) Option { } } +// WithUUIDGenerator provides the Option for uuid generator +func WithUUIDGenerator(f func() uuid.Generator) Option { + return func(o *writerOptions) { + if f != nil { + o.getUUIDGenerator = f + } + } +} + // Writer is a redo log event Writer which writes redo log events to a file. type Writer struct { cfg *FileWriterConfig @@ -117,10 +129,13 @@ type Writer struct { gcRunning atomic.Bool size int64 file *os.File - bw *pioutil.PageWriter - uint64buf []byte - storage storage.ExternalStorage + // record the filepath that is being written, and has not been flushed + ongoingFilePath string + bw *pioutil.PageWriter + uint64buf []byte + storage storage.ExternalStorage sync.RWMutex + uuidGenerator uuid.Generator metricFsyncDuration prometheus.Observer metricFlushAllDuration prometheus.Observer @@ -163,6 +178,11 @@ func NewWriter(ctx context.Context, cfg *FileWriterConfig, opts ...Option) (*Wri metricFlushAllDuration: redoFlushAllDurationHistogram.WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), metricWriteBytes: redoWriteBytesGauge.WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), } + if w.op.getUUIDGenerator != nil { + w.uuidGenerator = w.op.getUUIDGenerator() + } else { + w.uuidGenerator = uuid.NewGenerator() + } w.running.Store(true) go w.runFlushToDisk(ctx, cfg.FlushIntervalInMs) @@ -308,7 +328,7 @@ func (w *Writer) close() error { ctx, cancel := context.WithTimeout(context.Background(), defaultS3Timeout) defer cancel() - err = w.renameInS3(ctx, w.file.Name(), w.filePath()) + err = w.renameInS3(ctx, w.file.Name(), w.ongoingFilePath) if err != nil { return cerror.WrapError(cerror.ErrS3StorageAPI, err) } @@ -331,11 +351,18 @@ func (w *Writer) getLogFileName() string { if w.op != nil && w.op.getLogFileName != nil { return w.op.getLogFileName() } - return fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, w.commitTS.Load(), common.LogEXT) + uid := w.uuidGenerator.NewString() + return fmt.Sprintf(common.RedoLogFileFormatV1, + w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.FileType, + w.commitTS.Load(), uid, common.LogEXT) } +// filePath always creates a new, unique file path, note this function is not +// thread-safe, writer needs to ensure lock is acquired when calling it. func (w *Writer) filePath() string { - return filepath.Join(w.cfg.Dir, w.getLogFileName()) + fp := filepath.Join(w.cfg.Dir, w.getLogFileName()) + w.ongoingFilePath = fp + return fp } func openTruncFile(name string) (*os.File, error) { @@ -366,7 +393,10 @@ func (w *Writer) openNew() error { } func (w *Writer) openOrNew(writeLen int) error { - path := w.filePath() + path := w.ongoingFilePath + if path == "" { + return w.openNew() + } info, err := os.Stat(path) if os.IsNotExist(err) { return w.openNew() diff --git a/cdc/redo/writer/file_test.go b/cdc/redo/writer/file_test.go index e22fdf430bf..9527daa07f5 100644 --- a/cdc/redo/writer/file_test.go +++ b/cdc/redo/writer/file_test.go @@ -28,6 +28,7 @@ import ( mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/require" "github.com/uber-go/atomic" ) @@ -47,6 +48,7 @@ func TestWriterWrite(t *testing.T) { require.Nil(t, err) defer os.RemoveAll(dir) + uuidGen := uuid.NewConstGenerator("const-uuid") w := &Writer{ cfg: &FileWriterConfig{ MaxLogSize: 10, @@ -61,13 +63,17 @@ func TestWriterWrite(t *testing.T) { metricWriteBytes: redoWriteBytesGauge.WithLabelValues("cp", "test-cf"), metricFsyncDuration: redoFsyncDurationHistogram.WithLabelValues("cp", "test-cf"), metricFlushAllDuration: redoFlushAllDurationHistogram.WithLabelValues("cp", "test-cf"), + uuidGenerator: uuidGen, } w.eventCommitTS.Store(1) _, err = w.Write([]byte("tes1t11111")) require.Nil(t, err) + var fileName string // create a .tmp file - fileName := fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, 1, common.LogEXT) + common.TmpEXT + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + w.cfg.ChangeFeedID, + w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT path := filepath.Join(w.cfg.Dir, fileName) info, err := os.Stat(path) require.Nil(t, err) @@ -81,13 +87,17 @@ func TestWriterWrite(t *testing.T) { require.Nil(t, err) // after rotate, rename to .log - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, 1, common.LogEXT) + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + w.cfg.ChangeFeedID, + w.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) require.Nil(t, err) require.Equal(t, fileName, info.Name()) // create a .tmp file with first eventCommitTS as name - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, 12, common.LogEXT) + common.TmpEXT + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + w.cfg.ChangeFeedID, + w.cfg.FileType, 12, uuidGen.NewString(), common.LogEXT) + common.TmpEXT path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) require.Nil(t, err) @@ -96,7 +106,9 @@ func TestWriterWrite(t *testing.T) { require.Nil(t, err) require.False(t, w.IsRunning()) // safe close, rename to .log with max eventCommitTS as name - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w.cfg.CaptureID, w.cfg.ChangeFeedID, w.cfg.CreateTime.Unix(), w.cfg.FileType, 22, common.LogEXT) + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w.cfg.CaptureID, + w.cfg.ChangeFeedID, + w.cfg.FileType, 22, uuidGen.NewString(), common.LogEXT) path = filepath.Join(w.cfg.Dir, fileName) info, err = os.Stat(path) require.Nil(t, err) @@ -111,18 +123,24 @@ func TestWriterWrite(t *testing.T) { FileType: common.DefaultRowLogFileType, CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), }, - uint64buf: make([]byte, 8), - running: *atomic.NewBool(true), - metricWriteBytes: redoWriteBytesGauge.WithLabelValues("cp", "test-cf11"), - metricFsyncDuration: redoFsyncDurationHistogram.WithLabelValues("cp", "test-cf11"), - metricFlushAllDuration: redoFlushAllDurationHistogram.WithLabelValues("cp", "test-cf11"), + uint64buf: make([]byte, 8), + running: *atomic.NewBool(true), + metricWriteBytes: redoWriteBytesGauge. + WithLabelValues("default", "test-cf11"), + metricFsyncDuration: redoFsyncDurationHistogram. + WithLabelValues("default", "test-cf11"), + metricFlushAllDuration: redoFlushAllDurationHistogram. + WithLabelValues("default", "test-cf11"), + uuidGenerator: uuidGen, } w1.eventCommitTS.Store(1) _, err = w1.Write([]byte("tes1t11111")) require.Nil(t, err) // create a .tmp file - fileName = fmt.Sprintf("%s_%s_%d_%s_%d%s", w1.cfg.CaptureID, w1.cfg.ChangeFeedID, w1.cfg.CreateTime.Unix(), w1.cfg.FileType, 1, common.LogEXT) + common.TmpEXT + fileName = fmt.Sprintf(common.RedoLogFileFormatV1, w1.cfg.CaptureID, + w1.cfg.ChangeFeedID, + w1.cfg.FileType, 1, uuidGen.NewString(), common.LogEXT) + common.TmpEXT path = filepath.Join(w1.cfg.Dir, fileName) info, err = os.Stat(path) require.Nil(t, err) @@ -141,22 +159,34 @@ func TestWriterGC(t *testing.T) { require.Nil(t, err) defer os.RemoveAll(dir) + uuidGen := uuid.NewConstGenerator("const-uuid") controller := gomock.NewController(t) mockStorage := mockstorage.NewMockExternalStorage(controller) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_1.log.tmp", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_1.log", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_row_1.log.tmp").Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_1_const-uuid.log.tmp", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_1_const-uuid.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_1_const-uuid.log.tmp"). + Return(nil).Times(1) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_2.log.tmp", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_2.log", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_row_2.log.tmp").Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_2_const-uuid.log.tmp", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_2_const-uuid.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_2_const-uuid.log.tmp"). + Return(nil).Times(1) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_3.log.tmp", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_row_3.log", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_row_3.log.tmp").Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_3_const-uuid.log.tmp", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_3_const-uuid.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_3_const-uuid.log.tmp"). + Return(nil).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_row_1.log").Return(errors.New("ignore err")).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_row_2.log").Return(errors.New("ignore err")).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_1_const-uuid.log"). + Return(errors.New("ignore err")).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_2_const-uuid.log"). + Return(errors.New("ignore err")).Times(1) megabyte = 1 cfg := &FileWriterConfig{ @@ -170,12 +200,16 @@ func TestWriterGC(t *testing.T) { S3Storage: true, } w := &Writer{ - cfg: cfg, - uint64buf: make([]byte, 8), - storage: mockStorage, - metricWriteBytes: redoWriteBytesGauge.WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), - metricFsyncDuration: redoFsyncDurationHistogram.WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), - metricFlushAllDuration: redoFlushAllDurationHistogram.WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), + cfg: cfg, + uint64buf: make([]byte, 8), + storage: mockStorage, + metricWriteBytes: redoWriteBytesGauge. + WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), + metricFsyncDuration: redoFsyncDurationHistogram. + WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), + metricFlushAllDuration: redoFlushAllDurationHistogram. + WithLabelValues(cfg.CaptureID, cfg.ChangeFeedID), + uuidGenerator: uuidGen, } w.running.Store(true) w.eventCommitTS.Store(1) @@ -236,11 +270,14 @@ func TestNewWriter(t *testing.T) { require.Nil(t, err) defer os.RemoveAll(dir) + uuidGen := uuid.NewConstGenerator("const-uuid") w, err := NewWriter(context.Background(), &FileWriterConfig{ Dir: "sdfsf", S3Storage: true, S3URI: *s3URI, - }) + }, + WithUUIDGenerator(func() uuid.Generator { return uuidGen }), + ) require.Nil(t, err) time.Sleep(time.Duration(defaultFlushIntervalInMs+1) * time.Millisecond) err = w.Close() @@ -249,9 +286,12 @@ func TestNewWriter(t *testing.T) { controller := gomock.NewController(t) mockStorage := mockstorage.NewMockExternalStorage(controller) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_ddl_0.log.tmp", gomock.Any()).Return(nil).Times(2) - mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_946688461_ddl_0.log", gomock.Any()).Return(nil).Times(1) - mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_946688461_ddl_0.log.tmp").Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_ddl_0_const-uuid.log.tmp", + gomock.Any()).Return(nil).Times(2) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_ddl_0_const-uuid.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_ddl_0_const-uuid.log.tmp"). + Return(nil).Times(1) w = &Writer{ cfg: &FileWriterConfig{ @@ -263,11 +303,15 @@ func TestNewWriter(t *testing.T) { S3Storage: true, MaxLogSize: defaultMaxLogSize, }, - uint64buf: make([]byte, 8), - storage: mockStorage, - metricWriteBytes: redoWriteBytesGauge.WithLabelValues("cp", "test"), - metricFsyncDuration: redoFsyncDurationHistogram.WithLabelValues("cp", "test"), - metricFlushAllDuration: redoFlushAllDurationHistogram.WithLabelValues("cp", "test"), + uint64buf: make([]byte, 8), + storage: mockStorage, + metricWriteBytes: redoWriteBytesGauge. + WithLabelValues("default", "test"), + metricFsyncDuration: redoFsyncDurationHistogram. + WithLabelValues("default", "test"), + metricFlushAllDuration: redoFlushAllDurationHistogram. + WithLabelValues("default", "test"), + uuidGenerator: uuidGen, } w.running.Store(true) _, err = w.Write([]byte("test")) @@ -281,3 +325,68 @@ func TestNewWriter(t *testing.T) { require.Equal(t, w.running.Load(), false) time.Sleep(time.Duration(defaultFlushIntervalInMs+1) * time.Millisecond) } + +func TestRotateFile(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + _, err := NewWriter(ctx, nil) + require.NotNil(t, err) + + controller := gomock.NewController(t) + mockStorage := mockstorage.NewMockExternalStorage(controller) + + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_0_uuid-1.log.tmp", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_0_uuid-2.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_0_uuid-1.log.tmp"). + Return(nil).Times(1) + + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_0_uuid-3.log.tmp", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().WriteFile(gomock.Any(), "cp_test_row_100_uuid-4.log", + gomock.Any()).Return(nil).Times(1) + mockStorage.EXPECT().DeleteFile(gomock.Any(), "cp_test_row_0_uuid-3.log.tmp"). + Return(nil).Times(1) + + dir := t.TempDir() + uuidGen := uuid.NewMock() + uuidGen.Push("uuid-1") + uuidGen.Push("uuid-2") + uuidGen.Push("uuid-3") + uuidGen.Push("uuid-4") + uuidGen.Push("uuid-5") + w := &Writer{ + cfg: &FileWriterConfig{ + Dir: dir, + CaptureID: "cp", + ChangeFeedID: "test", + FileType: common.DefaultRowLogFileType, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + S3Storage: true, + MaxLogSize: defaultMaxLogSize, + }, + uint64buf: make([]byte, 8), + metricWriteBytes: redoWriteBytesGauge. + WithLabelValues("default", "test"), + metricFsyncDuration: redoFsyncDurationHistogram. + WithLabelValues("default", "test"), + metricFlushAllDuration: redoFlushAllDurationHistogram. + WithLabelValues("default", "test"), + storage: mockStorage, + uuidGenerator: uuidGen, + } + + w.running.Store(true) + _, err = w.Write([]byte("test")) + require.Nil(t, err) + + err = w.rotate() + require.Nil(t, err) + + w.AdvanceTs(100) + _, err = w.Write([]byte("test")) + require.Nil(t, err) + err = w.rotate() + require.Nil(t, err) +} diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index 5c104b4040d..a2ffc7b18c8 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -108,7 +108,9 @@ type LogWriter struct { } // NewLogWriter creates a LogWriter instance. It is guaranteed only one LogWriter per changefeed -func NewLogWriter(ctx context.Context, cfg *LogWriterConfig) (*LogWriter, error) { +func NewLogWriter( + ctx context.Context, cfg *LogWriterConfig, opts ...Option, +) (*LogWriter, error) { if cfg == nil { return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("LogWriterConfig can not be nil")) } @@ -150,11 +152,11 @@ func NewLogWriter(ctx context.Context, cfg *LogWriterConfig) (*LogWriter, error) logWriter = &LogWriter{ cfg: cfg, } - logWriter.rowWriter, err = NewWriter(ctx, rowCfg) + logWriter.rowWriter, err = NewWriter(ctx, rowCfg, opts...) if err != nil { return nil, err } - logWriter.ddlWriter, err = NewWriter(ctx, ddlCfg) + logWriter.ddlWriter, err = NewWriter(ctx, ddlCfg, opts...) if err != nil { return nil, err } diff --git a/cdc/redo/writer/writer_test.go b/cdc/redo/writer/writer_test.go index d071ca30474..15c59f1e4b8 100644 --- a/cdc/redo/writer/writer_test.go +++ b/cdc/redo/writer/writer_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/multierr" @@ -618,7 +619,10 @@ func TestNewLogWriter(t *testing.T) { CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), FlushIntervalInMs: 5, } - ll, err := NewLogWriter(ctx, cfg) + uuidGen := uuid.NewConstGenerator("const-uuid") + ll, err := NewLogWriter(ctx, cfg, + WithUUIDGenerator(func() uuid.Generator { return uuidGen }), + ) require.Nil(t, err) time.Sleep(time.Duration(defaultGCIntervalInMs+1) * time.Millisecond) require.Equal(t, map[int64]uint64{}, ll.meta.ResolvedTsList) diff --git a/cdc/server.go b/cdc/server.go index 9decacdbd77..5fc30c6673d 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -119,7 +119,6 @@ func (s *Server) Run(ctx context.Context) error { LogConfig: &logConfig, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpcTLSOption, grpc.WithBlock(), grpc.WithConnectParams(grpc.ConnectParams{ Backoff: backoff.Config{ diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index 2e424fb3bb4..cf8c92badb5 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -35,6 +35,11 @@ type blackHoleSink struct { lastAccumulated uint64 } +// Init table sink resources +func (b *blackHoleSink) Init(tableID model.TableID) error { + return nil +} + func (b *blackHoleSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { for _, row := range rows { log.Debug("BlockHoleSink: EmitRowChangedEvents", zap.Any("row", row)) diff --git a/cdc/sink/buffer_sink.go b/cdc/sink/buffer_sink.go index 0b19ba96763..048ecf4dadc 100644 --- a/cdc/sink/buffer_sink.go +++ b/cdc/sink/buffer_sink.go @@ -34,7 +34,6 @@ type bufferSink struct { buffer map[model.TableID][]*model.RowChangedEvent bufferMu sync.Mutex flushTsChan chan flushMsg - drawbackChan chan drawbackMsg } func newBufferSink( @@ -42,7 +41,6 @@ func newBufferSink( backendSink Sink, errCh chan error, checkpointTs model.Ts, - drawbackChan chan drawbackMsg, ) *bufferSink { sink := &bufferSink{ Sink: backendSink, @@ -50,7 +48,6 @@ func newBufferSink( buffer: make(map[model.TableID][]*model.RowChangedEvent), changeFeedCheckpointTs: checkpointTs, flushTsChan: make(chan flushMsg, 128), - drawbackChan: drawbackChan, } go sink.run(ctx, errCh) return sink @@ -77,11 +74,6 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { errCh <- err } return - case drawback := <-b.drawbackChan: - b.bufferMu.Lock() - delete(b.buffer, drawback.tableID) - b.bufferMu.Unlock() - close(drawback.callback) case flushEvent := <-b.flushTsChan: b.bufferMu.Lock() resolvedTs := flushEvent.resolvedTs @@ -133,6 +125,30 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { } } +// Init table sink resources +func (b *bufferSink) Init(tableID model.TableID) error { + b.clearBufferedTableData(tableID) + return b.Sink.Init(tableID) +} + +// Barrier delete buffer +func (b *bufferSink) Barrier(ctx context.Context, tableID model.TableID) error { + b.clearBufferedTableData(tableID) + return b.Sink.Barrier(ctx, tableID) +} + +func (b *bufferSink) clearBufferedTableData(tableID model.TableID) { + b.bufferMu.Lock() + defer b.bufferMu.Unlock() + delete(b.buffer, tableID) + checkpointTs, loaded := b.tableCheckpointTsMap.LoadAndDelete(tableID) + if loaded { + log.Info("clean up table checkpoint ts in buffer sink", + zap.Int64("tableID", tableID), + zap.Uint64("checkpointTs", checkpointTs.(uint64))) + } +} + func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { select { case <-ctx.Done(): diff --git a/cdc/sink/buffer_sink_test.go b/cdc/sink/buffer_sink_test.go index b529f2a416e..f3a50824f88 100644 --- a/cdc/sink/buffer_sink_test.go +++ b/cdc/sink/buffer_sink_test.go @@ -32,7 +32,7 @@ func TestTableIsNotFlushed(t *testing.T) { func TestFlushTable(t *testing.T) { ctx, cancel := context.WithCancel(context.TODO()) defer cancel() - b := newBufferSink(ctx, newBlackHoleSink(ctx), make(chan error), 5, make(chan drawbackMsg)) + b := newBufferSink(ctx, newBlackHoleSink(ctx), make(chan error), 5) require.Equal(t, uint64(5), b.getTableCheckpointTs(2)) require.Nil(t, b.EmitRowChangedEvents(ctx)) @@ -73,7 +73,7 @@ func TestFlushTable(t *testing.T) { func TestFlushFailed(t *testing.T) { ctx, cancel := context.WithCancel(context.TODO()) - b := newBufferSink(ctx, newBlackHoleSink(ctx), make(chan error), 5, make(chan drawbackMsg)) + b := newBufferSink(ctx, newBlackHoleSink(ctx), make(chan error), 5) checkpoint, err := b.FlushRowChangedEvents(ctx, 3, 8) require.True(t, checkpoint <= 8) @@ -89,3 +89,18 @@ func TestFlushFailed(t *testing.T) { require.Equal(t, uint64(8), b.getTableCheckpointTs(3)) require.Equal(t, uint64(5), b.getTableCheckpointTs(1)) } + +func TestCleanBufferedData(t *testing.T) { + t.Parallel() + + tblID := model.TableID(1) + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + b := newBufferSink(ctx, newBlackHoleSink(ctx), make(chan error), 5) + b.buffer[tblID] = []*model.RowChangedEvent{} + _, ok := b.buffer[tblID] + require.True(t, ok) + require.Nil(t, b.Init(tblID)) + _, ok = b.buffer[tblID] + require.False(t, ok) +} diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go index 23367e98b29..1addf1aea4f 100644 --- a/cdc/sink/cdclog/file.go +++ b/cdc/sink/cdclog/file.go @@ -192,6 +192,10 @@ type fileSink struct { ddlEncoder codec.EventBatchEncoder } +func (f *fileSink) Init(_ model.TableID) error { + return nil +} + func (f *fileSink) flushLogMeta() error { data, err := f.logMeta.Marshal() if err != nil { diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go index dc5b26b2791..dc0e8da0892 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -210,6 +210,10 @@ type s3Sink struct { ddlEncoder codec.EventBatchEncoder } +func (s *s3Sink) Init(_ model.TableID) error { + return nil +} + func (s *s3Sink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { return s.emitRowChangedEvents(ctx, newTableBuffer, rows...) } diff --git a/cdc/sink/common/common.go b/cdc/sink/common/common.go index 7fff1ec0082..702f7134bba 100644 --- a/cdc/sink/common/common.go +++ b/cdc/sink/common/common.go @@ -63,6 +63,13 @@ func NewUnresolvedTxnCache() *UnresolvedTxnCache { } } +// RemoveTableTxn removes unresolved rows from cache +func (c *UnresolvedTxnCache) RemoveTableTxn(tableID model.TableID) { + c.unresolvedTxnsMu.Lock() + defer c.unresolvedTxnsMu.Unlock() + delete(c.unresolvedTxns, tableID) +} + // Append adds unresolved rows to cache // the rows inputed into this function will go through the following handling logic // 1. group by tableID from one input stream diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index f210b3db9e7..14e26bbbec5 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -43,8 +43,6 @@ type Manager struct { flushMu sync.Mutex flushing int64 - drawbackChan chan drawbackMsg - captureAddr string changefeedID model.ChangeFeedID metricsTableSinkTotalRows prometheus.Counter @@ -55,12 +53,10 @@ func NewManager( ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts, captureAddr string, changefeedID model.ChangeFeedID, ) *Manager { - drawbackChan := make(chan drawbackMsg, 16) return &Manager{ - backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs, drawbackChan), + backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs), changeFeedCheckpointTs: checkpointTs, tableSinks: make(map[model.TableID]*tableSink), - drawbackChan: drawbackChan, captureAddr: captureAddr, changefeedID: changefeedID, metricsTableSinkTotalRows: tableSinkTotalRowsCountCounter.WithLabelValues(captureAddr, changefeedID), @@ -68,12 +64,11 @@ func NewManager( } // CreateTableSink creates a table sink -func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts, redoManager redo.LogManager) Sink { - m.tableSinksMu.Lock() - defer m.tableSinksMu.Unlock() - if _, exist := m.tableSinks[tableID]; exist { - log.Panic("the table sink already exists", zap.Uint64("tableID", uint64(tableID))) - } +func (m *Manager) CreateTableSink( + tableID model.TableID, + checkpointTs model.Ts, + redoManager redo.LogManager, +) (Sink, error) { sink := &tableSink{ tableID: tableID, manager: m, @@ -81,8 +76,17 @@ func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts, emittedTs: checkpointTs, redoManager: redoManager, } + + m.tableSinksMu.Lock() + defer m.tableSinksMu.Unlock() + if _, exist := m.tableSinks[tableID]; exist { + log.Panic("the table sink already exists", zap.Uint64("tableID", uint64(tableID))) + } + if err := sink.Init(tableID); err != nil { + return nil, errors.Trace(err) + } m.tableSinks[tableID] = sink - return sink + return sink, nil } // Close closes the Sink manager and backend Sink, this method can be reentrantly called @@ -149,17 +153,6 @@ func (m *Manager) destroyTableSink(ctx context.Context, tableID model.TableID) e m.tableSinksMu.Lock() delete(m.tableSinks, tableID) m.tableSinksMu.Unlock() - callback := make(chan struct{}) - select { - case <-ctx.Done(): - return ctx.Err() - case m.drawbackChan <- drawbackMsg{tableID: tableID, callback: callback}: - } - select { - case <-ctx.Done(): - return ctx.Err() - case <-callback: - } return m.backendSink.Barrier(ctx, tableID) } @@ -180,8 +173,3 @@ func (m *Manager) UpdateChangeFeedCheckpointTs(checkpointTs uint64) { m.backendSink.UpdateChangeFeedCheckpointTs(checkpointTs) } } - -type drawbackMsg struct { - tableID model.TableID - callback chan struct{} -} diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 7e901df53de..5d16cbd05cf 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -48,6 +48,11 @@ func newCheckSink(c *check.C) *checkSink { } } +// Init table sink resources +func (c *checkSink) Init(tableID model.TableID) error { + return nil +} + func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { c.rowsMu.Lock() defer c.rowsMu.Unlock() @@ -110,7 +115,10 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { wg.Add(1) go func() { defer wg.Done() - tableSinks[i] = manager.CreateTableSink(model.TableID(i), 0, redo.NewDisabledManager()) + var err error + tableSinks[i], err = manager.CreateTableSink(model.TableID(i), + 0, redo.NewDisabledManager()) + c.Assert(err, check.IsNil) }() } wg.Wait() @@ -203,7 +211,8 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { for i := 0; i < goroutineNum; i++ { if i%4 != 3 { // add table - table := manager.CreateTableSink(model.TableID(i), maxResolvedTs, redoManager) + table, err := manager.CreateTableSink(model.TableID(i), maxResolvedTs, redoManager) + c.Assert(err, check.IsNil) ctx, cancel := context.WithCancel(ctx) tableCancels = append(tableCancels, cancel) tableSinks = append(tableSinks, table) @@ -245,9 +254,11 @@ func (s *managerSuite) TestManagerDestroyTableSink(c *check.C) { defer manager.Close(ctx) tableID := int64(49) - tableSink := manager.CreateTableSink(tableID, 100, redo.NewDisabledManager()) - err := tableSink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ - Table: &model.TableName{TableID: tableID}, + table := &model.TableName{TableID: int64(49)} + tableSink, err := manager.CreateTableSink(table.TableID, 100, redo.NewDisabledManager()) + c.Assert(err, check.IsNil) + err = tableSink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + Table: table, CommitTs: uint64(110), }) c.Assert(err, check.IsNil) @@ -274,7 +285,10 @@ func BenchmarkManagerFlushing(b *testing.B) { wg.Add(1) go func() { defer wg.Done() - tableSinks[i] = manager.CreateTableSink(model.TableID(i), 0, redo.NewDisabledManager()) + var err error + tableSinks[i], err = manager.CreateTableSink(model.TableID(i), + 0, redo.NewDisabledManager()) + panic(err) }() } wg.Wait() @@ -340,6 +354,10 @@ type errorSink struct { *check.C } +func (e *errorSink) Init(_ model.TableID) error { + return nil +} + func (e *errorSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { return errors.New("error in emit row changed events") } @@ -371,8 +389,9 @@ func (s *managerSuite) TestManagerError(c *check.C) { errCh := make(chan error, 16) manager := NewManager(ctx, &errorSink{C: c}, errCh, 0, "", "") defer manager.Close(ctx) - sink := manager.CreateTableSink(1, 0, redo.NewDisabledManager()) - err := sink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + sink, err := manager.CreateTableSink(1, 0, redo.NewDisabledManager()) + c.Assert(err, check.IsNil) + err = sink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ CommitTs: 1, Table: &model.TableName{TableID: 1}, }) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 4fff5ee6081..4f9795a19c1 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -18,6 +18,7 @@ import ( "fmt" "net/url" "strings" + "sync" "sync/atomic" "time" @@ -53,12 +54,12 @@ type mqSink struct { filter *filter.Filter protocol codec.Protocol - partitionNum int32 - partitionInput []chan mqEvent - partitionResolvedTs []uint64 - tableCheckpointTs map[model.TableID]uint64 - resolvedNotifier *notify.Notifier - resolvedReceiver *notify.Receiver + partitionNum int32 + partitionInput []chan mqEvent + partitionResolvedTs []uint64 + tableCheckpointTsMap sync.Map + resolvedNotifier *notify.Notifier + resolvedReceiver *notify.Receiver statistics *Statistics @@ -117,7 +118,6 @@ func newMqSink( partitionNum: partitionNum, partitionInput: partitionInput, partitionResolvedTs: make([]uint64, partitionNum), - tableCheckpointTs: make(map[model.TableID]uint64), resolvedNotifier: notifier, resolvedReceiver: resolvedReceiver, @@ -168,7 +168,12 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha } func (k *mqSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { - if checkpointTs, ok := k.tableCheckpointTs[tableID]; ok && resolvedTs <= checkpointTs { + var checkpointTs uint64 + v, ok := k.tableCheckpointTsMap.Load(tableID) + if ok { + checkpointTs = v.(uint64) + } + if resolvedTs <= checkpointTs { return checkpointTs, nil } @@ -202,7 +207,7 @@ flushLoop: if err != nil { return 0, errors.Trace(err) } - k.tableCheckpointTs[tableID] = resolvedTs + k.tableCheckpointTsMap.Store(tableID, resolvedTs) k.statistics.PrintStatus(ctx) return resolvedTs, nil } @@ -255,6 +260,20 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { return errors.Trace(err) } +func (k *mqSink) Init(tableID model.TableID) error { + // We need to clean up the old values of the table, + // otherwise when the table is dispatched back again, + // it may read the old values. + // See: https://github.com/pingcap/tiflow/issues/4464#issuecomment-1085385382. + if checkpointTs, loaded := k.tableCheckpointTsMap.LoadAndDelete(tableID); loaded { + log.Info("clean up table checkpoint ts in MQ sink", + zap.Int64("tableID", tableID), + zap.Uint64("checkpointTs", checkpointTs.(uint64))) + } + + return nil +} + func (k *mqSink) Close(ctx context.Context) error { err := k.mqProducer.Close() return errors.Trace(err) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 1dc0fec8102..e6b82dfc8b1 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -866,6 +866,30 @@ func (w *mysqlSinkWorker) cleanup() { } } +func (s *mysqlSink) Init(tableID model.TableID) error { + s.cleanTableResource(tableID) + return nil +} + +func (s *mysqlSink) cleanTableResource(tableID model.TableID) { + // We need to clean up the old values of the table, + // otherwise when the table is dispatched back again, + // it may read the old values. + // See: https://github.com/pingcap/tiflow/issues/4464#issuecomment-1085385382. + if resolvedTs, loaded := s.tableMaxResolvedTs.LoadAndDelete(tableID); loaded { + log.Info("clean up table max resolved ts", + zap.Int64("tableID", tableID), + zap.Uint64("resolvedTs", resolvedTs.(uint64))) + } + if checkpointTs, loaded := s.tableCheckpointTs.LoadAndDelete(tableID); loaded { + log.Info("clean up table checkpoint ts", + zap.Int64("tableID", tableID), + zap.Uint64("checkpointTs", checkpointTs.(uint64))) + } + // try to remove table txn cache + s.txnCache.RemoveTableTxn(tableID) +} + func (s *mysqlSink) Close(ctx context.Context) error { s.execWaitNotifier.Close() s.resolvedNotifier.Close() @@ -875,6 +899,7 @@ func (s *mysqlSink) Close(ctx context.Context) error { } func (s *mysqlSink) Barrier(ctx context.Context, tableID model.TableID) error { + defer s.cleanTableResource(tableID) warnDuration := 3 * time.Minute ticker := time.NewTicker(warnDuration) defer ticker.Stop() diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index d9e8f25fcbf..e9b8dfc261d 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -1210,3 +1210,32 @@ func TestMySQLSinkFlushResovledTs(t *testing.T) { err = sink.Close(ctx) require.Nil(t, err) } + +func TestCleanTableResource(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + tblID := model.TableID(1) + f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) + require.Nil(t, err) + s := &mysqlSink{ + txnCache: common.NewUnresolvedTxnCache(), + filter: f, + statistics: NewStatistics(ctx, "db"), + } + require.Nil(t, s.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + Table: &model.TableName{TableID: tblID, Schema: "test", Table: "t1"}, + })) + s.tableCheckpointTs.Store(tblID, uint64(1)) + s.tableMaxResolvedTs.Store(tblID, uint64(2)) + require.Nil(t, s.Init(tblID)) + m := &sync.Map{} + m.Store(tblID, uint64(10)) + ret, _ := s.txnCache.Resolved(m) + require.True(t, len(ret) == 0) + _, ok := s.tableCheckpointTs.Load(tblID) + require.False(t, ok) + _, ok = s.tableMaxResolvedTs.Load(tblID) + require.False(t, ok) +} diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 39b7af6b080..7db3b59671b 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -106,6 +106,11 @@ func newSimpleMySQLSink(ctx context.Context, sinkURI *url.URL, config *config.Re return sink, nil } +// Init table sink resources +func (s *simpleMySQLSink) Init(tableID model.TableID) error { + return nil +} + // EmitRowChangedEvents sends Row Changed Event to Sink // EmitRowChangedEvents may write rows to downstream directly; func (s *simpleMySQLSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index c9e7addd7ab..055beb14bbf 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -34,6 +34,11 @@ const ( // Sink is an abstraction for anything that a changefeed may emit into. type Sink interface { + // Init initializes the sink resource + // when the sink is added, this function will be called + // init resource or clean up the old values in this function + Init(tableID model.TableID) error + // EmitRowChangedEvents sends Row Changed Event to Sink // EmitRowChangedEvents may write rows to downstream directly; EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 48858f525b0..dd33d663547 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -134,6 +134,11 @@ func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { return nil } +// Init table sink resources +func (t *tableSink) Init(tableID model.TableID) error { + return t.manager.backendSink.Init(tableID) +} + // Close once the method is called, no more events can be written to this table sink func (t *tableSink) Close(ctx context.Context) error { return t.manager.destroyTableSink(ctx, t.tableID) diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index c81497876da..1e97991f2c7 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -336,6 +336,7 @@ ErrSyncerReplaceEventNotExist,[code=36066:class=sync-unit:scope=internal:level=h ErrSyncerParseDDL,[code=36067:class=sync-unit:scope=internal:level=high], "Message: parse DDL: %s, Workaround: Please confirm your DDL statement is correct and needed. For TiDB compatible DDL, see https://docs.pingcap.com/tidb/stable/mysql-compatibility#ddl. You can use `handle-error` command to skip or replace the DDL or add a binlog filter rule to ignore it if the DDL is not needed." ErrSyncerUnsupportedStmt,[code=36068:class=sync-unit:scope=internal:level=high], "Message: `%s` statement not supported in %s mode" ErrSyncerGetEvent,[code=36069:class=sync-unit:scope=upstream:level=high], "Message: get binlog event error: %v, Workaround: Please check if the binlog file could be parsed by `mysqlbinlog`." +ErrSyncerDownstreamTableNotFound,[code=36070:class=sync-unit:scope=internal:level=high], "Message: downstream table %s not found" ErrMasterSQLOpNilRequest,[code=38001:class=dm-master:scope=internal:level=medium], "Message: nil request not valid" ErrMasterSQLOpNotSupport,[code=38002:class=dm-master:scope=internal:level=medium], "Message: op %s not supported" ErrMasterSQLOpWithoutSharding,[code=38003:class=dm-master:scope=internal:level=medium], "Message: operate request without --sharding specified not valid" diff --git a/dm/dm/config/source_config.go b/dm/dm/config/source_config.go index 448ea3c3898..c1d4be81651 100644 --- a/dm/dm/config/source_config.go +++ b/dm/dm/config/source_config.go @@ -286,7 +286,7 @@ func (c *SourceConfig) Adjust(ctx context.Context, db *sql.DB) (err error) { log.L().Warn("using an absolute relay path, relay log can't work when starting multiple relay worker") } - return c.AdjustCaseSensitive(ctx2, db) + return nil } // AdjustCaseSensitive adjust CaseSensitive from DB. diff --git a/dm/dm/master/bootstrap_test.go b/dm/dm/master/bootstrap_test.go index d4daaec6bd5..4b19a1e694e 100644 --- a/dm/dm/master/bootstrap_test.go +++ b/dm/dm/master/bootstrap_test.go @@ -78,7 +78,7 @@ func (t *testMaster) TestCollectSourceConfigFilesV1Import(c *C) { cfg1.From.User = user cfg1.From.Password = password cfg1.RelayDir = "relay-dir" - c.Assert(checkAndAdjustSourceConfigFunc(ctx, cfg1), IsNil) // adjust source config. + c.Assert(checkAndAdjustSourceConfigForDMCtlFunc(ctx, cfg1), IsNil) // adjust source config. cfg2 := cfg1.Clone() cfg2.SourceID = "mysql-replica-02" diff --git a/dm/dm/master/openapi.go b/dm/dm/master/openapi.go index 66ae60fd39b..74f5c786446 100644 --- a/dm/dm/master/openapi.go +++ b/dm/dm/master/openapi.go @@ -17,9 +17,12 @@ package master import ( "context" + "crypto/tls" "encoding/json" - "fmt" "net/http" + "net/http/httputil" + + "github.com/pingcap/failpoint" "github.com/deepmap/oapi-codegen/pkg/middleware" "github.com/labstack/echo/v4" @@ -45,7 +48,7 @@ const ( // redirectRequestToLeaderMW a middleware auto redirect request to leader. // because the leader has some data in memory, only the leader can process the request. -func (s *Server) redirectRequestToLeaderMW() echo.MiddlewareFunc { +func (s *Server) reverseRequestToLeaderMW(tlsCfg *tls.Config) echo.MiddlewareFunc { return func(next echo.HandlerFunc) echo.HandlerFunc { return func(ctx echo.Context) error { ctx2 := ctx.Request().Context() @@ -58,13 +61,36 @@ func (s *Server) redirectRequestToLeaderMW() echo.MiddlewareFunc { if err != nil { return err } - return ctx.Redirect(http.StatusTemporaryRedirect, fmt.Sprintf("http://%s%s", leaderOpenAPIAddr, ctx.Request().RequestURI)) + + failpoint.Inject("MockNotSetTls", func() { + tlsCfg = nil + }) + // simpleProxy just reverses to leader host + simpleProxy := httputil.ReverseProxy{ + Director: func(req *http.Request) { + if tlsCfg != nil { + req.URL.Scheme = "https" + } else { + req.URL.Scheme = "http" + } + req.URL.Host = leaderOpenAPIAddr + req.Host = leaderOpenAPIAddr + }, + } + if tlsCfg != nil { + transport := http.DefaultTransport.(*http.Transport).Clone() + transport.TLSClientConfig = tlsCfg + simpleProxy.Transport = transport + } + log.L().Info("reverse request to leader", zap.String("Request URL", ctx.Request().URL.String()), zap.String("leader", leaderOpenAPIAddr), zap.Bool("hasTLS", tlsCfg != nil)) + simpleProxy.ServeHTTP(ctx.Response(), ctx.Request()) + return nil } } } // InitOpenAPIHandles init openapi handlers. -func (s *Server) InitOpenAPIHandles() error { +func (s *Server) InitOpenAPIHandles(tlsCfg *tls.Config) error { swagger, err := openapi.GetSwagger() if err != nil { return err @@ -77,7 +103,7 @@ func (s *Server) InitOpenAPIHandles() error { // set logger e.Use(openapi.ZapLogger(logger)) e.Use(echomiddleware.Recover()) - e.Use(s.redirectRequestToLeaderMW()) + e.Use(s.reverseRequestToLeaderMW(tlsCfg)) // disables swagger server name validation. it seems to work poorly swagger.Servers = nil // use our validation middleware to check all requests against the OpenAPI schema. diff --git a/dm/dm/master/openapi_test.go b/dm/dm/master/openapi_test.go index e0cbc2c22b4..4c63ee1e06f 100644 --- a/dm/dm/master/openapi_test.go +++ b/dm/dm/master/openapi_test.go @@ -19,9 +19,12 @@ import ( "context" "fmt" "net/http" + "os" "testing" "time" + "github.com/stretchr/testify/require" + "github.com/DATA-DOG/go-sqlmock" "github.com/deepmap/oapi-codegen/pkg/testutil" "github.com/golang/mock/gomock" @@ -75,7 +78,7 @@ func (t *openAPISuite) SetUpTest(c *check.C) { c.Assert(ha.ClearTestInfoOperation(t.etcdTestCli), check.IsNil) } -func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { +func (t *openAPISuite) TestReverseRequestToLeader(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -134,9 +137,103 @@ func (t *openAPISuite) TestRedirectRequestToLeader(c *check.C) { c.Assert(resultListSource.Data, check.HasLen, 0) c.Assert(resultListSource.Total, check.Equals, 0) - // list source not from leader will get a redirect + // list source from non-leader will get result too result2 := testutil.NewRequest().Get(baseURL).Go(t.testT, s2.echo) - c.Assert(result2.Code(), check.Equals, http.StatusTemporaryRedirect) + c.Assert(result2.Code(), check.Equals, http.StatusOK) + var resultListSource2 openapi.GetSourceListResponse + err = result2.UnmarshalBodyToObject(&resultListSource2) + c.Assert(err, check.IsNil) + c.Assert(resultListSource2.Data, check.HasLen, 0) + c.Assert(resultListSource2.Total, check.Equals, 0) +} + +func (t *openAPISuite) TestReverseRequestToHttpsLeader(c *check.C) { + pwd, err := os.Getwd() + require.NoError(t.testT, err) + caPath := pwd + "/tls_for_test/ca.pem" + certPath := pwd + "/tls_for_test/dm.pem" + keyPath := pwd + "/tls_for_test/dm.key" + + // master1 + masterAddr1 := tempurl.Alloc()[len("http://"):] + peerAddr1 := tempurl.Alloc()[len("http://"):] + cfg1 := NewConfig() + require.NoError(t.testT, cfg1.Parse([]string{ + "--name=dm-master-tls-1", + fmt.Sprintf("--data-dir=%s", t.testT.TempDir()), + fmt.Sprintf("--master-addr=https://%s", masterAddr1), + fmt.Sprintf("--advertise-addr=https://%s", masterAddr1), + fmt.Sprintf("--peer-urls=https://%s", peerAddr1), + fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr1), + fmt.Sprintf("--initial-cluster=dm-master-tls-1=https://%s", peerAddr1), + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, + })) + cfg1.ExperimentalFeatures.OpenAPI = true + s1 := NewServer(cfg1) + ctx1, cancel1 := context.WithCancel(context.Background()) + require.NoError(t.testT, s1.Start(ctx1)) + defer func() { + cancel1() + s1.Close() + }() + // wait the first one become the leader + require.True(t.testT, utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return s1.election.IsLeader() && s1.scheduler.Started() + })) + + // master2 + masterAddr2 := tempurl.Alloc()[len("http://"):] + peerAddr2 := tempurl.Alloc()[len("http://"):] + cfg2 := NewConfig() + require.NoError(t.testT, cfg2.Parse([]string{ + "--name=dm-master-tls-2", + fmt.Sprintf("--data-dir=%s", t.testT.TempDir()), + fmt.Sprintf("--master-addr=https://%s", masterAddr2), + fmt.Sprintf("--advertise-addr=https://%s", masterAddr2), + fmt.Sprintf("--peer-urls=https://%s", peerAddr2), + fmt.Sprintf("--advertise-peer-urls=https://%s", peerAddr2), + "--ssl-ca=" + caPath, + "--ssl-cert=" + certPath, + "--ssl-key=" + keyPath, + })) + cfg2.ExperimentalFeatures.OpenAPI = true + cfg2.Join = s1.cfg.MasterAddr // join to an existing cluster + s2 := NewServer(cfg2) + ctx2, cancel2 := context.WithCancel(context.Background()) + require.NoError(t.testT, s2.Start(ctx2)) + defer func() { + cancel2() + s2.Close() + }() + // wait the second master ready + require.False(t.testT, utils.WaitSomething(30, 100*time.Millisecond, func() bool { + return s2.election.IsLeader() + })) + + baseURL := "/api/v1/sources" + // list source from leader + result := testutil.NewRequest().Get(baseURL).Go(t.testT, s1.echo) + require.Equal(t.testT, http.StatusOK, result.Code()) + var resultListSource openapi.GetSourceListResponse + require.NoError(t.testT, result.UnmarshalBodyToObject(&resultListSource)) + require.Len(t.testT, resultListSource.Data, 0) + require.Equal(t.testT, 0, resultListSource.Total) + + // with tls, list source not from leader will get result too + result = testutil.NewRequest().Get(baseURL).Go(t.testT, s2.echo) + require.Equal(t.testT, http.StatusOK, result.Code()) + var resultListSource2 openapi.GetSourceListResponse + require.NoError(t.testT, result.UnmarshalBodyToObject(&resultListSource2)) + require.Len(t.testT, resultListSource2.Data, 0) + require.Equal(t.testT, 0, resultListSource2.Total) + + // without tls, list source not from leader will be 502 + require.NoError(t.testT, failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/MockNotSetTls", `return()`)) + result = testutil.NewRequest().Get(baseURL).Go(t.testT, s2.echo) + require.Equal(t.testT, http.StatusBadGateway, result.Code()) + require.NoError(t.testT, failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/MockNotSetTls")) } func (t *openAPISuite) TestOpenAPIWillNotStartInDefaultConfig(c *check.C) { diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 42309798856..01588d3b9bb 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -15,6 +15,7 @@ package master import ( "context" + "database/sql" "fmt" "net" "net/http" @@ -81,7 +82,11 @@ var ( registerOnce sync.Once runBackgroundOnce sync.Once - checkAndAdjustSourceConfigFunc = checkAndAdjustSourceConfig + // the difference of below functions is checkAndAdjustSourceConfigForDMCtlFunc will not AdjustCaseSensitive. It's a + // compatibility compromise. + // When we need to change the implementation of dmctl to OpenAPI, we should notice the user about this change. + checkAndAdjustSourceConfigFunc = checkAndAdjustSourceConfig + checkAndAdjustSourceConfigForDMCtlFunc = checkAndAdjustSourceConfigForDMCtl ) // Server handles RPC requests for dm-master. @@ -190,8 +195,14 @@ func (s *Server) Start(ctx context.Context) (err error) { "/status": getStatusHandle(), "/debug/": getDebugHandler(), } + if s.cfg.ExperimentalFeatures.OpenAPI { - if initOpenAPIErr := s.InitOpenAPIHandles(); initOpenAPIErr != nil { + // tls3 is used to openapi reverse proxy + tls3, err1 := toolutils.NewTLS(s.cfg.SSLCA, s.cfg.SSLCert, s.cfg.SSLKey, s.cfg.AdvertiseAddr, s.cfg.CertAllowedCN) + if err1 != nil { + return terror.ErrMasterTLSConfigNotValid.Delegate(err1) + } + if initOpenAPIErr := s.InitOpenAPIHandles(tls3.TLSConfig()); initOpenAPIErr != nil { return terror.ErrOpenAPICommonError.Delegate(initOpenAPIErr) } userHandles["/api/v1/"] = s.echo @@ -1121,7 +1132,7 @@ func parseAndAdjustSourceConfig(ctx context.Context, contents []string) ([]*conf if err != nil { return cfgs, err } - if err := checkAndAdjustSourceConfigFunc(ctx, cfg); err != nil { + if err := checkAndAdjustSourceConfigForDMCtlFunc(ctx, cfg); err != nil { return cfgs, err } cfgs[i] = cfg @@ -1129,7 +1140,11 @@ func parseAndAdjustSourceConfig(ctx context.Context, contents []string) ([]*conf return cfgs, nil } -func checkAndAdjustSourceConfig(ctx context.Context, cfg *config.SourceConfig) error { +func innerCheckAndAdjustSourceConfig( + ctx context.Context, + cfg *config.SourceConfig, + hook func(sourceConfig *config.SourceConfig, ctx context.Context, db *sql.DB) error, +) error { dbConfig := cfg.GenerateDBConfig() fromDB, err := conn.DefaultDBProvider.Apply(dbConfig) if err != nil { @@ -1139,12 +1154,25 @@ func checkAndAdjustSourceConfig(ctx context.Context, cfg *config.SourceConfig) e if err = cfg.Adjust(ctx, fromDB.DB); err != nil { return err } + if hook != nil { + if err = hook(cfg, ctx, fromDB.DB); err != nil { + return err + } + } if _, err = cfg.Yaml(); err != nil { return err } return cfg.Verify() } +func checkAndAdjustSourceConfig(ctx context.Context, cfg *config.SourceConfig) error { + return innerCheckAndAdjustSourceConfig(ctx, cfg, (*config.SourceConfig).AdjustCaseSensitive) +} + +func checkAndAdjustSourceConfigForDMCtl(ctx context.Context, cfg *config.SourceConfig) error { + return innerCheckAndAdjustSourceConfig(ctx, cfg, nil) +} + func parseSourceConfig(contents []string) ([]*config.SourceConfig, error) { cfgs := make([]*config.SourceConfig, len(contents)) for i, content := range contents { diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index 1aeef2a89a3..266986b3538 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -191,12 +191,12 @@ func (t *testMaster) SetUpSuite(c *check.C) { t.workerClients = make(map[string]workerrpc.Client) t.saveMaxRetryNum = maxRetryNum maxRetryNum = 2 - checkAndAdjustSourceConfigFunc = checkAndNoAdjustSourceConfigMock + checkAndAdjustSourceConfigForDMCtlFunc = checkAndNoAdjustSourceConfigMock } func (t *testMaster) TearDownSuite(c *check.C) { maxRetryNum = t.saveMaxRetryNum - checkAndAdjustSourceConfigFunc = checkAndAdjustSourceConfig + checkAndAdjustSourceConfigForDMCtlFunc = checkAndAdjustSourceConfig } func (t *testMaster) SetUpTest(c *check.C) { diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 5989da6adee..0b1ef910c94 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -540,7 +540,7 @@ func (w *SourceWorker) UpdateSubTask(ctx context.Context, cfg *config.SubTaskCon return st.Update(ctx, cfg) } -// OperateSubTask stop/resume/pause sub task. +// OperateSubTask stop/resume/pause sub task. func (w *SourceWorker) OperateSubTask(name string, op pb.TaskOp) error { w.Lock() defer w.Unlock() diff --git a/dm/errors.toml b/dm/errors.toml index 9fd7af01ece..e07679a6174 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2026,6 +2026,12 @@ description = "" workaround = "Please check if the binlog file could be parsed by `mysqlbinlog`." tags = ["upstream", "high"] +[error.DM-sync-unit-36070] +message = "downstream table %s not found" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-master-38001] message = "nil request not valid" description = "" diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 43e47c1c4aa..4b4e757ee8e 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -20,6 +20,7 @@ import ( "os" "strings" + "github.com/docker/go-units" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" tidbConfig "github.com/pingcap/tidb/config" @@ -35,7 +36,9 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + unistoreConfig "github.com/pingcap/tidb/store/mockstore/unistore/config" "github.com/pingcap/tidb/types" + "go.uber.org/atomic" "go.uber.org/zap" tcontext "github.com/pingcap/tiflow/dm/pkg/context" @@ -60,6 +63,11 @@ var ( } ) +func init() { + unistoreConfig.DefaultConf.Engine.VlogFileSize = 4 * units.MiB + unistoreConfig.DefaultConf.Engine.L1Size = 128 * units.MiB +} + // Tracker is used to track schema locally. type Tracker struct { storePath string @@ -67,6 +75,7 @@ type Tracker struct { dom *domain.Domain se session.Session dsTracker *downstreamTracker + closed atomic.Bool } // downstreamTracker tracks downstream schema. @@ -265,10 +274,7 @@ func (tr *Tracker) GetCreateTable(ctx context.Context, table *filter.Table) (str row := req.GetRow(0) str := row.GetString(1) // the first column is the table name. - // returned as single line. - str = strings.ReplaceAll(str, "\n", "") - str = strings.ReplaceAll(str, " ", " ") - return str, nil + return utils.CreateTableSQLToOneRow(str), nil } // AllSchemas returns all schemas visible to the tracker (excluding system tables). @@ -348,6 +354,12 @@ func (tr *Tracker) Reset() error { // Close close a tracker. func (tr *Tracker) Close() error { + if tr == nil { + return nil + } + if !tr.closed.CAS(false, true) { + return nil + } tr.se.Close() tr.dom.Close() if err := tr.store.Close(); err != nil { diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 347192f5719..7003df2fb90 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -433,6 +433,7 @@ const ( codeSyncerParseDDL codeSyncerUnsupportedStmt codeSyncerGetEvent + codeSyncerDownstreamTableNotFound ) // DM-master error code. @@ -1067,6 +1068,7 @@ var ( ErrSyncerParseDDL = New(codeSyncerParseDDL, ClassSyncUnit, ScopeInternal, LevelHigh, "parse DDL: %s", "Please confirm your DDL statement is correct and needed. For TiDB compatible DDL, see https://docs.pingcap.com/tidb/stable/mysql-compatibility#ddl. You can use `handle-error` command to skip or replace the DDL or add a binlog filter rule to ignore it if the DDL is not needed.") ErrSyncerUnsupportedStmt = New(codeSyncerUnsupportedStmt, ClassSyncUnit, ScopeInternal, LevelHigh, "`%s` statement not supported in %s mode", "") ErrSyncerGetEvent = New(codeSyncerGetEvent, ClassSyncUnit, ScopeUpstream, LevelHigh, "get binlog event error: %v", "Please check if the binlog file could be parsed by `mysqlbinlog`.") + ErrSyncerDownstreamTableNotFound = New(codeSyncerDownstreamTableNotFound, ClassSyncUnit, ScopeInternal, LevelHigh, "downstream table %s not found", "") // DM-master error. ErrMasterSQLOpNilRequest = New(codeMasterSQLOpNilRequest, ClassDMMaster, ScopeInternal, LevelMedium, "nil request not valid", "") diff --git a/dm/pkg/utils/common.go b/dm/pkg/utils/common.go index 6f88836fd40..658dfdd2165 100644 --- a/dm/pkg/utils/common.go +++ b/dm/pkg/utils/common.go @@ -181,12 +181,13 @@ func FetchLowerCaseTableNamesSetting(ctx context.Context, conn *sql.Conn) (Lower return LowerCaseTableNamesFlavor(res), nil } -// GetDBCaseSensitive returns the case sensitive setting of target db. +// GetDBCaseSensitive returns the case-sensitive setting of target db. func GetDBCaseSensitive(ctx context.Context, db *sql.DB) (bool, error) { conn, err := db.Conn(ctx) if err != nil { return true, terror.DBErrorAdapt(err, terror.ErrDBDriverError) } + defer conn.Close() lcFlavor, err := FetchLowerCaseTableNamesSetting(ctx, conn) if err != nil { return true, err diff --git a/dm/pkg/utils/db.go b/dm/pkg/utils/db.go index c582a016823..806b48165d0 100644 --- a/dm/pkg/utils/db.go +++ b/dm/pkg/utils/db.go @@ -632,3 +632,10 @@ func GetTableCreateSQL(ctx context.Context, conn *sql.Conn, tableID string) (sql } return createStr, nil } + +// CreateTableSQLToOneRow formats the result of SHOW CREATE TABLE to one row. +func CreateTableSQLToOneRow(sql string) string { + sql = strings.ReplaceAll(sql, "\n", "") + sql = strings.ReplaceAll(sql, " ", " ") + return sql +} diff --git a/dm/pkg/utils/db_test.go b/dm/pkg/utils/db_test.go index 3687f1eaf82..04b640d2973 100644 --- a/dm/pkg/utils/db_test.go +++ b/dm/pkg/utils/db_test.go @@ -16,6 +16,7 @@ package utils import ( "context" "strconv" + "testing" "time" "github.com/DATA-DOG/go-sqlmock" @@ -25,6 +26,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" tmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/stretchr/testify/require" "github.com/pingcap/tiflow/dm/pkg/gtid" ) @@ -458,3 +460,9 @@ func (t *testDBSuite) TestAddGSetWithPurged(c *C) { c.Assert(originSet, DeepEquals, tc.originGSet) } } + +func TestCreateTableSQLToOneRow(t *testing.T) { + input := "CREATE TABLE `t1` (\n `id` bigint(20) NOT NULL,\n `c1` varchar(20) DEFAULT NULL,\n `c2` varchar(20) DEFAULT NULL,\n PRIMARY KEY (`id`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin" + expected := "CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c1` varchar(20) DEFAULT NULL, `c2` varchar(20) DEFAULT NULL, PRIMARY KEY (`id`) /*T![clustered_index] NONCLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin" + require.Equal(t, expected, CreateTableSQLToOneRow(input)) +} diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 709f7802ad1..87d6260bc91 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -105,7 +105,7 @@ func (b *binlogPoint) flush() { b.flushedTI = b.ti } -func (b *binlogPoint) rollback(schemaTracker *schema.Tracker, schema string) (isSchemaChanged bool) { +func (b *binlogPoint) rollback() { b.Lock() defer b.Unlock() @@ -113,23 +113,23 @@ func (b *binlogPoint) rollback(schemaTracker *schema.Tracker, schema string) (is b.flushedLocation.ResetSuffix() b.location = b.flushedLocation if b.ti == nil { + // TODO: if we forget to save table info for table checkpoint, this is also nil! + // And table checkpoint rollback to flushed point may also be nil! return // for global checkpoint, no need to rollback the schema. } // NOTE: no `Equal` function for `model.TableInfo` exists now, so we compare `pointer` directly, // and after a new DDL applied to the schema, the returned pointer of `model.TableInfo` changed now. - trackedTi, _ := schemaTracker.GetTableInfo(&filter.Table{Schema: schema, Name: b.ti.Name.O}) // ignore the returned error, only compare `trackerTi` is enough. - // may three versions of schema exist: - // - the one tracked in the TiDB-with-mockTiKV. + // there may be three versions of schema: + // - the one tracked in the schema tracker (TiDB-with-unistore). // - the one in the checkpoint but not flushed. // - the one in the checkpoint and flushed. - // if any of them are not equal, then we rollback them: + // schema tracker will be closed after task is paused, and it will load all schemas from checkpoint when task resumes. + // if the later two are not equal, then we rollback them: // - set the one in the checkpoint but not flushed to the one flushed. - // - set the one tracked to the one in the checkpoint by the caller of this method (both flushed and not flushed are the same now) - if isSchemaChanged = (trackedTi != b.ti) || (b.ti != b.flushedTI); isSchemaChanged { + if b.ti != b.flushedTI { b.ti = b.flushedTI } - return } func (b *binlogPoint) outOfDate() bool { @@ -235,6 +235,9 @@ type CheckPoint interface { // TablePoint returns all table's stream checkpoint TablePoint() map[string]map[string]binlog.Location + // GetTableInfo returns the saved table info from table checkpoint for the given table, return nil when not found + GetTableInfo(schema string, table string) *model.TableInfo + // FlushedGlobalPoint returns the flushed global binlog stream's checkpoint // corresponding to to Meta.Pos and gtid FlushedGlobalPoint() binlog.Location @@ -248,13 +251,16 @@ type CheckPoint interface { GetFlushedTableInfo(table *filter.Table) *model.TableInfo // Rollback rolls global checkpoint and all table checkpoints back to flushed checkpoints - Rollback(schemaTracker *schema.Tracker) + Rollback() // String return text of global position String() string // CheckAndUpdate check the checkpoint data consistency and try to fix them if possible CheckAndUpdate(ctx context.Context, schemas map[string]string, tables map[string]map[string]string) error + + // LoadIntoSchemaTracker loads table infos of all points into schema tracker. + LoadIntoSchemaTracker(ctx context.Context, schemaTracker *schema.Tracker) error } // RemoteCheckPoint implements CheckPoint @@ -594,10 +600,16 @@ func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, tabl args := make([][]interface{}, 0, 10) point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) - if tablePoints, ok := cp.points[sourceSchema]; ok { - if p, ok2 := tablePoints[sourceTable]; ok2 { - point = p - } + tablePoints, ok := cp.points[sourceSchema] + if !ok { + tablePoints = map[string]*binlogPoint{} + cp.points[sourceSchema] = tablePoints + } + p, ok2 := tablePoints[sourceTable] + if ok2 { + point = p + } else { + tablePoints[sourceTable] = point } tiBytes, err := json.Marshal(ti) @@ -674,6 +686,21 @@ func (cp *RemoteCheckPoint) TablePoint() map[string]map[string]binlog.Location { return tablePoint } +func (cp *RemoteCheckPoint) GetTableInfo(schema string, table string) *model.TableInfo { + cp.RLock() + defer cp.RUnlock() + + tables, ok := cp.points[schema] + if !ok { + return nil + } + tablePoint, ok := tables[table] + if !ok { + return nil + } + return tablePoint.TableInfo() +} + // FlushedGlobalPoint implements CheckPoint.FlushedGlobalPoint. func (cp *RemoteCheckPoint) FlushedGlobalPoint() binlog.Location { cp.RLock() @@ -698,10 +725,10 @@ func (cp *RemoteCheckPoint) CheckGlobalPoint() bool { } // Rollback implements CheckPoint.Rollback. -func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { +func (cp *RemoteCheckPoint) Rollback() { cp.RLock() defer cp.RUnlock() - cp.globalPoint.rollback(schemaTracker, "") + cp.globalPoint.rollback() for schemaName, mSchema := range cp.points { for tableName, point := range mSchema { table := &filter.Table{ @@ -710,38 +737,7 @@ func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { } logger := cp.logCtx.L().WithFields(zap.Stringer("table", table)) logger.Debug("try to rollback checkpoint", log.WrapStringerField("checkpoint", point)) - from := point.MySQLLocation() - if point.rollback(schemaTracker, schemaName) { - logger.Info("rollback checkpoint", zap.Stringer("from", from), zap.Stringer("to", point.FlushedMySQLLocation())) - // schema changed - if err := schemaTracker.DropTable(table); err != nil { - logger.Warn("failed to drop table from schema tracker", log.ShortError(err)) - } - if point.ti != nil { - // TODO: Figure out how to recover from errors. - if err := schemaTracker.CreateSchemaIfNotExists(schemaName); err != nil { - logger.Error("failed to rollback schema on schema tracker: cannot create schema", log.ShortError(err)) - } - if err := schemaTracker.CreateTableIfNotExists(table, point.ti); err != nil { - logger.Error("failed to rollback schema on schema tracker: cannot create table", log.ShortError(err)) - } - } - } - } - } - - // drop any tables in the tracker if no corresponding checkpoint exists. - for _, schema := range schemaTracker.AllSchemas() { - _, ok1 := cp.points[schema.Name.O] - for _, table := range schema.Tables { - var ok2 bool - if ok1 { - _, ok2 = cp.points[schema.Name.O][table.Name.O] - } - if !ok2 { - err := schemaTracker.DropTable(&filter.Table{Schema: schema.Name.O, Name: table.Name.O}) - cp.logCtx.L().Info("drop table in schema tracker because no checkpoint exists", zap.String("schema", schema.Name.O), zap.String("table", table.Name.O), log.ShortError(err)) - } + point.rollback() } } } @@ -897,6 +893,33 @@ func (cp *RemoteCheckPoint) Load(tctx *tcontext.Context) error { return terror.WithScope(terror.DBErrorAdapt(rows.Err(), terror.ErrDBDriverError), terror.ScopeDownstream) } +// LoadIntoSchemaTracker loads table infos of all points into schema tracker. +func (cp *RemoteCheckPoint) LoadIntoSchemaTracker(ctx context.Context, schemaTracker *schema.Tracker) error { + cp.RLock() + defer cp.RUnlock() + + for cpSchema, mSchema := range cp.points { + err := schemaTracker.CreateSchemaIfNotExists(cpSchema) + if err != nil { + return err + } + for cpTable, point := range mSchema { + // for create database DDL, we'll create a table point with no table name and table info, need to skip. + if point.flushedTI == nil { + continue + } + cp.logCtx.L().Debug("will init table info in schema tracker", + zap.String("database", cpSchema), + zap.String("table", cpTable)) + err := schemaTracker.CreateTableIfNotExists(&filter.Table{Schema: cpSchema, Name: cpTable}, point.flushedTI) + if err != nil { + return err + } + } + } + return nil +} + // CheckAndUpdate check the checkpoint data consistency and try to fix them if possible. func (cp *RemoteCheckPoint) CheckAndUpdate(ctx context.Context, schemas map[string]string, tables map[string]map[string]string) error { cp.Lock() diff --git a/dm/syncer/checkpoint_test.go b/dm/syncer/checkpoint_test.go index 4564d630bfd..81ddedbbfe4 100644 --- a/dm/syncer/checkpoint_test.go +++ b/dm/syncer/checkpoint_test.go @@ -19,7 +19,6 @@ import ( "fmt" "os" "path/filepath" - "strings" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" @@ -201,7 +200,7 @@ func (s *testCheckpointSuite) testGlobalCheckPoint(c *C, cp CheckPoint) { c.Assert(cp.FlushedGlobalPoint().Position, Equals, pos1) // test rollback - cp.Rollback(s.tracker) + cp.Rollback() c.Assert(cp.GlobalPoint().Position, Equals, pos1) c.Assert(cp.FlushedGlobalPoint().Position, Equals, pos1) @@ -216,7 +215,7 @@ func (s *testCheckpointSuite) testGlobalCheckPoint(c *C, cp CheckPoint) { s.mock.ExpectCommit() err = cp.FlushPointsExcept(tctx, nil, nil, nil) c.Assert(err, IsNil) - cp.Rollback(s.tracker) + cp.Rollback() c.Assert(cp.GlobalPoint().Position, Equals, pos2) c.Assert(cp.FlushedGlobalPoint().Position, Equals, pos2) @@ -354,7 +353,7 @@ func (s *testCheckpointSuite) testTableCheckPoint(c *C, cp CheckPoint) { c.Assert(older, IsTrue) // rollback, to min - cp.Rollback(s.tracker) + cp.Rollback() older = cp.IsOlderThanTablePoint(table, binlog.Location{Position: pos1}, false) c.Assert(older, IsFalse) @@ -369,7 +368,7 @@ func (s *testCheckpointSuite) testTableCheckPoint(c *C, cp CheckPoint) { s.mock.ExpectCommit() err = cp.FlushPointsExcept(tctx, nil, nil, nil) c.Assert(err, IsNil) - cp.Rollback(s.tracker) + cp.Rollback() older = cp.IsOlderThanTablePoint(table, binlog.Location{Position: pos1}, false) c.Assert(older, IsTrue) @@ -401,19 +400,12 @@ func (s *testCheckpointSuite) testTableCheckPoint(c *C, cp CheckPoint) { c.Assert(rcp.points[schemaName][tableName].TableInfo(), NotNil) c.Assert(rcp.points[schemaName][tableName].flushedTI, IsNil) - cp.Rollback(s.tracker) + cp.Rollback() rcp = cp.(*RemoteCheckPoint) c.Assert(rcp.points[schemaName][tableName].TableInfo(), IsNil) c.Assert(rcp.points[schemaName][tableName].flushedTI, IsNil) - _, err = s.tracker.GetTableInfo(table) - c.Assert(strings.Contains(err.Error(), "doesn't exist"), IsTrue) - // test save, flush and rollback to not nil table info - err = s.tracker.Exec(ctx, schemaName, "create table "+tableName+" (c int);") - c.Assert(err, IsNil) - ti, err = s.tracker.GetTableInfo(table) - c.Assert(err, IsNil) cp.SaveTablePoint(table, binlog.Location{Position: pos1}, ti) tiBytes, _ := json.Marshal(ti) s.mock.ExpectBegin() @@ -425,10 +417,7 @@ func (s *testCheckpointSuite) testTableCheckPoint(c *C, cp CheckPoint) { ti2, err := s.tracker.GetTableInfo(table) c.Assert(err, IsNil) cp.SaveTablePoint(table, binlog.Location{Position: pos2}, ti2) - cp.Rollback(s.tracker) - ti11, err := s.tracker.GetTableInfo(table) - c.Assert(err, IsNil) - c.Assert(ti11.Columns, HasLen, 1) + cp.Rollback() // clear, to min s.mock.ExpectBegin() @@ -456,7 +445,7 @@ func (s *testCheckpointSuite) testTableCheckPoint(c *C, cp CheckPoint) { s.mock.ExpectCommit() err = cp.FlushPointsExcept(tctx, []*filter.Table{table}, nil, nil) c.Assert(err, IsNil) - cp.Rollback(s.tracker) + cp.Rollback() older = cp.IsOlderThanTablePoint(table, binlog.Location{Position: pos1}, false) c.Assert(older, IsFalse) diff --git a/dm/syncer/dbconn/db.go b/dm/syncer/dbconn/db.go index 379ef4a6de2..8625e71b106 100644 --- a/dm/syncer/dbconn/db.go +++ b/dm/syncer/dbconn/db.go @@ -18,8 +18,11 @@ import ( "strings" "time" + "github.com/go-sql-driver/mysql" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/errno" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" @@ -190,7 +193,8 @@ func (conn *DBConn) ExecuteSQLWithIgnore(tctx *tcontext.Context, ignoreError fun metrics.SQLRetriesTotal.WithLabelValues("stmt_exec", conn.Cfg.Name).Add(1) return true } - return false + // TODO: move it to above IsRetryableError + return isRetryableError(err) }, } @@ -227,6 +231,16 @@ func (conn *DBConn) ExecuteSQLWithIgnore(tctx *tcontext.Context, ignoreError fun return ret.(int), nil } +func isRetryableError(err error) bool { + err = errors.Cause(err) // check the original error + mysqlErr, ok := err.(*mysql.MySQLError) + if !ok { + return false + } + + return mysqlErr.Number == errno.ErrKeyColumnDoesNotExits +} + // ExecuteSQL does some SQL executions. func (conn *DBConn) ExecuteSQL(tctx *tcontext.Context, queries []string, args ...[]interface{}) (int, error) { return conn.ExecuteSQLWithIgnore(tctx, nil, queries, args...) diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index cf26e69c499..247eb4d3e5f 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -241,7 +241,9 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { time.Sleep(time.Duration(t) * time.Second) }) // use background context to execute sqls as much as possible - ctx, cancel := w.tctx.WithTimeout(maxDMLExecutionDuration) + // set timeout to maxDMLConnectionDuration to make sure dmls can be replicated to downstream event if the latency is high + // if users need to quit this asap, we can support pause-task/stop-task --force in the future + ctx, cancel := w.tctx.WithTimeout(maxDMLConnectionDuration) defer cancel() affect, err = db.ExecuteSQL(ctx, queries, args...) failpoint.Inject("SafeModeExit", func(val failpoint.Value) { diff --git a/dm/syncer/filter.go b/dm/syncer/filter.go index b224a5ee99c..670b4cf54b1 100644 --- a/dm/syncer/filter.go +++ b/dm/syncer/filter.go @@ -62,6 +62,7 @@ func (s *Syncer) skipQueryEvent(qec *queryEventContext, ddlInfo *ddlInfo) (bool, if err != nil { s.tctx.L().Warn("track ddl failed", zap.Stringer("ddl info", ddlInfo)) } + s.saveTablePoint(table, *qec.lastLocation) s.tctx.L().Warn("track skipped ddl and return empty string", zap.String("origin sql", qec.originSQL), zap.Stringer("ddl info", ddlInfo)) ddlInfo.originDDL = "" return true, nil diff --git a/dm/syncer/filter_test.go b/dm/syncer/filter_test.go index 9ad9c4d37bb..662a6c75af6 100644 --- a/dm/syncer/filter_test.go +++ b/dm/syncer/filter_test.go @@ -18,13 +18,14 @@ import ( "database/sql" "github.com/DATA-DOG/go-sqlmock" + "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/schema" @@ -69,6 +70,7 @@ func (s *testFilterSuite) TestSkipQueryEvent(c *C) { syncer.ddlDBConn = &dbconn.DBConn{Cfg: syncer.cfg, BaseConn: s.baseConn} syncer.schemaTracker, err = schema.NewTracker(context.Background(), syncer.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn) c.Assert(err, IsNil) + defer syncer.schemaTracker.Close() syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) // test binlog filter @@ -125,9 +127,13 @@ func (s *testFilterSuite) TestSkipQueryEvent(c *C) { }, } p := parser.New() + loc := binlog.NewLocation(mysql.MySQLFlavor) qec := &queryEventContext{ - eventContext: &eventContext{tctx: tcontext.Background()}, - p: p, + eventContext: &eventContext{ + tctx: tcontext.Background(), + lastLocation: &loc, + }, + p: p, } for _, ca := range cases { ddlInfo, err := syncer.genDDLInfo(p, ca.schema, ca.sql) diff --git a/dm/syncer/schema.go b/dm/syncer/schema.go index a7d1c2360f5..e4ffbb02432 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -14,21 +14,25 @@ package syncer import ( + "bytes" "context" "encoding/json" "strings" "github.com/pingcap/tidb-tools/pkg/filter" + ddl2 "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/dm/pkg/utils" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -61,12 +65,21 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR } return string(tableListJSON), err case pb.SchemaOp_GetSchema: - // we only try to get schema from schema-tracker now. - // in other words, we can not get the schema if any DDL/DML has been replicated, or set a schema previously. - return s.schemaTracker.GetCreateTable(ctx, sourceTable) + // when task is paused, schemaTracker is closed. We get the table structure from checkpoint. + ti := s.checkpoint.GetTableInfo(req.Database, req.Table) + if ti == nil { + s.tctx.L().Info("table schema is not in checkpoint, fetch from downstream", + zap.String("table", sourceTable.String())) + targetTable := s.route(sourceTable) + return utils.GetTableCreateSQL(ctx, s.downstreamTrackConn.BaseConn.DBConn, targetTable.String()) + } + + result := bytes.NewBuffer(make([]byte, 0, 512)) + err2 := executor.ConstructResultOfShowCreateTable(s.sessCtx, ti, autoid.Allocators{}, result) + return utils.CreateTableSQLToOneRow(result.String()), err2 + case pb.SchemaOp_SetSchema: // for set schema, we must ensure it's a valid `CREATE TABLE` statement. - // now, we only set schema for schema-tracker, // if want to update the one in checkpoint, it should wait for the flush of checkpoint. parser2, err := s.fromDB.GetParser(ctx) if err != nil { @@ -92,37 +105,21 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR } newSQL := newCreateSQLBuilder.String() - // drop the previous schema first. - err = s.schemaTracker.DropTable(sourceTable) - if err != nil && !schema.IsTableNotExists(err) { - return "", terror.ErrSchemaTrackerCannotDropTable.Delegate(err, sourceTable) - } - err = s.schemaTracker.CreateSchemaIfNotExists(req.Database) - if err != nil { - return "", terror.ErrSchemaTrackerCannotCreateSchema.Delegate(err, req.Database) - } - err = s.schemaTracker.Exec(ctx, req.Database, newSQL) - if err != nil { - return "", terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) - } - s.exprFilterGroup.ResetExprs(sourceTable) - if !req.Flush && !req.Sync { - break + if !req.Flush { + s.tctx.L().Info("overwrite --flush to true for operate-schema") } - ti, err := s.schemaTracker.GetTableInfo(sourceTable) - if err != nil { - return "", err + ti, err2 := ddl2.BuildTableInfoFromAST(stmt) + if err2 != nil { + return "", terror.ErrSchemaTrackerRestoreStmtFail.Delegate(err2) } - if req.Flush { - log.L().Info("flush table info", zap.String("table info", newSQL)) - err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), sourceTable, ti) - if err != nil { - return "", err - } + s.tctx.L().Info("flush table info", zap.String("table info", newSQL)) + err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), sourceTable, ti) + if err != nil { + return "", err } if req.Sync { @@ -142,9 +139,9 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR } case pb.SchemaOp_RemoveSchema: - // we only drop the schema in the schema-tracker now, - // so if we drop the schema and continue to replicate any DDL/DML, it will try to get schema from downstream again. - return "", s.schemaTracker.DropTable(sourceTable) + // as the doc says, `operate-schema remove` will let DM-worker use table structure in checkpoint, which does not + // need further actions. + return "", nil } return "", nil } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 5a360875123..16520d897cd 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -87,7 +87,6 @@ var ( maxDDLConnectionTimeout = fmt.Sprintf("%dm", MaxDDLConnectionTimeoutMinute) maxDMLConnectionDuration, _ = time.ParseDuration(maxDMLConnectionTimeout) - maxDMLExecutionDuration = 30 * time.Second maxPauseOrStopWaitTime = 10 * time.Second @@ -322,11 +321,6 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } rollbackHolder.Add(fr.FuncRollback{Name: "close-DBs", Fn: s.closeDBs}) - s.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, s.cfg.To.Session, s.downstreamTrackConn) - if err != nil { - return terror.ErrSchemaTrackerInit.Delegate(err) - } - s.streamerController = NewStreamerController(s.notifier, s.syncCfg, s.cfg.EnableGTID, s.fromDB, s.binlogType, s.cfg.RelayDir, s.timezone) s.baList, err = filter.New(s.cfg.CaseSensitive, s.cfg.BAList) @@ -676,7 +670,7 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { // try to rollback checkpoints, if they already flushed, no effect prePos := s.checkpoint.GlobalPoint() - s.checkpoint.Rollback(s.schemaTracker) + s.checkpoint.Rollback() currPos := s.checkpoint.GlobalPoint() if binlog.CompareLocation(prePos, currPos, s.cfg.EnableGTID) != 0 { s.tctx.L().Warn("something wrong with rollback global checkpoint", zap.Stringer("previous position", prePos), zap.Stringer("current position", currPos)) @@ -1347,6 +1341,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { flushCheckpoint bool delLoadTask bool cleanDumpFile = s.cfg.CleanDumpFile + freshAndAllMode bool ) flushCheckpoint, err = s.adjustGlobalPointGTID(tctx) if err != nil { @@ -1355,15 +1350,24 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.cfg.Mode == config.ModeAll && fresh { delLoadTask = true flushCheckpoint = true - err = s.loadTableStructureFromDump(ctx) - if err != nil { - tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) - cleanDumpFile = false - } - } else { + freshAndAllMode = true + } + if s.cfg.Mode == config.ModeIncrement || !fresh { cleanDumpFile = false } + // some prepare work before the binlog event loop: + // 1. first we flush checkpoint as needed, so in next resume we won't go to Load unit. + // 2. then since we are confident that Load unit is done we can delete the load task etcd KV. + // TODO: we can't handle panic between 1. and 2., or fail to delete the load task etcd KV. + // 3. then we initiate schema tracker + // 4. - when it's a fresh task, load the table structure from dump files into schema tracker. + // if it's also a optimistic sharding task, also load the table structure into checkpoints because shard tables + // may not have same table structure so we can't fetch the downstream table structure for them lazily. + // - when it's a resumed task, load the table structure from checkpoints into schema tracker. + // TODO: we can't handle failure between 1. and 4. After 1. it's not a fresh task. + // 5. finally clean the dump files + if flushCheckpoint { if err = s.flushCheckPoints(); err != nil { tctx.L().Warn("fail to flush checkpoints when starting task", zap.Error(err)) @@ -1375,6 +1379,24 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Warn("error when del load task in etcd", zap.Error(err)) } } + + s.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, s.cfg.To.Session, s.downstreamTrackConn) + if err != nil { + return terror.ErrSchemaTrackerInit.Delegate(err) + } + + if freshAndAllMode { + err = s.loadTableStructureFromDump(ctx) + if err != nil { + tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) + cleanDumpFile = false + } + } else { + err = s.checkpoint.LoadIntoSchemaTracker(ctx, s.schemaTracker) + if err != nil { + return err + } + } if cleanDumpFile { tctx.L().Info("try to remove all dump files") if err = os.RemoveAll(s.cfg.Dir); err != nil { @@ -2933,6 +2955,9 @@ func (s *Syncer) loadTableStructureFromDump(ctx context.Context) error { zap.Error(err)) setFirstErr(err) } + // TODO: we should save table checkpoint here, but considering when + // the first time of flushing checkpoint, user may encounter https://github.com/pingcap/tiflow/issues/5010 + // we should fix that problem first. } } return firstErr @@ -3123,6 +3148,9 @@ func (s *Syncer) Pause() { return } s.stopSync() + if err := s.schemaTracker.Close(); err != nil { + s.tctx.L().Error("fail to close schema tracker", log.ShortError(err)) + } } // Resume resumes the paused process. diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index b924a935546..7209d5302ca 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -770,6 +770,10 @@ func (s *testSyncerSuite) TestRun(c *C) { s.cfg.Batch = 1000 s.cfg.WorkerCount = 2 s.cfg.MaxRetry = 1 + s.cfg.To.Session = map[string]string{ + "sql_mode": "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", + "tidb_skip_utf8_check": "0", + } cfg, err := s.cfg.Clone() c.Assert(err, IsNil) @@ -792,10 +796,6 @@ func (s *testSyncerSuite) TestRun(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("t_1", "create table t_1(id int primary key, name varchar(24), KEY `index1` (`name`))")) - s.mockGetServerUnixTS(mock) - mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_2`").WillReturnRows( - sqlmock.NewRows([]string{"Table", "Create Table"}). - AddRow("t_2", "create table t_2(id int primary key, name varchar(24))")) syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) c.Assert(err, IsNil) @@ -947,6 +947,14 @@ func (s *testSyncerSuite) TestRun(c *C) { mockDBProvider := conn.InitMockDB(c) mockDBProvider.ExpectQuery("SELECT cast\\(TIMEDIFF\\(NOW\\(6\\), UTC_TIMESTAMP\\(6\\)\\) as time\\);"). WillReturnRows(sqlmock.NewRows([]string{""}).AddRow("01:00:00")) + s.mockGetServerUnixTS(mock) + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", pmysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_2`").WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("t_2", "create table t_2(id int primary key, name varchar(24))")) + c.Assert(syncer.Update(context.Background(), s.cfg), IsNil) c.Assert(syncer.timezone.String(), Equals, "+01:00") @@ -1032,6 +1040,10 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { {Schema: "test_1", Name: "t_1"}, }, } + s.cfg.To.Session = map[string]string{ + "sql_mode": "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", + "tidb_skip_utf8_check": "0", + } cfg, err := s.cfg.Clone() c.Assert(err, IsNil) @@ -1110,6 +1122,13 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { // disable 1-minute safe mode c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds", "return(0)"), IsNil) go syncer.Process(ctx, resultCh) + go func() { + for r := range resultCh { + if len(r.Errors) > 0 { + c.Fatal(r.String()) + } + } + }() expectJobs := []*expectJob{ // now every ddl job will start with a flush job diff --git a/dm/tests/_utils/check_process_exit b/dm/tests/_utils/check_process_exit new file mode 100755 index 00000000000..f2f9aa932e6 --- /dev/null +++ b/dm/tests/_utils/check_process_exit @@ -0,0 +1,22 @@ +#!/bin/bash +# parameter 1: name +# parameter 2: retry count + +name=$1 +retry=$2 + +i=0 +while [ $i -lt $retry ]; do + ret=$(ps aux | grep $name | sed '/.*grep.*/d' | sed '/.*check_process_exit.*/d' | wc -l) + if [ $ret -lt 1 ]; then + break + fi + ((i++)) + echo "wait for process $name exits the $i-th time" + sleep 1 +done + +if [ $i -ge $retry ]; then + echo "process $name exits timeout" + exit 1 +fi diff --git a/dm/tests/_utils/check_sync_diff b/dm/tests/_utils/check_sync_diff index 2b7bcd17bdf..632eda279fe 100755 --- a/dm/tests/_utils/check_sync_diff +++ b/dm/tests/_utils/check_sync_diff @@ -9,7 +9,7 @@ conf=$2 if [ $# -ge 3 ]; then check_time=$3 else - check_time=10 + check_time=20 fi PWD=$(pwd) diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index ff24c716c5c..a91e1755372 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -307,15 +307,18 @@ function check_log_contain_with_retry() { function init_cluster(){ run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT - run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 } diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index c3d87bcf207..7d43a5e3d0f 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -95,14 +95,13 @@ function test_query_timeout() { # don't know why CI has turned on Event Scheduler run_sql_both_source 'SET GLOBAL event_scheduler = OFF;' - # there's only 2 rows in result, one for dm-worker's source-level status, one for SHOW PROCESSLIST + # there's only 1 row in result, which is for SHOW PROCESSLIST run_sql_source1 'SHOW PROCESSLIST;' - check_rows_equal 2 + check_rows_equal 1 run_sql_source2 'SHOW PROCESSLIST;' - check_rows_equal 2 + check_rows_equal 1 - # there's only 1 row in result, which is for SHOW PROCESSLIST run_sql_tidb 'SHOW PROCESSLIST;' check_rows_equal 1 @@ -130,14 +129,13 @@ function test_query_timeout() { "stop-task $ILLEGAL_CHAR_NAME" \ "\"result\": true" 3 - # there's only 2 rows in result, one for dm-worker's source-level status, one for SHOW PROCESSLIST + # there's only 1 row in result, which is for SHOW PROCESSLIST run_sql_source1 'SHOW PROCESSLIST;' - check_rows_equal 2 + check_rows_equal 1 run_sql_source2 'SHOW PROCESSLIST;' - check_rows_equal 2 + check_rows_equal 1 - # there's only 1 row in result, which is for SHOW PROCESSLIST run_sql_tidb 'SHOW PROCESSLIST;' check_rows_equal 1 diff --git a/dm/tests/case_sensitive/data/db2.prepare.sql b/dm/tests/case_sensitive/data/db2.prepare.sql index 98efbb4b44f..b0e049b679a 100644 --- a/dm/tests/case_sensitive/data/db2.prepare.sql +++ b/dm/tests/case_sensitive/data/db2.prepare.sql @@ -8,6 +8,7 @@ create table Upper_Table ( PRIMARY KEY (id)); insert into Upper_Table (name, ts) values ('Arya', now()), ('Bran', '2021-05-11 10:01:05'), ('Sansa', NULL); +-- if case-insensitive, this should report conflict with Upper_Table create table upper_table(id int NOT NULL PRIMARY KEY); -- test block-allow-list diff --git a/dm/tests/case_sensitive/run.sh b/dm/tests/case_sensitive/run.sh index 3fd1fdaea3d..1beeff327f8 100755 --- a/dm/tests/case_sensitive/run.sh +++ b/dm/tests/case_sensitive/run.sh @@ -7,7 +7,22 @@ source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME API_VERSION="v1alpha1" -function run() { +function prepare_sensitive_task() { + cp $cur/data/db1.prepare.sql $WORK_DIR/db1.prepare.sql + cp $cur/data/db2.prepare.sql $WORK_DIR/db2.prepare.sql + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml +} + +function prepare_insensitive_task() { + cp $cur/data/db1.prepare.sql $WORK_DIR/db1.prepare.sql + cp $cur/data/db2.prepare.sql $WORK_DIR/db2.prepare.sql + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml + + sed -i "/sensitive/d" $WORK_DIR/dm-task.yaml + sed -i "/create table upper_table/d" $WORK_DIR/db2.prepare.sql +} + +function run_with_prepared() { run_sql_both_source "SET @@GLOBAL.SQL_MODE='ANSI_QUOTES,NO_AUTO_VALUE_ON_ZERO'" inject_points=( "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" @@ -15,9 +30,9 @@ function run() { ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" - run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $WORK_DIR/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' - run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + run_sql_file $WORK_DIR/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_contains 'Query OK, 3 rows affected' # manually create the route table run_sql 'CREATE DATABASE IF NOT EXISTS `UPPER_DB_ROUTE`' $TIDB_PORT $TIDB_PASSWORD @@ -40,7 +55,6 @@ function run() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # start DM task only - cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" # check task has started check_metric $WORKER1_PORT "dm_worker_task_state{source_id=\"mysql-replica-01\",task=\"test\",worker=\"worker1\"}" 10 1 3 @@ -82,7 +96,6 @@ function run() { # test block-allow-list by the way run_sql "show databases;" $TIDB_PORT $TIDB_PASSWORD - check_not_contains "Upper_Db_IGNORE" check_contains "Upper_DB1" check_contains "lower_db" # test route-rule @@ -90,7 +103,6 @@ function run() { run_sql "show tables from UPPER_DB_ROUTE" $TIDB_PORT $TIDB_PASSWORD check_contains "do_table_route" - check_not_contains "Do_table_ignore" run_sql_tidb_with_retry "select count(*) from UPPER_DB_ROUTE.do_table_route" "count(*): 5" # test binlog event filter @@ -101,16 +113,37 @@ function run() { # ensure the truncate is ignored and the new row is inserted run_sql_tidb_with_retry "select count(*) from UPPER_DB_ROUTE.do_table_route" "count(*): 6" + dmctl_stop_task test + dmctl_operate_source stop $WORK_DIR/source1.yaml $SOURCE_ID1 + dmctl_operate_source stop $WORK_DIR/source2.yaml $SOURCE_ID2 + export GO_FAILPOINTS='' } +function check_ignore_when_sensitive() { + run_sql "show databases;" $TIDB_PORT $TIDB_PASSWORD + check_not_contains "Upper_Db_IGNORE" + run_sql "show tables from UPPER_DB_ROUTE" $TIDB_PORT $TIDB_PASSWORD + check_not_contains "Do_table_ignore" +} + trap cleanup_process EXIT -trap "cleanup_data Upper_DB Upper_DB1 lower_db UPPER_DB_ROUTE sync_diff_inspector" EXIT +trap "cleanup_data Upper_DB Upper_DB1 lower_db UPPER_DB_ROUTE Upper_Db_IGNORE sync_diff_inspector" EXIT # also cleanup dm processes in case of last run failed cleanup_process $* -cleanup_data Upper_DB Upper_DB1 lower_db UPPER_DB_ROUTE -run +cleanup_data Upper_DB Upper_DB1 lower_db UPPER_DB_ROUTE Upper_Db_IGNORE + +prepare_sensitive_task +run_with_prepared +check_ignore_when_sensitive + +cleanup_process $* +cleanup_data Upper_DB Upper_DB1 lower_db UPPER_DB_ROUTE Upper_Db_IGNORE + +prepare_insensitive_task +run_with_prepared + cleanup_process $* echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" diff --git a/dm/tests/dmctl_basic/conf/dm-task.yaml b/dm/tests/dmctl_basic/conf/dm-task.yaml index ed9f54cfeaf..b9ea640e7f1 100644 --- a/dm/tests/dmctl_basic/conf/dm-task.yaml +++ b/dm/tests/dmctl_basic/conf/dm-task.yaml @@ -39,6 +39,8 @@ block-allow-list: do-tables: - db-name: "dmctl" tbl-name: "~^t_[\\d]+" + - db-name: "dmctl" + tbl-name: "flush_trigger" routes: sharding-route-rules-table: diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index e26a00dae17..c7f79655a9b 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -123,6 +123,8 @@ block-allow-list: do-tables: - db-name: dmctl tbl-name: ~^t_[\d]+ + - db-name: dmctl + tbl-name: flush_trigger do-dbs: - dmctl ignore-tables: [] diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index 25b56a061f8..22c361f3074 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -379,12 +379,12 @@ function run() { # make sure every shard table in source 1 has be forwarded to newer binlog, so older relay log could be purged run_sql_source1 "flush logs" + run_sql_source1 "create table dmctl.flush_trigger (c int primary key);" run_sql_source1 "update dmctl.t_1 set d = '' where id = 13" run_sql_source1 "update dmctl.t_2 set d = '' where id = 12" - # sleep 2*1s to ensure syncer unit has flushed global checkpoint and updates - # updated ActiveRelayLog - sleep 2 + # sleep to ensure syncer unit has resumed, read next binlog files and updated ActiveRelayLog + sleep 5 server_uuid=$(tail -n 1 $WORK_DIR/worker1/relay_log/server-uuid.index) run_sql_source1 "show binary logs\G" max_binlog_name=$(grep Log_name "$SQL_RESULT_FILE" | tail -n 1 | awk -F":" '{print $NF}') diff --git a/dm/tests/expression_filter/run.sh b/dm/tests/expression_filter/run.sh index 1b33c27051a..c5662e6ae89 100755 --- a/dm/tests/expression_filter/run.sh +++ b/dm/tests/expression_filter/run.sh @@ -19,11 +19,13 @@ function complex_behaviour() { run_sql_file $cur/data/db1.prepare2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 - # test about schema-tracker can't create its storage + # test no permission chmod -w $WORK_DIR/worker1 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $cur/conf/dm-task2.yaml" \ - "failed to create schema tracker" 1 \ + "start-task $cur/conf/dm-task2.yaml" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "dumpling runs with error" 1 \ "permission denied" 1 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" diff --git a/dm/tests/handle_error_3/run.sh b/dm/tests/handle_error_3/run.sh index f0726acd0da..4cb30f7c34e 100644 --- a/dm/tests/handle_error_3/run.sh +++ b/dm/tests/handle_error_3/run.sh @@ -187,6 +187,10 @@ function DM_4193_CASE() { "binlog revert test -s $source2 -b $first_name2:$second_pos2" \ "operator not exist" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + '"stage": "Paused"' 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "binlog skip test" \ "\"result\": true" 3 diff --git a/dm/tests/many_tables/conf/source1.yaml b/dm/tests/many_tables/conf/source1.yaml index 0018c1b61e3..04cc575da29 100644 --- a/dm/tests/many_tables/conf/source1.yaml +++ b/dm/tests/many_tables/conf/source1.yaml @@ -10,4 +10,5 @@ from: password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= port: 3306 checker: - check-enable: false + check-enable: true + backoff-max: 1s diff --git a/dm/tests/many_tables/run.sh b/dm/tests/many_tables/run.sh index 4291c34c624..1ec0154ad7d 100644 --- a/dm/tests/many_tables/run.sh +++ b/dm/tests/many_tables/run.sh @@ -27,6 +27,13 @@ function incremental_data() { done } +function incremental_data_2() { + j=6 + for i in $(seq $TABLE_NUM); do + run_sql "INSERT INTO many_tables_db.t$i VALUES ($j,${j}000$j);" $MYSQL_PORT1 $MYSQL_PASSWORD1 + done +} + function run() { echo "start prepare_data" prepare_data @@ -69,8 +76,36 @@ function run() { echo "start incremental_data" incremental_data echo "finish incremental_data" - check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + # test https://github.com/pingcap/tiflow/issues/5344 + kill_dm_worker + # let some binlog event save table checkpoint before meet downstream error + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1)' + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + run_sql_source1 "CREATE TABLE many_tables_db.flush (c INT PRIMARY KEY);" + sleep 5 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + '"synced": true' 1 + + pkill -hup tidb-server 2>/dev/null || true + wait_process_exit tidb-server + # now worker will process some binlog events, save table checkpoint and meet downstream error + incremental_data_2 + sleep 30 + + resume_num=$(grep 'unit process error' $WORK_DIR/worker1/log/dm-worker.log | wc -l) + echo "resume_num: $resume_num" + # because we check auto resume every 5 seconds... + [ $resume_num -ge 4 ] + folder_size=$(du -d0 $WORK_DIR/worker1/ --exclude="$WORK_DIR/worker1/log" | cut -f1) + echo "folder_size: $folder_size" + # less than 10M + [ $folder_size -lt 10000 ] + + export GO_FAILPOINTS='' } cleanup_data many_tables_db diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index 42f0fda61ce..f4162ba1db4 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -245,6 +245,8 @@ function run() { check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + # wait syncer begin to sync so it has deleted load task etcd KV. + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # relay task tranfer to worker1 with no error. check_metric $WORKER1_PORT "dm_relay_data_corruption" 3 -1 1 diff --git a/dm/tests/openapi/client/openapi_source_check b/dm/tests/openapi/client/openapi_source_check index 5b7c1dfd0c1..fd205281334 100755 --- a/dm/tests/openapi/client/openapi_source_check +++ b/dm/tests/openapi/client/openapi_source_check @@ -1,6 +1,7 @@ #!/usr/bin/env python import sys import requests +import ssl SOURCE1_NAME = "mysql-01" SOURCE2_NAME = "mysql-02" @@ -11,6 +12,10 @@ WORKER2_NAME = "worker2" API_ENDPOINT = "http://127.0.0.1:8261/api/v1/sources" API_ENDPOINT_NOT_LEADER = "http://127.0.0.1:8361/api/v1/sources" +API_ENDPOINT_HTTPS = "https://127.0.0.1:8261/api/v1/sources" +API_ENDPOINT_NOT_LEADER_HTTPS = "https://127.0.0.1:8361/api/v1/sources" + + def create_source_failed(): resp = requests.post(url=API_ENDPOINT) @@ -47,6 +52,19 @@ def create_source2_success(): print("create_source1_success resp=", resp.json()) assert resp.status_code == 201 +def create_source_success_https(ssl_ca, ssl_cert, ssl_key): + req = { + "case_sensitive": False, + "enable_gtid": False, + "host": "127.0.0.1", + "password": "123456", + "port": 3306, + "source_name": SOURCE1_NAME, + "user": "root", + } + resp = requests.post(url=API_ENDPOINT_HTTPS, json=req, verify=ssl_ca, cert=(ssl_cert, ssl_key)) + print("create_source_success_https resp=", resp.json()) + assert resp.status_code == 201 def list_source_success(source_count): resp = requests.get(url=API_ENDPOINT) @@ -55,6 +73,12 @@ def list_source_success(source_count): print("list_source_by_openapi_success resp=", data) assert data["total"] == int(source_count) +def list_source_success_https(source_count, ssl_ca, ssl_cert, ssl_key): + resp = requests.get(url=API_ENDPOINT_HTTPS, verify=ssl_ca, cert=(ssl_cert, ssl_key)) + assert resp.status_code == 200 + data = resp.json() + print("list_source_success_https resp=", data) + assert data["total"] == int(source_count) def list_source_with_status_success(source_count, status_count): resp = requests.get(url=API_ENDPOINT + "?with_status=true") @@ -66,13 +90,19 @@ def list_source_with_status_success(source_count, status_count): assert len(data["data"][i]["status_list"]) == int(status_count) -def list_source_with_redirect(source_count): +def list_source_with_reverse(source_count): resp = requests.get(url=API_ENDPOINT_NOT_LEADER) assert resp.status_code == 200 data = resp.json() - print("list_source_by_openapi_redirect resp=", data) + print("list_source_with_reverse resp=", data) assert data["total"] == int(source_count) +def list_source_with_reverse_https(source_count, ssl_ca, ssl_cert, ssl_key): + resp = requests.get(url=API_ENDPOINT_NOT_LEADER_HTTPS, verify=ssl_ca, cert=(ssl_cert, ssl_key)) + assert resp.status_code == 200 + data = resp.json() + print("list_source_with_reverse_https resp=", data) + assert data["total"] == int(source_count) def delete_source_success(source_name): resp = requests.delete(url=API_ENDPOINT + "/" + source_name) @@ -215,8 +245,11 @@ if __name__ == "__main__": "create_source_failed": create_source_failed, "create_source1_success": create_source1_success, "create_source2_success": create_source2_success, + "create_source_success_https": create_source_success_https, "list_source_success": list_source_success, - "list_source_with_redirect": list_source_with_redirect, + "list_source_success_https": list_source_success_https, + "list_source_with_reverse_https": list_source_with_reverse_https, + "list_source_with_reverse": list_source_with_reverse, "list_source_with_status_success": list_source_with_status_success, "delete_source_failed": delete_source_failed, "delete_source_success": delete_source_success, diff --git a/dm/tests/openapi/client/openapi_task_check b/dm/tests/openapi/client/openapi_task_check index ac763a8aa36..04824830cb4 100755 --- a/dm/tests/openapi/client/openapi_task_check +++ b/dm/tests/openapi/client/openapi_task_check @@ -239,17 +239,7 @@ def operate_schema_and_table_success(task_name, source_name, schema_name, table_ assert create_table["schema_name"] == schema_name assert table_name in create_table["schema_create_sql"] - # delete table - resp = requests.delete(url=single_table_url) - assert resp.status_code == 204 - - # after delete, no table in schema - resp = requests.get(url=table_url) - assert resp.status_code == 200 - print("get_task_schema_success table resp=", resp.json()) - assert len(resp.json()) == 0 - - # add table back again + # overwrite table set_table_data = { "sql_content": "CREATE TABLE openapi.t1(i TINYINT, j INT UNIQUE KEY);", "flush": True, diff --git a/dm/tests/openapi/run.sh b/dm/tests/openapi/run.sh index 969f4f767b6..87bcc6986e3 100644 --- a/dm/tests/openapi/run.sh +++ b/dm/tests/openapi/run.sh @@ -67,7 +67,7 @@ function test_source() { openapi_source_check "delete_source_failed" "mysql-01" # send request to not leader node - openapi_source_check "list_source_with_redirect" 0 + openapi_source_check "list_source_with_reverse" 0 echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: SOURCE SUCCESS" } @@ -294,6 +294,62 @@ function test_noshard_task() { echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: NO SHARD TASK SUCCESS" } +function test_reverse_https() { + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: REVERSE HTTPS" + cleanup_data openapi + cleanup_process + + cp $cur/tls_conf/dm-master1.toml $WORK_DIR/ + cp $cur/tls_conf/dm-master2.toml $WORK_DIR/ + cp $cur/tls_conf/dm-worker1.toml $WORK_DIR/ + cp $cur/tls_conf/dm-worker2.toml $WORK_DIR/ + sed -i "s%dir-placeholer%$cur\/tls_conf%g" $WORK_DIR/dm-master1.toml + sed -i "s%dir-placeholer%$cur\/tls_conf%g" $WORK_DIR/dm-master2.toml + sed -i "s%dir-placeholer%$cur\/tls_conf%g" $WORK_DIR/dm-worker1.toml + sed -i "s%dir-placeholer%$cur\/tls_conf%g" $WORK_DIR/dm-worker2.toml + + # run dm-master1 + run_dm_master $WORK_DIR/master1 $MASTER_PORT1 $WORK_DIR/dm-master1.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + # join master2 + run_dm_master $WORK_DIR/master2 $MASTER_PORT2 $WORK_DIR/dm-master2.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT2 "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + # run dm-worker1 + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $WORK_DIR/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + # run dm-worker2 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $WORK_DIR/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + + prepare_database + # create source successfully + openapi_source_check "create_source_success_https" "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + + # get source list success + openapi_source_check "list_source_success_https" 1 "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + + # send request to not leader node + openapi_source_check "list_source_with_reverse_https" 1 "$cur/tls_conf/ca.pem" "$cur/tls_conf/dm.pem" "$cur/tls_conf/dm.key" + + cleanup_data openapi + cleanup_process + + # run dm-master1 + run_dm_master $WORK_DIR/master1 $MASTER_PORT1 $cur/conf/dm-master1.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 + # join master2 + run_dm_master $WORK_DIR/master2 $MASTER_PORT2 $cur/conf/dm-master2.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT2 + # run dm-worker1 + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + # run dm-worker2 + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: REVERSE HTTPS" +} + function test_cluster() { # list master and worker node openapi_cluster_check "list_master_success" 2 @@ -333,6 +389,9 @@ function run() { test_shard_task test_noshard_task + test_reverse_https + + # NOTE: this test case MUST running at last, because it will offline some members of cluster test_cluster } diff --git a/dm/tests/openapi/tls_conf/ca.pem b/dm/tests/openapi/tls_conf/ca.pem new file mode 100644 index 00000000000..9fc215fa83b --- /dev/null +++ b/dm/tests/openapi/tls_conf/ca.pem @@ -0,0 +1,8 @@ +-----BEGIN CERTIFICATE----- +MIIBGDCBwAIJAOjYXLFw5V1HMAoGCCqGSM49BAMCMBQxEjAQBgNVBAMMCWxvY2Fs +aG9zdDAgFw0yMDAzMTcxMjAwMzNaGA8yMjkzMTIzMTEyMDAzM1owFDESMBAGA1UE +AwwJbG9jYWxob3N0MFkwEwYHKoZIzj0CAQYIKoZIzj0DAQcDQgAEglCIJD8uVBfD +kuM+UQP+VA7Srbz17WPLA0Sqc+sQ2p6fT6HYKCW60EXiZ/yEC0925iyVbXEEbX4J +xCc2Heow5TAKBggqhkjOPQQDAgNHADBEAiAILL3Zt/3NFeDW9c9UAcJ9lc92E0ZL +GNDuH6i19Fex3wIgT0ZMAKAFSirGGtcLu0emceuk+zVKjJzmYbsLdpj/JuQ= +-----END CERTIFICATE----- diff --git a/dm/tests/openapi/tls_conf/dm-master1.toml b/dm/tests/openapi/tls_conf/dm-master1.toml new file mode 100644 index 00000000000..477916bf8a9 --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm-master1.toml @@ -0,0 +1,18 @@ +# Master Configuration. +advertise-addr = "127.0.0.1:8261" +initial-cluster = "master1=https://127.0.0.1:8291" +master-addr = ":8261" +name = "master1" +peer-urls = "127.0.0.1:8291" + +ssl-ca = "dir-placeholer/ca.pem" +ssl-cert = "dir-placeholer/dm.pem" +ssl-key = "dir-placeholer/dm.key" +cert-allowed-cn = ["dm"] +auto-compaction-retention = "3s" + +[experimental] +openapi = true + + + diff --git a/dm/tests/openapi/tls_conf/dm-master2.toml b/dm/tests/openapi/tls_conf/dm-master2.toml new file mode 100644 index 00000000000..8605fc1c853 --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm-master2.toml @@ -0,0 +1,15 @@ +# Master Configuration. +name = "master2" +master-addr = ":8361" +advertise-addr = "127.0.0.1:8361" +peer-urls = "http://127.0.0.1:8292" +join = "127.0.0.1:8261" + +ssl-ca = "dir-placeholer/ca.pem" +ssl-cert = "dir-placeholer/dm.pem" +ssl-key = "dir-placeholer/dm.key" +cert-allowed-cn = ["dm"] +auto-compaction-retention = "3s" + +[experimental] +openapi = true diff --git a/dm/tests/openapi/tls_conf/dm-worker1.toml b/dm/tests/openapi/tls_conf/dm-worker1.toml new file mode 100644 index 00000000000..7e57bcf2744 --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm-worker1.toml @@ -0,0 +1,7 @@ +name = "worker1" +join = "127.0.0.1:8261" + +ssl-ca = "dir-placeholer/ca.pem" +ssl-cert = "dir-placeholer/dm.pem" +ssl-key = "dir-placeholer/dm.key" +cert-allowed-cn = ["dm"] \ No newline at end of file diff --git a/dm/tests/openapi/tls_conf/dm-worker2.toml b/dm/tests/openapi/tls_conf/dm-worker2.toml new file mode 100644 index 00000000000..96301e0cffb --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm-worker2.toml @@ -0,0 +1,7 @@ +name = "worker2" +join = "127.0.0.1:8261" + +ssl-ca = "dir-placeholer/ca.pem" +ssl-cert = "dir-placeholer/dm.pem" +ssl-key = "dir-placeholer/dm.key" +cert-allowed-cn = ["dm"] \ No newline at end of file diff --git a/dm/tests/openapi/tls_conf/dm.key b/dm/tests/openapi/tls_conf/dm.key new file mode 100644 index 00000000000..dfdc077bc4d --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm.key @@ -0,0 +1,8 @@ +-----BEGIN EC PARAMETERS----- +BggqhkjOPQMBBw== +-----END EC PARAMETERS----- +-----BEGIN EC PRIVATE KEY----- +MHcCAQEEICF/GDtVxhTPTP501nOu4jgwGSDY01xN+61xd9MfChw+oAoGCCqGSM49 +AwEHoUQDQgAEgQOv5bQO7xK16vZWhwJqlz2vl19+AXW2Ql7KQyGiBJVSvLbyDLOr +kIeFlHN04iqQ39SKSOSfeGSfRt6doU6IcA== +-----END EC PRIVATE KEY----- diff --git a/dm/tests/openapi/tls_conf/dm.pem b/dm/tests/openapi/tls_conf/dm.pem new file mode 100644 index 00000000000..d4f846e3a22 --- /dev/null +++ b/dm/tests/openapi/tls_conf/dm.pem @@ -0,0 +1,10 @@ +-----BEGIN CERTIFICATE----- +MIIBZDCCAQqgAwIBAgIJAIT/lgXUc1JqMAoGCCqGSM49BAMCMBQxEjAQBgNVBAMM +CWxvY2FsaG9zdDAgFw0yMDAzMTcxMjAwMzNaGA8yMjkzMTIzMTEyMDAzM1owDTEL +MAkGA1UEAwwCZG0wWTATBgcqhkjOPQIBBggqhkjOPQMBBwNCAASBA6/ltA7vErXq +9laHAmqXPa+XX34BdbZCXspDIaIElVK8tvIMs6uQh4WUc3TiKpDf1IpI5J94ZJ9G +3p2hTohwo0owSDAaBgNVHREEEzARgglsb2NhbGhvc3SHBH8AAAEwCwYDVR0PBAQD +AgWgMB0GA1UdJQQWMBQGCCsGAQUFBwMCBggrBgEFBQcDATAKBggqhkjOPQQDAgNI +ADBFAiEAx6ljJ+tNa55ypWLGNqmXlB4UdMmKmE4RSKJ8mmEelfECIG2ZmCE59rv5 +wImM6KnK+vM2QnEiISH3PeYyyRzQzycu +-----END CERTIFICATE----- diff --git a/dm/tests/relay_interrupt/run.sh b/dm/tests/relay_interrupt/run.sh index 4ba58be24a9..8ffa6de84c6 100644 --- a/dm/tests/relay_interrupt/run.sh +++ b/dm/tests/relay_interrupt/run.sh @@ -94,8 +94,8 @@ function run() { "start-task $task_conf" \ "\"result\": false" 1 \ "subtasks with name test for sources \[mysql-replica-01\] already exist" 1 - - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + # wait relay unit up + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "\"binlogType\": \"local\"" 1 diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index d414646e899..b6cc06ccbe6 100755 --- a/dm/tests/sequence_sharding_optimistic/run.sh +++ b/dm/tests/sequence_sharding_optimistic/run.sh @@ -71,8 +71,9 @@ run() { "\"stage\": \"Paused\"" 2 # try to get schema for the table, but can't get because no DDL/DML replicated yet. - curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 + # uncomment it after #5824 + # curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log + # check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c1` varchar(20) DEFAULT NULL, `c2` varchar(20) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # resume task manually. run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -164,17 +165,13 @@ run() { # drop the schema. curl -X PUT ${API_URL} -d '{"op":3, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/remove_schema.log - # try to get schema again, but can't get. - curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 - # try to set an invalid schema. curl -X PUT ${API_URL} -d '{"op":2, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1", "schema":"invalid create table statement"}' >${WORK_DIR}/get_schema.log >${WORK_DIR}/set_schema.log check_log_contains ${WORK_DIR}/set_schema.log 'is not a valid `CREATE TABLE` statement' 1 - # try to get schema again, no one exist. + # try to get schema again, still the old schema. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c2` varchar(20) DEFAULT NULL, `c3` int(11) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "binlog-schema -s mysql-replica-01,mysql-replica-02 sequence_sharding_optimistic sharding_seq_opt t2" \ diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 290e573a54c..9e6bee3d682 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -594,8 +594,8 @@ function DM_COMPACT_CASE() { function DM_COMPACT() { # mock downstream has a high latency and upstream has a high workload ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 + check_process_exit worker1 20 + check_process_exit worker2 20 export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(5)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -637,8 +637,8 @@ function DM_COMPACT_USE_DOWNSTREAM_SCHEMA_CASE() { function DM_COMPACT_USE_DOWNSTREAM_SCHEMA() { # downstream pk/uk/column is diffrent with upstream, compact use downstream schema. ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 + check_process_exit worker1 20 + check_process_exit worker2 20 # DownstreamIdentifyKeyCheckInCompact=return(20) will check whether the key value in compact is less than 20, if false, it will be panic. # This goal is check whether it use downstream schema in compator. # if use downstream schema, key will be 'b' with value less than 20. @@ -716,8 +716,8 @@ function DM_MULTIPLE_ROWS_CASE() { function DM_MULTIPLE_ROWS() { ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 + check_process_exit worker1 20 + check_process_exit worker2 20 export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(5)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -729,8 +729,8 @@ function DM_MULTIPLE_ROWS() { "clean_table" "" ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 + check_process_exit worker1 20 + check_process_exit worker2 20 export GO_FAILPOINTS='' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -796,8 +796,8 @@ function DM_DML_EXECUTE_ERROR_CASE() { function DM_DML_EXECUTE_ERROR() { ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true - check_port_offline $WORKER1_PORT 20 - check_port_offline $WORKER2_PORT 20 + check_process_exit worker1 20 + check_process_exit worker2 20 export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ErrorOnLastDML=return()' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml diff --git a/dm/tests/shardddl4/run.sh b/dm/tests/shardddl4/run.sh index 0bd6426a434..9ff6582f08f 100644 --- a/dm/tests/shardddl4/run.sh +++ b/dm/tests/shardddl4/run.sh @@ -422,6 +422,11 @@ function DM_130_CASE() { run_sql_source2 "insert into ${shardddl1}.${tb1} values(5,5);" run_sql_source2 "insert into ${shardddl1}.${tb2} values(6,6);" + if [[ "$1" = "optimistic" ]]; then + check_log_contain_with_retry "finish to handle ddls in optimistic shard mode.*alter table ${shardddl1}.${tb1} modify b int default 0" \ + $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + fi + run_sql_source2 "alter table ${shardddl1}.${tb1} modify b int default -1;" run_sql_source1 "insert into ${shardddl1}.${tb1}(a) values(7);" run_sql_source2 "insert into ${shardddl1}.${tb1}(a) values(8);" diff --git a/dm/tests/start_task/run.sh b/dm/tests/start_task/run.sh index 90e51cb24f7..2e79fc57933 100644 --- a/dm/tests/start_task/run.sh +++ b/dm/tests/start_task/run.sh @@ -55,8 +55,8 @@ function lazy_init_tracker() { done check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 20 - check_log_contains $WORK_DIR/worker1/log/dm-worker.log 'lazy init table info.*t50' 1 - check_log_not_contains $WORK_DIR/worker1/log/dm-worker.log 'lazy init table info.*t51' + check_log_contains $WORK_DIR/worker1/log/dm-worker.log 'init table info.*t50' 1 + check_log_not_contains $WORK_DIR/worker1/log/dm-worker.log 'init table info.*t51' cleanup_data start_task cleanup_process diff --git a/dm/tests/tracker_ignored_ddl/run.sh b/dm/tests/tracker_ignored_ddl/run.sh index 90b9c13869b..9d66f154db0 100644 --- a/dm/tests/tracker_ignored_ddl/run.sh +++ b/dm/tests/tracker_ignored_ddl/run.sh @@ -27,11 +27,18 @@ function run() { check_not_contains "ignore_1" echo "increment1 check success" + # a not ignored DDL to trigger a checkpoint flush + run_sql_source1 "create table tracker_ignored_ddl.test (c int primary key);" + run_sql_file $cur/data/db.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "Error 1054: Unknown column" 1 + # force a resume, the error is still there, but we want to check https://github.com/pingcap/tiflow/issues/5272#issuecomment-1109283279 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" + # need operate tidb run_sql_tidb "alter table $TEST_NAME.t1 add column ignore_1 int;" @@ -39,7 +46,7 @@ function run() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "resume-task test" \ "\"result\": true" 2 - sleep 1 + sleep 3 run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ "\"stage\": \"Running\"" 1 diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 550f4c5d5a7..06e42d60b80 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -2783,7 +2783,7 @@ "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", + "legendFormat": "0-2 txn/m worker", "refId": "C" }, { @@ -2792,7 +2792,7 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "2-10 row/s worker", + "legendFormat": "2-10 txn/m worker", "refId": "D" }, { @@ -2801,7 +2801,7 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "10-100 row/s worker", + "legendFormat": "10-100 txn/m worker", "refId": "E" }, { @@ -2810,7 +2810,7 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": ">100 row/s worker", + "legendFormat": ">100 txn/m worker", "refId": "F" } ], diff --git a/pkg/uuid/mock.go b/pkg/uuid/mock.go new file mode 100644 index 00000000000..c3055a8e1d7 --- /dev/null +++ b/pkg/uuid/mock.go @@ -0,0 +1,56 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package uuid + +import "github.com/pingcap/log" + +// MockGenerator is a mocked uuid generator +type MockGenerator struct { + list []string +} + +// NewMock creates a new MockGenerator instance +func NewMock() *MockGenerator { + return &MockGenerator{} +} + +// NewString implements Generator.NewString +func (g *MockGenerator) NewString() (ret string) { + if len(g.list) == 0 { + log.L().Panic("Empty uuid list. Please use Push() to add a uuid to the list.") + } + + ret, g.list = g.list[0], g.list[1:] + return +} + +// Push adds a candidate uuid in FIFO list +func (g *MockGenerator) Push(uuid string) { + g.list = append(g.list, uuid) +} + +// ConstGenerator is a mocked uuid generator, which always generate a pre defined uuid +type ConstGenerator struct { + uid string +} + +// NewConstGenerator creates a new ConstGenerator instance +func NewConstGenerator(uid string) *ConstGenerator { + return &ConstGenerator{uid: uid} +} + +// NewString implements Generator.NewString +func (g *ConstGenerator) NewString() string { + return g.uid +} diff --git a/pkg/uuid/uuid.go b/pkg/uuid/uuid.go new file mode 100644 index 00000000000..f542ff87282 --- /dev/null +++ b/pkg/uuid/uuid.go @@ -0,0 +1,32 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package uuid + +import guuid "github.com/google/uuid" + +// Generator defines an interface that can generate a uuid +type Generator interface { + NewString() string +} + +type generatorImpl struct{} + +func (g *generatorImpl) NewString() string { + return guuid.New().String() +} + +// NewGenerator creates a new generatorImpl instance +func NewGenerator() Generator { + return &generatorImpl{} +} diff --git a/pkg/uuid/uuid_test.go b/pkg/uuid/uuid_test.go new file mode 100644 index 00000000000..1513c67185a --- /dev/null +++ b/pkg/uuid/uuid_test.go @@ -0,0 +1,53 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package uuid + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGenerator(t *testing.T) { + t.Parallel() + + gen := NewGenerator() + uuid1 := gen.NewString() + uuid2 := gen.NewString() + require.NotEqual(t, uuid1, uuid2) +} + +func TestMockGenerator(t *testing.T) { + t.Parallel() + + gen := NewMock() + require.Panics(t, func() { gen.NewString() }) + + uuids := []string{"uuid1", "uuid2", "uuid3"} + for _, uid := range uuids { + gen.Push(uid) + } + for _, uid := range uuids { + require.Equal(t, uid, gen.NewString()) + } +} + +func TestConstGenerator(t *testing.T) { + t.Parallel() + + uid := "const-uuid" + gen := NewConstGenerator(uid) + for i := 0; i < 3; i++ { + require.Equal(t, uid, gen.NewString()) + } +} diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index 2f10a4c8b65..b8d82f8d576 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -167,6 +167,9 @@ func (h *defaultEventHandle) SetTimer(ctx context.Context, interval time.Duratio func (h *defaultEventHandle) Unregister() { if !atomic.CompareAndSwapInt32(&h.status, handleRunning, handleCancelled) { + // call synchronize so that the returning of Unregister cannot race + // with the calling of the errorHandler, if an error is already being processed. + h.worker.synchronize() // already cancelled return } diff --git a/pkg/workerpool/pool_test.go b/pkg/workerpool/pool_test.go index 9e2592f89d2..5a0480dd155 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -259,6 +259,36 @@ func TestCancelTimer(t *testing.T) { require.Regexp(t, "context canceled", err) } +func TestErrorAndCancelRace(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + + pool := newDefaultPoolImpl(&defaultHasher{}, 4) + errg, ctx := errgroup.WithContext(ctx) + errg.Go(func() error { + return pool.Run(ctx) + }) + + var racedVar int + handle := pool.RegisterEvent(func(ctx context.Context, event interface{}) error { + return errors.New("fake") + }).OnExit(func(err error) { + time.Sleep(100 * time.Millisecond) + racedVar++ + }) + + err := handle.AddEvent(ctx, 0) + require.NoError(t, err) + + time.Sleep(50 * time.Millisecond) + handle.Unregister() + racedVar++ + + cancel() + err = errg.Wait() + require.Regexp(t, "context canceled", err) +} + func TestTimer(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() diff --git a/tests/integration_tests/http_api/run.sh b/tests/integration_tests/http_api/run.sh index ecd8a8bab2e..fde461425bc 100644 --- a/tests/integration_tests/http_api/run.sh +++ b/tests/integration_tests/http_api/run.sh @@ -15,7 +15,7 @@ function run() { return fi - sudo pip install -U requests==2.26.0 + sudo python3 -m pip install -U requests==2.26.0 rm -rf $WORK_DIR && mkdir -p $WORK_DIR @@ -59,10 +59,10 @@ function run() { SINK_URI="mysql://normal:123456@127.0.0.1:3306/" - python $CUR/util/test_case.py check_health $TLS_DIR - python $CUR/util/test_case.py get_status $TLS_DIR + python3 $CUR/util/test_case.py check_health $TLS_DIR + python3 $CUR/util/test_case.py get_status $TLS_DIR - python $CUR/util/test_case.py create_changefeed $TLS_DIR "$SINK_URI" + python3 $CUR/util/test_case.py create_changefeed $TLS_DIR "$SINK_URI" # wait for changefeed created sleep 2 @@ -105,7 +105,7 @@ function run() { ) for case in ${sequential_cases[@]}; do - python $CUR/util/test_case.py "$case" $TLS_DIR + python3 $CUR/util/test_case.py "$case" $TLS_DIR done cleanup_process $CDC_BINARY