diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 2d5ed4e2efc..243a9fc46fd 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -402,6 +402,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, @@ -621,6 +622,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, @@ -783,6 +785,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/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/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/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), 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 } 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/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 1cfe9b9d3ed..0816f2b0d4c 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 { @@ -234,10 +251,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( @@ -251,12 +298,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(): @@ -264,12 +310,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/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/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/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 f3471ddd357..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": "", @@ -210,9 +211,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, @@ -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": "", @@ -349,9 +351,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, @@ -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 8b4b52acad9..c8f93c6eac8 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" @@ -63,6 +64,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/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/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)) +} 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/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 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 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/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" 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"