github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/simple/encoder_test.go (about)

     1  // Copyright 2023 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 simple
    15  
    16  import (
    17  	"context"
    18  	"database/sql/driver"
    19  	"fmt"
    20  	"math/rand"
    21  	"sort"
    22  	"strconv"
    23  	"testing"
    24  	"time"
    25  
    26  	"github.com/DATA-DOG/go-sqlmock"
    27  	"github.com/golang/mock/gomock"
    28  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    29  	"github.com/pingcap/tidb/pkg/parser/mysql"
    30  	"github.com/pingcap/tiflow/cdc/entry"
    31  	"github.com/pingcap/tiflow/cdc/model"
    32  	"github.com/pingcap/tiflow/pkg/compression"
    33  	"github.com/pingcap/tiflow/pkg/config"
    34  	"github.com/pingcap/tiflow/pkg/errors"
    35  	"github.com/pingcap/tiflow/pkg/integrity"
    36  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    37  	mock_simple "github.com/pingcap/tiflow/pkg/sink/codec/simple/mock"
    38  	"github.com/pingcap/tiflow/pkg/sink/codec/utils"
    39  	"github.com/stretchr/testify/require"
    40  )
    41  
    42  func TestEncodeCheckpoint(t *testing.T) {
    43  	t.Parallel()
    44  
    45  	ctx := context.Background()
    46  	codecConfig := common.NewConfig(config.ProtocolSimple)
    47  	for _, format := range []common.EncodingFormatType{
    48  		common.EncodingFormatAvro,
    49  		common.EncodingFormatJSON,
    50  	} {
    51  		codecConfig.EncodingFormat = format
    52  
    53  		for _, compressionType := range []string{
    54  			compression.None,
    55  			compression.Snappy,
    56  			compression.LZ4,
    57  		} {
    58  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
    59  			b, err := NewBuilder(ctx, codecConfig)
    60  			require.NoError(t, err)
    61  			enc := b.Build()
    62  
    63  			checkpoint := 446266400629063682
    64  			m, err := enc.EncodeCheckpointEvent(uint64(checkpoint))
    65  			require.NoError(t, err)
    66  
    67  			dec, err := NewDecoder(ctx, codecConfig, nil)
    68  			require.NoError(t, err)
    69  
    70  			err = dec.AddKeyValue(m.Key, m.Value)
    71  			require.NoError(t, err)
    72  
    73  			messageType, hasNext, err := dec.HasNext()
    74  			require.NoError(t, err)
    75  			require.True(t, hasNext)
    76  			require.Equal(t, model.MessageTypeResolved, messageType)
    77  			require.NotEqual(t, 0, dec.msg.BuildTs)
    78  
    79  			ts, err := dec.NextResolvedEvent()
    80  			require.NoError(t, err)
    81  			require.Equal(t, uint64(checkpoint), ts)
    82  		}
    83  	}
    84  }
    85  
    86  func TestEncodeDMLEnableChecksum(t *testing.T) {
    87  	replicaConfig := config.GetDefaultReplicaConfig()
    88  	replicaConfig.Integrity.IntegrityCheckLevel = integrity.CheckLevelCorrectness
    89  	createTableDDL, _, updateEvent, _ := utils.NewLargeEvent4Test(t, replicaConfig)
    90  	rand.New(rand.NewSource(time.Now().Unix())).Shuffle(len(createTableDDL.TableInfo.Columns), func(i, j int) {
    91  		createTableDDL.TableInfo.Columns[i], createTableDDL.TableInfo.Columns[j] = createTableDDL.TableInfo.Columns[j], createTableDDL.TableInfo.Columns[i]
    92  	})
    93  
    94  	ctx := context.Background()
    95  	codecConfig := common.NewConfig(config.ProtocolSimple)
    96  	codecConfig.EnableRowChecksum = true
    97  	for _, format := range []common.EncodingFormatType{
    98  		common.EncodingFormatAvro,
    99  		common.EncodingFormatJSON,
   100  	} {
   101  		codecConfig.EncodingFormat = format
   102  		for _, compressionType := range []string{
   103  			compression.None,
   104  			compression.Snappy,
   105  			compression.LZ4,
   106  		} {
   107  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
   108  
   109  			b, err := NewBuilder(ctx, codecConfig)
   110  			require.NoError(t, err)
   111  			enc := b.Build()
   112  
   113  			dec, err := NewDecoder(ctx, codecConfig, nil)
   114  			require.NoError(t, err)
   115  
   116  			m, err := enc.EncodeDDLEvent(createTableDDL)
   117  			require.NoError(t, err)
   118  
   119  			err = dec.AddKeyValue(m.Key, m.Value)
   120  			require.NoError(t, err)
   121  
   122  			messageType, hasNext, err := dec.HasNext()
   123  			require.NoError(t, err)
   124  			require.True(t, hasNext)
   125  			require.Equal(t, model.MessageTypeDDL, messageType)
   126  
   127  			_, err = dec.NextDDLEvent()
   128  			require.NoError(t, err)
   129  
   130  			err = enc.AppendRowChangedEvent(ctx, "", updateEvent, func() {})
   131  			require.NoError(t, err)
   132  
   133  			messages := enc.Build()
   134  			require.Len(t, messages, 1)
   135  
   136  			err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
   137  			require.NoError(t, err)
   138  
   139  			messageType, hasNext, err = dec.HasNext()
   140  			require.NoError(t, err)
   141  			require.True(t, hasNext)
   142  			require.Equal(t, model.MessageTypeRow, messageType)
   143  
   144  			decodedRow, err := dec.NextRowChangedEvent()
   145  			require.NoError(t, err)
   146  			require.Equal(t, updateEvent.Checksum.Current, decodedRow.Checksum.Current)
   147  			require.Equal(t, updateEvent.Checksum.Previous, decodedRow.Checksum.Previous)
   148  			require.False(t, decodedRow.Checksum.Corrupted)
   149  		}
   150  	}
   151  
   152  	// tamper the checksum, to test error case
   153  	updateEvent.Checksum.Current = 1
   154  	updateEvent.Checksum.Previous = 2
   155  
   156  	b, err := NewBuilder(ctx, codecConfig)
   157  	require.NoError(t, err)
   158  	enc := b.Build()
   159  
   160  	dec, err := NewDecoder(ctx, codecConfig, nil)
   161  	require.NoError(t, err)
   162  	m, err := enc.EncodeDDLEvent(createTableDDL)
   163  	require.NoError(t, err)
   164  
   165  	err = dec.AddKeyValue(m.Key, m.Value)
   166  	require.NoError(t, err)
   167  
   168  	messageType, hasNext, err := dec.HasNext()
   169  	require.NoError(t, err)
   170  	require.True(t, hasNext)
   171  	require.Equal(t, model.MessageTypeDDL, messageType)
   172  
   173  	_, err = dec.NextDDLEvent()
   174  	require.NoError(t, err)
   175  
   176  	err = enc.AppendRowChangedEvent(ctx, "", updateEvent, func() {})
   177  	require.NoError(t, err)
   178  
   179  	messages := enc.Build()
   180  	require.Len(t, messages, 1)
   181  
   182  	err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
   183  	require.NoError(t, err)
   184  
   185  	messageType, hasNext, err = dec.HasNext()
   186  	require.NoError(t, err)
   187  	require.True(t, hasNext)
   188  	require.Equal(t, model.MessageTypeRow, messageType)
   189  
   190  	decodedRow, err := dec.NextRowChangedEvent()
   191  	require.Error(t, err)
   192  	require.Nil(t, decodedRow)
   193  }
   194  
   195  func TestEncodeDDLSequence(t *testing.T) {
   196  	helper := entry.NewSchemaTestHelper(t)
   197  	defer helper.Close()
   198  
   199  	dropDBEvent := helper.DDL2Event(`DROP DATABASE IF EXISTS test`)
   200  	createDBDDLEvent := helper.DDL2Event(`CREATE DATABASE IF NOT EXISTS test`)
   201  	helper.Tk().MustExec("use test")
   202  
   203  	createTableDDLEvent := helper.DDL2Event("CREATE TABLE `TBL1` (`id` INT PRIMARY KEY AUTO_INCREMENT,`value` VARCHAR(255),`payload` VARCHAR(2000),`a` INT)")
   204  
   205  	addColumnDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` ADD COLUMN `nn` INT")
   206  
   207  	dropColumnDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` DROP COLUMN `nn`")
   208  
   209  	changeColumnDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` CHANGE COLUMN `value` `value2` VARCHAR(512)")
   210  
   211  	modifyColumnDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` MODIFY COLUMN `value2` VARCHAR(512) FIRST")
   212  
   213  	setDefaultDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` ALTER COLUMN `payload` SET DEFAULT _UTF8MB4'a'")
   214  
   215  	dropDefaultDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` ALTER COLUMN `payload` DROP DEFAULT")
   216  
   217  	autoIncrementDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` AUTO_INCREMENT = 5")
   218  
   219  	modifyColumnNullDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` MODIFY COLUMN `a` INT NULL")
   220  
   221  	modifyColumnNotNullDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` MODIFY COLUMN `a` INT NOT NULL")
   222  
   223  	addIndexDDLEvent := helper.DDL2Event("CREATE INDEX `idx_a` ON `TBL1` (`a`)")
   224  
   225  	renameIndexDDLEvent := helper.DDL2Event("ALTER TABLE `TBL1` RENAME INDEX `idx_a` TO `new_idx_a`")
   226  
   227  	indexVisibilityDDLEvent := helper.DDL2Event("ALTER TABLE TBL1 ALTER INDEX `new_idx_a` INVISIBLE")
   228  
   229  	dropIndexDDLEvent := helper.DDL2Event("DROP INDEX `new_idx_a` ON `TBL1`")
   230  
   231  	truncateTableDDLEvent := helper.DDL2Event("TRUNCATE TABLE TBL1")
   232  
   233  	multiSchemaChangeDDLEvent := helper.DDL2Event("ALTER TABLE TBL1 ADD COLUMN `new_col` INT, ADD INDEX `idx_new_col` (`a`)")
   234  
   235  	multiSchemaChangeDropDDLEvent := helper.DDL2Event("ALTER TABLE TBL1 DROP COLUMN `new_col`, DROP INDEX `idx_new_col`")
   236  
   237  	renameTableDDLEvent := helper.DDL2Event("RENAME TABLE TBL1 TO TBL2")
   238  
   239  	helper.Tk().MustExec("set @@tidb_allow_remove_auto_inc = 1")
   240  	renameColumnDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 CHANGE COLUMN `id` `id2` INT")
   241  
   242  	partitionTableDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 PARTITION BY RANGE (id2) (PARTITION p0 VALUES LESS THAN (10), PARTITION p1 VALUES LESS THAN (20))")
   243  
   244  	addPartitionDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 ADD PARTITION (PARTITION p2 VALUES LESS THAN (30))")
   245  
   246  	dropPartitionDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 DROP PARTITION p2")
   247  
   248  	truncatePartitionDDLevent := helper.DDL2Event("ALTER TABLE TBL2 TRUNCATE PARTITION p1")
   249  
   250  	reorganizePartitionDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 REORGANIZE PARTITION p1 INTO (PARTITION p3 VALUES LESS THAN (40))")
   251  
   252  	removePartitionDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 REMOVE PARTITIONING")
   253  
   254  	alterCharsetCollateDDLEvent := helper.DDL2Event("ALTER TABLE TBL2 CHARACTER SET = utf8mb4 COLLATE = utf8mb4_bin")
   255  
   256  	dropTableDDLEvent := helper.DDL2Event("DROP TABLE TBL2")
   257  
   258  	ctx := context.Background()
   259  	codecConfig := common.NewConfig(config.ProtocolSimple)
   260  	for _, format := range []common.EncodingFormatType{
   261  		common.EncodingFormatAvro,
   262  		common.EncodingFormatJSON,
   263  	} {
   264  		codecConfig.EncodingFormat = format
   265  		for _, compressionType := range []string{
   266  			compression.None,
   267  			compression.Snappy,
   268  			compression.LZ4,
   269  		} {
   270  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
   271  
   272  			b, err := NewBuilder(ctx, codecConfig)
   273  			require.NoError(t, err)
   274  
   275  			enc := b.Build()
   276  			dec, err := NewDecoder(ctx, codecConfig, nil)
   277  			require.NoError(t, err)
   278  
   279  			m, err := enc.EncodeDDLEvent(dropDBEvent)
   280  			require.NoError(t, err)
   281  
   282  			err = dec.AddKeyValue(m.Key, m.Value)
   283  			require.NoError(t, err)
   284  
   285  			messageType, hasNext, err := dec.HasNext()
   286  			require.NoError(t, err)
   287  			require.True(t, hasNext)
   288  			require.Equal(t, model.MessageTypeDDL, messageType)
   289  			require.Equal(t, DDLTypeQuery, dec.msg.Type)
   290  
   291  			_, err = dec.NextDDLEvent()
   292  			require.NoError(t, err)
   293  
   294  			m, err = enc.EncodeDDLEvent(createDBDDLEvent)
   295  			require.NoError(t, err)
   296  
   297  			err = dec.AddKeyValue(m.Key, m.Value)
   298  			require.NoError(t, err)
   299  
   300  			messageType, hasNext, err = dec.HasNext()
   301  			require.NoError(t, err)
   302  			require.True(t, hasNext)
   303  			require.Equal(t, model.MessageTypeDDL, messageType)
   304  			require.Equal(t, DDLTypeQuery, dec.msg.Type)
   305  
   306  			_, err = dec.NextDDLEvent()
   307  			require.NoError(t, err)
   308  
   309  			m, err = enc.EncodeDDLEvent(createTableDDLEvent)
   310  			require.NoError(t, err)
   311  
   312  			err = dec.AddKeyValue(m.Key, m.Value)
   313  			require.NoError(t, err)
   314  
   315  			messageType, hasNext, err = dec.HasNext()
   316  			require.NoError(t, err)
   317  			require.True(t, hasNext)
   318  			require.Equal(t, model.MessageTypeDDL, messageType)
   319  			require.Equal(t, DDLTypeCreate, dec.msg.Type)
   320  
   321  			event, err := dec.NextDDLEvent()
   322  			require.NoError(t, err)
   323  			require.Len(t, event.TableInfo.Indices, 1)
   324  			require.Len(t, event.TableInfo.Columns, 4)
   325  
   326  			m, err = enc.EncodeDDLEvent(addColumnDDLEvent)
   327  			require.NoError(t, err)
   328  
   329  			err = dec.AddKeyValue(m.Key, m.Value)
   330  			require.NoError(t, err)
   331  
   332  			_, _, err = dec.HasNext()
   333  			require.NoError(t, err)
   334  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   335  
   336  			event, err = dec.NextDDLEvent()
   337  			require.NoError(t, err)
   338  			require.Len(t, event.TableInfo.Indices, 1)
   339  			require.Len(t, event.TableInfo.Columns, 5)
   340  
   341  			m, err = enc.EncodeDDLEvent(dropColumnDDLEvent)
   342  			require.NoError(t, err)
   343  
   344  			err = dec.AddKeyValue(m.Key, m.Value)
   345  			require.NoError(t, err)
   346  
   347  			_, _, err = dec.HasNext()
   348  			require.NoError(t, err)
   349  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   350  
   351  			event, err = dec.NextDDLEvent()
   352  			require.NoError(t, err)
   353  			require.Len(t, event.TableInfo.Indices, 1)
   354  			require.Len(t, event.TableInfo.Columns, 4)
   355  
   356  			m, err = enc.EncodeDDLEvent(changeColumnDDLEvent)
   357  			require.NoError(t, err)
   358  
   359  			err = dec.AddKeyValue(m.Key, m.Value)
   360  			require.NoError(t, err)
   361  
   362  			_, _, err = dec.HasNext()
   363  			require.NoError(t, err)
   364  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   365  
   366  			event, err = dec.NextDDLEvent()
   367  			require.NoError(t, err)
   368  			require.Len(t, event.TableInfo.Indices, 1)
   369  			require.Len(t, event.TableInfo.Columns, 4)
   370  
   371  			m, err = enc.EncodeDDLEvent(modifyColumnDDLEvent)
   372  			require.NoError(t, err)
   373  
   374  			err = dec.AddKeyValue(m.Key, m.Value)
   375  			require.NoError(t, err)
   376  
   377  			_, _, err = dec.HasNext()
   378  			require.NoError(t, err)
   379  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   380  
   381  			event, err = dec.NextDDLEvent()
   382  			require.NoError(t, err)
   383  			require.Equal(t, 1, len(event.TableInfo.Indices), string(format), compressionType)
   384  			require.Equal(t, 4, len(event.TableInfo.Columns))
   385  
   386  			m, err = enc.EncodeDDLEvent(setDefaultDDLEvent)
   387  			require.NoError(t, err)
   388  
   389  			err = dec.AddKeyValue(m.Key, m.Value)
   390  			require.NoError(t, err)
   391  
   392  			_, _, err = dec.HasNext()
   393  			require.NoError(t, err)
   394  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   395  
   396  			event, err = dec.NextDDLEvent()
   397  			require.NoError(t, err)
   398  			require.Equal(t, 1, len(event.TableInfo.Indices))
   399  			require.Equal(t, 4, len(event.TableInfo.Columns))
   400  			for _, col := range event.TableInfo.Columns {
   401  				if col.Name.O == "payload" {
   402  					require.Equal(t, "a", col.DefaultValue)
   403  				}
   404  			}
   405  
   406  			m, err = enc.EncodeDDLEvent(dropDefaultDDLEvent)
   407  			require.NoError(t, err)
   408  
   409  			err = dec.AddKeyValue(m.Key, m.Value)
   410  			require.NoError(t, err)
   411  
   412  			_, _, err = dec.HasNext()
   413  			require.NoError(t, err)
   414  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   415  
   416  			event, err = dec.NextDDLEvent()
   417  			require.NoError(t, err)
   418  			require.Equal(t, 1, len(event.TableInfo.Indices))
   419  			require.Equal(t, 4, len(event.TableInfo.Columns))
   420  			for _, col := range event.TableInfo.Columns {
   421  				if col.Name.O == "payload" {
   422  					require.Nil(t, col.DefaultValue)
   423  				}
   424  			}
   425  
   426  			m, err = enc.EncodeDDLEvent(autoIncrementDDLEvent)
   427  			require.NoError(t, err)
   428  
   429  			err = dec.AddKeyValue(m.Key, m.Value)
   430  			require.NoError(t, err)
   431  
   432  			_, _, err = dec.HasNext()
   433  			require.NoError(t, err)
   434  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   435  
   436  			event, err = dec.NextDDLEvent()
   437  			require.NoError(t, err)
   438  			require.Equal(t, 1, len(event.TableInfo.Indices))
   439  			require.Equal(t, 4, len(event.TableInfo.Columns))
   440  
   441  			m, err = enc.EncodeDDLEvent(modifyColumnNullDDLEvent)
   442  			require.NoError(t, err)
   443  
   444  			err = dec.AddKeyValue(m.Key, m.Value)
   445  			require.NoError(t, err)
   446  
   447  			_, _, err = dec.HasNext()
   448  			require.NoError(t, err)
   449  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   450  
   451  			event, err = dec.NextDDLEvent()
   452  			require.NoError(t, err)
   453  			require.Equal(t, 1, len(event.TableInfo.Indices))
   454  			require.Equal(t, 4, len(event.TableInfo.Columns))
   455  			for _, col := range event.TableInfo.Columns {
   456  				if col.Name.O == "a" {
   457  					require.True(t, !mysql.HasNotNullFlag(col.GetFlag()))
   458  				}
   459  			}
   460  
   461  			m, err = enc.EncodeDDLEvent(modifyColumnNotNullDDLEvent)
   462  			require.NoError(t, err)
   463  
   464  			err = dec.AddKeyValue(m.Key, m.Value)
   465  			require.NoError(t, err)
   466  
   467  			_, _, err = dec.HasNext()
   468  			require.NoError(t, err)
   469  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   470  
   471  			event, err = dec.NextDDLEvent()
   472  			require.NoError(t, err)
   473  			require.Equal(t, 1, len(event.TableInfo.Indices))
   474  			require.Equal(t, 4, len(event.TableInfo.Columns))
   475  			for _, col := range event.TableInfo.Columns {
   476  				if col.Name.O == "a" {
   477  					require.True(t, mysql.HasNotNullFlag(col.GetFlag()))
   478  				}
   479  			}
   480  
   481  			m, err = enc.EncodeDDLEvent(addIndexDDLEvent)
   482  			require.NoError(t, err)
   483  
   484  			err = dec.AddKeyValue(m.Key, m.Value)
   485  			require.NoError(t, err)
   486  
   487  			_, _, err = dec.HasNext()
   488  			require.NoError(t, err)
   489  			require.Equal(t, DDLTypeCIndex, dec.msg.Type)
   490  
   491  			event, err = dec.NextDDLEvent()
   492  			require.NoError(t, err)
   493  			require.Equal(t, 2, len(event.TableInfo.Indices))
   494  			require.Equal(t, 4, len(event.TableInfo.Columns))
   495  
   496  			m, err = enc.EncodeDDLEvent(renameIndexDDLEvent)
   497  			require.NoError(t, err)
   498  
   499  			err = dec.AddKeyValue(m.Key, m.Value)
   500  			require.NoError(t, err)
   501  
   502  			_, _, err = dec.HasNext()
   503  			require.NoError(t, err)
   504  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   505  
   506  			event, err = dec.NextDDLEvent()
   507  			require.NoError(t, err)
   508  			require.Equal(t, 2, len(event.TableInfo.Indices))
   509  			require.Equal(t, 4, len(event.TableInfo.Columns))
   510  			hasNewIndex := false
   511  			noOldIndex := true
   512  			for _, index := range event.TableInfo.Indices {
   513  				if index.Name.O == "new_idx_a" {
   514  					hasNewIndex = true
   515  				}
   516  				if index.Name.O == "idx_a" {
   517  					noOldIndex = false
   518  				}
   519  			}
   520  			require.True(t, hasNewIndex)
   521  			require.True(t, noOldIndex)
   522  
   523  			m, err = enc.EncodeDDLEvent(indexVisibilityDDLEvent)
   524  			require.NoError(t, err)
   525  
   526  			err = dec.AddKeyValue(m.Key, m.Value)
   527  			require.NoError(t, err)
   528  
   529  			_, _, err = dec.HasNext()
   530  			require.NoError(t, err)
   531  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   532  
   533  			event, err = dec.NextDDLEvent()
   534  			require.NoError(t, err)
   535  			require.Equal(t, 2, len(event.TableInfo.Indices))
   536  			require.Equal(t, 4, len(event.TableInfo.Columns))
   537  
   538  			m, err = enc.EncodeDDLEvent(dropIndexDDLEvent)
   539  			require.NoError(t, err)
   540  
   541  			err = dec.AddKeyValue(m.Key, m.Value)
   542  			require.NoError(t, err)
   543  
   544  			_, _, err = dec.HasNext()
   545  			require.NoError(t, err)
   546  			require.Equal(t, DDLTypeDIndex, dec.msg.Type)
   547  
   548  			event, err = dec.NextDDLEvent()
   549  			require.NoError(t, err)
   550  			require.Equal(t, 1, len(event.TableInfo.Indices))
   551  			require.Equal(t, 4, len(event.TableInfo.Columns))
   552  
   553  			m, err = enc.EncodeDDLEvent(truncateTableDDLEvent)
   554  			require.NoError(t, err)
   555  
   556  			err = dec.AddKeyValue(m.Key, m.Value)
   557  			require.NoError(t, err)
   558  
   559  			_, _, err = dec.HasNext()
   560  			require.NoError(t, err)
   561  			require.Equal(t, DDLTypeTruncate, dec.msg.Type)
   562  
   563  			event, err = dec.NextDDLEvent()
   564  			require.NoError(t, err)
   565  			require.Equal(t, 1, len(event.TableInfo.Indices))
   566  			require.Equal(t, 4, len(event.TableInfo.Columns))
   567  
   568  			m, err = enc.EncodeDDLEvent(multiSchemaChangeDDLEvent)
   569  			require.NoError(t, err)
   570  
   571  			err = dec.AddKeyValue(m.Key, m.Value)
   572  			require.NoError(t, err)
   573  
   574  			_, _, err = dec.HasNext()
   575  			require.NoError(t, err)
   576  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   577  
   578  			event, err = dec.NextDDLEvent()
   579  			require.NoError(t, err)
   580  			require.Equal(t, 2, len(event.TableInfo.Indices))
   581  			require.Equal(t, 5, len(event.TableInfo.Columns))
   582  
   583  			m, err = enc.EncodeDDLEvent(multiSchemaChangeDropDDLEvent)
   584  			require.NoError(t, err)
   585  
   586  			err = dec.AddKeyValue(m.Key, m.Value)
   587  			require.NoError(t, err)
   588  
   589  			_, _, err = dec.HasNext()
   590  			require.NoError(t, err)
   591  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   592  
   593  			event, err = dec.NextDDLEvent()
   594  			require.NoError(t, err)
   595  			require.Equal(t, 1, len(event.TableInfo.Indices))
   596  			require.Equal(t, 4, len(event.TableInfo.Columns))
   597  
   598  			m, err = enc.EncodeDDLEvent(renameTableDDLEvent)
   599  			require.NoError(t, err)
   600  
   601  			err = dec.AddKeyValue(m.Key, m.Value)
   602  			require.NoError(t, err)
   603  
   604  			_, _, err = dec.HasNext()
   605  			require.NoError(t, err)
   606  			require.Equal(t, DDLTypeRename, dec.msg.Type)
   607  
   608  			event, err = dec.NextDDLEvent()
   609  			require.NoError(t, err)
   610  			require.Equal(t, 1, len(event.TableInfo.Indices))
   611  			require.Equal(t, 4, len(event.TableInfo.Columns))
   612  
   613  			m, err = enc.EncodeDDLEvent(renameColumnDDLEvent)
   614  			require.NoError(t, err)
   615  
   616  			err = dec.AddKeyValue(m.Key, m.Value)
   617  			require.NoError(t, err)
   618  
   619  			_, _, err = dec.HasNext()
   620  			require.NoError(t, err)
   621  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   622  
   623  			event, err = dec.NextDDLEvent()
   624  			require.NoError(t, err)
   625  			require.Equal(t, 1, len(event.TableInfo.Indices))
   626  			require.Equal(t, 4, len(event.TableInfo.Columns))
   627  
   628  			m, err = enc.EncodeDDLEvent(partitionTableDDLEvent)
   629  			require.NoError(t, err)
   630  
   631  			err = dec.AddKeyValue(m.Key, m.Value)
   632  			require.NoError(t, err)
   633  
   634  			_, _, err = dec.HasNext()
   635  			require.NoError(t, err)
   636  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   637  
   638  			event, err = dec.NextDDLEvent()
   639  			require.NoError(t, err)
   640  			require.Equal(t, 1, len(event.TableInfo.Indices))
   641  			require.Equal(t, 4, len(event.TableInfo.Columns))
   642  
   643  			m, err = enc.EncodeDDLEvent(addPartitionDDLEvent)
   644  			require.NoError(t, err)
   645  
   646  			err = dec.AddKeyValue(m.Key, m.Value)
   647  			require.NoError(t, err)
   648  
   649  			_, _, err = dec.HasNext()
   650  			require.NoError(t, err)
   651  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   652  
   653  			event, err = dec.NextDDLEvent()
   654  			require.NoError(t, err)
   655  			require.Equal(t, 1, len(event.TableInfo.Indices))
   656  			require.Equal(t, 4, len(event.TableInfo.Columns))
   657  
   658  			m, err = enc.EncodeDDLEvent(dropPartitionDDLEvent)
   659  			require.NoError(t, err)
   660  
   661  			err = dec.AddKeyValue(m.Key, m.Value)
   662  			require.NoError(t, err)
   663  
   664  			_, _, err = dec.HasNext()
   665  			require.NoError(t, err)
   666  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   667  
   668  			event, err = dec.NextDDLEvent()
   669  			require.NoError(t, err)
   670  			require.Equal(t, 1, len(event.TableInfo.Indices))
   671  			require.Equal(t, 4, len(event.TableInfo.Columns))
   672  
   673  			m, err = enc.EncodeDDLEvent(truncatePartitionDDLevent)
   674  			require.NoError(t, err)
   675  
   676  			err = dec.AddKeyValue(m.Key, m.Value)
   677  			require.NoError(t, err)
   678  
   679  			_, _, err = dec.HasNext()
   680  			require.NoError(t, err)
   681  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   682  
   683  			event, err = dec.NextDDLEvent()
   684  			require.NoError(t, err)
   685  			require.Equal(t, 1, len(event.TableInfo.Indices))
   686  			require.Equal(t, 4, len(event.TableInfo.Columns))
   687  
   688  			m, err = enc.EncodeDDLEvent(reorganizePartitionDDLEvent)
   689  			require.NoError(t, err)
   690  
   691  			err = dec.AddKeyValue(m.Key, m.Value)
   692  			require.NoError(t, err)
   693  
   694  			_, _, err = dec.HasNext()
   695  			require.NoError(t, err)
   696  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   697  
   698  			event, err = dec.NextDDLEvent()
   699  			require.NoError(t, err)
   700  			require.Equal(t, 1, len(event.TableInfo.Indices))
   701  			require.Equal(t, 4, len(event.TableInfo.Columns))
   702  
   703  			m, err = enc.EncodeDDLEvent(removePartitionDDLEvent)
   704  			require.NoError(t, err)
   705  
   706  			err = dec.AddKeyValue(m.Key, m.Value)
   707  			require.NoError(t, err)
   708  
   709  			_, _, err = dec.HasNext()
   710  			require.NoError(t, err)
   711  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   712  
   713  			event, err = dec.NextDDLEvent()
   714  			require.NoError(t, err)
   715  			require.Equal(t, 1, len(event.TableInfo.Indices))
   716  			require.Equal(t, 4, len(event.TableInfo.Columns))
   717  
   718  			m, err = enc.EncodeDDLEvent(alterCharsetCollateDDLEvent)
   719  			require.NoError(t, err)
   720  
   721  			err = dec.AddKeyValue(m.Key, m.Value)
   722  			require.NoError(t, err)
   723  
   724  			_, _, err = dec.HasNext()
   725  			require.NoError(t, err)
   726  			require.Equal(t, DDLTypeAlter, dec.msg.Type)
   727  
   728  			event, err = dec.NextDDLEvent()
   729  			require.NoError(t, err)
   730  			require.Equal(t, 1, len(event.TableInfo.Indices))
   731  			require.Equal(t, 4, len(event.TableInfo.Columns))
   732  
   733  			m, err = enc.EncodeDDLEvent(dropTableDDLEvent)
   734  			require.NoError(t, err)
   735  
   736  			err = dec.AddKeyValue(m.Key, m.Value)
   737  			require.NoError(t, err)
   738  
   739  			_, _, err = dec.HasNext()
   740  			require.NoError(t, err)
   741  			require.Equal(t, DDLTypeErase, dec.msg.Type)
   742  
   743  			event, err = dec.NextDDLEvent()
   744  			require.NoError(t, err)
   745  			require.Equal(t, 1, len(event.TableInfo.Indices))
   746  			require.Equal(t, 4, len(event.TableInfo.Columns))
   747  		}
   748  	}
   749  }
   750  
   751  func TestEncodeDDLEvent(t *testing.T) {
   752  	replicaConfig := config.GetDefaultReplicaConfig()
   753  	replicaConfig.Integrity.IntegrityCheckLevel = integrity.CheckLevelCorrectness
   754  	helper := entry.NewSchemaTestHelperWithReplicaConfig(t, replicaConfig)
   755  	defer helper.Close()
   756  
   757  	createTableSQL := `create table test.t(id int primary key, name varchar(255) not null, gender enum('male', 'female'), email varchar(255) null, key idx_name_email(name, email))`
   758  	createTableDDLEvent := helper.DDL2Event(createTableSQL)
   759  
   760  	insertEvent := helper.DML2Event(`insert into test.t values (1, "jack", "male", "jack@abc.com")`, "test", "t")
   761  
   762  	renameTableDDLEvent := helper.DDL2Event(`rename table test.t to test.abc`)
   763  
   764  	insertEvent2 := helper.DML2Event(`insert into test.abc values (2, "anna", "female", "anna@abc.com")`, "test", "abc")
   765  	helper.Tk().MustExec("drop table test.abc")
   766  
   767  	ctx := context.Background()
   768  	codecConfig := common.NewConfig(config.ProtocolSimple)
   769  	codecConfig.EnableRowChecksum = true
   770  	for _, format := range []common.EncodingFormatType{
   771  		common.EncodingFormatAvro,
   772  		common.EncodingFormatJSON,
   773  	} {
   774  		codecConfig.EncodingFormat = format
   775  		for _, compressionType := range []string{
   776  			compression.None,
   777  			compression.Snappy,
   778  			compression.LZ4,
   779  		} {
   780  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
   781  			b, err := NewBuilder(ctx, codecConfig)
   782  			require.NoError(t, err)
   783  			enc := b.Build()
   784  
   785  			dec, err := NewDecoder(ctx, codecConfig, nil)
   786  			require.NoError(t, err)
   787  
   788  			m, err := enc.EncodeDDLEvent(createTableDDLEvent)
   789  			require.NoError(t, err)
   790  
   791  			err = dec.AddKeyValue(m.Key, m.Value)
   792  			require.NoError(t, err)
   793  
   794  			messageType, hasNext, err := dec.HasNext()
   795  			require.NoError(t, err)
   796  			require.True(t, hasNext)
   797  			require.Equal(t, model.MessageTypeDDL, messageType)
   798  			require.NotEqual(t, 0, dec.msg.BuildTs)
   799  			require.True(t, dec.msg.TableSchema.Indexes[0].Nullable)
   800  
   801  			columnSchemas := dec.msg.TableSchema.Columns
   802  			sortedColumns := make([]*timodel.ColumnInfo, len(createTableDDLEvent.TableInfo.Columns))
   803  			copy(sortedColumns, createTableDDLEvent.TableInfo.Columns)
   804  			sort.Slice(sortedColumns, func(i, j int) bool {
   805  				return sortedColumns[i].ID < sortedColumns[j].ID
   806  			})
   807  
   808  			for idx, column := range sortedColumns {
   809  				require.Equal(t, column.Name.O, columnSchemas[idx].Name)
   810  			}
   811  
   812  			event, err := dec.NextDDLEvent()
   813  
   814  			require.NoError(t, err)
   815  			require.Equal(t, createTableDDLEvent.TableInfo.TableName.TableID, event.TableInfo.TableName.TableID)
   816  			require.Equal(t, createTableDDLEvent.CommitTs, event.CommitTs)
   817  
   818  			// because we don't we don't set startTs in the encoded message,
   819  			// so the startTs is equal to commitTs
   820  
   821  			require.Equal(t, createTableDDLEvent.CommitTs, event.StartTs)
   822  			require.Equal(t, createTableDDLEvent.Query, event.Query)
   823  			require.Equal(t, len(createTableDDLEvent.TableInfo.Columns), len(event.TableInfo.Columns))
   824  			require.Equal(t, 2, len(event.TableInfo.Indices))
   825  			require.Nil(t, event.PreTableInfo)
   826  
   827  			item := dec.memo.Read(createTableDDLEvent.TableInfo.TableName.Schema,
   828  				createTableDDLEvent.TableInfo.TableName.Table, createTableDDLEvent.TableInfo.UpdateTS)
   829  			require.NotNil(t, item)
   830  
   831  			err = enc.AppendRowChangedEvent(ctx, "", insertEvent, func() {})
   832  			require.NoError(t, err)
   833  
   834  			messages := enc.Build()
   835  			require.Len(t, messages, 1)
   836  
   837  			err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
   838  			require.NoError(t, err)
   839  
   840  			messageType, hasNext, err = dec.HasNext()
   841  			require.NoError(t, err)
   842  			require.True(t, hasNext)
   843  			require.Equal(t, model.MessageTypeRow, messageType)
   844  			require.NotEqual(t, 0, dec.msg.BuildTs)
   845  
   846  			decodedRow, err := dec.NextRowChangedEvent()
   847  			require.NoError(t, err)
   848  			require.Equal(t, decodedRow.CommitTs, insertEvent.CommitTs)
   849  			require.Equal(t, decodedRow.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName())
   850  			require.Equal(t, decodedRow.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName())
   851  			require.Nil(t, decodedRow.PreColumns)
   852  
   853  			m, err = enc.EncodeDDLEvent(renameTableDDLEvent)
   854  			require.NoError(t, err)
   855  
   856  			err = dec.AddKeyValue(m.Key, m.Value)
   857  			require.NoError(t, err)
   858  
   859  			messageType, hasNext, err = dec.HasNext()
   860  			require.NoError(t, err)
   861  			require.True(t, hasNext)
   862  			require.Equal(t, model.MessageTypeDDL, messageType)
   863  			require.NotEqual(t, 0, dec.msg.BuildTs)
   864  
   865  			event, err = dec.NextDDLEvent()
   866  			require.NoError(t, err)
   867  			require.Equal(t, renameTableDDLEvent.TableInfo.TableName.TableID, event.TableInfo.TableName.TableID)
   868  			require.Equal(t, renameTableDDLEvent.CommitTs, event.CommitTs)
   869  			// because we don't we don't set startTs in the encoded message,
   870  			// so the startTs is equal to commitTs
   871  			require.Equal(t, renameTableDDLEvent.CommitTs, event.StartTs)
   872  			require.Equal(t, renameTableDDLEvent.Query, event.Query)
   873  			require.Equal(t, len(renameTableDDLEvent.TableInfo.Columns), len(event.TableInfo.Columns))
   874  			require.Equal(t, len(renameTableDDLEvent.TableInfo.Indices)+1, len(event.TableInfo.Indices))
   875  			require.NotNil(t, event.PreTableInfo)
   876  
   877  			item = dec.memo.Read(renameTableDDLEvent.TableInfo.TableName.Schema,
   878  				renameTableDDLEvent.TableInfo.TableName.Table, renameTableDDLEvent.TableInfo.UpdateTS)
   879  			require.NotNil(t, item)
   880  
   881  			err = enc.AppendRowChangedEvent(context.Background(), "", insertEvent2, func() {})
   882  			require.NoError(t, err)
   883  
   884  			messages = enc.Build()
   885  			require.Len(t, messages, 1)
   886  
   887  			err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
   888  			require.NoError(t, err)
   889  
   890  			messageType, hasNext, err = dec.HasNext()
   891  			require.NoError(t, err)
   892  			require.True(t, hasNext)
   893  			require.Equal(t, model.MessageTypeRow, messageType)
   894  			require.NotEqual(t, 0, dec.msg.BuildTs)
   895  
   896  			decodedRow, err = dec.NextRowChangedEvent()
   897  			require.NoError(t, err)
   898  			require.Equal(t, insertEvent2.CommitTs, decodedRow.CommitTs)
   899  			require.Equal(t, insertEvent2.TableInfo.GetSchemaName(), decodedRow.TableInfo.GetSchemaName())
   900  			require.Equal(t, insertEvent2.TableInfo.GetTableName(), decodedRow.TableInfo.GetTableName())
   901  			require.Nil(t, decodedRow.PreColumns)
   902  		}
   903  	}
   904  }
   905  
   906  func TestEncodeIntegerTypes(t *testing.T) {
   907  	replicaConfig := config.GetDefaultReplicaConfig()
   908  	replicaConfig.Integrity.IntegrityCheckLevel = integrity.CheckLevelCorrectness
   909  	helper := entry.NewSchemaTestHelperWithReplicaConfig(t, replicaConfig)
   910  	defer helper.Close()
   911  
   912  	createTableDDL := `create table test.t(
   913  		id int primary key auto_increment,
   914   		a tinyint, b tinyint unsigned,
   915   		c smallint, d smallint unsigned,
   916   		e mediumint, f mediumint unsigned,
   917   		g int, h int unsigned,
   918   		i bigint, j bigint unsigned)`
   919  	ddlEvent := helper.DDL2Event(createTableDDL)
   920  
   921  	sql := `insert into test.t values(
   922  		1,
   923  		-128, 0,
   924  		-32768, 0,
   925  		-8388608, 0,
   926  		-2147483648, 0,
   927  		-9223372036854775808, 0)`
   928  	minValues := helper.DML2Event(sql, "test", "t")
   929  
   930  	sql = `insert into test.t values (
   931  		2,
   932   		127, 255,
   933   		32767, 65535,
   934   		8388607, 16777215,
   935   		2147483647, 4294967295,
   936   		9223372036854775807, 18446744073709551615)`
   937  	maxValues := helper.DML2Event(sql, "test", "t")
   938  
   939  	ctx := context.Background()
   940  	codecConfig := common.NewConfig(config.ProtocolSimple)
   941  	codecConfig.EnableRowChecksum = true
   942  	for _, format := range []common.EncodingFormatType{
   943  		common.EncodingFormatAvro,
   944  		common.EncodingFormatJSON,
   945  	} {
   946  		codecConfig.EncodingFormat = format
   947  		b, err := NewBuilder(ctx, codecConfig)
   948  		require.NoError(t, err)
   949  		enc := b.Build()
   950  
   951  		m, err := enc.EncodeDDLEvent(ddlEvent)
   952  		require.NoError(t, err)
   953  
   954  		dec, err := NewDecoder(ctx, codecConfig, nil)
   955  		require.NoError(t, err)
   956  
   957  		err = dec.AddKeyValue(m.Key, m.Value)
   958  		require.NoError(t, err)
   959  
   960  		messageType, hasNext, err := dec.HasNext()
   961  		require.NoError(t, err)
   962  		require.True(t, hasNext)
   963  		require.Equal(t, model.MessageTypeDDL, messageType)
   964  
   965  		_, err = dec.NextDDLEvent()
   966  		require.NoError(t, err)
   967  
   968  		for _, event := range []*model.RowChangedEvent{
   969  			minValues,
   970  			maxValues,
   971  		} {
   972  			err = enc.AppendRowChangedEvent(ctx, "", event, func() {})
   973  			require.NoError(t, err)
   974  
   975  			messages := enc.Build()
   976  			err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
   977  			require.NoError(t, err)
   978  
   979  			messageType, hasNext, err = dec.HasNext()
   980  			require.NoError(t, err)
   981  			require.True(t, hasNext)
   982  			require.Equal(t, model.MessageTypeRow, messageType)
   983  
   984  			decodedRow, err := dec.NextRowChangedEvent()
   985  			require.NoError(t, err)
   986  			require.Equal(t, decodedRow.CommitTs, event.CommitTs)
   987  
   988  			decodedColumns := make(map[string]*model.ColumnData, len(decodedRow.Columns))
   989  			for _, column := range decodedRow.Columns {
   990  				colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
   991  				decodedColumns[colName] = column
   992  			}
   993  
   994  			for _, expected := range event.Columns {
   995  				colName := event.TableInfo.ForceGetColumnName(expected.ColumnID)
   996  				decoded, ok := decodedColumns[colName]
   997  				require.True(t, ok)
   998  				require.EqualValues(t, expected.Value, decoded.Value)
   999  			}
  1000  		}
  1001  	}
  1002  }
  1003  
  1004  func TestEncoderOtherTypes(t *testing.T) {
  1005  	helper := entry.NewSchemaTestHelper(t)
  1006  	defer helper.Close()
  1007  
  1008  	sql := `create table test.t(
  1009  			a int primary key auto_increment,
  1010  			b enum('a', 'b', 'c'),
  1011  			c set('a', 'b', 'c'),
  1012  			d bit(64),
  1013  			e json)`
  1014  	ddlEvent := helper.DDL2Event(sql)
  1015  
  1016  	sql = `insert into test.t() values (1, 'a', 'a,b', b'1000001', '{
  1017  		  "key1": "value1",
  1018  		  "key2": "value2"
  1019  		}');`
  1020  	row := helper.DML2Event(sql, "test", "t")
  1021  
  1022  	ctx := context.Background()
  1023  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1024  	for _, format := range []common.EncodingFormatType{
  1025  		common.EncodingFormatAvro,
  1026  		common.EncodingFormatJSON,
  1027  	} {
  1028  		codecConfig.EncodingFormat = format
  1029  		b, err := NewBuilder(ctx, codecConfig)
  1030  		require.NoError(t, err)
  1031  		enc := b.Build()
  1032  
  1033  		m, err := enc.EncodeDDLEvent(ddlEvent)
  1034  		require.NoError(t, err)
  1035  
  1036  		dec, err := NewDecoder(ctx, codecConfig, nil)
  1037  		require.NoError(t, err)
  1038  
  1039  		err = dec.AddKeyValue(m.Key, m.Value)
  1040  		require.NoError(t, err)
  1041  
  1042  		messageType, hasNext, err := dec.HasNext()
  1043  		require.NoError(t, err)
  1044  		require.True(t, hasNext)
  1045  		require.Equal(t, model.MessageTypeDDL, messageType)
  1046  
  1047  		_, err = dec.NextDDLEvent()
  1048  		require.NoError(t, err)
  1049  
  1050  		err = enc.AppendRowChangedEvent(ctx, "", row, func() {})
  1051  		require.NoError(t, err)
  1052  
  1053  		messages := enc.Build()
  1054  		require.Len(t, messages, 1)
  1055  
  1056  		err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
  1057  		require.NoError(t, err)
  1058  
  1059  		messageType, hasNext, err = dec.HasNext()
  1060  		require.NoError(t, err)
  1061  		require.True(t, hasNext)
  1062  		require.Equal(t, model.MessageTypeRow, messageType)
  1063  
  1064  		decodedRow, err := dec.NextRowChangedEvent()
  1065  		require.NoError(t, err)
  1066  
  1067  		decodedColumns := make(map[string]*model.ColumnData, len(decodedRow.Columns))
  1068  		for _, column := range decodedRow.Columns {
  1069  			colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1070  			decodedColumns[colName] = column
  1071  		}
  1072  		for _, expected := range row.Columns {
  1073  			colName := row.TableInfo.ForceGetColumnName(expected.ColumnID)
  1074  			decoded, ok := decodedColumns[colName]
  1075  			require.True(t, ok)
  1076  			require.EqualValues(t, expected.Value, decoded.Value)
  1077  		}
  1078  	}
  1079  }
  1080  
  1081  func TestEncodeDMLBeforeDDL(t *testing.T) {
  1082  	helper := entry.NewSchemaTestHelper(t)
  1083  	defer helper.Close()
  1084  
  1085  	sql := `create table test.t(a int primary key, b int)`
  1086  	ddlEvent := helper.DDL2Event(sql)
  1087  
  1088  	sql = `insert into test.t values (1, 2)`
  1089  	row := helper.DML2Event(sql, "test", "t")
  1090  
  1091  	ctx := context.Background()
  1092  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1093  
  1094  	b, err := NewBuilder(ctx, codecConfig)
  1095  	require.NoError(t, err)
  1096  	enc := b.Build()
  1097  
  1098  	err = enc.AppendRowChangedEvent(ctx, "", row, func() {})
  1099  	require.NoError(t, err)
  1100  
  1101  	messages := enc.Build()
  1102  	require.Len(t, messages, 1)
  1103  
  1104  	dec, err := NewDecoder(ctx, codecConfig, nil)
  1105  	require.NoError(t, err)
  1106  
  1107  	err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
  1108  	require.NoError(t, err)
  1109  
  1110  	messageType, hasNext, err := dec.HasNext()
  1111  	require.NoError(t, err)
  1112  	require.True(t, hasNext)
  1113  	require.Equal(t, model.MessageTypeRow, messageType)
  1114  
  1115  	decodedRow, err := dec.NextRowChangedEvent()
  1116  	require.NoError(t, err)
  1117  	require.Nil(t, decodedRow)
  1118  
  1119  	m, err := enc.EncodeDDLEvent(ddlEvent)
  1120  	require.NoError(t, err)
  1121  
  1122  	err = dec.AddKeyValue(m.Key, m.Value)
  1123  	require.NoError(t, err)
  1124  
  1125  	messageType, hasNext, err = dec.HasNext()
  1126  	require.NoError(t, err)
  1127  	require.True(t, hasNext)
  1128  	require.Equal(t, model.MessageTypeDDL, messageType)
  1129  
  1130  	event, err := dec.NextDDLEvent()
  1131  	require.NoError(t, err)
  1132  	require.NotNil(t, event)
  1133  
  1134  	cachedEvents := dec.GetCachedEvents()
  1135  	for _, decodedRow = range cachedEvents {
  1136  		require.NotNil(t, decodedRow)
  1137  		require.NotNil(t, decodedRow.TableInfo)
  1138  		require.Equal(t, decodedRow.TableInfo.ID, event.TableInfo.ID)
  1139  	}
  1140  }
  1141  
  1142  func TestEncodeBootstrapEvent(t *testing.T) {
  1143  	helper := entry.NewSchemaTestHelper(t)
  1144  	defer helper.Close()
  1145  
  1146  	sql := `create table test.t(
  1147      	id int,
  1148      	name varchar(255) not null,
  1149      	age int,
  1150      	email varchar(255) not null,
  1151      	primary key(id, name),
  1152      	key idx_name_email(name, email))`
  1153  	ddlEvent := helper.DDL2Event(sql)
  1154  	ddlEvent.IsBootstrap = true
  1155  
  1156  	sql = `insert into test.t values (1, "jack", 23, "jack@abc.com")`
  1157  	row := helper.DML2Event(sql, "test", "t")
  1158  
  1159  	helper.Tk().MustExec("drop table test.t")
  1160  
  1161  	ctx := context.Background()
  1162  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1163  	for _, format := range []common.EncodingFormatType{
  1164  		common.EncodingFormatAvro,
  1165  		common.EncodingFormatJSON,
  1166  	} {
  1167  		codecConfig.EncodingFormat = format
  1168  		for _, compressionType := range []string{
  1169  			compression.None,
  1170  			compression.Snappy,
  1171  			compression.LZ4,
  1172  		} {
  1173  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
  1174  			b, err := NewBuilder(ctx, codecConfig)
  1175  			require.NoError(t, err)
  1176  			enc := b.Build()
  1177  
  1178  			m, err := enc.EncodeDDLEvent(ddlEvent)
  1179  			require.NoError(t, err)
  1180  
  1181  			dec, err := NewDecoder(ctx, codecConfig, nil)
  1182  			require.NoError(t, err)
  1183  
  1184  			err = dec.AddKeyValue(m.Key, m.Value)
  1185  			require.NoError(t, err)
  1186  
  1187  			messageType, hasNext, err := dec.HasNext()
  1188  			require.NoError(t, err)
  1189  			require.True(t, hasNext)
  1190  			require.Equal(t, model.MessageTypeDDL, messageType)
  1191  			require.NotEqual(t, 0, dec.msg.BuildTs)
  1192  
  1193  			event, err := dec.NextDDLEvent()
  1194  			require.NoError(t, err)
  1195  			require.Equal(t, ddlEvent.TableInfo.TableName.TableID, event.TableInfo.TableName.TableID)
  1196  			// Bootstrap event doesn't have query
  1197  			require.Equal(t, "", event.Query)
  1198  			require.Equal(t, len(ddlEvent.TableInfo.Columns), len(event.TableInfo.Columns))
  1199  			require.Equal(t, len(ddlEvent.TableInfo.Indices), len(event.TableInfo.Indices))
  1200  
  1201  			item := dec.memo.Read(ddlEvent.TableInfo.TableName.Schema,
  1202  				ddlEvent.TableInfo.TableName.Table, ddlEvent.TableInfo.UpdateTS)
  1203  			require.NotNil(t, item)
  1204  
  1205  			err = enc.AppendRowChangedEvent(context.Background(), "", row, func() {})
  1206  			require.NoError(t, err)
  1207  
  1208  			messages := enc.Build()
  1209  			require.Len(t, messages, 1)
  1210  
  1211  			err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
  1212  			require.NoError(t, err)
  1213  
  1214  			messageType, hasNext, err = dec.HasNext()
  1215  			require.NoError(t, err)
  1216  			require.True(t, hasNext)
  1217  			require.Equal(t, model.MessageTypeRow, messageType)
  1218  			require.NotEqual(t, 0, dec.msg.BuildTs)
  1219  
  1220  			decodedRow, err := dec.NextRowChangedEvent()
  1221  			require.NoError(t, err)
  1222  			require.Equal(t, decodedRow.CommitTs, row.CommitTs)
  1223  			require.Equal(t, decodedRow.TableInfo.GetSchemaName(), row.TableInfo.GetSchemaName())
  1224  			require.Equal(t, decodedRow.TableInfo.GetTableName(), row.TableInfo.GetTableName())
  1225  			require.Nil(t, decodedRow.PreColumns)
  1226  		}
  1227  	}
  1228  }
  1229  
  1230  func TestEncodeLargeEventsNormal(t *testing.T) {
  1231  	ddlEvent, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
  1232  
  1233  	ctx := context.Background()
  1234  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1235  	for _, format := range []common.EncodingFormatType{
  1236  		common.EncodingFormatAvro,
  1237  		common.EncodingFormatJSON,
  1238  	} {
  1239  		codecConfig.EncodingFormat = format
  1240  		for _, compressionType := range []string{
  1241  			compression.None,
  1242  			compression.Snappy,
  1243  			compression.LZ4,
  1244  		} {
  1245  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
  1246  
  1247  			b, err := NewBuilder(ctx, codecConfig)
  1248  			require.NoError(t, err)
  1249  			enc := b.Build()
  1250  
  1251  			dec, err := NewDecoder(ctx, codecConfig, nil)
  1252  			require.NoError(t, err)
  1253  
  1254  			m, err := enc.EncodeDDLEvent(ddlEvent)
  1255  			require.NoError(t, err)
  1256  
  1257  			err = dec.AddKeyValue(m.Key, m.Value)
  1258  			require.NoError(t, err)
  1259  
  1260  			messageType, hasNext, err := dec.HasNext()
  1261  			require.NoError(t, err)
  1262  			require.True(t, hasNext)
  1263  			require.Equal(t, model.MessageTypeDDL, messageType)
  1264  
  1265  			obtainedDDL, err := dec.NextDDLEvent()
  1266  			require.NoError(t, err)
  1267  			require.NotNil(t, obtainedDDL)
  1268  
  1269  			obtainedDefaultValues := make(map[string]interface{}, len(obtainedDDL.TableInfo.Columns))
  1270  			for _, col := range obtainedDDL.TableInfo.Columns {
  1271  				obtainedDefaultValues[col.Name.O] = model.GetColumnDefaultValue(col)
  1272  				switch col.GetType() {
  1273  				case mysql.TypeFloat, mysql.TypeDouble:
  1274  					require.Equal(t, 0, col.GetDecimal())
  1275  				default:
  1276  				}
  1277  			}
  1278  			for _, col := range ddlEvent.TableInfo.Columns {
  1279  				expected := model.GetColumnDefaultValue(col)
  1280  				obtained := obtainedDefaultValues[col.Name.O]
  1281  				require.Equal(t, expected, obtained)
  1282  			}
  1283  
  1284  			for _, event := range []*model.RowChangedEvent{insertEvent, updateEvent, deleteEvent} {
  1285  				err = enc.AppendRowChangedEvent(ctx, "", event, func() {})
  1286  				require.NoError(t, err)
  1287  
  1288  				messages := enc.Build()
  1289  				require.Len(t, messages, 1)
  1290  
  1291  				err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
  1292  				require.NoError(t, err)
  1293  
  1294  				messageType, hasNext, err = dec.HasNext()
  1295  				require.NoError(t, err)
  1296  				require.True(t, hasNext)
  1297  				require.Equal(t, model.MessageTypeRow, messageType)
  1298  
  1299  				if event.IsDelete() {
  1300  					require.Equal(t, dec.msg.Type, DMLTypeDelete)
  1301  				} else if event.IsUpdate() {
  1302  					require.Equal(t, dec.msg.Type, DMLTypeUpdate)
  1303  				} else {
  1304  					require.Equal(t, dec.msg.Type, DMLTypeInsert)
  1305  				}
  1306  
  1307  				decodedRow, err := dec.NextRowChangedEvent()
  1308  				require.NoError(t, err)
  1309  
  1310  				require.Equal(t, decodedRow.CommitTs, event.CommitTs)
  1311  				require.Equal(t, decodedRow.TableInfo.GetSchemaName(), event.TableInfo.GetSchemaName())
  1312  				require.Equal(t, decodedRow.TableInfo.GetTableName(), event.TableInfo.GetTableName())
  1313  				require.Equal(t, decodedRow.PhysicalTableID, event.PhysicalTableID)
  1314  
  1315  				decodedColumns := make(map[string]*model.ColumnData, len(decodedRow.Columns))
  1316  				for _, column := range decodedRow.Columns {
  1317  					colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1318  					decodedColumns[colName] = column
  1319  				}
  1320  				for _, col := range event.Columns {
  1321  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1322  					decoded, ok := decodedColumns[colName]
  1323  					require.True(t, ok)
  1324  					require.EqualValues(t, col.Value, decoded.Value)
  1325  				}
  1326  
  1327  				decodedPreviousColumns := make(map[string]*model.ColumnData, len(decodedRow.PreColumns))
  1328  				for _, column := range decodedRow.PreColumns {
  1329  					colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1330  					decodedPreviousColumns[colName] = column
  1331  				}
  1332  				for _, col := range event.PreColumns {
  1333  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1334  					decoded, ok := decodedPreviousColumns[colName]
  1335  					require.True(t, ok)
  1336  					require.EqualValues(t, col.Value, decoded.Value)
  1337  				}
  1338  			}
  1339  		}
  1340  	}
  1341  }
  1342  
  1343  func TestDDLMessageTooLarge(t *testing.T) {
  1344  	ddlEvent, _, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
  1345  
  1346  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1347  	codecConfig.MaxMessageBytes = 100
  1348  	for _, format := range []common.EncodingFormatType{
  1349  		common.EncodingFormatAvro,
  1350  		common.EncodingFormatJSON,
  1351  	} {
  1352  		codecConfig.EncodingFormat = format
  1353  		b, err := NewBuilder(context.Background(), codecConfig)
  1354  		require.NoError(t, err)
  1355  		enc := b.Build()
  1356  
  1357  		_, err = enc.EncodeDDLEvent(ddlEvent)
  1358  		require.ErrorIs(t, err, errors.ErrMessageTooLarge)
  1359  	}
  1360  }
  1361  
  1362  func TestDMLMessageTooLarge(t *testing.T) {
  1363  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
  1364  
  1365  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1366  	codecConfig.MaxMessageBytes = 50
  1367  
  1368  	for _, format := range []common.EncodingFormatType{
  1369  		common.EncodingFormatAvro,
  1370  		common.EncodingFormatJSON,
  1371  	} {
  1372  		codecConfig.EncodingFormat = format
  1373  
  1374  		for _, handle := range []string{
  1375  			config.LargeMessageHandleOptionNone,
  1376  			config.LargeMessageHandleOptionHandleKeyOnly,
  1377  			config.LargeMessageHandleOptionClaimCheck,
  1378  		} {
  1379  			codecConfig.LargeMessageHandle.LargeMessageHandleOption = handle
  1380  			if handle == config.LargeMessageHandleOptionClaimCheck {
  1381  				codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "file:///tmp/simple-claim-check"
  1382  			}
  1383  			b, err := NewBuilder(context.Background(), codecConfig)
  1384  			require.NoError(t, err)
  1385  			enc := b.Build()
  1386  
  1387  			err = enc.AppendRowChangedEvent(context.Background(), "", insertEvent, func() {})
  1388  			require.ErrorIs(t, err, errors.ErrMessageTooLarge, string(format), handle)
  1389  		}
  1390  	}
  1391  }
  1392  
  1393  func TestLargerMessageHandleClaimCheck(t *testing.T) {
  1394  	ddlEvent, _, updateEvent, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
  1395  
  1396  	ctx := context.Background()
  1397  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1398  	codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionClaimCheck
  1399  
  1400  	codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "unsupported:///"
  1401  	b, err := NewBuilder(ctx, codecConfig)
  1402  	require.Error(t, err)
  1403  	require.Nil(t, b)
  1404  
  1405  	badDec, err := NewDecoder(ctx, codecConfig, nil)
  1406  	require.Error(t, err)
  1407  	require.Nil(t, badDec)
  1408  
  1409  	codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "file:///tmp/simple-claim-check"
  1410  	for _, format := range []common.EncodingFormatType{
  1411  		common.EncodingFormatAvro,
  1412  		common.EncodingFormatJSON,
  1413  	} {
  1414  		codecConfig.EncodingFormat = format
  1415  		for _, compressionType := range []string{
  1416  			compression.None,
  1417  			compression.Snappy,
  1418  			compression.LZ4,
  1419  		} {
  1420  			codecConfig.MaxMessageBytes = config.DefaultMaxMessageBytes
  1421  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
  1422  
  1423  			b, err = NewBuilder(ctx, codecConfig)
  1424  			require.NoError(t, err)
  1425  			enc := b.Build()
  1426  
  1427  			m, err := enc.EncodeDDLEvent(ddlEvent)
  1428  			require.NoError(t, err)
  1429  
  1430  			dec, err := NewDecoder(ctx, codecConfig, nil)
  1431  			require.NoError(t, err)
  1432  
  1433  			err = dec.AddKeyValue(m.Key, m.Value)
  1434  			require.NoError(t, err)
  1435  
  1436  			messageType, hasNext, err := dec.HasNext()
  1437  			require.NoError(t, err)
  1438  			require.True(t, hasNext)
  1439  			require.Equal(t, model.MessageTypeDDL, messageType)
  1440  
  1441  			_, err = dec.NextDDLEvent()
  1442  			require.NoError(t, err)
  1443  
  1444  			enc.(*encoder).config.MaxMessageBytes = 500
  1445  			err = enc.AppendRowChangedEvent(ctx, "", updateEvent, func() {})
  1446  			require.NoError(t, err)
  1447  
  1448  			claimCheckLocationM := enc.Build()[0]
  1449  
  1450  			dec.config.MaxMessageBytes = 500
  1451  			err = dec.AddKeyValue(claimCheckLocationM.Key, claimCheckLocationM.Value)
  1452  			require.NoError(t, err)
  1453  
  1454  			messageType, hasNext, err = dec.HasNext()
  1455  			require.NoError(t, err)
  1456  			require.True(t, hasNext)
  1457  			require.Equal(t, model.MessageTypeRow, messageType)
  1458  			require.NotEqual(t, "", dec.msg.ClaimCheckLocation)
  1459  
  1460  			decodedRow, err := dec.NextRowChangedEvent()
  1461  			require.NoError(t, err)
  1462  
  1463  			require.Equal(t, decodedRow.CommitTs, updateEvent.CommitTs)
  1464  			require.Equal(t, decodedRow.TableInfo.GetSchemaName(), updateEvent.TableInfo.GetSchemaName())
  1465  			require.Equal(t, decodedRow.TableInfo.GetTableName(), updateEvent.TableInfo.GetTableName())
  1466  
  1467  			decodedColumns := make(map[string]*model.ColumnData, len(decodedRow.Columns))
  1468  			for _, column := range decodedRow.Columns {
  1469  				colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1470  				decodedColumns[colName] = column
  1471  			}
  1472  			for _, col := range updateEvent.Columns {
  1473  				colName := updateEvent.TableInfo.ForceGetColumnName(col.ColumnID)
  1474  				decoded, ok := decodedColumns[colName]
  1475  				require.True(t, ok)
  1476  				require.EqualValues(t, col.Value, decoded.Value)
  1477  			}
  1478  
  1479  			for _, column := range decodedRow.PreColumns {
  1480  				colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1481  				decodedColumns[colName] = column
  1482  			}
  1483  			for _, col := range updateEvent.PreColumns {
  1484  				colName := updateEvent.TableInfo.ForceGetColumnName(col.ColumnID)
  1485  				decoded, ok := decodedColumns[colName]
  1486  				require.True(t, ok)
  1487  				require.EqualValues(t, col.Value, decoded.Value)
  1488  			}
  1489  		}
  1490  	}
  1491  }
  1492  
  1493  func TestLargeMessageHandleKeyOnly(t *testing.T) {
  1494  	db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual))
  1495  	mock.MatchExpectationsInOrder(false)
  1496  	require.NoError(t, err)
  1497  
  1498  	ddlEvent, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
  1499  	ctx := context.Background()
  1500  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1501  	codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionHandleKeyOnly
  1502  
  1503  	badDec, err := NewDecoder(ctx, codecConfig, nil)
  1504  	require.Error(t, err)
  1505  	require.Nil(t, badDec)
  1506  
  1507  	events := []*model.RowChangedEvent{
  1508  		insertEvent,
  1509  		updateEvent,
  1510  		deleteEvent,
  1511  	}
  1512  
  1513  	for _, format := range []common.EncodingFormatType{
  1514  		common.EncodingFormatJSON,
  1515  		common.EncodingFormatAvro,
  1516  	} {
  1517  		codecConfig.EncodingFormat = format
  1518  		for _, compressionType := range []string{
  1519  			compression.None,
  1520  			compression.Snappy,
  1521  			compression.LZ4,
  1522  		} {
  1523  			codecConfig.MaxMessageBytes = config.DefaultMaxMessageBytes
  1524  			codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compressionType
  1525  
  1526  			b, err := NewBuilder(ctx, codecConfig)
  1527  			require.NoError(t, err)
  1528  			enc := b.Build()
  1529  
  1530  			dec, err := NewDecoder(ctx, codecConfig, db)
  1531  			require.NoError(t, err)
  1532  
  1533  			enc.(*encoder).config.MaxMessageBytes = 500
  1534  			dec.config.MaxMessageBytes = 500
  1535  			for _, event = range events {
  1536  				err = enc.AppendRowChangedEvent(ctx, "", event, func() {})
  1537  				require.NoError(t, err)
  1538  
  1539  				messages := enc.Build()
  1540  				require.Len(t, messages, 1)
  1541  
  1542  				err = dec.AddKeyValue(messages[0].Key, messages[0].Value)
  1543  				require.NoError(t, err)
  1544  
  1545  				messageType, hasNext, err := dec.HasNext()
  1546  				require.NoError(t, err)
  1547  				require.True(t, hasNext)
  1548  				require.Equal(t, model.MessageTypeRow, messageType)
  1549  				require.True(t, dec.msg.HandleKeyOnly)
  1550  
  1551  				obtainedValues := make(map[string]interface{}, len(dec.msg.Data))
  1552  				for name, value := range dec.msg.Data {
  1553  					obtainedValues[name] = value
  1554  				}
  1555  				for _, col := range event.Columns {
  1556  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1557  					colFlag := event.TableInfo.ForceGetColumnFlagType(col.ColumnID)
  1558  					if colFlag.IsHandleKey() {
  1559  						require.Contains(t, dec.msg.Data, colName)
  1560  						obtained := obtainedValues[colName]
  1561  						switch v := obtained.(type) {
  1562  						case string:
  1563  							var err error
  1564  							obtained, err = strconv.ParseInt(v, 10, 64)
  1565  							require.NoError(t, err)
  1566  						}
  1567  						require.EqualValues(t, col.Value, obtained)
  1568  					} else {
  1569  						require.NotContains(t, dec.msg.Data, colName)
  1570  					}
  1571  				}
  1572  
  1573  				clear(obtainedValues)
  1574  				for name, value := range dec.msg.Old {
  1575  					obtainedValues[name] = value
  1576  				}
  1577  				for _, col := range event.PreColumns {
  1578  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1579  					colFlag := event.TableInfo.ForceGetColumnFlagType(col.ColumnID)
  1580  					if colFlag.IsHandleKey() {
  1581  						require.Contains(t, dec.msg.Old, colName)
  1582  						obtained := obtainedValues[colName]
  1583  						switch v := obtained.(type) {
  1584  						case string:
  1585  							var err error
  1586  							obtained, err = strconv.ParseInt(v, 10, 64)
  1587  							require.NoError(t, err)
  1588  						}
  1589  						require.EqualValues(t, col.Value, obtained)
  1590  					} else {
  1591  						require.NotContains(t, dec.msg.Data, colName)
  1592  					}
  1593  				}
  1594  
  1595  				decodedRow, err := dec.NextRowChangedEvent()
  1596  				require.NoError(t, err)
  1597  				require.Nil(t, decodedRow)
  1598  			}
  1599  
  1600  			enc.(*encoder).config.MaxMessageBytes = config.DefaultMaxMessageBytes
  1601  			dec.config.MaxMessageBytes = config.DefaultMaxMessageBytes
  1602  			m, err := enc.EncodeDDLEvent(ddlEvent)
  1603  			require.NoError(t, err)
  1604  
  1605  			err = dec.AddKeyValue(m.Key, m.Value)
  1606  			require.NoError(t, err)
  1607  
  1608  			messageType, hasNext, err := dec.HasNext()
  1609  			require.NoError(t, err)
  1610  			require.True(t, hasNext)
  1611  			require.Equal(t, model.MessageTypeDDL, messageType)
  1612  
  1613  			for _, event = range events {
  1614  				mock.ExpectQuery("SELECT @@global.time_zone").
  1615  					WillReturnRows(mock.NewRows([]string{""}).AddRow("SYSTEM"))
  1616  
  1617  				query := fmt.Sprintf("set @@tidb_snapshot=%v", event.CommitTs)
  1618  				mock.ExpectExec(query).WillReturnResult(driver.ResultNoRows)
  1619  
  1620  				query = fmt.Sprintf("set @@tidb_snapshot=%v", event.CommitTs-1)
  1621  				mock.ExpectExec(query).WillReturnResult(driver.ResultNoRows)
  1622  
  1623  				names, values := utils.LargeColumnKeyValues()
  1624  				mock.ExpectQuery("select * from test.t where t = 127").
  1625  					WillReturnRows(mock.NewRows(names).AddRow(values...))
  1626  
  1627  				mock.ExpectQuery("select * from test.t where t = 127").
  1628  					WillReturnRows(mock.NewRows(names).AddRow(values...))
  1629  
  1630  			}
  1631  			_, err = dec.NextDDLEvent()
  1632  			require.NoError(t, err)
  1633  
  1634  			decodedRows := dec.GetCachedEvents()
  1635  			for idx, decodedRow := range decodedRows {
  1636  				event := events[idx]
  1637  
  1638  				require.Equal(t, decodedRow.CommitTs, event.CommitTs)
  1639  				require.Equal(t, decodedRow.TableInfo.GetSchemaName(), event.TableInfo.GetSchemaName())
  1640  				require.Equal(t, decodedRow.TableInfo.GetTableName(), event.TableInfo.GetTableName())
  1641  
  1642  				decodedColumns := make(map[string]*model.ColumnData, len(decodedRow.Columns))
  1643  				for _, column := range decodedRow.Columns {
  1644  					colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1645  					decodedColumns[colName] = column
  1646  				}
  1647  				for _, col := range event.Columns {
  1648  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1649  					decoded, ok := decodedColumns[colName]
  1650  					require.True(t, ok)
  1651  					colInfo := event.TableInfo.ForceGetColumnFlagType(col.ColumnID)
  1652  					if colInfo.IsBinary() {
  1653  						switch v := col.Value.(type) {
  1654  						case []byte:
  1655  							length := len(decoded.Value.([]uint8))
  1656  							require.Equal(t, v[:length], decoded.Value, colName)
  1657  						default:
  1658  							require.EqualValues(t, col.Value, decoded.Value, colName)
  1659  						}
  1660  					} else {
  1661  						require.EqualValues(t, col.Value, decoded.Value, colName)
  1662  					}
  1663  				}
  1664  
  1665  				clear(decodedColumns)
  1666  				for _, column := range decodedRow.PreColumns {
  1667  					colName := decodedRow.TableInfo.ForceGetColumnName(column.ColumnID)
  1668  					decodedColumns[colName] = column
  1669  				}
  1670  				for _, col := range event.PreColumns {
  1671  					colName := event.TableInfo.ForceGetColumnName(col.ColumnID)
  1672  					decoded, ok := decodedColumns[colName]
  1673  					require.True(t, ok)
  1674  					colInfo := event.TableInfo.ForceGetColumnFlagType(col.ColumnID)
  1675  					if colInfo.IsBinary() {
  1676  						switch v := col.Value.(type) {
  1677  						case []byte:
  1678  							length := len(decoded.Value.([]uint8))
  1679  							require.Equal(t, v[:length], decoded.Value, colName)
  1680  						default:
  1681  							require.EqualValues(t, col.Value, decoded.Value, colName)
  1682  						}
  1683  					} else {
  1684  						require.EqualValues(t, col.Value, decoded.Value, colName)
  1685  					}
  1686  				}
  1687  			}
  1688  		}
  1689  	}
  1690  }
  1691  
  1692  func TestDecoder(t *testing.T) {
  1693  	ctx := context.Background()
  1694  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1695  	decoder, err := NewDecoder(ctx, codecConfig, nil)
  1696  	require.NoError(t, err)
  1697  	require.NotNil(t, decoder)
  1698  
  1699  	messageType, hasNext, err := decoder.HasNext()
  1700  	require.NoError(t, err)
  1701  	require.False(t, hasNext)
  1702  	require.Equal(t, model.MessageTypeUnknown, messageType)
  1703  
  1704  	ddl, err := decoder.NextDDLEvent()
  1705  	require.ErrorIs(t, err, errors.ErrCodecDecode)
  1706  	require.Nil(t, ddl)
  1707  
  1708  	decoder.msg = new(message)
  1709  	checkpoint, err := decoder.NextResolvedEvent()
  1710  	require.ErrorIs(t, err, errors.ErrCodecDecode)
  1711  	require.Equal(t, uint64(0), checkpoint)
  1712  
  1713  	event, err := decoder.NextRowChangedEvent()
  1714  	require.ErrorIs(t, err, errors.ErrCodecDecode)
  1715  	require.Nil(t, event)
  1716  
  1717  	decoder.value = []byte("invalid")
  1718  	err = decoder.AddKeyValue(nil, nil)
  1719  	require.ErrorIs(t, err, errors.ErrCodecDecode)
  1720  }
  1721  
  1722  func TestMarshallerError(t *testing.T) {
  1723  	ctx := context.Background()
  1724  	codecConfig := common.NewConfig(config.ProtocolSimple)
  1725  
  1726  	b, err := NewBuilder(ctx, codecConfig)
  1727  	require.NoError(t, err)
  1728  	enc := b.Build()
  1729  
  1730  	mockMarshaller := mock_simple.NewMockmarshaller(gomock.NewController(t))
  1731  	enc.(*encoder).marshaller = mockMarshaller
  1732  
  1733  	mockMarshaller.EXPECT().MarshalCheckpoint(gomock.Any()).Return(nil, errors.ErrEncodeFailed)
  1734  	_, err = enc.EncodeCheckpointEvent(123)
  1735  	require.ErrorIs(t, err, errors.ErrEncodeFailed)
  1736  
  1737  	mockMarshaller.EXPECT().MarshalDDLEvent(gomock.Any()).Return(nil, errors.ErrEncodeFailed)
  1738  	_, err = enc.EncodeDDLEvent(&model.DDLEvent{})
  1739  	require.ErrorIs(t, err, errors.ErrEncodeFailed)
  1740  
  1741  	mockMarshaller.EXPECT().MarshalRowChangedEvent(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, errors.ErrEncodeFailed)
  1742  	err = enc.AppendRowChangedEvent(ctx, "", &model.RowChangedEvent{}, func() {})
  1743  	require.ErrorIs(t, err, errors.ErrEncodeFailed)
  1744  
  1745  	dec, err := NewDecoder(ctx, codecConfig, nil)
  1746  	require.NoError(t, err)
  1747  	dec.marshaller = mockMarshaller
  1748  
  1749  	mockMarshaller.EXPECT().Unmarshal(gomock.Any(), gomock.Any()).Return(errors.ErrDecodeFailed)
  1750  	err = dec.AddKeyValue([]byte("key"), []byte("value"))
  1751  	require.NoError(t, err)
  1752  
  1753  	messageType, hasNext, err := dec.HasNext()
  1754  	require.ErrorIs(t, err, errors.ErrDecodeFailed)
  1755  	require.False(t, hasNext)
  1756  	require.Equal(t, model.MessageTypeUnknown, messageType)
  1757  }