github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/json.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 "encoding/base64" 19 "encoding/binary" 20 "encoding/json" 21 "sort" 22 "strconv" 23 "strings" 24 25 "github.com/pingcap/errors" 26 "github.com/pingcap/log" 27 timodel "github.com/pingcap/parser/model" 28 "github.com/pingcap/parser/mysql" 29 cerror "github.com/pingcap/ticdc/pkg/errors" 30 "go.uber.org/zap" 31 32 "github.com/pingcap/ticdc/cdc/model" 33 ) 34 35 const ( 36 // BatchVersion1 represents the version of batch format 37 BatchVersion1 uint64 = 1 38 // DefaultMaxMessageBytes sets the default value for max-message-bytes 39 DefaultMaxMessageBytes int = 64 * 1024 * 1024 // 64M 40 // DefaultMaxBatchSize sets the default value for max-batch-size 41 DefaultMaxBatchSize int = 16 42 ) 43 44 type column struct { 45 Type byte `json:"t"` 46 47 // WhereHandle is deprecation 48 // WhereHandle is replaced by HandleKey in Flag 49 WhereHandle *bool `json:"h,omitempty"` 50 Flag model.ColumnFlagType `json:"f"` 51 Value interface{} `json:"v"` 52 } 53 54 func (c *column) FromSinkColumn(col *model.Column) { 55 c.Type = col.Type 56 c.Flag = col.Flag 57 if c.Flag.IsHandleKey() { 58 whereHandle := true 59 c.WhereHandle = &whereHandle 60 } 61 if col.Value == nil { 62 c.Value = nil 63 return 64 } 65 switch col.Type { 66 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: 67 str := string(col.Value.([]byte)) 68 if c.Flag.IsBinary() { 69 str = strconv.Quote(str) 70 str = str[1 : len(str)-1] 71 } 72 c.Value = str 73 default: 74 c.Value = col.Value 75 } 76 } 77 78 func (c *column) ToSinkColumn(name string) *model.Column { 79 col := new(model.Column) 80 col.Type = c.Type 81 col.Flag = c.Flag 82 col.Name = name 83 col.Value = c.Value 84 if c.Value == nil { 85 return col 86 } 87 switch col.Type { 88 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: 89 str := col.Value.(string) 90 var err error 91 if c.Flag.IsBinary() { 92 str, err = strconv.Unquote("\"" + str + "\"") 93 if err != nil { 94 log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) 95 } 96 } 97 col.Value = []byte(str) 98 default: 99 col.Value = c.Value 100 } 101 return col 102 } 103 104 func formatColumnVal(c column) column { 105 switch c.Type { 106 case mysql.TypeTinyBlob, mysql.TypeMediumBlob, 107 mysql.TypeLongBlob, mysql.TypeBlob: 108 if s, ok := c.Value.(string); ok { 109 var err error 110 c.Value, err = base64.StdEncoding.DecodeString(s) 111 if err != nil { 112 log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) 113 } 114 } 115 case mysql.TypeBit: 116 if s, ok := c.Value.(json.Number); ok { 117 intNum, err := s.Int64() 118 if err != nil { 119 log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) 120 } 121 c.Value = uint64(intNum) 122 } 123 } 124 return c 125 } 126 127 type mqMessageKey struct { 128 // TODO: should we rename it to CRTs 129 Ts uint64 `json:"ts"` 130 Schema string `json:"scm,omitempty"` 131 Table string `json:"tbl,omitempty"` 132 RowID int64 `json:"rid,omitempty"` 133 Partition *int64 `json:"ptn,omitempty"` 134 Type model.MqMessageType `json:"t"` 135 } 136 137 func (m *mqMessageKey) Encode() ([]byte, error) { 138 data, err := json.Marshal(m) 139 return data, cerror.WrapError(cerror.ErrMarshalFailed, err) 140 } 141 142 func (m *mqMessageKey) Decode(data []byte) error { 143 return cerror.WrapError(cerror.ErrUnmarshalFailed, json.Unmarshal(data, m)) 144 } 145 146 type mqMessageRow struct { 147 Update map[string]column `json:"u,omitempty"` 148 PreColumns map[string]column `json:"p,omitempty"` 149 Delete map[string]column `json:"d,omitempty"` 150 } 151 152 func (m *mqMessageRow) Encode() ([]byte, error) { 153 data, err := json.Marshal(m) 154 return data, cerror.WrapError(cerror.ErrMarshalFailed, err) 155 } 156 157 func (m *mqMessageRow) Decode(data []byte) error { 158 decoder := json.NewDecoder(bytes.NewReader(data)) 159 decoder.UseNumber() 160 err := decoder.Decode(m) 161 if err != nil { 162 return cerror.WrapError(cerror.ErrUnmarshalFailed, err) 163 } 164 for colName, column := range m.Update { 165 m.Update[colName] = formatColumnVal(column) 166 } 167 for colName, column := range m.Delete { 168 m.Delete[colName] = formatColumnVal(column) 169 } 170 for colName, column := range m.PreColumns { 171 m.PreColumns[colName] = formatColumnVal(column) 172 } 173 return nil 174 } 175 176 type mqMessageDDL struct { 177 Query string `json:"q"` 178 Type timodel.ActionType `json:"t"` 179 } 180 181 func (m *mqMessageDDL) Encode() ([]byte, error) { 182 data, err := json.Marshal(m) 183 return data, cerror.WrapError(cerror.ErrMarshalFailed, err) 184 } 185 186 func (m *mqMessageDDL) Decode(data []byte) error { 187 return cerror.WrapError(cerror.ErrUnmarshalFailed, json.Unmarshal(data, m)) 188 } 189 190 func newResolvedMessage(ts uint64) *mqMessageKey { 191 return &mqMessageKey{ 192 Ts: ts, 193 Type: model.MqMessageTypeResolved, 194 } 195 } 196 197 func rowEventToMqMessage(e *model.RowChangedEvent) (*mqMessageKey, *mqMessageRow) { 198 var partition *int64 199 if e.Table.IsPartition { 200 partition = &e.Table.TableID 201 } 202 key := &mqMessageKey{ 203 Ts: e.CommitTs, 204 Schema: e.Table.Schema, 205 Table: e.Table.Table, 206 RowID: e.RowID, 207 Partition: partition, 208 Type: model.MqMessageTypeRow, 209 } 210 value := &mqMessageRow{} 211 if e.IsDelete() { 212 value.Delete = sinkColumns2JsonColumns(e.PreColumns) 213 } else { 214 value.Update = sinkColumns2JsonColumns(e.Columns) 215 value.PreColumns = sinkColumns2JsonColumns(e.PreColumns) 216 } 217 return key, value 218 } 219 220 func sinkColumns2JsonColumns(cols []*model.Column) map[string]column { 221 jsonCols := make(map[string]column, len(cols)) 222 for _, col := range cols { 223 if col == nil { 224 continue 225 } 226 c := column{} 227 c.FromSinkColumn(col) 228 jsonCols[col.Name] = c 229 } 230 if len(jsonCols) == 0 { 231 return nil 232 } 233 return jsonCols 234 } 235 236 func jsonColumns2SinkColumns(cols map[string]column) []*model.Column { 237 sinkCols := make([]*model.Column, 0, len(cols)) 238 for name, col := range cols { 239 c := col.ToSinkColumn(name) 240 sinkCols = append(sinkCols, c) 241 } 242 if len(sinkCols) == 0 { 243 return nil 244 } 245 sort.Slice(sinkCols, func(i, j int) bool { 246 return strings.Compare(sinkCols[i].Name, sinkCols[j].Name) > 0 247 }) 248 return sinkCols 249 } 250 251 func mqMessageToRowEvent(key *mqMessageKey, value *mqMessageRow) *model.RowChangedEvent { 252 e := new(model.RowChangedEvent) 253 // TODO: we lost the startTs from kafka message 254 // startTs-based txn filter is out of work 255 e.CommitTs = key.Ts 256 e.Table = &model.TableName{ 257 Schema: key.Schema, 258 Table: key.Table, 259 } 260 // TODO: we lost the tableID from kafka message 261 if key.Partition != nil { 262 e.Table.TableID = *key.Partition 263 e.Table.IsPartition = true 264 } 265 266 if len(value.Delete) != 0 { 267 e.PreColumns = jsonColumns2SinkColumns(value.Delete) 268 } else { 269 e.Columns = jsonColumns2SinkColumns(value.Update) 270 e.PreColumns = jsonColumns2SinkColumns(value.PreColumns) 271 } 272 return e 273 } 274 275 func ddlEventtoMqMessage(e *model.DDLEvent) (*mqMessageKey, *mqMessageDDL) { 276 key := &mqMessageKey{ 277 Ts: e.CommitTs, 278 Schema: e.TableInfo.Schema, 279 Table: e.TableInfo.Table, 280 Type: model.MqMessageTypeDDL, 281 } 282 value := &mqMessageDDL{ 283 Query: e.Query, 284 Type: e.Type, 285 } 286 return key, value 287 } 288 289 func mqMessageToDDLEvent(key *mqMessageKey, value *mqMessageDDL) *model.DDLEvent { 290 e := new(model.DDLEvent) 291 e.TableInfo = new(model.SimpleTableInfo) 292 // TODO: we lost the startTs from kafka message 293 // startTs-based txn filter is out of work 294 e.CommitTs = key.Ts 295 e.TableInfo.Table = key.Table 296 e.TableInfo.Schema = key.Schema 297 e.Type = value.Type 298 e.Query = value.Query 299 return e 300 } 301 302 // JSONEventBatchEncoder encodes the events into the byte of a batch into. 303 type JSONEventBatchEncoder struct { 304 // TODO remove deprecated fields 305 keyBuf *bytes.Buffer // Deprecated: only used for MixedBuild for now 306 valueBuf *bytes.Buffer // Deprecated: only used for MixedBuild for now 307 supportMixedBuild bool // TODO decouple this out 308 309 messageBuf []*MQMessage 310 curBatchSize int 311 // configs 312 maxKafkaMessageSize int 313 maxBatchSize int 314 } 315 316 // GetMaxKafkaMessageSize is only for unit testing. 317 func (d *JSONEventBatchEncoder) GetMaxKafkaMessageSize() int { 318 return d.maxKafkaMessageSize 319 } 320 321 // GetMaxBatchSize is only for unit testing. 322 func (d *JSONEventBatchEncoder) GetMaxBatchSize() int { 323 return d.maxBatchSize 324 } 325 326 // SetMixedBuildSupport is used by CDC Log 327 func (d *JSONEventBatchEncoder) SetMixedBuildSupport(enabled bool) { 328 d.supportMixedBuild = enabled 329 } 330 331 // AppendResolvedEvent is no-op 332 func (d *JSONEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { 333 return EncoderNoOperation, nil 334 } 335 336 // EncodeCheckpointEvent implements the EventBatchEncoder interface 337 func (d *JSONEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { 338 keyMsg := newResolvedMessage(ts) 339 key, err := keyMsg.Encode() 340 if err != nil { 341 return nil, errors.Trace(err) 342 } 343 344 var keyLenByte [8]byte 345 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 346 var valueLenByte [8]byte 347 binary.BigEndian.PutUint64(valueLenByte[:], 0) 348 349 if d.supportMixedBuild { 350 d.keyBuf.Write(keyLenByte[:]) 351 d.keyBuf.Write(key) 352 d.valueBuf.Write(valueLenByte[:]) 353 return nil, nil 354 } 355 356 keyBuf := new(bytes.Buffer) 357 var versionByte [8]byte 358 binary.BigEndian.PutUint64(versionByte[:], BatchVersion1) 359 keyBuf.Write(versionByte[:]) 360 keyBuf.Write(keyLenByte[:]) 361 keyBuf.Write(key) 362 363 valueBuf := new(bytes.Buffer) 364 valueBuf.Write(valueLenByte[:]) 365 366 ret := newResolvedMQMessage(ProtocolDefault, keyBuf.Bytes(), valueBuf.Bytes(), ts) 367 return ret, nil 368 } 369 370 // AppendRowChangedEvent implements the EventBatchEncoder interface 371 func (d *JSONEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { 372 // Some transactions could generate empty row change event, such as 373 // begin; insert into t (id) values (1); delete from t where id=1; commit; 374 // Just ignore these row changed events 375 if len(e.Columns) == 0 && len(e.PreColumns) == 0 { 376 return EncoderNoOperation, nil 377 } 378 keyMsg, valueMsg := rowEventToMqMessage(e) 379 key, err := keyMsg.Encode() 380 if err != nil { 381 return EncoderNoOperation, errors.Trace(err) 382 } 383 value, err := valueMsg.Encode() 384 if err != nil { 385 return EncoderNoOperation, errors.Trace(err) 386 } 387 388 var keyLenByte [8]byte 389 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 390 var valueLenByte [8]byte 391 binary.BigEndian.PutUint64(valueLenByte[:], uint64(len(value))) 392 393 if d.supportMixedBuild { 394 d.keyBuf.Write(keyLenByte[:]) 395 d.keyBuf.Write(key) 396 397 d.valueBuf.Write(valueLenByte[:]) 398 d.valueBuf.Write(value) 399 } else { 400 if len(d.messageBuf) == 0 || 401 d.curBatchSize >= d.maxBatchSize || 402 d.messageBuf[len(d.messageBuf)-1].Length()+len(key)+len(value)+16 > d.maxKafkaMessageSize { 403 404 versionHead := make([]byte, 8) 405 binary.BigEndian.PutUint64(versionHead, BatchVersion1) 406 407 d.messageBuf = append(d.messageBuf, NewMQMessage(ProtocolDefault, versionHead, nil, 0, model.MqMessageTypeRow, nil, nil)) 408 d.curBatchSize = 0 409 } 410 411 message := d.messageBuf[len(d.messageBuf)-1] 412 message.Key = append(message.Key, keyLenByte[:]...) 413 message.Key = append(message.Key, key...) 414 message.Value = append(message.Value, valueLenByte[:]...) 415 message.Value = append(message.Value, value...) 416 message.Ts = e.CommitTs 417 message.Schema = &e.Table.Schema 418 message.Table = &e.Table.Table 419 420 if message.Length() > d.maxKafkaMessageSize { 421 // `len(d.messageBuf) == 1` is implied 422 log.Warn("Event does not fit into max-message-bytes. Adjust relevant configurations to avoid service interruptions.", 423 zap.Int("event-len", message.Length()), zap.Int("max-message-bytes", d.maxKafkaMessageSize)) 424 } 425 d.curBatchSize++ 426 } 427 return EncoderNoOperation, nil 428 } 429 430 // EncodeDDLEvent implements the EventBatchEncoder interface 431 func (d *JSONEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) { 432 keyMsg, valueMsg := ddlEventtoMqMessage(e) 433 key, err := keyMsg.Encode() 434 if err != nil { 435 return nil, errors.Trace(err) 436 } 437 value, err := valueMsg.Encode() 438 if err != nil { 439 return nil, errors.Trace(err) 440 } 441 442 var keyLenByte [8]byte 443 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 444 var valueLenByte [8]byte 445 binary.BigEndian.PutUint64(valueLenByte[:], uint64(len(value))) 446 447 if d.supportMixedBuild { 448 d.keyBuf.Write(keyLenByte[:]) 449 d.keyBuf.Write(key) 450 d.valueBuf.Write(valueLenByte[:]) 451 d.valueBuf.Write(value) 452 return nil, nil 453 } 454 455 keyBuf := new(bytes.Buffer) 456 var versionByte [8]byte 457 binary.BigEndian.PutUint64(versionByte[:], BatchVersion1) 458 keyBuf.Write(versionByte[:]) 459 keyBuf.Write(keyLenByte[:]) 460 keyBuf.Write(key) 461 462 valueBuf := new(bytes.Buffer) 463 valueBuf.Write(valueLenByte[:]) 464 valueBuf.Write(value) 465 466 ret := newDDLMQMessage(ProtocolDefault, keyBuf.Bytes(), valueBuf.Bytes(), e) 467 return ret, nil 468 } 469 470 // Build implements the EventBatchEncoder interface 471 func (d *JSONEventBatchEncoder) Build() (mqMessages []*MQMessage) { 472 if d.supportMixedBuild { 473 if d.valueBuf.Len() == 0 { 474 return nil 475 } 476 /* there could be multiple types of event encoded within a single message which means the type is not sure */ 477 ret := NewMQMessage(ProtocolDefault, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeUnknown, nil, nil) 478 return []*MQMessage{ret} 479 } 480 481 ret := d.messageBuf 482 d.messageBuf = make([]*MQMessage, 0) 483 return ret 484 } 485 486 // MixedBuild implements the EventBatchEncoder interface 487 func (d *JSONEventBatchEncoder) MixedBuild(withVersion bool) []byte { 488 if !d.supportMixedBuild { 489 log.Panic("mixedBuildSupport not enabled!") 490 return nil 491 } 492 keyBytes := d.keyBuf.Bytes() 493 valueBytes := d.valueBuf.Bytes() 494 mixedBytes := make([]byte, len(keyBytes)+len(valueBytes)) 495 496 index := uint64(0) 497 keyIndex := uint64(0) 498 valueIndex := uint64(0) 499 500 if withVersion { 501 // the first 8 bytes is the version, we should copy directly 502 // then skip 8 bytes for next round key value parse 503 copy(mixedBytes[:8], keyBytes[:8]) 504 index = uint64(8) // skip version 505 keyIndex = uint64(8) // skip version 506 } 507 508 for { 509 if keyIndex >= uint64(len(keyBytes)) { 510 break 511 } 512 keyLen := binary.BigEndian.Uint64(keyBytes[keyIndex : keyIndex+8]) 513 offset := keyLen + 8 514 copy(mixedBytes[index:index+offset], keyBytes[keyIndex:keyIndex+offset]) 515 keyIndex += offset 516 index += offset 517 518 valueLen := binary.BigEndian.Uint64(valueBytes[valueIndex : valueIndex+8]) 519 offset = valueLen + 8 520 copy(mixedBytes[index:index+offset], valueBytes[valueIndex:valueIndex+offset]) 521 valueIndex += offset 522 index += offset 523 } 524 return mixedBytes 525 } 526 527 // Size implements the EventBatchEncoder interface 528 func (d *JSONEventBatchEncoder) Size() int { 529 return d.keyBuf.Len() + d.valueBuf.Len() 530 } 531 532 // Reset implements the EventBatchEncoder interface 533 func (d *JSONEventBatchEncoder) Reset() { 534 d.keyBuf.Reset() 535 d.valueBuf.Reset() 536 } 537 538 // SetParams reads relevant parameters for Open Protocol 539 func (d *JSONEventBatchEncoder) SetParams(params map[string]string) error { 540 var err error 541 if maxMessageBytes, ok := params["max-message-bytes"]; ok { 542 d.maxKafkaMessageSize, err = strconv.Atoi(maxMessageBytes) 543 if err != nil { 544 return cerror.ErrKafkaInvalidConfig.Wrap(err) 545 } 546 } else { 547 d.maxKafkaMessageSize = DefaultMaxMessageBytes 548 } 549 550 if d.maxKafkaMessageSize <= 0 { 551 return cerror.ErrKafkaInvalidConfig.Wrap(errors.Errorf("invalid max-message-bytes %d", d.maxKafkaMessageSize)) 552 } 553 554 if maxBatchSize, ok := params["max-batch-size"]; ok { 555 d.maxBatchSize, err = strconv.Atoi(maxBatchSize) 556 if err != nil { 557 return cerror.ErrKafkaInvalidConfig.Wrap(err) 558 } 559 } else { 560 d.maxBatchSize = DefaultMaxBatchSize 561 } 562 563 if d.maxBatchSize <= 0 { 564 return cerror.ErrKafkaInvalidConfig.Wrap(errors.Errorf("invalid max-batch-size %d", d.maxBatchSize)) 565 } 566 return nil 567 } 568 569 // NewJSONEventBatchEncoder creates a new JSONEventBatchEncoder. 570 func NewJSONEventBatchEncoder() EventBatchEncoder { 571 batch := &JSONEventBatchEncoder{ 572 keyBuf: &bytes.Buffer{}, 573 valueBuf: &bytes.Buffer{}, 574 } 575 var versionByte [8]byte 576 binary.BigEndian.PutUint64(versionByte[:], BatchVersion1) 577 batch.keyBuf.Write(versionByte[:]) 578 return batch 579 } 580 581 // JSONEventBatchMixedDecoder decodes the byte of a batch into the original messages. 582 type JSONEventBatchMixedDecoder struct { 583 mixedBytes []byte 584 nextKey *mqMessageKey 585 nextKeyLen uint64 586 } 587 588 // HasNext implements the EventBatchDecoder interface 589 func (b *JSONEventBatchMixedDecoder) HasNext() (model.MqMessageType, bool, error) { 590 if !b.hasNext() { 591 return 0, false, nil 592 } 593 if err := b.decodeNextKey(); err != nil { 594 return 0, false, err 595 } 596 return b.nextKey.Type, true, nil 597 } 598 599 // NextResolvedEvent implements the EventBatchDecoder interface 600 func (b *JSONEventBatchMixedDecoder) NextResolvedEvent() (uint64, error) { 601 if b.nextKey == nil { 602 if err := b.decodeNextKey(); err != nil { 603 return 0, err 604 } 605 } 606 b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] 607 if b.nextKey.Type != model.MqMessageTypeResolved { 608 return 0, cerror.ErrJSONCodecInvalidData.GenWithStack("not found resolved event message") 609 } 610 valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) 611 b.mixedBytes = b.mixedBytes[valueLen+8:] 612 resolvedTs := b.nextKey.Ts 613 b.nextKey = nil 614 return resolvedTs, nil 615 } 616 617 // NextRowChangedEvent implements the EventBatchDecoder interface 618 func (b *JSONEventBatchMixedDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { 619 if b.nextKey == nil { 620 if err := b.decodeNextKey(); err != nil { 621 return nil, err 622 } 623 } 624 b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] 625 if b.nextKey.Type != model.MqMessageTypeRow { 626 return nil, cerror.ErrJSONCodecInvalidData.GenWithStack("not found row event message") 627 } 628 valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) 629 value := b.mixedBytes[8 : valueLen+8] 630 b.mixedBytes = b.mixedBytes[valueLen+8:] 631 rowMsg := new(mqMessageRow) 632 if err := rowMsg.Decode(value); err != nil { 633 return nil, errors.Trace(err) 634 } 635 rowEvent := mqMessageToRowEvent(b.nextKey, rowMsg) 636 b.nextKey = nil 637 return rowEvent, nil 638 } 639 640 // NextDDLEvent implements the EventBatchDecoder interface 641 func (b *JSONEventBatchMixedDecoder) NextDDLEvent() (*model.DDLEvent, error) { 642 if b.nextKey == nil { 643 if err := b.decodeNextKey(); err != nil { 644 return nil, err 645 } 646 } 647 b.mixedBytes = b.mixedBytes[b.nextKeyLen+8:] 648 if b.nextKey.Type != model.MqMessageTypeDDL { 649 return nil, cerror.ErrJSONCodecInvalidData.GenWithStack("not found ddl event message") 650 } 651 valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) 652 value := b.mixedBytes[8 : valueLen+8] 653 b.mixedBytes = b.mixedBytes[valueLen+8:] 654 ddlMsg := new(mqMessageDDL) 655 if err := ddlMsg.Decode(value); err != nil { 656 return nil, errors.Trace(err) 657 } 658 ddlEvent := mqMessageToDDLEvent(b.nextKey, ddlMsg) 659 b.nextKey = nil 660 return ddlEvent, nil 661 } 662 663 func (b *JSONEventBatchMixedDecoder) hasNext() bool { 664 return len(b.mixedBytes) > 0 665 } 666 667 func (b *JSONEventBatchMixedDecoder) decodeNextKey() error { 668 keyLen := binary.BigEndian.Uint64(b.mixedBytes[:8]) 669 key := b.mixedBytes[8 : keyLen+8] 670 // drop value bytes 671 msgKey := new(mqMessageKey) 672 err := msgKey.Decode(key) 673 if err != nil { 674 return errors.Trace(err) 675 } 676 b.nextKey = msgKey 677 b.nextKeyLen = keyLen 678 return nil 679 } 680 681 // JSONEventBatchDecoder decodes the byte of a batch into the original messages. 682 type JSONEventBatchDecoder struct { 683 keyBytes []byte 684 valueBytes []byte 685 nextKey *mqMessageKey 686 nextKeyLen uint64 687 } 688 689 // HasNext implements the EventBatchDecoder interface 690 func (b *JSONEventBatchDecoder) HasNext() (model.MqMessageType, bool, error) { 691 if !b.hasNext() { 692 return 0, false, nil 693 } 694 if err := b.decodeNextKey(); err != nil { 695 return 0, false, err 696 } 697 return b.nextKey.Type, true, nil 698 } 699 700 // NextResolvedEvent implements the EventBatchDecoder interface 701 func (b *JSONEventBatchDecoder) NextResolvedEvent() (uint64, error) { 702 if b.nextKey == nil { 703 if err := b.decodeNextKey(); err != nil { 704 return 0, err 705 } 706 } 707 b.keyBytes = b.keyBytes[b.nextKeyLen+8:] 708 if b.nextKey.Type != model.MqMessageTypeResolved { 709 return 0, cerror.ErrJSONCodecInvalidData.GenWithStack("not found resolved event message") 710 } 711 valueLen := binary.BigEndian.Uint64(b.valueBytes[:8]) 712 b.valueBytes = b.valueBytes[valueLen+8:] 713 resolvedTs := b.nextKey.Ts 714 b.nextKey = nil 715 return resolvedTs, nil 716 } 717 718 // NextRowChangedEvent implements the EventBatchDecoder interface 719 func (b *JSONEventBatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { 720 if b.nextKey == nil { 721 if err := b.decodeNextKey(); err != nil { 722 return nil, err 723 } 724 } 725 b.keyBytes = b.keyBytes[b.nextKeyLen+8:] 726 if b.nextKey.Type != model.MqMessageTypeRow { 727 return nil, cerror.ErrJSONCodecInvalidData.GenWithStack("not found row event message") 728 } 729 valueLen := binary.BigEndian.Uint64(b.valueBytes[:8]) 730 value := b.valueBytes[8 : valueLen+8] 731 b.valueBytes = b.valueBytes[valueLen+8:] 732 rowMsg := new(mqMessageRow) 733 if err := rowMsg.Decode(value); err != nil { 734 return nil, errors.Trace(err) 735 } 736 rowEvent := mqMessageToRowEvent(b.nextKey, rowMsg) 737 b.nextKey = nil 738 return rowEvent, nil 739 } 740 741 // NextDDLEvent implements the EventBatchDecoder interface 742 func (b *JSONEventBatchDecoder) NextDDLEvent() (*model.DDLEvent, error) { 743 if b.nextKey == nil { 744 if err := b.decodeNextKey(); err != nil { 745 return nil, err 746 } 747 } 748 b.keyBytes = b.keyBytes[b.nextKeyLen+8:] 749 if b.nextKey.Type != model.MqMessageTypeDDL { 750 return nil, cerror.ErrJSONCodecInvalidData.GenWithStack("not found ddl event message") 751 } 752 valueLen := binary.BigEndian.Uint64(b.valueBytes[:8]) 753 value := b.valueBytes[8 : valueLen+8] 754 b.valueBytes = b.valueBytes[valueLen+8:] 755 ddlMsg := new(mqMessageDDL) 756 if err := ddlMsg.Decode(value); err != nil { 757 return nil, errors.Trace(err) 758 } 759 ddlEvent := mqMessageToDDLEvent(b.nextKey, ddlMsg) 760 b.nextKey = nil 761 return ddlEvent, nil 762 } 763 764 func (b *JSONEventBatchDecoder) hasNext() bool { 765 return len(b.keyBytes) > 0 && len(b.valueBytes) > 0 766 } 767 768 func (b *JSONEventBatchDecoder) decodeNextKey() error { 769 keyLen := binary.BigEndian.Uint64(b.keyBytes[:8]) 770 key := b.keyBytes[8 : keyLen+8] 771 msgKey := new(mqMessageKey) 772 err := msgKey.Decode(key) 773 if err != nil { 774 return errors.Trace(err) 775 } 776 b.nextKey = msgKey 777 b.nextKeyLen = keyLen 778 return nil 779 } 780 781 // NewJSONEventBatchDecoder creates a new JSONEventBatchDecoder. 782 func NewJSONEventBatchDecoder(key []byte, value []byte) (EventBatchDecoder, error) { 783 version := binary.BigEndian.Uint64(key[:8]) 784 key = key[8:] 785 if version != BatchVersion1 { 786 return nil, cerror.ErrJSONCodecInvalidData.GenWithStack("unexpected key format version") 787 } 788 // if only decode one byte slice, we choose MixedDecoder 789 if len(key) > 0 && len(value) == 0 { 790 return &JSONEventBatchMixedDecoder{ 791 mixedBytes: key, 792 }, nil 793 } 794 return &JSONEventBatchDecoder{ 795 keyBytes: key, 796 valueBytes: value, 797 }, nil 798 }