github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/open/open_protocol_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 open 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/binary" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/cdc/model" 24 "github.com/pingcap/tiflow/pkg/config" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/sink/codec" 27 "github.com/pingcap/tiflow/pkg/sink/codec/common" 28 "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" 29 "go.uber.org/zap" 30 ) 31 32 // BatchEncoder encodes the events into the byte of a batch into. 33 type BatchEncoder struct { 34 messageBuf []*common.Message 35 callbackBuff []func() 36 curBatchSize int 37 38 claimCheck *claimcheck.ClaimCheck 39 40 config *common.Config 41 } 42 43 func (d *BatchEncoder) buildMessageOnlyHandleKeyColumns(e *model.RowChangedEvent) ([]byte, []byte, error) { 44 // set the `largeMessageOnlyHandleKeyColumns` to true to only encode handle key columns. 45 keyMsg, valueMsg, err := rowChangeToMsg(e, d.config, true) 46 if err != nil { 47 return nil, nil, errors.Trace(err) 48 } 49 key, err := keyMsg.Encode() 50 if err != nil { 51 return nil, nil, errors.Trace(err) 52 } 53 value, err := valueMsg.encode() 54 if err != nil { 55 return nil, nil, errors.Trace(err) 56 } 57 58 value, err = common.Compress( 59 d.config.ChangefeedID, d.config.LargeMessageHandle.LargeMessageHandleCompression, value, 60 ) 61 if err != nil { 62 return nil, nil, err 63 } 64 65 // for single message that is longer than max-message-bytes 66 // 16 is the length of `keyLenByte` and `valueLenByte`, 8 is the length of `versionHead` 67 length := len(key) + len(value) + common.MaxRecordOverhead + 16 + 8 68 if length > d.config.MaxMessageBytes { 69 log.Warn("Single message is too large for open-protocol, only encode handle key columns", 70 zap.Int("maxMessageBytes", d.config.MaxMessageBytes), 71 zap.Int("length", length), 72 zap.Any("table", e.TableInfo.TableName), 73 zap.Any("key", key)) 74 return nil, nil, cerror.ErrMessageTooLarge.GenWithStackByArgs() 75 } 76 77 return key, value, nil 78 } 79 80 // AppendRowChangedEvent implements the RowEventEncoder interface 81 func (d *BatchEncoder) AppendRowChangedEvent( 82 ctx context.Context, 83 _ string, 84 e *model.RowChangedEvent, 85 callback func(), 86 ) error { 87 keyMsg, valueMsg, err := rowChangeToMsg(e, d.config, false) 88 if err != nil { 89 return errors.Trace(err) 90 } 91 key, err := keyMsg.Encode() 92 if err != nil { 93 return errors.Trace(err) 94 } 95 value, err := valueMsg.encode() 96 if err != nil { 97 return errors.Trace(err) 98 } 99 100 value, err = common.Compress( 101 d.config.ChangefeedID, d.config.LargeMessageHandle.LargeMessageHandleCompression, value, 102 ) 103 if err != nil { 104 return errors.Trace(err) 105 } 106 107 // for single message that is longer than max-message-bytes 108 // 16 is the length of `keyLenByte` and `valueLenByte`, 8 is the length of `versionHead` 109 length := len(key) + len(value) + common.MaxRecordOverhead + 16 + 8 110 if length > d.config.MaxMessageBytes { 111 if d.config.LargeMessageHandle.Disabled() { 112 log.Warn("Single message is too large for open-protocol", 113 zap.Int("maxMessageBytes", d.config.MaxMessageBytes), 114 zap.Int("length", length), 115 zap.Any("table", e.TableInfo.TableName), 116 zap.Any("key", key)) 117 return cerror.ErrMessageTooLarge.GenWithStackByArgs() 118 } 119 120 if d.config.LargeMessageHandle.EnableClaimCheck() { 121 // send the large message to the external storage first, then 122 // create a new message contains the reference of the large message. 123 claimCheckFileName := claimcheck.NewFileName() 124 m := newMessage(key, value) 125 err = d.claimCheck.WriteMessage(ctx, m.Key, m.Value, claimCheckFileName) 126 if err != nil { 127 return errors.Trace(err) 128 } 129 130 key, value, err = d.newClaimCheckLocationMessage(e, claimCheckFileName) 131 if err != nil { 132 return errors.Trace(err) 133 } 134 } 135 136 if d.config.LargeMessageHandle.HandleKeyOnly() { 137 // it's must that `LargeMessageHandle == LargeMessageHandleOnlyHandleKeyColumns` here. 138 key, value, err = d.buildMessageOnlyHandleKeyColumns(e) 139 if err != nil { 140 return errors.Trace(err) 141 } 142 } 143 } 144 145 if len(d.messageBuf) == 0 || 146 d.curBatchSize >= d.config.MaxBatchSize || 147 d.messageBuf[len(d.messageBuf)-1].Length()+len(key)+len(value)+16 > d.config.MaxMessageBytes { 148 // Before we create a new message, we should handle the previous callbacks. 149 d.tryBuildCallback() 150 versionHead := make([]byte, 8) 151 binary.BigEndian.PutUint64(versionHead, codec.BatchVersion1) 152 msg := common.NewMsg(config.ProtocolOpen, versionHead, nil, 153 0, model.MessageTypeRow, nil, nil) 154 d.messageBuf = append(d.messageBuf, msg) 155 d.curBatchSize = 0 156 } 157 158 var ( 159 keyLenByte [8]byte 160 valueLenByte [8]byte 161 ) 162 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 163 binary.BigEndian.PutUint64(valueLenByte[:], uint64(len(value))) 164 165 message := d.messageBuf[len(d.messageBuf)-1] 166 message.Key = append(message.Key, keyLenByte[:]...) 167 message.Key = append(message.Key, key...) 168 message.Value = append(message.Value, valueLenByte[:]...) 169 message.Value = append(message.Value, value...) 170 message.Ts = e.CommitTs 171 message.Schema = e.TableInfo.GetSchemaNamePtr() 172 message.Table = e.TableInfo.GetTableNamePtr() 173 message.IncRowsCount() 174 175 if callback != nil { 176 d.callbackBuff = append(d.callbackBuff, callback) 177 } 178 179 d.curBatchSize++ 180 return nil 181 } 182 183 func newMessage(key, value []byte) *common.Message { 184 versionHead := make([]byte, 8) 185 binary.BigEndian.PutUint64(versionHead, codec.BatchVersion1) 186 message := common.NewMsg(config.ProtocolOpen, versionHead, nil, 0, model.MessageTypeRow, nil, nil) 187 188 var ( 189 keyLenByte [8]byte 190 valueLenByte [8]byte 191 ) 192 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 193 binary.BigEndian.PutUint64(valueLenByte[:], uint64(len(value))) 194 195 message.Key = append(message.Key, keyLenByte[:]...) 196 message.Key = append(message.Key, key...) 197 message.Value = append(message.Value, valueLenByte[:]...) 198 message.Value = append(message.Value, value...) 199 200 return message 201 } 202 203 // EncodeDDLEvent implements the RowEventEncoder interface 204 func (d *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { 205 keyMsg, valueMsg := ddlEventToMsg(e) 206 key, err := keyMsg.Encode() 207 if err != nil { 208 return nil, errors.Trace(err) 209 } 210 value, err := valueMsg.encode() 211 if err != nil { 212 return nil, errors.Trace(err) 213 } 214 215 value, err = common.Compress( 216 d.config.ChangefeedID, d.config.LargeMessageHandle.LargeMessageHandleCompression, value, 217 ) 218 if err != nil { 219 return nil, errors.Trace(err) 220 } 221 222 var keyLenByte [8]byte 223 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 224 var valueLenByte [8]byte 225 binary.BigEndian.PutUint64(valueLenByte[:], uint64(len(value))) 226 227 keyBuf := new(bytes.Buffer) 228 var versionByte [8]byte 229 binary.BigEndian.PutUint64(versionByte[:], codec.BatchVersion1) 230 keyBuf.Write(versionByte[:]) 231 keyBuf.Write(keyLenByte[:]) 232 keyBuf.Write(key) 233 234 valueBuf := new(bytes.Buffer) 235 valueBuf.Write(valueLenByte[:]) 236 valueBuf.Write(value) 237 238 ret := common.NewDDLMsg(config.ProtocolOpen, keyBuf.Bytes(), valueBuf.Bytes(), e) 239 return ret, nil 240 } 241 242 // EncodeCheckpointEvent implements the RowEventEncoder interface 243 func (d *BatchEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { 244 keyMsg := newResolvedMessage(ts) 245 key, err := keyMsg.Encode() 246 if err != nil { 247 return nil, errors.Trace(err) 248 } 249 250 var keyLenByte [8]byte 251 binary.BigEndian.PutUint64(keyLenByte[:], uint64(len(key))) 252 var valueLenByte [8]byte 253 binary.BigEndian.PutUint64(valueLenByte[:], 0) 254 255 keyBuf := new(bytes.Buffer) 256 var versionByte [8]byte 257 binary.BigEndian.PutUint64(versionByte[:], codec.BatchVersion1) 258 keyBuf.Write(versionByte[:]) 259 keyBuf.Write(keyLenByte[:]) 260 keyBuf.Write(key) 261 262 valueBuf := new(bytes.Buffer) 263 valueBuf.Write(valueLenByte[:]) 264 265 ret := common.NewResolvedMsg(config.ProtocolOpen, keyBuf.Bytes(), valueBuf.Bytes(), ts) 266 return ret, nil 267 } 268 269 // Build implements the RowEventEncoder interface 270 func (d *BatchEncoder) Build() (messages []*common.Message) { 271 d.tryBuildCallback() 272 ret := d.messageBuf 273 d.messageBuf = make([]*common.Message, 0) 274 return ret 275 } 276 277 // tryBuildCallback will collect all the callbacks into one message's callback. 278 func (d *BatchEncoder) tryBuildCallback() { 279 if len(d.messageBuf) != 0 && len(d.callbackBuff) != 0 { 280 lastMsg := d.messageBuf[len(d.messageBuf)-1] 281 callbacks := d.callbackBuff 282 lastMsg.Callback = func() { 283 for _, cb := range callbacks { 284 cb() 285 } 286 } 287 d.callbackBuff = make([]func(), 0) 288 } 289 } 290 291 // NewClaimCheckLocationMessage implement the ClaimCheckLocationEncoder interface. 292 func (d *BatchEncoder) newClaimCheckLocationMessage( 293 event *model.RowChangedEvent, fileName string, 294 ) ([]byte, []byte, error) { 295 keyMsg, valueMsg, err := rowChangeToMsg(event, d.config, true) 296 if err != nil { 297 return nil, nil, errors.Trace(err) 298 } 299 300 keyMsg.OnlyHandleKey = false 301 keyMsg.ClaimCheckLocation = d.claimCheck.FileNameWithPrefix(fileName) 302 key, err := keyMsg.Encode() 303 if err != nil { 304 return nil, nil, errors.Trace(err) 305 } 306 307 value, err := valueMsg.encode() 308 if err != nil { 309 return nil, nil, errors.Trace(err) 310 } 311 312 value, err = common.Compress( 313 d.config.ChangefeedID, d.config.LargeMessageHandle.LargeMessageHandleCompression, value, 314 ) 315 if err != nil { 316 return nil, nil, errors.Trace(err) 317 } 318 319 // for single message that is longer than max-message-bytes 320 // 16 is the length of `keyLenByte` and `valueLenByte`, 8 is the length of `versionHead` 321 length := len(key) + len(value) + common.MaxRecordOverhead + 16 + 8 322 if length > d.config.MaxMessageBytes { 323 log.Warn("Single message is too large for open-protocol, "+ 324 "when create the claim-check location message", 325 zap.Int("maxMessageBytes", d.config.MaxMessageBytes), 326 zap.Int("length", length), 327 zap.Any("key", key)) 328 return nil, nil, cerror.ErrMessageTooLarge.GenWithStackByArgs() 329 } 330 331 return key, value, nil 332 } 333 334 type batchEncoderBuilder struct { 335 claimCheck *claimcheck.ClaimCheck 336 config *common.Config 337 } 338 339 // Build a BatchEncoder 340 func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { 341 return NewBatchEncoder(b.config, b.claimCheck) 342 } 343 344 func (b *batchEncoderBuilder) CleanMetrics() { 345 if b.claimCheck != nil { 346 b.claimCheck.CleanMetrics() 347 } 348 } 349 350 // NewBatchEncoderBuilder creates an open-protocol batchEncoderBuilder. 351 func NewBatchEncoderBuilder( 352 ctx context.Context, config *common.Config, 353 ) (codec.RowEventEncoderBuilder, error) { 354 var ( 355 claimCheck *claimcheck.ClaimCheck 356 err error 357 ) 358 if config.LargeMessageHandle.EnableClaimCheck() { 359 claimCheck, err = claimcheck.New(ctx, config.LargeMessageHandle.ClaimCheckStorageURI, config.ChangefeedID) 360 if err != nil { 361 return nil, errors.Trace(err) 362 } 363 } 364 return &batchEncoderBuilder{ 365 config: config, 366 claimCheck: claimCheck, 367 }, nil 368 } 369 370 // NewBatchEncoder creates a new BatchEncoder. 371 func NewBatchEncoder(config *common.Config, claimCheck *claimcheck.ClaimCheck) codec.RowEventEncoder { 372 return &BatchEncoder{ 373 config: config, 374 claimCheck: claimCheck, 375 } 376 }