github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/canal/canal_json_row_event_encoder.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 "context" 18 "time" 19 20 "github.com/goccy/go-json" 21 "github.com/mailru/easyjson/jwriter" 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 "github.com/pingcap/tiflow/pkg/config" 26 cerror "github.com/pingcap/tiflow/pkg/errors" 27 "github.com/pingcap/tiflow/pkg/sink/codec" 28 "github.com/pingcap/tiflow/pkg/sink/codec/common" 29 "github.com/pingcap/tiflow/pkg/sink/codec/utils" 30 "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" 31 "go.uber.org/zap" 32 ) 33 34 func fillColumns( 35 columns []*model.Column, 36 onlyOutputUpdatedColumn bool, 37 onlyHandleKeyColumn bool, 38 newColumnMap map[string]*model.Column, 39 out *jwriter.Writer, 40 builder *canalEntryBuilder, 41 ) error { 42 if len(columns) == 0 { 43 out.RawString("null") 44 return nil 45 } 46 out.RawByte('[') 47 out.RawByte('{') 48 isFirst := true 49 for _, col := range columns { 50 if col != nil { 51 // column equal, do not output it 52 if onlyOutputUpdatedColumn && shouldIgnoreColumn(col, newColumnMap) { 53 continue 54 } 55 if onlyHandleKeyColumn && !col.Flag.IsHandleKey() { 56 continue 57 } 58 if isFirst { 59 isFirst = false 60 } else { 61 out.RawByte(',') 62 } 63 value, err := builder.formatValue(col.Value, col.Flag.IsBinary()) 64 if err != nil { 65 return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 66 } 67 out.String(col.Name) 68 out.RawByte(':') 69 if col.Value == nil { 70 out.RawString("null") 71 } else { 72 out.String(value) 73 } 74 } 75 } 76 out.RawByte('}') 77 out.RawByte(']') 78 return nil 79 } 80 81 func newJSONMessageForDML( 82 builder *canalEntryBuilder, 83 e *model.RowChangedEvent, 84 config *common.Config, 85 messageTooLarge bool, 86 claimCheckFileName string, 87 ) ([]byte, error) { 88 isDelete := e.IsDelete() 89 90 onlyHandleKey := messageTooLarge 91 if isDelete && config.DeleteOnlyHandleKeyColumns { 92 onlyHandleKey = true 93 } 94 95 mysqlTypeMap := make(map[string]string, len(e.Columns)) 96 out := &jwriter.Writer{} 97 out.RawByte('{') 98 { 99 const prefix string = ",\"id\":" 100 out.RawString(prefix[1:]) 101 out.Int64(0) // ignored by both Canal Adapter and Flink 102 } 103 { 104 const prefix string = ",\"database\":" 105 out.RawString(prefix) 106 out.String(e.TableInfo.GetSchemaName()) 107 } 108 { 109 const prefix string = ",\"table\":" 110 out.RawString(prefix) 111 out.String(e.TableInfo.GetTableName()) 112 } 113 { 114 const prefix string = ",\"pkNames\":" 115 out.RawString(prefix) 116 pkNames := e.PrimaryKeyColumnNames() 117 if pkNames == nil { 118 out.RawString("null") 119 } else { 120 out.RawByte('[') 121 for v25, v26 := range pkNames { 122 if v25 > 0 { 123 out.RawByte(',') 124 } 125 out.String(v26) 126 } 127 out.RawByte(']') 128 } 129 } 130 { 131 const prefix string = ",\"isDdl\":" 132 out.RawString(prefix) 133 out.Bool(false) 134 } 135 { 136 const prefix string = ",\"type\":" 137 out.RawString(prefix) 138 out.String(eventTypeString(e)) 139 } 140 { 141 const prefix string = ",\"es\":" 142 out.RawString(prefix) 143 out.Int64(convertToCanalTs(e.CommitTs)) 144 } 145 { 146 const prefix string = ",\"ts\":" 147 out.RawString(prefix) 148 out.Int64(time.Now().UnixMilli()) // ignored by both Canal Adapter and Flink 149 } 150 { 151 const prefix string = ",\"sql\":" 152 out.RawString(prefix) 153 out.String("") 154 } 155 { 156 columns := e.PreColumns 157 if !isDelete { 158 columns = e.Columns 159 } 160 const prefix string = ",\"sqlType\":" 161 out.RawString(prefix) 162 emptyColumn := true 163 tableInfo := e.TableInfo 164 for _, col := range columns { 165 if col != nil { 166 colFlag := tableInfo.ForceGetColumnFlagType(col.ColumnID) 167 columnInfo := tableInfo.ForceGetColumnInfo(col.ColumnID) 168 colType := columnInfo.GetType() 169 colName := tableInfo.ForceGetColumnName(col.ColumnID) 170 if onlyHandleKey && !colFlag.IsHandleKey() { 171 continue 172 } 173 if emptyColumn { 174 out.RawByte('{') 175 emptyColumn = false 176 } else { 177 out.RawByte(',') 178 } 179 javaType, err := getJavaSQLType(col.Value, colType, *colFlag) 180 if err != nil { 181 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 182 } 183 out.String(colName) 184 out.RawByte(':') 185 out.Int32(int32(javaType)) 186 mysqlTypeMap[colName] = utils.GetMySQLType(columnInfo, config.ContentCompatible) 187 } 188 } 189 if emptyColumn { 190 out.RawString(`null`) 191 } else { 192 out.RawByte('}') 193 } 194 } 195 { 196 const prefix string = ",\"mysqlType\":" 197 out.RawString(prefix) 198 if mysqlTypeMap == nil { 199 out.RawString(`null`) 200 } else { 201 out.RawByte('{') 202 isFirst := true 203 for typeKey, typeValue := range mysqlTypeMap { 204 if isFirst { 205 isFirst = false 206 } else { 207 out.RawByte(',') 208 } 209 out.String(typeKey) 210 out.RawByte(':') 211 out.String(typeValue) 212 } 213 out.RawByte('}') 214 } 215 } 216 217 if e.IsDelete() { 218 out.RawString(",\"old\":null") 219 out.RawString(",\"data\":") 220 if err := fillColumns( 221 e.GetPreColumns(), 222 false, onlyHandleKey, nil, out, builder, 223 ); err != nil { 224 return nil, err 225 } 226 } else if e.IsInsert() { 227 out.RawString(",\"old\":null") 228 out.RawString(",\"data\":") 229 if err := fillColumns( 230 e.GetColumns(), 231 false, onlyHandleKey, nil, out, builder, 232 ); err != nil { 233 return nil, err 234 } 235 } else if e.IsUpdate() { 236 var newColsMap map[string]*model.Column 237 if config.OnlyOutputUpdatedColumns { 238 newColsMap = make(map[string]*model.Column, len(e.Columns)) 239 for _, col := range e.GetColumns() { 240 newColsMap[col.Name] = col 241 } 242 } 243 out.RawString(",\"old\":") 244 if err := fillColumns( 245 e.GetPreColumns(), 246 config.OnlyOutputUpdatedColumns, onlyHandleKey, newColsMap, out, builder, 247 ); err != nil { 248 return nil, err 249 } 250 out.RawString(",\"data\":") 251 if err := fillColumns( 252 e.GetColumns(), 253 false, onlyHandleKey, nil, out, builder, 254 ); err != nil { 255 return nil, err 256 } 257 } else { 258 log.Panic("unreachable event type", zap.Any("event", e)) 259 } 260 261 if config.EnableTiDBExtension { 262 const prefix string = ",\"_tidb\":" 263 out.RawString(prefix) 264 out.RawByte('{') 265 out.RawString("\"commitTs\":") 266 out.Uint64(e.CommitTs) 267 268 // only send handle key may happen in 2 cases: 269 // 1. delete event, and set only handle key config. no need to encode `onlyHandleKey` field 270 // 2. event larger than the max message size, and enable large message handle to the `handleKeyOnly`, encode `onlyHandleKey` field 271 if messageTooLarge { 272 if config.LargeMessageHandle.HandleKeyOnly() { 273 out.RawByte(',') 274 out.RawString("\"onlyHandleKey\":true") 275 } 276 if config.LargeMessageHandle.EnableClaimCheck() { 277 out.RawByte(',') 278 out.RawString("\"claimCheckLocation\":") 279 out.String(claimCheckFileName) 280 } 281 } 282 out.RawByte('}') 283 } 284 out.RawByte('}') 285 286 value, err := out.BuildBytes() 287 if err != nil { 288 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 289 } 290 return value, nil 291 } 292 293 func eventTypeString(e *model.RowChangedEvent) string { 294 if e.IsDelete() { 295 return "DELETE" 296 } 297 if len(e.PreColumns) == 0 { 298 return "INSERT" 299 } 300 return "UPDATE" 301 } 302 303 // JSONRowEventEncoder encodes row event in JSON format 304 type JSONRowEventEncoder struct { 305 builder *canalEntryBuilder 306 messages []*common.Message 307 308 claimCheck *claimcheck.ClaimCheck 309 310 config *common.Config 311 } 312 313 // newJSONRowEventEncoder creates a new JSONRowEventEncoder 314 func newJSONRowEventEncoder( 315 config *common.Config, claimCheck *claimcheck.ClaimCheck, 316 ) codec.RowEventEncoder { 317 return &JSONRowEventEncoder{ 318 builder: newCanalEntryBuilder(config), 319 messages: make([]*common.Message, 0, 1), 320 config: config, 321 claimCheck: claimCheck, 322 } 323 } 324 325 func (c *JSONRowEventEncoder) newJSONMessageForDDL(e *model.DDLEvent) canalJSONMessageInterface { 326 msg := &JSONMessage{ 327 ID: 0, // ignored by both Canal Adapter and Flink 328 Schema: e.TableInfo.TableName.Schema, 329 Table: e.TableInfo.TableName.Table, 330 IsDDL: true, 331 EventType: convertDdlEventType(e).String(), 332 ExecutionTime: convertToCanalTs(e.CommitTs), 333 BuildTime: time.Now().UnixMilli(), // timestamp 334 Query: e.Query, 335 } 336 337 if !c.config.EnableTiDBExtension { 338 return msg 339 } 340 341 return &canalJSONMessageWithTiDBExtension{ 342 JSONMessage: msg, 343 Extensions: &tidbExtension{CommitTs: e.CommitTs}, 344 } 345 } 346 347 func (c *JSONRowEventEncoder) newJSONMessage4CheckpointEvent( 348 ts uint64, 349 ) *canalJSONMessageWithTiDBExtension { 350 return &canalJSONMessageWithTiDBExtension{ 351 JSONMessage: &JSONMessage{ 352 ID: 0, 353 IsDDL: false, 354 EventType: tidbWaterMarkType, 355 ExecutionTime: convertToCanalTs(ts), 356 BuildTime: time.Now().UnixNano() / int64(time.Millisecond), // converts to milliseconds 357 }, 358 Extensions: &tidbExtension{WatermarkTs: ts}, 359 } 360 } 361 362 // EncodeCheckpointEvent implements the RowEventEncoder interface 363 func (c *JSONRowEventEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { 364 if !c.config.EnableTiDBExtension { 365 return nil, nil 366 } 367 368 msg := c.newJSONMessage4CheckpointEvent(ts) 369 value, err := json.Marshal(msg) 370 if err != nil { 371 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 372 } 373 374 value, err = common.Compress( 375 c.config.ChangefeedID, c.config.LargeMessageHandle.LargeMessageHandleCompression, value, 376 ) 377 if err != nil { 378 return nil, errors.Trace(err) 379 } 380 381 return common.NewResolvedMsg(config.ProtocolCanalJSON, nil, value, ts), nil 382 } 383 384 // AppendRowChangedEvent implements the interface EventJSONBatchEncoder 385 func (c *JSONRowEventEncoder) AppendRowChangedEvent( 386 ctx context.Context, 387 _ string, 388 e *model.RowChangedEvent, 389 callback func(), 390 ) error { 391 value, err := newJSONMessageForDML(c.builder, e, c.config, false, "") 392 if err != nil { 393 return errors.Trace(err) 394 } 395 396 value, err = common.Compress( 397 c.config.ChangefeedID, c.config.LargeMessageHandle.LargeMessageHandleCompression, value, 398 ) 399 if err != nil { 400 return errors.Trace(err) 401 } 402 m := &common.Message{ 403 Key: nil, 404 Value: value, 405 Ts: e.CommitTs, 406 Schema: e.TableInfo.GetSchemaNamePtr(), 407 Table: e.TableInfo.GetTableNamePtr(), 408 Type: model.MessageTypeRow, 409 Protocol: config.ProtocolCanalJSON, 410 Callback: callback, 411 } 412 m.IncRowsCount() 413 414 originLength := m.Length() 415 if m.Length() > c.config.MaxMessageBytes { 416 // for single message that is longer than max-message-bytes, do not send it. 417 if c.config.LargeMessageHandle.Disabled() { 418 log.Error("Single message is too large for canal-json", 419 zap.Int("maxMessageBytes", c.config.MaxMessageBytes), 420 zap.Int("length", originLength), 421 zap.Any("table", e.TableInfo.TableName)) 422 return cerror.ErrMessageTooLarge.GenWithStackByArgs() 423 } 424 425 if c.config.LargeMessageHandle.HandleKeyOnly() { 426 value, err = newJSONMessageForDML(c.builder, e, c.config, true, "") 427 if err != nil { 428 return cerror.ErrMessageTooLarge.GenWithStackByArgs() 429 } 430 value, err = common.Compress( 431 c.config.ChangefeedID, c.config.LargeMessageHandle.LargeMessageHandleCompression, value, 432 ) 433 if err != nil { 434 return errors.Trace(err) 435 } 436 437 m.Value = value 438 length := m.Length() 439 if length > c.config.MaxMessageBytes { 440 log.Error("Single message is still too large for canal-json only encode handle-key columns", 441 zap.Int("maxMessageBytes", c.config.MaxMessageBytes), 442 zap.Int("originLength", originLength), 443 zap.Int("length", length), 444 zap.Any("table", e.TableInfo.TableName)) 445 return cerror.ErrMessageTooLarge.GenWithStackByArgs() 446 } 447 log.Warn("Single message is too large for canal-json, only encode handle-key columns", 448 zap.Int("maxMessageBytes", c.config.MaxMessageBytes), 449 zap.Int("originLength", originLength), 450 zap.Int("length", length), 451 zap.Any("table", e.TableInfo.TableName)) 452 } 453 454 if c.config.LargeMessageHandle.EnableClaimCheck() { 455 claimCheckFileName := claimcheck.NewFileName() 456 if err := c.claimCheck.WriteMessage(ctx, m.Key, m.Value, claimCheckFileName); err != nil { 457 return errors.Trace(err) 458 } 459 460 m, err = c.newClaimCheckLocationMessage(e, callback, claimCheckFileName) 461 if err != nil { 462 return errors.Trace(err) 463 } 464 } 465 } 466 467 c.messages = append(c.messages, m) 468 return nil 469 } 470 471 func (c *JSONRowEventEncoder) newClaimCheckLocationMessage( 472 event *model.RowChangedEvent, callback func(), fileName string, 473 ) (*common.Message, error) { 474 claimCheckLocation := c.claimCheck.FileNameWithPrefix(fileName) 475 value, err := newJSONMessageForDML(c.builder, event, c.config, true, claimCheckLocation) 476 if err != nil { 477 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 478 } 479 480 value, err = common.Compress( 481 c.config.ChangefeedID, c.config.LargeMessageHandle.LargeMessageHandleCompression, value, 482 ) 483 if err != nil { 484 return nil, errors.Trace(err) 485 } 486 487 result := common.NewMsg(config.ProtocolCanalJSON, nil, value, 0, model.MessageTypeRow, nil, nil) 488 result.Callback = callback 489 result.IncRowsCount() 490 491 length := result.Length() 492 if length > c.config.MaxMessageBytes { 493 log.Warn("Single message is too large for canal-json, when create the claim check location message", 494 zap.Int("maxMessageBytes", c.config.MaxMessageBytes), 495 zap.Int("length", length), 496 zap.Any("table", event.TableInfo.TableName)) 497 return nil, cerror.ErrMessageTooLarge.GenWithStackByArgs(length) 498 } 499 return result, nil 500 } 501 502 // Build implements the RowEventEncoder interface 503 func (c *JSONRowEventEncoder) Build() []*common.Message { 504 if len(c.messages) == 0 { 505 return nil 506 } 507 508 result := c.messages 509 c.messages = nil 510 return result 511 } 512 513 // EncodeDDLEvent encodes DDL events 514 func (c *JSONRowEventEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { 515 message := c.newJSONMessageForDDL(e) 516 value, err := json.Marshal(message) 517 if err != nil { 518 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 519 } 520 value, err = common.Compress( 521 c.config.ChangefeedID, c.config.LargeMessageHandle.LargeMessageHandleCompression, value, 522 ) 523 if err != nil { 524 return nil, errors.Trace(err) 525 } 526 return common.NewDDLMsg(config.ProtocolCanalJSON, nil, value, e), nil 527 } 528 529 type jsonRowEventEncoderBuilder struct { 530 config *common.Config 531 532 claimCheck *claimcheck.ClaimCheck 533 } 534 535 // NewJSONRowEventEncoderBuilder creates a canal-json batchEncoderBuilder. 536 func NewJSONRowEventEncoderBuilder(ctx context.Context, config *common.Config) (codec.RowEventEncoderBuilder, error) { 537 var ( 538 claimCheck *claimcheck.ClaimCheck 539 err error 540 ) 541 if config.LargeMessageHandle.EnableClaimCheck() { 542 claimCheck, err = claimcheck.New(ctx, config.LargeMessageHandle.ClaimCheckStorageURI, config.ChangefeedID) 543 if err != nil { 544 return nil, errors.Trace(err) 545 } 546 } 547 return &jsonRowEventEncoderBuilder{ 548 config: config, 549 claimCheck: claimCheck, 550 }, nil 551 } 552 553 // Build a `jsonRowEventEncoderBuilder` 554 func (b *jsonRowEventEncoderBuilder) Build() codec.RowEventEncoder { 555 return newJSONRowEventEncoder(b.config, b.claimCheck) 556 } 557 558 func shouldIgnoreColumn(col *model.Column, 559 newColumnMap map[string]*model.Column, 560 ) bool { 561 newCol, ok := newColumnMap[col.Name] 562 if ok && newCol != nil { 563 // sql type is not equal 564 if newCol.Type != col.Type { 565 return false 566 } 567 // value equal 568 if codec.IsColumnValueEqual(newCol.Value, col.Value) { 569 return true 570 } 571 } 572 return false 573 } 574 575 func (b *jsonRowEventEncoderBuilder) CleanMetrics() { 576 if b.claimCheck != nil { 577 b.claimCheck.CleanMetrics() 578 } 579 }