github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/table_sink_advancer_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 sinkmanager
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  	"testing"
    20  	"time"
    21  
    22  	"github.com/pingcap/tiflow/cdc/model"
    23  	"github.com/pingcap/tiflow/cdc/processor/memquota"
    24  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter"
    25  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    26  	"github.com/pingcap/tiflow/pkg/spanz"
    27  	"github.com/stretchr/testify/require"
    28  	"github.com/stretchr/testify/suite"
    29  )
    30  
    31  type tableSinkAdvancerSuite struct {
    32  	suite.Suite
    33  	testChangefeedID    model.ChangeFeedID
    34  	testSpan            tablepb.Span
    35  	defaultTestMemQuota uint64
    36  }
    37  
    38  func (suite *tableSinkAdvancerSuite) SetupSuite() {
    39  	requestMemSize = 256
    40  	maxUpdateIntervalSize = 512
    41  	suite.testChangefeedID = model.DefaultChangeFeedID("1")
    42  	suite.testSpan = spanz.TableIDToComparableSpan(1)
    43  	suite.defaultTestMemQuota = 1024
    44  }
    45  
    46  func (suite *tableSinkAdvancerSuite) SetupTest() {
    47  	// reset batchID
    48  	// We set batchID to 1 because we want to test the case that
    49  	// the first batchID is 1. Normally, the first batchID should
    50  	// never be 0.
    51  	batchID.Store(1)
    52  }
    53  
    54  func (suite *tableSinkAdvancerSuite) TearDownSuite() {
    55  	requestMemSize = defaultRequestMemSize
    56  	maxUpdateIntervalSize = defaultMaxUpdateIntervalSize
    57  }
    58  
    59  func TestTableSinkAdvancerSuite(t *testing.T) {
    60  	suite.Run(t, new(tableSinkAdvancerSuite))
    61  }
    62  
    63  func (suite *tableSinkAdvancerSuite) genSinkTask() (*sinkTask, *mockSink) {
    64  	wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan)
    65  	task := &sinkTask{
    66  		span:      suite.testSpan,
    67  		tableSink: wrapper,
    68  	}
    69  
    70  	return task, sink
    71  }
    72  
    73  func (suite *tableSinkAdvancerSuite) genMemQuota(initMemQuota uint64) *memquota.MemQuota {
    74  	memoryQuota := memquota.NewMemQuota(suite.testChangefeedID, suite.defaultTestMemQuota, "sink")
    75  	memoryQuota.ForceAcquire(initMemQuota)
    76  	memoryQuota.AddTable(suite.testSpan)
    77  	return memoryQuota
    78  }
    79  
    80  func (suite *tableSinkAdvancerSuite) TestNeedEmitAndAdvance() {
    81  	for _, tc := range []struct {
    82  		name             string
    83  		splitTxn         bool
    84  		committedTxnSize uint64
    85  		pendingTxnSize   uint64
    86  		expected         bool
    87  	}{
    88  		{
    89  			name:             "split txn and not reach maxUpdateIntervalSize",
    90  			splitTxn:         true,
    91  			committedTxnSize: maxUpdateIntervalSize - 2,
    92  			pendingTxnSize:   1,
    93  			expected:         false,
    94  		},
    95  		{
    96  			name:             "split txn and reach maxUpdateIntervalSize",
    97  			splitTxn:         true,
    98  			committedTxnSize: maxUpdateIntervalSize,
    99  			pendingTxnSize:   1,
   100  			expected:         true,
   101  		},
   102  		{
   103  			name:             "not split txn and not reach maxUpdateIntervalSize",
   104  			splitTxn:         false,
   105  			committedTxnSize: maxUpdateIntervalSize - 1,
   106  			// Do not care about pendingTxnSize
   107  			pendingTxnSize: maxUpdateIntervalSize + 100,
   108  			expected:       false,
   109  		},
   110  		{
   111  			name:             "not split txn and reach maxUpdateIntervalSize",
   112  			splitTxn:         false,
   113  			committedTxnSize: maxUpdateIntervalSize + 100,
   114  			// Do not care about pendingTxnSize
   115  			pendingTxnSize: maxUpdateIntervalSize + 100,
   116  			expected:       true,
   117  		},
   118  	} {
   119  		suite.Run(tc.name, func() {
   120  			require.Equal(suite.T(), tc.expected,
   121  				needEmitAndAdvance(tc.splitTxn, tc.committedTxnSize, tc.pendingTxnSize))
   122  		})
   123  	}
   124  }
   125  
   126  func (suite *tableSinkAdvancerSuite) TestAdvanceTableSinkWithBatchID() {
   127  	task, _ := suite.genSinkTask()
   128  	memoryQuota := suite.genMemQuota(512)
   129  	defer memoryQuota.Close()
   130  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   131  	require.NotNil(suite.T(), advancer)
   132  
   133  	err := advanceTableSinkWithBatchID(task, 2, 256, 1, memoryQuota)
   134  	require.NoError(suite.T(), err)
   135  
   136  	expectedResolvedTs := model.NewResolvedTs(2)
   137  	expectedResolvedTs.Mode = model.BatchResolvedMode
   138  	expectedResolvedTs.BatchID = 1
   139  	checkpointTs := task.tableSink.getCheckpointTs()
   140  	require.Equal(suite.T(), expectedResolvedTs, checkpointTs)
   141  }
   142  
   143  func (suite *tableSinkAdvancerSuite) TestAdvanceTableSink() {
   144  	task, _ := suite.genSinkTask()
   145  	memoryQuota := suite.genMemQuota(512)
   146  	defer memoryQuota.Close()
   147  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   148  	require.NotNil(suite.T(), advancer)
   149  
   150  	err := advanceTableSink(task, 2, 256, memoryQuota)
   151  	require.NoError(suite.T(), err)
   152  
   153  	expectedResolvedTs := model.NewResolvedTs(2)
   154  	checkpointTs := task.tableSink.getCheckpointTs()
   155  	require.Equal(suite.T(), expectedResolvedTs, checkpointTs)
   156  }
   157  
   158  func (suite *tableSinkAdvancerSuite) TestNewTableSinkAdvancer() {
   159  	task, _ := suite.genSinkTask()
   160  	memoryQuota := suite.genMemQuota(512)
   161  	defer memoryQuota.Close()
   162  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   163  	require.NotNil(suite.T(), advancer)
   164  	require.Equal(suite.T(), uint64(512), advancer.availableMem)
   165  }
   166  
   167  func (suite *tableSinkAdvancerSuite) TestHasEnoughMem() {
   168  	memoryQuota := suite.genMemQuota(512)
   169  	defer memoryQuota.Close()
   170  	task, _ := suite.genSinkTask()
   171  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   172  	require.NotNil(suite.T(), advancer)
   173  	require.True(suite.T(), advancer.hasEnoughMem())
   174  	for i := 0; i < 6; i++ {
   175  		// 6 * 256 = 1536 > 1024
   176  		advancer.appendEvents([]*model.RowChangedEvent{{}}, 256)
   177  	}
   178  	require.False(suite.T(), advancer.hasEnoughMem(),
   179  		"hasEnoughMem should return false when usedMem > availableMem")
   180  }
   181  
   182  func (suite *tableSinkAdvancerSuite) TestCleanup() {
   183  	memoryQuota := suite.genMemQuota(512)
   184  	defer memoryQuota.Close()
   185  	task, _ := suite.genSinkTask()
   186  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   187  	require.NotNil(suite.T(), advancer)
   188  	require.Equal(suite.T(), uint64(512), advancer.availableMem)
   189  	require.Equal(suite.T(), uint64(0), advancer.usedMem)
   190  	require.Equal(suite.T(), uint64(512), memoryQuota.GetUsedBytes())
   191  	advancer.cleanup()
   192  	require.Equal(suite.T(), uint64(0), memoryQuota.GetUsedBytes(),
   193  		"memory quota should be released after cleanup")
   194  }
   195  
   196  func (suite *tableSinkAdvancerSuite) TestAppendEvents() {
   197  	memoryQuota := suite.genMemQuota(512)
   198  	defer memoryQuota.Close()
   199  	task, _ := suite.genSinkTask()
   200  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   201  	require.NotNil(suite.T(), advancer)
   202  	require.True(suite.T(), advancer.hasEnoughMem())
   203  	for i := 0; i < 2; i++ {
   204  		advancer.appendEvents([]*model.RowChangedEvent{{}}, 256)
   205  	}
   206  	require.Equal(suite.T(), uint64(512), advancer.pendingTxnSize)
   207  	require.Equal(suite.T(), uint64(512), advancer.usedMem)
   208  	require.False(suite.T(), advancer.hasEnoughMem())
   209  	require.Len(suite.T(), advancer.events, 2)
   210  }
   211  
   212  func (suite *tableSinkAdvancerSuite) TestTryMoveMoveToNextTxn() {
   213  	memoryQuota := suite.genMemQuota(512)
   214  	defer memoryQuota.Close()
   215  	task, _ := suite.genSinkTask()
   216  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 512)
   217  	require.NotNil(suite.T(), advancer)
   218  
   219  	// Initial state.
   220  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   221  	require.Equal(suite.T(), uint64(0), advancer.lastTxnCommitTs)
   222  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   223  	require.Equal(suite.T(), uint64(0), advancer.currTxnCommitTs)
   224  
   225  	// Append 1 event with commit ts 1
   226  	advancer.appendEvents([]*model.RowChangedEvent{
   227  		{CommitTs: 1},
   228  	}, 256)
   229  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   230  	advancer.tryMoveToNextTxn(1)
   231  	require.Equal(suite.T(), uint64(256), advancer.committedTxnSize)
   232  	require.Equal(suite.T(), uint64(0), advancer.lastTxnCommitTs)
   233  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   234  	require.Equal(suite.T(), uint64(1), advancer.currTxnCommitTs)
   235  
   236  	// Append 2 events with commit ts 2
   237  	for i := 0; i < 2; i++ {
   238  		advancer.appendEvents([]*model.RowChangedEvent{
   239  			{CommitTs: 2},
   240  		}, 256)
   241  	}
   242  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   243  	require.Equal(suite.T(), uint64(256), advancer.committedTxnSize)
   244  	require.Equal(suite.T(), uint64(0), advancer.lastTxnCommitTs)
   245  	require.Equal(suite.T(), uint64(512), advancer.pendingTxnSize)
   246  	require.Equal(suite.T(), uint64(1), advancer.currTxnCommitTs)
   247  
   248  	// Try to move to next txn.
   249  	advancer.tryMoveToNextTxn(2)
   250  	require.Equal(suite.T(), uint64(768), advancer.committedTxnSize)
   251  	require.Equal(suite.T(), uint64(1), advancer.lastTxnCommitTs)
   252  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   253  	require.Equal(suite.T(), uint64(2), advancer.currTxnCommitTs)
   254  }
   255  
   256  // Test Scenario:
   257  // When we meet a commit fence, we should flush all the events and advance the
   258  // table sink with the commit ts of the commit fence.
   259  func (suite *tableSinkAdvancerSuite) TestAdvanceTheSameCommitTsEventsWithCommitFence() {
   260  	memoryQuota := suite.genMemQuota(768)
   261  	defer memoryQuota.Close()
   262  	task, sink := suite.genSinkTask()
   263  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   264  	require.NotNil(suite.T(), advancer)
   265  
   266  	// 1. append 1 event with commit ts 1
   267  	advancer.appendEvents([]*model.RowChangedEvent{
   268  		{CommitTs: 1},
   269  	}, 256)
   270  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   271  	advancer.tryMoveToNextTxn(1)
   272  
   273  	// 2. append 2 events with commit ts 2
   274  	for i := 0; i < 2; i++ {
   275  		advancer.appendEvents([]*model.RowChangedEvent{
   276  			{CommitTs: 2},
   277  		}, 256)
   278  	}
   279  	advancer.tryMoveToNextTxn(2)
   280  
   281  	// 3. advance with commit fence
   282  	// Last pos is a commit fence.
   283  	advancer.lastPos = sorter.Position{
   284  		StartTs:  1,
   285  		CommitTs: 2,
   286  	}
   287  	err := advancer.advance(false)
   288  	require.NoError(suite.T(), err)
   289  
   290  	require.Len(suite.T(), sink.GetEvents(), 3)
   291  	sink.AckAllEvents()
   292  	require.Eventually(suite.T(), func() bool {
   293  		checkpointTs := task.tableSink.getCheckpointTs()
   294  		return checkpointTs == model.NewResolvedTs(2)
   295  	}, 5*time.Second, 10*time.Millisecond)
   296  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   297  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   298  }
   299  
   300  // Test Scenario:
   301  // When we do not meet a commit fence, we should flush all the events and advance the
   302  // table sink with the commit ts and batch ID.
   303  func (suite *tableSinkAdvancerSuite) TestAdvanceTheSameCommitTsEventsWithoutCommitFence() {
   304  	memoryQuota := suite.genMemQuota(768)
   305  	defer memoryQuota.Close()
   306  	task, sink := suite.genSinkTask()
   307  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   308  	require.NotNil(suite.T(), advancer)
   309  
   310  	// 1. append 1 event with commit ts 1
   311  	advancer.appendEvents([]*model.RowChangedEvent{
   312  		{CommitTs: 1},
   313  	}, 256)
   314  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   315  	advancer.tryMoveToNextTxn(1)
   316  
   317  	// 2. append 2 events with commit ts 3
   318  	for i := 0; i < 2; i++ {
   319  		advancer.appendEvents([]*model.RowChangedEvent{
   320  			{CommitTs: 3},
   321  		}, 256)
   322  	}
   323  	advancer.tryMoveToNextTxn(3)
   324  
   325  	// 3. advance without commit fence
   326  	// Last pos is **not** a commit fence.
   327  	advancer.lastPos = sorter.Position{
   328  		StartTs:  1,
   329  		CommitTs: 3,
   330  	}
   331  	err := advancer.advance(false)
   332  	require.NoError(suite.T(), err)
   333  
   334  	require.Len(suite.T(), sink.GetEvents(), 3)
   335  	sink.AckAllEvents()
   336  	require.Eventually(suite.T(), func() bool {
   337  		expectedResolvedTs := model.NewResolvedTs(3)
   338  		expectedResolvedTs.Mode = model.BatchResolvedMode
   339  		expectedResolvedTs.BatchID = 1
   340  		checkpointTs := task.tableSink.getCheckpointTs()
   341  		return checkpointTs == expectedResolvedTs
   342  	}, 5*time.Second, 10*time.Millisecond)
   343  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   344  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   345  	require.Equal(suite.T(), uint64(2), batchID.Load(), "batch ID should be increased")
   346  }
   347  
   348  // Test Scenario:
   349  // When we meet a different commit ts event, and we support split txn,
   350  // we should flush all the events and advance the
   351  // table sink with the current commit ts and batch ID.
   352  func (suite *tableSinkAdvancerSuite) TestAdvanceDifferentCommitTsEventsWithSplitTxn() {
   353  	memoryQuota := suite.genMemQuota(768)
   354  	defer memoryQuota.Close()
   355  	task, sink := suite.genSinkTask()
   356  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   357  	require.NotNil(suite.T(), advancer)
   358  
   359  	// 1. append 1 event with commit ts 2
   360  	advancer.appendEvents([]*model.RowChangedEvent{
   361  		{CommitTs: 2},
   362  	}, 256)
   363  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   364  	advancer.tryMoveToNextTxn(2)
   365  
   366  	// 2. meet a txn finished event
   367  	advancer.lastPos = sorter.Position{
   368  		StartTs:  1,
   369  		CommitTs: 2,
   370  	}
   371  
   372  	// 3. append 2 events with commit ts 3
   373  	for i := 0; i < 2; i++ {
   374  		advancer.appendEvents([]*model.RowChangedEvent{
   375  			{CommitTs: 3},
   376  		}, 256)
   377  	}
   378  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   379  	advancer.tryMoveToNextTxn(3)
   380  
   381  	// 4. advance without commit fence and with split txn
   382  	err := advancer.advance(false)
   383  	require.NoError(suite.T(), err)
   384  
   385  	require.Len(suite.T(), sink.GetEvents(), 3)
   386  	sink.AckAllEvents()
   387  	require.Eventually(suite.T(), func() bool {
   388  		expectedResolvedTs := model.NewResolvedTs(3)
   389  		expectedResolvedTs.Mode = model.BatchResolvedMode
   390  		expectedResolvedTs.BatchID = 1
   391  		checkpointTs := task.tableSink.getCheckpointTs()
   392  		return checkpointTs == expectedResolvedTs
   393  	}, 5*time.Second, 10*time.Millisecond)
   394  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   395  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   396  	require.Equal(suite.T(), uint64(2), batchID.Load(), "batch ID should be increased")
   397  }
   398  
   399  // Test Scenario:
   400  // When we meet a different commit ts event, and we do **not** support split txn,
   401  // we should flush all the events and advance the
   402  // table sink with the current commit of the last event.
   403  func (suite *tableSinkAdvancerSuite) TestAdvanceDifferentCommitTsEventsWithoutSplitTxn() {
   404  	memoryQuota := suite.genMemQuota(768)
   405  	defer memoryQuota.Close()
   406  	task, sink := suite.genSinkTask()
   407  	// Do not split txn.
   408  	advancer := newTableSinkAdvancer(task, false, memoryQuota, 768)
   409  	require.NotNil(suite.T(), advancer)
   410  
   411  	// 1. append 1 event with commit ts 2
   412  	advancer.appendEvents([]*model.RowChangedEvent{
   413  		{CommitTs: 2},
   414  	}, 256)
   415  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   416  	advancer.tryMoveToNextTxn(2)
   417  
   418  	// 2. meet a txn finished event
   419  	advancer.lastPos = sorter.Position{
   420  		StartTs:  1,
   421  		CommitTs: 2,
   422  	}
   423  
   424  	// 3. append 1 event with commit ts 3
   425  	advancer.appendEvents([]*model.RowChangedEvent{
   426  		{CommitTs: 3},
   427  	}, 256)
   428  	require.Equal(suite.T(), uint64(512), advancer.usedMem)
   429  	advancer.tryMoveToNextTxn(3)
   430  
   431  	// 4. append 1 event with commit ts 3
   432  	advancer.appendEvents([]*model.RowChangedEvent{
   433  		{CommitTs: 3},
   434  	}, 256)
   435  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   436  	advancer.tryMoveToNextTxn(3)
   437  
   438  	// 5. advance without commit fence and split txn
   439  	err := advancer.advance(false)
   440  	require.NoError(suite.T(), err)
   441  
   442  	require.Len(suite.T(), sink.GetEvents(), 1)
   443  	sink.AckAllEvents()
   444  	require.Eventually(suite.T(), func() bool {
   445  		expectedResolvedTs := model.NewResolvedTs(2)
   446  		checkpointTs := task.tableSink.getCheckpointTs()
   447  		return checkpointTs == expectedResolvedTs
   448  	}, 5*time.Second, 10*time.Millisecond)
   449  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   450  	require.Equal(suite.T(), uint64(256), advancer.pendingTxnSize)
   451  	require.Equal(suite.T(), uint64(1), batchID.Load(), "batch ID should not be increased")
   452  }
   453  
   454  // Test Scenario:
   455  // When we meet a different commit ts event, and we do **not** support split txn,
   456  // we should flush all the events and advance the
   457  // table sink with the current commit of the last event. Also we should clear the
   458  // pending txn size.
   459  func (suite *tableSinkAdvancerSuite) TestLastTimeAdvanceDifferentCommitTsEventsWithoutSplitTxn() {
   460  	memoryQuota := suite.genMemQuota(768)
   461  	defer memoryQuota.Close()
   462  	task, sink := suite.genSinkTask()
   463  	// Do not split txn.
   464  	advancer := newTableSinkAdvancer(task, false, memoryQuota, 768)
   465  	require.NotNil(suite.T(), advancer)
   466  
   467  	// 1. append 1 event with commit ts 2
   468  	advancer.appendEvents([]*model.RowChangedEvent{
   469  		{CommitTs: 2},
   470  	}, 256)
   471  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   472  	advancer.tryMoveToNextTxn(2)
   473  
   474  	// 2. meet a txn finished event
   475  	advancer.lastPos = sorter.Position{
   476  		StartTs:  1,
   477  		CommitTs: 2,
   478  	}
   479  
   480  	// 3. append 1 event with commit ts 3
   481  	advancer.appendEvents([]*model.RowChangedEvent{
   482  		{CommitTs: 3},
   483  	}, 256)
   484  	require.Equal(suite.T(), uint64(512), advancer.usedMem)
   485  	advancer.tryMoveToNextTxn(3)
   486  
   487  	// 4. append 1 event with commit ts 3
   488  	advancer.appendEvents([]*model.RowChangedEvent{
   489  		{CommitTs: 3},
   490  	}, 256)
   491  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   492  	advancer.tryMoveToNextTxn(3)
   493  
   494  	// 5. advance without commit fence and split txn
   495  	err := advancer.lastTimeAdvance()
   496  	require.NoError(suite.T(), err)
   497  
   498  	require.Len(suite.T(), sink.GetEvents(), 1)
   499  	sink.AckAllEvents()
   500  	require.Eventually(suite.T(), func() bool {
   501  		expectedResolvedTs := model.NewResolvedTs(2)
   502  		checkpointTs := task.tableSink.getCheckpointTs()
   503  		return checkpointTs == expectedResolvedTs
   504  	}, 5*time.Second, 10*time.Millisecond)
   505  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   506  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize,
   507  		"Last time advance should clear pending txn size,"+
   508  			"otherwise the memory quota will be leaked.")
   509  	require.Equal(suite.T(), uint64(1), batchID.Load())
   510  }
   511  
   512  // Test Scenario:
   513  // We receive some events and exceed the available memory quota.
   514  // We should advance the table sink and also make up the difference
   515  // between the available memory quota and the used memory quota.
   516  func (suite *tableSinkAdvancerSuite) TestTryAdvanceWhenExceedAvailableMem() {
   517  	memoryQuota := suite.genMemQuota(768)
   518  	defer memoryQuota.Close()
   519  	task, sink := suite.genSinkTask()
   520  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   521  	require.NotNil(suite.T(), advancer)
   522  
   523  	// 1. append 1 event with commit ts 2
   524  	advancer.appendEvents([]*model.RowChangedEvent{
   525  		{CommitTs: 2},
   526  	}, 256)
   527  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   528  	advancer.tryMoveToNextTxn(2)
   529  
   530  	// 2. append 3 events with commit ts 3
   531  	for i := 0; i < 3; i++ {
   532  		advancer.appendEvents([]*model.RowChangedEvent{
   533  			{CommitTs: 3},
   534  		}, 256)
   535  	}
   536  	require.Equal(suite.T(), uint64(1024), advancer.usedMem)
   537  	advancer.tryMoveToNextTxn(3)
   538  
   539  	// 3. Last pos is a commit fence.
   540  	advancer.lastPos = sorter.Position{
   541  		StartTs:  2,
   542  		CommitTs: 3,
   543  	}
   544  
   545  	require.Equal(suite.T(), uint64(768), memoryQuota.GetUsedBytes())
   546  	// 4. Try advance with txn is finished.
   547  	err := advancer.tryAdvanceAndAcquireMem(
   548  		false,
   549  		true,
   550  	)
   551  	require.NoError(suite.T(), err)
   552  	require.Equal(suite.T(), uint64(1024), memoryQuota.GetUsedBytes(),
   553  		"Memory quota should be force acquired when exceed available memory.",
   554  	)
   555  
   556  	require.Len(suite.T(), sink.GetEvents(), 4)
   557  	sink.AckAllEvents()
   558  	require.Eventually(suite.T(), func() bool {
   559  		expectedResolvedTs := model.NewResolvedTs(3)
   560  		checkpointTs := task.tableSink.getCheckpointTs()
   561  		return checkpointTs == expectedResolvedTs
   562  	}, 5*time.Second, 10*time.Millisecond)
   563  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   564  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   565  	require.Equal(suite.T(), uint64(1), batchID.Load())
   566  }
   567  
   568  // Test Scenario:
   569  // We receive some events and reach the max update interval size.
   570  // We should advance the table sink.
   571  func (suite *tableSinkAdvancerSuite) TestTryAdvanceWhenReachTheMaxUpdateIntSizeAndTxnNotFinished() {
   572  	memoryQuota := suite.genMemQuota(768)
   573  	defer memoryQuota.Close()
   574  	task, sink := suite.genSinkTask()
   575  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   576  	require.NotNil(suite.T(), advancer)
   577  
   578  	// 1. append 1 event with commit ts 2
   579  	advancer.appendEvents([]*model.RowChangedEvent{
   580  		{CommitTs: 2},
   581  	}, 256)
   582  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   583  	advancer.tryMoveToNextTxn(2)
   584  
   585  	// 2. append 2 events with commit ts 3
   586  	for i := 0; i < 2; i++ {
   587  		advancer.appendEvents([]*model.RowChangedEvent{
   588  			{CommitTs: 3},
   589  		}, 256)
   590  	}
   591  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   592  	advancer.tryMoveToNextTxn(3)
   593  
   594  	// 3. Last pos is a commit fence.
   595  	advancer.lastPos = sorter.Position{
   596  		StartTs:  2,
   597  		CommitTs: 3,
   598  	}
   599  
   600  	// 4. Try advance with txn is not finished.
   601  	err := advancer.tryAdvanceAndAcquireMem(
   602  		false,
   603  		false,
   604  	)
   605  	require.NoError(suite.T(), err)
   606  	require.Len(suite.T(), sink.GetEvents(), 3)
   607  	sink.AckAllEvents()
   608  	require.Eventually(suite.T(), func() bool {
   609  		expectedResolvedTs := model.NewResolvedTs(3)
   610  		checkpointTs := task.tableSink.getCheckpointTs()
   611  		return checkpointTs == expectedResolvedTs
   612  	}, 5*time.Second, 10*time.Millisecond)
   613  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   614  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   615  	require.Equal(suite.T(), uint64(1), batchID.Load())
   616  }
   617  
   618  // Test Scenario:
   619  // We receive some events and the task is finished.
   620  // We should advance the table sink.
   621  func (suite *tableSinkAdvancerSuite) TestFinish() {
   622  	memoryQuota := suite.genMemQuota(768)
   623  	defer memoryQuota.Close()
   624  	task, sink := suite.genSinkTask()
   625  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   626  	require.NotNil(suite.T(), advancer)
   627  
   628  	// 1. append 1 event with commit ts 2
   629  	advancer.appendEvents([]*model.RowChangedEvent{
   630  		{CommitTs: 2},
   631  	}, 256)
   632  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   633  	advancer.tryMoveToNextTxn(2)
   634  
   635  	// 2. append 2 events with commit ts 3
   636  	for i := 0; i < 2; i++ {
   637  		advancer.appendEvents([]*model.RowChangedEvent{
   638  			{CommitTs: 3},
   639  		}, 256)
   640  	}
   641  	require.Equal(suite.T(), uint64(768), advancer.usedMem)
   642  	advancer.tryMoveToNextTxn(3)
   643  
   644  	require.Equal(suite.T(), uint64(2), advancer.lastTxnCommitTs)
   645  	require.Equal(suite.T(), uint64(3), advancer.currTxnCommitTs)
   646  	// 3. Try finish.
   647  	finishedPos := sorter.Position{
   648  		StartTs:  3,
   649  		CommitTs: 4,
   650  	}
   651  	err := advancer.finish(finishedPos)
   652  	require.NoError(suite.T(), err)
   653  
   654  	// All events should be flushed and the last pos should be updated.
   655  	require.Equal(suite.T(), finishedPos, advancer.lastPos)
   656  	require.Equal(suite.T(), uint64(4), advancer.lastTxnCommitTs)
   657  	require.Equal(suite.T(), uint64(4), advancer.currTxnCommitTs)
   658  
   659  	require.Len(suite.T(), sink.GetEvents(), 3)
   660  	sink.AckAllEvents()
   661  	require.Eventually(suite.T(), func() bool {
   662  		expectedResolvedTs := model.NewResolvedTs(4)
   663  		checkpointTs := task.tableSink.getCheckpointTs()
   664  		return checkpointTs == expectedResolvedTs
   665  	}, 5*time.Second, 10*time.Millisecond)
   666  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   667  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   668  	require.Equal(suite.T(), uint64(1), batchID.Load())
   669  }
   670  
   671  // Test Scenario:
   672  // We receive some events and do not support split txn.
   673  // We should advance the table sink and force acquire memory for next txn.
   674  func (suite *tableSinkAdvancerSuite) TestTryAdvanceAndForceAcquireWithoutSplitTxn() {
   675  	memoryQuota := suite.genMemQuota(768)
   676  	defer memoryQuota.Close()
   677  	task, sink := suite.genSinkTask()
   678  	advancer := newTableSinkAdvancer(task, false, memoryQuota, 768)
   679  	require.NotNil(suite.T(), advancer)
   680  
   681  	// 1. append 1 event with commit ts 2
   682  	advancer.appendEvents([]*model.RowChangedEvent{
   683  		{CommitTs: 2},
   684  	}, 256)
   685  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   686  	advancer.tryMoveToNextTxn(2)
   687  
   688  	// 2. append 3 events with commit ts 3, this will exceed the memory quota.
   689  	for i := 0; i < 3; i++ {
   690  		advancer.appendEvents([]*model.RowChangedEvent{
   691  			{CommitTs: 3},
   692  		}, 256)
   693  	}
   694  	require.Equal(suite.T(), uint64(1024), advancer.usedMem)
   695  	advancer.tryMoveToNextTxn(3)
   696  
   697  	// 3. Last pos is a commit fence.
   698  	advancer.lastPos = sorter.Position{
   699  		StartTs:  2,
   700  		CommitTs: 3,
   701  	}
   702  
   703  	// 4. Try advance.
   704  	err := advancer.tryAdvanceAndAcquireMem(
   705  		false,
   706  		false,
   707  	)
   708  	require.NoError(suite.T(), err)
   709  	require.Len(suite.T(), sink.GetEvents(), 4)
   710  	sink.AckAllEvents()
   711  	require.Eventually(suite.T(), func() bool {
   712  		expectedResolvedTs := model.NewResolvedTs(3)
   713  		checkpointTs := task.tableSink.getCheckpointTs()
   714  		return checkpointTs == expectedResolvedTs
   715  	}, 5*time.Second, 10*time.Millisecond)
   716  	require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   717  	require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   718  	require.Equal(suite.T(), uint64(1), batchID.Load())
   719  }
   720  
   721  // Test Scenario:
   722  // We receive some events and support split txn.
   723  // We should advance the table sink and block acquire memory for next txn.
   724  func (suite *tableSinkAdvancerSuite) TestTryAdvanceAndBlockAcquireWithSplitTxn() {
   725  	memoryQuota := suite.genMemQuota(768)
   726  	defer memoryQuota.Close()
   727  	task, sink := suite.genSinkTask()
   728  	advancer := newTableSinkAdvancer(task, true, memoryQuota, 768)
   729  	require.NotNil(suite.T(), advancer)
   730  
   731  	// 1. append 1 event with commit ts 2
   732  	advancer.appendEvents([]*model.RowChangedEvent{
   733  		{CommitTs: 2},
   734  	}, 256)
   735  	require.Equal(suite.T(), uint64(256), advancer.usedMem)
   736  	advancer.tryMoveToNextTxn(2)
   737  
   738  	// 2. append 3 events with commit ts 3, this will exceed the memory quota.
   739  	for i := 0; i < 3; i++ {
   740  		advancer.appendEvents([]*model.RowChangedEvent{
   741  			{CommitTs: 3},
   742  		}, 256)
   743  	}
   744  	require.Equal(suite.T(), uint64(1024), advancer.usedMem)
   745  	advancer.tryMoveToNextTxn(3)
   746  
   747  	// 3. Last pos is a commit fence.
   748  	advancer.lastPos = sorter.Position{
   749  		StartTs:  2,
   750  		CommitTs: 3,
   751  	}
   752  
   753  	down := make(chan struct{})
   754  	go func() {
   755  		// 4. Try advance and block acquire.
   756  		err := advancer.tryAdvanceAndAcquireMem(
   757  			false,
   758  			false,
   759  		)
   760  		require.ErrorIs(suite.T(), err, context.Canceled)
   761  		down <- struct{}{}
   762  	}()
   763  
   764  	var wg sync.WaitGroup
   765  	wg.Add(1)
   766  	go func() {
   767  		// Wait all events are flushed.
   768  		require.Eventually(suite.T(), func() bool {
   769  			return len(sink.GetEvents()) == 4
   770  		}, 5*time.Second, 10*time.Millisecond)
   771  		sink.AckAllEvents()
   772  		// After ack, abort the blocked acquire.
   773  		memoryQuota.Close()
   774  		// Wait the blocked acquire is aborted, otherwise the test data race.
   775  		<-down
   776  		require.Eventually(suite.T(), func() bool {
   777  			expectedResolvedTs := model.NewResolvedTs(3)
   778  			checkpointTs := task.tableSink.getCheckpointTs()
   779  			return checkpointTs == expectedResolvedTs
   780  		}, 5*time.Second, 10*time.Millisecond)
   781  		require.Equal(suite.T(), uint64(0), advancer.committedTxnSize)
   782  		require.Equal(suite.T(), uint64(0), advancer.pendingTxnSize)
   783  		wg.Done()
   784  	}()
   785  	wg.Wait()
   786  }