github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/json_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 codec 15 16 import ( 17 "math" 18 "strconv" 19 "testing" 20 21 "github.com/pingcap/check" 22 "github.com/pingcap/parser/mysql" 23 "github.com/pingcap/ticdc/cdc/model" 24 "github.com/pingcap/ticdc/pkg/util/testleak" 25 ) 26 27 func Test(t *testing.T) { check.TestingT(t) } 28 29 type batchSuite struct { 30 rowCases [][]*model.RowChangedEvent 31 ddlCases [][]*model.DDLEvent 32 resolvedTsCases [][]uint64 33 } 34 35 var _ = check.Suite(&batchSuite{ 36 rowCases: [][]*model.RowChangedEvent{{{ 37 CommitTs: 1, 38 Table: &model.TableName{Schema: "a", Table: "b"}, 39 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, 40 }}, {{ 41 CommitTs: 1, 42 Table: &model.TableName{Schema: "a", Table: "b"}, 43 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, 44 }, { 45 CommitTs: 2, 46 Table: &model.TableName{Schema: "a", Table: "b"}, 47 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, 48 }, { 49 CommitTs: 3, 50 Table: &model.TableName{Schema: "a", Table: "b"}, 51 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, 52 }, { 53 CommitTs: 4, 54 Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, 55 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, 56 }}, {}}, 57 ddlCases: [][]*model.DDLEvent{{{ 58 CommitTs: 1, 59 TableInfo: &model.SimpleTableInfo{ 60 Schema: "a", Table: "b", 61 }, 62 Query: "create table a", 63 Type: 1, 64 }}, {{ 65 CommitTs: 1, 66 TableInfo: &model.SimpleTableInfo{ 67 Schema: "a", Table: "b", 68 }, 69 Query: "create table a", 70 Type: 1, 71 }, { 72 CommitTs: 2, 73 TableInfo: &model.SimpleTableInfo{ 74 Schema: "a", Table: "b", 75 }, 76 Query: "create table b", 77 Type: 2, 78 }, { 79 CommitTs: 3, 80 TableInfo: &model.SimpleTableInfo{ 81 Schema: "a", Table: "b", 82 }, 83 Query: "create table c", 84 Type: 3, 85 }}, {}}, 86 resolvedTsCases: [][]uint64{{1}, {1, 2, 3}, {}}, 87 }) 88 89 func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEncoder, newDecoder func(key []byte, value []byte) (EventBatchDecoder, error)) { 90 checkRowDecoder := func(decoder EventBatchDecoder, cs []*model.RowChangedEvent) { 91 index := 0 92 for { 93 tp, hasNext, err := decoder.HasNext() 94 c.Assert(err, check.IsNil) 95 if !hasNext { 96 break 97 } 98 c.Assert(tp, check.Equals, model.MqMessageTypeRow) 99 row, err := decoder.NextRowChangedEvent() 100 c.Assert(err, check.IsNil) 101 c.Assert(row, check.DeepEquals, cs[index]) 102 index++ 103 } 104 } 105 checkDDLDecoder := func(decoder EventBatchDecoder, cs []*model.DDLEvent) { 106 index := 0 107 for { 108 tp, hasNext, err := decoder.HasNext() 109 c.Assert(err, check.IsNil) 110 if !hasNext { 111 break 112 } 113 c.Assert(tp, check.Equals, model.MqMessageTypeDDL) 114 ddl, err := decoder.NextDDLEvent() 115 c.Assert(err, check.IsNil) 116 c.Assert(ddl, check.DeepEquals, cs[index]) 117 index++ 118 } 119 } 120 checkTSDecoder := func(decoder EventBatchDecoder, cs []uint64) { 121 index := 0 122 for { 123 tp, hasNext, err := decoder.HasNext() 124 c.Assert(err, check.IsNil) 125 if !hasNext { 126 break 127 } 128 c.Assert(tp, check.Equals, model.MqMessageTypeResolved) 129 ts, err := decoder.NextResolvedEvent() 130 c.Assert(err, check.IsNil) 131 c.Assert(ts, check.DeepEquals, cs[index]) 132 index++ 133 } 134 } 135 136 for _, cs := range s.rowCases { 137 encoder := newEncoder() 138 err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) 139 c.Assert(err, check.IsNil) 140 141 mixedEncoder := newEncoder() 142 mixedEncoder.(*JSONEventBatchEncoder).SetMixedBuildSupport(true) 143 for _, row := range cs { 144 _, err := encoder.AppendRowChangedEvent(row) 145 c.Assert(err, check.IsNil) 146 147 op, err := mixedEncoder.AppendRowChangedEvent(row) 148 c.Assert(op, check.Equals, EncoderNoOperation) 149 c.Assert(err, check.IsNil) 150 } 151 // test mixed decode 152 mixed := mixedEncoder.MixedBuild(true) 153 c.Assert(len(mixed), check.Equals, mixedEncoder.Size()) 154 mixedDecoder, err := newDecoder(mixed, nil) 155 c.Assert(err, check.IsNil) 156 checkRowDecoder(mixedDecoder, cs) 157 // test normal decode 158 if len(cs) > 0 { 159 res := encoder.Build() 160 c.Assert(res, check.HasLen, 1) 161 decoder, err := newDecoder(res[0].Key, res[0].Value) 162 c.Assert(err, check.IsNil) 163 checkRowDecoder(decoder, cs) 164 } 165 } 166 167 for _, cs := range s.ddlCases { 168 encoder := newEncoder() 169 mixedEncoder := newEncoder() 170 err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) 171 c.Assert(err, check.IsNil) 172 173 mixedEncoder.(*JSONEventBatchEncoder).SetMixedBuildSupport(true) 174 for i, ddl := range cs { 175 msg, err := encoder.EncodeDDLEvent(ddl) 176 c.Assert(err, check.IsNil) 177 c.Assert(msg, check.NotNil) 178 decoder, err := newDecoder(msg.Key, msg.Value) 179 c.Assert(err, check.IsNil) 180 checkDDLDecoder(decoder, cs[i:i+1]) 181 182 msg, err = mixedEncoder.EncodeDDLEvent(ddl) 183 c.Assert(msg, check.IsNil) 184 c.Assert(err, check.IsNil) 185 } 186 187 // test mixed encode 188 mixed := mixedEncoder.MixedBuild(true) 189 c.Assert(len(mixed), check.Equals, mixedEncoder.Size()) 190 mixedDecoder, err := newDecoder(mixed, nil) 191 c.Assert(err, check.IsNil) 192 checkDDLDecoder(mixedDecoder, cs) 193 } 194 195 for _, cs := range s.resolvedTsCases { 196 encoder := newEncoder() 197 mixedEncoder := newEncoder() 198 err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) 199 c.Assert(err, check.IsNil) 200 201 mixedEncoder.(*JSONEventBatchEncoder).SetMixedBuildSupport(true) 202 for i, ts := range cs { 203 msg, err := encoder.EncodeCheckpointEvent(ts) 204 c.Assert(err, check.IsNil) 205 c.Assert(msg, check.NotNil) 206 decoder, err := newDecoder(msg.Key, msg.Value) 207 c.Assert(err, check.IsNil) 208 checkTSDecoder(decoder, cs[i:i+1]) 209 210 msg, err = mixedEncoder.EncodeCheckpointEvent(ts) 211 c.Assert(msg, check.IsNil) 212 c.Assert(err, check.IsNil) 213 } 214 215 // test mixed encode 216 mixed := mixedEncoder.MixedBuild(true) 217 c.Assert(len(mixed), check.Equals, mixedEncoder.Size()) 218 mixedDecoder, err := newDecoder(mixed, nil) 219 c.Assert(err, check.IsNil) 220 checkTSDecoder(mixedDecoder, cs) 221 } 222 } 223 224 func (s *batchSuite) TestParamsEdgeCases(c *check.C) { 225 defer testleak.AfterTest(c)() 226 encoder := NewJSONEventBatchEncoder().(*JSONEventBatchEncoder) 227 err := encoder.SetParams(map[string]string{}) 228 c.Assert(err, check.IsNil) 229 c.Assert(encoder.maxBatchSize, check.Equals, 16) 230 c.Assert(encoder.maxKafkaMessageSize, check.Equals, 64*1024*1024) 231 232 err = encoder.SetParams(map[string]string{"max-message-bytes": "0"}) 233 c.Assert(err, check.ErrorMatches, ".*invalid.*") 234 235 err = encoder.SetParams(map[string]string{"max-message-bytes": "-1"}) 236 c.Assert(err, check.ErrorMatches, ".*invalid.*") 237 238 err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxInt32)}) 239 c.Assert(err, check.IsNil) 240 c.Assert(encoder.maxBatchSize, check.Equals, 16) 241 c.Assert(encoder.maxKafkaMessageSize, check.Equals, math.MaxInt32) 242 243 err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxUint32)}) 244 c.Assert(err, check.IsNil) 245 c.Assert(encoder.maxBatchSize, check.Equals, 16) 246 c.Assert(encoder.maxKafkaMessageSize, check.Equals, math.MaxUint32) 247 248 err = encoder.SetParams(map[string]string{"max-batch-size": "0"}) 249 c.Assert(err, check.ErrorMatches, ".*invalid.*") 250 251 err = encoder.SetParams(map[string]string{"max-batch-size": "-1"}) 252 c.Assert(err, check.ErrorMatches, ".*invalid.*") 253 254 err = encoder.SetParams(map[string]string{"max-batch-size": strconv.Itoa(math.MaxInt32)}) 255 c.Assert(err, check.IsNil) 256 c.Assert(encoder.maxBatchSize, check.Equals, math.MaxInt32) 257 c.Assert(encoder.maxKafkaMessageSize, check.Equals, 64*1024*1024) 258 259 err = encoder.SetParams(map[string]string{"max-batch-size": strconv.Itoa(math.MaxUint32)}) 260 c.Assert(err, check.IsNil) 261 c.Assert(encoder.maxBatchSize, check.Equals, math.MaxUint32) 262 c.Assert(encoder.maxKafkaMessageSize, check.Equals, 64*1024*1024) 263 } 264 265 func (s *batchSuite) TestMaxMessageBytes(c *check.C) { 266 defer testleak.AfterTest(c)() 267 encoder := NewJSONEventBatchEncoder() 268 err := encoder.SetParams(map[string]string{"max-message-bytes": "256"}) 269 c.Check(err, check.IsNil) 270 271 testEvent := &model.RowChangedEvent{ 272 CommitTs: 1, 273 Table: &model.TableName{Schema: "a", Table: "b"}, 274 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, 275 } 276 277 for i := 0; i < 10000; i++ { 278 r, err := encoder.AppendRowChangedEvent(testEvent) 279 c.Check(r, check.Equals, EncoderNoOperation) 280 c.Check(err, check.IsNil) 281 } 282 283 messages := encoder.Build() 284 for _, msg := range messages { 285 c.Assert(msg.Length(), check.LessEqual, 256) 286 } 287 } 288 289 func (s *batchSuite) TestMaxBatchSize(c *check.C) { 290 defer testleak.AfterTest(c)() 291 encoder := NewJSONEventBatchEncoder() 292 err := encoder.SetParams(map[string]string{"max-batch-size": "64"}) 293 c.Check(err, check.IsNil) 294 295 testEvent := &model.RowChangedEvent{ 296 CommitTs: 1, 297 Table: &model.TableName{Schema: "a", Table: "b"}, 298 Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, 299 } 300 301 for i := 0; i < 10000; i++ { 302 r, err := encoder.AppendRowChangedEvent(testEvent) 303 c.Check(r, check.Equals, EncoderNoOperation) 304 c.Check(err, check.IsNil) 305 } 306 307 messages := encoder.Build() 308 sum := 0 309 for _, msg := range messages { 310 decoder, err := NewJSONEventBatchDecoder(msg.Key, msg.Value) 311 c.Check(err, check.IsNil) 312 count := 0 313 for { 314 t, hasNext, err := decoder.HasNext() 315 c.Check(err, check.IsNil) 316 if !hasNext { 317 break 318 } 319 320 c.Check(t, check.Equals, model.MqMessageTypeRow) 321 _, err = decoder.NextRowChangedEvent() 322 c.Check(err, check.IsNil) 323 count++ 324 } 325 c.Check(count, check.LessEqual, 64) 326 sum += count 327 } 328 c.Check(sum, check.Equals, 10000) 329 } 330 331 func (s *batchSuite) TestEmptyMessage(c *check.C) { 332 defer testleak.AfterTest(c)() 333 encoder := NewJSONEventBatchEncoder() 334 err := encoder.SetParams(map[string]string{"max-batch-size": "64"}) 335 c.Check(err, check.IsNil) 336 337 emptyEvent := &model.RowChangedEvent{ 338 CommitTs: 1, 339 Table: &model.TableName{Schema: "a", Table: "b"}, 340 Columns: []*model.Column{}, 341 } 342 343 for i := 0; i < 10000; i++ { 344 r, err := encoder.AppendRowChangedEvent(emptyEvent) 345 c.Check(r, check.Equals, EncoderNoOperation) 346 c.Check(err, check.IsNil) 347 } 348 349 messages := encoder.Build() 350 c.Assert(messages, check.HasLen, 0) 351 } 352 353 func (s *batchSuite) TestDefaultEventBatchCodec(c *check.C) { 354 defer testleak.AfterTest(c)() 355 s.testBatchCodec(c, func() EventBatchEncoder { 356 encoder := NewJSONEventBatchEncoder() 357 return encoder 358 }, NewJSONEventBatchDecoder) 359 } 360 361 var _ = check.Suite(&columnSuite{}) 362 363 type columnSuite struct{} 364 365 func (s *columnSuite) TestFormatCol(c *check.C) { 366 defer testleak.AfterTest(c)() 367 row := &mqMessageRow{Update: map[string]column{"test": { 368 Type: mysql.TypeString, 369 Value: "测", 370 }}} 371 rowEncode, err := row.Encode() 372 c.Assert(err, check.IsNil) 373 row2 := new(mqMessageRow) 374 err = row2.Decode(rowEncode) 375 c.Assert(err, check.IsNil) 376 c.Assert(row2, check.DeepEquals, row) 377 378 row = &mqMessageRow{Update: map[string]column{"test": { 379 Type: mysql.TypeBlob, 380 Value: []byte("测"), 381 }}} 382 rowEncode, err = row.Encode() 383 c.Assert(err, check.IsNil) 384 row2 = new(mqMessageRow) 385 err = row2.Decode(rowEncode) 386 c.Assert(err, check.IsNil) 387 c.Assert(row2, check.DeepEquals, row) 388 } 389 390 func (s *columnSuite) TestVarBinaryCol(c *check.C) { 391 defer testleak.AfterTest(c)() 392 col := &model.Column{ 393 Name: "test", 394 Type: mysql.TypeString, 395 Flag: model.BinaryFlag, 396 Value: []byte{0x89, 0x50, 0x4E, 0x47, 0x0D, 0x0A, 0x1A, 0x0A}, 397 } 398 jsonCol := column{} 399 jsonCol.FromSinkColumn(col) 400 row := &mqMessageRow{Update: map[string]column{"test": jsonCol}} 401 rowEncode, err := row.Encode() 402 c.Assert(err, check.IsNil) 403 row2 := new(mqMessageRow) 404 err = row2.Decode(rowEncode) 405 c.Assert(err, check.IsNil) 406 c.Assert(row2, check.DeepEquals, row) 407 jsonCol2 := row2.Update["test"] 408 col2 := jsonCol2.ToSinkColumn("test") 409 c.Assert(col2, check.DeepEquals, col) 410 }