github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/avro/avro_test.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 avro 15 16 import ( 17 "context" 18 "math/rand" 19 "testing" 20 "time" 21 22 "github.com/linkedin/goavro/v2" 23 timodel "github.com/pingcap/tidb/pkg/parser/model" 24 "github.com/pingcap/tidb/pkg/parser/mysql" 25 "github.com/pingcap/tiflow/cdc/model" 26 "github.com/pingcap/tiflow/pkg/config" 27 "github.com/pingcap/tiflow/pkg/sink/codec/common" 28 "github.com/pingcap/tiflow/pkg/sink/codec/utils" 29 "github.com/pingcap/tiflow/pkg/uuid" 30 "github.com/stretchr/testify/require" 31 ) 32 33 func TestDMLEventE2E(t *testing.T) { 34 codecConfig := common.NewConfig(config.ProtocolAvro) 35 codecConfig.EnableTiDBExtension = true 36 ctx, cancel := context.WithCancel(context.Background()) 37 defer cancel() 38 39 _, event, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) 40 colInfos := event.TableInfo.GetColInfosForRowChangedEvent() 41 42 rand.New(rand.NewSource(time.Now().Unix())).Shuffle(len(event.Columns), func(i, j int) { 43 event.Columns[i], event.Columns[j] = event.Columns[j], event.Columns[i] 44 colInfos[i], colInfos[j] = colInfos[j], colInfos[i] 45 }) 46 47 for _, decimalHandling := range []string{"precise", "string"} { 48 for _, unsignedBigintHandling := range []string{"long", "string"} { 49 codecConfig.AvroDecimalHandlingMode = decimalHandling 50 codecConfig.AvroBigintUnsignedHandlingMode = unsignedBigintHandling 51 52 encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) 53 require.NoError(t, err) 54 require.NotNil(t, encoder) 55 56 topic := "avro-test-topic" 57 err = encoder.AppendRowChangedEvent(ctx, topic, event, func() {}) 58 require.NoError(t, err) 59 60 messages := encoder.Build() 61 require.Len(t, messages, 1) 62 message := messages[0] 63 64 schemaM, err := NewConfluentSchemaManager(ctx, "http://127.0.0.1:8081", nil) 65 require.NoError(t, err) 66 67 decoder := NewDecoder(codecConfig, schemaM, topic) 68 err = decoder.AddKeyValue(message.Key, message.Value) 69 require.NoError(t, err) 70 71 messageType, exist, err := decoder.HasNext() 72 require.NoError(t, err) 73 require.True(t, exist) 74 require.Equal(t, model.MessageTypeRow, messageType) 75 76 decodedEvent, err := decoder.NextRowChangedEvent() 77 require.NoError(t, err) 78 require.NotNil(t, decodedEvent) 79 80 TeardownEncoderAndSchemaRegistry4Testing() 81 } 82 } 83 } 84 85 func TestDDLEventE2E(t *testing.T) { 86 codecConfig := common.NewConfig(config.ProtocolAvro) 87 codecConfig.EnableTiDBExtension = true 88 codecConfig.AvroEnableWatermark = true 89 90 encoder := NewAvroEncoder(model.DefaultNamespace, nil, codecConfig) 91 92 ddl, _, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) 93 message, err := encoder.EncodeDDLEvent(ddl) 94 require.NoError(t, err) 95 require.NotNil(t, message) 96 97 topic := "test-topic" 98 decoder := NewDecoder(codecConfig, nil, topic) 99 err = decoder.AddKeyValue(message.Key, message.Value) 100 require.NoError(t, err) 101 102 messageType, exist, err := decoder.HasNext() 103 require.NoError(t, err) 104 require.True(t, exist) 105 require.Equal(t, model.MessageTypeDDL, messageType) 106 107 decodedEvent, err := decoder.NextDDLEvent() 108 require.NoError(t, err) 109 require.NotNil(t, decodedEvent) 110 require.Equal(t, ddl.CommitTs, decodedEvent.CommitTs) 111 require.Equal(t, timodel.ActionCreateTable, decodedEvent.Type) 112 require.NotEmpty(t, decodedEvent.Query) 113 require.NotEmpty(t, decodedEvent.TableInfo.TableName.Schema) 114 require.NotEmpty(t, decodedEvent.TableInfo.TableName.Table) 115 } 116 117 func TestResolvedE2E(t *testing.T) { 118 t.Parallel() 119 120 codecConfig := common.NewConfig(config.ProtocolAvro) 121 codecConfig.EnableTiDBExtension = true 122 codecConfig.AvroEnableWatermark = true 123 124 encoder := NewAvroEncoder(model.DefaultNamespace, nil, codecConfig) 125 126 resolvedTs := uint64(1591943372224) 127 message, err := encoder.EncodeCheckpointEvent(resolvedTs) 128 require.NoError(t, err) 129 require.NotNil(t, message) 130 131 topic := "test-topic" 132 decoder := NewDecoder(codecConfig, nil, topic) 133 err = decoder.AddKeyValue(message.Key, message.Value) 134 require.NoError(t, err) 135 136 messageType, exist, err := decoder.HasNext() 137 require.NoError(t, err) 138 require.True(t, exist) 139 require.Equal(t, model.MessageTypeResolved, messageType) 140 141 obtained, err := decoder.NextResolvedEvent() 142 require.NoError(t, err) 143 require.Equal(t, resolvedTs, obtained) 144 } 145 146 func TestAvroEncode4EnableChecksum(t *testing.T) { 147 codecConfig := common.NewConfig(config.ProtocolAvro) 148 codecConfig.EnableTiDBExtension = true 149 codecConfig.EnableRowChecksum = true 150 codecConfig.AvroDecimalHandlingMode = "string" 151 codecConfig.AvroBigintUnsignedHandlingMode = "string" 152 153 ctx, cancel := context.WithCancel(context.Background()) 154 defer cancel() 155 156 encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) 157 defer TeardownEncoderAndSchemaRegistry4Testing() 158 require.NoError(t, err) 159 require.NotNil(t, encoder) 160 161 _, event, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) 162 topic := "default" 163 bin, err := encoder.encodeValue(ctx, "default", event) 164 require.NoError(t, err) 165 166 cid, data, err := extractConfluentSchemaIDAndBinaryData(bin) 167 require.NoError(t, err) 168 169 avroValueCodec, err := encoder.schemaM.Lookup(ctx, topic, schemaID{confluentSchemaID: cid}) 170 require.NoError(t, err) 171 172 res, _, err := avroValueCodec.NativeFromBinary(data) 173 require.NoError(t, err) 174 require.NotNil(t, res) 175 176 m, ok := res.(map[string]interface{}) 177 require.True(t, ok) 178 179 _, found := m[tidbRowLevelChecksum] 180 require.True(t, found) 181 182 _, found = m[tidbCorrupted] 183 require.True(t, found) 184 185 _, found = m[tidbChecksumVersion] 186 require.True(t, found) 187 } 188 189 func TestAvroEncode(t *testing.T) { 190 codecConfig := common.NewConfig(config.ProtocolAvro) 191 codecConfig.EnableTiDBExtension = true 192 193 ctx, cancel := context.WithCancel(context.Background()) 194 defer cancel() 195 196 encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) 197 defer TeardownEncoderAndSchemaRegistry4Testing() 198 require.NoError(t, err) 199 require.NotNil(t, encoder) 200 201 _, event, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) 202 topic := "default" 203 bin, err := encoder.encodeKey(ctx, topic, event) 204 require.NoError(t, err) 205 206 cid, data, err := extractConfluentSchemaIDAndBinaryData(bin) 207 require.NoError(t, err) 208 209 avroKeyCodec, err := encoder.schemaM.Lookup(ctx, topic, schemaID{confluentSchemaID: cid}) 210 require.NoError(t, err) 211 212 res, _, err := avroKeyCodec.NativeFromBinary(data) 213 require.NoError(t, err) 214 require.NotNil(t, res) 215 for k := range res.(map[string]interface{}) { 216 if k == "_tidb_commit_ts" || k == "_tidb_op" || k == "_tidb_commit_physical_time" { 217 require.Fail(t, "key shall not include extension fields") 218 } 219 } 220 require.Equal(t, int32(127), res.(map[string]interface{})["t"]) 221 222 bin, err = encoder.encodeValue(ctx, topic, event) 223 require.NoError(t, err) 224 225 cid, data, err = extractConfluentSchemaIDAndBinaryData(bin) 226 require.NoError(t, err) 227 228 avroValueCodec, err := encoder.schemaM.Lookup(ctx, topic, schemaID{confluentSchemaID: cid}) 229 require.NoError(t, err) 230 231 res, _, err = avroValueCodec.NativeFromBinary(data) 232 require.NoError(t, err) 233 require.NotNil(t, res) 234 235 for k, v := range res.(map[string]interface{}) { 236 if k == "_tidb_op" { 237 require.Equal(t, "c", v.(string)) 238 } 239 if k == "float" { 240 require.Equal(t, float32(3.14), v) 241 } 242 } 243 } 244 245 func TestAvroEnvelope(t *testing.T) { 246 t.Parallel() 247 cManager := &confluentSchemaManager{} 248 gManager := &glueSchemaManager{} 249 avroCodec, err := goavro.NewCodec(` 250 { 251 "type": "record", 252 "name": "testdb.avroenvelope", 253 "fields" : [ 254 {"name": "id", "type": "int", "default": 0} 255 ] 256 }`) 257 258 require.NoError(t, err) 259 260 testNativeData := make(map[string]interface{}) 261 testNativeData["id"] = 7 262 263 bin, err := avroCodec.BinaryFromNative(nil, testNativeData) 264 require.NoError(t, err) 265 266 // test confluent schema message 267 header, err := cManager.getMsgHeader(8) 268 require.NoError(t, err) 269 res := avroEncodeResult{ 270 data: bin, 271 header: header, 272 } 273 274 evlp, err := res.toEnvelope() 275 require.NoError(t, err) 276 require.Equal(t, header, evlp[0:5]) 277 278 parsed, _, err := avroCodec.NativeFromBinary(evlp[5:]) 279 require.NoError(t, err) 280 require.NotNil(t, parsed) 281 282 id, exists := parsed.(map[string]interface{})["id"] 283 require.True(t, exists) 284 require.Equal(t, int32(7), id) 285 286 // test glue schema message 287 uuidGenerator := uuid.NewGenerator() 288 uuidS := uuidGenerator.NewString() 289 header, err = gManager.getMsgHeader(uuidS) 290 require.NoError(t, err) 291 res = avroEncodeResult{ 292 data: bin, 293 header: header, 294 } 295 evlp, err = res.toEnvelope() 296 require.NoError(t, err) 297 require.Equal(t, header, evlp[0:18]) 298 299 parsed, _, err = avroCodec.NativeFromBinary(evlp[18:]) 300 require.NoError(t, err) 301 require.NotNil(t, parsed) 302 id, exists = parsed.(map[string]interface{})["id"] 303 require.True(t, exists) 304 require.Equal(t, int32(7), id) 305 } 306 307 func TestSanitizeName(t *testing.T) { 308 t.Parallel() 309 310 require.Equal(t, "normalColumnName123", sanitizeName("normalColumnName123")) 311 require.Equal( 312 t, 313 "_1ColumnNameStartWithNumber", 314 sanitizeName("1ColumnNameStartWithNumber"), 315 ) 316 require.Equal(t, "A_B", sanitizeName("A.B")) 317 require.Equal(t, "columnNameWith__", sanitizeName("columnNameWith䏿–‡")) 318 } 319 320 func TestGetAvroNamespace(t *testing.T) { 321 t.Parallel() 322 323 require.Equal( 324 t, 325 "normalNamespace.normalSchema", 326 getAvroNamespace("normalNamespace", "normalSchema"), 327 ) 328 require.Equal( 329 t, 330 "_1Namespace._1Schema", 331 getAvroNamespace("1Namespace", "1Schema"), 332 ) 333 require.Equal( 334 t, 335 "N_amespace.S_chema", 336 getAvroNamespace("N-amespace", "S.chema"), 337 ) 338 } 339 340 func TestArvoAppendRowChangedEventWithCallback(t *testing.T) { 341 codecConfig := common.NewConfig(config.ProtocolAvro) 342 codecConfig.EnableTiDBExtension = true 343 344 ctx, cancel := context.WithCancel(context.Background()) 345 defer cancel() 346 encoder, err := SetupEncoderAndSchemaRegistry4Testing(ctx, codecConfig) 347 defer TeardownEncoderAndSchemaRegistry4Testing() 348 require.NoError(t, err) 349 require.NotNil(t, encoder) 350 351 // Empty build makes sure that the callback build logic not broken. 352 msgs := encoder.Build() 353 require.Len(t, msgs, 0, "no message should be built and no panic") 354 355 cols := []*model.Column{{ 356 Name: "col1", 357 Type: mysql.TypeVarchar, 358 Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, 359 }} 360 tableInfo := model.BuildTableInfo("a", "b", cols, [][]int{{0}}) 361 row := &model.RowChangedEvent{ 362 CommitTs: 1, 363 TableInfo: tableInfo, 364 Columns: model.Columns2ColumnDatas([]*model.Column{{ 365 Name: "col1", 366 Value: []byte("aa"), 367 }}, tableInfo), 368 } 369 370 expected := 0 371 count := 0 372 for i := 0; i < 5; i++ { 373 expected += i 374 bit := i 375 err := encoder.AppendRowChangedEvent(ctx, "", row, func() { 376 count += bit 377 }) 378 require.NoError(t, err) 379 380 msgs = encoder.Build() 381 require.Len(t, msgs, 1, "one message should be built") 382 383 msgs[0].Callback() 384 require.Equal(t, expected, count, "expected one callback be called") 385 } 386 }