github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/avro/avro.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 "bytes" 18 "context" 19 "encoding/binary" 20 "encoding/json" 21 "math/big" 22 "sort" 23 "strconv" 24 "strings" 25 26 "github.com/linkedin/goavro/v2" 27 "github.com/pingcap/errors" 28 "github.com/pingcap/log" 29 timodel "github.com/pingcap/tidb/pkg/parser/model" 30 "github.com/pingcap/tidb/pkg/parser/mysql" 31 "github.com/pingcap/tidb/pkg/types" 32 "github.com/pingcap/tidb/pkg/util/rowcodec" 33 "github.com/pingcap/tiflow/cdc/model" 34 "github.com/pingcap/tiflow/pkg/config" 35 cerror "github.com/pingcap/tiflow/pkg/errors" 36 "github.com/pingcap/tiflow/pkg/sink/codec" 37 "github.com/pingcap/tiflow/pkg/sink/codec/common" 38 "github.com/tikv/client-go/v2/oracle" 39 "go.uber.org/zap" 40 ) 41 42 // BatchEncoder converts the events to binary Avro data 43 type BatchEncoder struct { 44 namespace string 45 schemaM SchemaManager 46 result []*common.Message 47 48 config *common.Config 49 } 50 51 type avroEncodeInput struct { 52 columns []*model.Column 53 colInfos []rowcodec.ColInfo 54 } 55 56 func (r *avroEncodeInput) Less(i, j int) bool { 57 return r.colInfos[i].ID < r.colInfos[j].ID 58 } 59 60 func (r *avroEncodeInput) Len() int { 61 return len(r.columns) 62 } 63 64 func (r *avroEncodeInput) Swap(i, j int) { 65 r.colInfos[i], r.colInfos[j] = r.colInfos[j], r.colInfos[i] 66 r.columns[i], r.columns[j] = r.columns[j], r.columns[i] 67 } 68 69 type avroEncodeResult struct { 70 data []byte 71 // header is the message header, it will be encoder into the head 72 // of every single avro message. Note: Confluent schema registry and 73 // Aws Glue schema registry have different header format. 74 header []byte 75 } 76 77 func (a *BatchEncoder) encodeKey(ctx context.Context, topic string, e *model.RowChangedEvent) ([]byte, error) { 78 cols, colInfos := e.HandleKeyColInfos() 79 // result may be nil if the event has no handle key columns, this may happen in the force replicate mode. 80 // todo: disallow force replicate mode if using the avro. 81 if len(cols) == 0 { 82 return nil, nil 83 } 84 85 keyColumns := &avroEncodeInput{ 86 columns: cols, 87 colInfos: colInfos, 88 } 89 avroCodec, header, err := a.getKeySchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, keyColumns) 90 if err != nil { 91 return nil, errors.Trace(err) 92 } 93 94 native, err := a.columns2AvroData(keyColumns) 95 if err != nil { 96 log.Error("avro: key converting to native failed", zap.Error(err)) 97 return nil, errors.Trace(err) 98 } 99 100 bin, err := avroCodec.BinaryFromNative(nil, native) 101 if err != nil { 102 log.Error("avro: key converting to Avro binary failed", zap.Error(err)) 103 return nil, cerror.WrapError(cerror.ErrAvroEncodeToBinary, err) 104 } 105 106 result := &avroEncodeResult{ 107 data: bin, 108 header: header, 109 } 110 data, err := result.toEnvelope() 111 if err != nil { 112 return nil, errors.Trace(err) 113 } 114 return data, nil 115 } 116 117 func topicName2SchemaSubjects(topicName, subjectSuffix string) string { 118 return topicName + subjectSuffix 119 } 120 121 func (a *BatchEncoder) getValueSchemaCodec( 122 ctx context.Context, topic string, tableName *model.TableName, tableVersion uint64, input *avroEncodeInput, 123 ) (*goavro.Codec, []byte, error) { 124 schemaGen := func() (string, error) { 125 schema, err := a.value2AvroSchema(tableName, input) 126 if err != nil { 127 log.Error("avro: generating value schema failed", zap.Error(err)) 128 return "", errors.Trace(err) 129 } 130 return schema, nil 131 } 132 133 subject := topicName2SchemaSubjects(topic, valueSchemaSuffix) 134 avroCodec, header, err := a.schemaM.GetCachedOrRegister(ctx, subject, tableVersion, schemaGen) 135 if err != nil { 136 return nil, nil, errors.Trace(err) 137 } 138 return avroCodec, header, nil 139 } 140 141 func (a *BatchEncoder) getKeySchemaCodec( 142 ctx context.Context, topic string, tableName *model.TableName, tableVersion uint64, keyColumns *avroEncodeInput, 143 ) (*goavro.Codec, []byte, error) { 144 schemaGen := func() (string, error) { 145 schema, err := a.key2AvroSchema(tableName, keyColumns) 146 if err != nil { 147 log.Error("AvroEventBatchEncoder: generating key schema failed", zap.Error(err)) 148 return "", errors.Trace(err) 149 } 150 return schema, nil 151 } 152 153 subject := topicName2SchemaSubjects(topic, keySchemaSuffix) 154 avroCodec, header, err := a.schemaM.GetCachedOrRegister(ctx, subject, tableVersion, schemaGen) 155 if err != nil { 156 return nil, nil, errors.Trace(err) 157 } 158 return avroCodec, header, nil 159 } 160 161 func (a *BatchEncoder) encodeValue(ctx context.Context, topic string, e *model.RowChangedEvent) ([]byte, error) { 162 if e.IsDelete() { 163 return nil, nil 164 } 165 166 input := &avroEncodeInput{ 167 columns: e.GetColumns(), 168 colInfos: e.TableInfo.GetColInfosForRowChangedEvent(), 169 } 170 if len(input.columns) == 0 { 171 return nil, nil 172 } 173 174 avroCodec, header, err := a.getValueSchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, input) 175 if err != nil { 176 return nil, errors.Trace(err) 177 } 178 179 native, err := a.columns2AvroData(input) 180 if err != nil { 181 log.Error("avro: converting value to native failed", zap.Error(err)) 182 return nil, errors.Trace(err) 183 } 184 if a.config.EnableTiDBExtension { 185 native = a.nativeValueWithExtension(native, e) 186 } 187 188 bin, err := avroCodec.BinaryFromNative(nil, native) 189 if err != nil { 190 log.Error("avro: converting value to Avro binary failed", zap.Error(err)) 191 return nil, cerror.WrapError(cerror.ErrAvroEncodeToBinary, err) 192 } 193 194 result := &avroEncodeResult{ 195 data: bin, 196 header: header, 197 } 198 data, err := result.toEnvelope() 199 if err != nil { 200 return nil, errors.Trace(err) 201 } 202 return data, nil 203 } 204 205 // AppendRowChangedEvent appends a row change event to the encoder 206 // NOTE: the encoder can only store one RowChangedEvent! 207 func (a *BatchEncoder) AppendRowChangedEvent( 208 ctx context.Context, 209 topic string, 210 e *model.RowChangedEvent, 211 callback func(), 212 ) error { 213 topic = sanitizeTopic(topic) 214 215 key, err := a.encodeKey(ctx, topic, e) 216 if err != nil { 217 log.Error("avro encoding key failed", zap.Error(err), zap.Any("event", e)) 218 return errors.Trace(err) 219 } 220 221 value, err := a.encodeValue(ctx, topic, e) 222 if err != nil { 223 log.Error("avro encoding value failed", zap.Error(err), zap.Any("event", e)) 224 return errors.Trace(err) 225 } 226 227 message := common.NewMsg( 228 config.ProtocolAvro, 229 key, 230 value, 231 e.CommitTs, 232 model.MessageTypeRow, 233 e.TableInfo.GetSchemaNamePtr(), 234 e.TableInfo.GetTableNamePtr(), 235 ) 236 message.Callback = callback 237 message.IncRowsCount() 238 239 if message.Length() > a.config.MaxMessageBytes { 240 log.Warn("Single message is too large for avro", 241 zap.Int("maxMessageBytes", a.config.MaxMessageBytes), 242 zap.Int("length", message.Length()), 243 zap.Any("table", e.TableInfo.TableName)) 244 return cerror.ErrMessageTooLarge.GenWithStackByArgs(message.Length()) 245 } 246 247 a.result = append(a.result, message) 248 return nil 249 } 250 251 // EncodeCheckpointEvent only encode checkpoint event if the watermark event is enabled 252 // it's only used for the testing purpose. 253 func (a *BatchEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { 254 if a.config.EnableTiDBExtension && a.config.AvroEnableWatermark { 255 buf := new(bytes.Buffer) 256 data := []interface{}{checkpointByte, ts} 257 for _, v := range data { 258 err := binary.Write(buf, binary.BigEndian, v) 259 if err != nil { 260 return nil, cerror.WrapError(cerror.ErrAvroToEnvelopeError, err) 261 } 262 } 263 264 value := buf.Bytes() 265 return common.NewResolvedMsg(config.ProtocolAvro, nil, value, ts), nil 266 } 267 return nil, nil 268 } 269 270 type ddlEvent struct { 271 Query string `json:"query"` 272 Type timodel.ActionType `json:"type"` 273 Schema string `json:"schema"` 274 Table string `json:"table"` 275 CommitTs uint64 `json:"commitTs"` 276 } 277 278 // EncodeDDLEvent only encode DDL event if the watermark event is enabled 279 // it's only used for the testing purpose. 280 func (a *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { 281 if a.config.EnableTiDBExtension && a.config.AvroEnableWatermark { 282 buf := new(bytes.Buffer) 283 _ = binary.Write(buf, binary.BigEndian, ddlByte) 284 285 event := &ddlEvent{ 286 Query: e.Query, 287 Type: e.Type, 288 Schema: e.TableInfo.TableName.Schema, 289 Table: e.TableInfo.TableName.Table, 290 CommitTs: e.CommitTs, 291 } 292 data, err := json.Marshal(event) 293 if err != nil { 294 return nil, cerror.WrapError(cerror.ErrAvroToEnvelopeError, err) 295 } 296 buf.Write(data) 297 298 value := buf.Bytes() 299 return common.NewDDLMsg(config.ProtocolAvro, nil, value, e), nil 300 } 301 302 return nil, nil 303 } 304 305 // Build Messages 306 func (a *BatchEncoder) Build() (messages []*common.Message) { 307 result := a.result 308 a.result = nil 309 return result 310 } 311 312 const ( 313 insertOperation = "c" 314 updateOperation = "u" 315 ) 316 317 func getOperation(e *model.RowChangedEvent) string { 318 if e.IsInsert() { 319 return insertOperation 320 } else if e.IsUpdate() { 321 return updateOperation 322 } 323 return "" 324 } 325 326 func (a *BatchEncoder) nativeValueWithExtension( 327 native map[string]interface{}, 328 e *model.RowChangedEvent, 329 ) map[string]interface{} { 330 native[tidbOp] = getOperation(e) 331 native[tidbCommitTs] = int64(e.CommitTs) 332 native[tidbPhysicalTime] = oracle.ExtractPhysical(e.CommitTs) 333 334 if a.config.EnableRowChecksum && e.Checksum != nil { 335 native[tidbRowLevelChecksum] = strconv.FormatUint(uint64(e.Checksum.Current), 10) 336 native[tidbCorrupted] = e.Checksum.Corrupted 337 native[tidbChecksumVersion] = e.Checksum.Version 338 } 339 return native 340 } 341 342 type avroSchemaTop struct { 343 Tp string `json:"type"` 344 Name string `json:"name"` 345 Namespace string `json:"namespace"` 346 Fields []map[string]interface{} `json:"fields"` 347 } 348 349 const ( 350 tidbType = "tidb_type" 351 tidbOp = "_tidb_op" 352 tidbCommitTs = "_tidb_commit_ts" 353 tidbPhysicalTime = "_tidb_commit_physical_time" 354 355 // row level checksum related fields 356 tidbRowLevelChecksum = "_tidb_row_level_checksum" 357 tidbChecksumVersion = "_tidb_checksum_version" 358 tidbCorrupted = "_tidb_corrupted" 359 ) 360 361 var type2TiDBType = map[byte]string{ 362 mysql.TypeTiny: "INT", 363 mysql.TypeShort: "INT", 364 mysql.TypeInt24: "INT", 365 mysql.TypeLong: "INT", 366 mysql.TypeLonglong: "BIGINT", 367 mysql.TypeFloat: "FLOAT", 368 mysql.TypeDouble: "DOUBLE", 369 mysql.TypeBit: "BIT", 370 mysql.TypeNewDecimal: "DECIMAL", 371 mysql.TypeTinyBlob: "TEXT", 372 mysql.TypeMediumBlob: "TEXT", 373 mysql.TypeBlob: "TEXT", 374 mysql.TypeLongBlob: "TEXT", 375 mysql.TypeVarchar: "TEXT", 376 mysql.TypeVarString: "TEXT", 377 mysql.TypeString: "TEXT", 378 mysql.TypeEnum: "ENUM", 379 mysql.TypeSet: "SET", 380 mysql.TypeJSON: "JSON", 381 mysql.TypeDate: "DATE", 382 mysql.TypeDatetime: "DATETIME", 383 mysql.TypeTimestamp: "TIMESTAMP", 384 mysql.TypeDuration: "TIME", 385 mysql.TypeYear: "YEAR", 386 } 387 388 func getTiDBTypeFromColumn(col *model.Column) string { 389 tt := type2TiDBType[col.Type] 390 if col.Flag.IsUnsigned() && (tt == "INT" || tt == "BIGINT") { 391 return tt + " UNSIGNED" 392 } 393 if col.Flag.IsBinary() && tt == "TEXT" { 394 return "BLOB" 395 } 396 return tt 397 } 398 399 func flagFromTiDBType(tp string) model.ColumnFlagType { 400 var flag model.ColumnFlagType 401 if strings.Contains(tp, "UNSIGNED") { 402 flag.SetIsUnsigned() 403 } 404 return flag 405 } 406 407 func mysqlTypeFromTiDBType(tidbType string) byte { 408 var result byte 409 switch tidbType { 410 case "INT", "INT UNSIGNED": 411 result = mysql.TypeLong 412 case "BIGINT", "BIGINT UNSIGNED": 413 result = mysql.TypeLonglong 414 case "FLOAT": 415 result = mysql.TypeFloat 416 case "DOUBLE": 417 result = mysql.TypeDouble 418 case "BIT": 419 result = mysql.TypeBit 420 case "DECIMAL": 421 result = mysql.TypeNewDecimal 422 case "TEXT": 423 result = mysql.TypeVarchar 424 case "BLOB": 425 result = mysql.TypeLongBlob 426 case "ENUM": 427 result = mysql.TypeEnum 428 case "SET": 429 result = mysql.TypeSet 430 case "JSON": 431 result = mysql.TypeJSON 432 case "DATE": 433 result = mysql.TypeDate 434 case "DATETIME": 435 result = mysql.TypeDatetime 436 case "TIMESTAMP": 437 result = mysql.TypeTimestamp 438 case "TIME": 439 result = mysql.TypeDuration 440 case "YEAR": 441 result = mysql.TypeYear 442 default: 443 log.Panic("this should not happen, unknown TiDB type", zap.String("type", tidbType)) 444 } 445 return result 446 } 447 448 const ( 449 replacementChar = "_" 450 numberPrefix = "_" 451 ) 452 453 // sanitizeName escapes not permitted chars for avro 454 // debezium-core/src/main/java/io/debezium/schema/FieldNameSelector.java 455 // https://avro.apache.org/docs/current/spec.html#names 456 func sanitizeName(name string) string { 457 changed := false 458 var sb strings.Builder 459 for i, c := range name { 460 if i == 0 && (c >= '0' && c <= '9') { 461 sb.WriteString(numberPrefix) 462 sb.WriteRune(c) 463 changed = true 464 } else if !(c == '_' || 465 ('a' <= c && c <= 'z') || 466 ('A' <= c && c <= 'Z') || 467 ('0' <= c && c <= '9')) { 468 sb.WriteString(replacementChar) 469 changed = true 470 } else { 471 sb.WriteRune(c) 472 } 473 } 474 475 sanitizedName := sb.String() 476 if changed { 477 log.Warn( 478 "Name is potentially not safe for serialization, replace it", 479 zap.String("name", name), 480 zap.String("replacedName", sanitizedName), 481 ) 482 } 483 return sanitizedName 484 } 485 486 // sanitizeTopic escapes ".", it may have special meanings for sink connectors 487 func sanitizeTopic(name string) string { 488 return strings.ReplaceAll(name, ".", replacementChar) 489 } 490 491 // https://github.com/debezium/debezium/blob/9f7ede0e0695f012c6c4e715e96aed85eecf6b5f \ 492 // /debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/antlr/ \ 493 // MySqlAntlrDdlParser.java#L374 494 func escapeEnumAndSetOptions(option string) string { 495 option = strings.ReplaceAll(option, ",", "\\,") 496 option = strings.ReplaceAll(option, "\\'", "'") 497 option = strings.ReplaceAll(option, "''", "'") 498 return option 499 } 500 501 func getAvroNamespace(namespace string, schema string) string { 502 return sanitizeName(namespace) + "." + sanitizeName(schema) 503 } 504 505 type avroSchema struct { 506 Type string `json:"type"` 507 // connect.parameters is designated field extracted by schema registry 508 Parameters map[string]string `json:"connect.parameters"` 509 } 510 511 type avroLogicalTypeSchema struct { 512 avroSchema 513 LogicalType string `json:"logicalType"` 514 Precision interface{} `json:"precision,omitempty"` 515 Scale interface{} `json:"scale,omitempty"` 516 } 517 518 func (a *BatchEncoder) schemaWithExtension( 519 top *avroSchemaTop, 520 ) *avroSchemaTop { 521 top.Fields = append(top.Fields, 522 map[string]interface{}{ 523 "name": tidbOp, 524 "type": "string", 525 "default": "", 526 }, 527 map[string]interface{}{ 528 "name": tidbCommitTs, 529 "type": "long", 530 "default": 0, 531 }, 532 map[string]interface{}{ 533 "name": tidbPhysicalTime, 534 "type": "long", 535 "default": 0, 536 }, 537 ) 538 539 if a.config.EnableRowChecksum { 540 top.Fields = append(top.Fields, 541 map[string]interface{}{ 542 "name": tidbRowLevelChecksum, 543 "type": "string", 544 "default": "", 545 }, 546 map[string]interface{}{ 547 "name": tidbCorrupted, 548 "type": "boolean", 549 "default": false, 550 }, 551 map[string]interface{}{ 552 "name": tidbChecksumVersion, 553 "type": "int", 554 "default": 0, 555 }) 556 } 557 558 return top 559 } 560 561 func (a *BatchEncoder) columns2AvroSchema( 562 tableName *model.TableName, 563 input *avroEncodeInput, 564 ) (*avroSchemaTop, error) { 565 top := &avroSchemaTop{ 566 Tp: "record", 567 Name: sanitizeName(tableName.Table), 568 Namespace: getAvroNamespace(a.namespace, tableName.Schema), 569 Fields: nil, 570 } 571 for i, col := range input.columns { 572 if col == nil { 573 continue 574 } 575 avroType, err := a.columnToAvroSchema(col, input.colInfos[i].Ft) 576 if err != nil { 577 return nil, err 578 } 579 field := make(map[string]interface{}) 580 field["name"] = sanitizeName(col.Name) 581 582 copied := *col 583 copied.Value = copied.Default 584 defaultValue, _, err := a.columnToAvroData(&copied, input.colInfos[i].Ft) 585 if err != nil { 586 log.Error("fail to get default value for avro schema") 587 return nil, errors.Trace(err) 588 } 589 // goavro doesn't support set default value for logical type 590 // https://github.com/linkedin/goavro/issues/202 591 if _, ok := avroType.(avroLogicalTypeSchema); ok { 592 if col.Flag.IsNullable() { 593 field["type"] = []interface{}{"null", avroType} 594 field["default"] = nil 595 } else { 596 field["type"] = avroType 597 } 598 } else { 599 if col.Flag.IsNullable() { 600 // https://stackoverflow.com/questions/22938124/avro-field-default-values 601 if defaultValue == nil { 602 field["type"] = []interface{}{"null", avroType} 603 } else { 604 field["type"] = []interface{}{avroType, "null"} 605 } 606 field["default"] = defaultValue 607 } else { 608 field["type"] = avroType 609 if defaultValue != nil { 610 field["default"] = defaultValue 611 } 612 } 613 } 614 top.Fields = append(top.Fields, field) 615 } 616 return top, nil 617 } 618 619 func (a *BatchEncoder) value2AvroSchema( 620 tableName *model.TableName, 621 input *avroEncodeInput, 622 ) (string, error) { 623 if a.config.EnableRowChecksum { 624 sort.Sort(input) 625 } 626 627 top, err := a.columns2AvroSchema(tableName, input) 628 if err != nil { 629 return "", err 630 } 631 632 if a.config.EnableTiDBExtension { 633 top = a.schemaWithExtension(top) 634 } 635 636 str, err := json.Marshal(top) 637 if err != nil { 638 return "", cerror.WrapError(cerror.ErrAvroMarshalFailed, err) 639 } 640 log.Info("avro: row to schema", 641 zap.ByteString("schema", str), 642 zap.Bool("enableTiDBExtension", a.config.EnableRowChecksum), 643 zap.Bool("enableRowLevelChecksum", a.config.EnableRowChecksum)) 644 return string(str), nil 645 } 646 647 func (a *BatchEncoder) key2AvroSchema( 648 tableName *model.TableName, 649 keyColumns *avroEncodeInput, 650 ) (string, error) { 651 top, err := a.columns2AvroSchema(tableName, keyColumns) 652 if err != nil { 653 return "", err 654 } 655 656 str, err := json.Marshal(top) 657 if err != nil { 658 return "", cerror.WrapError(cerror.ErrAvroMarshalFailed, err) 659 } 660 log.Info("avro: key to schema", zap.ByteString("schema", str)) 661 return string(str), nil 662 } 663 664 func (a *BatchEncoder) columns2AvroData( 665 input *avroEncodeInput, 666 ) (map[string]interface{}, error) { 667 ret := make(map[string]interface{}, len(input.columns)) 668 for i, col := range input.columns { 669 if col == nil { 670 continue 671 } 672 data, str, err := a.columnToAvroData(col, input.colInfos[i].Ft) 673 if err != nil { 674 return nil, err 675 } 676 677 // https: //pkg.go.dev/github.com/linkedin/goavro/v2#Union 678 if col.Flag.IsNullable() { 679 ret[sanitizeName(col.Name)] = goavro.Union(str, data) 680 } else { 681 ret[sanitizeName(col.Name)] = data 682 } 683 } 684 685 log.Debug("rowToAvroData", zap.Any("data", ret)) 686 return ret, nil 687 } 688 689 func (a *BatchEncoder) columnToAvroSchema( 690 col *model.Column, 691 ft *types.FieldType, 692 ) (interface{}, error) { 693 tt := getTiDBTypeFromColumn(col) 694 switch col.Type { 695 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: 696 // BOOL/TINYINT/SMALLINT/MEDIUMINT 697 return avroSchema{ 698 Type: "int", 699 Parameters: map[string]string{tidbType: tt}, 700 }, nil 701 case mysql.TypeLong: // INT 702 if col.Flag.IsUnsigned() { 703 return avroSchema{ 704 Type: "long", 705 Parameters: map[string]string{tidbType: tt}, 706 }, nil 707 } 708 return avroSchema{ 709 Type: "int", 710 Parameters: map[string]string{tidbType: tt}, 711 }, nil 712 case mysql.TypeLonglong: // BIGINT 713 t := "long" 714 if col.Flag.IsUnsigned() && 715 a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeString { 716 t = "string" 717 } 718 return avroSchema{ 719 Type: t, 720 Parameters: map[string]string{tidbType: tt}, 721 }, nil 722 case mysql.TypeFloat: 723 return avroSchema{ 724 Type: "float", 725 Parameters: map[string]string{tidbType: tt}, 726 }, nil 727 case mysql.TypeDouble: 728 return avroSchema{ 729 Type: "double", 730 Parameters: map[string]string{tidbType: tt}, 731 }, nil 732 case mysql.TypeBit: 733 displayFlen := ft.GetFlen() 734 if displayFlen == -1 { 735 displayFlen, _ = mysql.GetDefaultFieldLengthAndDecimal(col.Type) 736 } 737 return avroSchema{ 738 Type: "bytes", 739 Parameters: map[string]string{ 740 tidbType: tt, 741 "length": strconv.Itoa(displayFlen), 742 }, 743 }, nil 744 case mysql.TypeNewDecimal: 745 if a.config.AvroDecimalHandlingMode == common.DecimalHandlingModePrecise { 746 defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) 747 displayFlen, displayDecimal := ft.GetFlen(), ft.GetDecimal() 748 // length not specified, set it to system type default 749 if displayFlen == -1 { 750 displayFlen = defaultFlen 751 } 752 if displayDecimal == -1 { 753 displayDecimal = defaultDecimal 754 } 755 return avroLogicalTypeSchema{ 756 avroSchema: avroSchema{ 757 Type: "bytes", 758 Parameters: map[string]string{tidbType: tt}, 759 }, 760 LogicalType: "decimal", 761 Precision: displayFlen, 762 Scale: displayDecimal, 763 }, nil 764 } 765 // decimalHandlingMode == string 766 return avroSchema{ 767 Type: "string", 768 Parameters: map[string]string{tidbType: tt}, 769 }, nil 770 // TINYTEXT/MEDIUMTEXT/TEXT/LONGTEXT/CHAR/VARCHAR 771 // TINYBLOB/MEDIUMBLOB/BLOB/LONGBLOB/BINARY/VARBINARY 772 case mysql.TypeVarchar, 773 mysql.TypeString, 774 mysql.TypeVarString, 775 mysql.TypeTinyBlob, 776 mysql.TypeMediumBlob, 777 mysql.TypeLongBlob, 778 mysql.TypeBlob: 779 t := "string" 780 if col.Flag.IsBinary() { 781 t = "bytes" 782 } 783 return avroSchema{ 784 Type: t, 785 Parameters: map[string]string{tidbType: tt}, 786 }, nil 787 case mysql.TypeEnum, mysql.TypeSet: 788 es := make([]string, 0, len(ft.GetElems())) 789 for _, e := range ft.GetElems() { 790 e = escapeEnumAndSetOptions(e) 791 es = append(es, e) 792 } 793 return avroSchema{ 794 Type: "string", 795 Parameters: map[string]string{ 796 tidbType: tt, 797 "allowed": strings.Join(es, ","), 798 }, 799 }, nil 800 case mysql.TypeJSON: 801 return avroSchema{ 802 Type: "string", 803 Parameters: map[string]string{tidbType: tt}, 804 }, nil 805 case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDuration: 806 return avroSchema{ 807 Type: "string", 808 Parameters: map[string]string{tidbType: tt}, 809 }, nil 810 case mysql.TypeYear: 811 return avroSchema{ 812 Type: "int", 813 Parameters: map[string]string{tidbType: tt}, 814 }, nil 815 default: 816 log.Error("unknown mysql type", zap.Any("mysqlType", col.Type)) 817 return nil, cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") 818 } 819 } 820 821 func (a *BatchEncoder) columnToAvroData( 822 col *model.Column, 823 ft *types.FieldType, 824 ) (interface{}, string, error) { 825 if col.Value == nil { 826 return nil, "null", nil 827 } 828 829 switch col.Type { 830 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: 831 if v, ok := col.Value.(string); ok { 832 n, err := strconv.ParseInt(v, 10, 32) 833 if err != nil { 834 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 835 } 836 return int32(n), "int", nil 837 } 838 if col.Flag.IsUnsigned() { 839 return int32(col.Value.(uint64)), "int", nil 840 } 841 return int32(col.Value.(int64)), "int", nil 842 case mysql.TypeLong: 843 if v, ok := col.Value.(string); ok { 844 n, err := strconv.ParseInt(v, 10, 64) 845 if err != nil { 846 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 847 } 848 if col.Flag.IsUnsigned() { 849 return n, "long", nil 850 } 851 return int32(n), "int", nil 852 } 853 if col.Flag.IsUnsigned() { 854 return int64(col.Value.(uint64)), "long", nil 855 } 856 return int32(col.Value.(int64)), "int", nil 857 case mysql.TypeLonglong: 858 if v, ok := col.Value.(string); ok { 859 if col.Flag.IsUnsigned() { 860 if a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeString { 861 return v, "string", nil 862 } 863 n, err := strconv.ParseUint(v, 10, 64) 864 if err != nil { 865 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 866 } 867 return int64(n), "long", nil 868 } 869 n, err := strconv.ParseInt(v, 10, 64) 870 if err != nil { 871 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 872 } 873 return n, "long", nil 874 } 875 if col.Flag.IsUnsigned() { 876 if a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeLong { 877 return int64(col.Value.(uint64)), "long", nil 878 } 879 // bigintUnsignedHandlingMode == "string" 880 return strconv.FormatUint(col.Value.(uint64), 10), "string", nil 881 } 882 return col.Value.(int64), "long", nil 883 case mysql.TypeFloat: 884 if v, ok := col.Value.(string); ok { 885 n, err := strconv.ParseFloat(v, 32) 886 if err != nil { 887 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 888 } 889 return n, "float", nil 890 } 891 return col.Value.(float32), "float", nil 892 case mysql.TypeDouble: 893 if v, ok := col.Value.(string); ok { 894 n, err := strconv.ParseFloat(v, 64) 895 if err != nil { 896 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 897 } 898 return n, "double", nil 899 } 900 return col.Value.(float64), "double", nil 901 case mysql.TypeBit: 902 if v, ok := col.Value.(string); ok { 903 return []byte(v), "bytes", nil 904 } 905 return []byte(types.NewBinaryLiteralFromUint(col.Value.(uint64), -1)), "bytes", nil 906 case mysql.TypeNewDecimal: 907 if a.config.AvroDecimalHandlingMode == common.DecimalHandlingModePrecise { 908 v, succ := new(big.Rat).SetString(col.Value.(string)) 909 if !succ { 910 return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack( 911 "fail to encode Decimal value", 912 ) 913 } 914 return v, "bytes.decimal", nil 915 } 916 // decimalHandlingMode == "string" 917 return col.Value.(string), "string", nil 918 case mysql.TypeVarchar, 919 mysql.TypeString, 920 mysql.TypeVarString, 921 mysql.TypeTinyBlob, 922 mysql.TypeBlob, 923 mysql.TypeMediumBlob, 924 mysql.TypeLongBlob: 925 if col.Flag.IsBinary() { 926 if v, ok := col.Value.(string); ok { 927 return []byte(v), "bytes", nil 928 } 929 return col.Value, "bytes", nil 930 } 931 if v, ok := col.Value.(string); ok { 932 return v, "string", nil 933 } 934 return string(col.Value.([]byte)), "string", nil 935 case mysql.TypeEnum: 936 if v, ok := col.Value.(string); ok { 937 return v, "string", nil 938 } 939 elements := ft.GetElems() 940 number := col.Value.(uint64) 941 enumVar, err := types.ParseEnumValue(elements, number) 942 if err != nil { 943 log.Info("avro encoder parse enum value failed", zap.Strings("elements", elements), zap.Uint64("number", number)) 944 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 945 } 946 return enumVar.Name, "string", nil 947 case mysql.TypeSet: 948 if v, ok := col.Value.(string); ok { 949 return v, "string", nil 950 } 951 elements := ft.GetElems() 952 number := col.Value.(uint64) 953 setVar, err := types.ParseSetValue(elements, number) 954 if err != nil { 955 log.Info("avro encoder parse set value failed", 956 zap.Strings("elements", elements), zap.Uint64("number", number), zap.Error(err)) 957 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 958 } 959 return setVar.Name, "string", nil 960 case mysql.TypeJSON: 961 return col.Value.(string), "string", nil 962 case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDuration: 963 return col.Value.(string), "string", nil 964 case mysql.TypeYear: 965 if v, ok := col.Value.(string); ok { 966 n, err := strconv.ParseInt(v, 10, 32) 967 if err != nil { 968 log.Info("avro encoder parse year value failed", zap.String("value", v), zap.Error(err)) 969 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 970 } 971 return int32(n), "int", nil 972 } 973 return int32(col.Value.(int64)), "int", nil 974 default: 975 log.Error("unknown mysql type", zap.Any("value", col.Value), zap.Any("mysqlType", col.Type)) 976 return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") 977 } 978 } 979 980 const ( 981 // avro does not send ddl and checkpoint message, the following 2 field is used to distinguish 982 // TiCDC DDL event and checkpoint event, only used for testing purpose, not for production 983 ddlByte = uint8(1) 984 checkpointByte = uint8(2) 985 ) 986 987 func (r *avroEncodeResult) toEnvelope() ([]byte, error) { 988 buf := new(bytes.Buffer) 989 data := []interface{}{r.header, r.data} 990 for _, v := range data { 991 err := binary.Write(buf, binary.BigEndian, v) 992 if err != nil { 993 return nil, cerror.WrapError(cerror.ErrAvroToEnvelopeError, err) 994 } 995 } 996 return buf.Bytes(), nil 997 } 998 999 type batchEncoderBuilder struct { 1000 namespace string 1001 config *common.Config 1002 schemaM SchemaManager 1003 } 1004 1005 const ( 1006 keySchemaSuffix = "-key" 1007 valueSchemaSuffix = "-value" 1008 ) 1009 1010 // NewBatchEncoderBuilder creates an avro batchEncoderBuilder. 1011 func NewBatchEncoderBuilder( 1012 ctx context.Context, config *common.Config, 1013 ) (codec.RowEventEncoderBuilder, error) { 1014 var schemaM SchemaManager 1015 var err error 1016 1017 schemaRegistryType := config.SchemaRegistryType() 1018 switch schemaRegistryType { 1019 case common.SchemaRegistryTypeConfluent: 1020 schemaM, err = NewConfluentSchemaManager(ctx, config.AvroConfluentSchemaRegistry, nil) 1021 if err != nil { 1022 return nil, errors.Trace(err) 1023 } 1024 case common.SchemaRegistryTypeGlue: 1025 schemaM, err = NewGlueSchemaManager(ctx, config.AvroGlueSchemaRegistry) 1026 if err != nil { 1027 return nil, errors.Trace(err) 1028 } 1029 default: 1030 return nil, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs(schemaRegistryType) 1031 } 1032 1033 return &batchEncoderBuilder{ 1034 namespace: config.ChangefeedID.Namespace, 1035 config: config, 1036 schemaM: schemaM, 1037 }, nil 1038 } 1039 1040 // Build an AvroEventBatchEncoder. 1041 func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { 1042 return NewAvroEncoder(b.namespace, b.schemaM, b.config) 1043 } 1044 1045 // CleanMetrics is a no-op for AvroEventBatchEncoder. 1046 func (b *batchEncoderBuilder) CleanMetrics() {} 1047 1048 // NewAvroEncoder return a avro encoder. 1049 func NewAvroEncoder(namespace string, schemaM SchemaManager, config *common.Config) codec.RowEventEncoder { 1050 return &BatchEncoder{ 1051 namespace: namespace, 1052 schemaM: schemaM, 1053 result: make([]*common.Message, 0, 1), 1054 config: config, 1055 } 1056 } 1057 1058 // SetupEncoderAndSchemaRegistry4Testing start a local schema registry for testing. 1059 func SetupEncoderAndSchemaRegistry4Testing( 1060 ctx context.Context, 1061 config *common.Config, 1062 ) (*BatchEncoder, error) { 1063 startHTTPInterceptForTestingRegistry() 1064 schemaM, err := NewConfluentSchemaManager(ctx, "http://127.0.0.1:8081", nil) 1065 if err != nil { 1066 return nil, errors.Trace(err) 1067 } 1068 1069 return &BatchEncoder{ 1070 namespace: model.DefaultNamespace, 1071 schemaM: schemaM, 1072 result: make([]*common.Message, 0, 1), 1073 config: config, 1074 }, nil 1075 } 1076 1077 // TeardownEncoderAndSchemaRegistry4Testing stop the local schema registry for testing. 1078 func TeardownEncoderAndSchemaRegistry4Testing() { 1079 stopHTTPInterceptForTestingRegistry() 1080 }