github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/canal/canal_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 19 "github.com/golang/protobuf/proto" 20 "github.com/pingcap/errors" 21 "github.com/pingcap/log" 22 "github.com/pingcap/tiflow/cdc/model" 23 "github.com/pingcap/tiflow/pkg/config" 24 cerror "github.com/pingcap/tiflow/pkg/errors" 25 "github.com/pingcap/tiflow/pkg/sink/codec" 26 "github.com/pingcap/tiflow/pkg/sink/codec/common" 27 canal "github.com/pingcap/tiflow/proto/canal" 28 "go.uber.org/zap" 29 ) 30 31 // BatchEncoder encodes the events into the byte of a batch into. 32 type BatchEncoder struct { 33 messages *canal.Messages 34 callbackBuf []func() 35 packet *canal.Packet 36 entryBuilder *canalEntryBuilder 37 38 config *common.Config 39 } 40 41 // EncodeCheckpointEvent implements the RowEventEncoder interface 42 func (d *BatchEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { 43 // For canal now, there is no such a corresponding type to ResolvedEvent so far. 44 // Therefore, the event is ignored. 45 return nil, nil 46 } 47 48 // AppendRowChangedEvent implements the RowEventEncoder interface 49 func (d *BatchEncoder) AppendRowChangedEvent( 50 _ context.Context, 51 _ string, 52 e *model.RowChangedEvent, 53 callback func(), 54 ) error { 55 entry, err := d.entryBuilder.fromRowEvent(e, d.config.DeleteOnlyHandleKeyColumns) 56 if err != nil { 57 return errors.Trace(err) 58 } 59 b, err := proto.Marshal(entry) 60 if err != nil { 61 return cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 62 } 63 d.messages.Messages = append(d.messages.Messages, b) 64 if callback != nil { 65 d.callbackBuf = append(d.callbackBuf, callback) 66 } 67 return nil 68 } 69 70 // EncodeDDLEvent implements the RowEventEncoder interface 71 func (d *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { 72 entry, err := d.entryBuilder.fromDDLEvent(e) 73 if err != nil { 74 return nil, errors.Trace(err) 75 } 76 b, err := proto.Marshal(entry) 77 if err != nil { 78 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 79 } 80 81 messages := new(canal.Messages) 82 messages.Messages = append(messages.Messages, b) 83 b, err = messages.Marshal() 84 if err != nil { 85 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 86 } 87 88 packet := &canal.Packet{ 89 VersionPresent: &canal.Packet_Version{ 90 Version: CanalPacketVersion, 91 }, 92 Type: canal.PacketType_MESSAGES, 93 } 94 packet.Body = b 95 b, err = packet.Marshal() 96 if err != nil { 97 return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) 98 } 99 100 return common.NewDDLMsg(config.ProtocolCanal, nil, b, e), nil 101 } 102 103 // Build implements the RowEventEncoder interface 104 func (d *BatchEncoder) Build() []*common.Message { 105 rowCount := len(d.messages.Messages) 106 if rowCount == 0 { 107 return nil 108 } 109 110 err := d.refreshPacketBody() 111 if err != nil { 112 log.Panic("Error when generating Canal packet", zap.Error(err)) 113 } 114 115 value, err := proto.Marshal(d.packet) 116 if err != nil { 117 log.Panic("Error when serializing Canal packet", zap.Error(err)) 118 } 119 ret := common.NewMsg(config.ProtocolCanal, nil, value, 0, model.MessageTypeRow, nil, nil) 120 ret.SetRowsCount(rowCount) 121 d.messages.Reset() 122 d.resetPacket() 123 124 if len(d.callbackBuf) != 0 && len(d.callbackBuf) == rowCount { 125 callbacks := d.callbackBuf 126 ret.Callback = func() { 127 for _, cb := range callbacks { 128 cb() 129 } 130 } 131 d.callbackBuf = make([]func(), 0) 132 } 133 return []*common.Message{ret} 134 } 135 136 // refreshPacketBody() marshals the messages to the packet body 137 func (d *BatchEncoder) refreshPacketBody() error { 138 oldSize := len(d.packet.Body) 139 newSize := proto.Size(d.messages) 140 if newSize > oldSize { 141 // resize packet body slice 142 d.packet.Body = append(d.packet.Body, make([]byte, newSize-oldSize)...) 143 } else { 144 d.packet.Body = d.packet.Body[:newSize] 145 } 146 147 _, err := d.messages.MarshalToSizedBuffer(d.packet.Body) 148 return err 149 } 150 151 func (d *BatchEncoder) resetPacket() { 152 d.packet = &canal.Packet{ 153 VersionPresent: &canal.Packet_Version{ 154 Version: CanalPacketVersion, 155 }, 156 Type: canal.PacketType_MESSAGES, 157 } 158 } 159 160 // newBatchEncoder creates a new canalBatchEncoder. 161 func newBatchEncoder(config *common.Config) codec.RowEventEncoder { 162 encoder := &BatchEncoder{ 163 messages: &canal.Messages{}, 164 callbackBuf: make([]func(), 0), 165 entryBuilder: newCanalEntryBuilder(config), 166 167 config: config, 168 } 169 170 encoder.resetPacket() 171 return encoder 172 } 173 174 type batchEncoderBuilder struct { 175 config *common.Config 176 } 177 178 // Build a `canalBatchEncoder` 179 func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { 180 return newBatchEncoder(b.config) 181 } 182 183 // CleanMetrics is a no-op for canalBatchEncoder. 184 func (b *batchEncoderBuilder) CleanMetrics() {} 185 186 // NewBatchEncoderBuilder creates a canal batchEncoderBuilder. 187 func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { 188 return &batchEncoderBuilder{ 189 config: config, 190 } 191 }