github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/canal/canal_entry.go (about) 1 // Copyright 2022 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 canal 15 16 import ( 17 "fmt" 18 "math" 19 "reflect" 20 "strconv" 21 22 "github.com/golang/protobuf/proto" // nolint:staticcheck 23 "github.com/pingcap/errors" 24 mm "github.com/pingcap/tidb/pkg/parser/model" 25 timodel "github.com/pingcap/tidb/pkg/parser/model" 26 "github.com/pingcap/tidb/pkg/parser/mysql" 27 "github.com/pingcap/tiflow/cdc/model" 28 cerror "github.com/pingcap/tiflow/pkg/errors" 29 "github.com/pingcap/tiflow/pkg/sink/codec/common" 30 "github.com/pingcap/tiflow/pkg/sink/codec/internal" 31 "github.com/pingcap/tiflow/pkg/sink/codec/utils" 32 canal "github.com/pingcap/tiflow/proto/canal" 33 "golang.org/x/text/encoding" 34 "golang.org/x/text/encoding/charmap" 35 ) 36 37 // compatible with canal-1.1.4 38 // https://github.com/alibaba/canal/tree/canal-1.1.4 39 const ( 40 CanalPacketVersion int32 = 1 41 CanalProtocolVersion int32 = 1 42 CanalServerEncode string = "UTF-8" 43 ) 44 45 type canalEntryBuilder struct { 46 bytesDecoder *encoding.Decoder // default charset is ISO-8859-1 47 config *common.Config 48 } 49 50 // newCanalEntryBuilder creates a new canalEntryBuilder 51 func newCanalEntryBuilder(config *common.Config) *canalEntryBuilder { 52 return &canalEntryBuilder{ 53 bytesDecoder: charmap.ISO8859_1.NewDecoder(), 54 config: config, 55 } 56 } 57 58 // build the header of a canal entry 59 func (b *canalEntryBuilder) buildHeader(commitTs uint64, schema string, table string, eventType canal.EventType, rowCount int) *canal.Header { 60 t := convertToCanalTs(commitTs) 61 h := &canal.Header{ 62 VersionPresent: &canal.Header_Version{Version: CanalProtocolVersion}, 63 ServerenCode: CanalServerEncode, 64 ExecuteTime: t, 65 SourceTypePresent: &canal.Header_SourceType{SourceType: canal.Type_MYSQL}, 66 SchemaName: schema, 67 TableName: table, 68 EventTypePresent: &canal.Header_EventType{EventType: eventType}, 69 } 70 if rowCount > 0 { 71 p := &canal.Pair{ 72 Key: "rowsCount", 73 Value: strconv.Itoa(rowCount), 74 } 75 h.Props = append(h.Props, p) 76 } 77 return h 78 } 79 80 // In the official canal-json implementation, value were extracted from binlog buffer. 81 // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/dbsync/src/main/java/com/taobao/tddl/dbsync/binlog/event/RowsLogBuffer.java#L276-L1147 82 // all value will be represented in string type 83 // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L760-L855 84 func (b *canalEntryBuilder) formatValue(value interface{}, isBinary bool) (result string, err error) { 85 // value would be nil, if no value insert for the column. 86 if value == nil { 87 return "", nil 88 } 89 90 switch v := value.(type) { 91 case int64: 92 result = strconv.FormatInt(v, 10) 93 case uint64: 94 result = strconv.FormatUint(v, 10) 95 case float32: 96 result = strconv.FormatFloat(float64(v), 'f', -1, 32) 97 case float64: 98 result = strconv.FormatFloat(v, 'f', -1, 64) 99 case string: 100 result = v 101 case []byte: 102 // see https://github.com/alibaba/canal/blob/9f6021cf36f78cc8ac853dcf37a1769f359b868b/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L801 103 if isBinary { 104 decoded, err := b.bytesDecoder.Bytes(v) 105 if err != nil { 106 return "", err 107 } 108 result = string(decoded) 109 } else { 110 result = string(v) 111 } 112 default: 113 result = fmt.Sprintf("%v", v) 114 } 115 return result, nil 116 } 117 118 // build the Column in the canal RowData 119 // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L756-L872 120 func (b *canalEntryBuilder) buildColumn(c *model.Column, columnInfo *timodel.ColumnInfo, updated bool) (*canal.Column, error) { 121 mysqlType := utils.GetMySQLType(columnInfo, b.config.ContentCompatible) 122 javaType, err := getJavaSQLType(c.Value, c.Type, c.Flag) 123 if err != nil { 124 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 125 } 126 127 value, err := b.formatValue(c.Value, c.Flag.IsBinary()) 128 if err != nil { 129 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 130 } 131 132 canalColumn := &canal.Column{ 133 SqlType: int32(javaType), 134 Name: c.Name, 135 IsKey: c.Flag.IsPrimaryKey(), 136 Updated: updated, 137 IsNullPresent: &canal.Column_IsNull{IsNull: c.Value == nil}, 138 Value: value, 139 MysqlType: mysqlType, 140 } 141 return canalColumn, nil 142 } 143 144 // build the RowData of a canal entry 145 func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*canal.RowData, error) { 146 var columns []*canal.Column 147 colInfos := e.TableInfo.GetColInfosForRowChangedEvent() 148 for idx, column := range e.GetColumns() { 149 if column == nil { 150 continue 151 } 152 columnInfo, ok := e.TableInfo.GetColumnInfo(colInfos[idx].ID) 153 if !ok { 154 return nil, cerror.ErrCanalEncodeFailed.GenWithStack( 155 "column info not found for column id: %d", colInfos[idx].ID) 156 } 157 c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) 158 if err != nil { 159 return nil, errors.Trace(err) 160 } 161 columns = append(columns, c) 162 } 163 164 onlyHandleKeyColumns = onlyHandleKeyColumns && e.IsDelete() 165 var preColumns []*canal.Column 166 for idx, column := range e.GetPreColumns() { 167 if column == nil { 168 continue 169 } 170 if onlyHandleKeyColumns && !column.Flag.IsHandleKey() { 171 continue 172 } 173 columnInfo, ok := e.TableInfo.GetColumnInfo(colInfos[idx].ID) 174 if !ok { 175 return nil, cerror.ErrCanalEncodeFailed.GenWithStack( 176 "column info not found for column id: %d", colInfos[idx].ID) 177 } 178 c, err := b.buildColumn(column, columnInfo, !e.IsDelete()) 179 if err != nil { 180 return nil, errors.Trace(err) 181 } 182 preColumns = append(preColumns, c) 183 } 184 185 rowData := &canal.RowData{} 186 rowData.BeforeColumns = preColumns 187 rowData.AfterColumns = columns 188 return rowData, nil 189 } 190 191 // fromRowEvent builds canal entry from cdc RowChangedEvent 192 func (b *canalEntryBuilder) fromRowEvent(e *model.RowChangedEvent, onlyHandleKeyColumns bool) (*canal.Entry, error) { 193 eventType := convertRowEventType(e) 194 header := b.buildHeader(e.CommitTs, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName(), eventType, 1) 195 isDdl := isCanalDDL(eventType) // false 196 rowData, err := b.buildRowData(e, onlyHandleKeyColumns) 197 if err != nil { 198 return nil, errors.Trace(err) 199 } 200 rc := &canal.RowChange{ 201 EventTypePresent: &canal.RowChange_EventType{EventType: eventType}, 202 IsDdlPresent: &canal.RowChange_IsDdl{IsDdl: isDdl}, 203 RowDatas: []*canal.RowData{rowData}, 204 } 205 rcBytes, err := proto.Marshal(rc) 206 if err != nil { 207 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 208 } 209 210 // build entry 211 entry := &canal.Entry{ 212 Header: header, 213 EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA}, 214 StoreValue: rcBytes, 215 } 216 return entry, nil 217 } 218 219 // fromDDLEvent builds canal entry from cdc DDLEvent 220 func (b *canalEntryBuilder) fromDDLEvent(e *model.DDLEvent) (*canal.Entry, error) { 221 eventType := convertDdlEventType(e) 222 header := b.buildHeader(e.CommitTs, e.TableInfo.TableName.Schema, e.TableInfo.TableName.Table, eventType, -1) 223 isDdl := isCanalDDL(eventType) 224 rc := &canal.RowChange{ 225 EventTypePresent: &canal.RowChange_EventType{EventType: eventType}, 226 IsDdlPresent: &canal.RowChange_IsDdl{IsDdl: isDdl}, 227 Sql: e.Query, 228 RowDatas: nil, 229 DdlSchemaName: e.TableInfo.TableName.Schema, 230 } 231 rcBytes, err := proto.Marshal(rc) 232 if err != nil { 233 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 234 } 235 236 // build entry 237 entry := &canal.Entry{ 238 Header: header, 239 EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA}, 240 StoreValue: rcBytes, 241 } 242 return entry, nil 243 } 244 245 // convert ts in tidb to timestamp(in ms) in canal 246 func convertToCanalTs(commitTs uint64) int64 { 247 return int64(commitTs >> 18) 248 } 249 250 // get the canal EventType according to the RowChangedEvent 251 func convertRowEventType(e *model.RowChangedEvent) canal.EventType { 252 if e.IsDelete() { 253 return canal.EventType_DELETE 254 } 255 if len(e.PreColumns) == 0 { 256 return canal.EventType_INSERT 257 } 258 return canal.EventType_UPDATE 259 } 260 261 // get the canal EventType according to the DDLEvent 262 func convertDdlEventType(e *model.DDLEvent) canal.EventType { 263 // see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/ddl/DruidDdlParser.java#L59-L178 264 switch e.Type { 265 case mm.ActionCreateSchema, mm.ActionDropSchema, mm.ActionShardRowID, mm.ActionCreateView, 266 mm.ActionDropView, mm.ActionRecoverTable, mm.ActionModifySchemaCharsetAndCollate, 267 mm.ActionLockTable, mm.ActionUnlockTable, mm.ActionRepairTable, mm.ActionSetTiFlashReplica, 268 mm.ActionUpdateTiFlashReplicaStatus, mm.ActionCreateSequence, mm.ActionAlterSequence, 269 mm.ActionDropSequence, mm.ActionModifyTableAutoIdCache, mm.ActionRebaseAutoRandomBase: 270 return canal.EventType_QUERY 271 case mm.ActionCreateTable: 272 return canal.EventType_CREATE 273 case mm.ActionRenameTable, mm.ActionRenameTables: 274 return canal.EventType_RENAME 275 case mm.ActionAddIndex, mm.ActionAddForeignKey, mm.ActionAddPrimaryKey: 276 return canal.EventType_CINDEX 277 case mm.ActionDropIndex, mm.ActionDropForeignKey, mm.ActionDropPrimaryKey: 278 return canal.EventType_DINDEX 279 case mm.ActionAddColumn, mm.ActionDropColumn, mm.ActionModifyColumn, mm.ActionRebaseAutoID, 280 mm.ActionSetDefaultValue, mm.ActionModifyTableComment, mm.ActionRenameIndex, mm.ActionAddTablePartition, 281 mm.ActionDropTablePartition, mm.ActionModifyTableCharsetAndCollate, mm.ActionTruncateTablePartition, 282 mm.ActionAlterIndexVisibility, mm.ActionMultiSchemaChange, mm.ActionReorganizePartition, 283 mm.ActionAlterTablePartitioning, mm.ActionRemovePartitioning, 284 // AddColumns and DropColumns are removed in TiDB v6.2.0, see https://github.com/pingcap/tidb/pull/35862. 285 mm.ActionAddColumns, mm.ActionDropColumns: 286 return canal.EventType_ALTER 287 case mm.ActionDropTable: 288 return canal.EventType_ERASE 289 case mm.ActionTruncateTable: 290 return canal.EventType_TRUNCATE 291 default: 292 return canal.EventType_QUERY 293 } 294 } 295 296 func isCanalDDL(t canal.EventType) bool { 297 // see https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L297 298 switch t { 299 case canal.EventType_CREATE, 300 canal.EventType_RENAME, 301 canal.EventType_CINDEX, 302 canal.EventType_DINDEX, 303 canal.EventType_ALTER, 304 canal.EventType_ERASE, 305 canal.EventType_TRUNCATE, 306 canal.EventType_QUERY: 307 return true 308 } 309 return false 310 } 311 312 func getJavaSQLType(value interface{}, tp byte, flag model.ColumnFlagType) (result internal.JavaSQLType, err error) { 313 javaType := internal.MySQLType2JavaType(tp, flag.IsBinary()) 314 // flag `isUnsigned` only for `numerical` and `bit`, `year` data type. 315 if !flag.IsUnsigned() { 316 return javaType, nil 317 } 318 319 switch tp { 320 // for year, to `int64`, others to `uint64`. 321 // no need to promote type for `year` and `bit` 322 case mysql.TypeYear, mysql.TypeBit: 323 return javaType, nil 324 case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeNewDecimal: 325 return javaType, nil 326 } 327 328 // for **unsigned** integral types, type would be `uint64` or `string`. see reference: 329 // https://github.com/pingcap/tiflow/blob/1e3dd155049417e3fd7bf9b0a0c7b08723b33791/cdc/entry/mounter.go#L501 330 // https://github.com/pingcap/tidb/blob/6495a5a116a016a3e077d181b8c8ad81f76ac31b/types/datum.go#L423-L455 331 if value == nil { 332 return javaType, nil 333 } 334 var number uint64 335 switch v := value.(type) { 336 case uint64: 337 number = v 338 case string: 339 a, err := strconv.ParseUint(v, 10, 64) 340 if err != nil { 341 return javaType, err 342 } 343 number = a 344 default: 345 return javaType, errors.Errorf("unexpected type for unsigned value: %+v, tp: %+v", reflect.TypeOf(v), tp) 346 } 347 348 // Some special cases handled in canal 349 // see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L733 350 // For unsigned type, promote by the following rule: 351 // TinyInt, 1byte, [-128, 127], [0, 255], if a > 127 352 // SmallInt, 2byte, [-32768, 32767], [0, 65535], if a > 32767 353 // Int, 4byte, [-2147483648, 2147483647], [0, 4294967295], if a > 2147483647 354 // BigInt, 8byte, [-2<<63, 2 << 63 - 1], [0, 2 << 64 - 1], if a > 2 << 63 - 1 355 switch tp { 356 case mysql.TypeTiny: 357 if number > math.MaxInt8 { 358 javaType = internal.JavaSQLTypeSMALLINT 359 } 360 case mysql.TypeShort: 361 if number > math.MaxInt16 { 362 javaType = internal.JavaSQLTypeINTEGER 363 } 364 case mysql.TypeLong: 365 if number > math.MaxInt32 { 366 javaType = internal.JavaSQLTypeBIGINT 367 } 368 case mysql.TypeLonglong: 369 if number > math.MaxInt64 { 370 javaType = internal.JavaSQLTypeDECIMAL 371 } 372 } 373 374 return javaType, nil 375 }