github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/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 codec 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/binary" 20 "encoding/json" 21 "fmt" 22 "math" 23 "math/big" 24 "strconv" 25 "time" 26 27 "github.com/pingcap/errors" 28 "github.com/pingcap/log" 29 "github.com/pingcap/parser/mysql" 30 "github.com/pingcap/ticdc/cdc/model" 31 cerror "github.com/pingcap/ticdc/pkg/errors" 32 "github.com/pingcap/tidb/types" 33 tijson "github.com/pingcap/tidb/types/json" 34 "go.uber.org/zap" 35 ) 36 37 // AvroEventBatchEncoder converts the events to binary Avro data 38 type AvroEventBatchEncoder struct { 39 keySchemaManager *AvroSchemaManager 40 valueSchemaManager *AvroSchemaManager 41 resultBuf []*MQMessage 42 43 tz *time.Location 44 } 45 46 type avroEncodeResult struct { 47 data []byte 48 registryID int 49 } 50 51 // NewAvroEventBatchEncoder creates an AvroEventBatchEncoder 52 func NewAvroEventBatchEncoder() EventBatchEncoder { 53 return &AvroEventBatchEncoder{ 54 valueSchemaManager: nil, 55 keySchemaManager: nil, 56 resultBuf: make([]*MQMessage, 0, 4096), 57 } 58 } 59 60 // SetValueSchemaManager sets the value schema manager for an Avro encoder 61 func (a *AvroEventBatchEncoder) SetValueSchemaManager(manager *AvroSchemaManager) { 62 a.valueSchemaManager = manager 63 } 64 65 // GetValueSchemaManager gets the value schema manager for an Avro encoder 66 func (a *AvroEventBatchEncoder) GetValueSchemaManager() *AvroSchemaManager { 67 return a.valueSchemaManager 68 } 69 70 // SetKeySchemaManager sets the value schema manager for an Avro encoder 71 func (a *AvroEventBatchEncoder) SetKeySchemaManager(manager *AvroSchemaManager) { 72 a.keySchemaManager = manager 73 } 74 75 // GetKeySchemaManager gets the value schema manager for an Avro encoder 76 func (a *AvroEventBatchEncoder) GetKeySchemaManager() *AvroSchemaManager { 77 return a.keySchemaManager 78 } 79 80 // SetTimeZone sets the time-zone that is used to serialize Avro date-time types 81 func (a *AvroEventBatchEncoder) SetTimeZone(tz *time.Location) { 82 log.Debug("Setting Avro serializer timezone", zap.String("tz", tz.String())) 83 a.tz = tz 84 } 85 86 // AppendRowChangedEvent appends a row change event to the encoder 87 // NOTE: the encoder can only store one RowChangedEvent! 88 func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { 89 mqMessage := NewMQMessage(ProtocolAvro, nil, nil, e.CommitTs, model.MqMessageTypeRow, &e.Table.Schema, &e.Table.Table) 90 91 if !e.IsDelete() { 92 res, err := avroEncode(e.Table, a.valueSchemaManager, e.TableInfoVersion, e.Columns, a.tz) 93 if err != nil { 94 log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) 95 return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") 96 } 97 98 evlp, err := res.toEnvelope() 99 if err != nil { 100 log.Warn("AppendRowChangedEvent: could not construct Avro envelope", zap.String("table", e.Table.String())) 101 return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") 102 } 103 104 mqMessage.Value = evlp 105 } else { 106 mqMessage.Value = nil 107 } 108 109 pkeyCols := e.HandleKeyColumns() 110 111 res, err := avroEncode(e.Table, a.keySchemaManager, e.TableInfoVersion, pkeyCols, a.tz) 112 if err != nil { 113 log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) 114 return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") 115 } 116 117 evlp, err := res.toEnvelope() 118 if err != nil { 119 log.Warn("AppendRowChangedEvent: could not construct Avro envelope", zap.String("table", e.Table.String())) 120 return EncoderNoOperation, errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") 121 } 122 123 mqMessage.Key = evlp 124 a.resultBuf = append(a.resultBuf, mqMessage) 125 126 return EncoderNeedAsyncWrite, nil 127 } 128 129 // AppendResolvedEvent is no-op for Avro 130 func (a *AvroEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { 131 return EncoderNoOperation, nil 132 } 133 134 // EncodeCheckpointEvent is no-op for now 135 func (a *AvroEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { 136 return nil, nil 137 } 138 139 // EncodeDDLEvent is no-op now 140 func (a *AvroEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) { 141 return nil, nil 142 } 143 144 // Build MQ Messages 145 func (a *AvroEventBatchEncoder) Build() (mqMessages []*MQMessage) { 146 old := a.resultBuf 147 a.resultBuf = nil 148 return old 149 } 150 151 // MixedBuild implements the EventBatchEncoder interface 152 func (a *AvroEventBatchEncoder) MixedBuild(withVersion bool) []byte { 153 panic("Mixed Build only use for JsonEncoder") 154 } 155 156 // Reset implements the EventBatchEncoder interface 157 func (a *AvroEventBatchEncoder) Reset() { 158 panic("Reset only used for JsonEncoder") 159 } 160 161 // Size is the current size of resultBuf 162 func (a *AvroEventBatchEncoder) Size() int { 163 if a.resultBuf == nil { 164 return 0 165 } 166 sum := 0 167 for _, msg := range a.resultBuf { 168 sum += len(msg.Key) 169 sum += len(msg.Value) 170 } 171 return sum 172 } 173 174 // SetParams is no-op for now 175 func (a *AvroEventBatchEncoder) SetParams(params map[string]string) error { 176 // no op 177 return nil 178 } 179 180 func avroEncode(table *model.TableName, manager *AvroSchemaManager, tableVersion uint64, cols []*model.Column, tz *time.Location) (*avroEncodeResult, error) { 181 schemaGen := func() (string, error) { 182 schema, err := ColumnInfoToAvroSchema(table.Table, cols) 183 if err != nil { 184 return "", errors.Annotate(err, "AvroEventBatchEncoder: generating schema failed") 185 } 186 return schema, nil 187 } 188 189 // TODO pass ctx from the upper function. Need to modify the EventBatchEncoder interface. 190 avroCodec, registryID, err := manager.GetCachedOrRegister(context.Background(), *table, tableVersion, schemaGen) 191 if err != nil { 192 return nil, errors.Annotate(err, "AvroEventBatchEncoder: get-or-register failed") 193 } 194 195 native, err := rowToAvroNativeData(cols, tz) 196 if err != nil { 197 return nil, errors.Annotate(err, "AvroEventBatchEncoder: converting to native failed") 198 } 199 200 bin, err := avroCodec.BinaryFromNative(nil, native) 201 if err != nil { 202 return nil, errors.Annotate( 203 cerror.WrapError(cerror.ErrAvroEncodeToBinary, err), "AvroEventBatchEncoder: converting to Avro binary failed") 204 } 205 206 return &avroEncodeResult{ 207 data: bin, 208 registryID: registryID, 209 }, nil 210 } 211 212 type avroSchemaTop struct { 213 Tp string `json:"type"` 214 Name string `json:"name"` 215 Fields []map[string]interface{} `json:"fields"` 216 } 217 218 type logicalType string 219 220 type avroLogicalType struct { 221 Type string `json:"type"` 222 LogicalType logicalType `json:"logicalType"` 223 Precision interface{} `json:"precision,omitempty"` 224 Scale interface{} `json:"scale,omitempty"` 225 } 226 227 const ( 228 timestampMillis logicalType = "timestamp-millis" 229 timeMillis logicalType = "time-millis" 230 decimalType logicalType = "decimal" 231 ) 232 233 // ColumnInfoToAvroSchema generates the Avro schema JSON for the corresponding columns 234 func ColumnInfoToAvroSchema(name string, columnInfo []*model.Column) (string, error) { 235 top := avroSchemaTop{ 236 Tp: "record", 237 Name: name, 238 Fields: nil, 239 } 240 241 for _, col := range columnInfo { 242 avroType, err := getAvroDataTypeFromColumn(col) 243 if err != nil { 244 return "", err 245 } 246 field := make(map[string]interface{}) 247 field["name"] = col.Name 248 if col.Flag.IsHandleKey() { 249 field["type"] = avroType 250 } else { 251 field["type"] = []interface{}{"null", avroType} 252 field["default"] = nil 253 } 254 255 top.Fields = append(top.Fields, field) 256 } 257 258 str, err := json.Marshal(&top) 259 if err != nil { 260 return "", cerror.WrapError(cerror.ErrAvroMarshalFailed, err) 261 } 262 log.Debug("Avro Schema JSON generated", zap.ByteString("schema", str)) 263 return string(str), nil 264 } 265 266 func rowToAvroNativeData(cols []*model.Column, tz *time.Location) (interface{}, error) { 267 ret := make(map[string]interface{}, len(cols)) 268 for _, col := range cols { 269 if col == nil { 270 continue 271 } 272 data, str, err := columnToAvroNativeData(col, tz) 273 if err != nil { 274 return nil, err 275 } 276 277 if col.Flag.IsHandleKey() { 278 ret[col.Name] = data 279 continue 280 } 281 union := make(map[string]interface{}, 1) 282 union[str] = data 283 ret[col.Name] = union 284 } 285 return ret, nil 286 } 287 288 func getAvroDataTypeFallback(v interface{}) (string, error) { 289 switch tp := v.(type) { 290 case bool: 291 return "boolean", nil 292 case []byte: 293 return "bytes", nil 294 case float64: 295 return "double", nil 296 case float32: 297 return "float", nil 298 case int64, uint64: 299 return "long", nil 300 case int, int32, uint32: 301 return "int", nil 302 case nil: 303 return "null", nil 304 case string: 305 return "string", nil 306 default: 307 log.Warn("getAvroDataTypeFallback: unknown type") 308 return "", cerror.ErrAvroUnknownType.GenWithStackByArgs(tp) 309 } 310 } 311 312 var unsignedLongAvroType = avroLogicalType{ 313 Type: "bytes", 314 LogicalType: decimalType, 315 Precision: 8, 316 Scale: 0, 317 } 318 319 func getAvroDataTypeFromColumn(col *model.Column) (interface{}, error) { 320 log.Info("DEBUG: getAvroDataTypeFromColumn", zap.Reflect("col", col)) 321 switch col.Type { 322 case mysql.TypeFloat: 323 return "float", nil 324 case mysql.TypeDouble: 325 return "double", nil 326 case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: 327 return "string", nil 328 case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: 329 return avroLogicalType{ 330 Type: "long", 331 LogicalType: timestampMillis, 332 }, nil 333 case mysql.TypeDuration: 334 return avroLogicalType{ 335 Type: "int", 336 LogicalType: timeMillis, 337 }, nil 338 case mysql.TypeEnum: 339 return unsignedLongAvroType, nil 340 case mysql.TypeSet: 341 return unsignedLongAvroType, nil 342 case mysql.TypeBit: 343 return unsignedLongAvroType, nil 344 case mysql.TypeNewDecimal: 345 return "string", nil 346 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: 347 return "int", nil 348 case mysql.TypeLong: 349 if col.Flag.IsUnsigned() { 350 return "long", nil 351 } 352 return "int", nil 353 case mysql.TypeLonglong: 354 if col.Flag.IsUnsigned() { 355 return unsignedLongAvroType, nil 356 } 357 return "long", nil 358 case mysql.TypeNull: 359 return "null", nil 360 case mysql.TypeJSON: 361 return "string", nil 362 case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: 363 return "bytes", nil 364 case mysql.TypeYear: 365 return "long", nil 366 default: 367 log.Panic("Unknown MySql type", zap.Reflect("mysql-type", col.Type)) 368 return "", errors.New("Unknown Mysql type") 369 } 370 } 371 372 var ( 373 zeroTimeStr = types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, 0).String() 374 zeroDateStr = types.NewTime(types.ZeroCoreTime, mysql.TypeDate, 0).String() 375 ) 376 377 func columnToAvroNativeData(col *model.Column, tz *time.Location) (interface{}, string, error) { 378 if col.Value == nil { 379 return nil, "null", nil 380 } 381 382 handleUnsignedInt64 := func() (interface{}, string, error) { 383 var retVal interface{} 384 switch v := col.Value.(type) { 385 case uint64: 386 retVal = big.NewRat(0, 1).SetUint64(v) 387 case int64: 388 retVal = big.NewRat(0, 1).SetInt64(v) 389 } 390 return retVal, string("bytes." + decimalType), nil 391 } 392 393 switch col.Type { 394 case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: 395 // Refer to `unflatten` in cdc/entry/codec.go for why this piece of code is like this. 396 const fullType = "long." + timestampMillis 397 str := col.Value.(string) 398 399 if (col.Type == mysql.TypeDate && str == zeroDateStr) || 400 (col.Type != mysql.TypeDate && str == zeroTimeStr) { 401 402 return time.Time{}, string(fullType), nil 403 } 404 405 var actualTz *time.Location 406 if col.Type != mysql.TypeTimestamp { 407 actualTz = time.UTC 408 } else { 409 actualTz = tz 410 } 411 412 t, err := time.ParseInLocation(types.DateFormat, str, actualTz) 413 414 if err == nil { 415 return t, string(fullType), nil 416 } 417 418 t, err = time.ParseInLocation(types.TimeFormat, str, actualTz) 419 if err == nil { 420 return t, string(fullType), nil 421 } 422 423 t, err = time.ParseInLocation(types.TimeFSPFormat, str, actualTz) 424 if err != nil { 425 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 426 } 427 return t, string(fullType), nil 428 case mysql.TypeDuration: 429 str := col.Value.(string) 430 var ( 431 hours int 432 minutes int 433 seconds int 434 frac string 435 ) 436 _, err := fmt.Sscanf(str, "%d:%d:%d.%s", &hours, &minutes, &seconds, &frac) 437 if err != nil { 438 _, err := fmt.Sscanf(str, "%d:%d:%d", &hours, &minutes, &seconds) 439 frac = "0" 440 441 if err != nil { 442 return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) 443 } 444 } 445 446 fsp := len(frac) 447 fracInt, err := strconv.ParseInt(frac, 10, 32) 448 if err != nil { 449 return nil, "", err 450 } 451 fracInt = int64(float64(fracInt) * math.Pow10(6-fsp)) 452 453 d := types.NewDuration(hours, minutes, seconds, int(fracInt), int8(fsp)).Duration 454 const fullType = "int." + timeMillis 455 return d, string(fullType), nil 456 case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: 457 if col.Flag.IsBinary() { 458 switch val := col.Value.(type) { 459 case string: 460 return []byte(val), "bytes", nil 461 case []byte: 462 return val, "bytes", nil 463 } 464 } else { 465 switch val := col.Value.(type) { 466 case string: 467 return val, "string", nil 468 case []byte: 469 return string(val), "string", nil 470 } 471 } 472 log.Panic("Avro could not process text-like type", zap.Reflect("col", col)) 473 return nil, "", errors.New("Unknown datum type") 474 case mysql.TypeYear: 475 return col.Value.(int64), "long", nil 476 case mysql.TypeJSON: 477 return col.Value.(tijson.BinaryJSON).String(), "string", nil 478 case mysql.TypeNewDecimal: 479 return col.Value.(string), "string", nil 480 case mysql.TypeEnum: 481 return handleUnsignedInt64() 482 case mysql.TypeSet: 483 return handleUnsignedInt64() 484 case mysql.TypeBit: 485 return handleUnsignedInt64() 486 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: 487 if col.Flag.IsUnsigned() { 488 return int32(col.Value.(uint64)), "int", nil 489 } 490 return int32(col.Value.(int64)), "int", nil 491 case mysql.TypeLong: 492 if col.Flag.IsUnsigned() { 493 return int64(col.Value.(uint64)), "long", nil 494 } 495 return col.Value.(int64), "int", nil 496 case mysql.TypeLonglong: 497 if col.Flag.IsUnsigned() { 498 return handleUnsignedInt64() 499 } 500 return col.Value.(int64), "long", nil 501 case mysql.TypeFloat: 502 return col.Value.(float64), "float", nil 503 default: 504 avroType, err := getAvroDataTypeFallback(col.Value) 505 if err != nil { 506 return nil, "", err 507 } 508 return col.Value, avroType, nil 509 } 510 } 511 512 const magicByte = uint8(0) 513 514 func (r *avroEncodeResult) toEnvelope() ([]byte, error) { 515 buf := new(bytes.Buffer) 516 data := []interface{}{magicByte, int32(r.registryID), r.data} 517 for _, v := range data { 518 err := binary.Write(buf, binary.BigEndian, v) 519 if err != nil { 520 return nil, cerror.WrapError(cerror.ErrAvroToEnvelopeError, err) 521 } 522 } 523 return buf.Bytes(), nil 524 }