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

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package canal
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"encoding/json"
    20  	"testing"
    21  
    22  	"github.com/pingcap/tiflow/cdc/entry"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  	"github.com/pingcap/tiflow/pkg/compression"
    25  	"github.com/pingcap/tiflow/pkg/config"
    26  	cerror "github.com/pingcap/tiflow/pkg/errors"
    27  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    28  	"github.com/pingcap/tiflow/pkg/sink/codec/utils"
    29  	"github.com/stretchr/testify/require"
    30  )
    31  
    32  func TestBuildCanalJSONRowEventEncoder(t *testing.T) {
    33  	t.Parallel()
    34  	cfg := common.NewConfig(config.ProtocolCanalJSON)
    35  
    36  	builder, err := NewJSONRowEventEncoderBuilder(context.Background(), cfg)
    37  	require.NoError(t, err)
    38  	encoder, ok := builder.Build().(*JSONRowEventEncoder)
    39  	require.True(t, ok)
    40  	require.NotNil(t, encoder.config)
    41  }
    42  
    43  func TestDMLE2E(t *testing.T) {
    44  	_, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
    45  
    46  	ctx := context.Background()
    47  
    48  	for _, enableTiDBExtension := range []bool{true, false} {
    49  		codecConfig := common.NewConfig(config.ProtocolCanalJSON)
    50  		codecConfig.EnableTiDBExtension = enableTiDBExtension
    51  		builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
    52  		require.NoError(t, err)
    53  
    54  		encoder := builder.Build()
    55  
    56  		err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {})
    57  		require.NoError(t, err)
    58  
    59  		message := encoder.Build()[0]
    60  
    61  		decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
    62  		require.NoError(t, err)
    63  
    64  		err = decoder.AddKeyValue(message.Key, message.Value)
    65  		require.NoError(t, err)
    66  
    67  		messageType, hasNext, err := decoder.HasNext()
    68  		require.NoError(t, err)
    69  		require.True(t, hasNext)
    70  		require.Equal(t, messageType, model.MessageTypeRow)
    71  
    72  		decodedEvent, err := decoder.NextRowChangedEvent()
    73  		require.NoError(t, err)
    74  
    75  		require.True(t, decodedEvent.IsInsert())
    76  		if enableTiDBExtension {
    77  			require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs)
    78  		}
    79  		require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName())
    80  		require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName())
    81  
    82  		decodedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns))
    83  		for _, column := range decodedEvent.Columns {
    84  			colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID)
    85  			decodedColumns[colName] = column
    86  		}
    87  		for _, col := range insertEvent.Columns {
    88  			colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID)
    89  			decoded, ok := decodedColumns[colName]
    90  			require.True(t, ok)
    91  			require.EqualValues(t, col.Value, decoded.Value)
    92  		}
    93  
    94  		err = encoder.AppendRowChangedEvent(ctx, "", updateEvent, func() {})
    95  		require.NoError(t, err)
    96  
    97  		message = encoder.Build()[0]
    98  
    99  		err = decoder.AddKeyValue(message.Key, message.Value)
   100  		require.NoError(t, err)
   101  
   102  		messageType, hasNext, err = decoder.HasNext()
   103  		require.NoError(t, err)
   104  		require.True(t, hasNext)
   105  		require.EqualValues(t, messageType, model.MessageTypeRow)
   106  
   107  		decodedEvent, err = decoder.NextRowChangedEvent()
   108  		require.NoError(t, err)
   109  		require.True(t, decodedEvent.IsUpdate())
   110  
   111  		err = encoder.AppendRowChangedEvent(ctx, "", deleteEvent, func() {})
   112  		require.NoError(t, err)
   113  
   114  		message = encoder.Build()[0]
   115  		err = decoder.AddKeyValue(message.Key, message.Value)
   116  		require.NoError(t, err)
   117  
   118  		messageType, hasNext, err = decoder.HasNext()
   119  		require.NoError(t, err)
   120  		require.True(t, hasNext)
   121  		require.EqualValues(t, messageType, model.MessageTypeRow)
   122  
   123  		decodedEvent, err = decoder.NextRowChangedEvent()
   124  		require.NoError(t, err)
   125  		require.True(t, decodedEvent.IsDelete())
   126  	}
   127  }
   128  
   129  func TestCanalJSONCompressionE2E(t *testing.T) {
   130  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   131  
   132  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   133  	codecConfig.EnableTiDBExtension = true
   134  	codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.LZ4
   135  
   136  	ctx := context.Background()
   137  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   138  	require.NoError(t, err)
   139  	encoder := builder.Build()
   140  
   141  	// encode normal row changed event
   142  	err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {})
   143  	require.NoError(t, err)
   144  
   145  	message := encoder.Build()[0]
   146  
   147  	decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   148  	require.NoError(t, err)
   149  
   150  	err = decoder.AddKeyValue(message.Key, message.Value)
   151  	require.NoError(t, err)
   152  
   153  	messageType, hasNext, err := decoder.HasNext()
   154  	require.NoError(t, err)
   155  	require.True(t, hasNext)
   156  	require.Equal(t, messageType, model.MessageTypeRow)
   157  
   158  	decodedEvent, err := decoder.NextRowChangedEvent()
   159  	require.NoError(t, err)
   160  	require.Equal(t, decodedEvent.CommitTs, insertEvent.CommitTs)
   161  	require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName())
   162  	require.Equal(t, decodedEvent.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName())
   163  
   164  	// encode DDL event
   165  	helper := entry.NewSchemaTestHelper(t)
   166  	defer helper.Close()
   167  
   168  	sql := `create table test.person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))`
   169  	ddlEvent := helper.DDL2Event(sql)
   170  
   171  	message, err = encoder.EncodeDDLEvent(ddlEvent)
   172  	require.NoError(t, err)
   173  
   174  	err = decoder.AddKeyValue(message.Key, message.Value)
   175  	require.NoError(t, err)
   176  
   177  	messageType, hasNext, err = decoder.HasNext()
   178  	require.NoError(t, err)
   179  	require.True(t, hasNext)
   180  	require.Equal(t, messageType, model.MessageTypeDDL)
   181  
   182  	decodedDDL, err := decoder.NextDDLEvent()
   183  	require.NoError(t, err)
   184  
   185  	require.Equal(t, decodedDDL.Query, ddlEvent.Query)
   186  	require.Equal(t, decodedDDL.CommitTs, ddlEvent.CommitTs)
   187  	require.Equal(t, decodedDDL.TableInfo.TableName.Schema, ddlEvent.TableInfo.TableName.Schema)
   188  	require.Equal(t, decodedDDL.TableInfo.TableName.Table, ddlEvent.TableInfo.TableName.Table)
   189  
   190  	// encode checkpoint event
   191  	waterMark := uint64(2333)
   192  	message, err = encoder.EncodeCheckpointEvent(waterMark)
   193  	require.NoError(t, err)
   194  
   195  	err = decoder.AddKeyValue(message.Key, message.Value)
   196  	require.NoError(t, err)
   197  
   198  	messageType, hasNext, err = decoder.HasNext()
   199  	require.NoError(t, err)
   200  	require.True(t, hasNext)
   201  	require.Equal(t, messageType, model.MessageTypeResolved)
   202  
   203  	decodedWatermark, err := decoder.NextResolvedEvent()
   204  	require.NoError(t, err)
   205  	require.Equal(t, decodedWatermark, waterMark)
   206  }
   207  
   208  func TestCanalJSONClaimCheckE2E(t *testing.T) {
   209  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   210  	codecConfig.EnableTiDBExtension = true
   211  	codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionClaimCheck
   212  	codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.Snappy
   213  	codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "file:///tmp/canal-json-claim-check"
   214  	codecConfig.MaxMessageBytes = 500
   215  	ctx := context.Background()
   216  
   217  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   218  	require.NoError(t, err)
   219  	encoder := builder.Build()
   220  
   221  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   222  	err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {})
   223  	require.NoError(t, err)
   224  
   225  	// this is a large message, should be delivered to the external storage.
   226  	claimCheckLocationMessage := encoder.Build()[0]
   227  
   228  	decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   229  	require.NoError(t, err)
   230  
   231  	err = decoder.AddKeyValue(claimCheckLocationMessage.Key, claimCheckLocationMessage.Value)
   232  	require.NoError(t, err)
   233  
   234  	messageType, ok, err := decoder.HasNext()
   235  	require.NoError(t, err)
   236  	require.Equal(t, messageType, model.MessageTypeRow)
   237  	require.True(t, ok)
   238  
   239  	decodedLargeEvent, err := decoder.NextRowChangedEvent()
   240  	require.NoError(t, err)
   241  
   242  	require.Equal(t, insertEvent.CommitTs, decodedLargeEvent.CommitTs)
   243  	require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedLargeEvent.TableInfo.GetSchemaName())
   244  	require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedLargeEvent.TableInfo.GetTableName())
   245  	require.Nil(t, nil, decodedLargeEvent.PreColumns)
   246  
   247  	decodedColumns := make(map[string]*model.ColumnData, len(decodedLargeEvent.Columns))
   248  	for _, column := range decodedLargeEvent.Columns {
   249  		colName := decodedLargeEvent.TableInfo.ForceGetColumnName(column.ColumnID)
   250  		decodedColumns[colName] = column
   251  	}
   252  	for _, col := range insertEvent.Columns {
   253  		colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID)
   254  		decoded, ok := decodedColumns[colName]
   255  		require.True(t, ok)
   256  		require.EqualValues(t, col.Value, decoded.Value)
   257  	}
   258  }
   259  
   260  func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) {
   261  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   262  	codecConfig.EnableTiDBExtension = true
   263  	codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionHandleKeyOnly
   264  	codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.LZ4
   265  	codecConfig.MaxMessageBytes = 500
   266  
   267  	ctx := context.Background()
   268  
   269  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   270  	require.NoError(t, err)
   271  	encoder := builder.Build()
   272  
   273  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   274  	err = encoder.AppendRowChangedEvent(context.Background(), "", insertEvent, func() {})
   275  	require.NoError(t, err)
   276  
   277  	message := encoder.Build()[0]
   278  
   279  	decoder, err := NewBatchDecoder(context.Background(), codecConfig, &sql.DB{})
   280  	require.NoError(t, err)
   281  
   282  	err = decoder.AddKeyValue(message.Key, message.Value)
   283  	require.NoError(t, err)
   284  
   285  	messageType, ok, err := decoder.HasNext()
   286  	require.NoError(t, err)
   287  	require.True(t, ok)
   288  	require.Equal(t, messageType, model.MessageTypeRow)
   289  
   290  	handleKeyOnlyMessage := decoder.(*batchDecoder).msg.(*canalJSONMessageWithTiDBExtension)
   291  	require.True(t, handleKeyOnlyMessage.Extensions.OnlyHandleKey)
   292  
   293  	for _, col := range insertEvent.Columns {
   294  		colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID)
   295  		if insertEvent.TableInfo.ForceGetColumnFlagType(col.ColumnID).IsHandleKey() {
   296  			require.Contains(t, handleKeyOnlyMessage.Data[0], colName)
   297  			require.Contains(t, handleKeyOnlyMessage.SQLType, colName)
   298  			require.Contains(t, handleKeyOnlyMessage.MySQLType, colName)
   299  		} else {
   300  			require.NotContains(t, handleKeyOnlyMessage.Data[0], colName)
   301  			require.NotContains(t, handleKeyOnlyMessage.SQLType, colName)
   302  			require.NotContains(t, handleKeyOnlyMessage.MySQLType, colName)
   303  		}
   304  	}
   305  }
   306  
   307  func TestNewCanalJSONMessageFromDDL(t *testing.T) {
   308  	helper := entry.NewSchemaTestHelper(t)
   309  	defer helper.Close()
   310  
   311  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   312  	ctx := context.Background()
   313  
   314  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   315  	require.NoError(t, err)
   316  	encoder := builder.Build().(*JSONRowEventEncoder)
   317  
   318  	sql := `create table test.person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))`
   319  	ddlEvent := helper.DDL2Event(sql)
   320  
   321  	message := encoder.newJSONMessageForDDL(ddlEvent)
   322  	require.NotNil(t, message)
   323  
   324  	msg, ok := message.(*JSONMessage)
   325  	require.True(t, ok)
   326  	require.Equal(t, convertToCanalTs(ddlEvent.CommitTs), msg.ExecutionTime)
   327  	require.True(t, msg.IsDDL)
   328  	require.Equal(t, "test", msg.Schema)
   329  	require.Equal(t, "person", msg.Table)
   330  	require.Equal(t, ddlEvent.Query, msg.Query)
   331  	require.Equal(t, "CREATE", msg.EventType)
   332  
   333  	codecConfig.EnableTiDBExtension = true
   334  	builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   335  	require.NoError(t, err)
   336  
   337  	encoder = builder.Build().(*JSONRowEventEncoder)
   338  	message = encoder.newJSONMessageForDDL(ddlEvent)
   339  	require.NotNil(t, message)
   340  
   341  	withExtension, ok := message.(*canalJSONMessageWithTiDBExtension)
   342  	require.True(t, ok)
   343  
   344  	require.NotNil(t, withExtension.Extensions)
   345  	require.Equal(t, ddlEvent.CommitTs, withExtension.Extensions.CommitTs)
   346  }
   347  
   348  func TestBatching(t *testing.T) {
   349  	ctx := context.Background()
   350  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   351  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   352  	require.NoError(t, err)
   353  	encoder := builder.Build()
   354  	require.NotNil(t, encoder)
   355  
   356  	_, _, updateEvent, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   357  	updateCase := *updateEvent
   358  	for i := 1; i <= 1000; i++ {
   359  		ts := uint64(i)
   360  		updateCase.CommitTs = ts
   361  		err := encoder.AppendRowChangedEvent(context.Background(), "", &updateCase, nil)
   362  		require.NoError(t, err)
   363  
   364  		if i%100 == 0 {
   365  			msgs := encoder.Build()
   366  			require.NotNil(t, msgs)
   367  			require.Len(t, msgs, 100)
   368  
   369  			for j := range msgs {
   370  				require.Equal(t, 1, msgs[j].GetRowsCount())
   371  
   372  				var msg JSONMessage
   373  				err := json.Unmarshal(msgs[j].Value, &msg)
   374  				require.NoError(t, err)
   375  				require.Equal(t, "UPDATE", msg.EventType)
   376  			}
   377  		}
   378  	}
   379  
   380  	require.Len(t, encoder.(*JSONRowEventEncoder).messages, 0)
   381  }
   382  
   383  func TestEncodeCheckpointEvent(t *testing.T) {
   384  	t.Parallel()
   385  
   386  	ctx := context.Background()
   387  	var watermark uint64 = 2333
   388  	for _, enable := range []bool{false, true} {
   389  		codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   390  		codecConfig.EnableTiDBExtension = enable
   391  
   392  		builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   393  		require.NoError(t, err)
   394  
   395  		encoder := builder.Build()
   396  
   397  		msg, err := encoder.EncodeCheckpointEvent(watermark)
   398  		require.NoError(t, err)
   399  
   400  		if !enable {
   401  			require.Nil(t, msg)
   402  			continue
   403  		}
   404  
   405  		require.NotNil(t, msg)
   406  
   407  		ctx := context.Background()
   408  		decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   409  		require.NoError(t, err)
   410  
   411  		err = decoder.AddKeyValue(msg.Key, msg.Value)
   412  		require.NoError(t, err)
   413  
   414  		ty, hasNext, err := decoder.HasNext()
   415  		require.NoError(t, err)
   416  		if enable {
   417  			require.True(t, hasNext)
   418  			require.Equal(t, model.MessageTypeResolved, ty)
   419  			consumed, err := decoder.NextResolvedEvent()
   420  			require.NoError(t, err)
   421  			require.Equal(t, watermark, consumed)
   422  		} else {
   423  			require.False(t, hasNext)
   424  			require.Equal(t, model.MessageTypeUnknown, ty)
   425  		}
   426  
   427  		ty, hasNext, err = decoder.HasNext()
   428  		require.NoError(t, err)
   429  		require.False(t, hasNext)
   430  		require.Equal(t, model.MessageTypeUnknown, ty)
   431  	}
   432  }
   433  
   434  func TestCheckpointEventValueMarshal(t *testing.T) {
   435  	t.Parallel()
   436  
   437  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   438  	codecConfig.EnableTiDBExtension = true
   439  
   440  	ctx := context.Background()
   441  
   442  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   443  	require.NoError(t, err)
   444  
   445  	encoder := builder.Build()
   446  	var watermark uint64 = 1024
   447  	msg, err := encoder.EncodeCheckpointEvent(watermark)
   448  	require.NoError(t, err)
   449  	require.NotNil(t, msg)
   450  
   451  	// Unmarshal from the data we have encoded.
   452  	jsonMsg := canalJSONMessageWithTiDBExtension{
   453  		&JSONMessage{},
   454  		&tidbExtension{},
   455  	}
   456  	err = json.Unmarshal(msg.Value, &jsonMsg)
   457  	require.NoError(t, err)
   458  	require.Equal(t, watermark, jsonMsg.Extensions.WatermarkTs)
   459  	require.Equal(t, tidbWaterMarkType, jsonMsg.EventType)
   460  	require.Equal(t, "", jsonMsg.Schema)
   461  	require.Equal(t, "", jsonMsg.Table)
   462  	require.Equal(t, "", jsonMsg.Query)
   463  	require.False(t, jsonMsg.IsDDL)
   464  	require.EqualValues(t, 0, jsonMsg.ExecutionTime)
   465  	require.Nil(t, jsonMsg.Data)
   466  	require.Nil(t, jsonMsg.Old)
   467  	require.Nil(t, jsonMsg.SQLType)
   468  	require.Nil(t, jsonMsg.MySQLType)
   469  }
   470  
   471  func TestDDLEventWithExtension(t *testing.T) {
   472  	helper := entry.NewSchemaTestHelper(t)
   473  	defer helper.Close()
   474  
   475  	ctx := context.Background()
   476  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   477  	codecConfig.EnableTiDBExtension = true
   478  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   479  	require.NoError(t, err)
   480  	encoder := builder.Build()
   481  	require.NotNil(t, encoder)
   482  
   483  	sql := `create table test.person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))`
   484  	ddlEvent := helper.DDL2Event(sql)
   485  
   486  	message, err := encoder.EncodeDDLEvent(ddlEvent)
   487  	require.NoError(t, err)
   488  
   489  	decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   490  	require.NoError(t, err)
   491  
   492  	err = decoder.AddKeyValue(message.Key, message.Value)
   493  	require.NoError(t, err)
   494  
   495  	messageType, hasNext, err := decoder.HasNext()
   496  	require.NoError(t, err)
   497  	require.True(t, hasNext)
   498  	require.Equal(t, messageType, model.MessageTypeDDL)
   499  
   500  	decodedDDL, err := decoder.NextDDLEvent()
   501  	require.NoError(t, err)
   502  	require.Equal(t, ddlEvent.Query, decodedDDL.Query)
   503  	require.Equal(t, ddlEvent.CommitTs, decodedDDL.CommitTs)
   504  	require.Equal(t, ddlEvent.TableInfo.TableName.Schema, decodedDDL.TableInfo.TableName.Schema)
   505  	require.Equal(t, ddlEvent.TableInfo.TableName.Table, decodedDDL.TableInfo.TableName.Table)
   506  }
   507  
   508  func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) {
   509  	helper := entry.NewSchemaTestHelper(t)
   510  	defer helper.Close()
   511  
   512  	sql := `create table test.t(a varchar(255) primary key)`
   513  	_ = helper.DDL2Event(sql)
   514  
   515  	sql = `insert into test.t values ("aa")`
   516  	row := helper.DML2Event(sql, "test", "t")
   517  
   518  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   519  	codecConfig.EnableTiDBExtension = true
   520  	ctx := context.Background()
   521  
   522  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   523  	require.NoError(t, err)
   524  	encoder := builder.Build()
   525  
   526  	count := 0
   527  	tests := []struct {
   528  		row      *model.RowChangedEvent
   529  		callback func()
   530  	}{
   531  		{
   532  			row: row,
   533  			callback: func() {
   534  				count += 1
   535  			},
   536  		},
   537  		{
   538  			row: row,
   539  			callback: func() {
   540  				count += 2
   541  			},
   542  		},
   543  		{
   544  			row: row,
   545  			callback: func() {
   546  				count += 3
   547  			},
   548  		},
   549  		{
   550  			row: row,
   551  			callback: func() {
   552  				count += 4
   553  			},
   554  		},
   555  		{
   556  			row: row,
   557  			callback: func() {
   558  				count += 5
   559  			},
   560  		},
   561  	}
   562  
   563  	// Empty build makes sure that the callback build logic not broken.
   564  	msgs := encoder.Build()
   565  	require.Len(t, msgs, 0, "no message should be built and no panic")
   566  
   567  	// Append the events.
   568  	for _, test := range tests {
   569  		err := encoder.AppendRowChangedEvent(context.Background(), "", test.row, test.callback)
   570  		require.NoError(t, err)
   571  	}
   572  	require.Equal(t, 0, count, "nothing should be called")
   573  
   574  	msgs = encoder.Build()
   575  	require.Len(t, msgs, 5, "expected 5 messages")
   576  	msgs[0].Callback()
   577  	require.Equal(t, 1, count, "expected one callback be called")
   578  	msgs[1].Callback()
   579  	require.Equal(t, 3, count, "expected one callback be called")
   580  	msgs[2].Callback()
   581  	require.Equal(t, 6, count, "expected one callback be called")
   582  	msgs[3].Callback()
   583  	require.Equal(t, 10, count, "expected one callback be called")
   584  	msgs[4].Callback()
   585  	require.Equal(t, 15, count, "expected one callback be called")
   586  }
   587  
   588  func TestMaxMessageBytes(t *testing.T) {
   589  	helper := entry.NewSchemaTestHelper(t)
   590  	defer helper.Close()
   591  
   592  	sql := `create table test.t(a varchar(255) primary key)`
   593  	_ = helper.DDL2Event(sql)
   594  
   595  	sql = `insert into test.t values ("aa")`
   596  	row := helper.DML2Event(sql, "test", "t")
   597  
   598  	ctx := context.Background()
   599  	topic := ""
   600  
   601  	// the test message length is smaller than max-message-bytes
   602  	maxMessageBytes := 300
   603  	codecConfig := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes)
   604  
   605  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   606  	require.NoError(t, err)
   607  	encoder := builder.Build()
   608  
   609  	err = encoder.AppendRowChangedEvent(ctx, topic, row, nil)
   610  	require.NoError(t, err)
   611  
   612  	// the test message length is larger than max-message-bytes
   613  	codecConfig = codecConfig.WithMaxMessageBytes(100)
   614  
   615  	builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   616  	require.NoError(t, err)
   617  
   618  	encoder = builder.Build()
   619  	err = encoder.AppendRowChangedEvent(ctx, topic, row, nil)
   620  	require.Error(t, err, cerror.ErrMessageTooLarge)
   621  }
   622  
   623  func TestCanalJSONContentCompatibleE2E(t *testing.T) {
   624  	ctx := context.Background()
   625  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   626  	codecConfig.EnableTiDBExtension = true
   627  	codecConfig.ContentCompatible = true
   628  
   629  	builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   630  	require.NoError(t, err)
   631  
   632  	encoder := builder.Build()
   633  
   634  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   635  	err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {})
   636  	require.NoError(t, err)
   637  
   638  	message := encoder.Build()[0]
   639  
   640  	decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   641  	require.NoError(t, err)
   642  
   643  	err = decoder.AddKeyValue(message.Key, message.Value)
   644  	require.NoError(t, err)
   645  
   646  	messageType, hasNext, err := decoder.HasNext()
   647  	require.NoError(t, err)
   648  	require.True(t, hasNext)
   649  	require.Equal(t, messageType, model.MessageTypeRow)
   650  
   651  	decodedEvent, err := decoder.NextRowChangedEvent()
   652  	require.NoError(t, err)
   653  	require.Equal(t, decodedEvent.CommitTs, insertEvent.CommitTs)
   654  	require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName())
   655  	require.Equal(t, decodedEvent.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName())
   656  
   657  	obtainedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns))
   658  	for _, column := range decodedEvent.Columns {
   659  		colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID)
   660  		obtainedColumns[colName] = column
   661  	}
   662  	for _, col := range insertEvent.Columns {
   663  		colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID)
   664  		decoded, ok := obtainedColumns[colName]
   665  		require.True(t, ok)
   666  		require.EqualValues(t, col.Value, decoded.Value)
   667  	}
   668  }
   669  
   670  func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) {
   671  	_, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig())
   672  	ctx := context.Background()
   673  
   674  	for _, encodeEnable := range []bool{false, true} {
   675  		encodeConfig := common.NewConfig(config.ProtocolCanalJSON)
   676  		encodeConfig.EnableTiDBExtension = encodeEnable
   677  		encodeConfig.Terminator = config.CRLF
   678  
   679  		builder, err := NewJSONRowEventEncoderBuilder(ctx, encodeConfig)
   680  		require.NoError(t, err)
   681  		encoder := builder.Build()
   682  
   683  		err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, nil)
   684  		require.NoError(t, err)
   685  
   686  		messages := encoder.Build()
   687  		require.Equal(t, 1, len(messages))
   688  		msg := messages[0]
   689  
   690  		for _, decodeEnable := range []bool{false, true} {
   691  			decodeConfig := common.NewConfig(config.ProtocolCanalJSON)
   692  			decodeConfig.EnableTiDBExtension = decodeEnable
   693  			decoder, err := NewBatchDecoder(ctx, decodeConfig, nil)
   694  			require.NoError(t, err)
   695  			err = decoder.AddKeyValue(msg.Key, msg.Value)
   696  			require.NoError(t, err)
   697  
   698  			ty, hasNext, err := decoder.HasNext()
   699  			require.NoError(t, err)
   700  			require.True(t, hasNext)
   701  			require.Equal(t, model.MessageTypeRow, ty)
   702  
   703  			decodedEvent, err := decoder.NextRowChangedEvent()
   704  			require.NoError(t, err)
   705  
   706  			if encodeEnable && decodeEnable {
   707  				require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs)
   708  			}
   709  			require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName())
   710  			require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName())
   711  
   712  			decodedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns))
   713  			for _, column := range decodedEvent.Columns {
   714  				colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID)
   715  				decodedColumns[colName] = column
   716  			}
   717  			for _, col := range insertEvent.Columns {
   718  				colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID)
   719  				decoded, ok := decodedColumns[colName]
   720  				require.True(t, ok)
   721  				require.EqualValues(t, col.Value, decoded.Value)
   722  			}
   723  
   724  			_, hasNext, _ = decoder.HasNext()
   725  			require.False(t, hasNext)
   726  
   727  			decodedEvent, err = decoder.NextRowChangedEvent()
   728  			require.Error(t, err)
   729  			require.Nil(t, decodedEvent)
   730  		}
   731  	}
   732  }
   733  
   734  func TestNewCanalJSONBatchDecoder4DDLMessage(t *testing.T) {
   735  	helper := entry.NewSchemaTestHelper(t)
   736  	defer helper.Close()
   737  
   738  	sql := `create table test.person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))`
   739  	ddlEvent := helper.DDL2Event(sql)
   740  
   741  	ctx := context.Background()
   742  	for _, encodeEnable := range []bool{false, true} {
   743  		codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   744  		codecConfig.EnableTiDBExtension = encodeEnable
   745  
   746  		builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig)
   747  		require.NoError(t, err)
   748  		encoder := builder.Build()
   749  
   750  		result, err := encoder.EncodeDDLEvent(ddlEvent)
   751  		require.NoError(t, err)
   752  		require.NotNil(t, result)
   753  
   754  		for _, decodeEnable := range []bool{false, true} {
   755  			codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   756  			codecConfig.EnableTiDBExtension = decodeEnable
   757  			decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   758  			require.NoError(t, err)
   759  			err = decoder.AddKeyValue(nil, result.Value)
   760  			require.NoError(t, err)
   761  
   762  			ty, hasNext, err := decoder.HasNext()
   763  			require.Nil(t, err)
   764  			require.True(t, hasNext)
   765  			require.Equal(t, model.MessageTypeDDL, ty)
   766  
   767  			consumed, err := decoder.NextDDLEvent()
   768  			require.Nil(t, err)
   769  
   770  			if encodeEnable && decodeEnable {
   771  				require.Equal(t, ddlEvent.CommitTs, consumed.CommitTs)
   772  			} else {
   773  				require.Equal(t, uint64(0), consumed.CommitTs)
   774  			}
   775  
   776  			require.Equal(t, ddlEvent.TableInfo.TableName.Schema, consumed.TableInfo.TableName.Schema)
   777  			require.Equal(t, ddlEvent.TableInfo.TableName.Table, consumed.TableInfo.TableName.Table)
   778  			require.Equal(t, ddlEvent.Query, consumed.Query)
   779  
   780  			ty, hasNext, err = decoder.HasNext()
   781  			require.Nil(t, err)
   782  			require.False(t, hasNext)
   783  			require.Equal(t, model.MessageTypeUnknown, ty)
   784  
   785  			consumed, err = decoder.NextDDLEvent()
   786  			require.NotNil(t, err)
   787  			require.Nil(t, consumed)
   788  		}
   789  	}
   790  }
   791  
   792  func TestCanalJSONBatchDecoderWithTerminator(t *testing.T) {
   793  	encodedValue := `{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"INSERT","es":1668067205238,"ts":1668067206650,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}],"old":null}
   794  {"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"UPDATE","es":1668067229137,"ts":1668067230720,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}]}
   795  {"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"DELETE","es":1668067230388,"ts":1668067231725,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":null}`
   796  	ctx := context.Background()
   797  	codecConfig := common.NewConfig(config.ProtocolCanalJSON)
   798  	codecConfig.Terminator = "\n"
   799  	decoder, err := NewBatchDecoder(ctx, codecConfig, nil)
   800  	require.NoError(t, err)
   801  
   802  	err = decoder.AddKeyValue(nil, []byte(encodedValue))
   803  	require.NoError(t, err)
   804  
   805  	cnt := 0
   806  	for {
   807  		tp, hasNext, err := decoder.HasNext()
   808  		if !hasNext {
   809  			break
   810  		}
   811  		require.NoError(t, err)
   812  		require.Equal(t, model.MessageTypeRow, tp)
   813  		cnt++
   814  		event, err := decoder.NextRowChangedEvent()
   815  		require.NoError(t, err)
   816  		require.NotNil(t, event)
   817  	}
   818  	require.Equal(t, 3, cnt)
   819  }