github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/bootstraper_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 codec
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"testing"
    20  	"time"
    21  
    22  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    23  	"github.com/pingcap/tiflow/cdc/entry"
    24  	"github.com/pingcap/tiflow/cdc/model"
    25  	"github.com/pingcap/tiflow/pkg/config"
    26  	"github.com/stretchr/testify/require"
    27  )
    28  
    29  func getMockTableStatus(tableName string,
    30  	tableID int64,
    31  	totalPartition int32,
    32  ) (model.TopicPartitionKey, *model.RowChangedEvent, *tableStatistic) {
    33  	schema := "test"
    34  	tableInfo := &model.TableInfo{
    35  		TableName: model.TableName{
    36  			Schema:  schema,
    37  			Table:   tableName,
    38  			TableID: tableID,
    39  		},
    40  		TableInfo: &timodel.TableInfo{
    41  			ID:       tableID,
    42  			UpdateTS: 1,
    43  		},
    44  	}
    45  	key := model.TopicPartitionKey{
    46  		Topic:          fmt.Sprintf("%s.%s", schema, tableName),
    47  		Partition:      1,
    48  		TotalPartition: totalPartition,
    49  	}
    50  	row := &model.RowChangedEvent{
    51  		PhysicalTableID: tableID,
    52  		TableInfo:       tableInfo,
    53  	}
    54  	tb := newTableStatistic(key, row)
    55  	return key, row, tb
    56  }
    57  
    58  func TestShouldSendBootstrapMsg(t *testing.T) {
    59  	t.Parallel()
    60  	defaultSendBootstrapInterval := time.Duration(config.DefaultSendBootstrapIntervalInSec) * time.Second
    61  	defaultSendBootstrapInMsgCount := config.DefaultSendBootstrapInMsgCount
    62  
    63  	_, _, tb1 := getMockTableStatus("t1", int64(1), int32(3))
    64  
    65  	// case 1: A new added table should send bootstrap message immediately
    66  	require.True(t, tb1.
    67  		shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount))
    68  
    69  	// case 2: A table which has sent bootstrap message should not send bootstrap message
    70  	tb1.lastSendTime.Store(time.Now())
    71  	require.False(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount))
    72  
    73  	// case 3: When the table receive message more than sendBootstrapInMsgCount,
    74  	// it should send bootstrap message
    75  	tb1.counter.Add(defaultSendBootstrapInMsgCount)
    76  	require.True(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount))
    77  
    78  	// case 4: When the table does not send bootstrap message for a sendBootstrapInterval time,
    79  	// it should send bootstrap message
    80  	tb1.lastSendTime.Store(time.Now().Add(-defaultSendBootstrapInterval))
    81  	require.True(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount))
    82  }
    83  
    84  func TestIsActive(t *testing.T) {
    85  	t.Parallel()
    86  	_, row, tb1 := getMockTableStatus("t1", int64(1), int32(3))
    87  	// case 1: A new added table should be active
    88  	require.False(t, tb1.isInactive(defaultMaxInactiveDuration))
    89  
    90  	// case 2: A table which does not receive message for a long time should be inactive
    91  	tb1.lastMsgReceivedTime.Store(time.Now().Add(-defaultMaxInactiveDuration))
    92  	require.True(t, tb1.isInactive(defaultMaxInactiveDuration))
    93  
    94  	// case 3: A table which receive message recently should be active
    95  	// Note: A table's update method will be call any time it receive message
    96  	// So use update method to simulate the table receive message
    97  	tb1.update(row, 1)
    98  	require.False(t, tb1.isInactive(defaultMaxInactiveDuration))
    99  }
   100  
   101  func TestBootstrapWorker(t *testing.T) {
   102  	t.Parallel()
   103  	// new builder
   104  	cfID := model.DefaultChangeFeedID("test")
   105  	builder := &MockRowEventEncoderBuilder{}
   106  	outCh := make(chan *future, defaultInputChanSize)
   107  	worker := newBootstrapWorker(
   108  		cfID,
   109  		outCh,
   110  		builder.Build(),
   111  		config.DefaultSendBootstrapIntervalInSec,
   112  		config.DefaultSendBootstrapInMsgCount,
   113  		false,
   114  		defaultMaxInactiveDuration)
   115  
   116  	// Start the worker in a separate goroutine
   117  	ctx, cancel := context.WithCancel(context.Background())
   118  	defer cancel()
   119  	go func() {
   120  		_ = worker.run(ctx)
   121  	}()
   122  
   123  	// case 1: A new added table should send bootstrap message immediately
   124  	// Configure `sendBootstrapToAllPartition` to false
   125  	// The bootstrap message number should be equal to 1
   126  	// Event if we send the same table twice, it should only send bootstrap message once
   127  	key1, row1, _ := getMockTableStatus("t1", int64(1), int32(3))
   128  	err := worker.addEvent(ctx, key1, row1)
   129  	require.NoError(t, err)
   130  	err = worker.addEvent(ctx, key1, row1)
   131  	require.NoError(t, err)
   132  	var msgCount int32
   133  	c1ctx, c1Cancel := context.WithTimeout(context.Background(), 3*time.Second)
   134  	defer c1Cancel()
   135  l1:
   136  	for {
   137  		select {
   138  		case future := <-outCh:
   139  			require.NotNil(t, future)
   140  			require.Equal(t, key1.Topic, future.Key.Topic)
   141  			msgCount++
   142  			if msgCount == 1 {
   143  				break l1
   144  			}
   145  		case <-c1ctx.Done():
   146  			break l1
   147  		}
   148  	}
   149  	// The bootstrap event is only sent to the first partition
   150  	require.Equal(t, int32(1), msgCount)
   151  
   152  	// case 2: Configure `sendBootstrapToAllPartition` to true
   153  	// The messages number should be equal to the total partition number
   154  	worker.sendBootstrapToAllPartition = true
   155  	key2, row2, _ := getMockTableStatus("t2", int64(2), int32(2))
   156  	err = worker.addEvent(ctx, key2, row2)
   157  	require.NoError(t, err)
   158  	err = worker.addEvent(ctx, key2, row2)
   159  	require.NoError(t, err)
   160  	msgCount = 0
   161  	c2ctx, c2Cancel := context.WithTimeout(context.Background(), 3*time.Second)
   162  	defer c2Cancel()
   163  l2:
   164  	for {
   165  		select {
   166  		case future := <-outCh:
   167  			require.NotNil(t, future)
   168  			require.Equal(t, key2.Topic, future.Key.Topic)
   169  			msgCount++
   170  			if msgCount == key2.TotalPartition {
   171  				break l2
   172  			}
   173  		case <-c2ctx.Done():
   174  			break l2
   175  		}
   176  	}
   177  	// The bootstrap events are sent to all partition
   178  	require.Equal(t, key2.TotalPartition, msgCount)
   179  }
   180  
   181  func TestUpdateTableStatistic(t *testing.T) {
   182  	helper := entry.NewSchemaTestHelper(t)
   183  	defer helper.Close()
   184  
   185  	sql := `create table test.t1(
   186  		id int primary key,
   187  		name varchar(64) not null,
   188  		age int,
   189  		email varchar(255) not null,
   190  		unique index idx_name(name),
   191  		index idx_age_email(age,email)
   192  	);`
   193  	tableInfo1 := helper.DDL2Event(sql).TableInfo
   194  	row1 := &model.RowChangedEvent{
   195  		PhysicalTableID: tableInfo1.ID,
   196  		TableInfo:       tableInfo1,
   197  	}
   198  	tableStatistic := newTableStatistic(model.TopicPartitionKey{}, row1)
   199  
   200  	// case 1: The tableStatistic should not be updated if the tableInfo is the same
   201  	tableStatistic.update(row1, 1)
   202  	require.Equal(t, tableInfo1, tableStatistic.tableInfo.Load().(*model.TableInfo))
   203  
   204  	// case 2: The tableStatistic should be updated if the tableInfo is different
   205  	sql = `alter table test.t1 add column address varchar(255) not null;`
   206  	tableInfo2 := helper.DDL2Event(sql).TableInfo
   207  	row2 := &model.RowChangedEvent{
   208  		PhysicalTableID: tableInfo2.ID,
   209  		TableInfo:       tableInfo2,
   210  	}
   211  	tableStatistic.update(row2, 1)
   212  	require.Equal(t, tableInfo2, tableStatistic.tableInfo.Load().(*model.TableInfo))
   213  
   214  	// case 3: The tableStatistic should be updated when rename table
   215  	sql = `alter table test.t1 rename to test.t2;`
   216  	tableInfo3 := helper.DDL2Event(sql).TableInfo
   217  	row3 := &model.RowChangedEvent{
   218  		PhysicalTableID: tableInfo3.ID,
   219  		TableInfo:       tableInfo3,
   220  	}
   221  	tableStatistic.update(row3, 1)
   222  	require.Equal(t, tableInfo3, tableStatistic.tableInfo.Load().(*model.TableInfo))
   223  }