github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/avro/decoder.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 avro 15 16 import ( 17 "context" 18 "encoding/binary" 19 "encoding/json" 20 "fmt" 21 "strconv" 22 "strings" 23 24 "github.com/pingcap/log" 25 "github.com/pingcap/tidb/pkg/parser/mysql" 26 "github.com/pingcap/tidb/pkg/types" 27 "github.com/pingcap/tiflow/cdc/model" 28 "github.com/pingcap/tiflow/pkg/errors" 29 "github.com/pingcap/tiflow/pkg/sink/codec" 30 "github.com/pingcap/tiflow/pkg/sink/codec/common" 31 "go.uber.org/zap" 32 ) 33 34 type decoder struct { 35 config *common.Config 36 topic string 37 38 schemaM SchemaManager 39 40 key []byte 41 value []byte 42 } 43 44 // NewDecoder return an avro decoder 45 func NewDecoder( 46 config *common.Config, 47 schemaM SchemaManager, 48 topic string, 49 ) codec.RowEventDecoder { 50 return &decoder{ 51 config: config, 52 topic: topic, 53 schemaM: schemaM, 54 } 55 } 56 57 func (d *decoder) AddKeyValue(key, value []byte) error { 58 if d.key != nil || d.value != nil { 59 return errors.New("key or value is not nil") 60 } 61 d.key = key 62 d.value = value 63 return nil 64 } 65 66 func (d *decoder) HasNext() (model.MessageType, bool, error) { 67 if d.key == nil && d.value == nil { 68 return model.MessageTypeUnknown, false, nil 69 } 70 71 // it must a row event. 72 if d.key != nil { 73 return model.MessageTypeRow, true, nil 74 } 75 if len(d.value) < 1 { 76 return model.MessageTypeUnknown, false, errors.ErrAvroInvalidMessage.FastGenByArgs(d.value) 77 } 78 switch d.value[0] { 79 case magicByte: 80 return model.MessageTypeRow, true, nil 81 case ddlByte: 82 return model.MessageTypeDDL, true, nil 83 case checkpointByte: 84 return model.MessageTypeResolved, true, nil 85 } 86 return model.MessageTypeUnknown, false, errors.ErrAvroInvalidMessage.FastGenByArgs(d.value) 87 } 88 89 // NextRowChangedEvent returns the next row changed event if exists 90 func (d *decoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { 91 var ( 92 valueMap map[string]interface{} 93 valueSchema map[string]interface{} 94 err error 95 ) 96 97 ctx := context.Background() 98 keyMap, keySchema, err := d.decodeKey(ctx) 99 if err != nil { 100 return nil, errors.Trace(err) 101 } 102 103 // for the delete event, only have key part, it holds primary key or the unique key columns. 104 // for the insert / update, extract the value part, it holds all columns. 105 isDelete := len(d.value) == 0 106 if isDelete { 107 // delete event only have key part, treat it as the value part also. 108 valueMap = keyMap 109 valueSchema = keySchema 110 } else { 111 valueMap, valueSchema, err = d.decodeValue(ctx) 112 if err != nil { 113 return nil, errors.Trace(err) 114 } 115 } 116 117 event, err := assembleEvent(keyMap, valueMap, valueSchema, isDelete) 118 if err != nil { 119 return nil, errors.Trace(err) 120 } 121 122 // Delete event only has Primary Key Columns, but the checksum is calculated based on the whole row columns, 123 // checksum verification cannot be done here, so skip it. 124 if isDelete { 125 return event, nil 126 } 127 128 expectedChecksum, found, err := extractExpectedChecksum(valueMap) 129 if err != nil { 130 return nil, errors.Trace(err) 131 } 132 133 if isCorrupted(valueMap) { 134 log.Warn("row data is corrupted", 135 zap.String("topic", d.topic), zap.Uint64("checksum", expectedChecksum)) 136 for _, col := range event.Columns { 137 colInfo := event.TableInfo.ForceGetColumnInfo(col.ColumnID) 138 log.Info("data corrupted, print each column for debugging", 139 zap.String("name", colInfo.Name.O), 140 zap.Any("type", colInfo.GetType()), 141 zap.Any("charset", colInfo.GetCharset()), 142 zap.Any("flag", colInfo.GetFlag()), 143 zap.Any("value", col.Value), 144 zap.Any("default", colInfo.GetDefaultValue())) 145 } 146 } 147 148 if found { 149 if err := common.VerifyChecksum(event.Columns, event.TableInfo.Columns, uint32(expectedChecksum)); err != nil { 150 return nil, errors.Trace(err) 151 } 152 } 153 154 return event, nil 155 } 156 157 // assembleEvent return a row changed event 158 // keyMap hold primary key or unique key columns 159 // valueMap hold all columns information 160 // schema is corresponding to the valueMap, it can be used to decode the valueMap to construct columns. 161 func assembleEvent( 162 keyMap, valueMap, schema map[string]interface{}, isDelete bool, 163 ) (*model.RowChangedEvent, error) { 164 fields, ok := schema["fields"].([]interface{}) 165 if !ok { 166 return nil, errors.New("schema fields should be a map") 167 } 168 169 columns := make([]*model.Column, 0, len(valueMap)) 170 // fields is ordered by the column id, so iterate over it to build columns 171 // it's also the order to calculate the checksum. 172 for _, item := range fields { 173 field, ok := item.(map[string]interface{}) 174 if !ok { 175 return nil, errors.New("schema field should be a map") 176 } 177 178 // `tidbOp` is the first extension field in the schema, 179 // it's not real columns, so break here. 180 colName := field["name"].(string) 181 if colName == tidbOp { 182 break 183 } 184 185 // query the field to get `tidbType`, and get the mysql type from it. 186 var holder map[string]interface{} 187 switch ty := field["type"].(type) { 188 case []interface{}: 189 if m, ok := ty[0].(map[string]interface{}); ok { 190 holder = m["connect.parameters"].(map[string]interface{}) 191 } else if m, ok := ty[1].(map[string]interface{}); ok { 192 holder = m["connect.parameters"].(map[string]interface{}) 193 } else { 194 log.Panic("type info is anything else", zap.Any("typeInfo", field["type"])) 195 } 196 case map[string]interface{}: 197 holder = ty["connect.parameters"].(map[string]interface{}) 198 default: 199 log.Panic("type info is anything else", zap.Any("typeInfo", field["type"])) 200 } 201 tidbType := holder["tidb_type"].(string) 202 203 mysqlType := mysqlTypeFromTiDBType(tidbType) 204 205 flag := flagFromTiDBType(tidbType) 206 if _, ok := keyMap[colName]; ok { 207 flag.SetIsHandleKey() 208 flag.SetIsPrimaryKey() 209 } 210 211 value, ok := valueMap[colName] 212 if !ok { 213 return nil, errors.New("value not found") 214 } 215 value, err := getColumnValue(value, holder, mysqlType) 216 if err != nil { 217 return nil, errors.Trace(err) 218 } 219 220 col := &model.Column{ 221 Name: colName, 222 Type: mysqlType, 223 Flag: flag, 224 Value: value, 225 } 226 columns = append(columns, col) 227 } 228 229 // "namespace.schema" 230 namespace := schema["namespace"].(string) 231 schemaName := strings.Split(namespace, ".")[1] 232 tableName := schema["name"].(string) 233 234 var commitTs int64 235 if !isDelete { 236 o, ok := valueMap[tidbCommitTs] 237 if !ok { 238 return nil, errors.New("commit ts not found") 239 } 240 commitTs = o.(int64) 241 } 242 243 event := new(model.RowChangedEvent) 244 event.CommitTs = uint64(commitTs) 245 pkNameSet := make(map[string]struct{}, len(keyMap)) 246 for name := range keyMap { 247 pkNameSet[name] = struct{}{} 248 } 249 event.TableInfo = model.BuildTableInfoWithPKNames4Test(schemaName, tableName, columns, pkNameSet) 250 251 if isDelete { 252 event.PreColumns = model.Columns2ColumnDatas(columns, event.TableInfo) 253 } else { 254 event.Columns = model.Columns2ColumnDatas(columns, event.TableInfo) 255 } 256 257 return event, nil 258 } 259 260 func isCorrupted(valueMap map[string]interface{}) bool { 261 o, ok := valueMap[tidbCorrupted] 262 if !ok { 263 return false 264 } 265 266 corrupted := o.(bool) 267 return corrupted 268 } 269 270 // extract the checksum from the received value map 271 // return true if the checksum found, and return error if the checksum is not valid 272 func extractExpectedChecksum(valueMap map[string]interface{}) (uint64, bool, error) { 273 o, ok := valueMap[tidbRowLevelChecksum] 274 if !ok { 275 return 0, false, nil 276 } 277 checksum := o.(string) 278 if checksum == "" { 279 return 0, false, nil 280 } 281 result, err := strconv.ParseUint(checksum, 10, 64) 282 if err != nil { 283 return 0, true, errors.Trace(err) 284 } 285 return result, true, nil 286 } 287 288 // value is an interface, need to convert it to the real value with the help of type info. 289 // holder has the value's column info. 290 func getColumnValue( 291 value interface{}, holder map[string]interface{}, mysqlType byte, 292 ) (interface{}, error) { 293 switch t := value.(type) { 294 // for nullable columns, the value is encoded as a map with one pair. 295 // key is the encoded type, value is the encoded value, only care about the value here. 296 case map[string]interface{}: 297 for _, v := range t { 298 value = v 299 } 300 } 301 if value == nil { 302 return nil, nil 303 } 304 305 switch mysqlType { 306 case mysql.TypeEnum: 307 // enum type is encoded as string, 308 // we need to convert it to int by the order of the enum values definition. 309 allowed := strings.Split(holder["allowed"].(string), ",") 310 enum, err := types.ParseEnum(allowed, value.(string), "") 311 if err != nil { 312 return nil, errors.Trace(err) 313 } 314 value = enum.Value 315 case mysql.TypeSet: 316 // set type is encoded as string, 317 // we need to convert it to int by the order of the set values definition. 318 elems := strings.Split(holder["allowed"].(string), ",") 319 s, err := types.ParseSet(elems, value.(string), "") 320 if err != nil { 321 return nil, errors.Trace(err) 322 } 323 value = s.Value 324 } 325 return value, nil 326 } 327 328 // NextResolvedEvent returns the next resolved event if exists 329 func (d *decoder) NextResolvedEvent() (uint64, error) { 330 if len(d.value) == 0 { 331 return 0, errors.New("value should not be empty") 332 } 333 ts := binary.BigEndian.Uint64(d.value[1:]) 334 d.value = nil 335 return ts, nil 336 } 337 338 // NextDDLEvent returns the next DDL event if exists 339 func (d *decoder) NextDDLEvent() (*model.DDLEvent, error) { 340 if len(d.value) == 0 { 341 return nil, errors.New("value should not be empty") 342 } 343 if d.value[0] != ddlByte { 344 return nil, fmt.Errorf("first byte is not the ddl byte, but got: %+v", d.value[0]) 345 } 346 347 data := d.value[1:] 348 var baseDDLEvent ddlEvent 349 err := json.Unmarshal(data, &baseDDLEvent) 350 if err != nil { 351 return nil, errors.WrapError(errors.ErrDecodeFailed, err) 352 } 353 d.value = nil 354 355 result := new(model.DDLEvent) 356 result.TableInfo = new(model.TableInfo) 357 result.CommitTs = baseDDLEvent.CommitTs 358 result.TableInfo.TableName = model.TableName{ 359 Schema: baseDDLEvent.Schema, 360 Table: baseDDLEvent.Table, 361 } 362 result.Type = baseDDLEvent.Type 363 result.Query = baseDDLEvent.Query 364 365 return result, nil 366 } 367 368 // return the schema ID and the encoded binary data 369 // schemaID can be used to fetch the corresponding schema from schema registry, 370 // which should be used to decode the binary data. 371 func extractConfluentSchemaIDAndBinaryData(data []byte) (int, []byte, error) { 372 if len(data) < 5 { 373 return 0, nil, errors.ErrAvroInvalidMessage. 374 FastGenByArgs("an avro message using confluent schema registry should have at least 5 bytes") 375 } 376 if data[0] != magicByte { 377 return 0, nil, errors.ErrAvroInvalidMessage. 378 FastGenByArgs("magic byte is not match, it should be 0") 379 } 380 id, err := getConfluentSchemaIDFromHeader(data[0:5]) 381 if err != nil { 382 return 0, nil, errors.Trace(err) 383 } 384 return int(id), data[5:], nil 385 } 386 387 func extractGlueSchemaIDAndBinaryData(data []byte) (string, []byte, error) { 388 if len(data) < 18 { 389 return "", nil, errors.ErrAvroInvalidMessage. 390 FastGenByArgs("an avro message using glue schema registry should have at least 18 bytes") 391 } 392 if data[0] != headerVersionByte { 393 return "", nil, errors.ErrAvroInvalidMessage. 394 FastGenByArgs("header version byte is not match, it should be %d", headerVersionByte) 395 } 396 if data[1] != compressionDefaultByte { 397 return "", nil, errors.ErrAvroInvalidMessage. 398 FastGenByArgs("compression byte is not match, it should be %d", compressionDefaultByte) 399 } 400 id, err := getGlueSchemaIDFromHeader(data[0:18]) 401 if err != nil { 402 return "", nil, errors.Trace(err) 403 } 404 return id, data[18:], nil 405 } 406 407 func decodeRawBytes( 408 ctx context.Context, schemaM SchemaManager, data []byte, topic string, 409 ) (map[string]interface{}, map[string]interface{}, error) { 410 var schemaID schemaID 411 var binary []byte 412 var err error 413 var cid int 414 var gid string 415 416 switch schemaM.RegistryType() { 417 case common.SchemaRegistryTypeConfluent: 418 cid, binary, err = extractConfluentSchemaIDAndBinaryData(data) 419 if err != nil { 420 return nil, nil, err 421 } 422 schemaID.confluentSchemaID = cid 423 case common.SchemaRegistryTypeGlue: 424 gid, binary, err = extractGlueSchemaIDAndBinaryData(data) 425 if err != nil { 426 return nil, nil, err 427 } 428 schemaID.glueSchemaID = gid 429 default: 430 return nil, nil, errors.New("unknown schema registry type") 431 } 432 433 codec, err := schemaM.Lookup(ctx, topic, schemaID) 434 if err != nil { 435 return nil, nil, err 436 } 437 438 native, _, err := codec.NativeFromBinary(binary) 439 if err != nil { 440 return nil, nil, err 441 } 442 443 result, ok := native.(map[string]interface{}) 444 if !ok { 445 return nil, nil, errors.New("raw avro message is not a map") 446 } 447 448 schema := make(map[string]interface{}) 449 if err := json.Unmarshal([]byte(codec.Schema()), &schema); err != nil { 450 return nil, nil, errors.Trace(err) 451 } 452 453 return result, schema, nil 454 } 455 456 func (d *decoder) decodeKey(ctx context.Context) (map[string]interface{}, map[string]interface{}, error) { 457 data := d.key 458 d.key = nil 459 return decodeRawBytes(ctx, d.schemaM, data, d.topic) 460 } 461 462 func (d *decoder) decodeValue(ctx context.Context) (map[string]interface{}, map[string]interface{}, error) { 463 data := d.value 464 d.value = nil 465 return decodeRawBytes(ctx, d.schemaM, data, d.topic) 466 }