github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/canal/canal_json_decoder.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 "bytes" 18 "context" 19 "database/sql" 20 "path/filepath" 21 "strconv" 22 "strings" 23 24 "github.com/goccy/go-json" 25 "github.com/pingcap/errors" 26 "github.com/pingcap/log" 27 "github.com/pingcap/tidb/br/pkg/storage" 28 "github.com/pingcap/tiflow/cdc/model" 29 cerror "github.com/pingcap/tiflow/pkg/errors" 30 "github.com/pingcap/tiflow/pkg/sink/codec" 31 "github.com/pingcap/tiflow/pkg/sink/codec/common" 32 "github.com/pingcap/tiflow/pkg/sink/codec/utils" 33 "github.com/pingcap/tiflow/pkg/util" 34 "go.uber.org/zap" 35 "golang.org/x/text/encoding" 36 "golang.org/x/text/encoding/charmap" 37 ) 38 39 // batchDecoder decodes the byte into the original message. 40 type batchDecoder struct { 41 data []byte 42 msg canalJSONMessageInterface 43 44 config *common.Config 45 46 storage storage.ExternalStorage 47 48 upstreamTiDB *sql.DB 49 bytesDecoder *encoding.Decoder 50 } 51 52 // NewBatchDecoder return a decoder for canal-json 53 func NewBatchDecoder( 54 ctx context.Context, codecConfig *common.Config, db *sql.DB, 55 ) (codec.RowEventDecoder, error) { 56 var ( 57 externalStorage storage.ExternalStorage 58 err error 59 ) 60 if codecConfig.LargeMessageHandle.EnableClaimCheck() { 61 storageURI := codecConfig.LargeMessageHandle.ClaimCheckStorageURI 62 externalStorage, err = util.GetExternalStorageFromURI(ctx, storageURI) 63 if err != nil { 64 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 65 } 66 } 67 68 if codecConfig.LargeMessageHandle.HandleKeyOnly() && db == nil { 69 return nil, cerror.ErrCodecDecode. 70 GenWithStack("handle-key-only is enabled, but upstream TiDB is not provided") 71 } 72 73 return &batchDecoder{ 74 config: codecConfig, 75 storage: externalStorage, 76 upstreamTiDB: db, 77 bytesDecoder: charmap.ISO8859_1.NewDecoder(), 78 }, nil 79 } 80 81 // AddKeyValue implements the RowEventDecoder interface 82 func (b *batchDecoder) AddKeyValue(_, value []byte) error { 83 value, err := common.Decompress(b.config.LargeMessageHandle.LargeMessageHandleCompression, value) 84 if err != nil { 85 log.Error("decompress data failed", 86 zap.String("compression", b.config.LargeMessageHandle.LargeMessageHandleCompression), 87 zap.Error(err)) 88 89 return errors.Trace(err) 90 } 91 b.data = value 92 return nil 93 } 94 95 // HasNext implements the RowEventDecoder interface 96 func (b *batchDecoder) HasNext() (model.MessageType, bool, error) { 97 if b.data == nil { 98 return model.MessageTypeUnknown, false, nil 99 } 100 var ( 101 msg canalJSONMessageInterface = &JSONMessage{} 102 encodedData []byte 103 ) 104 105 if b.config.EnableTiDBExtension { 106 msg = &canalJSONMessageWithTiDBExtension{ 107 JSONMessage: &JSONMessage{}, 108 Extensions: &tidbExtension{}, 109 } 110 } 111 112 if len(b.config.Terminator) > 0 { 113 idx := bytes.IndexAny(b.data, b.config.Terminator) 114 if idx >= 0 { 115 encodedData = b.data[:idx] 116 b.data = b.data[idx+len(b.config.Terminator):] 117 } else { 118 encodedData = b.data 119 b.data = nil 120 } 121 } else { 122 encodedData = b.data 123 b.data = nil 124 } 125 126 if len(encodedData) == 0 { 127 return model.MessageTypeUnknown, false, nil 128 } 129 130 if err := json.Unmarshal(encodedData, msg); err != nil { 131 log.Error("canal-json decoder unmarshal data failed", 132 zap.Error(err), zap.ByteString("data", encodedData)) 133 return model.MessageTypeUnknown, false, err 134 } 135 b.msg = msg 136 return b.msg.messageType(), true, nil 137 } 138 139 func (b *batchDecoder) assembleClaimCheckRowChangedEvent(ctx context.Context, claimCheckLocation string) (*model.RowChangedEvent, error) { 140 _, claimCheckFileName := filepath.Split(claimCheckLocation) 141 data, err := b.storage.ReadFile(ctx, claimCheckFileName) 142 if err != nil { 143 return nil, err 144 } 145 claimCheckM, err := common.UnmarshalClaimCheckMessage(data) 146 if err != nil { 147 return nil, err 148 } 149 150 value, err := common.Decompress(b.config.LargeMessageHandle.LargeMessageHandleCompression, claimCheckM.Value) 151 if err != nil { 152 return nil, err 153 } 154 message := &canalJSONMessageWithTiDBExtension{} 155 err = json.Unmarshal(value, message) 156 if err != nil { 157 return nil, err 158 } 159 160 b.msg = message 161 return b.NextRowChangedEvent() 162 } 163 164 func (b *batchDecoder) buildData(holder *common.ColumnsHolder) (map[string]interface{}, map[string]string, error) { 165 columnsCount := holder.Length() 166 data := make(map[string]interface{}, columnsCount) 167 mysqlTypeMap := make(map[string]string, columnsCount) 168 169 for i := 0; i < columnsCount; i++ { 170 t := holder.Types[i] 171 name := holder.Types[i].Name() 172 mysqlType := strings.ToLower(t.DatabaseTypeName()) 173 174 var value string 175 rawValue := holder.Values[i].([]uint8) 176 if utils.IsBinaryMySQLType(mysqlType) { 177 rawValue, err := b.bytesDecoder.Bytes(rawValue) 178 if err != nil { 179 return nil, nil, errors.Trace(err) 180 } 181 value = string(rawValue) 182 } else if strings.Contains(mysqlType, "bit") || strings.Contains(mysqlType, "set") { 183 bitValue := common.MustBinaryLiteralToInt(rawValue) 184 value = strconv.FormatUint(bitValue, 10) 185 } else { 186 value = string(rawValue) 187 } 188 mysqlTypeMap[name] = mysqlType 189 data[name] = value 190 } 191 192 return data, mysqlTypeMap, nil 193 } 194 195 func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( 196 ctx context.Context, message *canalJSONMessageWithTiDBExtension, 197 ) (*model.RowChangedEvent, error) { 198 var ( 199 commitTs = message.Extensions.CommitTs 200 schema = message.Schema 201 table = message.Table 202 eventType = message.EventType 203 ) 204 205 handleKeyData := message.getData() 206 pkNames := make([]string, 0, len(handleKeyData)) 207 for name := range handleKeyData { 208 pkNames = append(pkNames, name) 209 } 210 211 result := &canalJSONMessageWithTiDBExtension{ 212 JSONMessage: &JSONMessage{ 213 Schema: schema, 214 Table: table, 215 PKNames: pkNames, 216 217 EventType: eventType, 218 }, 219 Extensions: &tidbExtension{ 220 CommitTs: commitTs, 221 }, 222 } 223 switch eventType { 224 case "INSERT": 225 holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, handleKeyData) 226 data, mysqlType, err := b.buildData(holder) 227 if err != nil { 228 return nil, err 229 } 230 result.MySQLType = mysqlType 231 result.Data = []map[string]interface{}{data} 232 case "UPDATE": 233 holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, handleKeyData) 234 data, mysqlType, err := b.buildData(holder) 235 if err != nil { 236 return nil, err 237 } 238 result.MySQLType = mysqlType 239 result.Data = []map[string]interface{}{data} 240 241 holder = common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, message.getOld()) 242 old, _, err := b.buildData(holder) 243 if err != nil { 244 return nil, err 245 } 246 result.Old = []map[string]interface{}{old} 247 case "DELETE": 248 holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, handleKeyData) 249 data, mysqlType, err := b.buildData(holder) 250 if err != nil { 251 return nil, err 252 } 253 result.MySQLType = mysqlType 254 result.Data = []map[string]interface{}{data} 255 } 256 257 b.msg = result 258 return b.NextRowChangedEvent() 259 } 260 261 // NextRowChangedEvent implements the RowEventDecoder interface 262 // `HasNext` should be called before this. 263 func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { 264 if b.msg == nil || b.msg.messageType() != model.MessageTypeRow { 265 return nil, cerror.ErrCanalDecodeFailed. 266 GenWithStack("not found row changed event message") 267 } 268 269 message, withExtension := b.msg.(*canalJSONMessageWithTiDBExtension) 270 if withExtension { 271 ctx := context.Background() 272 if message.Extensions.OnlyHandleKey { 273 return b.assembleHandleKeyOnlyRowChangedEvent(ctx, message) 274 } 275 if message.Extensions.ClaimCheckLocation != "" { 276 return b.assembleClaimCheckRowChangedEvent(ctx, message.Extensions.ClaimCheckLocation) 277 } 278 } 279 280 result, err := canalJSONMessage2RowChange(b.msg) 281 if err != nil { 282 return nil, err 283 } 284 b.msg = nil 285 return result, nil 286 } 287 288 // NextDDLEvent implements the RowEventDecoder interface 289 // `HasNext` should be called before this. 290 func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) { 291 if b.msg == nil || b.msg.messageType() != model.MessageTypeDDL { 292 return nil, cerror.ErrCanalDecodeFailed. 293 GenWithStack("not found ddl event message") 294 } 295 296 result := canalJSONMessage2DDLEvent(b.msg) 297 b.msg = nil 298 return result, nil 299 } 300 301 // NextResolvedEvent implements the RowEventDecoder interface 302 // `HasNext` should be called before this. 303 func (b *batchDecoder) NextResolvedEvent() (uint64, error) { 304 if b.msg == nil || b.msg.messageType() != model.MessageTypeResolved { 305 return 0, cerror.ErrCanalDecodeFailed. 306 GenWithStack("not found resolved event message") 307 } 308 309 withExtensionEvent, ok := b.msg.(*canalJSONMessageWithTiDBExtension) 310 if !ok { 311 log.Error("canal-json resolved event message should have tidb extension, but not found", 312 zap.Any("msg", b.msg)) 313 return 0, cerror.ErrCanalDecodeFailed. 314 GenWithStack("MessageTypeResolved tidb extension not found") 315 } 316 b.msg = nil 317 return withExtensionEvent.Extensions.WatermarkTs, nil 318 }