From ba5bced2d411e0dae27a997592390357fb0ffe48 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 3 Nov 2023 14:15:38 +0800 Subject: [PATCH 01/23] redo(ticdc): fix redo initialization block the owner (#9887) (#9992) close pingcap/tiflow#9886 --- cdc/owner/changefeed.go | 35 +--- cdc/processor/processor.go | 5 +- cdc/processor/processor_test.go | 9 +- cdc/redo/manager.go | 98 ++++----- cdc/redo/manager_test.go | 72 +++---- cdc/redo/meta_manager.go | 194 +++++++++++------- cdc/redo/meta_manager_test.go | 103 ++++++---- .../writer/blackhole/blackhole_log_writer.go | 37 ++-- cdc/redo/writer/factory/factory.go | 21 +- cdc/redo/writer/file/file.go | 4 +- cdc/redo/writer/memory/encoding_worker.go | 2 +- cdc/redo/writer/memory/file_worker.go | 2 +- cdc/redo/writer/memory/mem_log_writer.go | 4 +- cdc/redo/writer/memory/mem_log_writer_test.go | 2 +- cdc/redo/writer/writer.go | 2 +- cdc/scheduler/internal/v3/agent/table.go | 2 +- .../replication/replication_manager_test.go | 4 + cmd/kafka-consumer/main.go | 3 +- pkg/redo/config.go | 14 +- .../integration_tests/changefeed_error/run.sh | 2 +- 20 files changed, 331 insertions(+), 284 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 9b6e0413738..e79b39421fa 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -336,6 +336,12 @@ func (c *changefeed) tick(ctx cdcContext.Context, captures map[model.CaptureID]* default: } + if c.redoMetaMgr.Enabled() { + if !c.redoMetaMgr.Running() { + return nil + } + } + // TODO: pass table checkpointTs when we support concurrent process ddl allPhysicalTables, barrier, err := c.ddlManager.tick(ctx, preCheckpointTs, nil) if err != nil { @@ -578,32 +584,21 @@ LOOP2: } c.observerLastTick = atomic.NewTime(time.Time{}) - stdCtx := contextutil.PutChangefeedIDInCtx(cancelCtx, c.id) - c.redoDDLMgr, err = redo.NewDDLManager(stdCtx, c.state.Info.Config.Consistent, ddlStartTs) - failpoint.Inject("ChangefeedNewRedoManagerError", func() { - err = errors.New("changefeed new redo manager injected error") - }) - if err != nil { - return err - } + c.redoDDLMgr = redo.NewDDLManager(c.id, c.state.Info.Config.Consistent, ddlStartTs) if c.redoDDLMgr.Enabled() { c.wg.Add(1) go func() { defer c.wg.Done() - ctx.Throw(c.redoDDLMgr.Run(stdCtx)) + ctx.Throw(c.redoDDLMgr.Run(cancelCtx)) }() } - c.redoMetaMgr, err = redo.NewMetaManagerWithInit(stdCtx, - c.state.Info.Config.Consistent, checkpointTs) - if err != nil { - return err - } + c.redoMetaMgr = redo.NewMetaManager(c.id, c.state.Info.Config.Consistent, checkpointTs) if c.redoMetaMgr.Enabled() { c.wg.Add(1) go func() { defer c.wg.Done() - ctx.Throw(c.redoMetaMgr.Run(stdCtx)) + ctx.Throw(c.redoMetaMgr.Run(cancelCtx)) }() } log.Info("owner creates redo manager", @@ -765,15 +760,7 @@ func (c *changefeed) cleanupRedoManager(ctx context.Context) { } // when removing a paused changefeed, the redo manager is nil, create a new one if c.redoMetaMgr == nil { - redoMetaMgr, err := redo.NewMetaManager(ctx, c.state.Info.Config.Consistent) - if err != nil { - log.Info("owner creates redo manager for clean fail", - zap.String("namespace", c.id.Namespace), - zap.String("changefeed", c.id.ID), - zap.Error(err)) - return - } - c.redoMetaMgr = redoMetaMgr + c.redoMetaMgr = redo.NewMetaManager(c.id, c.state.Info.Config.Consistent, 0) } err := c.redoMetaMgr.Cleanup(ctx) if err != nil { diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 52b3fdc229f..e9202dc321e 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -670,10 +670,7 @@ func (p *processor) lazyInitImpl(etcdCtx cdcContext.Context) (err error) { } p.changefeed.Info.Config.Sink.TiDBSourceID = sourceID - p.redo.r, err = redo.NewDMLManager(stdCtx, p.changefeed.Info.Config.Consistent) - if err != nil { - return err - } + p.redo.r = redo.NewDMLManager(p.changefeedID, p.changefeed.Info.Config.Consistent) p.redo.name = "RedoManager" p.redo.spawn(stdCtx) diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index 9bc810a6c4b..8db8104b22c 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/entry/schema" "github.com/pingcap/tiflow/cdc/model" @@ -64,20 +63,18 @@ func newProcessor4Test( return nil } - stdCtx := contextutil.PutChangefeedIDInCtx(ctx, changefeedID) if !enableRedo { p.redo.r = redo.NewDisabledDMLManager() } else { tmpDir := t.TempDir() redoDir := fmt.Sprintf("%s/%s", tmpDir, changefeedID) - dmlMgr, err := redo.NewDMLManager(ctx, &config.ConsistentConfig{ + dmlMgr := redo.NewDMLManager(changefeedID, &config.ConsistentConfig{ Level: string(redoPkg.ConsistentLevelEventual), MaxLogSize: redoPkg.DefaultMaxLogSize, FlushIntervalInMs: redoPkg.DefaultFlushIntervalInMs, Storage: "file://" + redoDir, UseFileBackend: false, }) - require.NoError(t, err) p.redo.r = dmlMgr } p.redo.name = "RedoManager" @@ -87,9 +84,9 @@ func newProcessor4Test( p.sinkManager.r, p.sourceManager.r, _ = sinkmanager.NewManagerWithMemEngine( t, changefeedID, state.Info, p.redo.r) p.sinkManager.name = "SinkManager" - p.sinkManager.spawn(stdCtx) + p.sinkManager.spawn(ctx) p.sourceManager.name = "SourceManager" - p.sourceManager.spawn(stdCtx) + p.sourceManager.spawn(ctx) // NOTICE: we have to bind the sourceManager to the sinkManager // otherwise the sinkManager will not receive the resolvedTs. diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 496a03bcfaa..5d76886d5fc 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -19,9 +19,8 @@ import ( "sync/atomic" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/tidb/br/pkg/storage" - "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/redo/common" @@ -66,19 +65,17 @@ func NewDisabledDDLManager() *ddlManager { // NewDDLManager creates a new ddl Manager. func NewDDLManager( - ctx context.Context, cfg *config.ConsistentConfig, ddlStartTs model.Ts, -) (*ddlManager, error) { - logManager, err := newLogManager(ctx, cfg, redo.RedoDDLLogFileType) - if err != nil { - return nil, err - } + changefeedID model.ChangeFeedID, + cfg *config.ConsistentConfig, ddlStartTs model.Ts, +) *ddlManager { + m := newLogManager(changefeedID, cfg, redo.RedoDDLLogFileType) span := spanz.TableIDToComparableSpan(0) - logManager.AddTable(span, ddlStartTs) + m.AddTable(span, ddlStartTs) return &ddlManager{ - logManager: logManager, - // The current fakeSpan is meaningless, find a meaningful sapn in the future. + logManager: m, + // The current fakeSpan is meaningless, find a meaningful span in the future. fakeSpan: span, - }, nil + } } type ddlManager struct { @@ -115,12 +112,12 @@ type DMLManager interface { } // NewDMLManager creates a new dml Manager. -func NewDMLManager(ctx context.Context, cfg *config.ConsistentConfig) (*dmlManager, error) { - logManager, err := newLogManager(ctx, cfg, redo.RedoRowLogFileType) - if err != nil { - return nil, err +func NewDMLManager( + changefeedID model.ChangeFeedID, cfg *config.ConsistentConfig, +) *dmlManager { + return &dmlManager{ + logManager: newLogManager(changefeedID, cfg, redo.RedoRowLogFileType), } - return &dmlManager{logManager: logManager}, nil } // NewDisabledDMLManager creates a disabled dml Manager. @@ -226,28 +223,22 @@ type logManager struct { } func newLogManager( - ctx context.Context, cfg *config.ConsistentConfig, logType string, -) (*logManager, error) { + changefeedID model.ChangeFeedID, + cfg *config.ConsistentConfig, logType string, +) *logManager { // return a disabled Manager if no consistent config or normal consistent level if cfg == nil || !redo.IsConsistentEnabled(cfg.Level) { - return &logManager{enabled: false}, nil + return &logManager{enabled: false} } - uri, err := storage.ParseRawURL(cfg.Storage) - if err != nil { - return nil, err - } - changefeedID := contextutil.ChangefeedIDFromCtx(ctx) - m := &logManager{ + return &logManager{ enabled: true, cfg: &writer.LogWriterConfig{ - ConsistentConfig: *cfg, - LogType: logType, - CaptureID: contextutil.CaptureAddrFromCtx(ctx), - ChangeFeedID: changefeedID, - URI: *uri, - UseExternalStorage: redo.IsExternalStorage(uri.Scheme), - MaxLogSizeInBytes: cfg.MaxLogSize * redo.Megabyte, + ConsistentConfig: *cfg, + LogType: logType, + CaptureID: config.GetGlobalServerConfig().AdvertiseAddr, + ChangeFeedID: changefeedID, + MaxLogSizeInBytes: cfg.MaxLogSize * redo.Megabyte, }, logBuffer: chann.NewAutoDrainChann[cacheEvents](), rtsMap: spanz.SyncMap{}, @@ -260,21 +251,30 @@ func newLogManager( metricRedoWorkerBusyRatio: common.RedoWorkerBusyRatio. WithLabelValues(changefeedID.Namespace, changefeedID.ID), } - - m.writer, err = factory.NewRedoLogWriter(ctx, m.cfg) - if err != nil { - return nil, err - } - return m, nil } // Run implements pkg/util.Runnable. func (m *logManager) Run(ctx context.Context, _ ...chan<- error) error { - if m.Enabled() { - defer m.close() - return m.bgUpdateLog(ctx) + failpoint.Inject("ChangefeedNewRedoManagerError", func() { + failpoint.Return(errors.New("changefeed new redo manager injected error")) + }) + if !m.Enabled() { + return nil } - return nil + + defer m.close() + start := time.Now() + w, err := factory.NewRedoLogWriter(ctx, m.cfg) + if err != nil { + log.Error("redo: failed to create redo log writer", + zap.String("namespace", m.cfg.ChangeFeedID.Namespace), + zap.String("changefeed", m.cfg.ChangeFeedID.ID), + zap.Duration("duration", time.Since(start)), + zap.Error(err)) + return err + } + m.writer = w + return m.bgUpdateLog(ctx) } // WaitForReady implements pkg/util.Runnable. @@ -546,11 +546,13 @@ func (m *logManager) close() { atomic.StoreInt32(&m.closed, 1) m.logBuffer.CloseAndDrain() - if err := m.writer.Close(); err != nil { - log.Error("redo manager fails to close writer", - zap.String("namespace", m.cfg.ChangeFeedID.Namespace), - zap.String("changefeed", m.cfg.ChangeFeedID.ID), - zap.Error(err)) + if m.writer != nil { + if err := m.writer.Close(); err != nil && errors.Cause(err) != context.Canceled { + log.Error("redo manager fails to close writer", + zap.String("namespace", m.cfg.ChangeFeedID.Namespace), + zap.String("changefeed", m.cfg.ChangeFeedID.ID), + zap.Error(err)) + } } log.Info("redo manager closed", zap.String("namespace", m.cfg.ChangeFeedID.Namespace), diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index 8224f6b959d..14db8ffc251 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/redo/writer" - "github.com/pingcap/tiflow/cdc/redo/writer/blackhole" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/spanz" @@ -121,15 +120,11 @@ func TestLogManagerInProcessor(t *testing.T) { FlushIntervalInMs: redo.MinFlushIntervalInMs, UseFileBackend: useFileBackend, } - dmlMgr, err := NewDMLManager(ctx, cfg) - require.NoError(t, err) - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - defer wg.Done() - dmlMgr.Run(ctx) - }() - + dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) + var eg errgroup.Group + eg.Go(func() error { + return dmlMgr.Run(ctx) + }) // check emit row changed events can move forward resolved ts spans := []tablepb.Span{ spanz.TableIDToComparableSpan(53), @@ -202,7 +197,7 @@ func TestLogManagerInProcessor(t *testing.T) { checkResolvedTs(t, dmlMgr.logManager, flushResolvedTs) cancel() - wg.Wait() + require.ErrorIs(t, eg.Wait(), context.Canceled) } testWriteDMLs("blackhole://", true) @@ -233,18 +228,16 @@ func TestLogManagerInOwner(t *testing.T) { UseFileBackend: useFileBackend, } startTs := model.Ts(10) - ddlMgr, err := NewDDLManager(ctx, cfg, startTs) - require.NoError(t, err) - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - defer wg.Done() - ddlMgr.Run(ctx) - }() + ddlMgr := NewDDLManager(model.DefaultChangeFeedID("test"), cfg, startTs) + + var eg errgroup.Group + eg.Go(func() error { + return ddlMgr.Run(ctx) + }) require.Equal(t, startTs, ddlMgr.GetResolvedTs()) ddl := &model.DDLEvent{StartTs: 100, CommitTs: 120, Query: "CREATE TABLE `TEST.T1`"} - err = ddlMgr.EmitDDLEvent(ctx, ddl) + err := ddlMgr.EmitDDLEvent(ctx, ddl) require.NoError(t, err) require.Equal(t, startTs, ddlMgr.GetResolvedTs()) @@ -252,7 +245,7 @@ func TestLogManagerInOwner(t *testing.T) { checkResolvedTs(t, ddlMgr.logManager, ddl.CommitTs) cancel() - wg.Wait() + require.ErrorIs(t, eg.Wait(), context.Canceled) } testWriteDDLs("blackhole://", true) @@ -275,23 +268,14 @@ func TestLogManagerError(t *testing.T) { cfg := &config.ConsistentConfig{ Level: string(redo.ConsistentLevelEventual), MaxLogSize: redo.DefaultMaxLogSize, - Storage: "blackhole://", + Storage: "blackhole-invalid://", FlushIntervalInMs: redo.MinFlushIntervalInMs, } - logMgr, err := NewDMLManager(ctx, cfg) - require.NoError(t, err) - err = logMgr.writer.Close() - require.NoError(t, err) - logMgr.writer = blackhole.NewInvalidLogWriter(logMgr.writer) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - defer wg.Done() - err := logMgr.Run(ctx) - require.Regexp(t, ".*invalid black hole writer.*", err) - require.Regexp(t, ".*WriteLog.*", err) - }() + logMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) + var eg errgroup.Group + eg.Go(func() error { + return logMgr.Run(ctx) + }) testCases := []struct { span tablepb.Span @@ -310,7 +294,10 @@ func TestLogManagerError(t *testing.T) { err := logMgr.emitRedoEvents(ctx, tc.span, nil, tc.rows...) require.NoError(t, err) } - wg.Wait() + + err := eg.Wait() + require.Regexp(t, ".*invalid black hole writer.*", err) + require.Regexp(t, ".*WriteLog.*", err) } func BenchmarkBlackhole(b *testing.B) { @@ -336,9 +323,8 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { FlushIntervalInMs: redo.MinFlushIntervalInMs, UseFileBackend: useFileBackend, } - dmlMgr, err := NewDMLManager(ctx, cfg) - require.Nil(b, err) - eg := errgroup.Group{} + dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) + var eg errgroup.Group eg.Go(func() error { return dmlMgr.Run(ctx) }) @@ -366,7 +352,7 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { go func(span tablepb.Span) { defer wg.Done() maxCommitTs := maxTsMap.GetV(span) - rows := []*model.RowChangedEvent{} + var rows []*model.RowChangedEvent for i := 0; i < maxRowCount; i++ { if i%100 == 0 { // prepare new row change events @@ -409,6 +395,6 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { time.Sleep(time.Millisecond * 500) } cancel() - err = eg.Wait() - require.ErrorIs(b, err, context.Canceled) + + require.ErrorIs(b, eg.Wait(), context.Canceled) } diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index 54f888a13f0..a19086b9bd0 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" - "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/pkg/config" @@ -31,6 +30,7 @@ import ( "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/uuid" "github.com/prometheus/client_golang/prometheus" + "go.uber.org/atomic" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -47,6 +47,9 @@ type MetaManager interface { // Cleanup deletes all redo logs, which are only called from the owner // when changefeed is deleted. Cleanup(ctx context.Context) error + + // Running return true if the meta manager is running or not. + Running() bool } type metaManager struct { @@ -54,6 +57,9 @@ type metaManager struct { changeFeedID model.ChangeFeedID enabled bool + // running means the meta manager now running normally. + running atomic.Bool + metaCheckpointTs statefulRts metaResolvedTs statefulRts @@ -63,8 +69,10 @@ type metaManager struct { uuidGenerator uuid.Generator preMetaFile string + startTs model.Ts + lastFlushTime time.Time - flushIntervalInMs int64 + cfg *config.ConsistentConfig metricFlushLogDuration prometheus.Observer } @@ -75,91 +83,87 @@ func NewDisabledMetaManager() *metaManager { } } -// NewMetaManagerWithInit creates a new Manager and initializes the meta. -func NewMetaManagerWithInit( - ctx context.Context, cfg *config.ConsistentConfig, startTs model.Ts, -) (*metaManager, error) { - m, err := NewMetaManager(ctx, cfg) - if err != nil { - return nil, err - } - - // There is no need to perform initialize operation if metaMgr is disabled - // or the scheme is blackhole. - if m.extStorage != nil { - m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. - WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) - if err = m.preCleanupExtStorage(ctx); err != nil { - log.Warn("pre clean redo logs fail", - zap.String("namespace", m.changeFeedID.Namespace), - zap.String("changefeed", m.changeFeedID.ID), - zap.Error(err)) - return nil, err - } - if err = m.initMeta(ctx, startTs); err != nil { - log.Warn("init redo meta fail", - zap.String("namespace", m.changeFeedID.Namespace), - zap.String("changefeed", m.changeFeedID.ID), - zap.Error(err)) - return nil, err - } - } - - return m, nil -} - // NewMetaManager creates a new meta Manager. -func NewMetaManager(ctx context.Context, cfg *config.ConsistentConfig) (*metaManager, error) { +func NewMetaManager( + changefeedID model.ChangeFeedID, + cfg *config.ConsistentConfig, checkpoint model.Ts, +) *metaManager { // return a disabled Manager if no consistent config or normal consistent level if cfg == nil || !redo.IsConsistentEnabled(cfg.Level) { - return &metaManager{enabled: false}, nil + return &metaManager{enabled: false} } m := &metaManager{ - captureID: contextutil.CaptureAddrFromCtx(ctx), - changeFeedID: contextutil.ChangefeedIDFromCtx(ctx), - uuidGenerator: uuid.NewGenerator(), - enabled: true, - flushIntervalInMs: cfg.FlushIntervalInMs, + captureID: config.GetGlobalServerConfig().AdvertiseAddr, + changeFeedID: changefeedID, + uuidGenerator: uuid.NewGenerator(), + enabled: true, + cfg: cfg, + startTs: checkpoint, } - uri, err := storage.ParseRawURL(cfg.Storage) + return m +} + +// Enabled returns whether this log manager is enabled +func (m *metaManager) Enabled() bool { + return m.enabled +} + +// Running return whether the meta manager is initialized, +// which means the external storage is accessible to the meta manager. +func (m *metaManager) Running() bool { + return m.running.Load() +} + +func (m *metaManager) preStart(ctx context.Context) error { + uri, err := storage.ParseRawURL(m.cfg.Storage) if err != nil { - return nil, err - } - if redo.IsBlackholeStorage(uri.Scheme) { - return m, nil + return err } - // "nfs" and "local" scheme are converted to "file" scheme redo.FixLocalScheme(uri) extStorage, err := redo.InitExternalStorage(ctx, *uri) if err != nil { - return nil, err + return err } m.extStorage = extStorage - return m, nil -} -// Enabled returns whether this log manager is enabled -func (m *metaManager) Enabled() bool { - return m.enabled + m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. + WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + + err = m.preCleanupExtStorage(ctx) + if err != nil { + log.Warn("redo: pre clean redo logs fail", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Error(err)) + return err + } + err = m.initMeta(ctx) + if err != nil { + log.Warn("redo: init redo meta fail", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Error(err)) + return err + } + return nil } // Run runs bgFlushMeta and bgGC. func (m *metaManager) Run(ctx context.Context, _ ...chan<- error) error { - if m.extStorage == nil { - log.Warn("extStorage of redo meta manager is nil, skip running") - return nil + if err := m.preStart(ctx); err != nil { + return err } - eg, egCtx := errgroup.WithContext(ctx) eg.Go(func() error { - return m.bgFlushMeta(egCtx, m.flushIntervalInMs) + return m.bgFlushMeta(egCtx, m.cfg.FlushIntervalInMs) }) eg.Go(func() error { return m.bgGC(egCtx) }) + m.running.Store(true) return eg.Wait() } @@ -192,9 +196,9 @@ func (m *metaManager) GetFlushedMeta() common.LogMeta { return common.LogMeta{CheckpointTs: checkpointTs, ResolvedTs: resolvedTs} } -// initMeta will read the meta file from external storage and initialize the meta -// field of metaManager. -func (m *metaManager) initMeta(ctx context.Context, startTs model.Ts) error { +// initMeta will read the meta file from external storage and +// use it to initialize the meta field of the metaManager. +func (m *metaManager) initMeta(ctx context.Context) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) @@ -202,10 +206,14 @@ func (m *metaManager) initMeta(ctx context.Context, startTs model.Ts) error { } metas := []*common.LogMeta{ - {CheckpointTs: startTs, ResolvedTs: startTs}, + {CheckpointTs: m.startTs, ResolvedTs: m.startTs}, } var toRemoveMetaFiles []string err := m.extStorage.WalkDir(ctx, nil, func(path string, size int64) error { + log.Info("redo: meta manager walk dir", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.String("path", path), zap.Int64("size", size)) // TODO: use prefix to accelerate traverse operation if !strings.HasSuffix(path, redo.MetaEXT) { return nil @@ -213,22 +221,30 @@ func (m *metaManager) initMeta(ctx context.Context, startTs model.Ts) error { toRemoveMetaFiles = append(toRemoveMetaFiles, path) data, err := m.extStorage.ReadFile(ctx, path) - if err != nil && !util.IsNotExistInExtStorage(err) { - return err - } - if len(data) != 0 { - var meta common.LogMeta - _, err = meta.UnmarshalMsg(data) - if err != nil { + if err != nil { + log.Warn("redo: read meta file failed", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.String("path", path), zap.Error(err)) + if !util.IsNotExistInExtStorage(err) { return err } - metas = append(metas, &meta) + return nil + } + var meta common.LogMeta + _, err = meta.UnmarshalMsg(data) + if err != nil { + log.Error("redo: unmarshal meta data failed", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Error(err), zap.ByteString("data", data)) + return err } + metas = append(metas, &meta) return nil }) if err != nil { - return errors.WrapError(errors.ErrRedoMetaInitialize, - errors.Annotate(err, "read meta file fail")) + return errors.WrapError(errors.ErrRedoMetaInitialize, err) } var checkpointTs, resolvedTs uint64 @@ -241,9 +257,16 @@ func (m *metaManager) initMeta(ctx context.Context, startTs model.Ts) error { m.metaResolvedTs.unflushed = resolvedTs m.metaCheckpointTs.unflushed = checkpointTs if err := m.maybeFlushMeta(ctx); err != nil { - return errors.WrapError(errors.ErrRedoMetaInitialize, - errors.Annotate(err, "flush meta file fail")) + return errors.WrapError(errors.ErrRedoMetaInitialize, err) } + + flushedMeta := m.GetFlushedMeta() + log.Info("redo: meta manager flush init meta success", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Uint64("checkpointTs", flushedMeta.CheckpointTs), + zap.Uint64("resolvedTs", flushedMeta.ResolvedTs)) + return util.DeleteFilesInExtStorage(ctx, m.extStorage, toRemoveMetaFiles) } @@ -303,8 +326,19 @@ func (m *metaManager) shouldRemoved(path string, checkPointTs uint64) bool { // deleteAllLogs delete all redo logs and leave a deleted mark. func (m *metaManager) deleteAllLogs(ctx context.Context) error { + // when one changefeed with redo enabled gets deleted, it's extStorage should always be set to not nil + // otherwise it should have already meet panic during changefeed running time. + // the extStorage may be nil in the unit test, so just set the external storage to make unit test happy. if m.extStorage == nil { - return nil + uri, err := storage.ParseRawURL(m.cfg.Storage) + redo.FixLocalScheme(uri) + if err != nil { + return err + } + m.extStorage, err = redo.InitExternalStorage(ctx, *uri) + if err != nil { + return err + } } // Write deleted mark before clean any files. deleteMarker := getDeletedChangefeedMarker(m.changeFeedID) @@ -380,6 +414,10 @@ func (m *metaManager) flush(ctx context.Context, meta common.LogMeta) error { } metaFile := getMetafileName(m.captureID, m.changeFeedID, m.uuidGenerator) if err := m.extStorage.WriteFile(ctx, metaFile, data); err != nil { + log.Error("redo: meta manager flush meta write file failed", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Error(err)) return errors.WrapError(errors.ErrExternalStorageAPI, err) } @@ -390,6 +428,10 @@ func (m *metaManager) flush(ctx context.Context, meta common.LogMeta) error { } err := m.extStorage.DeleteFile(ctx, m.preMetaFile) if err != nil && !util.IsNotExistInExtStorage(err) { + log.Error("redo: meta manager flush meta delete file failed", + zap.String("namespace", m.changeFeedID.Namespace), + zap.String("changefeed", m.changeFeedID.ID), + zap.Error(err)) return errors.WrapError(errors.ErrExternalStorageAPI, err) } } diff --git a/cdc/redo/meta_manager_test.go b/cdc/redo/meta_manager_test.go index 8879e7c4864..478d50fdd36 100644 --- a/cdc/redo/meta_manager_test.go +++ b/cdc/redo/meta_manager_test.go @@ -21,7 +21,6 @@ import ( "testing" "time" - "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/pkg/config" @@ -38,9 +37,7 @@ func TestInitAndWriteMeta(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() captureID := "test-capture" - ctx = contextutil.PutCaptureAddrInCtx(ctx, captureID) changefeedID := model.DefaultChangeFeedID("test-changefeed") - ctx = contextutil.PutChangefeedIDInCtx(ctx, changefeedID) extStorage, uri, err := util.GetTestExtStorage(ctx, t.TempDir()) require.NoError(t, err) @@ -51,17 +48,18 @@ func TestInitAndWriteMeta(t *testing.T) { {CheckpointTs: 8, ResolvedTs: 9}, {CheckpointTs: 9, ResolvedTs: 11}, } - toReomoveFiles := []string{} + + var toRemoveFiles []string for _, meta := range metas { data, err := meta.MarshalMsg(nil) require.NoError(t, err) metaName := getMetafileName(captureID, changefeedID, uuid.NewGenerator()) err = extStorage.WriteFile(ctx, metaName, data) require.NoError(t, err) - toReomoveFiles = append(toReomoveFiles, metaName) + toRemoveFiles = append(toRemoveFiles, metaName) } - // err = extStorage.WriteFile(ctx, getDeletedChangefeedMarker(changefeedID), []byte{}) - notRemoveFiles := []string{} + + var notRemoveFiles []string require.NoError(t, err) for i := 0; i < 10; i++ { fileName := "dummy" + getChangefeedMatcher(changefeedID) + strconv.Itoa(i) @@ -77,11 +75,22 @@ func TestInitAndWriteMeta(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, } - m, err := NewMetaManagerWithInit(ctx, cfg, startTs) - require.NoError(t, err) - require.Equal(t, startTs, m.metaCheckpointTs.getFlushed()) - require.Equal(t, uint64(11), m.metaResolvedTs.getFlushed()) - for _, fileName := range toReomoveFiles { + m := NewMetaManager(changefeedID, cfg, startTs) + + var eg errgroup.Group + eg.Go(func() error { + return m.Run(ctx) + }) + + require.Eventually(t, func() bool { + return startTs == m.metaCheckpointTs.getFlushed() + }, time.Second, 50*time.Millisecond) + + require.Eventually(t, func() bool { + return uint64(11) == m.metaResolvedTs.getFlushed() + }, time.Second, 50*time.Millisecond) + + for _, fileName := range toRemoveFiles { ret, err := extStorage.FileExists(ctx, fileName) require.NoError(t, err) require.False(t, ret, "file %s should be removed", fileName) @@ -92,7 +101,10 @@ func TestInitAndWriteMeta(t *testing.T) { require.True(t, ret, "file %s should not be removed", fileName) } - testWriteMeta(t, m) + testWriteMeta(ctx, t, m) + + cancel() + require.ErrorIs(t, eg.Wait(), context.Canceled) } func TestPreCleanupAndWriteMeta(t *testing.T) { @@ -101,9 +113,7 @@ func TestPreCleanupAndWriteMeta(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() captureID := "test-capture" - ctx = contextutil.PutCaptureAddrInCtx(ctx, captureID) changefeedID := model.DefaultChangeFeedID("test-changefeed") - ctx = contextutil.PutChangefeedIDInCtx(ctx, changefeedID) extStorage, uri, err := util.GetTestExtStorage(ctx, t.TempDir()) require.NoError(t, err) @@ -115,7 +125,7 @@ func TestPreCleanupAndWriteMeta(t *testing.T) { {CheckpointTs: 9, ResolvedTs: 11}, {CheckpointTs: 11, ResolvedTs: 12}, } - toRemoveFiles := []string{} + var toRemoveFiles []string for _, meta := range metas { data, err := meta.MarshalMsg(nil) require.NoError(t, err) @@ -141,24 +151,36 @@ func TestPreCleanupAndWriteMeta(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, } - m, err := NewMetaManagerWithInit(ctx, cfg, startTs) - require.NoError(t, err) - require.Equal(t, startTs, m.metaCheckpointTs.getFlushed()) - require.Equal(t, startTs, m.metaResolvedTs.getFlushed()) + m := NewMetaManager(changefeedID, cfg, startTs) + + var eg errgroup.Group + eg.Go(func() error { + return m.Run(ctx) + }) + + require.Eventually(t, func() bool { + return startTs == m.metaCheckpointTs.getFlushed() + }, time.Second, 50*time.Millisecond) + + require.Eventually(t, func() bool { + return startTs == m.metaResolvedTs.getFlushed() + }, time.Second, 50*time.Millisecond) + for _, fileName := range toRemoveFiles { ret, err := extStorage.FileExists(ctx, fileName) require.NoError(t, err) require.False(t, ret, "file %s should be removed", fileName) } - testWriteMeta(t, m) -} + testWriteMeta(ctx, t, m) -func testWriteMeta(t *testing.T, m *metaManager) { - ctx, cancel := context.WithCancel(context.Background()) + cancel() + require.ErrorIs(t, eg.Wait(), context.Canceled) +} - checkMeta := func(targetCkpts, targetRts uint64) { +func testWriteMeta(ctx context.Context, t *testing.T, m *metaManager) { + checkMeta := func(targetCheckpointTs, targetResolvedTs uint64) { var checkpointTs, resolvedTs uint64 - metas := []*common.LogMeta{} + var metas []*common.LogMeta cnt := 0 m.extStorage.WalkDir(ctx, nil, func(path string, size int64) error { if !strings.HasSuffix(path, redo.MetaEXT) { @@ -175,14 +197,10 @@ func testWriteMeta(t *testing.T, m *metaManager) { }) require.Equal(t, 1, cnt) common.ParseMeta(metas, &checkpointTs, &resolvedTs) - require.Equal(t, targetCkpts, checkpointTs) - require.Equal(t, targetRts, resolvedTs) + require.Equal(t, targetCheckpointTs, checkpointTs) + require.Equal(t, targetResolvedTs, resolvedTs) } - eg := errgroup.Group{} - eg.Go(func() error { - return m.Run(ctx) - }) // test both regressed meta := m.GetFlushedMeta() m.UpdateMeta(1, 2) @@ -208,9 +226,6 @@ func testWriteMeta(t *testing.T, m *metaManager) { return m.metaCheckpointTs.getFlushed() == 16 }, time.Second, 50*time.Millisecond) checkMeta(16, 21) - - cancel() - require.ErrorIs(t, eg.Wait(), context.Canceled) } func TestGCAndCleanup(t *testing.T) { @@ -219,9 +234,7 @@ func TestGCAndCleanup(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() captureID := "test-capture" - ctx = contextutil.PutCaptureAddrInCtx(ctx, captureID) changefeedID := model.DefaultChangeFeedID("test-changefeed") - ctx = contextutil.PutChangefeedIDInCtx(ctx, changefeedID) extStorage, uri, err := util.GetTestExtStorage(ctx, t.TempDir()) require.NoError(t, err) @@ -266,15 +279,21 @@ func TestGCAndCleanup(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, } - m, err := NewMetaManagerWithInit(ctx, cfg, startTs) - require.NoError(t, err) - require.Equal(t, startTs, m.metaCheckpointTs.getFlushed()) - require.Equal(t, startTs, m.metaResolvedTs.getFlushed()) + m := NewMetaManager(changefeedID, cfg, startTs) - eg := errgroup.Group{} + var eg errgroup.Group eg.Go(func() error { return m.Run(ctx) }) + + require.Eventually(t, func() bool { + return startTs == m.metaCheckpointTs.getFlushed() + }, time.Second, 50*time.Millisecond) + + require.Eventually(t, func() bool { + return startTs == m.metaResolvedTs.getFlushed() + }, time.Second, 50*time.Millisecond) + checkGC(startTs) for i := startTs; i <= uint64(maxCommitTs); i++ { diff --git a/cdc/redo/writer/blackhole/blackhole_log_writer.go b/cdc/redo/writer/blackhole/blackhole_log_writer.go index 24cc03bd3df..9a877f262f3 100644 --- a/cdc/redo/writer/blackhole/blackhole_log_writer.go +++ b/cdc/redo/writer/blackhole/blackhole_log_writer.go @@ -26,14 +26,21 @@ var _ writer.RedoLogWriter = (*blackHoleWriter)(nil) // blackHoleSink defines a blackHole storage, it receives events and persists // without any latency -type blackHoleWriter struct{} +type blackHoleWriter struct { + invalid bool +} // NewLogWriter creates a blackHole writer -func NewLogWriter() *blackHoleWriter { - return &blackHoleWriter{} +func NewLogWriter(invalid bool) *blackHoleWriter { + return &blackHoleWriter{ + invalid: invalid, + } } func (bs *blackHoleWriter) WriteEvents(_ context.Context, events ...writer.RedoEvent) (err error) { + if bs.invalid { + return errors.New("[WriteLog] invalid black hole writer") + } if len(events) == 0 { return nil } @@ -45,30 +52,12 @@ func (bs *blackHoleWriter) WriteEvents(_ context.Context, events ...writer.RedoE } func (bs *blackHoleWriter) FlushLog(_ context.Context) error { + if bs.invalid { + return errors.New("[FlushLog] invalid black hole writer") + } return nil } func (bs *blackHoleWriter) Close() error { return nil } - -type invalidBlackHoleWriter struct { - *blackHoleWriter -} - -// NewInvalidLogWriter creates a invalid blackHole writer -func NewInvalidLogWriter(rl writer.RedoLogWriter) *invalidBlackHoleWriter { - return &invalidBlackHoleWriter{ - blackHoleWriter: rl.(*blackHoleWriter), - } -} - -func (ibs *invalidBlackHoleWriter) WriteEvents( - _ context.Context, _ ...writer.RedoEvent, -) (err error) { - return errors.New("[WriteLog] invalid black hole writer") -} - -func (ibs *invalidBlackHoleWriter) FlushLog(_ context.Context) error { - return errors.New("[FlushLog] invalid black hole writer") -} diff --git a/cdc/redo/writer/factory/factory.go b/cdc/redo/writer/factory/factory.go index 7684b3f06f0..796d803ae45 100644 --- a/cdc/redo/writer/factory/factory.go +++ b/cdc/redo/writer/factory/factory.go @@ -15,7 +15,9 @@ package factory import ( "context" + "strings" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/cdc/redo/writer/blackhole" "github.com/pingcap/tiflow/cdc/redo/writer/file" @@ -28,14 +30,23 @@ import ( func NewRedoLogWriter( ctx context.Context, lwCfg *writer.LogWriterConfig, ) (writer.RedoLogWriter, error) { - scheme := lwCfg.URI.Scheme - if !redo.IsValidConsistentStorage(scheme) { - return nil, errors.ErrConsistentStorage.GenWithStackByArgs(scheme) + uri, err := storage.ParseRawURL(lwCfg.Storage) + if err != nil { + return nil, err } - if redo.IsBlackholeStorage(scheme) { - return blackhole.NewLogWriter(), nil + if !redo.IsValidConsistentStorage(uri.Scheme) { + return nil, errors.ErrConsistentStorage.GenWithStackByArgs(uri.Scheme) } + + lwCfg.URI = uri + lwCfg.UseExternalStorage = redo.IsExternalStorage(uri.Scheme) + + if redo.IsBlackholeStorage(uri.Scheme) { + invalid := strings.HasSuffix(uri.Scheme, "invalid") + return blackhole.NewLogWriter(invalid), nil + } + if lwCfg.UseFileBackend { return file.NewLogWriter(ctx, lwCfg) } diff --git a/cdc/redo/writer/file/file.go b/cdc/redo/writer/file/file.go index e895bffe8c6..e12a5df59de 100644 --- a/cdc/redo/writer/file/file.go +++ b/cdc/redo/writer/file/file.go @@ -86,7 +86,7 @@ func NewFileWriter( var extStorage storage.ExternalStorage if cfg.UseExternalStorage { var err error - extStorage, err = redo.InitExternalStorage(ctx, cfg.URI) + extStorage, err = redo.InitExternalStorage(ctx, *cfg.URI) if err != nil { return nil, err } @@ -129,7 +129,7 @@ func NewFileWriter( // if we use S3 as the remote storage, a file allocator can be leveraged to // pre-allocate files for us. // TODO: test whether this improvement can also be applied to NFS. - if cfg.UseExternalStorage { + if w.cfg.UseExternalStorage { w.allocator = fsutil.NewFileAllocator(cfg.Dir, cfg.LogType, cfg.MaxLogSizeInBytes) } diff --git a/cdc/redo/writer/memory/encoding_worker.go b/cdc/redo/writer/memory/encoding_worker.go index 337379f3b50..57b15ce45f3 100644 --- a/cdc/redo/writer/memory/encoding_worker.go +++ b/cdc/redo/writer/memory/encoding_worker.go @@ -124,7 +124,7 @@ func newEncodingWorkerGroup(workerNum int) *encodingWorkerGroup { func (e *encodingWorkerGroup) Run(ctx context.Context) (err error) { defer func() { close(e.closed) - if err != nil { + if err != nil && errors.Cause(err) != context.Canceled { log.Warn("redo fileWorkerGroup closed with error", zap.Error(err)) } }() diff --git a/cdc/redo/writer/memory/file_worker.go b/cdc/redo/writer/memory/file_worker.go index 771888a418d..45b366117c0 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -138,7 +138,7 @@ func (f *fileWorkerGroup) Run( ) (err error) { defer func() { f.close() - if err != nil { + if err != nil && errors.Cause(err) != context.Canceled { log.Warn("redo file workers closed with error", zap.Error(err)) } }() diff --git a/cdc/redo/writer/memory/mem_log_writer.go b/cdc/redo/writer/memory/mem_log_writer.go index a47a83e855b..0460fdbf08b 100644 --- a/cdc/redo/writer/memory/mem_log_writer.go +++ b/cdc/redo/writer/memory/mem_log_writer.go @@ -54,11 +54,11 @@ func NewLogWriter( } // "nfs" and "local" scheme are converted to "file" scheme if !cfg.UseExternalStorage { - redo.FixLocalScheme(&cfg.URI) + redo.FixLocalScheme(cfg.URI) cfg.UseExternalStorage = redo.IsExternalStorage(cfg.URI.Scheme) } - extStorage, err := redo.InitExternalStorage(ctx, cfg.URI) + extStorage, err := redo.InitExternalStorage(ctx, *cfg.URI) if err != nil { return nil, err } diff --git a/cdc/redo/writer/memory/mem_log_writer_test.go b/cdc/redo/writer/memory/mem_log_writer_test.go index 60ea189a875..bba18c8068f 100644 --- a/cdc/redo/writer/memory/mem_log_writer_test.go +++ b/cdc/redo/writer/memory/mem_log_writer_test.go @@ -61,7 +61,7 @@ func testWriteEvents(t *testing.T, events []writer.RedoEvent) { LogType: redo.RedoDDLLogFileType, CaptureID: "test-capture", ChangeFeedID: model.DefaultChangeFeedID("test-changefeed"), - URI: *uri, + URI: uri, UseExternalStorage: true, MaxLogSizeInBytes: 10 * redo.Megabyte, } diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index 18779997b30..42ac64d4ff6 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -52,7 +52,7 @@ type LogWriterConfig struct { CaptureID model.CaptureID ChangeFeedID model.ChangeFeedID - URI url.URL + URI *url.URL UseExternalStorage bool Dir string MaxLogSizeInBytes int64 diff --git a/cdc/scheduler/internal/v3/agent/table.go b/cdc/scheduler/internal/v3/agent/table.go index 506bef29a54..e6a8525af98 100644 --- a/cdc/scheduler/internal/v3/agent/table.go +++ b/cdc/scheduler/internal/v3/agent/table.go @@ -311,7 +311,7 @@ func newTableSpanManager( func (tm *tableSpanManager) poll(ctx context.Context) ([]*schedulepb.Message, error) { result := make([]*schedulepb.Message, 0) var err error - toBeDropped := []tablepb.Span{} + var toBeDropped []tablepb.Span tm.tables.Ascend(func(span tablepb.Span, table *tableSpan) bool { message, err1 := table.poll(ctx) if err != nil { diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index e8a7f8c6431..6d8cb9f3be5 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -604,6 +604,10 @@ func (m *mockRedoMetaManager) Enabled() bool { return m.enable } +func (m *mockRedoMetaManager) Running() bool { + return true +} + func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { t.Parallel() diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 4e1c2aa9f1e..29da7cf23e7 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -702,10 +702,11 @@ func (c *Consumer) appendDDL(ddl *model.DDLEvent) { defer c.ddlListMu.Unlock() // DDL CommitTs fallback, just crash it to indicate the bug. if c.ddlWithMaxCommitTs != nil && ddl.CommitTs < c.ddlWithMaxCommitTs.CommitTs { - log.Panic("DDL CommitTs < maxCommitTsDDL.CommitTs", + log.Warn("DDL CommitTs < maxCommitTsDDL.CommitTs", zap.Uint64("commitTs", ddl.CommitTs), zap.Uint64("maxCommitTs", c.ddlWithMaxCommitTs.CommitTs), zap.Any("DDL", ddl)) + return } // A rename tables DDL job contains multiple DDL events with same CommitTs. diff --git a/pkg/redo/config.go b/pkg/redo/config.go index fbcdb3f1eb3..82c2e6f3457 100644 --- a/pkg/redo/config.go +++ b/pkg/redo/config.go @@ -172,7 +172,7 @@ func FixLocalScheme(uri *url.URL) { // IsBlackholeStorage returns whether a blackhole storage is used. func IsBlackholeStorage(scheme string) bool { - return ConsistentStorage(scheme) == consistentStorageBlackhole + return strings.HasPrefix(scheme, string(consistentStorageBlackhole)) } // InitExternalStorage init an external storage. @@ -220,6 +220,18 @@ func ValidateStorage(uri *url.URL) error { return errors.WrapError(errors.ErrStorageInitialize, errors.Annotate(err, fmt.Sprintf("can't make dir for new redo log: %+v", uri))) } + + file := filepath.Join(uri.Path, "file.test") + if err := os.WriteFile(file, []byte(""), DefaultFileMode); err != nil { + return errors.WrapError(errors.ErrStorageInitialize, errors.Annotate(err, + fmt.Sprintf("can't write file for new redo log: %+v", uri))) + } + + if _, err := os.ReadFile(file); err != nil { + return errors.WrapError(errors.ErrStorageInitialize, errors.Annotate(err, + fmt.Sprintf("can't read file for new redo log: %+v", uri))) + } + _ = os.Remove(file) return nil } diff --git a/tests/integration_tests/changefeed_error/run.sh b/tests/integration_tests/changefeed_error/run.sh index e4b032829cb..525fc22042e 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -106,7 +106,7 @@ function run() { cleanup_process $CDC_BINARY # make sure initialize changefeed error will not stuck the owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ChangefeedNewRedoManagerError=2*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/redo/ChangefeedNewRedoManagerError=2*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY changefeedid_3="changefeed-initialize-error" From bdeeffe2d76396bd68035ad2313e1f177b0f1e28 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 16 Nov 2023 13:27:51 +0800 Subject: [PATCH 02/23] sink(cdc): avoid sinking redundant events in some rare cases with redo enabled (#10085) (#10093) close pingcap/tiflow#10065 --- cdc/processor/sinkmanager/redo_cache.go | 62 +++++++------- cdc/processor/sinkmanager/redo_cache_test.go | 82 ++++++++++++++++--- .../sinkmanager/table_sink_worker.go | 17 ++-- .../sinkmanager/table_sink_worker_test.go | 48 +++++++++++ 4 files changed, 163 insertions(+), 46 deletions(-) diff --git a/cdc/processor/sinkmanager/redo_cache.go b/cdc/processor/sinkmanager/redo_cache.go index 17572cd34b0..719396bee59 100644 --- a/cdc/processor/sinkmanager/redo_cache.go +++ b/cdc/processor/sinkmanager/redo_cache.go @@ -59,11 +59,20 @@ type popResult struct { events []*model.RowChangedEvent size uint64 // size of events. releaseSize uint64 // size of all released events. - pushCount int - success bool - // If success, boundary is the upperBound of poped events. - // Otherwise, boundary is the lowerBound of cached events. - boundary engine.Position + + // many RowChangedEvent can come from one same PolymorphicEvent. + // pushCount indicates the count of raw PolymorphicEvents. + pushCount int + + // success indicates whether there is a gap between cached events and required events. + success bool + + // If success, upperBoundIfSuccess is the upperBound of poped events. + // The caller should fetch events (upperBoundIfSuccess, upperBound] from engine. + upperBoundIfSuccess engine.Position + // If fail, lowerBoundIfFail is the lowerBound of cached events. + // The caller should fetch events [lowerBound, lowerBoundIfFail) from engine. + lowerBoundIfFail engine.Position } // newRedoEventCache creates a redoEventCache instance. @@ -150,26 +159,28 @@ func (e *eventAppender) pop(lowerBound, upperBound engine.Position) (res popResu // NOTE: the caller will fetch events [lowerBound, res.boundary) from engine. res.success = false if e.lowerBound.Compare(upperBound.Next()) <= 0 { - res.boundary = e.lowerBound + res.lowerBoundIfFail = e.lowerBound } else { - res.boundary = upperBound.Next() + res.lowerBoundIfFail = upperBound.Next() } return } + if !e.upperBound.Valid() { - // if e.upperBound is invalid, it means there are no resolved transactions - // in the cache. + // It means there are no resolved cached transactions in the required range. // NOTE: the caller will fetch events [lowerBound, res.boundary) from engine. res.success = false - res.boundary = upperBound.Next() + res.lowerBoundIfFail = upperBound.Next() return } res.success = true - if upperBound.Compare(e.upperBound) > 0 { - res.boundary = e.upperBound + if lowerBound.Compare(e.upperBound) > 0 { + res.upperBoundIfSuccess = lowerBound.Prev() + } else if upperBound.Compare(e.upperBound) > 0 { + res.upperBoundIfSuccess = e.upperBound } else { - res.boundary = upperBound + res.upperBoundIfSuccess = upperBound } startIdx := sort.Search(e.readyCount, func(i int) bool { @@ -181,28 +192,23 @@ func (e *eventAppender) pop(lowerBound, upperBound engine.Position) (res popResu res.releaseSize += e.sizes[i] } - var endIdx int - if startIdx == e.readyCount { - endIdx = startIdx - } else { - endIdx = sort.Search(e.readyCount, func(i int) bool { - pos := engine.Position{CommitTs: e.events[i].CommitTs, StartTs: e.events[i].StartTs} - return pos.Compare(res.boundary) > 0 - }) - res.events = e.events[startIdx:endIdx] - for i := startIdx; i < endIdx; i++ { - res.size += e.sizes[i] - res.pushCount += int(e.pushCounts[i]) - } - res.releaseSize += res.size + endIdx := sort.Search(e.readyCount, func(i int) bool { + pos := engine.Position{CommitTs: e.events[i].CommitTs, StartTs: e.events[i].StartTs} + return pos.Compare(res.upperBoundIfSuccess) > 0 + }) + res.events = e.events[startIdx:endIdx] + for i := startIdx; i < endIdx; i++ { + res.size += e.sizes[i] + res.pushCount += int(e.pushCounts[i]) } + res.releaseSize += res.size e.events = e.events[endIdx:] e.sizes = e.sizes[endIdx:] e.pushCounts = e.pushCounts[endIdx:] e.readyCount -= endIdx // Update boundaries. Set upperBound to invalid if the range has been drained. - e.lowerBound = res.boundary.Next() + e.lowerBound = res.upperBoundIfSuccess.Next() if e.lowerBound.Compare(e.upperBound) > 0 { e.upperBound = engine.Position{} } diff --git a/cdc/processor/sinkmanager/redo_cache_test.go b/cdc/processor/sinkmanager/redo_cache_test.go index 005971e57e2..3de5958a03a 100644 --- a/cdc/processor/sinkmanager/redo_cache_test.go +++ b/cdc/processor/sinkmanager/redo_cache_test.go @@ -48,14 +48,14 @@ func TestRedoEventCache(t *testing.T) { // Try to pop [{0,1}, {0,4}], shoud fail. And the returned boundary should be {1,4}. popRes = appender.pop(engine.Position{StartTs: 0, CommitTs: 1}, engine.Position{StartTs: 0, CommitTs: 4}) require.False(t, popRes.success) - require.Equal(t, uint64(1), popRes.boundary.StartTs) - require.Equal(t, uint64(4), popRes.boundary.CommitTs) + require.Equal(t, uint64(1), popRes.lowerBoundIfFail.StartTs) + require.Equal(t, uint64(4), popRes.lowerBoundIfFail.CommitTs) // Try to pop [{0,2}, {0,4}], shoud fail. And the returned boundary should be {3,4}. popRes = appender.pop(engine.Position{StartTs: 0, CommitTs: 1}, engine.Position{StartTs: 5, CommitTs: 6}) require.False(t, popRes.success) - require.Equal(t, uint64(3), popRes.boundary.StartTs) - require.Equal(t, uint64(4), popRes.boundary.CommitTs) + require.Equal(t, uint64(3), popRes.lowerBoundIfFail.StartTs) + require.Equal(t, uint64(4), popRes.lowerBoundIfFail.CommitTs) // Try to pop [{3,4}, {3,4}], should success. popRes = appender.pop(engine.Position{StartTs: 3, CommitTs: 4}, engine.Position{StartTs: 3, CommitTs: 4}) @@ -63,22 +63,22 @@ func TestRedoEventCache(t *testing.T) { require.Equal(t, 2, len(popRes.events)) require.Equal(t, uint64(300), popRes.size) require.Equal(t, 2, popRes.pushCount) - require.Equal(t, uint64(3), popRes.boundary.StartTs) - require.Equal(t, uint64(4), popRes.boundary.CommitTs) + require.Equal(t, uint64(3), popRes.upperBoundIfSuccess.StartTs) + require.Equal(t, uint64(4), popRes.upperBoundIfSuccess.CommitTs) // Try to pop [{3,4}, {3,4}] again, shoud fail. And the returned boundary should be {4,4}. popRes = appender.pop(engine.Position{StartTs: 3, CommitTs: 4}, engine.Position{StartTs: 3, CommitTs: 4}) require.False(t, popRes.success) - require.Equal(t, uint64(4), popRes.boundary.StartTs) - require.Equal(t, uint64(4), popRes.boundary.CommitTs) + require.Equal(t, uint64(4), popRes.lowerBoundIfFail.StartTs) + require.Equal(t, uint64(4), popRes.lowerBoundIfFail.CommitTs) popRes = appender.pop(engine.Position{StartTs: 4, CommitTs: 4}, engine.Position{StartTs: 9, CommitTs: 10}) require.True(t, popRes.success) require.Equal(t, 1, len(popRes.events)) require.Equal(t, uint64(300), popRes.size) require.Equal(t, 1, popRes.pushCount) - require.Equal(t, uint64(5), popRes.boundary.StartTs) - require.Equal(t, uint64(6), popRes.boundary.CommitTs) + require.Equal(t, uint64(5), popRes.upperBoundIfSuccess.StartTs) + require.Equal(t, uint64(6), popRes.upperBoundIfSuccess.CommitTs) require.Equal(t, 0, len(appender.events)) require.True(t, appender.broken) @@ -87,3 +87,65 @@ func TestRedoEventCache(t *testing.T) { require.Equal(t, uint64(0), appender.upperBound.StartTs) require.Equal(t, uint64(0), appender.upperBound.CommitTs) } + +func TestRedoEventCacheAllPopBranches(t *testing.T) { + cache := newRedoEventCache(model.ChangeFeedID{}, 1000) + span := spanz.TableIDToComparableSpan(3) + appender := cache.maybeCreateAppender(span, engine.Position{StartTs: 101, CommitTs: 111}) + var batch []*model.RowChangedEvent + var ok bool + var popRes popResult + + batch = []*model.RowChangedEvent{{StartTs: 1, CommitTs: 11}, {StartTs: 1, CommitTs: 11}} + ok, _ = appender.pushBatch(batch, 0, engine.Position{}) + require.True(t, ok) + + batch = []*model.RowChangedEvent{{StartTs: 2, CommitTs: 12}} + ok, _ = appender.pushBatch(batch, 0, engine.Position{}) + require.True(t, ok) + + popRes = appender.pop(engine.Position{StartTs: 1, CommitTs: 2}, engine.Position{StartTs: 3, CommitTs: 4}) + require.False(t, popRes.success) + require.Equal(t, engine.Position{StartTs: 4, CommitTs: 4}, popRes.lowerBoundIfFail) + + popRes = appender.pop(engine.Position{StartTs: 1, CommitTs: 2}, engine.Position{StartTs: 300, CommitTs: 400}) + require.False(t, popRes.success) + require.Equal(t, engine.Position{StartTs: 101, CommitTs: 111}, popRes.lowerBoundIfFail) + + popRes = appender.pop(engine.Position{StartTs: 1, CommitTs: 11}, engine.Position{StartTs: 2, CommitTs: 12}) + require.False(t, popRes.success) + require.Equal(t, engine.Position{StartTs: 3, CommitTs: 12}, popRes.lowerBoundIfFail) + + batch = []*model.RowChangedEvent{{StartTs: 101, CommitTs: 111}, {StartTs: 101, CommitTs: 111}} + ok, _ = appender.pushBatch(batch, 0, engine.Position{StartTs: 101, CommitTs: 111}) + require.True(t, ok) + + batch = []*model.RowChangedEvent{{StartTs: 102, CommitTs: 112}} + ok, _ = appender.pushBatch(batch, 0, engine.Position{}) + require.True(t, ok) + require.Equal(t, 5, appender.readyCount) + + popRes = appender.pop(engine.Position{StartTs: 101, CommitTs: 111}, engine.Position{StartTs: 102, CommitTs: 112}) + require.True(t, popRes.success) + require.Equal(t, engine.Position{StartTs: 101, CommitTs: 111}, popRes.upperBoundIfSuccess) + require.Equal(t, 2, len(popRes.events)) + require.Equal(t, 1, popRes.pushCount) + require.Equal(t, uint64(101), popRes.events[1].StartTs) + require.Equal(t, 0, appender.readyCount) + + popRes = appender.pop(engine.Position{StartTs: 102, CommitTs: 111}, engine.Position{StartTs: 102, CommitTs: 112}) + require.False(t, popRes.success) + require.Equal(t, engine.Position{StartTs: 103, CommitTs: 112}, popRes.lowerBoundIfFail) + + batch = []*model.RowChangedEvent{{StartTs: 102, CommitTs: 112}} + ok, _ = appender.pushBatch(batch, 0, engine.Position{StartTs: 102, CommitTs: 102}) + require.True(t, ok) + require.Equal(t, 2, appender.readyCount) + + popRes = appender.pop(engine.Position{StartTs: 501, CommitTs: 502}, engine.Position{StartTs: 701, CommitTs: 702}) + require.True(t, popRes.success) + require.Equal(t, 0, len(popRes.events)) + require.Equal(t, engine.Position{StartTs: 500, CommitTs: 502}, popRes.upperBoundIfSuccess) + require.Equal(t, 0, appender.readyCount) + require.Equal(t, 0, len(appender.events)) +} diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index 9d7de864397..b04ce8f7c40 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -201,13 +201,14 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e if err != nil { return errors.Trace(err) } + // NOTE: lowerBound can be updated by `fetchFromCache`, so `lastPos` should also be updated. + advancer.lastPos = lowerBound.Prev() if drained { // If drained is true it means we have drained all events from the cache, // we can return directly instead of get events from the source manager again. - performCallback(lowerBound.Prev()) + performCallback(advancer.lastPos) return nil } - advancer.lastPos = lowerBound.Prev() } // lowerBound and upperBound are both closed intervals. @@ -277,7 +278,7 @@ func (w *sinkWorker) fetchFromCache( } popRes := cache.pop(*lowerBound, *upperBound) if popRes.success { - newLowerBound = popRes.boundary.Next() + newLowerBound = popRes.upperBoundIfSuccess.Next() if len(popRes.events) > 0 { w.metricOutputEventCountKV.Add(float64(popRes.pushCount)) w.metricRedoEventCacheHit.Add(float64(popRes.size)) @@ -288,9 +289,9 @@ func (w *sinkWorker) fetchFromCache( // Get a resolvedTs so that we can record it into sink memory quota. var resolvedTs model.ResolvedTs - isCommitFence := popRes.boundary.IsCommitFence() + isCommitFence := popRes.upperBoundIfSuccess.IsCommitFence() if w.splitTxn { - resolvedTs = model.NewResolvedTs(popRes.boundary.CommitTs) + resolvedTs = model.NewResolvedTs(popRes.upperBoundIfSuccess.CommitTs) if !isCommitFence { resolvedTs.Mode = model.BatchResolvedMode resolvedTs.BatchID = batchID.Load() @@ -298,9 +299,9 @@ func (w *sinkWorker) fetchFromCache( } } else { if isCommitFence { - resolvedTs = model.NewResolvedTs(popRes.boundary.CommitTs) + resolvedTs = model.NewResolvedTs(popRes.upperBoundIfSuccess.CommitTs) } else { - resolvedTs = model.NewResolvedTs(popRes.boundary.CommitTs - 1) + resolvedTs = model.NewResolvedTs(popRes.upperBoundIfSuccess.CommitTs - 1) } } // Transfer the memory usage from redoMemQuota to sinkMemQuota. @@ -316,7 +317,7 @@ func (w *sinkWorker) fetchFromCache( zap.Any("resolvedTs", resolvedTs), zap.Error(err)) } else { - newUpperBound = popRes.boundary.Prev() + newUpperBound = popRes.lowerBoundIfFail.Prev() } cacheDrained = newLowerBound.Compare(newUpperBound) > 0 log.Debug("fetchFromCache is performed", diff --git a/cdc/processor/sinkmanager/table_sink_worker_test.go b/cdc/processor/sinkmanager/table_sink_worker_test.go index 3a8c0667753..0bb42a1f9ee 100644 --- a/cdc/processor/sinkmanager/table_sink_worker_test.go +++ b/cdc/processor/sinkmanager/table_sink_worker_test.go @@ -760,3 +760,51 @@ func (suite *tableSinkWorkerSuite) TestHandleTaskWithoutMemory() { cancel() wg.Wait() } + +func (suite *tableSinkWorkerSuite) TestHandleTaskWithCache() { + ctx, cancel := context.WithCancel(context.Background()) + events := []*model.PolymorphicEvent{ + genPolymorphicEvent(2, 4, suite.testSpan), + genPolymorphicEvent(2, 4, suite.testSpan), + genPolymorphicResolvedEvent(4), + } + w, e := suite.createWorker(ctx, 0, true) + w.eventCache = newRedoEventCache(suite.testChangefeedID, 1024*1024) + appender := w.eventCache.maybeCreateAppender(suite.testSpan, engine.Position{StartTs: 1, CommitTs: 3}) + appender.pushBatch( + []*model.RowChangedEvent{events[0].Row, events[1].Row}, + uint64(0), engine.Position{StartTs: 2, CommitTs: 4}, + ) + defer w.sinkMemQuota.Close() + suite.addEventsToSortEngine(events, e) + + taskChan := make(chan *sinkTask) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := w.handleTasks(ctx, taskChan) + require.Equal(suite.T(), context.Canceled, err) + }() + + wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) + defer sink.Close() + + chShouldBeClosed := make(chan struct{}, 1) + callback := func(lastWrittenPos engine.Position) { + require.Equal(suite.T(), engine.Position{StartTs: 2, CommitTs: 4}, lastWrittenPos) + close(chShouldBeClosed) + } + taskChan <- &sinkTask{ + span: suite.testSpan, + lowerBound: engine.Position{StartTs: 1, CommitTs: 3}, + getUpperBound: genUpperBoundGetter(4), + tableSink: wrapper, + callback: callback, + isCanceled: func() bool { return true }, + } + + <-chShouldBeClosed + cancel() + wg.Wait() +} From 1b69de9e6ca57889c9a4c33897dde2b1728831c3 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 16 Nov 2023 14:08:19 +0800 Subject: [PATCH 03/23] kv-client(cdc): add more logs to help debug slow regions (#9981) (#9983) close pingcap/tiflow#9980 --- cdc/kv/client.go | 134 +++++++++++++----- cdc/kv/region_state.go | 91 ++++++------ cdc/kv/region_state_bench_test.go | 12 +- cdc/kv/region_worker.go | 142 ++++++++++--------- cdc/kv/region_worker_test.go | 40 ++++-- cdc/kv/regionlock/region_range_lock.go | 148 +++++++++++++++++--- cdc/kv/regionlock/region_range_lock_test.go | 8 +- 7 files changed, 387 insertions(+), 188 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 8805713e234..a419c787ecd 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -19,6 +19,7 @@ import ( "io" "math/rand" "strconv" + "strings" "sync" "sync/atomic" "time" @@ -42,6 +43,7 @@ import ( "github.com/pingcap/tiflow/pkg/version" "github.com/prometheus/client_golang/prometheus" tidbkv "github.com/tikv/client-go/v2/kv" + "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" "go.uber.org/zap" @@ -271,7 +273,7 @@ func (c *CDCClient) EventFeed( eventCh chan<- model.RegionFeedEvent, ) error { s := newEventFeedSession(c, span, lockResolver, ts, eventCh) - return s.eventFeed(ctx, ts) + return s.eventFeed(ctx) } // RegionCount returns the number of captured regions. @@ -365,7 +367,6 @@ type eventFeedSession struct { type rangeRequestTask struct { span tablepb.Span - ts uint64 } func newEventFeedSession( @@ -375,9 +376,10 @@ func newEventFeedSession( startTs uint64, eventCh chan<- model.RegionFeedEvent, ) *eventFeedSession { - id := strconv.FormatUint(allocID(), 10) + id := allocID() + idStr := strconv.FormatUint(id, 10) rangeLock := regionlock.NewRegionRangeLock( - totalSpan.StartKey, totalSpan.EndKey, startTs, + id, totalSpan.StartKey, totalSpan.EndKey, startTs, client.changefeed.Namespace+"."+client.changefeed.ID) return &eventFeedSession{ client: client, @@ -390,7 +392,7 @@ func newEventFeedSession( eventCh: eventCh, rangeLock: rangeLock, lockResolver: lockResolver, - id: id, + id: idStr, regionChSizeGauge: clientChannelSize.WithLabelValues("region"), errChSizeGauge: clientChannelSize.WithLabelValues("err"), rangeChSizeGauge: clientChannelSize.WithLabelValues("range"), @@ -406,7 +408,7 @@ func newEventFeedSession( } } -func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { +func (s *eventFeedSession) eventFeed(ctx context.Context) error { s.requestRangeCh = chann.NewAutoDrainChann[rangeRequestTask]() s.regionCh = chann.NewAutoDrainChann[singleRegionInfo]() s.regionRouter = chann.NewAutoDrainChann[singleRegionInfo]() @@ -423,13 +425,11 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { g, ctx := errgroup.WithContext(ctx) - g.Go(func() error { - return s.dispatchRequest(ctx) - }) + g.Go(func() error { return s.dispatchRequest(ctx) }) - g.Go(func() error { - return s.requestRegionToStore(ctx, g) - }) + g.Go(func() error { return s.requestRegionToStore(ctx, g) }) + + g.Go(func() error { return s.logSlowRegions(ctx) }) g.Go(func() error { for { @@ -447,7 +447,7 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { // Besides the count or frequency of range request is limited, // we use ephemeral goroutine instead of permanent goroutine. g.Go(func() error { - return s.divideAndSendEventFeedToRegions(ctx, task.span, task.ts) + return s.divideAndSendEventFeedToRegions(ctx, task.span) }) } } @@ -468,7 +468,7 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { } }) - s.requestRangeCh.In() <- rangeRequestTask{span: s.totalSpan, ts: ts} + s.requestRangeCh.In() <- rangeRequestTask{span: s.totalSpan} s.rangeChSizeGauge.Inc() log.Info("event feed started", @@ -476,7 +476,7 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { zap.String("changefeed", s.changefeed.ID), zap.Int64("tableID", s.tableID), zap.String("tableName", s.tableName), - zap.Uint64("startTs", ts), + zap.Uint64("startTs", s.startTs), zap.Stringer("span", &s.totalSpan)) return g.Wait() @@ -485,9 +485,9 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { // scheduleDivideRegionAndRequest schedules a range to be divided by regions, // and these regions will be then scheduled to send ChangeData requests. func (s *eventFeedSession) scheduleDivideRegionAndRequest( - ctx context.Context, span tablepb.Span, ts uint64, + ctx context.Context, span tablepb.Span, ) { - task := rangeRequestTask{span: span, ts: ts} + task := rangeRequestTask{span: span} select { case s.requestRangeCh.In() <- task: s.rangeChSizeGauge.Inc() @@ -501,7 +501,7 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single handleResult := func(res regionlock.LockRangeResult) { switch res.Status { case regionlock.LockRangeStatusSuccess: - sri.resolvedTs = res.CheckpointTs + sri.lockedRange = res.LockedRange select { case s.regionCh.In() <- sri: s.regionChSizeGauge.Inc() @@ -513,12 +513,11 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single zap.String("changefeed", s.changefeed.ID), zap.Uint64("regionID", sri.verID.GetID()), zap.Stringer("span", &sri.span), - zap.Uint64("resolvedTs", sri.resolvedTs), zap.Any("retrySpans", res.RetryRanges)) for _, r := range res.RetryRanges { // This call is always blocking, otherwise if scheduling in a new // goroutine, it won't block the caller of `schedulerRegionRequest`. - s.scheduleDivideRegionAndRequest(ctx, r, sri.resolvedTs) + s.scheduleDivideRegionAndRequest(ctx, r) } case regionlock.LockRangeStatusCancel: return @@ -529,11 +528,12 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single res := s.rangeLock.LockRange( ctx, sri.span.StartKey, sri.span.EndKey, sri.verID.GetID(), sri.verID.GetVer()) + failpoint.Inject("kvClientMockRangeLock", func(val failpoint.Value) { // short sleep to wait region has split time.Sleep(time.Second) s.rangeLock.UnlockRange(sri.span.StartKey, sri.span.EndKey, - sri.verID.GetID(), sri.verID.GetVer(), sri.resolvedTs) + sri.verID.GetID(), sri.verID.GetVer()) regionNum := val.(int) retryRanges := make([]tablepb.Span, 0, regionNum) start := []byte("a") @@ -561,7 +561,7 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single // CAUTION: Note that this should only be called in a context that the region has locked its range. func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo) { s.rangeLock.UnlockRange(errorInfo.span.StartKey, errorInfo.span.EndKey, - errorInfo.verID.GetID(), errorInfo.verID.GetVer(), errorInfo.resolvedTs) + errorInfo.verID.GetID(), errorInfo.verID.GetVer(), errorInfo.resolvedTs()) log.Info("region failed", zap.Stringer("span", &errorInfo.span), zap.Any("regionId", errorInfo.verID.GetID()), zap.Error(errorInfo.err)) @@ -611,7 +611,7 @@ func (s *eventFeedSession) requestRegionToStore( RegionId: regionID, RequestId: requestID, RegionEpoch: regionEpoch, - CheckpointTs: sri.resolvedTs, + CheckpointTs: sri.resolvedTs(), StartKey: sri.span.StartKey, EndKey: sri.span.EndKey, ExtraOp: extraOp, @@ -689,13 +689,13 @@ func (s *eventFeedSession) requestRegionToStore( state := newRegionFeedState(sri, requestID) pendingRegions.setByRequestID(requestID, state) - log.Debug("start new request", + log.Info("start new request", zap.String("namespace", s.changefeed.Namespace), zap.String("changefeed", s.changefeed.ID), zap.Int64("tableID", s.tableID), zap.String("tableName", s.tableName), - zap.String("addr", storeAddr), - zap.Any("request", req)) + zap.Uint64("regionID", sri.verID.GetID()), + zap.String("addr", storeAddr)) err = stream.client.Send(req) @@ -782,7 +782,7 @@ func (s *eventFeedSession) dispatchRequest(ctx context.Context) error { Region: sri.verID.GetID(), }, }, - ResolvedTs: sri.resolvedTs, + ResolvedTs: sri.resolvedTs(), }, } select { @@ -804,7 +804,7 @@ func (s *eventFeedSession) dispatchRequest(ctx context.Context) error { zap.String("tableName", s.tableName), zap.Uint64("regionID", sri.verID.GetID()), zap.Stringer("span", &sri.span), - zap.Uint64("resolvedTs", sri.resolvedTs)) + zap.Uint64("resolvedTs", sri.resolvedTs())) errInfo := newRegionErrorInfo(sri, &rpcCtxUnavailableErr{verID: sri.verID}) s.onRegionFail(ctx, errInfo) continue @@ -818,7 +818,7 @@ func (s *eventFeedSession) dispatchRequest(ctx context.Context) error { // to region boundaries. When region merging happens, it's possible that it // will produce some overlapping spans. func (s *eventFeedSession) divideAndSendEventFeedToRegions( - ctx context.Context, span tablepb.Span, ts uint64, + ctx context.Context, span tablepb.Span, ) error { limit := 20 nextSpan := span @@ -870,7 +870,7 @@ func (s *eventFeedSession) divideAndSendEventFeedToRegions( // the End key return by the PD API will be nil to represent the biggest key, partialSpan = spanz.HackSpan(partialSpan) - sri := newSingleRegionInfo(tiRegion.VerID(), partialSpan, ts, nil) + sri := newSingleRegionInfo(tiRegion.VerID(), partialSpan, nil) s.scheduleRegionRequest(ctx, sri) // return if no more regions if spanz.EndCompare(nextSpan.StartKey, span.EndKey) >= 0 { @@ -889,17 +889,24 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI switch eerr := errors.Cause(err).(type) { case *eventError: innerErr := eerr.err + log.Info("cdc region error", + zap.String("namespace", s.changefeed.Namespace), + zap.String("changefeed", s.changefeed.ID), + zap.Int64("tableID", s.tableID), + zap.String("tableName", s.tableName), + zap.Stringer("error", innerErr)) + if notLeader := innerErr.GetNotLeader(); notLeader != nil { metricFeedNotLeaderCounter.Inc() s.client.regionCache.UpdateLeader(errInfo.verID, notLeader.GetLeader(), errInfo.rpcCtx.AccessIdx) } else if innerErr.GetEpochNotMatch() != nil { // TODO: If only confver is updated, we don't need to reload the region from region cache. metricFeedEpochNotMatchCounter.Inc() - s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.resolvedTs) + s.scheduleDivideRegionAndRequest(ctx, errInfo.span) return nil } else if innerErr.GetRegionNotFound() != nil { metricFeedRegionNotFoundCounter.Inc() - s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.resolvedTs) + s.scheduleDivideRegionAndRequest(ctx, errInfo.span) return nil } else if duplicatedRequest := innerErr.GetDuplicateRequest(); duplicatedRequest != nil { metricFeedDuplicateRequestCounter.Inc() @@ -929,7 +936,7 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI } case *rpcCtxUnavailableErr: metricFeedRPCCtxUnavailable.Inc() - s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.resolvedTs) + s.scheduleDivideRegionAndRequest(ctx, errInfo.span) return nil case *connectToStoreErr: metricConnectToStoreErr.Inc() @@ -1206,7 +1213,7 @@ func (s *eventFeedSession) sendRegionChangeEvents( } state.start() worker.setRegionState(event.RegionId, state) - } else if state.isStopped() { + } else if state.isStale() { log.Warn("drop event due to region feed stopped", zap.String("namespace", s.changefeed.Namespace), zap.String("changefeed", s.changefeed.ID), @@ -1216,6 +1223,17 @@ func (s *eventFeedSession) sendRegionChangeEvents( continue } + switch x := event.Event.(type) { + case *cdcpb.Event_Error: + log.Info("event feed receives a region error", + zap.String("namespace", s.changefeed.Namespace), + zap.String("changefeed", s.changefeed.ID), + zap.Int64("tableID", s.tableID), + zap.String("tableName", s.tableName), + zap.Uint64("regionID", event.RegionId), + zap.Any("error", x.Error)) + } + slot := worker.inputCalcSlot(event.RegionId) statefulEvents[slot] = append(statefulEvents[slot], ®ionStatefulEvent{ changeEvent: event, @@ -1308,6 +1326,54 @@ func (s *eventFeedSession) getStreamCancel(storeAddr string) (cancel context.Can return } +func (s *eventFeedSession) logSlowRegions(ctx context.Context) error { + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-ticker.C: + } + + currTime, err := s.client.pdClock.CurrentTime() + if err != nil { + continue + } + attr := s.rangeLock.CollectLockedRangeAttrs(nil) + if attr.SlowestRegion.Initialized { + ckptTime := oracle.GetTimeFromTS(attr.SlowestRegion.CheckpointTs) + if currTime.Sub(ckptTime) > 20*time.Second { + log.Info("event feed finds a slow region", + zap.String("namespace", s.changefeed.Namespace), + zap.String("changefeed", s.changefeed.ID), + zap.Int64("tableID", s.tableID), + zap.String("tableName", s.tableName), + zap.Any("slowRegion", attr.SlowestRegion)) + } + } else if currTime.Sub(attr.SlowestRegion.Created) > 10*time.Minute { + log.Info("event feed initializes a region too slow", + zap.String("namespace", s.changefeed.Namespace), + zap.String("changefeed", s.changefeed.ID), + zap.Int64("tableID", s.tableID), + zap.String("tableName", s.tableName), + zap.Any("slowRegion", attr.SlowestRegion)) + } + if len(attr.Holes) > 0 { + holes := make([]string, 0, len(attr.Holes)) + for _, hole := range attr.Holes { + holes = append(holes, fmt.Sprintf("[%s,%s)", hole.StartKey, hole.EndKey)) + } + log.Info("event feed holes exist", + zap.String("namespace", s.changefeed.Namespace), + zap.String("changefeed", s.changefeed.ID), + zap.Int64("tableID", s.tableID), + zap.String("tableName", s.tableName), + zap.String("holes", strings.Join(holes, ", "))) + } + } +} + func assembleRowEvent(regionID uint64, entry *cdcpb.Event_Row) (model.RegionFeedEvent, error) { var opType model.OpType switch entry.GetOpType() { diff --git a/cdc/kv/region_state.go b/cdc/kv/region_state.go index 52113f6c970..520c6366e8a 100644 --- a/cdc/kv/region_state.go +++ b/cdc/kv/region_state.go @@ -16,9 +16,8 @@ package kv import ( "runtime" "sync" - "sync/atomic" - "time" + "github.com/pingcap/tiflow/cdc/kv/regionlock" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/tikv/client-go/v2/tikv" ) @@ -26,68 +25,84 @@ import ( const ( minRegionStateBucket = 4 maxRegionStateBucket = 16 + + stateNormal uint32 = 0 + stateStopped uint32 = 1 + stateRemoved uint32 = 2 ) type singleRegionInfo struct { - verID tikv.RegionVerID - span tablepb.Span - resolvedTs uint64 - rpcCtx *tikv.RPCContext + verID tikv.RegionVerID + span tablepb.Span + rpcCtx *tikv.RPCContext + + lockedRange *regionlock.LockedRange } func newSingleRegionInfo( verID tikv.RegionVerID, span tablepb.Span, - ts uint64, rpcCtx *tikv.RPCContext, ) singleRegionInfo { return singleRegionInfo{ - verID: verID, - span: span, - resolvedTs: ts, - rpcCtx: rpcCtx, + verID: verID, + span: span, + rpcCtx: rpcCtx, } } +func (s singleRegionInfo) resolvedTs() uint64 { + return s.lockedRange.CheckpointTs.Load() +} + type regionFeedState struct { sri singleRegionInfo requestID uint64 - stopped int32 - - initialized atomic.Bool - matcher *matcher - startFeedTime time.Time - lastResolvedTs uint64 + matcher *matcher + + // Transform: normal -> stopped -> removed. + // normal: the region is in replicating. + // stopped: some error happens. + // removed: the region is returned into the pending list, + // will be re-resolved and re-scheduled later. + state struct { + sync.RWMutex + v uint32 + } } func newRegionFeedState(sri singleRegionInfo, requestID uint64) *regionFeedState { return ®ionFeedState{ sri: sri, requestID: requestID, - stopped: 0, } } func (s *regionFeedState) start() { - s.startFeedTime = time.Now() - s.lastResolvedTs = s.sri.resolvedTs s.matcher = newMatcher() } +// mark regionFeedState as stopped with the given error if possible. func (s *regionFeedState) markStopped() { - atomic.StoreInt32(&s.stopped, 1) + s.state.Lock() + defer s.state.Unlock() + if s.state.v == stateNormal { + s.state.v = stateStopped + } } -func (s *regionFeedState) isStopped() bool { - return atomic.LoadInt32(&s.stopped) > 0 +func (s *regionFeedState) isStale() bool { + s.state.RLock() + defer s.state.RUnlock() + return s.state.v == stateStopped || s.state.v == stateRemoved } func (s *regionFeedState) isInitialized() bool { - return s.initialized.Load() + return s.sri.lockedRange.Initialzied.Load() } func (s *regionFeedState) setInitialized() { - s.initialized.Store(true) + s.sri.lockedRange.Initialzied.Store(true) } func (s *regionFeedState) getRegionID() uint64 { @@ -95,31 +110,29 @@ func (s *regionFeedState) getRegionID() uint64 { } func (s *regionFeedState) getLastResolvedTs() uint64 { - return atomic.LoadUint64(&s.lastResolvedTs) + return s.sri.lockedRange.CheckpointTs.Load() } // updateResolvedTs update the resolved ts of the current region feed func (s *regionFeedState) updateResolvedTs(resolvedTs uint64) { - if resolvedTs > s.getLastResolvedTs() { - atomic.StoreUint64(&s.lastResolvedTs, resolvedTs) - } -} - -// setRegionInfoResolvedTs is only called when the region disconnect, -// to update the `singleRegionInfo` which is reused by reconnect. -func (s *regionFeedState) setRegionInfoResolvedTs() { - if s.getLastResolvedTs() <= s.sri.resolvedTs { - return + state := s.sri.lockedRange + for { + last := state.CheckpointTs.Load() + if last > resolvedTs { + return + } + if state.CheckpointTs.CompareAndSwap(last, resolvedTs) { + break + } } - s.sri.resolvedTs = s.lastResolvedTs } func (s *regionFeedState) getRegionInfo() singleRegionInfo { return s.sri } -func (s *regionFeedState) getRegionMeta() (uint64, tablepb.Span, time.Time, string) { - return s.sri.verID.GetID(), s.sri.span, s.startFeedTime, s.sri.rpcCtx.Addr +func (s *regionFeedState) getRegionMeta() (uint64, tablepb.Span, string) { + return s.sri.verID.GetID(), s.sri.span, s.sri.rpcCtx.Addr } type syncRegionFeedStateMap struct { diff --git a/cdc/kv/region_state_bench_test.go b/cdc/kv/region_state_bench_test.go index 5b9fe592080..250f265d28c 100644 --- a/cdc/kv/region_state_bench_test.go +++ b/cdc/kv/region_state_bench_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/pingcap/tiflow/cdc/kv/regionlock" "github.com/pingcap/tiflow/pkg/spanz" "github.com/tikv/client-go/v2/tikv" ) @@ -40,9 +41,9 @@ func TestSyncRegionFeedStateMapConcurrentAccess(t *testing.T) { return default: } - m.setByRequestID(1, ®ionFeedState{}) - m.setByRequestID(2, ®ionFeedState{}) - m.setByRequestID(3, ®ionFeedState{}) + m.setByRequestID(1, ®ionFeedState{sri: singleRegionInfo{lockedRange: ®ionlock.LockedRange{}}}) + m.setByRequestID(2, ®ionFeedState{sri: singleRegionInfo{lockedRange: ®ionlock.LockedRange{}}}) + m.setByRequestID(3, ®ionFeedState{sri: singleRegionInfo{lockedRange: ®ionlock.LockedRange{}}}) } }() wg.Add(1) @@ -55,7 +56,7 @@ func TestSyncRegionFeedStateMapConcurrentAccess(t *testing.T) { default: } m.iter(func(requestID uint64, state *regionFeedState) bool { - _ = state.initialized.Load() + state.isInitialized() return true }) } @@ -118,7 +119,8 @@ func benchmarkGetRegionState(b *testing.B, bench func(b *testing.B, sm regionSta state := newRegionFeedState(newSingleRegionInfo( tikv.RegionVerID{}, spanz.ToSpan([]byte{}, spanz.UpperBoundKey), - 0, &tikv.RPCContext{}), 0) + &tikv.RPCContext{}), 0) + state.sri.lockedRange = ®ionlock.LockedRange{} regionCount := []int{100, 1000, 10000, 20000, 40000, 80000, 160000, 320000} for _, count := range regionCount { diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index d41d55f8355..5a3771d65cb 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -61,19 +61,18 @@ const ( ) type regionWorkerMetrics struct { - // kv events related metrics - metricReceivedEventSize prometheus.Observer - metricDroppedEventSize prometheus.Observer + metricReceivedEventSize prometheus.Observer + metricDroppedEventSize prometheus.Observer + metricPullEventInitializedCounter prometheus.Counter + metricPullEventCommittedCounter prometheus.Counter metricPullEventPrewriteCounter prometheus.Counter metricPullEventCommitCounter prometheus.Counter - metricPullEventCommittedCounter prometheus.Counter metricPullEventRollbackCounter prometheus.Counter - metricSendEventResolvedCounter prometheus.Counter - metricSendEventCommitCounter prometheus.Counter - metricSendEventCommittedCounter prometheus.Counter - // TODO: add region runtime related metrics + metricSendEventResolvedCounter prometheus.Counter + metricSendEventCommitCounter prometheus.Counter + metricSendEventCommittedCounter prometheus.Counter } /* @@ -114,22 +113,22 @@ type regionWorker struct { inputPending int32 } -func newRegionWorker( - ctx context.Context, changefeedID model.ChangeFeedID, s *eventFeedSession, addr string, -) *regionWorker { +func newRegionWorkerMetrics(changefeedID model.ChangeFeedID) *regionWorkerMetrics { metrics := ®ionWorkerMetrics{} metrics.metricReceivedEventSize = eventSize.WithLabelValues("received") metrics.metricDroppedEventSize = eventSize.WithLabelValues("dropped") + metrics.metricPullEventInitializedCounter = pullEventCounter. WithLabelValues(cdcpb.Event_INITIALIZED.String(), changefeedID.Namespace, changefeedID.ID) metrics.metricPullEventCommittedCounter = pullEventCounter. WithLabelValues(cdcpb.Event_COMMITTED.String(), changefeedID.Namespace, changefeedID.ID) - metrics.metricPullEventCommitCounter = pullEventCounter. - WithLabelValues(cdcpb.Event_COMMIT.String(), changefeedID.Namespace, changefeedID.ID) metrics.metricPullEventPrewriteCounter = pullEventCounter. WithLabelValues(cdcpb.Event_PREWRITE.String(), changefeedID.Namespace, changefeedID.ID) + metrics.metricPullEventCommitCounter = pullEventCounter. + WithLabelValues(cdcpb.Event_COMMIT.String(), changefeedID.Namespace, changefeedID.ID) metrics.metricPullEventRollbackCounter = pullEventCounter. WithLabelValues(cdcpb.Event_ROLLBACK.String(), changefeedID.Namespace, changefeedID.ID) + metrics.metricSendEventResolvedCounter = sendEventCounter. WithLabelValues("native-resolved", changefeedID.Namespace, changefeedID.ID) metrics.metricSendEventCommitCounter = sendEventCounter. @@ -137,6 +136,12 @@ func newRegionWorker( metrics.metricSendEventCommittedCounter = sendEventCounter. WithLabelValues("committed", changefeedID.Namespace, changefeedID.ID) + return metrics +} + +func newRegionWorker( + ctx context.Context, changefeedID model.ChangeFeedID, s *eventFeedSession, addr string, +) *regionWorker { return ®ionWorker{ parentCtx: ctx, session: s, @@ -148,7 +153,7 @@ func newRegionWorker( rtsUpdateCh: make(chan *rtsUpdateEvent, 1024), storeAddr: addr, concurrency: s.client.config.KVClient.WorkerConcurrent, - metrics: metrics, + metrics: newRegionWorkerMetrics(changefeedID), inputPending: 0, } } @@ -193,21 +198,22 @@ func (w *regionWorker) checkShouldExit() error { } func (w *regionWorker) handleSingleRegionError(err error, state *regionFeedState) error { - state.setRegionInfoResolvedTs() regionID := state.getRegionID() + isStale := state.isStale() log.Info("single region event feed disconnected", zap.String("namespace", w.session.client.changefeed.Namespace), zap.String("changefeed", w.session.client.changefeed.ID), zap.Uint64("regionID", regionID), zap.Uint64("requestID", state.requestID), zap.Stringer("span", &state.sri.span), - zap.Uint64("resolvedTs", state.sri.resolvedTs), + zap.Uint64("resolvedTs", state.sri.resolvedTs()), + zap.Bool("isStale", isStale), zap.Error(err)) // if state is already marked stopped, it must have been or would be processed by `onRegionFail` - if state.isStopped() { + if isStale { return w.checkShouldExit() } - // We need to ensure when the error is handled, `isStopped` must be set. So set it before sending the error. + // We need to ensure when the error is handled, `isStale` must be set. So set it before sending the error. state.markStopped() w.delRegionState(regionID) failpoint.Inject("kvClientSingleFeedProcessDelay", nil) @@ -295,7 +301,7 @@ func (w *regionWorker) resolveLock(ctx context.Context) error { maxVersion := oracle.ComposeTS(oracle.GetPhysical(currentTimeFromPD.Add(-10*time.Second)), 0) for _, rts := range expired { state, ok := w.getRegionState(rts.regionID) - if !ok || state.isStopped() { + if !ok || state.isStale() { // state is already deleted or stopped, just continue, // and don't need to push resolved ts back to heap. continue @@ -336,7 +342,7 @@ func (w *regionWorker) resolveLock(ctx context.Context) error { zap.Uint64("resolvedTs", lastResolvedTs), ) } - err = w.session.lockResolver.Resolve(ctx, rts.regionID, maxVersion) + err := w.session.lockResolver.Resolve(ctx, rts.regionID, maxVersion) if err != nil { log.Warn("failed to resolve lock", zap.Uint64("regionID", rts.regionID), @@ -356,7 +362,7 @@ func (w *regionWorker) resolveLock(ctx context.Context) error { func (w *regionWorker) processEvent(ctx context.Context, event *regionStatefulEvent) error { // event.state is nil when resolvedTsEvent is not nil - skipEvent := event.state != nil && event.state.isStopped() + skipEvent := event.state != nil && event.state.isStale() if skipEvent { return nil } @@ -616,52 +622,50 @@ func (w *regionWorker) handleEventEntry( x *cdcpb.Event_Entries_, state *regionFeedState, ) error { - regionID, regionSpan, startTime, _ := state.getRegionMeta() + emit := func(assembled model.RegionFeedEvent) bool { + select { + case w.outputCh <- assembled: + return true + case <-ctx.Done(): + return false + } + } + return handleEventEntry(x, w.session.startTs, state, w.metrics, emit) +} + +func handleEventEntry( + x *cdcpb.Event_Entries_, + startTs uint64, + state *regionFeedState, + metrics *regionWorkerMetrics, + emit func(assembled model.RegionFeedEvent) bool, +) error { + regionID, regionSpan, _ := state.getRegionMeta() for _, entry := range x.Entries.GetEntries() { - // if a region with kv range [a, z), and we only want the get [b, c) from this region, - // tikv will return all key events in the region, although specified [b, c) int the request. - // we can make tikv only return the events about the keys in the specified range. + // NOTE: from TiKV 7.0.0, entries are already filtered out in TiKV side. + // We can remove the check in future. comparableKey := spanz.ToComparableKey(entry.GetKey()) - // key for initialized event is nil if entry.Type != cdcpb.Event_INITIALIZED && !spanz.KeyInSpan(comparableKey, regionSpan) { - w.metrics.metricDroppedEventSize.Observe(float64(entry.Size())) + metrics.metricDroppedEventSize.Observe(float64(entry.Size())) continue } switch entry.Type { case cdcpb.Event_INITIALIZED: - if time.Since(startTime) > 20*time.Second { - log.Warn("The time cost of initializing is too much", - zap.String("namespace", w.session.client.changefeed.Namespace), - zap.String("changefeed", w.session.client.changefeed.ID), - zap.Duration("duration", time.Since(startTime)), - zap.Uint64("regionID", regionID)) - } - w.metrics.metricPullEventInitializedCounter.Inc() - + metrics.metricPullEventInitializedCounter.Inc() state.setInitialized() - // state is just initialized, so we know this must be true - cachedEvents := state.matcher.matchCachedRow(true) - for _, cachedEvent := range cachedEvents { + for _, cachedEvent := range state.matcher.matchCachedRow(true) { revent, err := assembleRowEvent(regionID, cachedEvent) if err != nil { return errors.Trace(err) } - select { - case w.outputCh <- revent: - w.metrics.metricSendEventCommitCounter.Inc() - case <-ctx.Done(): - return errors.Trace(ctx.Err()) + if !emit(revent) { + return nil } + metrics.metricSendEventCommitCounter.Inc() } state.matcher.matchCachedRollbackRow(true) case cdcpb.Event_COMMITTED: - w.metrics.metricPullEventCommittedCounter.Inc() - revent, err := assembleRowEvent(regionID, entry) - if err != nil { - return errors.Trace(err) - } - resolvedTs := state.getLastResolvedTs() if entry.CommitTs <= resolvedTs { logPanic("The CommitTs must be greater than the resolvedTs", @@ -671,17 +675,21 @@ func (w *regionWorker) handleEventEntry( zap.Uint64("regionID", regionID)) return errUnreachable } - select { - case w.outputCh <- revent: - w.metrics.metricSendEventCommittedCounter.Inc() - case <-ctx.Done(): - return errors.Trace(ctx.Err()) + + metrics.metricPullEventCommittedCounter.Inc() + revent, err := assembleRowEvent(regionID, entry) + if err != nil { + return errors.Trace(err) + } + if !emit(revent) { + return nil } + metrics.metricSendEventCommittedCounter.Inc() case cdcpb.Event_PREWRITE: - w.metrics.metricPullEventPrewriteCounter.Inc() + metrics.metricPullEventPrewriteCounter.Inc() state.matcher.putPrewriteRow(entry) case cdcpb.Event_COMMIT: - w.metrics.metricPullEventCommitCounter.Inc() + metrics.metricPullEventCommitCounter.Inc() // NOTE: matchRow should always be called even if the event is stale. if !state.matcher.matchRow(entry, state.isInitialized()) { if !state.isInitialized() { @@ -695,7 +703,7 @@ func (w *regionWorker) handleEventEntry( } // TiKV can send events with StartTs/CommitTs less than startTs. - isStaleEvent := entry.CommitTs <= w.session.startTs + isStaleEvent := entry.CommitTs <= startTs if isStaleEvent { continue } @@ -715,15 +723,12 @@ func (w *regionWorker) handleEventEntry( if err != nil { return errors.Trace(err) } - select { - case w.outputCh <- revent: - w.metrics.metricSendEventCommitCounter.Inc() - case <-ctx.Done(): - return errors.Trace(ctx.Err()) + if !emit(revent) { + return nil } - w.metrics.metricSendEventCommitCounter.Inc() + metrics.metricSendEventCommitCounter.Inc() case cdcpb.Event_ROLLBACK: - w.metrics.metricPullEventRollbackCounter.Inc() + metrics.metricPullEventRollbackCounter.Inc() if !state.isInitialized() { state.matcher.cacheRollbackRow(entry) continue @@ -743,7 +748,7 @@ func (w *regionWorker) handleResolvedTs( regions := make([]uint64, 0, len(revents.regions)) for _, state := range revents.regions { - if state.isStopped() || !state.isInitialized() { + if state.isStale() || !state.isInitialized() { continue } regionID := state.getRegionID() @@ -778,7 +783,7 @@ func (w *regionWorker) handleResolvedTs( default: } for _, state := range revents.regions { - if state.isStopped() || !state.isInitialized() { + if state.isStale() || !state.isInitialized() { continue } state.updateResolvedTs(resolvedTs) @@ -804,7 +809,7 @@ func (w *regionWorker) evictAllRegions() { for _, states := range w.statesManager.states { deletes = deletes[:0] states.iter(func(regionID uint64, regionState *regionFeedState) bool { - if regionState.isStopped() { + if regionState.isStale() { return true } regionState.markStopped() @@ -818,7 +823,6 @@ func (w *regionWorker) evictAllRegions() { }) for _, del := range deletes { w.delRegionState(del.regionID) - del.regionState.setRegionInfoResolvedTs() // since the context used in region worker will be cancelled after // region worker exits, we must use the parent context to prevent // regionErrorInfo loss. diff --git a/cdc/kv/region_worker_test.go b/cdc/kv/region_worker_test.go index 1bbc4f9a876..9438d518651 100644 --- a/cdc/kv/region_worker_test.go +++ b/cdc/kv/region_worker_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/cdcpb" + "github.com/pingcap/tiflow/cdc/kv/regionlock" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/spanz" @@ -49,7 +50,11 @@ func TestRegionStateManagerThreadSafe(t *testing.T) { for i := 0; i < regionCount; i++ { regionID := uint64(1000 + i) regionIDs[i] = regionID - rsm.setState(regionID, ®ionFeedState{requestID: uint64(i + 1), lastResolvedTs: uint64(1000)}) + + state := ®ionFeedState{requestID: uint64(i + 1)} + state.sri.lockedRange = ®ionlock.LockedRange{} + state.updateResolvedTs(1000) + rsm.setState(regionID, state) } var wg sync.WaitGroup @@ -91,8 +96,8 @@ func TestRegionStateManagerThreadSafe(t *testing.T) { for _, regionID := range regionIDs { s, ok := rsm.getState(regionID) require.True(t, ok) - require.Greater(t, s.lastResolvedTs, uint64(1000)) - totalResolvedTs += s.lastResolvedTs + require.Greater(t, s.getLastResolvedTs(), uint64(1000)) + totalResolvedTs += s.getLastResolvedTs() } } @@ -151,7 +156,8 @@ func TestRegionWokerHandleEventEntryEventOutOfOrder(t *testing.T) { state := newRegionFeedState(newSingleRegionInfo( tikv.RegionVerID{}, spanz.ToSpan([]byte{}, spanz.UpperBoundKey), - 0, &tikv.RPCContext{}), 0) + &tikv.RPCContext{}), 0) + state.sri.lockedRange = ®ionlock.LockedRange{} state.start() worker := newRegionWorker(ctx, model.ChangeFeedID{}, s, "") require.Equal(t, 2, cap(worker.outputCh)) @@ -268,28 +274,30 @@ func TestRegionWorkerHandleResolvedTs(t *testing.T) { s1 := newRegionFeedState(singleRegionInfo{ verID: tikv.NewRegionVerID(1, 1, 1), }, 1) - s1.initialized.Store(true) - s1.lastResolvedTs = 9 + s1.sri.lockedRange = ®ionlock.LockedRange{} + s1.setInitialized() + s1.updateResolvedTs(9) s2 := newRegionFeedState(singleRegionInfo{ verID: tikv.NewRegionVerID(2, 2, 2), }, 2) - s2.initialized.Store(true) - s2.lastResolvedTs = 11 + s2.sri.lockedRange = ®ionlock.LockedRange{} + s2.setInitialized() + s2.updateResolvedTs(11) s3 := newRegionFeedState(singleRegionInfo{ verID: tikv.NewRegionVerID(3, 3, 3), }, 3) - s3.initialized.Store(false) - s3.lastResolvedTs = 8 + s3.sri.lockedRange = ®ionlock.LockedRange{} + s3.updateResolvedTs(8) err := w.handleResolvedTs(ctx, &resolvedTsEvent{ resolvedTs: 10, regions: []*regionFeedState{s1, s2, s3}, }) require.Nil(t, err) - require.Equal(t, uint64(10), s1.lastResolvedTs) - require.Equal(t, uint64(11), s2.lastResolvedTs) - require.Equal(t, uint64(8), s3.lastResolvedTs) + require.Equal(t, uint64(10), s1.getLastResolvedTs()) + require.Equal(t, uint64(11), s2.getLastResolvedTs()) + require.Equal(t, uint64(8), s3.getLastResolvedTs()) re := <-w.rtsUpdateCh require.Equal(t, uint64(10), re.resolvedTs) @@ -309,8 +317,10 @@ func TestRegionWorkerHandleEventsBeforeStartTs(t *testing.T) { s1 := newRegionFeedState(newSingleRegionInfo( tikv.RegionVerID{}, spanz.ToSpan([]byte{}, spanz.UpperBoundKey), - 9, &tikv.RPCContext{}), + &tikv.RPCContext{}), 0) + s1.sri.lockedRange = ®ionlock.LockedRange{} + s1.sri.lockedRange.CheckpointTs.Store(9) s1.start() w := newRegionWorker(ctx, model.ChangeFeedID{}, s, "") @@ -319,7 +329,7 @@ func TestRegionWorkerHandleEventsBeforeStartTs(t *testing.T) { regions: []*regionFeedState{s1}, }) require.Nil(t, err) - require.Equal(t, uint64(9), s1.lastResolvedTs) + require.Equal(t, uint64(9), s1.getLastResolvedTs()) timer := time.NewTimer(time.Second) select { diff --git a/cdc/kv/regionlock/region_range_lock.go b/cdc/kv/regionlock/region_range_lock.go index 1fa5762173b..3c375f843df 100644 --- a/cdc/kv/regionlock/region_range_lock.go +++ b/cdc/kv/regionlock/region_range_lock.go @@ -21,10 +21,12 @@ import ( "math" "sync" "sync/atomic" + "time" "github.com/google/btree" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/processor/tablepb" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -116,6 +118,7 @@ type rangeLockEntry struct { regionID uint64 version uint64 waiters []chan<- interface{} + state LockedRange } func rangeLockEntryWithKey(key []byte) *rangeLockEntry { @@ -137,36 +140,36 @@ func (e *rangeLockEntry) String() string { len(e.waiters)) } -var currentID uint64 = 0 - -func allocID() uint64 { - return atomic.AddUint64(¤tID, 1) -} - // RegionRangeLock is specifically used for kv client to manage exclusive region ranges. Acquiring lock will be blocked // if part of its range is already locked. It also manages checkpoint ts of all ranges. The ranges are marked by a // version number, which should comes from the Region's Epoch version. The version is used to compare which range is // new and which is old if two ranges are overlapping. type RegionRangeLock struct { + // ID to identify different RegionRangeLock instances, so logs of different instances can be distinguished. + id uint64 + totalSpan tablepb.Span changefeedLogInfo string + mu sync.Mutex rangeCheckpointTs *rangeTsMap rangeLock *btree.BTreeG[*rangeLockEntry] regionIDLock map[uint64]*rangeLockEntry - // ID to identify different RegionRangeLock instances, so logs of different instances can be distinguished. - id uint64 + stopped bool + refCount uint64 } // NewRegionRangeLock creates a new RegionRangeLock. func NewRegionRangeLock( + id uint64, startKey, endKey []byte, startTs uint64, changefeedLogInfo string, ) *RegionRangeLock { return &RegionRangeLock{ + id: id, + totalSpan: tablepb.Span{StartKey: startKey, EndKey: endKey}, changefeedLogInfo: changefeedLogInfo, rangeCheckpointTs: newRangeTsMap(startKey, endKey, startTs), rangeLock: btree.NewG(16, rangeLockEntryLess), regionIDLock: make(map[uint64]*rangeLockEntry), - id: allocID(), } } @@ -208,6 +211,9 @@ func (l *RegionRangeLock) getOverlappedEntries(startKey, endKey []byte, regionID func (l *RegionRangeLock) tryLockRange(startKey, endKey []byte, regionID, version uint64) (LockRangeResult, []<-chan interface{}) { l.mu.Lock() defer l.mu.Unlock() + if l.stopped { + return LockRangeResult{Status: LockRangeStatusCancel}, nil + } overlappingEntries := l.getOverlappedEntries(startKey, endKey, regionID) @@ -219,6 +225,8 @@ func (l *RegionRangeLock) tryLockRange(startKey, endKey []byte, regionID, versio regionID: regionID, version: version, } + newEntry.state.CheckpointTs.Store(checkpointTs) + newEntry.state.Created = time.Now() l.rangeLock.ReplaceOrInsert(newEntry) l.regionIDLock[regionID] = newEntry @@ -230,9 +238,11 @@ func (l *RegionRangeLock) tryLockRange(startKey, endKey []byte, regionID, versio zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey))) + l.refCount += 1 return LockRangeResult{ Status: LockRangeStatusSuccess, CheckpointTs: checkpointTs, + LockedRange: &newEntry.state, }, nil } @@ -308,7 +318,9 @@ func (l *RegionRangeLock) tryLockRange(startKey, endKey []byte, regionID, versio } // LockRange locks a range with specified version. -func (l *RegionRangeLock) LockRange(ctx context.Context, startKey, endKey []byte, regionID, version uint64) LockRangeResult { +func (l *RegionRangeLock) LockRange( + ctx context.Context, startKey, endKey []byte, regionID, version uint64, +) LockRangeResult { res, signalChs := l.tryLockRange(startKey, endKey, regionID, version) if res.Status != LockRangeStatusWait { @@ -337,22 +349,23 @@ func (l *RegionRangeLock) LockRange(ctx context.Context, startKey, endKey []byte } // UnlockRange unlocks a range and update checkpointTs of the range to specified value. -func (l *RegionRangeLock) UnlockRange(startKey, endKey []byte, regionID, version uint64, checkpointTs uint64) { +// If it returns true it means it is stopped and all ranges are unlocked correctly. +func (l *RegionRangeLock) UnlockRange( + startKey, endKey []byte, regionID, version uint64, + checkpointTs ...uint64, +) (drained bool) { l.mu.Lock() defer l.mu.Unlock() entry, ok := l.rangeLock.Get(rangeLockEntryWithKey(startKey)) - if !ok { log.Panic("unlocking a not locked range", zap.String("changefeed", l.changefeedLogInfo), zap.Uint64("regionID", regionID), zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey)), - zap.Uint64("version", version), - zap.Uint64("checkpointTs", checkpointTs)) + zap.Uint64("version", version)) } - if entry.regionID != regionID { log.Panic("unlocked a range but regionID mismatch", zap.String("changefeed", l.changefeedLogInfo), @@ -369,6 +382,8 @@ func (l *RegionRangeLock) UnlockRange(startKey, endKey []byte, regionID, version zap.String("regionIDLockEntry", l.regionIDLock[regionID].String())) } delete(l.regionIDLock, regionID) + l.refCount -= 1 + drained = l.stopped && l.refCount == 0 if entry.version != version || !bytes.Equal(entry.endKey, endKey) { log.Panic("unlocking region doesn't match the locked region", @@ -377,7 +392,6 @@ func (l *RegionRangeLock) UnlockRange(startKey, endKey []byte, regionID, version zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey)), zap.Uint64("version", version), - zap.Uint64("checkpointTs", checkpointTs), zap.String("foundLockEntry", entry.String())) } @@ -385,17 +399,40 @@ func (l *RegionRangeLock) UnlockRange(startKey, endKey []byte, regionID, version ch <- nil } - _, ok = l.rangeLock.Delete(entry) - if !ok { + if entry, ok = l.rangeLock.Delete(entry); !ok { panic("unreachable") } - l.rangeCheckpointTs.Set(startKey, endKey, checkpointTs) + + var newCheckpointTs uint64 + if len(checkpointTs) > 0 { + newCheckpointTs = checkpointTs[0] + } else { + newCheckpointTs = entry.state.CheckpointTs.Load() + } + + l.rangeCheckpointTs.Set(startKey, endKey, newCheckpointTs) log.Debug("unlocked range", zap.String("changefeed", l.changefeedLogInfo), zap.Uint64("lockID", l.id), zap.Uint64("regionID", entry.regionID), - zap.Uint64("checkpointTs", checkpointTs), + zap.Uint64("checkpointTs", newCheckpointTs), zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey))) + return +} + +// RefCount returns how many ranges are locked. +func (l *RegionRangeLock) RefCount() uint64 { + l.mu.Lock() + defer l.mu.Unlock() + return l.refCount +} + +// Stop stops the instance. +func (l *RegionRangeLock) Stop() (drained bool) { + l.mu.Lock() + defer l.mu.Unlock() + l.stopped = true + return l.stopped && l.refCount == 0 } const ( @@ -410,15 +447,82 @@ const ( ) // LockRangeResult represents the result of LockRange method of RegionRangeLock. -// If Status is LockRangeStatusSuccess, the CheckpointTs field will be the minimal checkpoint ts among the locked -// range. +// If Status is LockRangeStatusSuccess: +// - CheckpointTs will be the minimal checkpoint ts among the locked range; +// - LockedRange is for recording real-time state changes; +// // If Status is LockRangeStatusWait, it means the lock cannot be acquired immediately. WaitFn must be invoked to // continue waiting and acquiring the lock. +// // If Status is LockRangeStatusStale, it means the LockRange request is stale because there's already a overlapping // locked range, whose version is greater or equals to the requested one. type LockRangeResult struct { Status int CheckpointTs uint64 + LockedRange *LockedRange WaitFn func() LockRangeResult RetryRanges []tablepb.Span } + +// LockedRange is returned by `RegionRangeLock.LockRange`, which can be used to +// collect informations for the range. And collected informations can be accessed +// by iterating `RegionRangeLock`. +type LockedRange struct { + CheckpointTs atomic.Uint64 + Initialzied atomic.Bool + Created time.Time +} + +// CollectLockedRangeAttrs collects locked range attributes. +func (l *RegionRangeLock) CollectLockedRangeAttrs( + action func(regionID uint64, state *LockedRange), +) (r CollectedLockedRangeAttrs) { + l.mu.Lock() + defer l.mu.Unlock() + r.FastestRegion.CheckpointTs = 0 + r.SlowestRegion.CheckpointTs = math.MaxUint64 + + lastEnd := l.totalSpan.StartKey + l.rangeLock.Ascend(func(item *rangeLockEntry) bool { + if action != nil { + action(item.regionID, &item.state) + } + if spanz.EndCompare(lastEnd, item.startKey) < 0 { + r.Holes = append(r.Holes, tablepb.Span{StartKey: lastEnd, EndKey: item.startKey}) + } + ckpt := item.state.CheckpointTs.Load() + if ckpt > r.FastestRegion.CheckpointTs { + r.FastestRegion.RegionID = item.regionID + r.FastestRegion.CheckpointTs = ckpt + r.FastestRegion.Initialized = item.state.Initialzied.Load() + r.FastestRegion.Created = item.state.Created + } + if ckpt < r.SlowestRegion.CheckpointTs { + r.SlowestRegion.RegionID = item.regionID + r.SlowestRegion.CheckpointTs = ckpt + r.SlowestRegion.Initialized = item.state.Initialzied.Load() + r.SlowestRegion.Created = item.state.Created + } + lastEnd = item.endKey + return true + }) + if spanz.EndCompare(lastEnd, l.totalSpan.EndKey) < 0 { + r.Holes = append(r.Holes, tablepb.Span{StartKey: lastEnd, EndKey: l.totalSpan.EndKey}) + } + return +} + +// CollectedLockedRangeAttrs returns by `RegionRangeLock.CollectedLockedRangeAttrs`. +type CollectedLockedRangeAttrs struct { + Holes []tablepb.Span + FastestRegion LockedRangeAttrs + SlowestRegion LockedRangeAttrs +} + +// LockedRangeAttrs is like `LockedRange`, but only contains some read-only attributes. +type LockedRangeAttrs struct { + RegionID uint64 + CheckpointTs uint64 + Initialized bool + Created time.Time +} diff --git a/cdc/kv/regionlock/region_range_lock_test.go b/cdc/kv/regionlock/region_range_lock_test.go index 8b1a5690190..af887248164 100644 --- a/cdc/kv/regionlock/region_range_lock_test.go +++ b/cdc/kv/regionlock/region_range_lock_test.go @@ -90,7 +90,7 @@ func TestRegionRangeLock(t *testing.T) { t.Parallel() ctx := context.TODO() - l := NewRegionRangeLock([]byte("a"), []byte("h"), math.MaxUint64, "") + l := NewRegionRangeLock(1, []byte("a"), []byte("h"), math.MaxUint64, "") mustLockRangeSuccess(ctx, t, l, "a", "e", 1, 1, math.MaxUint64) unlockRange(l, "a", "e", 1, 1, 100) @@ -107,7 +107,7 @@ func TestRegionRangeLock(t *testing.T) { func TestRegionRangeLockStale(t *testing.T) { t.Parallel() - l := NewRegionRangeLock([]byte("a"), []byte("z"), math.MaxUint64, "") + l := NewRegionRangeLock(1, []byte("a"), []byte("z"), math.MaxUint64, "") ctx := context.TODO() mustLockRangeSuccess(ctx, t, l, "c", "g", 1, 10, math.MaxUint64) mustLockRangeSuccess(ctx, t, l, "j", "n", 2, 8, math.MaxUint64) @@ -130,7 +130,7 @@ func TestRegionRangeLockLockingRegionID(t *testing.T) { t.Parallel() ctx := context.TODO() - l := NewRegionRangeLock([]byte("a"), []byte("z"), math.MaxUint64, "") + l := NewRegionRangeLock(1, []byte("a"), []byte("z"), math.MaxUint64, "") mustLockRangeSuccess(ctx, t, l, "c", "d", 1, 10, math.MaxUint64) mustLockRangeStale(ctx, t, l, "e", "f", 1, 5, "e", "f") @@ -166,7 +166,7 @@ func TestRegionRangeLockCanBeCancelled(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) - l := NewRegionRangeLock([]byte("a"), []byte("z"), math.MaxUint64, "") + l := NewRegionRangeLock(1, []byte("a"), []byte("z"), math.MaxUint64, "") mustLockRangeSuccess(ctx, t, l, "g", "h", 1, 10, math.MaxUint64) wait := mustLockRangeWait(ctx, t, l, "g", "h", 1, 12) cancel() From 04685c5bd633ff26a5dc74967a269ca0eb12287d Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 16 Nov 2023 14:54:47 +0800 Subject: [PATCH 04/23] puller(cdc): add metrics for slowest changefeed puller (#10054) (#10068) close pingcap/tiflow#10053 --- .../internal/v3/replication/metrics.go | 18 +++++++++ .../v3/replication/replication_manager.go | 39 ++++++++++++++----- metrics/grafana/ticdc.json | 20 +++++++++- 3 files changed, 65 insertions(+), 12 deletions(-) diff --git a/cdc/scheduler/internal/v3/replication/metrics.go b/cdc/scheduler/internal/v3/replication/metrics.go index ccd546f67fe..a427035eff5 100644 --- a/cdc/scheduler/internal/v3/replication/metrics.go +++ b/cdc/scheduler/internal/v3/replication/metrics.go @@ -125,6 +125,21 @@ var ( Name: "slow_table_region_count", Help: "The number of regions captured by the slowest table", }, []string{"namespace", "changefeed"}) + + slowestTablePullerResolvedTs = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "scheduler", + Name: "slow_table_puller_resolved_ts", + Help: "Puller Slowest ResolvedTs", + }, []string{"namespace", "changefeed"}) + slowestTablePullerResolvedTsLag = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "scheduler", + Name: "slow_table_puller_resolved_ts_lag", + Help: "Puller Slowest ResolvedTs lag", + }, []string{"namespace", "changefeed"}) ) // InitMetrics registers all metrics used in scheduler @@ -144,4 +159,7 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(slowestTableStageCheckpointTsLagHistogramVec) registry.MustRegister(slowestTableStageResolvedTsLagHistogramVec) registry.MustRegister(slowestTableRegionGaugeVec) + + registry.MustRegister(slowestTablePullerResolvedTs) + registry.MustRegister(slowestTablePullerResolvedTsLag) } diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 0ec7c7a671b..4228cc0ec75 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -147,7 +147,8 @@ type Manager struct { //nolint:revive maxTaskConcurrency int changefeedID model.ChangeFeedID - slowestTableID tablepb.Span + slowestPuller tablepb.Span + slowestSink tablepb.Span acceptAddTableTask int acceptRemoveTableTask int acceptMoveTableTask int @@ -589,14 +590,16 @@ func (r *Manager) AdvanceCheckpoint( } }() + r.slowestPuller = tablepb.Span{} + r.slowestSink = tablepb.Span{} + var slowestPullerResolvedTs uint64 = math.MaxUint64 + newCheckpointTs, newResolvedTs = math.MaxUint64, math.MaxUint64 - slowestRange := tablepb.Span{} cannotProceed := false - lastSpan := tablepb.Span{} currentTables.Iter(func(tableID model.TableID, tableStart, tableEnd tablepb.Span) bool { tableSpanFound, tableHasHole := false, false tableSpanStartFound, tableSpanEndFound := false, false - lastSpan = tablepb.Span{} + lastSpan := tablepb.Span{} r.spans.AscendRange(tableStart, tableEnd, func(span tablepb.Span, table *ReplicationSet) bool { if lastSpan.TableID != 0 && !bytes.Equal(lastSpan.EndKey, span.StartKey) { @@ -620,11 +623,19 @@ func (r *Manager) AdvanceCheckpoint( // Find the minimum checkpoint ts and resolved ts. if newCheckpointTs > table.Checkpoint.CheckpointTs { newCheckpointTs = table.Checkpoint.CheckpointTs - slowestRange = span + r.slowestSink = span } if newResolvedTs > table.Checkpoint.ResolvedTs { newResolvedTs = table.Checkpoint.ResolvedTs } + + // Find the minimum puller resolved ts. + if pullerCkpt, ok := table.Stats.StageCheckpoints["puller-egress"]; ok { + if slowestPullerResolvedTs > pullerCkpt.ResolvedTs { + slowestPullerResolvedTs = pullerCkpt.ResolvedTs + r.slowestPuller = span + } + } return true }) if !tableSpanFound || !tableSpanStartFound || !tableSpanEndFound || tableHasHole { @@ -657,9 +668,6 @@ func (r *Manager) AdvanceCheckpoint( } return checkpointCannotProceed, checkpointCannotProceed } - if slowestRange.TableID != 0 { - r.slowestTableID = slowestRange - } // If currentTables is empty, we should advance newResolvedTs to global barrier ts and // advance newCheckpointTs to min table barrier ts. @@ -745,9 +753,9 @@ func (r *Manager) CollectMetrics() { cf := r.changefeedID tableGauge. WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.spans.Len())) - if table, ok := r.spans.Get(r.slowestTableID); ok { + if table, ok := r.spans.Get(r.slowestSink); ok { slowestTableIDGauge. - WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.slowestTableID.TableID)) + WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.slowestSink.TableID)) slowestTableStateGauge. WithLabelValues(cf.Namespace, cf.ID).Set(float64(table.State)) phyCkpTs := oracle.ExtractPhysical(table.Checkpoint.CheckpointTs) @@ -829,6 +837,17 @@ func (r *Manager) CollectMetrics() { WithLabelValues(cf.Namespace, cf.ID, ReplicationSetState(s).String()). Set(float64(counter)) } + + if table, ok := r.spans.Get(r.slowestSink); ok { + if pullerCkpt, ok := table.Stats.StageCheckpoints["puller-egress"]; ok { + phyCkptTs := oracle.ExtractPhysical(pullerCkpt.ResolvedTs) + slowestTablePullerResolvedTs.WithLabelValues(cf.Namespace, cf.ID).Set(float64(phyCkptTs)) + + phyCurrentTs := oracle.ExtractPhysical(table.Stats.CurrentTs) + lag := float64(phyCurrentTs-phyCkptTs) / 1e3 + slowestTablePullerResolvedTsLag.WithLabelValues(cf.Namespace, cf.ID).Set(lag) + } + } } // CleanMetrics cleans metrics. diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 6be8d5d5f0a..cdf9a324da5 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -3426,8 +3426,16 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{changefeed}}-barrier", "refId": "C" + }, + { + "exemplar": true, + "expr": "max(ticdc_scheduler_slow_table_puller_resolved_ts{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "hide": false, + "interval": "", + "legendFormat": "{{changefeed}}-puller", + "refId": "B" } ], "thresholds": [], @@ -3633,8 +3641,16 @@ "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{changefeed}}-barrier", "refId": "C" + }, + { + "exemplar": true, + "expr": "max(ticdc_scheduler_slow_table_puller_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "hide": false, + "interval": "", + "legendFormat": "{{changefeed}}-puller", + "refId": "A" } ], "thresholds": [], From 7b984d67a587c5968d218e75a6f7f2631e53c5ee Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Nov 2023 16:02:49 +0800 Subject: [PATCH 05/23] sink(cdc): always handle sink failures for cases with sync-point enabled (#10132) (#10141) close pingcap/tiflow#10091 --- cdc/processor/sinkmanager/manager.go | 45 +++++++++++++++++-- cdc/processor/sinkmanager/manager_test.go | 42 +++++++++++++++++ .../sinkmanager/table_sink_worker.go | 19 ++------ .../sinkmanager/table_sink_wrapper.go | 14 ++++++ .../dmlsink/blackhole/black_hole_dml_sink.go | 18 +++++--- cdc/sink/tablesink/table_sink.go | 2 + cdc/sink/tablesink/table_sink_impl.go | 8 ++++ 7 files changed, 121 insertions(+), 27 deletions(-) diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index 35391e1f44e..f126f13ec9c 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -530,13 +530,50 @@ func (m *SinkManager) generateSinkTasks(ctx context.Context) error { slowestTableProgress := progs[i] lowerBound := slowestTableProgress.nextLowerBoundPos upperBound := m.getUpperBound(tableSink.getUpperBoundTs()) - // The table has no available progress. - if lowerBound.Compare(upperBound) >= 0 { + + if !tableSink.initTableSink() { + // The table hasn't been attached to a sink. m.sinkProgressHeap.push(slowestTableProgress) continue } - // The table hasn't been attached to a sink. - if !tableSink.initTableSink() { + + if sinkErr := tableSink.checkTableSinkHealth(); sinkErr != nil { + switch errors.Cause(sinkErr).(type) { + case tablesink.SinkInternalError: + tableSink.closeAndClearTableSink() + if restartErr := tableSink.restart(ctx); restartErr == nil { + // Restart the table sink based on the checkpoint position. + ckpt := tableSink.getCheckpointTs().ResolvedMark() + lastWrittenPos := engine.Position{StartTs: ckpt - 1, CommitTs: ckpt} + p := &progress{ + span: tableSink.span, + nextLowerBoundPos: lastWrittenPos.Next(), + version: slowestTableProgress.version, + } + m.sinkProgressHeap.push(p) + log.Info("table sink has been restarted", + zap.String("namespace", m.changefeedID.Namespace), + zap.String("changefeed", m.changefeedID.ID), + zap.Stringer("span", &tableSink.span), + zap.Any("lastWrittenPos", lastWrittenPos), + zap.String("sinkError", sinkErr.Error())) + } else { + m.sinkProgressHeap.push(slowestTableProgress) + log.Warn("table sink restart fail", + zap.String("namespace", m.changefeedID.Namespace), + zap.String("changefeed", m.changefeedID.ID), + zap.Stringer("span", &tableSink.span), + zap.String("sinkError", sinkErr.Error()), + zap.Error(restartErr)) + } + default: + return sinkErr + } + continue + } + + // The table has no available progress. + if lowerBound.Compare(upperBound) >= 0 { m.sinkProgressHeap.push(slowestTableProgress) continue } diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index b70e17fc3ab..b1447fe19a9 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -372,3 +372,45 @@ func TestSinkManagerNeedsStuckCheck(t *testing.T) { require.False(t, manager.needsStuckCheck()) } + +func TestSinkManagerRestartTableSinks(t *testing.T) { + failpoint.Enable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/SinkWorkerTaskHandlePause", "return") + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/SinkWorkerTaskHandlePause") + + ctx, cancel := context.WithCancel(context.Background()) + errCh := make(chan error, 16) + changefeedInfo := getChangefeedInfo() + manager, _, _ := CreateManagerWithMemEngine(t, ctx, model.ChangeFeedID{}, changefeedInfo, errCh) + defer func() { + cancel() + manager.Close() + }() + + span := tablepb.Span{TableID: 1} + manager.AddTable(span, 1, 100) + require.Nil(t, manager.StartTable(span, 2)) + table, exists := manager.tableSinks.Load(span) + require.True(t, exists) + + table.(*tableSinkWrapper).updateReceivedSorterResolvedTs(4) + table.(*tableSinkWrapper).updateBarrierTs(4) + select { + case task := <-manager.sinkTaskChan: + require.Equal(t, engine.Position{StartTs: 0, CommitTs: 3}, task.lowerBound) + task.callback(engine.Position{StartTs: 3, CommitTs: 4}) + case <-time.After(2 * time.Second): + panic("should always get a sink task") + } + + // With the failpoint blackhole/WriteEventsFail enabled, sink manager should restarts + // the table sink at its checkpoint. + failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/dmlsink/blackhole/WriteEventsFail", "1*return") + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/dmlsink/blackhole/WriteEventsFail") + select { + case task := <-manager.sinkTaskChan: + require.Equal(t, engine.Position{StartTs: 2, CommitTs: 2}, task.lowerBound) + task.callback(engine.Position{StartTs: 3, CommitTs: 4}) + case <-time.After(2 * time.Second): + panic("should always get a sink task") + } +} diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index b04ce8f7c40..42c493f3918 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -96,6 +96,7 @@ func newSinkWorker( } func (w *sinkWorker) handleTasks(ctx context.Context, taskChan <-chan *sinkTask) error { + failpoint.Inject("SinkWorkerTaskHandlePause", func() { <-ctx.Done() }) for { select { case <-ctx.Done(): @@ -169,25 +170,11 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // events have been reported. Then we can continue the table // at the checkpoint position. case tablesink.SinkInternalError: - task.tableSink.closeAndClearTableSink() // After the table sink is cleared all pending events are sent out or dropped. // So we can re-add the table into sinkMemQuota. w.sinkMemQuota.ClearTable(task.tableSink.span) - - // Restart the table sink based on the checkpoint position. - if err := task.tableSink.restart(ctx); err == nil { - checkpointTs := task.tableSink.getCheckpointTs() - ckpt := checkpointTs.ResolvedMark() - lastWrittenPos := engine.Position{StartTs: ckpt - 1, CommitTs: ckpt} - performCallback(lastWrittenPos) - log.Info("table sink has been restarted", - zap.String("namespace", w.changefeedID.Namespace), - zap.String("changefeed", w.changefeedID.ID), - zap.Stringer("span", &task.span), - zap.Any("lastWrittenPos", lastWrittenPos), - zap.String("sinkError", finalErr.Error())) - finalErr = err - } + performCallback(advancer.lastPos) + finalErr = nil default: } } diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index feb7921d794..5e10a4bdcb7 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -166,6 +166,11 @@ func (t *tableSinkWrapper) start(ctx context.Context, startTs model.Ts) (err err break } } + if model.NewResolvedTs(startTs).Greater(t.tableSink.checkpointTs) { + t.tableSink.checkpointTs = model.NewResolvedTs(startTs) + t.tableSink.resolvedTs = model.NewResolvedTs(startTs) + t.tableSink.advanced = time.Now() + } t.state.Store(tablepb.TableStateReplicating) return nil } @@ -363,6 +368,15 @@ func (t *tableSinkWrapper) doTableSinkClear() { t.tableSink.version = 0 } +func (t *tableSinkWrapper) checkTableSinkHealth() (err error) { + t.tableSink.RLock() + defer t.tableSink.RUnlock() + if t.tableSink.s != nil { + err = t.tableSink.s.CheckHealth() + } + return +} + // When the attached sink fail, there can be some events that have already been // committed at downstream but we don't know. So we need to update `replicateTs` // of the table so that we can re-send those events later. diff --git a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go index 6681e70f508..3d146a50a89 100644 --- a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go +++ b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go @@ -14,6 +14,8 @@ package blackhole import ( + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" @@ -33,14 +35,16 @@ func NewDMLSink() *DMLSink { } // WriteEvents log the events. -func (s *DMLSink) WriteEvents(rows ...*dmlsink.CallbackableEvent[*model.RowChangedEvent]) error { - for _, row := range rows { - // NOTE: don't change the log, some tests depend on it. - log.Debug("BlackHoleSink: WriteEvents", zap.Any("row", row.Event)) - row.Callback() +func (s *DMLSink) WriteEvents(rows ...*dmlsink.CallbackableEvent[*model.RowChangedEvent]) (err error) { + failpoint.Inject("WriteEventsFail", func() { err = errors.New("InjectedErrorForWriteEventsFail") }) + if err == nil { + for _, row := range rows { + // NOTE: don't change the log, some tests depend on it. + log.Debug("BlackHoleSink: WriteEvents", zap.Any("row", row.Event)) + row.Callback() + } } - - return nil + return } // Close do nothing. diff --git a/cdc/sink/tablesink/table_sink.go b/cdc/sink/tablesink/table_sink.go index 588b30aabf9..69f6f61db11 100644 --- a/cdc/sink/tablesink/table_sink.go +++ b/cdc/sink/tablesink/table_sink.go @@ -37,6 +37,8 @@ type TableSink interface { Close() // AsyncClose closes the table sink asynchronously. Returns true if it's closed. AsyncClose() bool + // CheckHealth checks whether the associated sink backend is healthy or not. + CheckHealth() error } // SinkInternalError means the error comes from sink internal. diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index da067501727..5ee1b869dba 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -158,6 +158,14 @@ func (e *EventTableSink[E, P]) AsyncClose() bool { return false } +// CheckHealth checks whether the associated sink backend is healthy or not. +func (e *EventTableSink[E, P]) CheckHealth() error { + if err := e.backendSink.WriteEvents(); err != nil { + return SinkInternalError{err} + } + return nil +} + func (e *EventTableSink[E, P]) freeze() { // Notice: We have to set the state to stopping first, // otherwise the progressTracker may be advanced incorrectly. From 7ae0988cbaae8c8b29091aededec62cc51f89718 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 11:10:23 +0800 Subject: [PATCH 06/23] scheduler(ticdc): Updating ReplicationSet.stats only when stats is not empty (#10225) (#10229) close pingcap/tiflow#10224 --- cdc/scheduler/internal/v3/replication/replication_set.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cdc/scheduler/internal/v3/replication/replication_set.go b/cdc/scheduler/internal/v3/replication/replication_set.go index fec432499a8..eab841b246c 100644 --- a/cdc/scheduler/internal/v3/replication/replication_set.go +++ b/cdc/scheduler/internal/v3/replication/replication_set.go @@ -1020,7 +1020,11 @@ func (r *ReplicationSet) updateCheckpointAndStats( zap.Any("checkpointTs", r.Checkpoint.CheckpointTs), zap.Any("resolvedTs", r.Checkpoint.ResolvedTs)) } - r.Stats = stats + + // we only update stats when stats is not empty, because we only collect stats every 10s. + if stats.Size() > 0 { + r.Stats = stats + } } // SetHeap is a max-heap, it implements heap.Interface. From 1f6a44b0f7784abf9849f350e5379fce724e76c4 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 14:41:52 +0800 Subject: [PATCH 07/23] cdc: fixes minor bugs #10168 and #10169 (#10170) (#10190) close pingcap/tiflow#10168 --- cdc/kv/client.go | 4 ++++ cdc/processor/sinkmanager/tasks.go | 4 ++-- cdc/processor/sinkmanager/tasks_test.go | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index a419c787ecd..b1055693fd5 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -73,6 +73,8 @@ const ( // failed region will be reloaded via `BatchLoadRegionsWithKeyRange` API. So we // don't need to force reload region anymore. regionScheduleReload = false + + scanRegionsConcurrency = 1024 ) // time interval to force kv client to terminate gRPC stream and reconnect @@ -432,6 +434,8 @@ func (s *eventFeedSession) eventFeed(ctx context.Context) error { g.Go(func() error { return s.logSlowRegions(ctx) }) g.Go(func() error { + g, ctx := errgroup.WithContext(ctx) + g.SetLimit(scanRegionsConcurrency) for { select { case <-ctx.Done(): diff --git a/cdc/processor/sinkmanager/tasks.go b/cdc/processor/sinkmanager/tasks.go index e4b94551191..dd415428c83 100644 --- a/cdc/processor/sinkmanager/tasks.go +++ b/cdc/processor/sinkmanager/tasks.go @@ -39,8 +39,8 @@ var ( maxUpdateIntervalSize = defaultMaxUpdateIntervalSize // Sink manager schedules table tasks based on lag. Limit the max task range - // can be helpful to reduce changefeed latency. - maxTaskTimeRange = 5 * time.Second + // can be helpful to reduce changefeed latency for large initial data. + maxTaskTimeRange = 30 * time.Minute ) // Used to record the progress of the table. diff --git a/cdc/processor/sinkmanager/tasks_test.go b/cdc/processor/sinkmanager/tasks_test.go index fac3788a07f..15a1f559259 100644 --- a/cdc/processor/sinkmanager/tasks_test.go +++ b/cdc/processor/sinkmanager/tasks_test.go @@ -37,7 +37,7 @@ func TestValidateAndAdjustBound(t *testing.T) { StartTs: 439333515018895365, CommitTs: 439333515018895366, }, - taskTimeRange: 10 * time.Second, + taskTimeRange: 60 * time.Minute, expectAdjust: true, }, { From f6f03f7d598cb126290fdb6152866b9ca8a8de52 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 16:37:23 +0800 Subject: [PATCH 08/23] codec(ticdc): canal-json decouple java type from the mysql type (#10087) (#10127) close pingcap/tiflow#10086 --- cdc/kv/region_worker.go | 5 ++ cdc/server/server.go | 8 +-- pkg/sink/codec/canal/canal_entry.go | 67 +++++++++---------- pkg/sink/codec/canal/canal_entry_test.go | 10 +-- .../canal/canal_json_row_event_encoder.go | 12 +--- 5 files changed, 47 insertions(+), 55 deletions(-) diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index 5a3771d65cb..e49b5891fd9 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -41,6 +41,7 @@ import ( var ( regionWorkerPool workerpool.WorkerPool workerPoolOnce sync.Once + workerPoolLock sync.Mutex // The magic number here is keep the same with some magic numbers in some // other components in TiCDC, including worker pool task chan size, mounter // chan size etc. @@ -409,6 +410,8 @@ func (w *regionWorker) processEvent(ctx context.Context, event *regionStatefulEv func (w *regionWorker) initPoolHandles() { handles := make([]workerpool.EventHandle, 0, w.concurrency) + workerPoolLock.Lock() + defer workerPoolLock.Unlock() for i := 0; i < w.concurrency; i++ { poolHandle := regionWorkerPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error { event := eventI.(*regionStatefulEvent) @@ -864,6 +867,8 @@ func getWorkerPoolSize() (size int) { func InitWorkerPool() { workerPoolOnce.Do(func() { size := getWorkerPoolSize() + workerPoolLock.Lock() + defer workerPoolLock.Unlock() regionWorkerPool = workerpool.NewDefaultWorkerPool(size) }) } diff --git a/cdc/server/server.go b/cdc/server/server.go index dc7e047ce73..d585d402a4f 100644 --- a/cdc/server/server.go +++ b/cdc/server/server.go @@ -343,10 +343,6 @@ func (s *server) run(ctx context.Context) (err error) { eg, egCtx := errgroup.WithContext(ctx) - eg.Go(func() error { - return s.capture.Run(egCtx) - }) - eg.Go(func() error { return s.upstreamPDHealthChecker(egCtx) }) @@ -371,6 +367,10 @@ func (s *server) run(ctx context.Context) (err error) { return nil }) + eg.Go(func() error { + return s.capture.Run(egCtx) + }) + return eg.Wait() } diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index 4b897bf3f99..41f8e4bf6b2 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -78,7 +78,7 @@ func (b *canalEntryBuilder) buildHeader(commitTs uint64, schema string, table st // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/dbsync/src/main/java/com/taobao/tddl/dbsync/binlog/event/RowsLogBuffer.java#L276-L1147 // all value will be represented in string type // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L760-L855 -func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.JavaSQLType) (result string, err error) { +func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (result string, err error) { // value would be nil, if no value insert for the column. if value == nil { return "", nil @@ -96,20 +96,14 @@ func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.Jav case string: result = v case []byte: - // JavaSQLTypeVARCHAR / JavaSQLTypeCHAR / JavaSQLTypeBLOB / JavaSQLTypeCLOB / - // special handle for text and blob - // see https://github.com/alibaba/canal/blob/9f6021cf36f78cc8ac853dcf37a1769f359b868b/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L801 - switch javaType { - // for normal text - case internal.JavaSQLTypeVARCHAR, internal.JavaSQLTypeCHAR, internal.JavaSQLTypeCLOB: - result = string(v) - default: - // JavaSQLTypeBLOB + if isBinary { decoded, err := b.bytesDecoder.Bytes(v) if err != nil { return "", err } result = string(decoded) + } else { + result = string(v) } default: result = fmt.Sprintf("%v", v) @@ -119,21 +113,21 @@ func (b *canalEntryBuilder) formatValue(value interface{}, javaType internal.Jav // build the Column in the canal RowData // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L756-L872 -func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated bool) (*canal.Column, error) { - mysqlType := getMySQLType(c) - javaType, err := getJavaSQLType(c, mysqlType) +func (b *canalEntryBuilder) buildColumn(c *model.Column, updated bool) (*canal.Column, error) { + mysqlType := getMySQLType(c.Type, c.Flag) + javaType, err := getJavaSQLType(c.Value, c.Type, c.Flag) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } - value, err := b.formatValue(c.Value, javaType) + value, err := b.formatValue(c.Value, c.Flag.IsBinary()) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } canalColumn := &canal.Column{ SqlType: int32(javaType), - Name: colName, + Name: c.Name, IsKey: c.Flag.IsPrimaryKey(), Updated: updated, IsNullPresent: &canal.Column_IsNull{IsNull: c.Value == nil}, @@ -150,7 +144,7 @@ func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKey if column == nil { continue } - c, err := b.buildColumn(column, column.Name, !e.IsDelete()) + c, err := b.buildColumn(column, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -166,7 +160,7 @@ func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKey if onlyHandleKeyColumns && !column.Flag.IsHandleKey() { continue } - c, err := b.buildColumn(column, column.Name, !e.IsDelete()) + c, err := b.buildColumn(column, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -299,40 +293,39 @@ func isCanalDDL(t canal.EventType) bool { return false } -func getJavaSQLType(c *model.Column, mysqlType string) (result internal.JavaSQLType, err error) { - javaType := internal.MySQLType2JavaType(c.Type, c.Flag.IsBinary()) +func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (result internal.JavaSQLType, err error) { + javaType := internal.MySQLType2JavaType(tp, flag.IsBinary()) switch javaType { case internal.JavaSQLTypeBINARY, internal.JavaSQLTypeVARBINARY, internal.JavaSQLTypeLONGVARBINARY: - if strings.Contains(mysqlType, "text") { - return internal.JavaSQLTypeCLOB, nil + if flag.IsBinary() { + return internal.JavaSQLTypeBLOB, nil } - return internal.JavaSQLTypeBLOB, nil + return internal.JavaSQLTypeCLOB, nil } // flag `isUnsigned` only for `numerical` and `bit`, `year` data type. - if !c.Flag.IsUnsigned() { + if !flag.IsUnsigned() { return javaType, nil } + switch tp { // for year, to `int64`, others to `uint64`. // no need to promote type for `year` and `bit` - if c.Type == mysql.TypeYear || c.Type == mysql.TypeBit { + case mysql.TypeYear, mysql.TypeBit: return javaType, nil - } - - if c.Type == mysql.TypeFloat || c.Type == mysql.TypeDouble || c.Type == mysql.TypeNewDecimal { + case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeNewDecimal: return javaType, nil } // for **unsigned** integral types, type would be `uint64` or `string`. see reference: // https://github.com/pingcap/tiflow/blob/1e3dd155049417e3fd7bf9b0a0c7b08723b33791/cdc/entry/mounter.go#L501 // https://github.com/pingcap/tidb/blob/6495a5a116a016a3e077d181b8c8ad81f76ac31b/types/datum.go#L423-L455 - if c.Value == nil { + if value == nil { return javaType, nil } var number uint64 - switch v := c.Value.(type) { + switch v := value.(type) { case uint64: number = v case string: @@ -342,7 +335,7 @@ func getJavaSQLType(c *model.Column, mysqlType string) (result internal.JavaSQLT } number = a default: - return javaType, errors.Errorf("unexpected type for unsigned value: %+v, column: %+v", reflect.TypeOf(v), c) + return javaType, errors.Errorf("unexpected type for unsigned value: %+v, tp: %+v", reflect.TypeOf(v), tp) } // Some special cases handled in canal @@ -352,7 +345,7 @@ func getJavaSQLType(c *model.Column, mysqlType string) (result internal.JavaSQLT // SmallInt, 2byte, [-32768, 32767], [0, 65535], if a > 32767 // Int, 4byte, [-2147483648, 2147483647], [0, 4294967295], if a > 2147483647 // BigInt, 8byte, [-2<<63, 2 << 63 - 1], [0, 2 << 64 - 1], if a > 2 << 63 - 1 - switch c.Type { + switch tp { case mysql.TypeTiny: if number > math.MaxInt8 { javaType = internal.JavaSQLTypeSMALLINT @@ -388,20 +381,20 @@ func trimUnsignedFromMySQLType(mysqlType string) string { return strings.TrimSuffix(mysqlType, " unsigned") } -func getMySQLType(c *model.Column) string { - mysqlType := types.TypeStr(c.Type) +func getMySQLType(tp byte, flag model.ColumnFlagType) string { + mysqlType := types.TypeStr(tp) // make `mysqlType` representation keep the same as the canal official implementation - mysqlType = withUnsigned4MySQLType(mysqlType, c.Flag.IsUnsigned()) + mysqlType = withUnsigned4MySQLType(mysqlType, flag.IsUnsigned()) - if !c.Flag.IsBinary() { + if !flag.IsBinary() { return mysqlType } - if types.IsTypeBlob(c.Type) { + if types.IsTypeBlob(tp) { return strings.Replace(mysqlType, "text", "blob", 1) } - if types.IsTypeChar(c.Type) { + if types.IsTypeChar(tp) { return strings.Replace(mysqlType, "char", "binary", 1) } diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index 21f65f2b7ac..462dd09314e 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -30,16 +30,16 @@ func TestGetMySQLTypeAndJavaSQLType(t *testing.T) { t.Parallel() canalEntryBuilder := newCanalEntryBuilder() for _, item := range testColumnsTable { - obtainedMySQLType := getMySQLType(item.column) + obtainedMySQLType := getMySQLType(item.column.Type, item.column.Flag) require.Equal(t, item.expectedMySQLType, obtainedMySQLType) - obtainedJavaSQLType, err := getJavaSQLType(item.column, obtainedMySQLType) - require.Nil(t, err) + obtainedJavaSQLType, err := getJavaSQLType(item.column.Value, item.column.Type, item.column.Flag) + require.NoError(t, err) require.Equal(t, item.expectedJavaSQLType, obtainedJavaSQLType) + obtainedFinalValue, err := canalEntryBuilder.formatValue(item.column.Value, item.column.Flag.IsBinary()) + require.NoError(t, err) if !item.column.Flag.IsBinary() { - obtainedFinalValue, err := canalEntryBuilder.formatValue(item.column.Value, obtainedJavaSQLType) - require.Nil(t, err) require.Equal(t, item.expectedEncodedValue, obtainedFinalValue) } } diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index 618fae95c7b..2e4a1619ee7 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -56,12 +56,7 @@ func fillColumns(columns []*model.Column, out *jwriter.Writer, } else { out.RawByte(',') } - mysqlType := getMySQLType(col) - javaType, err := getJavaSQLType(col, mysqlType) - if err != nil { - return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) - } - value, err := builder.formatValue(col.Value, javaType) + value, err := builder.formatValue(col.Value, col.Flag.IsBinary()) if err != nil { return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } @@ -171,15 +166,14 @@ func newJSONMessageForDML( } else { out.RawByte(',') } - mysqlType := getMySQLType(col) - javaType, err := getJavaSQLType(col, mysqlType) + javaType, err := getJavaSQLType(col.Value, col.Type, col.Flag) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } out.String(col.Name) out.RawByte(':') out.Int32(int32(javaType)) - mysqlTypeMap[col.Name] = mysqlType + mysqlTypeMap[col.Name] = getMySQLType(col.Type, col.Flag) } } if emptyColumn { From 8de9f2721136b0069522d4244cebe17845428690 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 18:26:54 +0800 Subject: [PATCH 09/23] kv-client(cdc): correct conditions of canceling grpc streams (#10237) (#10241) close pingcap/tiflow#10239 --- cdc/kv/client.go | 4 ++-- cdc/kv/region_worker.go | 7 ++++++- cdc/kv/region_worker_test.go | 4 ++-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index b1055693fd5..65742439aa4 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -1016,7 +1016,7 @@ func (s *eventFeedSession) receiveFromStream( // always create a new region worker, because `receiveFromStream` is ensured // to call exactly once from outer code logic - worker := newRegionWorker(parentCtx, s.changefeed, s, addr) + worker := newRegionWorker(parentCtx, s.changefeed, s, addr, pendingRegions) defer worker.evictAllRegions() ctx, cancel := context.WithCancel(parentCtx) @@ -1062,7 +1062,7 @@ func (s *eventFeedSession) receiveFromStream( }) if err != nil { if status.Code(errors.Cause(err)) == codes.Canceled { - log.Debug( + log.Info( "receive from stream canceled", zap.String("namespace", s.changefeed.Namespace), zap.String("changefeed", s.changefeed.ID), diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index e49b5891fd9..ddd1c9046fa 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -112,6 +112,8 @@ type regionWorker struct { // how many pending input events inputPending int32 + + pendingRegions *syncRegionFeedStateMap } func newRegionWorkerMetrics(changefeedID model.ChangeFeedID) *regionWorkerMetrics { @@ -142,6 +144,7 @@ func newRegionWorkerMetrics(changefeedID model.ChangeFeedID) *regionWorkerMetric func newRegionWorker( ctx context.Context, changefeedID model.ChangeFeedID, s *eventFeedSession, addr string, + pendingRegions *syncRegionFeedStateMap, ) *regionWorker { return ®ionWorker{ parentCtx: ctx, @@ -156,6 +159,8 @@ func newRegionWorker( concurrency: s.client.config.KVClient.WorkerConcurrent, metrics: newRegionWorkerMetrics(changefeedID), inputPending: 0, + + pendingRegions: pendingRegions, } } @@ -191,7 +196,7 @@ func (w *regionWorker) checkShouldExit() error { empty := w.checkRegionStateEmpty() // If there is no region maintained by this region worker, exit it and // cancel the gRPC stream. - if empty { + if empty && w.pendingRegions.len() == 0 { w.cancelStream(time.Duration(0)) return cerror.ErrRegionWorkerExit.GenWithStackByArgs() } diff --git a/cdc/kv/region_worker_test.go b/cdc/kv/region_worker_test.go index 9438d518651..c8041246a96 100644 --- a/cdc/kv/region_worker_test.go +++ b/cdc/kv/region_worker_test.go @@ -159,7 +159,7 @@ func TestRegionWokerHandleEventEntryEventOutOfOrder(t *testing.T) { &tikv.RPCContext{}), 0) state.sri.lockedRange = ®ionlock.LockedRange{} state.start() - worker := newRegionWorker(ctx, model.ChangeFeedID{}, s, "") + worker := newRegionWorker(ctx, model.ChangeFeedID{}, s, "", newSyncRegionFeedStateMap()) require.Equal(t, 2, cap(worker.outputCh)) // Receive prewrite2 with empty value. @@ -322,7 +322,7 @@ func TestRegionWorkerHandleEventsBeforeStartTs(t *testing.T) { s1.sri.lockedRange = ®ionlock.LockedRange{} s1.sri.lockedRange.CheckpointTs.Store(9) s1.start() - w := newRegionWorker(ctx, model.ChangeFeedID{}, s, "") + w := newRegionWorker(ctx, model.ChangeFeedID{}, s, "", newSyncRegionFeedStateMap()) err := w.handleResolvedTs(ctx, &resolvedTsEvent{ resolvedTs: 5, From 32238ab715787a0f2d6b378b77630ebbf785f4f7 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 19:17:51 +0800 Subject: [PATCH 10/23] cloudstorage(ticdc): Remove nfs schema file check (#10138) (#10152) close pingcap/tiflow#10137 --- pkg/sink/cloudstorage/path.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/sink/cloudstorage/path.go b/pkg/sink/cloudstorage/path.go index 448a6217dfc..8b4a2550d51 100644 --- a/pkg/sink/cloudstorage/path.go +++ b/pkg/sink/cloudstorage/path.go @@ -220,7 +220,7 @@ func (f *FilePathGenerator) CheckOrWriteSchema( // Case 2: the table meta path is not empty. if schemaFileCnt != 0 { if lastVersion == 0 { - log.Panic("no table schema file found in an non-empty meta path", + log.Warn("no table schema file found in an non-empty meta path", zap.Any("versionedTableName", table), zap.Uint32("checksum", checksum)) } @@ -228,9 +228,9 @@ func (f *FilePathGenerator) CheckOrWriteSchema( return nil } - // Case 3: the table meta path is empty, which only happens when the table is - // existed before changefeed started. We need to write schema file to external - // storage. + // Case 3: the table meta path is empty, which happens when: + // a. the table is existed before changefeed started. We need to write schema file to external storage. + // b. the schema file is deleted by the consumer. We write schema file to external storage too. encodedDetail, err := def.MarshalWithQuery() if err != nil { return err From 06e345b5fe5eaf7220c127c882f435553b0ef6ad Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 21:26:48 +0800 Subject: [PATCH 11/23] config(ticdc): change case-sensitive default value (#10049) (#10076) close pingcap/tiflow#10047 --- cdc/api/v2/model_test.go | 2 +- cdc/model/changefeed_test.go | 2 +- pkg/config/replica_config.go | 2 +- tests/integration_tests/api_v2/cases.go | 2 +- tests/integration_tests/cli/run.sh | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index 2ed6dc97f84..c1bd97a816a 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -31,7 +31,7 @@ import ( // note: this is api published default value, not change it var defaultAPIConfig = &ReplicaConfig{ MemoryQuota: config.DefaultChangefeedMemoryQuota, - CaseSensitive: true, + CaseSensitive: false, EnableOldValue: true, CheckGCSafePoint: true, EnableSyncPoint: false, diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index 15eab536cf6..62756672cf2 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -146,7 +146,7 @@ func TestVerifyAndComplete(t *testing.T) { StartTs: 417257993615179777, Config: &config.ReplicaConfig{ MemoryQuota: 1073741824, - CaseSensitive: true, + CaseSensitive: false, EnableOldValue: true, CheckGCSafePoint: true, SyncPointInterval: time.Minute * 10, diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index c08b1bc6879..240891993f8 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -41,7 +41,7 @@ const ( var defaultReplicaConfig = &ReplicaConfig{ MemoryQuota: DefaultChangefeedMemoryQuota, - CaseSensitive: true, + CaseSensitive: false, EnableOldValue: true, CheckGCSafePoint: true, EnableSyncPoint: false, diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index 579fbd64d41..a68cac5ab90 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -109,7 +109,7 @@ var customReplicaConfig = &ReplicaConfig{ // defaultReplicaConfig check if the default values is changed var defaultReplicaConfig = &ReplicaConfig{ MemoryQuota: 1024 * 1024 * 1024, - CaseSensitive: true, + CaseSensitive: false, EnableOldValue: true, CheckGCSafePoint: true, EnableSyncPoint: false, diff --git a/tests/integration_tests/cli/run.sh b/tests/integration_tests/cli/run.sh index e1f8c95f3d7..8f0ffb37d28 100644 --- a/tests/integration_tests/cli/run.sh +++ b/tests/integration_tests/cli/run.sh @@ -71,7 +71,7 @@ function run() { # Update changefeed failed because changefeed is running cat - >"$WORK_DIR/changefeed.toml" <&1) - if [[ ! $changefeed_info == *"\"case_sensitive\":false"* ]]; then + if [[ ! $changefeed_info == *"\"case_sensitive\":true"* ]]; then echo "[$(date)] <<<<< changefeed info is not updated as expected ${changefeed_info} >>>>>" exit 1 fi From 1fde95e7947579577030e26307af2c00e33fb933 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 4 Dec 2023 22:47:49 +0800 Subject: [PATCH 12/23] puller(ticdc): detect resolved ts stuck in puller (#10182) (#10219) close pingcap/tiflow#10181 --- cdc/puller/puller.go | 37 ++++++++++++++++++++++++++++++++-- pkg/cmd/server/server_test.go | 16 +++++++++++++++ pkg/config/config_test_data.go | 6 +++++- pkg/config/debug.go | 11 ++++++++++ pkg/config/server_config.go | 4 ++++ 5 files changed, 71 insertions(+), 3 deletions(-) diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index 7853275a287..313daf06b82 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -72,6 +72,10 @@ type pullerImpl struct { changefeed model.ChangeFeedID tableID model.TableID tableName string + + cfg *config.ServerConfig + lastForwardTime time.Time + lastForwardResolvedTs uint64 } // New create a new Puller fetch event start from checkpointTs and put into buf. @@ -112,6 +116,7 @@ func New(ctx context.Context, changefeed: changefeed, tableID: tableID, tableName: tableName, + cfg: cfg, } return p } @@ -140,8 +145,8 @@ func (p *pullerImpl) Run(ctx context.Context) error { lastResolvedTs := p.checkpointTs g.Go(func() error { - metricsTicker := time.NewTicker(15 * time.Second) - defer metricsTicker.Stop() + stuckDetectorTicker := time.NewTicker(1 * time.Minute) + defer stuckDetectorTicker.Stop() output := func(raw *model.RawKVEntry) error { // even after https://github.com/pingcap/tiflow/pull/2038, kv client // could still miss region change notification, which leads to resolved @@ -178,6 +183,11 @@ func (p *pullerImpl) Run(ctx context.Context) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) + case <-stuckDetectorTicker.C: + if err := p.detectResolvedTsStuck(initialized); err != nil { + return errors.Trace(err) + } + continue case e = <-eventCh: } @@ -237,6 +247,29 @@ func (p *pullerImpl) Run(ctx context.Context) error { return g.Wait() } +func (p *pullerImpl) detectResolvedTsStuck(initialized bool) error { + if p.cfg.Debug.Puller.EnableResolvedTsStuckDetection && initialized { + resolvedTs := p.tsTracker.Frontier() + if resolvedTs == p.lastForwardResolvedTs { + log.Warn("ResolvedTs stuck detected in puller", + zap.String("namespace", p.changefeed.Namespace), + zap.String("changefeed", p.changefeed.ID), + zap.Int64("tableID", p.tableID), + zap.String("tableName", p.tableName), + zap.Uint64("lastResolvedTs", p.lastForwardResolvedTs), + zap.Uint64("resolvedTs", resolvedTs)) + if time.Since(p.lastForwardTime) > time.Duration(p.cfg.Debug.Puller.ResolvedTsStuckInterval) { + // throw an error to cause changefeed restart + return errors.New("resolved ts stuck") + } + } else { + p.lastForwardTime = time.Now() + p.lastForwardResolvedTs = resolvedTs + } + } + return nil +} + func (p *pullerImpl) Output() <-chan *model.RawKVEntry { return p.outputCh } diff --git a/pkg/cmd/server/server_test.go b/pkg/cmd/server/server_test.go index 843825a4344..6cfee19b5a9 100644 --- a/pkg/cmd/server/server_test.go +++ b/pkg/cmd/server/server_test.go @@ -201,6 +201,10 @@ func TestParseCfg(t *testing.T) { CheckBalanceInterval: 60000000000, AddTableBatchSize: 50, }, + Puller: &config.PullerConfig{ + EnableResolvedTsStuckDetection: false, + ResolvedTsStuckInterval: config.TomlDuration(5 * time.Minute), + }, }, ClusterID: "default", MaxMemoryPercentage: config.DisableMemoryLimit, @@ -348,6 +352,10 @@ check-balance-interval = "10s" CheckBalanceInterval: config.TomlDuration(10 * time.Second), AddTableBatchSize: 50, }, + Puller: &config.PullerConfig{ + EnableResolvedTsStuckDetection: false, + ResolvedTsStuckInterval: config.TomlDuration(5 * time.Minute), + }, }, ClusterID: "default", MaxMemoryPercentage: config.DisableMemoryLimit, @@ -483,6 +491,10 @@ cert-allowed-cn = ["dd","ee"] CheckBalanceInterval: 60000000000, AddTableBatchSize: 50, }, + Puller: &config.PullerConfig{ + EnableResolvedTsStuckDetection: false, + ResolvedTsStuckInterval: config.TomlDuration(5 * time.Minute), + }, }, ClusterID: "default", MaxMemoryPercentage: config.DisableMemoryLimit, @@ -546,5 +558,9 @@ unknown3 = 3 CheckBalanceInterval: 60000000000, AddTableBatchSize: 50, }, + Puller: &config.PullerConfig{ + EnableResolvedTsStuckDetection: false, + ResolvedTsStuckInterval: config.TomlDuration(5 * time.Minute), + }, }, o.serverConfig.Debug) } diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 5a44aac5aac..8e13ef911c1 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -149,7 +149,11 @@ const ( "check-balance-interval": 60000000000, "add-table-batch-size": 50 }, - "enable-kv-connect-backoff": false + "enable-kv-connect-backoff": false, + "puller": { + "enable-resolved-ts-stuck-detection": false, + "resolved-ts-stuck-interval": 300000000000 + } }, "cluster-id": "default", "max-memory-percentage": 0, diff --git a/pkg/config/debug.go b/pkg/config/debug.go index 3a7815090bc..59528b886a1 100644 --- a/pkg/config/debug.go +++ b/pkg/config/debug.go @@ -28,6 +28,9 @@ type DebugConfig struct { // EnableKVConnectBackOff enables the backoff for kv connect. EnableKVConnectBackOff bool `toml:"enable-kv-connect-backoff" json:"enable-kv-connect-backoff"` + + // Puller is the configuration of the puller. + Puller *PullerConfig `toml:"puller" json:"puller"` } // ValidateAndAdjust validates and adjusts the debug configuration @@ -44,3 +47,11 @@ func (c *DebugConfig) ValidateAndAdjust() error { return nil } + +// PullerConfig represents config for puller +type PullerConfig struct { + // EnableResolvedTsStuckDetection is used to enable resolved ts stuck detection. + EnableResolvedTsStuckDetection bool `toml:"enable-resolved-ts-stuck-detection" json:"enable-resolved-ts-stuck-detection"` + // ResolvedTsStuckInterval is the interval of checking resolved ts stuck. + ResolvedTsStuckInterval TomlDuration `toml:"resolved-ts-stuck-interval" json:"resolved-ts-stuck-interval"` +} diff --git a/pkg/config/server_config.go b/pkg/config/server_config.go index 5df56c4a1f1..27f40104544 100644 --- a/pkg/config/server_config.go +++ b/pkg/config/server_config.go @@ -140,6 +140,10 @@ var defaultServerConfig = &ServerConfig{ Scheduler: NewDefaultSchedulerConfig(), EnableKVConnectBackOff: false, + Puller: &PullerConfig{ + EnableResolvedTsStuckDetection: false, + ResolvedTsStuckInterval: TomlDuration(5 * time.Minute), + }, }, ClusterID: "default", GcTunerMemoryThreshold: DisableMemoryLimit, From fa92ec159e3b8a518add17eb9a7120cbeddfc7f2 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 00:34:19 +0800 Subject: [PATCH 13/23] redo(ticdc): add meta flush interval configuration (#9959) (#9972) close pingcap/tiflow#9960 --- cdc/api/v2/model.go | 33 ++++++++++++++----------- cdc/api/v2/model_test.go | 11 +++++---- cdc/owner/changefeed_test.go | 7 +++--- cdc/processor/processor_test.go | 11 +++++---- cdc/redo/meta_manager.go | 22 +++++++++++------ cdc/redo/meta_manager_test.go | 27 +++++++++++--------- pkg/config/config_test_data.go | 3 +++ pkg/config/consistent.go | 20 +++++++++++---- pkg/config/replica_config.go | 11 +++++---- pkg/redo/config.go | 2 ++ tests/integration_tests/api_v2/cases.go | 22 +++++++++-------- tests/integration_tests/api_v2/model.go | 11 +++++---- 12 files changed, 108 insertions(+), 72 deletions(-) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 3206f2f1899..989270e07fd 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -260,11 +260,12 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( } if c.Consistent != nil { res.Consistent = &config.ConsistentConfig{ - Level: c.Consistent.Level, - MaxLogSize: c.Consistent.MaxLogSize, - FlushIntervalInMs: c.Consistent.FlushIntervalInMs, - Storage: c.Consistent.Storage, - UseFileBackend: c.Consistent.UseFileBackend, + Level: c.Consistent.Level, + MaxLogSize: c.Consistent.MaxLogSize, + FlushIntervalInMs: c.Consistent.FlushIntervalInMs, + MetaFlushIntervalInMs: c.Consistent.MetaFlushIntervalInMs, + Storage: c.Consistent.Storage, + UseFileBackend: c.Consistent.UseFileBackend, } } if c.Sink != nil { @@ -627,11 +628,12 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { } if cloned.Consistent != nil { res.Consistent = &ConsistentConfig{ - Level: cloned.Consistent.Level, - MaxLogSize: cloned.Consistent.MaxLogSize, - FlushIntervalInMs: cloned.Consistent.FlushIntervalInMs, - Storage: cloned.Consistent.Storage, - UseFileBackend: cloned.Consistent.UseFileBackend, + Level: cloned.Consistent.Level, + MaxLogSize: cloned.Consistent.MaxLogSize, + FlushIntervalInMs: cloned.Consistent.FlushIntervalInMs, + MetaFlushIntervalInMs: cloned.Consistent.MetaFlushIntervalInMs, + Storage: cloned.Consistent.Storage, + UseFileBackend: cloned.Consistent.UseFileBackend, } } if cloned.Mounter != nil { @@ -807,11 +809,12 @@ type ColumnSelector struct { // ConsistentConfig represents replication consistency config for a changefeed // This is a duplicate of config.ConsistentConfig type ConsistentConfig struct { - Level string `json:"level"` - MaxLogSize int64 `json:"max_log_size"` - FlushIntervalInMs int64 `json:"flush_interval"` - Storage string `json:"storage"` - UseFileBackend bool `json:"use_file_backend"` + Level string `json:"level,omitempty"` + MaxLogSize int64 `json:"max_log_size"` + FlushIntervalInMs int64 `json:"flush_interval"` + MetaFlushIntervalInMs int64 `json:"meta_flush_interval"` + Storage string `json:"storage,omitempty"` + UseFileBackend bool `json:"use_file_backend"` } // ChangefeedSchedulerConfig is per changefeed scheduler settings. diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index c1bd97a816a..9b7f6ff9339 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -58,11 +58,12 @@ var defaultAPIConfig = &ReplicaConfig{ AdvanceTimeoutInSec: util.AddressOf(uint(150)), }, Consistent: &ConsistentConfig{ - Level: "none", - MaxLogSize: 64, - FlushIntervalInMs: redo.DefaultFlushIntervalInMs, - Storage: "", - UseFileBackend: false, + Level: "none", + MaxLogSize: 64, + FlushIntervalInMs: redo.DefaultFlushIntervalInMs, + MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + Storage: "", + UseFileBackend: false, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: config.GetDefaultReplicaConfig(). diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 81fb4716b2b..075a681fbb5 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -514,9 +514,10 @@ func TestRemoveChangefeed(t *testing.T) { info := ctx.ChangefeedVars().Info dir := t.TempDir() info.Config.Consistent = &config.ConsistentConfig{ - Level: "eventual", - Storage: filepath.Join("nfs://", dir), - FlushIntervalInMs: redo.DefaultFlushIntervalInMs, + Level: "eventual", + Storage: filepath.Join("nfs://", dir), + FlushIntervalInMs: redo.DefaultFlushIntervalInMs, + MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, } ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ ID: ctx.ChangefeedVars().ID, diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index 8db8104b22c..3a8a53c8ae2 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -69,11 +69,12 @@ func newProcessor4Test( tmpDir := t.TempDir() redoDir := fmt.Sprintf("%s/%s", tmpDir, changefeedID) dmlMgr := redo.NewDMLManager(changefeedID, &config.ConsistentConfig{ - Level: string(redoPkg.ConsistentLevelEventual), - MaxLogSize: redoPkg.DefaultMaxLogSize, - FlushIntervalInMs: redoPkg.DefaultFlushIntervalInMs, - Storage: "file://" + redoDir, - UseFileBackend: false, + Level: string(redoPkg.ConsistentLevelEventual), + MaxLogSize: redoPkg.DefaultMaxLogSize, + FlushIntervalInMs: redoPkg.DefaultFlushIntervalInMs, + MetaFlushIntervalInMs: redoPkg.DefaultMetaFlushIntervalInMs, + Storage: "file://" + redoDir, + UseFileBackend: false, }) p.redo.r = dmlMgr } diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index a19086b9bd0..164f04385a1 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -72,6 +72,7 @@ type metaManager struct { startTs model.Ts lastFlushTime time.Time + flushIntervalInMs int64 cfg *config.ConsistentConfig metricFlushLogDuration prometheus.Observer } @@ -94,12 +95,19 @@ func NewMetaManager( } m := &metaManager{ - captureID: config.GetGlobalServerConfig().AdvertiseAddr, - changeFeedID: changefeedID, - uuidGenerator: uuid.NewGenerator(), - enabled: true, - cfg: cfg, - startTs: checkpoint, + captureID: config.GetGlobalServerConfig().AdvertiseAddr, + changeFeedID: changefeedID, + uuidGenerator: uuid.NewGenerator(), + enabled: true, + cfg: cfg, + startTs: checkpoint, + flushIntervalInMs: cfg.MetaFlushIntervalInMs, + } + + if m.flushIntervalInMs < redo.MinFlushIntervalInMs { + log.Warn("redo flush interval is too small, use default value", + zap.Int64("interval", m.flushIntervalInMs)) + m.flushIntervalInMs = redo.DefaultMetaFlushIntervalInMs } return m @@ -158,7 +166,7 @@ func (m *metaManager) Run(ctx context.Context, _ ...chan<- error) error { } eg, egCtx := errgroup.WithContext(ctx) eg.Go(func() error { - return m.bgFlushMeta(egCtx, m.cfg.FlushIntervalInMs) + return m.bgFlushMeta(egCtx, m.flushIntervalInMs) }) eg.Go(func() error { return m.bgGC(egCtx) diff --git a/cdc/redo/meta_manager_test.go b/cdc/redo/meta_manager_test.go index 478d50fdd36..4bb7ae5ec32 100644 --- a/cdc/redo/meta_manager_test.go +++ b/cdc/redo/meta_manager_test.go @@ -70,10 +70,11 @@ func TestInitAndWriteMeta(t *testing.T) { startTs := uint64(10) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: uri.String(), - FlushIntervalInMs: redo.MinFlushIntervalInMs, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: uri.String(), + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, } m := NewMetaManager(changefeedID, cfg, startTs) @@ -146,10 +147,11 @@ func TestPreCleanupAndWriteMeta(t *testing.T) { startTs := uint64(10) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: uri.String(), - FlushIntervalInMs: redo.MinFlushIntervalInMs, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: uri.String(), + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, } m := NewMetaManager(changefeedID, cfg, startTs) @@ -274,10 +276,11 @@ func TestGCAndCleanup(t *testing.T) { startTs := uint64(3) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: uri.String(), - FlushIntervalInMs: redo.MinFlushIntervalInMs, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: uri.String(), + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, } m := NewMetaManager(changefeedID, cfg, startTs) diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 8e13ef911c1..3908b281818 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -60,6 +60,7 @@ const ( "level": "none", "max-log-size": 64, "flush-interval": 2000, + "meta-flush-interval": 200, "storage": "", "use-file-backend": false }, @@ -277,6 +278,7 @@ const ( "level": "none", "max-log-size": 64, "flush-interval": 2000, + "meta-flush-interval": 200, "storage": "", "use-file-backend": false }, @@ -410,6 +412,7 @@ const ( "level": "none", "max-log-size": 64, "flush-interval": 2000, + "meta-flush-interval": 200, "storage": "", "use-file-backend": false }, diff --git a/pkg/config/consistent.go b/pkg/config/consistent.go index 636edcf865f..4a521f98da5 100644 --- a/pkg/config/consistent.go +++ b/pkg/config/consistent.go @@ -24,11 +24,12 @@ import ( // ConsistentConfig represents replication consistency config for a changefeed. type ConsistentConfig struct { - Level string `toml:"level" json:"level"` - MaxLogSize int64 `toml:"max-log-size" json:"max-log-size"` - FlushIntervalInMs int64 `toml:"flush-interval" json:"flush-interval"` - Storage string `toml:"storage" json:"storage"` - UseFileBackend bool `toml:"use-file-backend" json:"use-file-backend"` + Level string `toml:"level" json:"level"` + MaxLogSize int64 `toml:"max-log-size" json:"max-log-size"` + FlushIntervalInMs int64 `toml:"flush-interval" json:"flush-interval"` + MetaFlushIntervalInMs int64 `toml:"meta-flush-interval" json:"meta-flush-interval"` + Storage string `toml:"storage" json:"storage"` + UseFileBackend bool `toml:"use-file-backend" json:"use-file-backend"` } // ValidateAndAdjust validates the consistency config and adjusts it if necessary. @@ -50,6 +51,15 @@ func (c *ConsistentConfig) ValidateAndAdjust() error { c.FlushIntervalInMs, redo.MinFlushIntervalInMs)) } + if c.MetaFlushIntervalInMs == 0 { + c.MetaFlushIntervalInMs = redo.DefaultMetaFlushIntervalInMs + } + if c.MetaFlushIntervalInMs < redo.MinFlushIntervalInMs { + return cerror.ErrInvalidReplicaConfig.FastGenByArgs( + fmt.Sprintf("The consistent.meta-flush-interval:%d must be equal or greater than %d", + c.MetaFlushIntervalInMs, redo.MinFlushIntervalInMs)) + } + uri, err := storage.ParseRawURL(c.Storage) if err != nil { return cerror.ErrInvalidReplicaConfig.GenWithStackByArgs( diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 240891993f8..c89845941c4 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -69,11 +69,12 @@ var defaultReplicaConfig = &ReplicaConfig{ AdvanceTimeoutInSec: util.AddressOf(DefaultAdvanceTimeoutInSec), }, Consistent: &ConsistentConfig{ - Level: "none", - MaxLogSize: redo.DefaultMaxLogSize, - FlushIntervalInMs: redo.DefaultFlushIntervalInMs, - Storage: "", - UseFileBackend: false, + Level: "none", + MaxLogSize: redo.DefaultMaxLogSize, + FlushIntervalInMs: redo.DefaultFlushIntervalInMs, + MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + Storage: "", + UseFileBackend: false, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, diff --git a/pkg/redo/config.go b/pkg/redo/config.go index 82c2e6f3457..d6b66de45ed 100644 --- a/pkg/redo/config.go +++ b/pkg/redo/config.go @@ -44,6 +44,8 @@ const ( FlushWarnDuration = time.Second * 20 // DefaultFlushIntervalInMs is the default flush interval for redo log. DefaultFlushIntervalInMs = 2000 + // DefaultMetaFlushIntervalInMs is the default flush interval for redo meta. + DefaultMetaFlushIntervalInMs = 200 // MinFlushIntervalInMs is the minimum flush interval for redo log. MinFlushIntervalInMs = 50 diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index a68cac5ab90..a1ee4082301 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -90,11 +90,12 @@ var customReplicaConfig = &ReplicaConfig{ EnablePartitionSeparator: true, }, Consistent: &ConsistentConfig{ - Level: "", - MaxLogSize: 65, - FlushIntervalInMs: 500, - Storage: "local://test", - UseFileBackend: true, + Level: "", + MaxLogSize: 65, + MetaFlushIntervalInMs: 201, + FlushIntervalInMs: 500, + Storage: "local://test", + UseFileBackend: true, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, @@ -133,11 +134,12 @@ var defaultReplicaConfig = &ReplicaConfig{ EnablePartitionSeparator: true, }, Consistent: &ConsistentConfig{ - Level: "none", - MaxLogSize: redo.DefaultMaxLogSize, - FlushIntervalInMs: redo.DefaultFlushIntervalInMs, - Storage: "", - UseFileBackend: false, + Level: "none", + MaxLogSize: redo.DefaultMaxLogSize, + MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + FlushIntervalInMs: redo.DefaultFlushIntervalInMs, + Storage: "", + UseFileBackend: false, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, diff --git a/tests/integration_tests/api_v2/model.go b/tests/integration_tests/api_v2/model.go index 3d7ac7c28f3..2e450c806c6 100644 --- a/tests/integration_tests/api_v2/model.go +++ b/tests/integration_tests/api_v2/model.go @@ -273,11 +273,12 @@ type ColumnSelector struct { // ConsistentConfig represents replication consistency config for a changefeed // This is a duplicate of config.ConsistentConfig type ConsistentConfig struct { - Level string `json:"level"` - MaxLogSize int64 `json:"max_log_size"` - FlushIntervalInMs int64 `json:"flush_interval"` - Storage string `json:"storage"` - UseFileBackend bool `json:"use_file_backend"` + Level string `json:"level"` + MaxLogSize int64 `json:"max_log_size"` + FlushIntervalInMs int64 `json:"flush_interval"` + MetaFlushIntervalInMs int64 `json:"meta_flush_interval"` + Storage string `json:"storage"` + UseFileBackend bool `json:"use_file_backend"` } // ChangefeedSchedulerConfig is per changefeed scheduler settings. From 7b988dd288ad4bf9ae07c4cf7a5664e943db9db5 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 08:25:18 +0800 Subject: [PATCH 14/23] redo(ticdc): fix meta manager wrong flush interval bug (#10031) (#10035) close pingcap/tiflow#10026 --- cdc/redo/manager.go | 1 + cdc/redo/meta_manager.go | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 5d76886d5fc..fd777b27f5e 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -427,6 +427,7 @@ func (m *logManager) flushLog( log.Debug("Flush redo log", zap.String("namespace", m.cfg.ChangeFeedID.Namespace), zap.String("changefeed", m.cfg.ChangeFeedID.ID), + zap.String("logType", m.cfg.LogType), zap.Any("tableRtsMap", tableRtsMap)) err := m.withLock(func(m *logManager) error { return m.writer.FlushLog(ctx) diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index 164f04385a1..bfa4dce69e0 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -166,7 +166,7 @@ func (m *metaManager) Run(ctx context.Context, _ ...chan<- error) error { } eg, egCtx := errgroup.WithContext(ctx) eg.Go(func() error { - return m.bgFlushMeta(egCtx, m.flushIntervalInMs) + return m.bgFlushMeta(egCtx) }) eg.Go(func() error { return m.bgGC(egCtx) @@ -466,8 +466,8 @@ func (m *metaManager) Cleanup(ctx context.Context) error { return m.deleteAllLogs(ctx) } -func (m *metaManager) bgFlushMeta(egCtx context.Context, flushIntervalInMs int64) (err error) { - ticker := time.NewTicker(time.Duration(flushIntervalInMs) * time.Millisecond) +func (m *metaManager) bgFlushMeta(egCtx context.Context) (err error) { + ticker := time.NewTicker(time.Duration(m.flushIntervalInMs) * time.Millisecond) defer func() { ticker.Stop() log.Info("redo metaManager bgFlushMeta exits", From 26be6f30b3ef4150885a6daa9ffc037aed9891f0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 09:31:18 +0800 Subject: [PATCH 15/23] redo(ticdc): use meta flush interval in redo ddl manager (#9999) (#10007) ref pingcap/tiflow#9960, close pingcap/tiflow#9998 --- cdc/redo/manager.go | 17 +++++++++++----- cdc/redo/manager_test.go | 42 ++++++++++++++++++++++------------------ 2 files changed, 35 insertions(+), 24 deletions(-) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index fd777b27f5e..872f591214f 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -274,7 +274,15 @@ func (m *logManager) Run(ctx context.Context, _ ...chan<- error) error { return err } m.writer = w - return m.bgUpdateLog(ctx) + return m.bgUpdateLog(ctx, m.getFlushDuration()) +} + +func (m *logManager) getFlushDuration() time.Duration { + flushIntervalInMs := m.cfg.FlushIntervalInMs + if m.cfg.LogType == redo.RedoDDLLogFileType { + flushIntervalInMs = m.cfg.MetaFlushIntervalInMs + } + return time.Duration(flushIntervalInMs) * time.Millisecond } // WaitForReady implements pkg/util.Runnable. @@ -484,15 +492,14 @@ func (m *logManager) onResolvedTsMsg(span tablepb.Span, resolvedTs model.Ts) { } } -func (m *logManager) bgUpdateLog(ctx context.Context) error { +func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duration) error { m.releaseMemoryCbs = make([]func(), 0, 1024) - flushIntervalInMs := m.cfg.FlushIntervalInMs - ticker := time.NewTicker(time.Duration(flushIntervalInMs) * time.Millisecond) + ticker := time.NewTicker(flushDuration) defer ticker.Stop() log.Info("redo manager bgUpdateLog is running", zap.String("namespace", m.cfg.ChangeFeedID.Namespace), zap.String("changefeed", m.cfg.ChangeFeedID.ID), - zap.Int64("flushIntervalInMs", flushIntervalInMs)) + zap.Duration("flushIntervalInMs", flushDuration)) var err error // logErrCh is used to retrieve errors from log flushing goroutines. diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index 14db8ffc251..a07eb39fee5 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -114,11 +114,12 @@ func TestLogManagerInProcessor(t *testing.T) { testWriteDMLs := func(storage string, useFileBackend bool) { ctx, cancel := context.WithCancel(ctx) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: storage, - FlushIntervalInMs: redo.MinFlushIntervalInMs, - UseFileBackend: useFileBackend, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: storage, + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + UseFileBackend: useFileBackend, } dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) var eg errgroup.Group @@ -221,11 +222,12 @@ func TestLogManagerInOwner(t *testing.T) { testWriteDDLs := func(storage string, useFileBackend bool) { ctx, cancel := context.WithCancel(ctx) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: storage, - FlushIntervalInMs: redo.MinFlushIntervalInMs, - UseFileBackend: useFileBackend, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: storage, + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + UseFileBackend: useFileBackend, } startTs := model.Ts(10) ddlMgr := NewDDLManager(model.DefaultChangeFeedID("test"), cfg, startTs) @@ -266,10 +268,11 @@ func TestLogManagerError(t *testing.T) { defer cancel() cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: "blackhole-invalid://", - FlushIntervalInMs: redo.MinFlushIntervalInMs, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: "blackhole-invalid://", + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, } logMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) var eg errgroup.Group @@ -317,11 +320,12 @@ func BenchmarkFileWriter(b *testing.B) { func runBenchTest(b *testing.B, storage string, useFileBackend bool) { ctx, cancel := context.WithCancel(context.Background()) cfg := &config.ConsistentConfig{ - Level: string(redo.ConsistentLevelEventual), - MaxLogSize: redo.DefaultMaxLogSize, - Storage: storage, - FlushIntervalInMs: redo.MinFlushIntervalInMs, - UseFileBackend: useFileBackend, + Level: string(redo.ConsistentLevelEventual), + MaxLogSize: redo.DefaultMaxLogSize, + Storage: storage, + FlushIntervalInMs: redo.MinFlushIntervalInMs, + MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + UseFileBackend: useFileBackend, } dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) var eg errgroup.Group From e876979526ec46074378c9681264f306e99cd100 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 12:59:49 +0800 Subject: [PATCH 16/23] redo(ticdc): add writer related parameter (#10025) (#10057) close pingcap/tiflow#10048 --- cdc/api/v2/model.go | 6 ++++++ cdc/api/v2/model_test.go | 2 ++ cdc/owner/changefeed_test.go | 2 ++ cdc/processor/processor_test.go | 2 ++ cdc/redo/manager.go | 5 ++++- cdc/redo/manager_test.go | 8 ++++++++ cdc/redo/meta_manager_test.go | 6 ++++++ cdc/redo/writer/memory/encoding_worker.go | 7 ++++--- cdc/redo/writer/memory/file_worker.go | 2 +- cdc/redo/writer/memory/main_test.go | 5 ----- cdc/redo/writer/memory/mem_log_writer.go | 13 ++----------- pkg/config/config_test_data.go | 6 ++++++ pkg/config/consistent.go | 9 +++++++++ pkg/config/replica_config.go | 2 ++ pkg/redo/config.go | 11 +++++++++++ tests/integration_tests/api_v2/cases.go | 4 ++++ tests/integration_tests/api_v2/model.go | 2 ++ 17 files changed, 71 insertions(+), 21 deletions(-) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 989270e07fd..f79772ac5f5 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -264,6 +264,8 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( MaxLogSize: c.Consistent.MaxLogSize, FlushIntervalInMs: c.Consistent.FlushIntervalInMs, MetaFlushIntervalInMs: c.Consistent.MetaFlushIntervalInMs, + EncodingWorkerNum: c.Consistent.EncodingWorkerNum, + FlushWorkerNum: c.Consistent.FlushWorkerNum, Storage: c.Consistent.Storage, UseFileBackend: c.Consistent.UseFileBackend, } @@ -632,6 +634,8 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { MaxLogSize: cloned.Consistent.MaxLogSize, FlushIntervalInMs: cloned.Consistent.FlushIntervalInMs, MetaFlushIntervalInMs: cloned.Consistent.MetaFlushIntervalInMs, + EncodingWorkerNum: c.Consistent.EncodingWorkerNum, + FlushWorkerNum: c.Consistent.FlushWorkerNum, Storage: cloned.Consistent.Storage, UseFileBackend: cloned.Consistent.UseFileBackend, } @@ -813,6 +817,8 @@ type ConsistentConfig struct { MaxLogSize int64 `json:"max_log_size"` FlushIntervalInMs int64 `json:"flush_interval"` MetaFlushIntervalInMs int64 `json:"meta_flush_interval"` + EncodingWorkerNum int `json:"encoding_worker_num"` + FlushWorkerNum int `json:"flush_worker_num"` Storage string `json:"storage,omitempty"` UseFileBackend bool `json:"use_file_backend"` } diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index 9b7f6ff9339..2430d7a04e1 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -62,6 +62,8 @@ var defaultAPIConfig = &ReplicaConfig{ MaxLogSize: 64, FlushIntervalInMs: redo.DefaultFlushIntervalInMs, MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, Storage: "", UseFileBackend: false, }, diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 075a681fbb5..5d125418437 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -518,6 +518,8 @@ func TestRemoveChangefeed(t *testing.T) { Storage: filepath.Join("nfs://", dir), FlushIntervalInMs: redo.DefaultFlushIntervalInMs, MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, } ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ ID: ctx.ChangefeedVars().ID, diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index 3a8a53c8ae2..370b8a6b6d1 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -73,6 +73,8 @@ func newProcessor4Test( MaxLogSize: redoPkg.DefaultMaxLogSize, FlushIntervalInMs: redoPkg.DefaultFlushIntervalInMs, MetaFlushIntervalInMs: redoPkg.DefaultMetaFlushIntervalInMs, + EncodingWorkerNum: redoPkg.DefaultEncodingWorkerNum, + FlushWorkerNum: redoPkg.DefaultFlushWorkerNum, Storage: "file://" + redoDir, UseFileBackend: false, }) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 872f591214f..6f3b0804185 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -499,7 +499,10 @@ func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duratio log.Info("redo manager bgUpdateLog is running", zap.String("namespace", m.cfg.ChangeFeedID.Namespace), zap.String("changefeed", m.cfg.ChangeFeedID.ID), - zap.Duration("flushIntervalInMs", flushDuration)) + zap.Duration("flushIntervalInMs", flushDuration), + zap.Int64("maxLogSize", m.cfg.MaxLogSize), + zap.Int("encoderWorkerNum", m.cfg.EncodingWorkerNum), + zap.Int("flushWorkerNum", m.cfg.FlushWorkerNum)) var err error // logErrCh is used to retrieve errors from log flushing goroutines. diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index a07eb39fee5..381f4e99743 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -119,6 +119,8 @@ func TestLogManagerInProcessor(t *testing.T) { Storage: storage, FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, UseFileBackend: useFileBackend, } dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) @@ -227,6 +229,8 @@ func TestLogManagerInOwner(t *testing.T) { Storage: storage, FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, UseFileBackend: useFileBackend, } startTs := model.Ts(10) @@ -273,6 +277,8 @@ func TestLogManagerError(t *testing.T) { Storage: "blackhole-invalid://", FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, } logMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) var eg errgroup.Group @@ -325,6 +331,8 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) { Storage: storage, FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, UseFileBackend: useFileBackend, } dmlMgr := NewDMLManager(model.DefaultChangeFeedID("test"), cfg) diff --git a/cdc/redo/meta_manager_test.go b/cdc/redo/meta_manager_test.go index 4bb7ae5ec32..24d2e404315 100644 --- a/cdc/redo/meta_manager_test.go +++ b/cdc/redo/meta_manager_test.go @@ -75,6 +75,8 @@ func TestInitAndWriteMeta(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, } m := NewMetaManager(changefeedID, cfg, startTs) @@ -152,6 +154,8 @@ func TestPreCleanupAndWriteMeta(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, } m := NewMetaManager(changefeedID, cfg, startTs) @@ -281,6 +285,8 @@ func TestGCAndCleanup(t *testing.T) { Storage: uri.String(), FlushIntervalInMs: redo.MinFlushIntervalInMs, MetaFlushIntervalInMs: redo.MinFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, } m := NewMetaManager(changefeedID, cfg, startTs) diff --git a/cdc/redo/writer/memory/encoding_worker.go b/cdc/redo/writer/memory/encoding_worker.go index 57b15ce45f3..bd878f65b2b 100644 --- a/cdc/redo/writer/memory/encoding_worker.go +++ b/cdc/redo/writer/memory/encoding_worker.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/model/codec" "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/redo" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -107,15 +108,15 @@ type encodingWorkerGroup struct { func newEncodingWorkerGroup(workerNum int) *encodingWorkerGroup { if workerNum <= 0 { - workerNum = defaultEncodingWorkerNum + workerNum = redo.DefaultEncodingWorkerNum } inputChs := make([]chan *polymorphicRedoEvent, workerNum) for i := 0; i < workerNum; i++ { - inputChs[i] = make(chan *polymorphicRedoEvent, defaultEncodingInputChanSize) + inputChs[i] = make(chan *polymorphicRedoEvent, redo.DefaultEncodingInputChanSize) } return &encodingWorkerGroup{ inputChs: inputChs, - outputCh: make(chan *polymorphicRedoEvent, defaultEncodingOutputChanSize), + outputCh: make(chan *polymorphicRedoEvent, redo.DefaultEncodingOutputChanSize), workerNum: workerNum, closed: make(chan struct{}), } diff --git a/cdc/redo/writer/memory/file_worker.go b/cdc/redo/writer/memory/file_worker.go index 45b366117c0..3258bf169c4 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -103,7 +103,7 @@ func newFileWorkerGroup( opts ...writer.Option, ) *fileWorkerGroup { if workerNum <= 0 { - workerNum = defaultFlushWorkerNum + workerNum = redo.DefaultFlushWorkerNum } op := &writer.LogWriterOptions{} diff --git a/cdc/redo/writer/memory/main_test.go b/cdc/redo/writer/memory/main_test.go index 6e92d8f867d..2f68850c54e 100644 --- a/cdc/redo/writer/memory/main_test.go +++ b/cdc/redo/writer/memory/main_test.go @@ -20,10 +20,5 @@ import ( ) func TestMain(m *testing.M) { - bak := defaultEncodingInputChanSize - defer func() { - defaultEncodingInputChanSize = bak - }() - defaultEncodingInputChanSize = 0 leakutil.SetUpLeakTest(m) } diff --git a/cdc/redo/writer/memory/mem_log_writer.go b/cdc/redo/writer/memory/mem_log_writer.go index 0460fdbf08b..25e968c513f 100644 --- a/cdc/redo/writer/memory/mem_log_writer.go +++ b/cdc/redo/writer/memory/mem_log_writer.go @@ -24,15 +24,6 @@ import ( "golang.org/x/sync/errgroup" ) -var ( - defaultEncodingWorkerNum = 16 - defaultEncodingInputChanSize = 128 - defaultEncodingOutputChanSize = 2048 - // Maximum allocated memory is flushWorkerNum*maxLogSize, which is - // `8*64MB = 512MB` by default. - defaultFlushWorkerNum = 8 -) - var _ writer.RedoLogWriter = (*memoryLogWriter)(nil) type memoryLogWriter struct { @@ -71,11 +62,11 @@ func NewLogWriter( cancel: lwCancel, } - lw.encodeWorkers = newEncodingWorkerGroup(defaultEncodingWorkerNum) + lw.encodeWorkers = newEncodingWorkerGroup(cfg.EncodingWorkerNum) eg.Go(func() error { return lw.encodeWorkers.Run(lwCtx) }) - lw.fileWorkers = newFileWorkerGroup(cfg, defaultFlushWorkerNum, extStorage, opts...) + lw.fileWorkers = newFileWorkerGroup(cfg, cfg.FlushWorkerNum, extStorage, opts...) eg.Go(func() error { return lw.fileWorkers.Run(lwCtx, lw.encodeWorkers.outputCh) }) diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 3908b281818..f3471ddd357 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -61,6 +61,8 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "encoding-worker-num": 16, + "flush-worker-num": 8, "storage": "", "use-file-backend": false }, @@ -279,6 +281,8 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "encoding-worker-num": 16, + "flush-worker-num": 8, "storage": "", "use-file-backend": false }, @@ -413,6 +417,8 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "encoding-worker-num": 16, + "flush-worker-num": 8, "storage": "", "use-file-backend": false }, diff --git a/pkg/config/consistent.go b/pkg/config/consistent.go index 4a521f98da5..14021e2074a 100644 --- a/pkg/config/consistent.go +++ b/pkg/config/consistent.go @@ -28,6 +28,8 @@ type ConsistentConfig struct { MaxLogSize int64 `toml:"max-log-size" json:"max-log-size"` FlushIntervalInMs int64 `toml:"flush-interval" json:"flush-interval"` MetaFlushIntervalInMs int64 `toml:"meta-flush-interval" json:"meta-flush-interval"` + EncodingWorkerNum int `toml:"encoding-worker-num" json:"encoding-worker-num"` + FlushWorkerNum int `toml:"flush-worker-num" json:"flush-worker-num"` Storage string `toml:"storage" json:"storage"` UseFileBackend bool `toml:"use-file-backend" json:"use-file-backend"` } @@ -60,6 +62,13 @@ func (c *ConsistentConfig) ValidateAndAdjust() error { c.MetaFlushIntervalInMs, redo.MinFlushIntervalInMs)) } + if c.EncodingWorkerNum == 0 { + c.EncodingWorkerNum = redo.DefaultEncodingWorkerNum + } + if c.FlushWorkerNum == 0 { + c.FlushWorkerNum = redo.DefaultFlushWorkerNum + } + uri, err := storage.ParseRawURL(c.Storage) if err != nil { return cerror.ErrInvalidReplicaConfig.GenWithStackByArgs( diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index c89845941c4..c4fc92c13cd 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -73,6 +73,8 @@ var defaultReplicaConfig = &ReplicaConfig{ MaxLogSize: redo.DefaultMaxLogSize, FlushIntervalInMs: redo.DefaultFlushIntervalInMs, MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, + EncodingWorkerNum: redo.DefaultEncodingWorkerNum, + FlushWorkerNum: redo.DefaultFlushWorkerNum, Storage: "", UseFileBackend: false, }, diff --git a/pkg/redo/config.go b/pkg/redo/config.go index d6b66de45ed..eb16f754ff0 100644 --- a/pkg/redo/config.go +++ b/pkg/redo/config.go @@ -49,6 +49,17 @@ const ( // MinFlushIntervalInMs is the minimum flush interval for redo log. MinFlushIntervalInMs = 50 + // DefaultEncodingWorkerNum is the default number of encoding workers. + DefaultEncodingWorkerNum = 16 + // DefaultEncodingInputChanSize is the default size of input channel for encoding worker. + DefaultEncodingInputChanSize = 128 + // DefaultEncodingOutputChanSize is the default size of output channel for encoding worker. + DefaultEncodingOutputChanSize = 2048 + // DefaultFlushWorkerNum is the default number of flush workers. + // Maximum allocated memory is flushWorkerNum*maxLogSize, which is + // `8*64MB = 512MB` by default. + DefaultFlushWorkerNum = 8 + // DefaultFileMode is the default mode when operation files DefaultFileMode = 0o644 // DefaultDirMode is the default mode when operation dir diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index a1ee4082301..ced863339b1 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -96,6 +96,8 @@ var customReplicaConfig = &ReplicaConfig{ FlushIntervalInMs: 500, Storage: "local://test", UseFileBackend: true, + EncoderWorkerNum: 14, + FlushWorkerNum: 23, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, @@ -140,6 +142,8 @@ var defaultReplicaConfig = &ReplicaConfig{ FlushIntervalInMs: redo.DefaultFlushIntervalInMs, Storage: "", UseFileBackend: false, + FlushWorkerNum: redo.DefaultFlushWorkerNum, + EncoderWorkerNum: redo.DefaultEncodingWorkerNum, }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, diff --git a/tests/integration_tests/api_v2/model.go b/tests/integration_tests/api_v2/model.go index 2e450c806c6..f33a0130d7b 100644 --- a/tests/integration_tests/api_v2/model.go +++ b/tests/integration_tests/api_v2/model.go @@ -277,6 +277,8 @@ type ConsistentConfig struct { MaxLogSize int64 `json:"max_log_size"` FlushIntervalInMs int64 `json:"flush_interval"` MetaFlushIntervalInMs int64 `json:"meta_flush_interval"` + EncoderWorkerNum int `json:"encoding_worker_num"` + FlushWorkerNum int `json:"flush_worker_num"` Storage string `json:"storage"` UseFileBackend bool `json:"use_file_backend"` } From 6aa39d0d036f45c4be38f9a7151047e2697c8097 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 13:58:19 +0800 Subject: [PATCH 17/23] redo(ticdc): fix redo zero flush interval panic issue (#10102) (#10104) close pingcap/tiflow#10101 --- cdc/redo/manager.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 6f3b0804185..7d2370c0c9a 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -279,8 +279,18 @@ func (m *logManager) Run(ctx context.Context, _ ...chan<- error) error { func (m *logManager) getFlushDuration() time.Duration { flushIntervalInMs := m.cfg.FlushIntervalInMs + defaultFlushIntervalInMs := redo.DefaultFlushIntervalInMs if m.cfg.LogType == redo.RedoDDLLogFileType { flushIntervalInMs = m.cfg.MetaFlushIntervalInMs + defaultFlushIntervalInMs = redo.DefaultMetaFlushIntervalInMs + } + if flushIntervalInMs < redo.MinFlushIntervalInMs { + log.Warn("redo flush interval is too small, use default value", + zap.Stringer("namespace", m.cfg.ChangeFeedID), + zap.Int("default", defaultFlushIntervalInMs), + zap.String("logType", m.cfg.LogType), + zap.Int64("interval", flushIntervalInMs)) + flushIntervalInMs = int64(defaultFlushIntervalInMs) } return time.Duration(flushIntervalInMs) * time.Millisecond } From 3f6e111255860290f0c545ace7870620e5f6ba0e Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 15:37:19 +0800 Subject: [PATCH 18/23] codec(ticdc): canal-json support compatible content by output detailed mysql type information (#10014) (#10128) close pingcap/tiflow#10106 --- cdc/api/v2/model.go | 3 + cdc/sink/dmlsink/mq/mq_dml_sink_test.go | 23 +- cdc/sink/dmlsink/mq/worker_test.go | 131 ++- pkg/config/config_test_data.go | 4 +- pkg/config/replica_config_test.go | 1 + pkg/config/sink.go | 3 + pkg/sink/codec/avro/avro_test.go | 47 +- pkg/sink/codec/canal/canal_encoder.go | 2 +- pkg/sink/codec/canal/canal_encoder_test.go | 109 +- pkg/sink/codec/canal/canal_entry.go | 72 +- pkg/sink/codec/canal/canal_entry_test.go | 154 +-- pkg/sink/codec/canal/canal_json_decoder.go | 3 +- .../codec/canal/canal_json_decoder_test.go | 15 +- pkg/sink/codec/canal/canal_json_message.go | 57 +- .../canal/canal_json_row_event_encoder.go | 11 +- .../canal_json_row_event_encoder_test.go | 308 ++++-- .../canal/canal_json_txn_event_encoder.go | 2 +- .../canal_json_txn_event_encoder_test.go | 48 +- pkg/sink/codec/canal/canal_test_util.go | 411 ++++---- pkg/sink/codec/canal/type_test.go | 960 ++++++++++++++++++ pkg/sink/codec/common/config.go | 15 + pkg/sink/codec/common/config_test.go | 17 + pkg/sink/codec/common/field_types.go | 52 + pkg/sink/codec/common/mysql_types.go | 65 ++ pkg/sink/codec/internal/java.go | 15 +- pkg/sink/codec/utils/field_types.go | 82 ++ .../conf/diff_config.toml | 29 + .../data/data.sql | 330 ++++++ .../data/data_gbk.sql | 101 ++ .../canal_json_content_compatible/run.sh | 56 + tests/integration_tests/run_group.sh | 2 +- 31 files changed, 2557 insertions(+), 571 deletions(-) create mode 100644 pkg/sink/codec/canal/type_test.go create mode 100644 pkg/sink/codec/common/field_types.go create mode 100644 pkg/sink/codec/common/mysql_types.go create mode 100644 pkg/sink/codec/utils/field_types.go create mode 100644 tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml create mode 100644 tests/integration_tests/canal_json_content_compatible/data/data.sql create mode 100644 tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql create mode 100644 tests/integration_tests/canal_json_content_compatible/run.sh diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index f79772ac5f5..b117e51133d 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -400,6 +400,7 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( FileIndexWidth: c.Sink.FileIndexWidth, EnableKafkaSinkV2: c.Sink.EnableKafkaSinkV2, OnlyOutputUpdatedColumns: c.Sink.OnlyOutputUpdatedColumns, + ContentCompatible: c.Sink.ContentCompatible, KafkaConfig: kafkaConfig, MySQLConfig: mysqlConfig, CloudStorageConfig: cloudStorageConfig, @@ -619,6 +620,7 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { FileIndexWidth: cloned.Sink.FileIndexWidth, EnableKafkaSinkV2: cloned.Sink.EnableKafkaSinkV2, OnlyOutputUpdatedColumns: cloned.Sink.OnlyOutputUpdatedColumns, + ContentCompatible: cloned.Sink.ContentCompatible, KafkaConfig: kafkaConfig, MySQLConfig: mysqlConfig, CloudStorageConfig: cloudStorageConfig, @@ -779,6 +781,7 @@ type SinkConfig struct { EnableKafkaSinkV2 bool `json:"enable_kafka_sink_v2"` OnlyOutputUpdatedColumns *bool `json:"only_output_updated_columns"` SafeMode *bool `json:"safe_mode,omitempty"` + ContentCompatible *bool `json:"content_compatible"` KafkaConfig *KafkaConfig `json:"kafka_config,omitempty"` MySQLConfig *MySQLConfig `json:"mysql_config,omitempty"` CloudStorageConfig *CloudStorageConfig `json:"cloud_storage_config,omitempty"` diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink_test.go b/cdc/sink/dmlsink/mq/mq_dml_sink_test.go index d393479ad00..e751efd4f86 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink_test.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink_test.go @@ -11,6 +11,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build intest +// +build intest + package mq import ( @@ -20,6 +23,8 @@ import ( "testing" "time" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer" @@ -56,8 +61,6 @@ func TestNewKafkaDMLSinkFailed(t *testing.T) { } func TestWriteEvents(t *testing.T) { - t.Parallel() - ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -79,11 +82,21 @@ func TestWriteEvents(t *testing.T) { require.NotNil(t, s) defer s.Close() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() + tableStatus := state.TableSinkSinking row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, } events := make([]*dmlsink.CallbackableEvent[*model.SingleTableTxn], 0, 3000) diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index de717e4e820..2add072ecb2 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -11,6 +11,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build intest +// +build intest + package mq import ( @@ -19,6 +22,8 @@ import ( "testing" "time" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer" @@ -57,7 +62,13 @@ func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlpro } func TestNonBatchEncode_SendMessages(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -70,9 +81,11 @@ func TestNonBatchEncode_SendMessages(t *testing.T) { Partition: 1, } row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, } tableStatus := state.TableSinkSinking @@ -250,7 +263,13 @@ func TestBatchEncode_Group(t *testing.T) { } func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() key1 := TopicPartitionKey{ Topic: "test", @@ -270,9 +289,11 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -317,8 +338,6 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { } func TestBatchEncode_SendMessages(t *testing.T) { - t.Parallel() - key1 := TopicPartitionKey{ Topic: "test", Partition: 1, @@ -337,13 +356,24 @@ func TestBatchEncode_SendMessages(t *testing.T) { defer cancel() worker, p := newBatchEncodeWorker(ctx, t) defer worker.close() + + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() + events := []mqEvent{ { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -353,9 +383,11 @@ func TestBatchEncode_SendMessages(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -365,9 +397,11 @@ func TestBatchEncode_SendMessages(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "cc"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -377,9 +411,11 @@ func TestBatchEncode_SendMessages(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "aa", Table: "bb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -389,9 +425,11 @@ func TestBatchEncode_SendMessages(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -401,9 +439,11 @@ func TestBatchEncode_SendMessages(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "aaa", Table: "bbb"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &tableStatus, @@ -461,8 +501,6 @@ func TestBatchEncodeWorker_Abort(t *testing.T) { } func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { - t.Parallel() - key1 := TopicPartitionKey{ Topic: "test", Partition: 1, @@ -477,13 +515,24 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { defer worker.close() replicatingStatus := state.TableSinkSinking stoppedStatus := state.TableSinkStopping + + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() + events := []mqEvent{ { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "aa"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -493,9 +542,11 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "a", Type: mysql.TypeVarchar, Value: "bb"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &replicatingStatus, @@ -505,9 +556,11 @@ func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { { rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: &model.RowChangedEvent{ - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, + CommitTs: 3, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: "cc"}}, + ColInfos: colInfo, }, Callback: func() {}, SinkState: &stoppedStatus, diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index f3471ddd357..d72b3aea310 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -210,9 +210,9 @@ const ( "terminator": "", "date-separator": "month", "enable-partition-separator": true, - "only-output-updated-columns": false, "enable-kafka-sink-v2": true, "only-output-updated-columns": true, + "content-compatible": true, "safe-mode": true, "kafka-config": { "partition-num": 1, @@ -349,9 +349,9 @@ const ( "large-message-handle-option": "handle-key-only" } }, - "only-output-updated-columns": false, "enable-kafka-sink-v2": true, "only-output-updated-columns": true, + "content-compatible": true, "safe-mode": true, "kafka-config": { "partition-num": 1, diff --git a/pkg/config/replica_config_test.go b/pkg/config/replica_config_test.go index 5a67b8db7d9..124fb0ea427 100644 --- a/pkg/config/replica_config_test.go +++ b/pkg/config/replica_config_test.go @@ -64,6 +64,7 @@ func TestReplicaConfigMarshal(t *testing.T) { conf.Scheduler.WriteKeyThreshold = 100001 conf.Sink.OnlyOutputUpdatedColumns = aws.Bool(true) + conf.Sink.ContentCompatible = aws.Bool(true) conf.Sink.SafeMode = aws.Bool(true) conf.Sink.AdvanceTimeoutInSec = util.AddressOf(uint(150)) conf.Sink.KafkaConfig = &KafkaConfig{ diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 4eb50ff215b..4e890d80aef 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -136,6 +136,9 @@ type SinkConfig struct { OnlyOutputUpdatedColumns *bool `toml:"only-output-updated-columns" json:"only-output-updated-columns"` + // ContentCompatible is only available when the downstream is MQ. + ContentCompatible *bool `toml:"content-compatible" json:"content-compatible,omitempty"` + // TiDBSourceID is the source ID of the upstream TiDB, // which is used to set the `tidb_cdc_write_source` session variable. // Note: This field is only used internally and only used in the MySQL sink. diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index 3296fb818fd..d8944a59e75 100644 --- a/pkg/sink/codec/avro/avro_test.go +++ b/pkg/sink/codec/avro/avro_test.go @@ -74,22 +74,6 @@ func teardownEncoderAndSchemaRegistry() { stopHTTPInterceptForTestingRegistry() } -func setBinChsClnFlag(ft *types.FieldType) *types.FieldType { - types.SetBinChsClnFlag(ft) - return ft -} - -//nolint:unparam -func setFlag(ft *types.FieldType, flag uint) *types.FieldType { - ft.SetFlag(flag) - return ft -} - -func setElems(ft *types.FieldType, elems []string) *types.FieldType { - ft.SetElems(elems) - return ft -} - type avroTestColumnTuple struct { col model.Column colInfo rowcodec.ColInfo @@ -165,7 +149,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 6, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeTiny), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeTiny)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -181,7 +165,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 7, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeShort), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeShort)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -197,7 +181,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 8, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeInt24), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeInt24)), }, avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int32(1), "int", @@ -213,7 +197,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 9, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeLong), uint(model.UnsignedFlag)), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLong)), }, avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, int64(1), "long", @@ -229,10 +213,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 10, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag( - types.NewFieldType(mysql.TypeLonglong), - uint(model.UnsignedFlag), - ), + Ft: common.SetUnsigned(types.NewFieldType(mysql.TypeLonglong)), }, avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}}, int64(1), "long", @@ -377,7 +358,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 22, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -393,7 +374,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 23, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -409,7 +390,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 24, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -425,7 +406,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 25, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -441,7 +422,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 26, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -457,7 +438,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 27, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -473,7 +454,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 28, IsPKHandle: false, VirtualGenCol: false, - Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + Ft: common.SetBinChsClnFlag(types.NewFieldType(mysql.TypeString)), }, avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, []byte("hello world"), "bytes", @@ -484,7 +465,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 29, IsPKHandle: false, VirtualGenCol: false, - Ft: setElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), + Ft: common.SetElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), }, avroSchema{ Type: "string", @@ -498,7 +479,7 @@ var avroTestColumns = []*avroTestColumnTuple{ ID: 30, IsPKHandle: false, VirtualGenCol: false, - Ft: setElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), + Ft: common.SetElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), }, avroSchema{ Type: "string", diff --git a/pkg/sink/codec/canal/canal_encoder.go b/pkg/sink/codec/canal/canal_encoder.go index 6e75e79d109..f3304d64264 100644 --- a/pkg/sink/codec/canal/canal_encoder.go +++ b/pkg/sink/codec/canal/canal_encoder.go @@ -162,7 +162,7 @@ func newBatchEncoder(config *common.Config) codec.RowEventEncoder { encoder := &BatchEncoder{ messages: &canal.Messages{}, callbackBuf: make([]func(), 0), - entryBuilder: newCanalEntryBuilder(), + entryBuilder: newCanalEntryBuilder(config), config: config, } diff --git a/pkg/sink/codec/canal/canal_encoder_test.go b/pkg/sink/codec/canal/canal_encoder_test.go index 2fe8bc06372..b56cbf9a730 100644 --- a/pkg/sink/codec/canal/canal_encoder_test.go +++ b/pkg/sink/codec/canal/canal_encoder_test.go @@ -19,6 +19,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" @@ -26,14 +27,87 @@ import ( "github.com/stretchr/testify/require" ) +var ( + rowCases = [][]*model.RowChangedEvent{ + {{ + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{ + Name: "col1", + Type: mysql.TypeVarchar, + Value: []byte("aa"), + }}, + }}, + { + { + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{ + Name: "col1", + Type: mysql.TypeVarchar, + Value: []byte("aa"), + }}, + }, + { + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, + }, + }, + } + + ddlCases = [][]*model.DDLEvent{ + {{ + CommitTs: 1, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table a", + Type: 1, + }}, + { + { + CommitTs: 2, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table b", + Type: 3, + }, + { + CommitTs: 3, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "a", Table: "b", + }, + }, + Query: "create table c", + Type: 3, + }, + }, + } +) + func TestCanalBatchEncoder(t *testing.T) { - t.Parallel() - s := defaultCanalBatchTester - for _, cs := range s.rowCases { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(10) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + for _, cs := range rowCases { encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) for _, row := range cs { + _, _, colInfo := tableInfo.GetRowColInfos() + row.TableInfo = tableInfo + row.ColInfos = colInfo err := encoder.AppendRowChangedEvent(context.Background(), "", row, nil) - require.Nil(t, err) + require.NoError(t, err) } res := encoder.Build() @@ -41,7 +115,6 @@ func TestCanalBatchEncoder(t *testing.T) { require.Nil(t, res) continue } - require.Len(t, res, 1) require.Nil(t, res[0].Key) require.Equal(t, len(cs), res[0].GetRowsCount()) @@ -56,33 +129,36 @@ func TestCanalBatchEncoder(t *testing.T) { require.Equal(t, len(cs), len(messages.GetMessages())) } - for _, cs := range s.ddlCases { + for _, cs := range ddlCases { encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) for _, ddl := range cs { msg, err := encoder.EncodeDDLEvent(ddl) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, msg) require.Nil(t, msg.Key) packet := &canal.Packet{} err = proto.Unmarshal(msg.Value, packet) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, canal.PacketType_MESSAGES, packet.GetType()) messages := &canal.Messages{} err = proto.Unmarshal(packet.GetBody(), messages) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, len(messages.GetMessages())) - require.Nil(t, err) + require.NoError(t, err) } } } func TestCanalAppendRowChangedEventWithCallback(t *testing.T) { - encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) - require.NotNil(t, encoder) + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() - count := 0 + sql := `create table test.t(a varchar(10) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() row := &model.RowChangedEvent{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, @@ -91,8 +167,15 @@ func TestCanalAppendRowChangedEventWithCallback(t *testing.T) { Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + TableInfo: tableInfo, + ColInfos: colInfo, } + encoder := newBatchEncoder(common.NewConfig(config.ProtocolCanal)) + require.NotNil(t, encoder) + + count := 0 + tests := []struct { row *model.RowChangedEvent callback func() diff --git a/pkg/sink/codec/canal/canal_entry.go b/pkg/sink/codec/canal/canal_entry.go index 41f8e4bf6b2..0442b7660d5 100644 --- a/pkg/sink/codec/canal/canal_entry.go +++ b/pkg/sink/codec/canal/canal_entry.go @@ -18,15 +18,14 @@ import ( "math" "reflect" "strconv" - "strings" "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/internal" canal "github.com/pingcap/tiflow/proto/canal" "golang.org/x/text/encoding" @@ -43,12 +42,14 @@ const ( type canalEntryBuilder struct { bytesDecoder *encoding.Decoder // default charset is ISO-8859-1 + config *common.Config } // newCanalEntryBuilder creates a new canalEntryBuilder -func newCanalEntryBuilder() *canalEntryBuilder { +func newCanalEntryBuilder(config *common.Config) *canalEntryBuilder { return &canalEntryBuilder{ bytesDecoder: charmap.ISO8859_1.NewDecoder(), + config: config, } } @@ -113,8 +114,8 @@ func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (resul // build the Column in the canal RowData // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L756-L872 -func (b *canalEntryBuilder) buildColumn(c *model.Column, updated bool) (*canal.Column, error) { - mysqlType := getMySQLType(c.Type, c.Flag) +func (b *canalEntryBuilder) buildColumn(c *model.Column, columnInfo *mm.ColumnInfo, updated bool) (*canal.Column, error) { + mysqlType := common.GetMySQLType(columnInfo, b.config.ContentCompatible) javaType, err := getJavaSQLType(c.Value, c.Type, c.Flag) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) @@ -140,11 +141,16 @@ func (b *canalEntryBuilder) buildColumn(c *model.Column, updated bool) (*canal.C // build the RowData of a canal entry func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*canal.RowData, error) { var columns []*canal.Column - for _, column := range e.Columns { + for idx, column := range e.Columns { if column == nil { continue } - c, err := b.buildColumn(column, !e.IsDelete()) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "column info not found for column id: %d", e.ColInfos[idx].ID) + } + c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -153,14 +159,19 @@ func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKey onlyHandleKeyColumns = onlyHandleKeyColumns && e.IsDelete() var preColumns []*canal.Column - for _, column := range e.PreColumns { + for idx, column := range e.PreColumns { if column == nil { continue } if onlyHandleKeyColumns && !column.Flag.IsHandleKey() { continue } - c, err := b.buildColumn(column, !e.IsDelete()) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "column info not found for column id: %d", e.ColInfos[idx].ID) + } + c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) if err != nil { return nil, errors.Trace(err) } @@ -295,15 +306,6 @@ func isCanalDDL(t canal.EventType) bool { func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (result internal.JavaSQLType, err error) { javaType := internal.MySQLType2JavaType(tp, flag.IsBinary()) - - switch javaType { - case internal.JavaSQLTypeBINARY, internal.JavaSQLTypeVARBINARY, internal.JavaSQLTypeLONGVARBINARY: - if flag.IsBinary() { - return internal.JavaSQLTypeBLOB, nil - } - return internal.JavaSQLTypeCLOB, nil - } - // flag `isUnsigned` only for `numerical` and `bit`, `year` data type. if !flag.IsUnsigned() { return javaType, nil @@ -366,37 +368,3 @@ func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (resu return javaType, nil } - -// when encoding the canal format, for unsigned mysql type, add `unsigned` keyword. -// it should have the form `t unsigned`, such as `int unsigned` -func withUnsigned4MySQLType(mysqlType string, unsigned bool) string { - if unsigned && mysqlType != "bit" && mysqlType != "year" { - return mysqlType + " unsigned" - } - return mysqlType -} - -// when decoding the canal format, remove `unsigned` to get the original `mysql type`. -func trimUnsignedFromMySQLType(mysqlType string) string { - return strings.TrimSuffix(mysqlType, " unsigned") -} - -func getMySQLType(tp byte, flag model.ColumnFlagType) string { - mysqlType := types.TypeStr(tp) - // make `mysqlType` representation keep the same as the canal official implementation - mysqlType = withUnsigned4MySQLType(mysqlType, flag.IsUnsigned()) - - if !flag.IsBinary() { - return mysqlType - } - - if types.IsTypeBlob(tp) { - return strings.Replace(mysqlType, "text", "blob", 1) - } - - if types.IsTypeChar(tp) { - return strings.Replace(mysqlType, "char", "binary", 1) - } - - return mysqlType -} diff --git a/pkg/sink/codec/canal/canal_entry_test.go b/pkg/sink/codec/canal/canal_entry_test.go index 462dd09314e..3614db8a1b6 100644 --- a/pkg/sink/codec/canal/canal_entry_test.go +++ b/pkg/sink/codec/canal/canal_entry_test.go @@ -19,47 +19,38 @@ import ( "github.com/golang/protobuf/proto" mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/internal" canal "github.com/pingcap/tiflow/proto/canal" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" ) -func TestGetMySQLTypeAndJavaSQLType(t *testing.T) { - t.Parallel() - canalEntryBuilder := newCanalEntryBuilder() - for _, item := range testColumnsTable { - obtainedMySQLType := getMySQLType(item.column.Type, item.column.Flag) - require.Equal(t, item.expectedMySQLType, obtainedMySQLType) +func TestInsert(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() - obtainedJavaSQLType, err := getJavaSQLType(item.column.Value, item.column.Type, item.column.Flag) - require.NoError(t, err) - require.Equal(t, item.expectedJavaSQLType, obtainedJavaSQLType) + sql := `create table test.t( + id int primary key, + name varchar(32), + tiny tinyint unsigned, + comment text, + bb blob)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) - obtainedFinalValue, err := canalEntryBuilder.formatValue(item.column.Value, item.column.Flag.IsBinary()) - require.NoError(t, err) - if !item.column.Flag.IsBinary() { - require.Equal(t, item.expectedEncodedValue, obtainedFinalValue) - } - } -} - -func TestConvertEntry(t *testing.T) { - t.Parallel() - testInsert(t) - testUpdate(t) - testDelete(t) - testDdl(t) -} + _, _, colInfos := tableInfo.GetRowColInfos() -func testInsert(t *testing.T) { - testCaseInsert := &model.RowChangedEvent{ + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, @@ -67,29 +58,31 @@ func testInsert(t *testing.T) { {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, {Name: "blob", Type: mysql.TypeBlob, Value: []byte("测试blob"), Flag: model.BinaryFlag}, }, + ColInfos: colInfos, } - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseInsert, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, testCaseInsert.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseInsert.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_INSERT, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) columns := rowDatas[0].AfterColumns - require.Equal(t, len(testCaseInsert.Columns), len(columns)) + require.Equal(t, len(event.Columns), len(columns)) for _, col := range columns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -114,28 +107,38 @@ func testInsert(t *testing.T) { require.Equal(t, int32(internal.JavaSQLTypeCLOB), col.GetSqlType()) require.False(t, col.GetIsKey()) require.False(t, col.GetIsNull()) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, "测试", col.GetValue()) require.Equal(t, "text", col.GetMysqlType()) - case "blob": + case "bb": require.Equal(t, int32(internal.JavaSQLTypeBLOB), col.GetSqlType()) require.False(t, col.GetIsKey()) require.False(t, col.GetIsNull()) s, err := charmap.ISO8859_1.NewEncoder().String(col.GetValue()) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, "测试blob", s) require.Equal(t, "blob", col.GetMysqlType()) } } } -func testUpdate(t *testing.T) { - testCaseUpdate := &model.RowChangedEvent{ +func TestUpdate(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(id int primary key, name varchar(32))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, Columns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, @@ -144,29 +147,31 @@ func testUpdate(t *testing.T) { {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 2}, {Name: "name", Type: mysql.TypeVarchar, Value: "Nancy"}, }, + ColInfos: colInfos, } - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseUpdate, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() require.Equal(t, int64(1591943372224), header.GetExecuteTime()) require.Equal(t, canal.Type_MYSQL, header.GetSourceType()) - require.Equal(t, testCaseUpdate.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseUpdate.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_UPDATE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) beforeColumns := rowDatas[0].BeforeColumns - require.Equal(t, len(testCaseUpdate.PreColumns), len(beforeColumns)) + require.Equal(t, len(event.PreColumns), len(beforeColumns)) for _, col := range beforeColumns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -186,7 +191,7 @@ func testUpdate(t *testing.T) { } afterColumns := rowDatas[0].AfterColumns - require.Equal(t, len(testCaseUpdate.Columns), len(afterColumns)) + require.Equal(t, len(event.Columns), len(afterColumns)) for _, col := range afterColumns { require.True(t, col.GetUpdated()) switch col.GetName() { @@ -206,37 +211,48 @@ func testUpdate(t *testing.T) { } } -func testDelete(t *testing.T) { - testCaseDelete := &model.RowChangedEvent{ +func TestDelete(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(id int primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + event := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, PreColumns: []*model.Column{ {Name: "id", Type: mysql.TypeLong, Flag: model.PrimaryKeyFlag, Value: 1}, }, + ColInfos: colInfos, } - - builder := newCanalEntryBuilder() - entry, err := builder.fromRowEvent(testCaseDelete, false) - require.Nil(t, err) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := newCanalEntryBuilder(codecConfig) + entry, err := builder.fromRowEvent(event, false) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() - require.Equal(t, testCaseDelete.Table.Schema, header.GetSchemaName()) - require.Equal(t, testCaseDelete.Table.Table, header.GetTableName()) + require.Equal(t, event.Table.Schema, header.GetSchemaName()) + require.Equal(t, event.Table.Table, header.GetTableName()) require.Equal(t, canal.EventType_DELETE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.False(t, rc.GetIsDdl()) rowDatas := rc.GetRowDatas() require.Equal(t, 1, len(rowDatas)) columns := rowDatas[0].BeforeColumns - require.Equal(t, len(testCaseDelete.PreColumns), len(columns)) + require.Equal(t, len(event.PreColumns), len(columns)) for _, col := range columns { require.False(t, col.GetUpdated()) switch col.GetName() { @@ -250,8 +266,8 @@ func testDelete(t *testing.T) { } } -func testDdl(t *testing.T) { - testCaseDdl := &model.DDLEvent{ +func TestDDL(t *testing.T) { + event := &model.DDLEvent{ CommitTs: 417318403368288260, TableInfo: &model.TableInfo{ TableName: model.TableName{ @@ -261,19 +277,19 @@ func testDdl(t *testing.T) { Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", Type: mm.ActionCreateTable, } - builder := newCanalEntryBuilder() - entry, err := builder.fromDDLEvent(testCaseDdl) - require.Nil(t, err) + builder := newCanalEntryBuilder(nil) + entry, err := builder.fromDDLEvent(event) + require.NoError(t, err) require.Equal(t, canal.EntryType_ROWDATA, entry.GetEntryType()) header := entry.GetHeader() - require.Equal(t, testCaseDdl.TableInfo.TableName.Schema, header.GetSchemaName()) - require.Equal(t, testCaseDdl.TableInfo.TableName.Table, header.GetTableName()) + require.Equal(t, event.TableInfo.TableName.Schema, header.GetSchemaName()) + require.Equal(t, event.TableInfo.TableName.Table, header.GetTableName()) require.Equal(t, canal.EventType_CREATE, header.GetEventType()) store := entry.GetStoreValue() require.NotNil(t, store) rc := &canal.RowChange{} err = proto.Unmarshal(store, rc) - require.Nil(t, err) + require.NoError(t, err) require.True(t, rc.GetIsDdl()) - require.Equal(t, testCaseDdl.TableInfo.TableName.Schema, rc.GetDdlSchemaName()) + require.Equal(t, event.TableInfo.TableName.Schema, rc.GetDdlSchemaName()) } diff --git a/pkg/sink/codec/canal/canal_json_decoder.go b/pkg/sink/codec/canal/canal_json_decoder.go index 6f09a1bf23c..b183768a01d 100644 --- a/pkg/sink/codec/canal/canal_json_decoder.go +++ b/pkg/sink/codec/canal/canal_json_decoder.go @@ -102,7 +102,6 @@ func (b *batchDecoder) HasNext() (model.MessageType, bool, error) { return model.MessageTypeUnknown, false, err } b.msg = msg - return b.msg.messageType(), true, nil } @@ -118,7 +117,7 @@ func (b *batchDecoder) buildData(holder *common.ColumnsHolder) (map[string]inter var value string rawValue := holder.Values[i].([]uint8) - if isBinaryMySQLType(mysqlType) { + if common.IsBinaryMySQLType(mysqlType) { rawValue, err := b.bytesDecoder.Bytes(rawValue) if err != nil { return nil, nil, errors.Trace(err) diff --git a/pkg/sink/codec/canal/canal_json_decoder_test.go b/pkg/sink/codec/canal/canal_json_decoder_test.go index d3ec988f432..7411248290b 100644 --- a/pkg/sink/codec/canal/canal_json_decoder_test.go +++ b/pkg/sink/codec/canal/canal_json_decoder_test.go @@ -24,8 +24,7 @@ import ( ) func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { - t.Parallel() - + insertEvent, _, _ := newLargeEvent4Test(t) ctx := context.Background() expectedDecodedValue := collectExpectedDecodedValue(testColumnsTable) for _, encodeEnable := range []bool{false, true} { @@ -36,8 +35,8 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { }) require.NotNil(t, encoder) - err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, nil) - require.Nil(t, err) + err := encoder.AppendRowChangedEvent(context.Background(), "", insertEvent, nil) + require.NoError(t, err) messages := encoder.Build() require.Equal(t, 1, len(messages)) @@ -52,16 +51,16 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { require.NoError(t, err) ty, hasNext, err := decoder.HasNext() - require.Nil(t, err) + require.NoError(t, err) require.True(t, hasNext) require.Equal(t, model.MessageTypeRow, ty) consumed, err := decoder.NextRowChangedEvent() require.Nil(t, err) - require.Equal(t, testCaseInsert.Table, consumed.Table) + require.Equal(t, insertEvent.Table, consumed.Table) if encodeEnable && decodeEnable { - require.Equal(t, testCaseInsert.CommitTs, consumed.CommitTs) + require.Equal(t, insertEvent.CommitTs, consumed.CommitTs) } else { require.Equal(t, uint64(0), consumed.CommitTs) } @@ -71,7 +70,7 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { require.True(t, ok) require.Equal(t, expected, col.Value) - for _, item := range testCaseInsert.Columns { + for _, item := range insertEvent.Columns { if item.Name == col.Name { require.Equal(t, item.Type, col.Type) } diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index 3b089f5e615..a58743202c6 100644 --- a/pkg/sink/codec/canal/canal_json_message.go +++ b/pkg/sink/codec/canal/canal_json_message.go @@ -15,14 +15,19 @@ package canal import ( "sort" + "strconv" "strings" + "github.com/pingcap/log" timodel "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/cdc/model" cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/sink/codec/internal" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/sink/codec/utils" canal "github.com/pingcap/tiflow/proto/canal" + "go.uber.org/zap" + "golang.org/x/text/encoding/charmap" ) const tidbWaterMarkType = "TIDB_WATERMARK" @@ -197,10 +202,7 @@ func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( "mysql type does not found, column: %+v, mysqlType: %+v", name, mysqlType) } - mysqlTypeStr = trimUnsignedFromMySQLType(mysqlTypeStr) - isBinary := isBinaryMySQLType(mysqlTypeStr) - mysqlType := types.StrToType(mysqlTypeStr) - col := internal.NewColumn(value, mysqlType).ToCanalJSONFormatColumn(name, isBinary) + col := canalJSONFormatColumn(value, name, mysqlTypeStr) result = append(result, col) } if len(result) == 0 { @@ -212,6 +214,45 @@ func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType return result, nil } +func canalJSONFormatColumn(value interface{}, name string, mysqlTypeStr string) *model.Column { + mysqlType := utils.ExtractBasicMySQLType(mysqlTypeStr) + result := &model.Column{ + Type: mysqlType, + Name: name, + Value: value, + } + if result.Value == nil { + return result + } + + data, ok := value.(string) + if !ok { + log.Panic("canal-json encoded message should have type in `string`") + } + + if mysqlType == mysql.TypeBit || mysqlType == mysql.TypeSet { + val, err := strconv.ParseUint(data, 10, 64) + if err != nil { + log.Panic("invalid column value for bit", zap.Any("col", result), zap.Error(err)) + } + result.Value = val + return result + } + + var err error + if common.IsBinaryMySQLType(mysqlTypeStr) { + // when encoding the `JavaSQLTypeBLOB`, use `ISO8859_1` decoder, now reverse it back. + encoder := charmap.ISO8859_1.NewEncoder() + value, err = encoder.String(data) + if err != nil { + log.Panic("invalid column value, please report a bug", zap.Any("col", result), zap.Error(err)) + } + } + + result.Value = value + return result +} + func canalJSONMessage2DDLEvent(msg canalJSONMessageInterface) *model.DDLEvent { result := new(model.DDLEvent) // we lost the startTs from kafka message @@ -245,7 +286,3 @@ func getDDLActionType(query string) timodel.ActionType { return timodel.ActionNone } - -func isBinaryMySQLType(mysqlType string) bool { - return strings.Contains(mysqlType, "blob") || strings.Contains(mysqlType, "binary") -} diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index 2e4a1619ee7..04cfa103cd1 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -155,7 +155,7 @@ func newJSONMessageForDML( const prefix string = ",\"sqlType\":" out.RawString(prefix) emptyColumn := true - for _, col := range columns { + for idx, col := range columns { if col != nil { if onlyHandleKey && !col.Flag.IsHandleKey() { continue @@ -173,7 +173,12 @@ func newJSONMessageForDML( out.String(col.Name) out.RawByte(':') out.Int32(int32(javaType)) - mysqlTypeMap[col.Name] = getMySQLType(col.Type, col.Flag) + columnInfo, ok := e.TableInfo.GetColumnInfo(e.ColInfos[idx].ID) + if !ok { + return nil, cerror.ErrCanalEncodeFailed.GenWithStack( + "cannot found the column info by the column ID: %d", e.ColInfos[idx].ID) + } + mysqlTypeMap[col.Name] = common.GetMySQLType(columnInfo, config.ContentCompatible) } } if emptyColumn { @@ -278,7 +283,7 @@ type JSONRowEventEncoder struct { // newJSONRowEventEncoder creates a new JSONRowEventEncoder func newJSONRowEventEncoder(config *common.Config) codec.RowEventEncoder { encoder := &JSONRowEventEncoder{ - builder: newCanalEntryBuilder(), + builder: newCanalEntryBuilder(config), messages: make([]*common.Message, 0, 1), config: config, diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go index 75651f08899..0dd6b0b4b37 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go @@ -11,16 +11,22 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build intest +// +build intest + package canal import ( "context" + "database/sql" "encoding/json" "testing" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" @@ -30,39 +36,38 @@ func TestBuildCanalJSONRowEventEncoder(t *testing.T) { t.Parallel() cfg := common.NewConfig(config.ProtocolCanalJSON) - builder := &jsonRowEventEncoderBuilder{config: cfg} + builder := NewJSONRowEventEncoderBuilder(cfg) encoder, ok := builder.Build().(*JSONRowEventEncoder) require.True(t, ok) require.NotNil(t, encoder.config) } func TestNewCanalJSONMessage4DML(t *testing.T) { - t.Parallel() - e := newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: false, - Terminator: "", - }) - require.NotNil(t, e) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := NewJSONRowEventEncoderBuilder(codecConfig) - encoder, ok := e.(*JSONRowEventEncoder) + encoder, ok := builder.Build().(*JSONRowEventEncoder) require.True(t, ok) - data, err := newJSONMessageForDML(testCaseInsert, encoder.config, encoder.builder, false) - require.Nil(t, err) + insertEvent, updateEvent, deleteEvent := newLargeEvent4Test(t) + data, err := newJSONMessageForDML(insertEvent, codecConfig, encoder.builder, false) + require.NoError(t, err) + var msg canalJSONMessageInterface = &JSONMessage{} err = json.Unmarshal(data, msg) - require.Nil(t, err) + require.NoError(t, err) + jsonMsg, ok := msg.(*JSONMessage) require.True(t, ok) require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) require.Equal(t, "INSERT", jsonMsg.EventType) - require.Equal(t, convertToCanalTs(testCaseInsert.CommitTs), jsonMsg.ExecutionTime) - require.Equal(t, "cdc", jsonMsg.Schema) - require.Equal(t, "person", jsonMsg.Table) + require.Equal(t, convertToCanalTs(insertEvent.CommitTs), jsonMsg.ExecutionTime) + require.Equal(t, "test", jsonMsg.Schema) + require.Equal(t, "t", jsonMsg.Table) require.False(t, jsonMsg.IsDDL) - for _, col := range testCaseInsert.Columns { + for _, col := range insertEvent.Columns { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) require.Contains(t, jsonMsg.MySQLType, col.Name) @@ -88,7 +93,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { if bytes, ok := item.column.Value.([]byte); ok { expectedValue, err := charmap.ISO8859_1.NewDecoder().Bytes(bytes) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, string(expectedValue), obtainedValue) continue } @@ -96,46 +101,51 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { require.Equal(t, item.expectedEncodedValue, obtainedValue) } - data, err = newJSONMessageForDML(testCaseUpdate, encoder.config, encoder.builder, false) - require.Nil(t, err) + data, err = newJSONMessageForDML(updateEvent, codecConfig, encoder.builder, false) + require.NoError(t, err) + jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) - require.Nil(t, err) + require.NoError(t, err) + require.NotNil(t, jsonMsg.Data) require.NotNil(t, jsonMsg.Old) require.Equal(t, "UPDATE", jsonMsg.EventType) - for _, col := range testCaseUpdate.Columns { + for _, col := range updateEvent.Columns { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) require.Contains(t, jsonMsg.MySQLType, col.Name) } - for _, col := range testCaseUpdate.PreColumns { + for _, col := range updateEvent.PreColumns { require.Contains(t, jsonMsg.Old[0], col.Name) } - data, err = newJSONMessageForDML(testCaseDelete, encoder.config, encoder.builder, false) - require.Nil(t, err) + data, err = newJSONMessageForDML(deleteEvent, codecConfig, encoder.builder, false) + require.NoError(t, err) + jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) require.Equal(t, "DELETE", jsonMsg.EventType) - for _, col := range testCaseDelete.PreColumns { + for _, col := range deleteEvent.PreColumns { require.Contains(t, jsonMsg.Data[0], col.Name) } - data, err = newJSONMessageForDML(testCaseDelete, &common.Config{DeleteOnlyHandleKeyColumns: true}, encoder.builder, false) + codecConfig = &common.Config{DeleteOnlyHandleKeyColumns: true} + data, err = newJSONMessageForDML(deleteEvent, codecConfig, encoder.builder, false) require.NoError(t, err) + jsonMsg = &JSONMessage{} err = json.Unmarshal(data, jsonMsg) require.NoError(t, err) require.NotNil(t, jsonMsg.Data) require.Nil(t, jsonMsg.Old) - for _, col := range testCaseDelete.PreColumns { + for _, col := range deleteEvent.PreColumns { if col.Flag.IsHandleKey() { require.Contains(t, jsonMsg.Data[0], col.Name) require.Contains(t, jsonMsg.SQLType, col.Name) @@ -147,67 +157,75 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { } } - e = newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: true, - Terminator: "", - OnlyOutputUpdatedColumns: true, - }) - require.NotNil(t, e) + codecConfig = common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + codecConfig.OnlyOutputUpdatedColumns = true - encoder, ok = e.(*JSONRowEventEncoder) + builder = NewJSONRowEventEncoderBuilder(codecConfig) + encoder, ok = builder.Build().(*JSONRowEventEncoder) require.True(t, ok) - data, err = newJSONMessageForDML(testCaseUpdate, encoder.config, encoder.builder, false) - require.Nil(t, err) + data, err = newJSONMessageForDML(updateEvent, codecConfig, encoder.builder, false) + require.NoError(t, err) withExtension := &canalJSONMessageWithTiDBExtension{} err = json.Unmarshal(data, withExtension) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, withExtension.Extensions) - require.Equal(t, testCaseUpdate.CommitTs, withExtension.Extensions.CommitTs) + require.Equal(t, updateEvent.CommitTs, withExtension.Extensions.CommitTs) - encoder, ok = e.(*JSONRowEventEncoder) + encoder, ok = builder.Build().(*JSONRowEventEncoder) require.True(t, ok) - data, err = newJSONMessageForDML(testCaseUpdate, encoder.config, encoder.builder, false) - require.Nil(t, err) + data, err = newJSONMessageForDML(updateEvent, codecConfig, encoder.builder, false) + require.NoError(t, err) withExtension = &canalJSONMessageWithTiDBExtension{} err = json.Unmarshal(data, withExtension) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 0, len(withExtension.JSONMessage.Old[0])) require.NotNil(t, withExtension.Extensions) - require.Equal(t, testCaseUpdate.CommitTs, withExtension.Extensions.CommitTs) + require.Equal(t, updateEvent.CommitTs, withExtension.Extensions.CommitTs) } func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { - t.Parallel() - codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionHandleKeyOnly codecConfig.MaxMessageBytes = 500 - encoder := newJSONRowEventEncoder(codecConfig) - err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, func() {}) + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() + + insertEvent, _, _ := newLargeEvent4Test(t) + err := encoder.AppendRowChangedEvent(context.Background(), "", insertEvent, func() {}) require.NoError(t, err) message := encoder.Build()[0] - var decoded canalJSONMessageWithTiDBExtension - err = json.Unmarshal(message.Value, &decoded) + decoder, err := NewBatchDecoder(context.Background(), codecConfig, &sql.DB{}) require.NoError(t, err) - require.True(t, decoded.Extensions.OnlyHandleKey) - for _, col := range testCaseInsert.Columns { + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + + messageType, ok, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, ok) + require.Equal(t, messageType, model.MessageTypeRow) + + handleKeyOnlyMessage := decoder.(*batchDecoder).msg.(*canalJSONMessageWithTiDBExtension) + require.True(t, handleKeyOnlyMessage.Extensions.OnlyHandleKey) + + for _, col := range insertEvent.Columns { if col.Flag.IsHandleKey() { - require.Contains(t, decoded.Data[0], col.Name) - require.Contains(t, decoded.SQLType, col.Name) - require.Contains(t, decoded.MySQLType, col.Name) + require.Contains(t, handleKeyOnlyMessage.Data[0], col.Name) + require.Contains(t, handleKeyOnlyMessage.SQLType, col.Name) + require.Contains(t, handleKeyOnlyMessage.MySQLType, col.Name) } else { - require.NotContains(t, decoded.Data[0], col.Name) - require.NotContains(t, decoded.SQLType, col.Name) - require.NotContains(t, decoded.MySQLType, col.Name) + require.NotContains(t, handleKeyOnlyMessage.Data[0], col.Name) + require.NotContains(t, handleKeyOnlyMessage.SQLType, col.Name) + require.NotContains(t, handleKeyOnlyMessage.MySQLType, col.Name) } } } @@ -215,8 +233,9 @@ func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { func TestNewCanalJSONMessageFromDDL(t *testing.T) { t.Parallel() - encoder, ok := newJSONRowEventEncoder(&common.Config{}).(*JSONRowEventEncoder) - require.True(t, ok) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build().(*JSONRowEventEncoder) message := encoder.newJSONMessageForDDL(testCaseDDL) require.NotNil(t, message) @@ -230,11 +249,10 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { require.Equal(t, testCaseDDL.Query, msg.Query) require.Equal(t, "CREATE", msg.EventType) - encoder, ok = newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: true, - }).(*JSONRowEventEncoder) - require.True(t, ok) + codecConfig.EnableTiDBExtension = true + builder = NewJSONRowEventEncoderBuilder(codecConfig) + encoder = builder.Build().(*JSONRowEventEncoder) message = encoder.newJSONMessageForDDL(testCaseDDL) require.NotNil(t, message) @@ -246,20 +264,18 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { } func TestBatching(t *testing.T) { - t.Parallel() - encoder := newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: false, - Terminator: "", - MaxMessageBytes: config.DefaultMaxMessageBytes, - }) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() require.NotNil(t, encoder) - updateCase := *testCaseUpdate + _, updateEvent, _ := newLargeEvent4Test(t) + updateCase := *updateEvent for i := 1; i <= 1000; i++ { ts := uint64(i) updateCase.CommitTs = ts err := encoder.AppendRowChangedEvent(context.Background(), "", &updateCase, nil) - require.Nil(t, err) + require.NoError(t, err) if i%100 == 0 { msgs := encoder.Build() @@ -271,7 +287,7 @@ func TestBatching(t *testing.T) { var msg JSONMessage err := json.Unmarshal(msgs[j].Value, &msg) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, "UPDATE", msg.EventType) } } @@ -282,15 +298,17 @@ func TestBatching(t *testing.T) { func TestEncodeCheckpointEvent(t *testing.T) { t.Parallel() + var watermark uint64 = 2333 for _, enable := range []bool{false, true} { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = enable - encoder := newJSONRowEventEncoder(codecConfig).(*JSONRowEventEncoder) - require.NotNil(t, encoder) + + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() msg, err := encoder.EncodeCheckpointEvent(watermark) - require.Nil(t, err) + require.NoError(t, err) if !enable { require.Nil(t, msg) @@ -307,12 +325,12 @@ func TestEncodeCheckpointEvent(t *testing.T) { require.NoError(t, err) ty, hasNext, err := decoder.HasNext() - require.Nil(t, err) + require.NoError(t, err) if enable { require.True(t, hasNext) require.Equal(t, model.MessageTypeResolved, ty) consumed, err := decoder.NextResolvedEvent() - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, watermark, consumed) } else { require.False(t, hasNext) @@ -320,7 +338,7 @@ func TestEncodeCheckpointEvent(t *testing.T) { } ty, hasNext, err = decoder.HasNext() - require.Nil(t, err) + require.NoError(t, err) require.False(t, hasNext) require.Equal(t, model.MessageTypeUnknown, ty) } @@ -328,14 +346,16 @@ func TestEncodeCheckpointEvent(t *testing.T) { func TestCheckpointEventValueMarshal(t *testing.T) { t.Parallel() + + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + + builder := NewJSONRowEventEncoderBuilder(codecConfig) + + encoder := builder.Build() var watermark uint64 = 1024 - encoder := &JSONRowEventEncoder{ - builder: newCanalEntryBuilder(), - config: &common.Config{EnableTiDBExtension: true}, - } - require.NotNil(t, encoder) msg, err := encoder.EncodeCheckpointEvent(watermark) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, msg) // Unmarshal from the data we have encoded. @@ -344,13 +364,13 @@ func TestCheckpointEventValueMarshal(t *testing.T) { &tidbExtension{}, } err = json.Unmarshal(msg.Value, &jsonMsg) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, watermark, jsonMsg.Extensions.WatermarkTs) // Hack the build time. // Otherwise, the timing will be inconsistent. jsonMsg.BuildTime = 1469579899 rawBytes, err := json.MarshalIndent(jsonMsg, "", " ") - require.Nil(t, err) + require.NoError(t, err) // No commit ts will be output. expectedJSON := `{ @@ -376,8 +396,9 @@ func TestCheckpointEventValueMarshal(t *testing.T) { func TestDDLEventWithExtensionValueMarshal(t *testing.T) { t.Parallel() + codecConfig := common.NewConfig(config.ProtocolCanalJSON) encoder := &JSONRowEventEncoder{ - builder: newCanalEntryBuilder(), + builder: newCanalEntryBuilder(codecConfig), config: &common.Config{EnableTiDBExtension: true}, } require.NotNil(t, encoder) @@ -391,7 +412,7 @@ func TestDDLEventWithExtensionValueMarshal(t *testing.T) { // Otherwise, the timing will be inconsistent. msg.BuildTime = 1469579899 rawBytes, err := json.MarshalIndent(msg, "", " ") - require.Nil(t, err) + require.NoError(t, err) // No watermark ts will be output. expectedJSON := `{ @@ -416,23 +437,31 @@ func TestDDLEventWithExtensionValueMarshal(t *testing.T) { } func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { - encoder := newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: true, - Terminator: "", - MaxMessageBytes: config.DefaultMaxMessageBytes, - }) - require.NotNil(t, encoder) + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() - count := 0 + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() + + count := 0 row := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ - Name: "col1", + Name: "a", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, } tests := []struct { @@ -478,7 +507,7 @@ func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { // Append the events. for _, test := range tests { err := encoder.AppendRowChangedEvent(context.Background(), "", test.row, test.callback) - require.Nil(t, err) + require.NoError(t, err) } require.Equal(t, 0, count, "nothing should be called") @@ -497,15 +526,26 @@ func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { } func TestMaxMessageBytes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + // the size of `testEvent` after being encoded by canal-json is 200 testEvent := &model.RowChangedEvent{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ - Name: "col1", + Name: "a", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, } ctx := context.Background() @@ -513,15 +553,65 @@ func TestMaxMessageBytes(t *testing.T) { // the test message length is smaller than max-message-bytes maxMessageBytes := 300 - cfg := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes) - encoder := NewJSONRowEventEncoderBuilder(cfg).Build() + codecConfig := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes) + + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() + err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) - require.Nil(t, err) + require.NoError(t, err) // the test message length is larger than max-message-bytes - cfg = cfg.WithMaxMessageBytes(100) - cfg.LargeMessageHandle = config.NewDefaultLargeMessageHandleConfig() - encoder = NewJSONRowEventEncoderBuilder(cfg).Build() + codecConfig = codecConfig.WithMaxMessageBytes(100) + builder = NewJSONRowEventEncoderBuilder(codecConfig) + + encoder = builder.Build() err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) - require.NotNil(t, err) + require.Error(t, err, cerror.ErrMessageTooLarge) +} + +func TestCanalJSONContentCompatibleE2E(t *testing.T) { + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true + codecConfig.ContentCompatible = true + + builder := NewJSONRowEventEncoderBuilder(codecConfig) + encoder := builder.Build() + + insertEvent, _, _ := newLargeEvent4Test(t) + err := encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {}) + require.NoError(t, err) + + message := encoder.Build()[0] + + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + + messageType, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, messageType, model.MessageTypeRow) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + require.Equal(t, decodedEvent.CommitTs, insertEvent.CommitTs) + require.Equal(t, decodedEvent.Table.Schema, insertEvent.Table.Schema) + require.Equal(t, decodedEvent.Table.Table, insertEvent.Table.Table) + + obtainedColumns := make(map[string]*model.Column, len(decodedEvent.Columns)) + for _, column := range decodedEvent.Columns { + obtainedColumns[column.Name] = column + } + + expectedValue := collectExpectedDecodedValue(testColumnsTable) + for _, actual := range insertEvent.Columns { + obtained, ok := obtainedColumns[actual.Name] + require.True(t, ok) + require.Equal(t, actual.Type, obtained.Type) + require.Equal(t, expectedValue[actual.Name], obtained.Value) + } } diff --git a/pkg/sink/codec/canal/canal_json_txn_event_encoder.go b/pkg/sink/codec/canal/canal_json_txn_event_encoder.go index bc47ba89abd..bd1cd5c5642 100644 --- a/pkg/sink/codec/canal/canal_json_txn_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_txn_event_encoder.go @@ -97,7 +97,7 @@ func (j *JSONTxnEventEncoder) Build() []*common.Message { // newJSONTxnEventEncoder creates a new JSONTxnEventEncoder func newJSONTxnEventEncoder(config *common.Config) codec.TxnEventEncoder { encoder := &JSONTxnEventEncoder{ - builder: newCanalEntryBuilder(), + builder: newCanalEntryBuilder(config), valueBuf: &bytes.Buffer{}, terminator: []byte(config.Terminator), diff --git a/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go index 71db664d285..042c25c3697 100644 --- a/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_txn_event_encoder_test.go @@ -11,12 +11,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build intest +// +build intest + package canal import ( "testing" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" @@ -34,20 +38,29 @@ func TestBuildCanalJSONTxnEventEncoder(t *testing.T) { } func TestCanalJSONTxnEventEncoderMaxMessageBytes(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() // the size of `testEvent` after being encoded by canal-json is 200 testEvent := &model.SingleTableTxn{ - Table: &model.TableName{Schema: "a", Table: "b"}, + Table: &model.TableName{Schema: "test", Table: "t"}, Rows: []*model.RowChangedEvent{ { - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, }, }, } @@ -67,7 +80,14 @@ func TestCanalJSONTxnEventEncoderMaxMessageBytes(t *testing.T) { } func TestCanalJSONAppendTxnEventEncoderWithCallback(t *testing.T) { - t.Parallel() + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a varchar(255) primary key)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() cfg := common.NewConfig(config.ProtocolCanalJSON) encoder := NewJSONTxnEventEncoderBuilder(cfg).Build() @@ -76,25 +96,29 @@ func TestCanalJSONAppendTxnEventEncoderWithCallback(t *testing.T) { count := 0 txn := &model.SingleTableTxn{ - Table: &model.TableName{Schema: "a", Table: "b"}, + Table: &model.TableName{Schema: "test", Table: "t"}, Rows: []*model.RowChangedEvent{ { - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 1, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ - Name: "col1", + Name: "a", Type: mysql.TypeVarchar, Value: []byte("aa"), }}, + ColInfos: colInfos, }, { - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, + CommitTs: 2, + Table: &model.TableName{Schema: "test", Table: "t"}, + TableInfo: tableInfo, Columns: []*model.Column{{ - Name: "col1", + Name: "a", Type: mysql.TypeVarchar, Value: []byte("bb"), }}, + ColInfos: colInfos, }, }, } diff --git a/pkg/sink/codec/canal/canal_test_util.go b/pkg/sink/codec/canal/canal_test_util.go index c5541b8b33a..7f41c3eaa54 100644 --- a/pkg/sink/codec/canal/canal_test_util.go +++ b/pkg/sink/codec/canal/canal_test_util.go @@ -11,19 +11,22 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build intest +// +build intest + package canal import ( + "testing" + mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/sink/codec/internal" ) type testColumnTuple struct { - column *model.Column - expectedMySQLType string - expectedJavaSQLType internal.JavaSQLType + column *model.Column // expectedEncodedValue is expected by encoding expectedEncodedValue string @@ -35,185 +38,185 @@ type testColumnTuple struct { var ( testColumnsTable = []*testColumnTuple{ { - &model.Column{Name: "tinyint", Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Type: mysql.TypeTiny, Value: int64(127)}, - "tinyint", internal.JavaSQLTypeTINYINT, "127", "127", + &model.Column{Name: "t", Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Type: mysql.TypeTiny, Value: int64(127)}, + "127", "127", }, { &model.Column{ - Name: "tinyint unsigned", Type: mysql.TypeTiny, Value: uint64(127), + Name: "tu1", Type: mysql.TypeTiny, Value: uint64(127), Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "127", "127", + "127", "127", }, { &model.Column{ - Name: "tinyint unsigned 2", Type: mysql.TypeTiny, Value: uint64(128), + Name: "tu2", Type: mysql.TypeTiny, Value: uint64(128), Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeSMALLINT, "128", "128", + "128", "128", }, { &model.Column{ - Name: "tinyint unsigned 3", Type: mysql.TypeTiny, Value: "0", + Name: "tu3", Type: mysql.TypeTiny, Value: "0", Flag: model.UnsignedFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "tinyint unsigned 4", Type: mysql.TypeTiny, Value: nil, + Name: "tu4", Type: mysql.TypeTiny, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "tinyint unsigned", internal.JavaSQLTypeTINYINT, "", nil, + "", nil, }, { - &model.Column{Name: "smallint", Type: mysql.TypeShort, Value: int64(32767)}, - "smallint", internal.JavaSQLTypeSMALLINT, "32767", "32767", + &model.Column{Name: "s", Type: mysql.TypeShort, Value: int64(32767)}, + "32767", "32767", }, { &model.Column{ - Name: "smallint unsigned", Type: mysql.TypeShort, Value: uint64(32767), + Name: "su1", Type: mysql.TypeShort, Value: uint64(32767), Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "32767", "32767", + "32767", "32767", }, { &model.Column{ - Name: "smallint unsigned 2", Type: mysql.TypeShort, Value: uint64(32768), + Name: "su2", Type: mysql.TypeShort, Value: uint64(32768), Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeINTEGER, "32768", "32768", + "32768", "32768", }, { &model.Column{ - Name: "smallint unsigned 3", Type: mysql.TypeShort, Value: "0", + Name: "su3", Type: mysql.TypeShort, Value: "0", Flag: model.UnsignedFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "smallint unsigned 4", Type: mysql.TypeShort, Value: nil, + Name: "su4", Type: mysql.TypeShort, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "smallint unsigned", internal.JavaSQLTypeSMALLINT, "", nil, + "", nil, }, { - &model.Column{Name: "mediumint", Type: mysql.TypeInt24, Value: int64(8388607)}, - "mediumint", internal.JavaSQLTypeINTEGER, "8388607", "8388607", + &model.Column{Name: "m", Type: mysql.TypeInt24, Value: int64(8388607)}, + "8388607", "8388607", }, { &model.Column{ - Name: "mediumint unsigned", Type: mysql.TypeInt24, Value: uint64(8388607), + Name: "mu1", Type: mysql.TypeInt24, Value: uint64(8388607), Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "8388607", "8388607", + "8388607", "8388607", }, { &model.Column{ - Name: "mediumint unsigned 2", Type: mysql.TypeInt24, Value: uint64(8388608), + Name: "mu2", Type: mysql.TypeInt24, Value: uint64(8388608), Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "8388608", "8388608", + "8388608", "8388608", }, { &model.Column{ - Name: "mediumint unsigned 3", Type: mysql.TypeInt24, Value: "0", + Name: "mu3", Type: mysql.TypeInt24, Value: "0", Flag: model.UnsignedFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "0", "0", + "0", "0", }, { &model.Column{ - Name: "mediumint unsigned 4", Type: mysql.TypeInt24, Value: nil, + Name: "mu4", Type: mysql.TypeInt24, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "mediumint unsigned", internal.JavaSQLTypeINTEGER, "", nil, + "", nil, }, { - &model.Column{Name: "int", Type: mysql.TypeLong, Value: int64(2147483647)}, - "int", internal.JavaSQLTypeINTEGER, "2147483647", "2147483647", + &model.Column{Name: "i", Type: mysql.TypeLong, Value: int64(2147483647)}, + "2147483647", "2147483647", }, { &model.Column{ - Name: "int unsigned", Type: mysql.TypeLong, Value: uint64(2147483647), + Name: "iu1", Type: mysql.TypeLong, Value: uint64(2147483647), Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "2147483647", "2147483647", + "2147483647", "2147483647", }, { &model.Column{ - Name: "int unsigned 2", Type: mysql.TypeLong, Value: uint64(2147483648), + Name: "iu2", Type: mysql.TypeLong, Value: uint64(2147483648), Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeBIGINT, "2147483648", "2147483648", + "2147483648", "2147483648", }, { &model.Column{ - Name: "int unsigned 3", Type: mysql.TypeLong, Value: "0", + Name: "iu3", Type: mysql.TypeLong, Value: "0", Flag: model.UnsignedFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "0", "0", + "0", "0", }, { &model.Column{ - Name: "int unsigned 4", Type: mysql.TypeLong, Value: nil, + Name: "iu4", Type: mysql.TypeLong, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "int unsigned", internal.JavaSQLTypeINTEGER, "", nil, + "", nil, }, { - &model.Column{Name: "bigint", Type: mysql.TypeLonglong, Value: int64(9223372036854775807)}, - "bigint", internal.JavaSQLTypeBIGINT, "9223372036854775807", "9223372036854775807", + &model.Column{Name: "bi", Type: mysql.TypeLonglong, Value: int64(9223372036854775807)}, + "9223372036854775807", "9223372036854775807", }, { &model.Column{ - Name: "bigint unsigned", Type: mysql.TypeLonglong, Value: uint64(9223372036854775807), + Name: "biu1", Type: mysql.TypeLonglong, Value: uint64(9223372036854775807), Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "9223372036854775807", "9223372036854775807", + "9223372036854775807", "9223372036854775807", }, { &model.Column{ - Name: "bigint unsigned 2", Type: mysql.TypeLonglong, Value: uint64(9223372036854775808), + Name: "biu2", Type: mysql.TypeLonglong, Value: uint64(9223372036854775808), Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeDECIMAL, "9223372036854775808", "9223372036854775808", + "9223372036854775808", "9223372036854775808", }, { &model.Column{ - Name: "bigint unsigned 3", Type: mysql.TypeLonglong, Value: "0", + Name: "biu3", Type: mysql.TypeLonglong, Value: "0", Flag: model.UnsignedFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "0", "0", + "0", "0", }, { &model.Column{ - Name: "bigint unsigned 4", Type: mysql.TypeLonglong, Value: nil, + Name: "biu4", Type: mysql.TypeLonglong, Value: nil, Flag: model.BinaryFlag | model.UnsignedFlag | model.NullableFlag, }, - "bigint unsigned", internal.JavaSQLTypeBIGINT, "", nil, + "", nil, }, { - &model.Column{Name: "float", Type: mysql.TypeFloat, Value: 3.14}, - "float", internal.JavaSQLTypeREAL, "3.14", "3.14", + &model.Column{Name: "floatT", Type: mysql.TypeFloat, Value: 3.14}, + "3.14", "3.14", }, { - &model.Column{Name: "double", Type: mysql.TypeDouble, Value: 2.71}, - "double", internal.JavaSQLTypeDOUBLE, "2.71", "2.71", + &model.Column{Name: "doubleT", Type: mysql.TypeDouble, Value: 2.71}, + "2.71", "2.71", }, { - &model.Column{Name: "decimal", Type: mysql.TypeNewDecimal, Value: "2333"}, - "decimal", internal.JavaSQLTypeDECIMAL, "2333", "2333", + &model.Column{Name: "decimalT", Type: mysql.TypeNewDecimal, Value: "2333"}, + "2333", "2333", }, { @@ -221,264 +224,264 @@ var ( Name: "float unsigned", Type: mysql.TypeFloat, Value: 3.14, Flag: model.UnsignedFlag, }, - "float unsigned", internal.JavaSQLTypeREAL, "3.14", "3.14", + "3.14", "3.14", }, { &model.Column{ Name: "double unsigned", Type: mysql.TypeDouble, Value: 2.71, Flag: model.UnsignedFlag, }, - "double unsigned", internal.JavaSQLTypeDOUBLE, "2.71", "2.71", + "2.71", "2.71", }, { &model.Column{ Name: "decimal unsigned", Type: mysql.TypeNewDecimal, Value: "2333", Flag: model.UnsignedFlag, }, - "decimal unsigned", internal.JavaSQLTypeDECIMAL, "2333", "2333", + "2333", "2333", }, // for column value type in `[]uint8` and have `BinaryFlag`, expectedEncodedValue is dummy. { - &model.Column{Name: "varchar", Type: mysql.TypeVarchar, Value: []uint8("测试Varchar")}, - "varchar", internal.JavaSQLTypeVARCHAR, "测试Varchar", "测试Varchar", + &model.Column{Name: "varcharT", Type: mysql.TypeVarchar, Value: []uint8("测试Varchar")}, + "测试Varchar", "测试Varchar", }, { - &model.Column{Name: "char", Type: mysql.TypeString, Value: []uint8("测试String")}, - "char", internal.JavaSQLTypeCHAR, "测试String", "测试String", + &model.Column{Name: "charT", Type: mysql.TypeString, Value: []uint8("测试String")}, + "测试String", "测试String", }, { &model.Column{ - Name: "binary", Type: mysql.TypeString, Value: []uint8("测试Binary"), + Name: "binaryT", Type: mysql.TypeString, Value: []uint8("测试Binary"), Flag: model.BinaryFlag, }, - "binary", internal.JavaSQLTypeBLOB, "测试Binary", "测试Binary", + "测试Binary", "测试Binary", }, { &model.Column{ - Name: "varbinary", Type: mysql.TypeVarchar, Value: []uint8("测试varbinary"), + Name: "varbinaryT", Type: mysql.TypeVarchar, Value: []uint8("测试varbinary"), Flag: model.BinaryFlag, }, - "varbinary", internal.JavaSQLTypeBLOB, "测试varbinary", "测试varbinary", + "测试varbinary", "测试varbinary", }, { - &model.Column{Name: "tinytext", Type: mysql.TypeTinyBlob, Value: []uint8("测试Tinytext")}, - "tinytext", internal.JavaSQLTypeCLOB, "测试Tinytext", "测试Tinytext", + &model.Column{Name: "tinytextT", Type: mysql.TypeTinyBlob, Value: []uint8("测试Tinytext")}, + "测试Tinytext", "测试Tinytext", }, + { - &model.Column{Name: "text", Type: mysql.TypeBlob, Value: []uint8("测试text")}, - "text", internal.JavaSQLTypeCLOB, "测试text", "测试text", + &model.Column{Name: "textT", Type: mysql.TypeBlob, Value: []uint8("测试text")}, + "测试text", "测试text", }, { - &model.Column{ - Name: "mediumtext", Type: mysql.TypeMediumBlob, - Value: []uint8("测试mediumtext"), - }, - "mediumtext", internal.JavaSQLTypeCLOB, "测试mediumtext", "测试mediumtext", + &model.Column{Name: "mediumtextT", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumtext")}, + "测试mediumtext", "测试mediumtext", }, { - &model.Column{Name: "longtext", Type: mysql.TypeLongBlob, Value: []uint8("测试longtext")}, - "longtext", internal.JavaSQLTypeCLOB, "测试longtext", "测试longtext", + &model.Column{Name: "longtextT", Type: mysql.TypeLongBlob, Value: []uint8("测试longtext")}, + "测试longtext", "测试longtext", }, { &model.Column{ - Name: "tinyblob", Type: mysql.TypeTinyBlob, Value: []uint8("测试tinyblob"), + Name: "tinyblobT", Type: mysql.TypeTinyBlob, Value: []uint8("测试tinyblob"), Flag: model.BinaryFlag, }, - "tinyblob", internal.JavaSQLTypeBLOB, "测试tinyblob", "测试tinyblob", + "测试tinyblob", "测试tinyblob", }, { &model.Column{ - Name: "blob", Type: mysql.TypeBlob, Value: []uint8("测试blob"), + Name: "blobT", Type: mysql.TypeBlob, Value: []uint8("测试blob"), Flag: model.BinaryFlag, }, - "blob", internal.JavaSQLTypeBLOB, "测试blob", "测试blob", + "测试blob", "测试blob", }, { &model.Column{ - Name: "mediumblob", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumblob"), + Name: "mediumblobT", Type: mysql.TypeMediumBlob, Value: []uint8("测试mediumblob"), Flag: model.BinaryFlag, }, - "mediumblob", internal.JavaSQLTypeBLOB, "测试mediumblob", "测试mediumblob", + "测试mediumblob", "测试mediumblob", }, { &model.Column{ - Name: "longblob", Type: mysql.TypeLongBlob, Value: []uint8("测试longblob"), + Name: "longblobT", Type: mysql.TypeLongBlob, Value: []uint8("测试longblob"), Flag: model.BinaryFlag, }, - "longblob", internal.JavaSQLTypeBLOB, "测试longblob", "测试longblob", + "测试longblob", "测试longblob", }, { - &model.Column{Name: "date", Type: mysql.TypeDate, Value: "2020-02-20"}, - "date", internal.JavaSQLTypeDATE, "2020-02-20", "2020-02-20", + &model.Column{Name: "dateT", Type: mysql.TypeDate, Value: "2020-02-20"}, + "2020-02-20", "2020-02-20", }, { - &model.Column{Name: "datetime", Type: mysql.TypeDatetime, Value: "2020-02-20 02:20:20"}, - "datetime", internal.JavaSQLTypeTIMESTAMP, "2020-02-20 02:20:20", "2020-02-20 02:20:20", + &model.Column{Name: "datetimeT", Type: mysql.TypeDatetime, Value: "2020-02-20 02:20:20"}, + "2020-02-20 02:20:20", "2020-02-20 02:20:20", }, { - &model.Column{Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2020-02-20 10:20:20"}, - "timestamp", internal.JavaSQLTypeTIMESTAMP, "2020-02-20 10:20:20", "2020-02-20 10:20:20", + &model.Column{Name: "timestampT", Type: mysql.TypeTimestamp, Value: "2020-02-20 10:20:20"}, + "2020-02-20 10:20:20", "2020-02-20 10:20:20", }, { - &model.Column{Name: "time", Type: mysql.TypeDuration, Value: "02:20:20"}, - "time", internal.JavaSQLTypeTIME, "02:20:20", "02:20:20", + &model.Column{Name: "timeT", Type: mysql.TypeDuration, Value: "02:20:20"}, + "02:20:20", "02:20:20", }, { - &model.Column{Name: "year", Type: mysql.TypeYear, Value: "2020", Flag: model.UnsignedFlag}, - "year", internal.JavaSQLTypeVARCHAR, "2020", "2020", + &model.Column{Name: "yearT", Type: mysql.TypeYear, Value: "2020", Flag: model.UnsignedFlag}, + "2020", "2020", }, { - &model.Column{Name: "enum", Type: mysql.TypeEnum, Value: uint64(1)}, - "enum", internal.JavaSQLTypeINTEGER, "1", "1", + &model.Column{Name: "enumT", Type: mysql.TypeEnum, Value: uint64(1)}, + "1", "1", }, { - &model.Column{Name: "set", Type: mysql.TypeSet, Value: uint64(3)}, - "set", internal.JavaSQLTypeBIT, "3", uint64(3), + &model.Column{Name: "setT", Type: mysql.TypeSet, Value: uint64(2)}, + "2", uint64(2), }, { &model.Column{ - Name: "bit", Type: mysql.TypeBit, Value: uint64(65), + Name: "bitT", Type: mysql.TypeBit, Value: uint64(65), Flag: model.UnsignedFlag | model.BinaryFlag, }, - "bit", internal.JavaSQLTypeBIT, "65", uint64(65), + "65", uint64(65), }, { &model.Column{ - Name: "json", Type: mysql.TypeJSON, Value: "{\"key1\": \"value1\"}", + Name: "jsonT", Type: mysql.TypeJSON, Value: "{\"key1\": \"value1\"}", Flag: model.BinaryFlag, }, - "json", internal.JavaSQLTypeVARCHAR, "{\"key1\": \"value1\"}", "{\"key1\": \"value1\"}", + "{\"key1\": \"value1\"}", "{\"key1\": \"value1\"}", }, } - defaultCanalBatchTester = &struct { - rowCases [][]*model.RowChangedEvent - ddlCases [][]*model.DDLEvent - }{ - rowCases: [][]*model.RowChangedEvent{ - {{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{ - Name: "col1", - Type: mysql.TypeVarchar, - Value: []byte("aa"), - }}, - }}, - { - { - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{ - Name: "col1", - Type: mysql.TypeVarchar, - Value: []byte("aa"), - }}, - }, - { - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, - }, - }, - }, - ddlCases: [][]*model.DDLEvent{ - {{ - CommitTs: 1, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table a", - Type: 1, - }}, - { - { - CommitTs: 2, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table b", - Type: 3, - }, - { - CommitTs: 3, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "a", Table: "b", - }, - }, - Query: "create table c", - Type: 3, - }, + testCaseDDL = &model.DDLEvent{ + CommitTs: 417318403368288260, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "cdc", Table: "person", }, }, + Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", + Type: mm.ActionCreateTable, + } +) + +func collectAllColumns(groups []*testColumnTuple) []*model.Column { + columns := make([]*model.Column, 0, len(groups)) + for _, item := range groups { + columns = append(columns, item.column) + } + return columns +} + +func collectExpectedDecodedValue(columns []*testColumnTuple) map[string]interface{} { + result := make(map[string]interface{}, len(columns)) + for _, item := range columns { + result[item.column.Name] = item.expectedDecodedValue } + return result +} + +func newLargeEvent4Test(t *testing.T) (*model.RowChangedEvent, *model.RowChangedEvent, *model.RowChangedEvent) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t( + t tinyint primary key, + tu1 tinyint unsigned, + tu2 tinyint unsigned, + tu3 tinyint unsigned, + tu4 tinyint unsigned, + s smallint, + su1 smallint unsigned, + su2 smallint unsigned, + su3 smallint unsigned, + su4 smallint unsigned, + m mediumint, + mu1 mediumint unsigned, + mu2 mediumint unsigned, + mu3 mediumint unsigned, + mu4 mediumint unsigned, + i int, + iu1 int unsigned, + iu2 int unsigned, + iu3 int unsigned, + iu4 int unsigned, + bi bigint, + biu1 bigint unsigned, + biu2 bigint unsigned, + biu3 bigint unsigned, + biu4 bigint unsigned, + floatT float, + doubleT double, + decimalT decimal, + floatTu float unsigned, + doubleTu double unsigned, + decimalTu decimal unsigned, + varcharT varchar(255), + charT char, + binaryT binary, + varbinaryT varbinary(255), + tinytextT tinytext, + textT text, + mediumtextT mediumtext, + longtextT longtext, + tinyblobT tinyblob, + blobT blob, + mediumblobT mediumblob, + longblobT longblob, + dateT date, + datetimeT datetime, + timestampT timestamp, + timeT time, + yearT year, + enumT enum('a', 'b', 'c'), + setT set('a', 'b', 'c'), + bitT bit(4), + jsonT json)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfo := tableInfo.GetRowColInfos() - testColumns = collectAllColumns(testColumnsTable) + testColumns := collectAllColumns(testColumnsTable) - testCaseInsert = &model.RowChangedEvent{ + insert := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ - Schema: "cdc", - Table: "person", + Schema: "test", + Table: "t", }, + TableInfo: tableInfo, Columns: testColumns, PreColumns: nil, + ColInfos: colInfo, } - testCaseUpdate = &model.RowChangedEvent{ + update := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: testColumns, PreColumns: testColumns, + ColInfos: colInfo, } - testCaseDelete = &model.RowChangedEvent{ + deleteE := &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ Schema: "cdc", Table: "person", }, + TableInfo: tableInfo, Columns: nil, PreColumns: testColumns, + ColInfos: colInfo, } - - testCaseDDL = &model.DDLEvent{ - CommitTs: 417318403368288260, - TableInfo: &model.TableInfo{ - TableName: model.TableName{ - Schema: "cdc", Table: "person", - }, - }, - Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", - Type: mm.ActionCreateTable, - } -) - -func collectAllColumns(groups []*testColumnTuple) []*model.Column { - result := make([]*model.Column, 0, len(groups)) - for _, item := range groups { - result = append(result, item.column) - } - return result -} - -func collectExpectedDecodedValue(columns []*testColumnTuple) map[string]interface{} { - result := make(map[string]interface{}, len(columns)) - for _, item := range columns { - result[item.column.Name] = item.expectedDecodedValue - } - return result + return insert, update, deleteE } diff --git a/pkg/sink/codec/canal/type_test.go b/pkg/sink/codec/canal/type_test.go new file mode 100644 index 00000000000..864a6faa84a --- /dev/null +++ b/pkg/sink/codec/canal/type_test.go @@ -0,0 +1,960 @@ +// Copyright 2023 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 canal + +import ( + "testing" + + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sink/codec/internal" + "github.com/pingcap/tiflow/pkg/sink/codec/utils" + "github.com/stretchr/testify/require" +) + +func TestGetMySQLType4IntTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1 ( + a int primary key, + b tinyint, + c smallint, + d mediumint, + e bigint)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "int", mysqlType) + // mysql type with the default type length + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "int(11)", mysqlType) + + flag := tableInfo.ColumnsFlag[colInfos[0].ID] + javaType, err := getJavaSQLType(int64(2147483647), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(4)", mysqlType) + + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + javaType, err = getJavaSQLType(int64(127), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(6)", mysqlType) + + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + javaType, err = getJavaSQLType(int64(32767), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(9)", mysqlType) + javaType, err = getJavaSQLType(int64(8388607), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20)", mysqlType) + javaType, err = getJavaSQLType(int64(9223372036854775807), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + + sql = `create table test.t2 ( + a int unsigned primary key, + b tinyint unsigned, + c smallint unsigned, + d mediumint unsigned, + e bigint unsigned)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[0].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned", mysqlType) + // mysql type with the default type length + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned", mysqlType) + + javaType, err = getJavaSQLType(uint64(2147483647), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(uint64(2147483648), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned", mysqlType) + + javaType, err = getJavaSQLType(uint64(127), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + javaType, err = getJavaSQLType(uint64(128), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeTINYINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(32767), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType(uint64(32768), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeSMALLINT, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(8388607), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(uint64(8388608), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20) unsigned", mysqlType) + javaType, err = getJavaSQLType(uint64(9223372036854775807), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType(uint64(9223372036854775808), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) + javaType, err = getJavaSQLType("0", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIGINT, javaType) + + sql = `create table test.t3 ( + a int(10) primary key, + b tinyint(3) , + c smallint(5), + d mediumint(8), + e bigint(19))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "int", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(19)", mysqlType) + + sql = `create table test.t4 ( + a int(10) unsigned primary key, + b tinyint(3) unsigned, + c smallint(5) unsigned, + d mediumint(8) unsigned, + e bigint(19) unsigned)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(19) unsigned", mysqlType) + + sql = `create table test.t5 ( + a int zerofill primary key, + b tinyint zerofill, + c smallint unsigned zerofill, + d mediumint zerofill, + e bigint zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[0].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "int unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "int(10) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(3) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "smallint unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "smallint(5) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumint unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumint(8) unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bigint unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bigint(20) unsigned zerofill", mysqlType) + + sql = `create table test.t6( + a int primary key, + b bit, + c bit(3), + d bool)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bit", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bit(1)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "bit", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "bit(3)", mysqlType) + javaType, err = getJavaSQLType(uint64(65), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIT, javaType) + + // bool is identical to tinyint in the TiDB. + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyint", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyint(1)", mysqlType) +} + +func TestGetMySQLType4FloatType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1( + a int primary key, + b float, + c double)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "float", mysqlType) + javaType, err := getJavaSQLType(3.14, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeREAL, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "double", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "double", mysqlType) + javaType, err = getJavaSQLType(2.71, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDOUBLE, javaType) + + sql = `create table test.t2(a int primary key, b float(10, 3), c float(10))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "float(10,3)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "float", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "float", mysqlType) + + sql = `create table test.t3(a int primary key, b double(20, 3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "double", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "double(20,3)", mysqlType) + + sql = `create table test.t4( + a int primary key, + b float unsigned, + c double unsigned, + d float zerofill, + e double zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "float unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "float unsigned", mysqlType) + javaType, err = getJavaSQLType(3.14, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeREAL, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "double unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "double unsigned", mysqlType) + javaType, err = getJavaSQLType(2.71, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDOUBLE, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "float unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "float unsigned zerofill", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "double unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "double unsigned zerofill", mysqlType) +} + +func TestGetMySQLType4Decimal(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b decimal, c numeric)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0)", mysqlType) + + sql = `create table test.t2(a int primary key, b decimal(5), c decimal(5, 2))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(5,0)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(5,2)", mysqlType) + javaType, err := getJavaSQLType("2333", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) + + sql = `create table test.t3(a int primary key, b decimal unsigned, c decimal zerofill)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal unsigned", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0) unsigned", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "decimal unsigned zerofill", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "decimal(10,0) unsigned zerofill", mysqlType) + javaType, err = getJavaSQLType("2333", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeDECIMAL, javaType) +} + +func TestGetMySQLType4TimeTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b time, c time(3))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "time", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "time", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "time", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "time(3)", mysqlType) + javaType, err := getJavaSQLType("02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIME) + + sql = `create table test.t2(a int primary key, b datetime, c datetime(3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "datetime", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "datetime", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "datetime", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "datetime(3)", mysqlType) + javaType, err = getJavaSQLType("2020-02-20 02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIMESTAMP) + + sql = `create table test.t3(a int primary key, b timestamp, c timestamp(3))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "timestamp", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "timestamp", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "timestamp", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "timestamp(3)", mysqlType) + javaType, err = getJavaSQLType("2020-02-20 02:20:20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeTIMESTAMP) + + sql = `create table test.t4(a int primary key, b date)` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "date", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "date", mysqlType) + javaType, err = getJavaSQLType("2020-02-20", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeDATE) + + sql = `create table test.t5(a int primary key, b year, c year(4))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "year", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "year(4)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "year", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "year(4)", mysqlType) + javaType, err = getJavaSQLType("2020", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeVARCHAR) +} + +func TestGetMySQLType4Char(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b char, c char(123))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "char", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "char(1)", mysqlType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "char", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "char(123)", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试char"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCHAR) + + sql = `create table test.t1(a int primary key, b varchar(123))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "varchar", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "varchar(123)", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试varchar"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeVARCHAR) +} + +func TestGetMySQLType4TextTypes(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b text, c tinytext, d mediumtext, e longtext)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "text", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "text", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试text"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinytext", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinytext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试tinytext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumtext", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumtext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试mediumtext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "longtext", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "longtext", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试longtext"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeCLOB) +} + +func TestGetMySQLType4BinaryType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b binary, c binary(10))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "binary", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "binary(1)", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试binary"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "binary", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "binary(10)", mysqlType) + + sql = `create table test.t2(a int primary key, b varbinary(23))` + job = helper.DDL2Job(sql) + tableInfo = model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos = tableInfo.GetRowColInfos() + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "varbinary", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "varbinary(23)", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试varbinary"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBLOB, javaType) +} + +func TestGetMySQLType4BlobType(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t1(a int primary key, b blob, c tinyblob, d mediumblob, e longblob)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "blob", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "blob", mysqlType) + javaType, err := getJavaSQLType([]uint8("测试blob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "tinyblob", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "tinyblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试tinyblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[3].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[3].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "mediumblob", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "mediumblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试mediumblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[4].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[4].ID] + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "longblob", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "longblob", mysqlType) + javaType, err = getJavaSQLType([]uint8("测试longblob"), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, javaType, internal.JavaSQLTypeBLOB) +} + +func TestGetMySQLType4EnumAndSet(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b enum('a', 'b', 'c'), c set('a', 'b', 'c'))` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "enum", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "enum('a','b','c')", mysqlType) + + javaType, err := getJavaSQLType(uint64(1), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeINTEGER, javaType) + + columnInfo, ok = tableInfo.GetColumnInfo(colInfos[2].ID) + require.True(t, ok) + flag = tableInfo.ColumnsFlag[colInfos[2].ID] + + mysqlType = utils.GetMySQLType(columnInfo, false) + require.Equal(t, "set", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "set('a','b','c')", mysqlType) + + javaType, err = getJavaSQLType(uint64(2), columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeBIT, javaType) +} + +func TestGetMySQLType4JSON(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + sql := `create table test.t(a int primary key, b json)` + job := helper.DDL2Job(sql) + tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) + + _, _, colInfos := tableInfo.GetRowColInfos() + + columnInfo, ok := tableInfo.GetColumnInfo(colInfos[1].ID) + require.True(t, ok) + flag := tableInfo.ColumnsFlag[colInfos[1].ID] + mysqlType := utils.GetMySQLType(columnInfo, false) + require.Equal(t, "json", mysqlType) + mysqlType = utils.GetMySQLType(columnInfo, true) + require.Equal(t, "json", mysqlType) + + javaType, err := getJavaSQLType("{\"key1\": \"value1\"}", columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeVARCHAR, javaType) + + javaType, err = getJavaSQLType(nil, columnInfo.FieldType.GetType(), flag) + require.NoError(t, err) + require.Equal(t, internal.JavaSQLTypeVARCHAR, javaType) +} diff --git a/pkg/sink/codec/common/config.go b/pkg/sink/codec/common/config.go index 5c9e59ab27c..1440fc28510 100644 --- a/pkg/sink/codec/common/config.go +++ b/pkg/sink/codec/common/config.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -48,6 +49,9 @@ type Config struct { AvroDecimalHandlingMode string AvroBigintUnsignedHandlingMode string + // canal-json only + ContentCompatible bool + // for sinking to cloud storage Delimiter string Quote string @@ -111,6 +115,7 @@ type urlConfig struct { AvroSchemaRegistry string `form:"schema-registry"` OnlyOutputUpdatedColumns *bool `form:"only-output-updated-columns"` + ContentCompatible *bool `form:"content-compatible"` } // Apply fill the Config @@ -185,6 +190,12 @@ func (c *Config) Apply(sinkURI *url.URL, replicaConfig *config.ReplicaConfig) er } c.DeleteOnlyHandleKeyColumns = !replicaConfig.EnableOldValue + if c.Protocol == config.ProtocolCanalJSON { + c.ContentCompatible = util.GetOrZero(urlParameter.ContentCompatible) + if c.ContentCompatible { + c.OnlyOutputUpdatedColumns = true + } + } return nil } @@ -197,6 +208,10 @@ func mergeConfig( if replicaConfig.Sink != nil { dest.AvroSchemaRegistry = replicaConfig.Sink.SchemaRegistry dest.OnlyOutputUpdatedColumns = replicaConfig.Sink.OnlyOutputUpdatedColumns + dest.ContentCompatible = replicaConfig.Sink.ContentCompatible + if util.GetOrZero(dest.ContentCompatible) { + dest.OnlyOutputUpdatedColumns = util.AddressOf(true) + } if replicaConfig.Sink.KafkaConfig != nil { dest.MaxMessageBytes = replicaConfig.Sink.KafkaConfig.MaxMessageBytes if replicaConfig.Sink.KafkaConfig.CodecConfig != nil { diff --git a/pkg/sink/codec/common/config_test.go b/pkg/sink/codec/common/config_test.go index 85d1d46e8d8..19914b30b4a 100644 --- a/pkg/sink/codec/common/config_test.go +++ b/pkg/sink/codec/common/config_test.go @@ -286,6 +286,7 @@ func TestMergeConfig(t *testing.T) { require.Equal(t, true, c.EnableTiDBExtension) require.Equal(t, "abc", c.AvroSchemaRegistry) require.True(t, c.OnlyOutputUpdatedColumns) + require.False(t, c.ContentCompatible) require.Equal(t, "ab", c.AvroBigintUnsignedHandlingMode) require.Equal(t, "cd", c.AvroDecimalHandlingMode) require.Equal(t, 123, c.MaxMessageBytes) @@ -313,6 +314,7 @@ func TestMergeConfig(t *testing.T) { require.Equal(t, true, c.EnableTiDBExtension) require.Equal(t, "abc", c.AvroSchemaRegistry) require.True(t, c.OnlyOutputUpdatedColumns) + require.False(t, c.ContentCompatible) require.Equal(t, "ab", c.AvroBigintUnsignedHandlingMode) require.Equal(t, "cd", c.AvroDecimalHandlingMode) require.Equal(t, 123, c.MaxMessageBytes) @@ -344,6 +346,7 @@ func TestMergeConfig(t *testing.T) { require.Equal(t, true, c.EnableTiDBExtension) require.Equal(t, "abc", c.AvroSchemaRegistry) require.True(t, c.OnlyOutputUpdatedColumns) + require.False(t, c.ContentCompatible) require.Equal(t, "ab", c.AvroBigintUnsignedHandlingMode) require.Equal(t, "cd", c.AvroDecimalHandlingMode) require.Equal(t, 123, c.MaxMessageBytes) @@ -397,6 +400,8 @@ func TestCanalJSONHandleKeyOnly(t *testing.T) { require.NoError(t, err) require.True(t, codecConfig.LargeMessageHandle.HandleKeyOnly()) + require.False(t, codecConfig.ContentCompatible) + require.False(t, codecConfig.OnlyOutputUpdatedColumns) // force-replicate is set to true, should return error replicaConfig.ForceReplicate = true @@ -451,3 +456,15 @@ func TestOpenProtocolHandleKeyOnly(t *testing.T) { err = codecConfig.Validate() require.NoError(t, err) } + +func TestApplyConfig4CanalJSON(t *testing.T) { + uri := "kafka://127.0.0.1:9092/abc?protocol=canal-json&content-compatible=true" + sinkURI, err := url.Parse(uri) + require.NoError(t, err) + + codecConfig := NewConfig(config.ProtocolCanalJSON) + err = codecConfig.Apply(sinkURI, config.GetDefaultReplicaConfig()) + require.NoError(t, err) + require.True(t, codecConfig.ContentCompatible) + require.True(t, codecConfig.OnlyOutputUpdatedColumns) +} diff --git a/pkg/sink/codec/common/field_types.go b/pkg/sink/codec/common/field_types.go new file mode 100644 index 00000000000..b76a9a74eda --- /dev/null +++ b/pkg/sink/codec/common/field_types.go @@ -0,0 +1,52 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" +) + +// SetBinChsClnFlag set the binary charset flag. +func SetBinChsClnFlag(ft *types.FieldType) *types.FieldType { + types.SetBinChsClnFlag(ft) + return ft +} + +// SetUnsigned set the unsigned flag. +func SetUnsigned(ft *types.FieldType) *types.FieldType { + ft.SetFlag(uint(model.UnsignedFlag)) + return ft +} + +// SetFlag set the specific flag to the ft +func SetFlag(ft *types.FieldType, flag uint) *types.FieldType { + ft.SetFlag(flag) + return ft +} + +// SetElems set the elems to the ft +func SetElems(ft *types.FieldType, elems []string) *types.FieldType { + ft.SetElems(elems) + return ft +} + +// NewTextFieldType create a new text field type. +func NewTextFieldType(tp byte) *types.FieldType { + ft := types.NewFieldType(tp) + ft.SetCollate(mysql.DefaultCollationName) + ft.SetCharset(mysql.DefaultCharset) + return ft +} diff --git a/pkg/sink/codec/common/mysql_types.go b/pkg/sink/codec/common/mysql_types.go new file mode 100644 index 00000000000..6aedd631c4f --- /dev/null +++ b/pkg/sink/codec/common/mysql_types.go @@ -0,0 +1,65 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "strings" + + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" +) + +// when encoding the canal format, for unsigned mysql type, add `unsigned` keyword. +// it should have the form `t unsigned`, such as `int unsigned` +func withUnsigned4MySQLType(mysqlType string, unsigned bool) string { + if unsigned && mysqlType != "bit" && mysqlType != "year" { + return mysqlType + " unsigned" + } + return mysqlType +} + +func withZerofill4MySQLType(mysqlType string, zerofill bool) string { + if zerofill && !strings.HasPrefix(mysqlType, "year") { + return mysqlType + " zerofill" + } + return mysqlType +} + +// GetMySQLType get the mysql type from column info +func GetMySQLType(columnInfo *timodel.ColumnInfo, fullType bool) string { + if !fullType { + result := types.TypeToStr(columnInfo.GetType(), columnInfo.GetCharset()) + result = withUnsigned4MySQLType(result, mysql.HasUnsignedFlag(columnInfo.GetFlag())) + result = withZerofill4MySQLType(result, mysql.HasZerofillFlag(columnInfo.GetFlag())) + return result + } + return columnInfo.GetTypeDesc() +} + +// ExtractBasicMySQLType return the mysql type +func ExtractBasicMySQLType(mysqlType string) byte { + for i := 0; i < len(mysqlType); i++ { + if mysqlType[i] == '(' || mysqlType[i] == ' ' { + return types.StrToType(mysqlType[:i]) + } + } + + return types.StrToType(mysqlType) +} + +// IsBinaryMySQLType return true if the given mysqlType string is a binary type +func IsBinaryMySQLType(mysqlType string) bool { + return strings.Contains(mysqlType, "blob") || strings.Contains(mysqlType, "binary") +} diff --git a/pkg/sink/codec/internal/java.go b/pkg/sink/codec/internal/java.go index 50e06434546..cbebe37606a 100644 --- a/pkg/sink/codec/internal/java.go +++ b/pkg/sink/codec/internal/java.go @@ -115,24 +115,25 @@ func MySQLType2JavaType(mysqlType byte, isBinary bool) JavaSQLType { // Blob related is not identical to the official implementation, since we do not know `meta` at the moment. // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/dbsync/src/main/java/com/taobao/tddl/dbsync/binlog/event/RowsLogBuffer.java#L222-L231 // But this does not matter, they will be `JavaSQLTypeBlob` or `JavaSQLTypeClob` finally. - case mysql.TypeTinyBlob: - return JavaSQLTypeVARBINARY - - case mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - return JavaSQLTypeLONGVARBINARY + case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if isBinary { + return JavaSQLTypeBLOB + } + return JavaSQLTypeCLOB case mysql.TypeVarString, mysql.TypeVarchar: if isBinary { - return JavaSQLTypeVARBINARY + return JavaSQLTypeBLOB } return JavaSQLTypeVARCHAR case mysql.TypeString: if isBinary { - return JavaSQLTypeBINARY + return JavaSQLTypeBLOB } return JavaSQLTypeCHAR + // Geometry is not supported, this should not hit. case mysql.TypeGeometry: return JavaSQLTypeBINARY diff --git a/pkg/sink/codec/utils/field_types.go b/pkg/sink/codec/utils/field_types.go new file mode 100644 index 00000000000..d327b4b5fcf --- /dev/null +++ b/pkg/sink/codec/utils/field_types.go @@ -0,0 +1,82 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "strings" + + "github.com/pingcap/tidb/parser/charset" + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/cdc/model" +) + +// SetBinChsClnFlag set the binary charset flag. +func SetBinChsClnFlag(ft *types.FieldType) *types.FieldType { + ft.SetCharset(charset.CharsetBin) + ft.SetCollate(charset.CollationBin) + ft.AddFlag(mysql.BinaryFlag) + return ft +} + +// SetUnsigned set the unsigned flag. +func SetUnsigned(ft *types.FieldType) *types.FieldType { + ft.SetFlag(uint(model.UnsignedFlag)) + return ft +} + +// SetElems set the elems to the ft +func SetElems(ft *types.FieldType, elems []string) *types.FieldType { + ft.SetElems(elems) + return ft +} + +// when encoding the canal format, for unsigned mysql type, add `unsigned` keyword. +// it should have the form `t unsigned`, such as `int unsigned` +func withUnsigned4MySQLType(mysqlType string, unsigned bool) string { + if unsigned && mysqlType != "bit" && mysqlType != "year" { + return mysqlType + " unsigned" + } + return mysqlType +} + +func withZerofill4MySQLType(mysqlType string, zerofill bool) string { + if zerofill && !strings.HasPrefix(mysqlType, "year") { + return mysqlType + " zerofill" + } + return mysqlType +} + +// GetMySQLType get the mysql type from column info +func GetMySQLType(columnInfo *timodel.ColumnInfo, fullType bool) string { + if !fullType { + result := types.TypeToStr(columnInfo.GetType(), columnInfo.GetCharset()) + result = withUnsigned4MySQLType(result, mysql.HasUnsignedFlag(columnInfo.GetFlag())) + result = withZerofill4MySQLType(result, mysql.HasZerofillFlag(columnInfo.GetFlag())) + return result + } + return columnInfo.GetTypeDesc() +} + +// ExtractBasicMySQLType return the mysql type +func ExtractBasicMySQLType(mysqlType string) byte { + for i := 0; i < len(mysqlType); i++ { + if mysqlType[i] == '(' || mysqlType[i] == ' ' { + return types.StrToType(mysqlType[:i]) + } + } + + return types.StrToType(mysqlType) +} diff --git a/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml b/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml new file mode 100644 index 00000000000..7f40bf15127 --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] +output-dir = "/tmp/tidb_cdc_test/canal_json_content_compatible/output" + +source-instances = ["mysql1"] + +target-instance = "tidb0" + +target-check-tables = ["test.?*"] + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 4000 +user = "root" +password = "" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "" diff --git a/tests/integration_tests/canal_json_content_compatible/data/data.sql b/tests/integration_tests/canal_json_content_compatible/data/data.sql new file mode 100644 index 00000000000..5d0ae96193a --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/data/data.sql @@ -0,0 +1,330 @@ +drop database if exists test; +create database test; +use test; + +create table tp_int +( + id int auto_increment, + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + constraint pk + primary key (id) +); + +insert into tp_int() +values (); + +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +-- insert min value +insert into tp_int(c_tinyint, c_smallint, c_mediumint, c_int, c_bigint) +values (-128, -32768, -8388608, -2147483648, -9223372036854775808); + +update tp_int set c_int = 0, c_tinyint = 0 where c_smallint = 2; +delete from tp_int where c_int = 0; + +-- unsigned int +create table tp_unsigned_int ( + id int auto_increment, + c_unsigned_tinyint tinyint unsigned null, + c_unsigned_smallint smallint unsigned null, + c_unsigned_mediumint mediumint unsigned null, + c_unsigned_int int unsigned null, + c_unsigned_bigint bigint unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_int() +values (); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (1, 2, 3, 4, 5); + +-- insert max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (255, 65535, 16777215, 4294967295, 18446744073709551615); + +-- insert signed max value +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (127, 32767, 8388607, 2147483647, 9223372036854775807); + +insert into tp_unsigned_int(c_unsigned_tinyint, c_unsigned_smallint, c_unsigned_mediumint, + c_unsigned_int, c_unsigned_bigint) +values (128, 32768, 8388608, 2147483648, 9223372036854775808); + +update tp_unsigned_int set c_unsigned_int = 0, c_unsigned_tinyint = 0 where c_unsigned_smallint = 65535; +delete from tp_unsigned_int where c_unsigned_int = 0; + +-- real +create table tp_real +( + id int auto_increment, + c_float float null, + c_double double null, + c_decimal decimal null, + c_decimal_2 decimal(10, 4) null, + constraint pk + primary key (id) +); + +insert into tp_real() +values (); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +insert into tp_real(c_float, c_double, c_decimal, c_decimal_2) +values (-2.7182818284, -3.1415926, -8000, -179394.233); + +update tp_real set c_double = 2.333 where c_double = 2020.0303; + +-- unsigned real +create table tp_unsigned_real ( + id int auto_increment, + c_unsigned_float float unsigned null, + c_unsigned_double double unsigned null, + c_unsigned_decimal decimal unsigned null, + c_unsigned_decimal_2 decimal(10, 4) unsigned null, + constraint pk + primary key (id) +); + +insert into tp_unsigned_real() +values (); + +insert into tp_unsigned_real(c_unsigned_float, c_unsigned_double, c_unsigned_decimal, c_unsigned_decimal_2) +values (2020.0202, 2020.0303, 2020.0404, 2021.1208); + +update tp_unsigned_real set c_unsigned_double = 2020.0404 where c_unsigned_double = 2020.0303; + +-- time +create table tp_time +( + id int auto_increment, + c_date date null, + c_datetime datetime null, + c_timestamp timestamp null, + c_time time null, + c_year year null, + constraint pk + primary key (id) +); + +insert into tp_time() +values (); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020'); + +insert into tp_time(c_date, c_datetime, c_timestamp, c_time, c_year) +values ('2022-02-22', '2022-02-22 22:22:22', '2020-02-20 02:20:20', '02:20:20', '2021'); + +update tp_time set c_year = '2022' where c_year = '2020'; +update tp_time set c_date = '2022-02-22' where c_datetime = '2020-02-20 02:20:20'; + +-- text +create table tp_text +( + id int auto_increment, + c_tinytext tinytext null, + c_text text null, + c_mediumtext mediumtext null, + c_longtext longtext null, + constraint pk + primary key (id) +); + +insert into tp_text() +values (); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A'); + +insert into tp_text(c_tinytext, c_text, c_mediumtext, c_longtext) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B', '89504E470D0A1A0B'); + +update tp_text set c_text = '89504E470D0A1A0B' where c_mediumtext = '89504E470D0A1A0A'; + +-- blob +create table tp_blob +( + id int auto_increment, + c_tinyblob tinyblob null, + c_blob blob null, + c_mediumblob mediumblob null, + c_longblob longblob null, + constraint pk + primary key (id) +); + +insert into tp_blob() +values (); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_blob(c_tinyblob, c_blob, c_mediumblob, c_longblob) +values (x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_blob set c_blob = x'89504E470D0A1A0B' where c_mediumblob = x'89504E470D0A1A0A'; + +-- char / binary +create table tp_char_binary +( + id int auto_increment, + c_char char(16) null, + c_varchar varchar(16) null, + c_binary binary(16) null, + c_varbinary varbinary(16) null, + constraint pk + primary key (id) +); + +insert into tp_char_binary() +values (); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A'); + +insert into tp_char_binary(c_char, c_varchar, c_binary, c_varbinary) +values ('89504E470D0A1A0B', '89504E470D0A1A0B', x'89504E470D0A1A0B', x'89504E470D0A1A0B'); + +update tp_char_binary set c_varchar = '89504E470D0A1A0B' where c_binary = x'89504E470D0A1A0A'; + +-- other +create table tp_other +( + id int auto_increment, + c_enum enum ('a','b','c') null, + c_set set ('a','b','c') null, + c_bit bit(64) null, + c_json json null, + constraint pk + primary key (id) +); + +insert into tp_other() +values (); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('a', 'a,b', b'1000001', '{ + "key1": "value1", + "key2": "value2" +}'); + +insert into tp_other(c_enum, c_set, c_bit, c_json) +values ('b', 'b,c', b'1000001', '{ + "key1": "value1", + "key2": "value2", + "key3": "123" +}'); + +update tp_other set c_enum = 'c' where c_set = 'b, c'; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '测试', "中国", "上海", "你好,世界" + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '部署', "美国", "纽约", "世界,你好" + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '开发' +WHERE name = '测试'; + +DELETE FROM cs_gbk +WHERE name = '部署' + AND country = '美国' + AND city = '纽约' + AND description = '世界,你好'; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + 名称 varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN 城市 char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN 城市 varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN 城市; + +/* this is a DDL test for table */ +CREATE TABLE 表1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 表1 TO 表2; + +DROP TABLE 表2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql b/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql new file mode 100644 index 00000000000..504d7b41174 --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/data/data_gbk.sql @@ -0,0 +1,101 @@ +-- This sql file is encoded in GBK by the 'iconv' command. +-- DO NOT EDIT. + +DROP DATABASE IF EXISTS test; +CREATE DATABASE test; +SET NAMES gbk; +USE test; + +-- gbk dmls +CREATE TABLE cs_gbk ( + id INT, + name varchar(128) CHARACTER SET gbk, + country char(32) CHARACTER SET gbk, + city varchar(64), + description text CHARACTER SET gbk, + image tinyblob, + PRIMARY KEY (id) +) ENGINE = InnoDB CHARSET = utf8mb4; + +INSERT INTO cs_gbk +VALUES (1, '', "й", "Ϻ", "," + , 0xC4E3BAC3CAC0BDE7); + +INSERT INTO cs_gbk +VALUES (2, '', "", "ŦԼ", "," + , 0xCAC0BDE7C4E3BAC3); + +UPDATE cs_gbk +SET name = '' +WHERE name = ''; + +DELETE FROM cs_gbk +WHERE name = '' + AND country = '' + AND city = 'ŦԼ' + AND description = ','; + +-- ddls +CREATE TABLE test_ddl1 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +RENAME TABLE test_ddl1 TO test_ddl; + +ALTER TABLE test_ddl + ADD INDEX test_add_index (c1); + +DROP INDEX test_add_index ON test_ddl; + +ALTER TABLE test_ddl + ADD COLUMN c2 INT NOT NULL; + +TRUNCATE TABLE test_ddl; + +DROP TABLE test_ddl2; + +CREATE TABLE test_ddl2 +( + id INT AUTO_INCREMENT, + c1 INT, + PRIMARY KEY (id) +); + +CREATE TABLE test_ddl3 ( + id INT, + varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +ALTER TABLE test_ddl3 + ADD COLUMN char(32); + +ALTER TABLE test_ddl3 + MODIFY COLUMN varchar(32); + +ALTER TABLE test_ddl3 + DROP COLUMN ; + +/* this is a DDL test for table */ +CREATE TABLE 1 ( + id INT, + name varchar(128), + PRIMARY KEY (id) +) ENGINE = InnoDB; + +RENAME TABLE 1 TO 2; + +create table finish_mark +( + id int PRIMARY KEY +); diff --git a/tests/integration_tests/canal_json_content_compatible/run.sh b/tests/integration_tests/canal_json_content_compatible/run.sh new file mode 100644 index 00000000000..1e98e486bd0 --- /dev/null +++ b/tests/integration_tests/canal_json_content_compatible/run.sh @@ -0,0 +1,56 @@ +#!/bin/bash + +set -e + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +# use kafka-consumer with canal-json decoder to sync data from kafka to mysql +function run() { + if [ "$SINK_TYPE" != "kafka" ]; then + return + fi + + # clean up environment + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + # start tidb cluster + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + TOPIC_NAME="ticdc-canal-json-content-compatible" + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + if [ "$SINK_TYPE" == "kafka" ]; then + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true&kafka-version=${KAFKA_VERSION}" + fi + + run_cdc_cli changefeed create --sink-uri="$SINK_URI" + sleep 5 # wait for changefeed to start + # determine the sink uri and run corresponding consumer + # currently only kafka and pulsar are supported + if [ "$SINK_TYPE" == "kafka" ]; then + run_kafka_consumer $WORK_DIR $SINK_URI + fi + + run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + # sync_diff can't check non-exist table, so we check expected tables are created in downstream first + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 5c714f70351..e17e42f5b63 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -15,7 +15,7 @@ mysql_only_http="http_api http_api_tls api_v2" mysql_only_consistent_replicate="consistent_replicate_ddl consistent_replicate_gbk consistent_replicate_nfs consistent_replicate_storage_file consistent_replicate_storage_file_large_value consistent_replicate_storage_s3 consistent_partition_table" kafka_only="kafka_big_messages kafka_compression kafka_messages kafka_sink_error_resume mq_sink_lost_callback" -kafka_only_protocol="canal_json_adapter_compatibility canal_json_basic multi_topics canal_json_handle_key_only open_protocol_handle_key_only" +kafka_only_protocol="canal_json_adapter_compatibility canal_json_basic canal_json_content_compatible multi_topics canal_json_handle_key_only open_protocol_handle_key_only" kafka_only_v2="kafka_big_txn_v2 kafka_big_messages_v2 multi_tables_ddl_v2 multi_topics_v2" storage_only_csv="csv_storage_basic csv_storage_multi_tables_ddl csv_storage_partition_table storage_csv_update" From a4df11e8ea67896e1e3b0edb401caebaddc9313a Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 5 Dec 2023 17:01:19 +0800 Subject: [PATCH 19/23] redo(ticdc): redo lz4 compression (#10171) (#10218) close pingcap/tiflow#10176 --- cdc/api/v2/model.go | 3 + cdc/model/sink.go | 2 +- cdc/model/sink_gen.go | 35 +----- cdc/redo/reader/file.go | 17 +++ cdc/redo/writer/memory/file_worker.go | 87 ++++++++++---- go.mod | 4 +- pkg/compression/compress.go | 112 ++++++++++++++++++ pkg/config/config_test_data.go | 3 + pkg/config/consistent.go | 7 ++ pkg/config/replica_config.go | 1 + .../conf/changefeed.toml | 1 + 11 files changed, 219 insertions(+), 53 deletions(-) create mode 100644 pkg/compression/compress.go diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index b117e51133d..9457278a49c 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -268,6 +268,7 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( FlushWorkerNum: c.Consistent.FlushWorkerNum, Storage: c.Consistent.Storage, UseFileBackend: c.Consistent.UseFileBackend, + Compression: c.Consistent.Compression, } } if c.Sink != nil { @@ -640,6 +641,7 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { FlushWorkerNum: c.Consistent.FlushWorkerNum, Storage: cloned.Consistent.Storage, UseFileBackend: cloned.Consistent.UseFileBackend, + Compression: cloned.Consistent.Compression, } } if cloned.Mounter != nil { @@ -824,6 +826,7 @@ type ConsistentConfig struct { FlushWorkerNum int `json:"flush_worker_num"` Storage string `json:"storage,omitempty"` UseFileBackend bool `json:"use_file_backend"` + Compression string `json:"compression,omitempty"` } // ChangefeedSchedulerConfig is per changefeed scheduler settings. diff --git a/cdc/model/sink.go b/cdc/model/sink.go index f9fc4372712..545608437ce 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -498,7 +498,7 @@ type Column struct { Default interface{} `json:"default" msg:"-"` // ApproximateBytes is approximate bytes consumed by the column. - ApproximateBytes int `json:"-"` + ApproximateBytes int `json:"-" msg:"-"` } // RedoColumn stores Column change diff --git a/cdc/model/sink_gen.go b/cdc/model/sink_gen.go index 8876f43e88f..a89615c5692 100644 --- a/cdc/model/sink_gen.go +++ b/cdc/model/sink_gen.go @@ -48,12 +48,6 @@ func (z *Column) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "Collation") return } - case "ApproximateBytes": - z.ApproximateBytes, err = dc.ReadInt() - if err != nil { - err = msgp.WrapError(err, "ApproximateBytes") - return - } default: err = dc.Skip() if err != nil { @@ -67,9 +61,9 @@ func (z *Column) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *Column) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 5 + // map header, size 4 // write "name" - err = en.Append(0x85, 0xa4, 0x6e, 0x61, 0x6d, 0x65) + err = en.Append(0x84, 0xa4, 0x6e, 0x61, 0x6d, 0x65) if err != nil { return } @@ -108,25 +102,15 @@ func (z *Column) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "Collation") return } - // write "ApproximateBytes" - err = en.Append(0xb0, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73) - if err != nil { - return - } - err = en.WriteInt(z.ApproximateBytes) - if err != nil { - err = msgp.WrapError(err, "ApproximateBytes") - return - } return } // MarshalMsg implements msgp.Marshaler func (z *Column) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 5 + // map header, size 4 // string "name" - o = append(o, 0x85, 0xa4, 0x6e, 0x61, 0x6d, 0x65) + o = append(o, 0x84, 0xa4, 0x6e, 0x61, 0x6d, 0x65) o = msgp.AppendString(o, z.Name) // string "type" o = append(o, 0xa4, 0x74, 0x79, 0x70, 0x65) @@ -137,9 +121,6 @@ func (z *Column) MarshalMsg(b []byte) (o []byte, err error) { // string "collation" o = append(o, 0xa9, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e) o = msgp.AppendString(o, z.Collation) - // string "ApproximateBytes" - o = append(o, 0xb0, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73) - o = msgp.AppendInt(o, z.ApproximateBytes) return } @@ -185,12 +166,6 @@ func (z *Column) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "Collation") return } - case "ApproximateBytes": - z.ApproximateBytes, bts, err = msgp.ReadIntBytes(bts) - if err != nil { - err = msgp.WrapError(err, "ApproximateBytes") - return - } default: bts, err = msgp.Skip(bts) if err != nil { @@ -205,7 +180,7 @@ func (z *Column) UnmarshalMsg(bts []byte) (o []byte, err error) { // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message func (z *Column) Msgsize() (s int) { - s = 1 + 5 + msgp.StringPrefixSize + len(z.Name) + 5 + msgp.ByteSize + 8 + msgp.StringPrefixSize + len(z.Charset) + 10 + msgp.StringPrefixSize + len(z.Collation) + 17 + msgp.IntSize + s = 1 + 5 + msgp.StringPrefixSize + len(z.Name) + 5 + msgp.ByteSize + 8 + msgp.StringPrefixSize + len(z.Charset) + 10 + msgp.StringPrefixSize + len(z.Collation) return } diff --git a/cdc/redo/reader/file.go b/cdc/redo/reader/file.go index 07744a884c5..8d5cff711ee 100644 --- a/cdc/redo/reader/file.go +++ b/cdc/redo/reader/file.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tiflow/cdc/model/codec" "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/cdc/redo/writer/file" + "github.com/pingcap/tiflow/pkg/compression" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/redo" "go.uber.org/zap" @@ -52,6 +53,9 @@ const ( defaultWorkerNum = 16 ) +// lz4MagicNumber is the magic number of lz4 compressed data +var lz4MagicNumber = []byte{0x04, 0x22, 0x4D, 0x18} + type fileReader interface { io.Closer // Read return the log from log file @@ -203,6 +207,13 @@ func selectDownLoadFile( return files, nil } +func isLZ4Compressed(data []byte) bool { + if len(data) < 4 { + return false + } + return bytes.Equal(data[:4], lz4MagicNumber) +} + func readAllFromBuffer(buf []byte) (logHeap, error) { r := &reader{ br: bytes.NewReader(buf), @@ -251,6 +262,12 @@ func sortAndWriteFile( log.Warn("download file is empty", zap.String("file", fileName)) return nil } + // it's lz4 compressed, decompress it + if isLZ4Compressed(fileContent) { + if fileContent, err = compression.Decode(compression.LZ4, fileContent); err != nil { + return err + } + } // sort data h, err := readAllFromBuffer(fileContent) diff --git a/cdc/redo/writer/memory/file_worker.go b/cdc/redo/writer/memory/file_worker.go index 3258bf169c4..3571a102c9e 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -14,16 +14,20 @@ package memory import ( + "bytes" "context" "fmt" + "io" "sync" "time" + "github.com/pierrec/lz4/v4" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/compression" "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/uuid" @@ -34,6 +38,7 @@ import ( type fileCache struct { data []byte + fileSize int64 maxCommitTs model.Ts // After memoryWriter become stable, this field would be used to // avoid traversing log files. @@ -41,17 +46,24 @@ type fileCache struct { filename string flushed chan struct{} + writer *dataWriter } -func newFileCache(event *polymorphicRedoEvent, buf []byte) *fileCache { - buf = buf[:0] - buf = append(buf, event.data.Bytes()...) - return &fileCache{ - data: buf, - maxCommitTs: event.commitTs, - minCommitTs: event.commitTs, - flushed: make(chan struct{}), +type dataWriter struct { + buf *bytes.Buffer + writer io.Writer + closer io.Closer +} + +func (w *dataWriter) Write(p []byte) (n int, err error) { + return w.writer.Write(p) +} + +func (w *dataWriter) Close() error { + if w.closer != nil { + return w.closer.Close() } + return nil } func (f *fileCache) waitFlushed(ctx context.Context) error { @@ -71,14 +83,19 @@ func (f *fileCache) markFlushed() { } } -func (f *fileCache) appendData(event *polymorphicRedoEvent) { - f.data = append(f.data, event.data.Bytes()...) +func (f *fileCache) appendData(event *polymorphicRedoEvent) error { + _, err := f.writer.Write(event.data.Bytes()) + if err != nil { + return err + } + f.fileSize += int64(event.data.Len()) if event.commitTs > f.maxCommitTs { f.maxCommitTs = event.commitTs } if event.commitTs < f.minCommitTs { f.minCommitTs = event.commitTs } + return nil } type fileWorkerGroup struct { @@ -170,7 +187,10 @@ func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { return errors.Trace(egCtx.Err()) case file := <-f.flushCh: start := time.Now() - err := f.extStorage.WriteFile(egCtx, file.filename, file.data) + if err := file.writer.Close(); err != nil { + return errors.Trace(err) + } + err := f.extStorage.WriteFile(egCtx, file.filename, file.writer.buf.Bytes()) f.metricFlushAllDuration.Observe(time.Since(start).Seconds()) if err != nil { return errors.Trace(err) @@ -213,10 +233,40 @@ func (f *fileWorkerGroup) bgWriteLogs( } // newFileCache write event to a new file cache. -func (f *fileWorkerGroup) newFileCache(event *polymorphicRedoEvent) { +func (f *fileWorkerGroup) newFileCache(event *polymorphicRedoEvent) error { bufPtr := f.pool.Get().(*[]byte) - file := newFileCache(event, *bufPtr) + buf := *bufPtr + buf = buf[:0] + var ( + wr io.Writer + closer io.Closer + ) + bufferWriter := bytes.NewBuffer(buf) + wr = bufferWriter + if f.cfg.Compression == compression.LZ4 { + wr = lz4.NewWriter(bufferWriter) + closer = wr.(io.Closer) + } + _, err := wr.Write(event.data.Bytes()) + if err != nil { + return errors.Trace(err) + } + + dw := &dataWriter{ + buf: bufferWriter, + writer: wr, + closer: closer, + } + file := &fileCache{ + data: buf, + fileSize: int64(len(event.data.Bytes())), + maxCommitTs: event.commitTs, + minCommitTs: event.commitTs, + flushed: make(chan struct{}), + writer: dw, + } f.files = append(f.files, file) + return nil } func (f *fileWorkerGroup) writeToCache( @@ -230,12 +280,11 @@ func (f *fileWorkerGroup) writeToCache( defer f.metricWriteBytes.Add(float64(writeLen)) if len(f.files) == 0 { - f.newFileCache(event) - return nil + return f.newFileCache(event) } file := f.files[len(f.files)-1] - if int64(len(file.data))+writeLen > f.cfg.MaxLogSizeInBytes { + if file.fileSize+writeLen > f.cfg.MaxLogSizeInBytes { file.filename = f.getLogFileName(file.maxCommitTs) select { case <-egCtx.Done(): @@ -243,12 +292,10 @@ func (f *fileWorkerGroup) writeToCache( case f.flushCh <- file: } - f.newFileCache(event) - return nil + return f.newFileCache(event) } - file.appendData(event) - return nil + return file.appendData(event) } func (f *fileWorkerGroup) flushAll(egCtx context.Context) error { diff --git a/go.mod b/go.mod index c2c8eb4a0cd..170c60b5912 100644 --- a/go.mod +++ b/go.mod @@ -51,12 +51,14 @@ require ( github.com/jcmturner/gokrb5/v8 v8.4.3 github.com/jmoiron/sqlx v1.3.3 github.com/kami-zh/go-capturer v0.0.0-20171211120116-e492ea43421d + github.com/klauspost/compress v1.15.14 github.com/labstack/gommon v0.3.0 github.com/linkedin/goavro/v2 v2.11.1 github.com/mailru/easyjson v0.7.7 github.com/mattn/go-shellwords v1.0.12 github.com/modern-go/reflect2 v1.0.2 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 + github.com/pierrec/lz4/v4 v4.1.17 github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c @@ -199,7 +201,6 @@ require ( github.com/jonboulle/clockwork v0.3.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.15.14 // indirect github.com/klauspost/cpuid v1.3.1 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect @@ -228,7 +229,6 @@ require ( github.com/pelletier/go-toml/v2 v2.0.1 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/philhofer/fwd v1.1.1 // indirect - github.com/pierrec/lz4/v4 v4.1.17 // indirect github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d // indirect github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 // indirect github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect diff --git a/pkg/compression/compress.go b/pkg/compression/compress.go new file mode 100644 index 00000000000..dab5fa3e570 --- /dev/null +++ b/pkg/compression/compress.go @@ -0,0 +1,112 @@ +// Copyright 2023 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 compression + +import ( + "bytes" + "fmt" + "sync" + + "github.com/klauspost/compress/snappy" + "github.com/pierrec/lz4/v4" + "github.com/pingcap/errors" +) + +const ( + // None no compression + None string = "none" + + // Snappy compression + Snappy string = "snappy" + + // LZ4 compression + LZ4 string = "lz4" +) + +var ( + lz4ReaderPool = sync.Pool{ + New: func() interface{} { + return lz4.NewReader(nil) + }, + } + + bufferPool = sync.Pool{ + New: func() interface{} { + return new(bytes.Buffer) + }, + } +) + +// Supported return true if the given compression is supported. +func Supported(cc string) bool { + switch cc { + case None, Snappy, LZ4: + return true + } + return false +} + +// Encode the given data by the given compression codec. +func Encode(cc string, data []byte) ([]byte, error) { + switch cc { + case None: + return data, nil + case Snappy: + return snappy.Encode(nil, data), nil + case LZ4: + var buf bytes.Buffer + writer := lz4.NewWriter(&buf) + if _, err := writer.Write(data); err != nil { + return nil, errors.Trace(err) + } + if err := writer.Close(); err != nil { + return nil, errors.Trace(err) + } + return buf.Bytes(), nil + default: + } + + return nil, errors.New(fmt.Sprintf("Unsupported compression %s", cc)) +} + +// Decode the given data by the given compression codec. +func Decode(cc string, data []byte) ([]byte, error) { + switch cc { + case None: + return data, nil + case Snappy: + return snappy.Decode(nil, data) + case LZ4: + reader, ok := lz4ReaderPool.Get().(*lz4.Reader) + if !ok { + reader = lz4.NewReader(bytes.NewReader(data)) + } else { + reader.Reset(bytes.NewReader(data)) + } + buffer := bufferPool.Get().(*bytes.Buffer) + _, err := buffer.ReadFrom(reader) + // copy the buffer to a new slice with the correct length + // reuse lz4Reader and buffer + lz4ReaderPool.Put(reader) + res := make([]byte, buffer.Len()) + copy(res, buffer.Bytes()) + buffer.Reset() + bufferPool.Put(buffer) + + return res, err + default: + } + + return nil, errors.New(fmt.Sprintf("Unsupported compression %s", cc)) +} diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index d72b3aea310..e0b7f6774d9 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -61,6 +61,7 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "compression": "", "encoding-worker-num": 16, "flush-worker-num": 8, "storage": "", @@ -281,6 +282,7 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "compression": "", "encoding-worker-num": 16, "flush-worker-num": 8, "storage": "", @@ -417,6 +419,7 @@ const ( "max-log-size": 64, "flush-interval": 2000, "meta-flush-interval": 200, + "compression": "", "encoding-worker-num": 16, "flush-worker-num": 8, "storage": "", diff --git a/pkg/config/consistent.go b/pkg/config/consistent.go index 14021e2074a..26f4950b4b3 100644 --- a/pkg/config/consistent.go +++ b/pkg/config/consistent.go @@ -17,6 +17,7 @@ import ( "fmt" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/pkg/compression" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/util" @@ -32,6 +33,7 @@ type ConsistentConfig struct { FlushWorkerNum int `toml:"flush-worker-num" json:"flush-worker-num"` Storage string `toml:"storage" json:"storage"` UseFileBackend bool `toml:"use-file-backend" json:"use-file-backend"` + Compression string `toml:"compression" json:"compression"` } // ValidateAndAdjust validates the consistency config and adjusts it if necessary. @@ -61,6 +63,11 @@ func (c *ConsistentConfig) ValidateAndAdjust() error { fmt.Sprintf("The consistent.meta-flush-interval:%d must be equal or greater than %d", c.MetaFlushIntervalInMs, redo.MinFlushIntervalInMs)) } + if len(c.Compression) > 0 && + c.Compression != compression.None && c.Compression != compression.LZ4 { + return cerror.ErrInvalidReplicaConfig.FastGenByArgs( + fmt.Sprintf("The consistent.compression:%s must be 'none' or 'lz4'", c.Compression)) + } if c.EncodingWorkerNum == 0 { c.EncodingWorkerNum = redo.DefaultEncodingWorkerNum diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index c4fc92c13cd..5c153de2905 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -77,6 +77,7 @@ var defaultReplicaConfig = &ReplicaConfig{ FlushWorkerNum: redo.DefaultFlushWorkerNum, Storage: "", UseFileBackend: false, + Compression: "", }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, diff --git a/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml b/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml index dadf83cf9fd..12c6fb0d09b 100644 --- a/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml +++ b/tests/integration_tests/consistent_replicate_storage_file/conf/changefeed.toml @@ -1,3 +1,4 @@ [consistent] level = "eventual" storage = "file:///tmp/tidb_cdc_test/consistent_replicate_storage_file/redo" +compression = "lz4" From be9ee4937ffd793b9992a1c057fb56ae29a80376 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 6 Dec 2023 11:37:19 +0800 Subject: [PATCH 20/23] puller(ticdc): fix stuck detect issue (#10258) (#10260) close pingcap/tiflow#10256 --- cdc/puller/puller.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index 313daf06b82..5d50228242d 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -76,6 +76,8 @@ type pullerImpl struct { cfg *config.ServerConfig lastForwardTime time.Time lastForwardResolvedTs uint64 + // startResolvedTs is the resolvedTs when puller is initialized + startResolvedTs uint64 } // New create a new Puller fetch event start from checkpointTs and put into buf. @@ -117,6 +119,8 @@ func New(ctx context.Context, tableID: tableID, tableName: tableName, cfg: cfg, + + startResolvedTs: checkpointTs, } return p } @@ -184,7 +188,7 @@ func (p *pullerImpl) Run(ctx context.Context) error { case <-ctx.Done(): return errors.Trace(ctx.Err()) case <-stuckDetectorTicker.C: - if err := p.detectResolvedTsStuck(initialized); err != nil { + if err := p.detectResolvedTsStuck(); err != nil { return errors.Trace(err) } continue @@ -247,9 +251,15 @@ func (p *pullerImpl) Run(ctx context.Context) error { return g.Wait() } -func (p *pullerImpl) detectResolvedTsStuck(initialized bool) error { - if p.cfg.Debug.Puller.EnableResolvedTsStuckDetection && initialized { +func (p *pullerImpl) detectResolvedTsStuck() error { + if p.cfg.Debug.Puller.EnableResolvedTsStuckDetection { resolvedTs := p.tsTracker.Frontier() + // check if the resolvedTs is advancing, + // If the resolvedTs in Frontier is less than startResolvedTs, it means that the incremental scan has + // not complete yet. We need to make no decision in this scenario. + if resolvedTs <= p.startResolvedTs { + return nil + } if resolvedTs == p.lastForwardResolvedTs { log.Warn("ResolvedTs stuck detected in puller", zap.String("namespace", p.changefeed.Namespace), From 2cadf0a9c101966c237ccb3cadd71e6a5873cbed Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 6 Dec 2023 13:08:49 +0800 Subject: [PATCH 21/23] sink(ticdc): use subDir in storage.Walk (#10027) (#10050) close pingcap/tiflow#10041, close pingcap/tiflow#10044 --- .../cloudstorage/cloud_storage_dml_sink.go | 10 ++-- pkg/sink/cloudstorage/path.go | 39 +++++++------- pkg/sink/cloudstorage/path_test.go | 53 +++++++++++++++++++ 3 files changed, 78 insertions(+), 24 deletions(-) diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go index 35e7b2e5f3e..1069ef4a1ce 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go @@ -81,7 +81,7 @@ type DMLSink struct { alive struct { sync.RWMutex // msgCh is a channel to hold eventFragment. - msgCh chan eventFragment + msgCh *chann.DrainableChann[eventFragment] isDead bool } @@ -140,7 +140,7 @@ func NewDMLSink(ctx context.Context, cancel: wgCancel, dead: make(chan struct{}), } - s.alive.msgCh = make(chan eventFragment, defaultChannelSize) + s.alive.msgCh = chann.NewAutoDrainChann[eventFragment]() encodedCh := make(chan eventFragment, defaultChannelSize) workerChannels := make([]*chann.DrainableChann[eventFragment], cfg.WorkerCount) @@ -148,7 +148,7 @@ func NewDMLSink(ctx context.Context, // create a group of encoding workers. for i := 0; i < defaultEncodingConcurrency; i++ { encoder := encoderBuilder.Build() - s.encodingWorkers[i] = newEncodingWorker(i, s.changefeedID, encoder, s.alive.msgCh, encodedCh) + s.encodingWorkers[i] = newEncodingWorker(i, s.changefeedID, encoder, s.alive.msgCh.Out(), encodedCh) } // create defragmenter. s.defragmenter = newDefragmenter(encodedCh, workerChannels) @@ -168,7 +168,7 @@ func NewDMLSink(ctx context.Context, s.alive.Lock() s.alive.isDead = true - close(s.alive.msgCh) + s.alive.msgCh.CloseAndDrain() s.alive.Unlock() close(s.dead) @@ -234,7 +234,7 @@ func (s *DMLSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa s.statistics.ObserveRows(txn.Event.Rows...) // emit a TxnCallbackableEvent encoupled with a sequence number starting from one. - s.alive.msgCh <- eventFragment{ + s.alive.msgCh.In() <- eventFragment{ seqNumber: seq, versionedTable: tbl, event: txn, diff --git a/pkg/sink/cloudstorage/path.go b/pkg/sink/cloudstorage/path.go index 8b4a2550d51..6ca2844a762 100644 --- a/pkg/sink/cloudstorage/path.go +++ b/pkg/sink/cloudstorage/path.go @@ -192,27 +192,28 @@ func (f *FilePathGenerator) CheckOrWriteSchema( _, checksum := mustParseSchemaName(tblSchemaFile) schemaFileCnt := 0 lastVersion := uint64(0) - prefix := fmt.Sprintf(tableSchemaPrefix+"schema_", def.Schema, def.Table) + subDir := fmt.Sprintf(tableSchemaPrefix, def.Schema, def.Table) checksumSuffix := fmt.Sprintf("%010d.json", checksum) - err = f.storage.WalkDir(ctx, &storage.WalkOption{ObjPrefix: prefix}, - func(path string, _ int64) error { - schemaFileCnt++ - if !strings.HasSuffix(path, checksumSuffix) { - return nil - } - version, parsedChecksum := mustParseSchemaName(path) - if parsedChecksum != checksum { - // TODO: parsedChecksum should be ignored, remove this panic - // after the new path protocol is verified. - log.Panic("invalid schema file name", - zap.String("path", path), zap.Any("checksum", checksum)) - } - if version > lastVersion { - lastVersion = version - } + err = f.storage.WalkDir(ctx, &storage.WalkOption{ + SubDir: subDir, /* use subDir to prevent walk the whole storage */ + ObjPrefix: subDir + "schema_", + }, func(path string, _ int64) error { + schemaFileCnt++ + if !strings.HasSuffix(path, checksumSuffix) { return nil - }, - ) + } + version, parsedChecksum := mustParseSchemaName(path) + if parsedChecksum != checksum { + // TODO: parsedChecksum should be ignored, remove this panic + // after the new path protocol is verified. + log.Panic("invalid schema file name", + zap.String("path", path), zap.Any("checksum", checksum)) + } + if version > lastVersion { + lastVersion = version + } + return nil + }) if err != nil { return err } diff --git a/pkg/sink/cloudstorage/path_test.go b/pkg/sink/cloudstorage/path_test.go index 6a0dbe56c4a..ead5254fff4 100644 --- a/pkg/sink/cloudstorage/path_test.go +++ b/pkg/sink/cloudstorage/path_test.go @@ -17,9 +17,14 @@ import ( "context" "fmt" "net/url" + "os" + "path/filepath" "testing" "time" + timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/engine/pkg/clock" "github.com/pingcap/tiflow/pkg/config" @@ -275,3 +280,51 @@ func TestIsSchemaFile(t *testing.T) { "testCase: %s, path: %v", tt.name, tt.path) } } + +func TestCheckOrWriteSchema(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + dir := t.TempDir() + f := testFilePathGenerator(ctx, t, dir) + + var columns []*timodel.ColumnInfo + ft := types.NewFieldType(mysql.TypeLong) + ft.SetFlag(mysql.PriKeyFlag | mysql.NotNullFlag) + col := &timodel.ColumnInfo{ + Name: timodel.NewCIStr("Id"), + FieldType: *ft, + DefaultValue: 10, + } + columns = append(columns, col) + tableInfo := &model.TableInfo{ + TableInfo: &timodel.TableInfo{Columns: columns}, + Version: 100, + TableName: model.TableName{ + Schema: "test", + Table: "table1", + TableID: 20, + }, + } + + table := VersionedTableName{ + TableNameWithPhysicTableID: tableInfo.TableName, + TableInfoVersion: tableInfo.Version, + } + + err := f.CheckOrWriteSchema(ctx, table, tableInfo) + require.NoError(t, err) + require.Equal(t, tableInfo.Version, f.versionMap[table]) + + // test only table version changed, schema file should be reused + table.TableInfoVersion = 101 + err = f.CheckOrWriteSchema(ctx, table, tableInfo) + require.NoError(t, err) + require.Equal(t, tableInfo.Version, f.versionMap[table]) + + dir = filepath.Join(dir, "test/table1/meta") + cnt, err := os.ReadDir(dir) + require.NoError(t, err) + require.Equal(t, 1, len(cnt)) +} From 7822a1b9c0d0970d1bdecdede5b191c6f9189639 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 6 Dec 2023 14:20:51 +0800 Subject: [PATCH 22/23] ddl_manager (ticdc): fix in bdr mode cdc can not replicate a table's dmls after drop and re-create it (#10080) (#10089) close pingcap/tiflow#10079 --- cdc/owner/ddl_manager.go | 27 ++++++++++++------- .../integration_tests/bdr_mode/data/start.sql | 1 + tests/integration_tests/bdr_mode/data/up.sql | 5 ++++ 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/cdc/owner/ddl_manager.go b/cdc/owner/ddl_manager.go index 21fa767f41f..1943ffb604a 100644 --- a/cdc/owner/ddl_manager.go +++ b/cdc/owner/ddl_manager.go @@ -220,15 +220,6 @@ func (m *ddlManager) tick( } for _, event := range events { - // If changefeed is in BDRMode, skip ddl. - if m.BDRMode { - log.Info("changefeed is in BDRMode, skip a ddl event", - zap.String("namespace", m.changfeedID.Namespace), - zap.String("ID", m.changfeedID.ID), - zap.Any("ddlEvent", event)) - continue - } - // TODO: find a better place to do this check // check if the ddl event is belong to an ineligible table. // If so, we should ignore it. @@ -348,6 +339,24 @@ func (m *ddlManager) executeDDL(ctx context.Context) error { if m.executingDDL == nil { return nil } + + // If changefeed is in BDRMode, skip ddl. + if m.BDRMode { + log.Info("changefeed is in BDRMode, skip a ddl event", + zap.String("namespace", m.changfeedID.Namespace), + zap.String("ID", m.changfeedID.ID), + zap.Any("ddlEvent", m.executingDDL)) + tableName := m.executingDDL.TableInfo.TableName + // Set it to nil first to accelerate GC. + m.pendingDDLs[tableName][0] = nil + m.pendingDDLs[tableName] = m.pendingDDLs[tableName][1:] + m.schema.DoGC(m.executingDDL.CommitTs - 1) + m.justSentDDL = m.executingDDL + m.executingDDL = nil + m.cleanCache() + return nil + } + failpoint.Inject("ExecuteNotDone", func() { // This ddl will never finish executing. // It is used to test the logic that a ddl only block the related table diff --git a/tests/integration_tests/bdr_mode/data/start.sql b/tests/integration_tests/bdr_mode/data/start.sql index 2d5007efbad..33e6cc75cba 100644 --- a/tests/integration_tests/bdr_mode/data/start.sql +++ b/tests/integration_tests/bdr_mode/data/start.sql @@ -3,3 +3,4 @@ create database `bdr_mode`; use `bdr_mode`; create table `t1` (id int primary key, name varchar(20)); +create table `t2` (id int primary key, name varchar(20)); \ No newline at end of file diff --git a/tests/integration_tests/bdr_mode/data/up.sql b/tests/integration_tests/bdr_mode/data/up.sql index dd926b15515..4e36df6157b 100644 --- a/tests/integration_tests/bdr_mode/data/up.sql +++ b/tests/integration_tests/bdr_mode/data/up.sql @@ -16,3 +16,8 @@ insert into `t1` values (22, '22'), (44, '44'), (66, '66'), (88, '88'), (108, '1 rollback; insert into `t1` values (100, '100'), (300, '300'), (500, '500'), (700, '700'), (900, '900'); + +drop table `t2`; +create table `t2` (id int primary key, name varchar(20)); +insert into `t2` values (1, '1'), (3, '3'), (5, '5'), (7, '7'), (9, '9'); +insert into `t2` values (2, '2'), (4, '4'), (6, '6'), (8, '8'), (10, '10'); \ No newline at end of file From de95a8ecc4b9709ce40156e564daee8e3ea65a0f Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 6 Dec 2023 18:13:20 +0800 Subject: [PATCH 23/23] redo(ticdc): fix redo balckhole storage issues (#10023) (#10120) close pingcap/tiflow#10024 --- cdc/redo/meta_manager.go | 4 ++++ tests/integration_tests/api_v2/cases.go | 8 ++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index bfa4dce69e0..f25f065a721 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -131,6 +131,10 @@ func (m *metaManager) preStart(ctx context.Context) error { } // "nfs" and "local" scheme are converted to "file" scheme redo.FixLocalScheme(uri) + // blackhole scheme is converted to "noop" scheme here, so we can use blackhole for testing + if redo.IsBlackholeStorage(uri.Scheme) { + uri, _ = storage.ParseRawURL("noop://") + } extStorage, err := redo.InitExternalStorage(ctx, *uri) if err != nil { return err diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index ced863339b1..f9f5842d5a5 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -196,7 +196,9 @@ func testChangefeed(ctx context.Context, client *CDCRESTClient) error { log.Panic("failed to unmarshal response", zap.String("body", string(resp.body)), zap.Error(err)) } if !reflect.DeepEqual(cfInfo.Config, defaultReplicaConfig) { - log.Panic("config is not equals", zap.Any("add", defaultReplicaConfig), zap.Any("get", cfInfo.Config)) + log.Panic("config is not equals", + zap.Any("add", defaultReplicaConfig), + zap.Any("get", cfInfo.Config)) } // pause changefeed @@ -244,7 +246,9 @@ func testChangefeed(ctx context.Context, client *CDCRESTClient) error { log.Panic("unmarshal failed", zap.String("body", string(resp.body)), zap.Error(err)) } if !reflect.DeepEqual(cf.Config, customReplicaConfig) { - log.Panic("config is not equals", zap.Any("update", customReplicaConfig), zap.Any("get", cf.Config)) + log.Panic("config is not equals", + zap.Any("update", customReplicaConfig), + zap.Any("get", cf.Config)) } // list changefeed