github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/canal/canal_json_message.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 "sort" 18 "strconv" 19 "strings" 20 21 "github.com/pingcap/log" 22 timodel "github.com/pingcap/tidb/pkg/parser/model" 23 "github.com/pingcap/tidb/pkg/parser/mysql" 24 "github.com/pingcap/tiflow/cdc/model" 25 cerrors "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/sink/codec/utils" 27 canal "github.com/pingcap/tiflow/proto/canal" 28 "go.uber.org/zap" 29 "golang.org/x/text/encoding/charmap" 30 ) 31 32 const tidbWaterMarkType = "TIDB_WATERMARK" 33 34 // The TiCDC Canal-JSON implementation extend the official format with a TiDB extension field. 35 // canalJSONMessageInterface is used to support this without affect the original format. 36 type canalJSONMessageInterface interface { 37 getSchema() *string 38 getTable() *string 39 getCommitTs() uint64 40 getQuery() string 41 getOld() map[string]interface{} 42 getData() map[string]interface{} 43 getMySQLType() map[string]string 44 getJavaSQLType() map[string]int32 45 messageType() model.MessageType 46 eventType() canal.EventType 47 pkNameSet() map[string]struct{} 48 } 49 50 // JSONMessage adapted from https://github.com/alibaba/canal/blob/b54bea5e3337c9597c427a53071d214ff04628d1/protocol/src/main/java/com/alibaba/otter/canal/protocol/FlatMessage.java#L1 51 type JSONMessage struct { 52 // ignored by consumers 53 ID int64 `json:"id"` 54 Schema string `json:"database"` 55 Table string `json:"table"` 56 PKNames []string `json:"pkNames"` 57 IsDDL bool `json:"isDdl"` 58 EventType string `json:"type"` 59 // officially the timestamp of the event-time of the message, in milliseconds since Epoch. 60 ExecutionTime int64 `json:"es"` 61 // officially the timestamp of building the message, in milliseconds since Epoch. 62 BuildTime int64 `json:"ts"` 63 // SQL that generated the change event, DDL or Query 64 Query string `json:"sql"` 65 // only works for INSERT / UPDATE / DELETE events, records each column's java representation type. 66 SQLType map[string]int32 `json:"sqlType"` 67 // only works for INSERT / UPDATE / DELETE events, records each column's mysql representation type. 68 MySQLType map[string]string `json:"mysqlType"` 69 // A Datum should be a string or nil 70 Data []map[string]interface{} `json:"data"` 71 Old []map[string]interface{} `json:"old"` 72 } 73 74 func (c *JSONMessage) getSchema() *string { 75 return &c.Schema 76 } 77 78 func (c *JSONMessage) getTable() *string { 79 return &c.Table 80 } 81 82 // for JSONMessage, we lost the commitTs. 83 func (c *JSONMessage) getCommitTs() uint64 { 84 return 0 85 } 86 87 func (c *JSONMessage) getQuery() string { 88 return c.Query 89 } 90 91 func (c *JSONMessage) getOld() map[string]interface{} { 92 if c.Old == nil { 93 return nil 94 } 95 return c.Old[0] 96 } 97 98 func (c *JSONMessage) getData() map[string]interface{} { 99 if c.Data == nil { 100 return nil 101 } 102 return c.Data[0] 103 } 104 105 func (c *JSONMessage) getMySQLType() map[string]string { 106 return c.MySQLType 107 } 108 109 func (c *JSONMessage) getJavaSQLType() map[string]int32 { 110 return c.SQLType 111 } 112 113 func (c *JSONMessage) messageType() model.MessageType { 114 if c.IsDDL { 115 return model.MessageTypeDDL 116 } 117 118 if c.EventType == tidbWaterMarkType { 119 return model.MessageTypeResolved 120 } 121 122 return model.MessageTypeRow 123 } 124 125 func (c *JSONMessage) eventType() canal.EventType { 126 return canal.EventType(canal.EventType_value[c.EventType]) 127 } 128 129 func (c *JSONMessage) pkNameSet() map[string]struct{} { 130 result := make(map[string]struct{}, len(c.PKNames)) 131 for _, item := range c.PKNames { 132 result[item] = struct{}{} 133 } 134 return result 135 } 136 137 type tidbExtension struct { 138 CommitTs uint64 `json:"commitTs,omitempty"` 139 WatermarkTs uint64 `json:"watermarkTs,omitempty"` 140 OnlyHandleKey bool `json:"onlyHandleKey,omitempty"` 141 ClaimCheckLocation string `json:"claimCheckLocation,omitempty"` 142 } 143 144 type canalJSONMessageWithTiDBExtension struct { 145 *JSONMessage 146 // Extensions is a TiCDC custom field that different from official Canal-JSON format. 147 // It would be useful to store something for special usage. 148 // At the moment, only store the `tso` of each event, 149 // which is useful if the message consumer needs to restore the original transactions. 150 Extensions *tidbExtension `json:"_tidb"` 151 } 152 153 func (c *canalJSONMessageWithTiDBExtension) getCommitTs() uint64 { 154 return c.Extensions.CommitTs 155 } 156 157 func canalJSONMessage2RowChange(msg canalJSONMessageInterface) (*model.RowChangedEvent, error) { 158 result := new(model.RowChangedEvent) 159 result.CommitTs = msg.getCommitTs() 160 mysqlType := msg.getMySQLType() 161 var err error 162 if msg.eventType() == canal.EventType_DELETE { 163 // for `DELETE` event, `data` contain the old data, set it as the `PreColumns` 164 preCols, err := canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType) 165 result.TableInfo = model.BuildTableInfoWithPKNames4Test(*msg.getSchema(), *msg.getTable(), preCols, msg.pkNameSet()) 166 result.PreColumns = model.Columns2ColumnDatas(preCols, result.TableInfo) 167 return result, err 168 } 169 170 // for `INSERT` and `UPDATE`, `data` contain fresh data, set it as the `Columns` 171 cols, err := canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType) 172 result.TableInfo = model.BuildTableInfoWithPKNames4Test(*msg.getSchema(), *msg.getTable(), cols, msg.pkNameSet()) 173 result.Columns = model.Columns2ColumnDatas(cols, result.TableInfo) 174 if err != nil { 175 return nil, err 176 } 177 178 // for `UPDATE`, `old` contain old data, set it as the `PreColumns` 179 if msg.eventType() == canal.EventType_UPDATE { 180 preCols, err := canalJSONColumnMap2RowChangeColumns(msg.getOld(), mysqlType) 181 if len(preCols) < len(cols) { 182 newPreCols := make([]*model.Column, 0, len(preCols)) 183 j := 0 184 // Columns are ordered by name 185 for _, col := range cols { 186 if j < len(preCols) && col.Name == preCols[j].Name { 187 newPreCols = append(newPreCols, preCols[j]) 188 j += 1 189 } else { 190 newPreCols = append(newPreCols, col) 191 } 192 } 193 preCols = newPreCols 194 } 195 if len(preCols) != len(cols) { 196 log.Panic("column count mismatch", zap.Any("preCols", preCols), zap.Any("cols", cols)) 197 } 198 result.PreColumns = model.Columns2ColumnDatas(preCols, result.TableInfo) 199 if err != nil { 200 return nil, err 201 } 202 } 203 204 return result, nil 205 } 206 207 func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType map[string]string) ([]*model.Column, error) { 208 result := make([]*model.Column, 0, len(cols)) 209 for name, value := range cols { 210 mysqlTypeStr, ok := mysqlType[name] 211 if !ok { 212 // this should not happen, else we have to check encoding for mysqlType. 213 return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( 214 "mysql type does not found, column: %+v, mysqlType: %+v", name, mysqlType) 215 } 216 col := canalJSONFormatColumn(value, name, mysqlTypeStr) 217 result = append(result, col) 218 } 219 if len(result) == 0 { 220 return nil, nil 221 } 222 sort.Slice(result, func(i, j int) bool { 223 return strings.Compare(result[i].Name, result[j].Name) > 0 224 }) 225 return result, nil 226 } 227 228 func canalJSONFormatColumn(value interface{}, name string, mysqlTypeStr string) *model.Column { 229 mysqlType := utils.ExtractBasicMySQLType(mysqlTypeStr) 230 result := &model.Column{ 231 Type: mysqlType, 232 Name: name, 233 Value: value, 234 } 235 if result.Value == nil { 236 return result 237 } 238 239 data, ok := value.(string) 240 if !ok { 241 log.Panic("canal-json encoded message should have type in `string`") 242 } 243 244 var err error 245 if utils.IsBinaryMySQLType(mysqlTypeStr) { 246 // when encoding the `JavaSQLTypeBLOB`, use `ISO8859_1` decoder, now reverse it back. 247 encoder := charmap.ISO8859_1.NewEncoder() 248 value, err = encoder.String(data) 249 if err != nil { 250 log.Panic("invalid column value, please report a bug", zap.Any("col", result), zap.Error(err)) 251 } 252 result.Value = value 253 return result 254 } 255 256 switch mysqlType { 257 case mysql.TypeBit, mysql.TypeSet: 258 value, err = strconv.ParseUint(data, 10, 64) 259 if err != nil { 260 log.Panic("invalid column value for bit", zap.Any("col", result), zap.Error(err)) 261 } 262 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeInt24, mysql.TypeYear: 263 value, err = strconv.ParseInt(data, 10, 64) 264 if err != nil { 265 log.Panic("invalid column value for int", zap.Any("col", result), zap.Error(err)) 266 } 267 case mysql.TypeEnum: 268 value, err = strconv.ParseInt(data, 10, 64) 269 if err != nil { 270 log.Panic("invalid column value for enum", zap.Any("col", result), zap.Error(err)) 271 } 272 case mysql.TypeLonglong: 273 value, err = strconv.ParseInt(data, 10, 64) 274 if err != nil { 275 value, err = strconv.ParseUint(data, 10, 64) 276 if err != nil { 277 log.Panic("invalid column value for bigint", zap.Any("col", result), zap.Error(err)) 278 } 279 } 280 case mysql.TypeFloat: 281 value, err = strconv.ParseFloat(data, 32) 282 if err != nil { 283 log.Panic("invalid column value for float", zap.Any("col", result), zap.Error(err)) 284 } 285 case mysql.TypeDouble: 286 value, err = strconv.ParseFloat(data, 64) 287 if err != nil { 288 log.Panic("invalid column value for double", zap.Any("col", result), zap.Error(err)) 289 } 290 } 291 292 result.Value = value 293 return result 294 } 295 296 func canalJSONMessage2DDLEvent(msg canalJSONMessageInterface) *model.DDLEvent { 297 result := new(model.DDLEvent) 298 // we lost the startTs from kafka message 299 result.CommitTs = msg.getCommitTs() 300 301 result.TableInfo = new(model.TableInfo) 302 result.TableInfo.TableName = model.TableName{ 303 Schema: *msg.getSchema(), 304 Table: *msg.getTable(), 305 } 306 307 // we lost DDL type from canal json format, only got the DDL SQL. 308 result.Query = msg.getQuery() 309 310 // hack the DDL Type to be compatible with MySQL sink's logic 311 // see https://github.com/pingcap/tiflow/blob/0578db337d/cdc/sink/mysql.go#L362-L370 312 result.Type = getDDLActionType(result.Query) 313 return result 314 } 315 316 // return DDL ActionType by the prefix 317 // see https://github.com/pingcap/tidb/blob/6dbf2de2f/parser/model/ddl.go#L101-L102 318 func getDDLActionType(query string) timodel.ActionType { 319 query = strings.ToLower(query) 320 if strings.HasPrefix(query, "create schema") || strings.HasPrefix(query, "create database") { 321 return timodel.ActionCreateSchema 322 } 323 if strings.HasPrefix(query, "drop schema") || strings.HasPrefix(query, "drop database") { 324 return timodel.ActionDropSchema 325 } 326 327 return timodel.ActionNone 328 }