github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/canal.go (about)

     1  // Copyright 2020 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package codec
    15  
    16  import (
    17  	"fmt"
    18  	"log"
    19  	"strconv"
    20  	"strings"
    21  
    22  	"github.com/golang/protobuf/proto"
    23  	"github.com/pingcap/errors"
    24  	mm "github.com/pingcap/parser/model"
    25  	"github.com/pingcap/parser/mysql"
    26  	parser_types "github.com/pingcap/parser/types"
    27  	"github.com/pingcap/ticdc/cdc/model"
    28  	cerror "github.com/pingcap/ticdc/pkg/errors"
    29  	canal "github.com/pingcap/ticdc/proto/canal"
    30  	"go.uber.org/zap"
    31  	"golang.org/x/text/encoding"
    32  	"golang.org/x/text/encoding/charmap"
    33  )
    34  
    35  // compatible with canal-1.1.4
    36  // https://github.com/alibaba/canal/tree/canal-1.1.4
    37  const (
    38  	CanalPacketVersion   int32  = 1
    39  	CanalProtocolVersion int32  = 1
    40  	CanalServerEncode    string = "UTF-8"
    41  )
    42  
    43  // convert ts in tidb to timestamp(in ms) in canal
    44  func convertToCanalTs(commitTs uint64) int64 {
    45  	return int64(commitTs >> 18)
    46  }
    47  
    48  // get the canal EventType according to the RowChangedEvent
    49  func convertRowEventType(e *model.RowChangedEvent) canal.EventType {
    50  	if e.IsDelete() {
    51  		return canal.EventType_DELETE
    52  	}
    53  	if len(e.PreColumns) == 0 {
    54  		return canal.EventType_INSERT
    55  	}
    56  	return canal.EventType_UPDATE
    57  }
    58  
    59  // get the canal EventType according to the DDLEvent
    60  func convertDdlEventType(e *model.DDLEvent) canal.EventType {
    61  	// see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/ddl/DruidDdlParser.java
    62  	switch e.Type {
    63  	case mm.ActionCreateSchema, mm.ActionDropSchema, mm.ActionShardRowID, mm.ActionCreateView,
    64  		mm.ActionDropView, mm.ActionRecoverTable, mm.ActionModifySchemaCharsetAndCollate,
    65  		mm.ActionLockTable, mm.ActionUnlockTable, mm.ActionRepairTable, mm.ActionSetTiFlashReplica,
    66  		mm.ActionUpdateTiFlashReplicaStatus, mm.ActionCreateSequence, mm.ActionAlterSequence,
    67  		mm.ActionDropSequence, mm.ActionModifyTableAutoIdCache, mm.ActionRebaseAutoRandomBase:
    68  		return canal.EventType_QUERY
    69  	case mm.ActionCreateTable:
    70  		return canal.EventType_CREATE
    71  	case mm.ActionRenameTable:
    72  		return canal.EventType_RENAME
    73  	case mm.ActionAddIndex, mm.ActionAddForeignKey, mm.ActionAddPrimaryKey:
    74  		return canal.EventType_CINDEX
    75  	case mm.ActionDropIndex, mm.ActionDropForeignKey, mm.ActionDropPrimaryKey:
    76  		return canal.EventType_DINDEX
    77  	case mm.ActionAddColumn, mm.ActionDropColumn, mm.ActionModifyColumn, mm.ActionRebaseAutoID,
    78  		mm.ActionSetDefaultValue, mm.ActionModifyTableComment, mm.ActionRenameIndex, mm.ActionAddTablePartition,
    79  		mm.ActionDropTablePartition, mm.ActionModifyTableCharsetAndCollate, mm.ActionTruncateTablePartition,
    80  		mm.ActionAddColumns, mm.ActionDropColumns:
    81  		return canal.EventType_ALTER
    82  	case mm.ActionDropTable:
    83  		return canal.EventType_ERASE
    84  	case mm.ActionTruncateTable:
    85  		return canal.EventType_TRUNCATE
    86  	default:
    87  		return canal.EventType_QUERY
    88  	}
    89  }
    90  
    91  func isCanalDdl(t canal.EventType) bool {
    92  	// EventType_QUERY is not a ddl type in canal, but in cdc it is.
    93  	// see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/ddl/DruidDdlParser.java
    94  	// &   https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L278
    95  	switch t {
    96  	case canal.EventType_CREATE,
    97  		canal.EventType_RENAME,
    98  		canal.EventType_CINDEX,
    99  		canal.EventType_DINDEX,
   100  		canal.EventType_ALTER,
   101  		canal.EventType_ERASE,
   102  		canal.EventType_TRUNCATE:
   103  		return true
   104  	}
   105  	return false
   106  }
   107  
   108  type canalEntryBuilder struct {
   109  	bytesDecoder *encoding.Decoder // default charset is ISO-8859-1
   110  }
   111  
   112  // build the header of a canal entry
   113  func (b *canalEntryBuilder) buildHeader(commitTs uint64, schema string, table string, eventType canal.EventType, rowCount int) *canal.Header {
   114  	t := convertToCanalTs(commitTs)
   115  	h := &canal.Header{
   116  		VersionPresent:    &canal.Header_Version{Version: CanalProtocolVersion},
   117  		ServerenCode:      CanalServerEncode,
   118  		ExecuteTime:       t,
   119  		SourceTypePresent: &canal.Header_SourceType{SourceType: canal.Type_MYSQL},
   120  		SchemaName:        schema,
   121  		TableName:         table,
   122  		EventTypePresent:  &canal.Header_EventType{EventType: eventType},
   123  	}
   124  	if rowCount > 0 {
   125  		p := &canal.Pair{
   126  			Key:   "rowsCount",
   127  			Value: strconv.Itoa(rowCount),
   128  		}
   129  		h.Props = append(h.Props, p)
   130  	}
   131  	return h
   132  }
   133  
   134  // build the Column in the canal RowData
   135  func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated bool) (*canal.Column, error) {
   136  	sqlType := MysqlToJavaType(c.Type)
   137  	mysqlType := parser_types.TypeStr(c.Type)
   138  	if c.Flag.IsBinary() {
   139  		if parser_types.IsTypeBlob(c.Type) {
   140  			mysqlType = strings.Replace(mysqlType, "text", "blob", 1)
   141  		} else if parser_types.IsTypeChar(c.Type) {
   142  			mysqlType = strings.Replace(mysqlType, "char", "binary", 1)
   143  		}
   144  	}
   145  	// Some special cases handled in canal
   146  	// see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L733
   147  	switch c.Type {
   148  	// Since we cannot get the signed/unsigned flag of the column in the RowChangedEvent currently,
   149  	// we promote the sqlTypes regardless of the flag.
   150  	case mysql.TypeTiny:
   151  		sqlType = JavaSQLTypeSMALLINT
   152  	case mysql.TypeShort:
   153  		sqlType = JavaSQLTypeINTEGER
   154  	case mysql.TypeInt24:
   155  		sqlType = JavaSQLTypeINTEGER
   156  	case mysql.TypeLong:
   157  		sqlType = JavaSQLTypeBIGINT
   158  	case mysql.TypeLonglong:
   159  		sqlType = JavaSQLTypeDECIMAL
   160  	}
   161  	switch sqlType {
   162  	case JavaSQLTypeBINARY, JavaSQLTypeVARBINARY, JavaSQLTypeLONGVARBINARY:
   163  		if c.Flag.IsBinary() {
   164  			sqlType = JavaSQLTypeBLOB
   165  		} else {
   166  			// In jdbc, text type is mapping to JavaSQLTypeVARCHAR
   167  			// see https://dev.mysql.com/doc/connector-j/5.1/en/connector-j-reference-type-conversions.html
   168  			sqlType = JavaSQLTypeVARCHAR
   169  		}
   170  	}
   171  
   172  	isKey := c.Flag.IsPrimaryKey()
   173  	isNull := c.Value == nil
   174  	value := ""
   175  	if !isNull {
   176  		switch v := c.Value.(type) {
   177  		case int64:
   178  			value = strconv.FormatInt(v, 10)
   179  		case uint64:
   180  			value = strconv.FormatUint(v, 10)
   181  		case float32:
   182  			value = strconv.FormatFloat(float64(v), 'f', -1, 32)
   183  		case float64:
   184  			value = strconv.FormatFloat(v, 'f', -1, 64)
   185  		case string:
   186  			value = v
   187  		case []byte:
   188  			// special handle for text and blob
   189  			// see https://github.com/alibaba/canal/blob/9f6021cf36f78cc8ac853dcf37a1769f359b868b/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L801
   190  			switch sqlType {
   191  			case JavaSQLTypeVARCHAR, JavaSQLTypeCHAR:
   192  				value = string(v)
   193  			default:
   194  				decoded, err := b.bytesDecoder.Bytes(v)
   195  				if err != nil {
   196  					return nil, cerror.WrapError(cerror.ErrCanalDecodeFailed, err)
   197  				}
   198  				value = string(decoded)
   199  				sqlType = JavaSQLTypeBLOB // change sql type to Blob when the type is []byte according to canal
   200  			}
   201  		default:
   202  			value = fmt.Sprintf("%v", v)
   203  		}
   204  	}
   205  
   206  	canalColumn := &canal.Column{
   207  		SqlType:       int32(sqlType),
   208  		Name:          colName,
   209  		IsKey:         isKey,
   210  		Updated:       updated,
   211  		IsNullPresent: &canal.Column_IsNull{IsNull: isNull},
   212  		Value:         value,
   213  		MysqlType:     mysqlType,
   214  	}
   215  	return canalColumn, nil
   216  }
   217  
   218  // build the RowData of a canal entry
   219  func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent) (*canal.RowData, error) {
   220  	var columns []*canal.Column
   221  	for _, column := range e.Columns {
   222  		if column == nil {
   223  			continue
   224  		}
   225  		c, err := b.buildColumn(column, column.Name, !e.IsDelete())
   226  		if err != nil {
   227  			return nil, errors.Trace(err)
   228  		}
   229  		columns = append(columns, c)
   230  	}
   231  	var preColumns []*canal.Column
   232  	for _, column := range e.PreColumns {
   233  		if column == nil {
   234  			continue
   235  		}
   236  		c, err := b.buildColumn(column, column.Name, !e.IsDelete())
   237  		if err != nil {
   238  			return nil, errors.Trace(err)
   239  		}
   240  		preColumns = append(preColumns, c)
   241  	}
   242  
   243  	rowData := &canal.RowData{}
   244  	rowData.BeforeColumns = preColumns
   245  	rowData.AfterColumns = columns
   246  	return rowData, nil
   247  }
   248  
   249  // FromRowEvent builds canal entry from cdc RowChangedEvent
   250  func (b *canalEntryBuilder) FromRowEvent(e *model.RowChangedEvent) (*canal.Entry, error) {
   251  	eventType := convertRowEventType(e)
   252  	header := b.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1)
   253  	isDdl := isCanalDdl(eventType) // false
   254  	rowData, err := b.buildRowData(e)
   255  	if err != nil {
   256  		return nil, errors.Trace(err)
   257  	}
   258  	rc := &canal.RowChange{
   259  		EventTypePresent: &canal.RowChange_EventType{EventType: eventType},
   260  		IsDdlPresent:     &canal.RowChange_IsDdl{IsDdl: isDdl},
   261  		RowDatas:         []*canal.RowData{rowData},
   262  	}
   263  	rcBytes, err := proto.Marshal(rc)
   264  	if err != nil {
   265  		return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   266  	}
   267  
   268  	// build entry
   269  	entry := &canal.Entry{
   270  		Header:           header,
   271  		EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA},
   272  		StoreValue:       rcBytes,
   273  	}
   274  	return entry, nil
   275  }
   276  
   277  // FromDdlEvent builds canal entry from cdc DDLEvent
   278  func (b *canalEntryBuilder) FromDdlEvent(e *model.DDLEvent) (*canal.Entry, error) {
   279  	eventType := convertDdlEventType(e)
   280  	header := b.buildHeader(e.CommitTs, e.TableInfo.Schema, e.TableInfo.Table, eventType, -1)
   281  	isDdl := isCanalDdl(eventType)
   282  	rc := &canal.RowChange{
   283  		EventTypePresent: &canal.RowChange_EventType{EventType: eventType},
   284  		IsDdlPresent:     &canal.RowChange_IsDdl{IsDdl: isDdl},
   285  		Sql:              e.Query,
   286  		RowDatas:         nil,
   287  		DdlSchemaName:    e.TableInfo.Schema,
   288  	}
   289  	rcBytes, err := proto.Marshal(rc)
   290  	if err != nil {
   291  		return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   292  	}
   293  
   294  	// build entry
   295  	entry := &canal.Entry{
   296  		Header:           header,
   297  		EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA},
   298  		StoreValue:       rcBytes,
   299  	}
   300  	return entry, nil
   301  }
   302  
   303  // NewCanalEntryBuilder creates a new canalEntryBuilder
   304  func NewCanalEntryBuilder() *canalEntryBuilder {
   305  	d := charmap.ISO8859_1.NewDecoder()
   306  	return &canalEntryBuilder{
   307  		bytesDecoder: d,
   308  	}
   309  }
   310  
   311  // CanalEventBatchEncoder encodes the events into the byte of a batch into.
   312  type CanalEventBatchEncoder struct {
   313  	messages     *canal.Messages
   314  	packet       *canal.Packet
   315  	entryBuilder *canalEntryBuilder
   316  }
   317  
   318  // AppendResolvedEvent appends a resolved event to the encoder
   319  // TODO TXN support
   320  func (d *CanalEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) {
   321  	return EncoderNoOperation, nil
   322  }
   323  
   324  // EncodeCheckpointEvent implements the EventBatchEncoder interface
   325  func (d *CanalEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) {
   326  	// For canal now, there is no such a corresponding type to ResolvedEvent so far.
   327  	// Therefore the event is ignored.
   328  	return nil, nil
   329  }
   330  
   331  // AppendRowChangedEvent implements the EventBatchEncoder interface
   332  func (d *CanalEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) {
   333  	entry, err := d.entryBuilder.FromRowEvent(e)
   334  	if err != nil {
   335  		return EncoderNoOperation, errors.Trace(err)
   336  	}
   337  	b, err := proto.Marshal(entry)
   338  	if err != nil {
   339  		return EncoderNoOperation, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   340  	}
   341  	d.messages.Messages = append(d.messages.Messages, b)
   342  	return EncoderNoOperation, nil
   343  }
   344  
   345  // EncodeDDLEvent implements the EventBatchEncoder interface
   346  func (d *CanalEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) {
   347  	entry, err := d.entryBuilder.FromDdlEvent(e)
   348  	if err != nil {
   349  		return nil, errors.Trace(err)
   350  	}
   351  	b, err := proto.Marshal(entry)
   352  	if err != nil {
   353  		return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   354  	}
   355  
   356  	messages := new(canal.Messages)
   357  	messages.Messages = append(messages.Messages, b)
   358  	b, err = messages.Marshal()
   359  	if err != nil {
   360  		return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   361  	}
   362  
   363  	packet := &canal.Packet{
   364  		VersionPresent: &canal.Packet_Version{
   365  			Version: CanalPacketVersion,
   366  		},
   367  		Type: canal.PacketType_MESSAGES,
   368  	}
   369  	packet.Body = b
   370  	b, err = packet.Marshal()
   371  	if err != nil {
   372  		return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err)
   373  	}
   374  
   375  	return newDDLMQMessage(ProtocolCanal, nil, b, e), nil
   376  }
   377  
   378  // Build implements the EventBatchEncoder interface
   379  func (d *CanalEventBatchEncoder) Build() []*MQMessage {
   380  	if len(d.messages.Messages) == 0 {
   381  		return nil
   382  	}
   383  
   384  	err := d.refreshPacketBody()
   385  	if err != nil {
   386  		log.Panic("Error when generating Canal packet", zap.Error(err))
   387  	}
   388  
   389  	value, err := proto.Marshal(d.packet)
   390  	if err != nil {
   391  		log.Panic("Error when serializing Canal packet", zap.Error(err))
   392  	}
   393  	ret := NewMQMessage(ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil)
   394  	d.messages.Reset()
   395  	d.resetPacket()
   396  	return []*MQMessage{ret}
   397  }
   398  
   399  // MixedBuild implements the EventBatchEncoder interface
   400  func (d *CanalEventBatchEncoder) MixedBuild(withVersion bool) []byte {
   401  	panic("Mixed Build only use for JsonEncoder")
   402  }
   403  
   404  // Size implements the EventBatchEncoder interface
   405  func (d *CanalEventBatchEncoder) Size() int {
   406  	// TODO: avoid marshaling the messages every time for calculating the size of the packet
   407  	err := d.refreshPacketBody()
   408  	if err != nil {
   409  		panic(err)
   410  	}
   411  	return proto.Size(d.packet)
   412  }
   413  
   414  // Reset implements the EventBatchEncoder interface
   415  func (d *CanalEventBatchEncoder) Reset() {
   416  	panic("Reset only used for JsonEncoder")
   417  }
   418  
   419  // SetParams is no-op for now
   420  func (d *CanalEventBatchEncoder) SetParams(params map[string]string) error {
   421  	// no op
   422  	return nil
   423  }
   424  
   425  // refreshPacketBody() marshals the messages to the packet body
   426  func (d *CanalEventBatchEncoder) refreshPacketBody() error {
   427  	oldSize := len(d.packet.Body)
   428  	newSize := proto.Size(d.messages)
   429  	if newSize > oldSize {
   430  		// resize packet body slice
   431  		d.packet.Body = append(d.packet.Body, make([]byte, newSize-oldSize)...)
   432  	} else {
   433  		d.packet.Body = d.packet.Body[:newSize]
   434  	}
   435  
   436  	_, err := d.messages.MarshalToSizedBuffer(d.packet.Body)
   437  	return err
   438  }
   439  
   440  func (d *CanalEventBatchEncoder) resetPacket() {
   441  	d.packet = &canal.Packet{
   442  		VersionPresent: &canal.Packet_Version{
   443  			Version: CanalPacketVersion,
   444  		},
   445  		Type: canal.PacketType_MESSAGES,
   446  	}
   447  }
   448  
   449  // NewCanalEventBatchEncoder creates a new CanalEventBatchEncoder.
   450  func NewCanalEventBatchEncoder() EventBatchEncoder {
   451  	encoder := &CanalEventBatchEncoder{
   452  		messages:     &canal.Messages{},
   453  		entryBuilder: NewCanalEntryBuilder(),
   454  	}
   455  
   456  	encoder.resetPacket()
   457  	return encoder
   458  }