github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/redo_log_worker_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/entry" 23 "github.com/pingcap/tiflow/cdc/model" 24 "github.com/pingcap/tiflow/cdc/processor/memquota" 25 "github.com/pingcap/tiflow/cdc/processor/sourcemanager" 26 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter" 27 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter/memory" 28 "github.com/pingcap/tiflow/cdc/processor/tablepb" 29 "github.com/pingcap/tiflow/pkg/spanz" 30 "github.com/pingcap/tiflow/pkg/upstream" 31 "github.com/stretchr/testify/require" 32 "github.com/stretchr/testify/suite" 33 ) 34 35 type redoLogWorkerSuite struct { 36 suite.Suite 37 testChangefeedID model.ChangeFeedID 38 testSpan tablepb.Span 39 } 40 41 func (suite *redoLogWorkerSuite) SetupSuite() { 42 requestMemSize = uint64(testEventSize) 43 // For one batch size. 44 // Advance table sink per 2 events. 45 maxUpdateIntervalSize = uint64(testEventSize * 2) 46 suite.testChangefeedID = model.DefaultChangeFeedID("1") 47 suite.testSpan = spanz.TableIDToComparableSpan(1) 48 } 49 50 func (suite *redoLogWorkerSuite) TearDownSuite() { 51 requestMemSize = defaultRequestMemSize 52 maxUpdateIntervalSize = defaultMaxUpdateIntervalSize 53 } 54 55 func TestRedoLogWorkerSuite(t *testing.T) { 56 suite.Run(t, new(redoLogWorkerSuite)) 57 } 58 59 //nolint:unparam 60 func (suite *redoLogWorkerSuite) createWorker( 61 ctx context.Context, memQuota uint64, 62 ) (*redoWorker, sorter.SortEngine, *mockRedoDMLManager) { 63 sortEngine := memory.New(context.Background()) 64 // Only sourcemanager.FetcyByTable is used, so NewForTest is fine. 65 sm := sourcemanager.NewForTest(suite.testChangefeedID, upstream.NewUpstream4Test(&MockPD{}), 66 &entry.MockMountGroup{}, sortEngine, false) 67 go func() { _ = sm.Run(ctx) }() 68 69 // To avoid refund or release panics. 70 quota := memquota.NewMemQuota(suite.testChangefeedID, memQuota, "sink") 71 // NOTICE: Do not forget the initial memory quota in the worker first time running. 72 quota.ForceAcquire(uint64(testEventSize)) 73 quota.AddTable(suite.testSpan) 74 redoDMLManager := newMockRedoDMLManager() 75 76 return newRedoWorker(suite.testChangefeedID, sm, quota, 77 redoDMLManager), sortEngine, redoDMLManager 78 } 79 80 func (suite *redoLogWorkerSuite) addEventsToSortEngine( 81 events []*model.PolymorphicEvent, 82 sortEngine sorter.SortEngine, 83 ) { 84 sortEngine.AddTable(suite.testSpan, 0) 85 for _, event := range events { 86 sortEngine.Add(suite.testSpan, event) 87 } 88 } 89 90 func (suite *redoLogWorkerSuite) TestHandleTaskGotSomeFilteredEvents() { 91 ctx, cancel := context.WithCancel(context.Background()) 92 events := []*model.PolymorphicEvent{ 93 genPolymorphicEvent(1, 2, suite.testSpan), 94 // This event will be filtered, so its Row will be nil. 95 genPolymorphicEventWithNilRow(1, 2), 96 genPolymorphicEventWithNilRow(1, 2), 97 genPolymorphicEvent(1, 3, suite.testSpan), 98 genPolymorphicEvent(1, 4, suite.testSpan), 99 genPolymorphicResolvedEvent(4), 100 } 101 102 // Only for three events. 103 eventSize := uint64(testEventSize * 3) 104 w, e, m := suite.createWorker(ctx, eventSize) 105 defer w.memQuota.Close() 106 suite.addEventsToSortEngine(events, e) 107 108 taskChan := make(chan *redoTask) 109 var wg sync.WaitGroup 110 wg.Add(1) 111 go func() { 112 defer wg.Done() 113 err := w.handleTasks(ctx, taskChan) 114 require.Equal(suite.T(), context.Canceled, err) 115 }() 116 117 callback := func(lastWritePos sorter.Position) { 118 require.Equal(suite.T(), sorter.Position{ 119 StartTs: 1, 120 CommitTs: 4, 121 }, lastWritePos) 122 require.Equal(suite.T(), sorter.Position{ 123 StartTs: 2, 124 CommitTs: 4, 125 }, lastWritePos.Next()) 126 cancel() 127 } 128 wrapper, _ := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) 129 taskChan <- &redoTask{ 130 span: suite.testSpan, 131 lowerBound: genLowerBound(), 132 getUpperBound: genUpperBoundGetter(4), 133 tableSink: wrapper, 134 callback: callback, 135 isCanceled: func() bool { return false }, 136 } 137 wg.Wait() 138 require.Len(suite.T(), m.getEvents(suite.testSpan), 3) 139 } 140 141 func (suite *redoLogWorkerSuite) TestHandleTaskAbortWhenNoMemAndOneTxnFinished() { 142 ctx, cancel := context.WithCancel(context.Background()) 143 events := []*model.PolymorphicEvent{ 144 genPolymorphicEvent(1, 2, suite.testSpan), 145 genPolymorphicEvent(1, 2, suite.testSpan), 146 genPolymorphicEvent(1, 3, suite.testSpan), 147 genPolymorphicEvent(2, 4, suite.testSpan), 148 genPolymorphicResolvedEvent(4), 149 } 150 151 // Only for three events. 152 eventSize := uint64(testEventSize * 3) 153 w, e, m := suite.createWorker(ctx, eventSize) 154 defer w.memQuota.Close() 155 suite.addEventsToSortEngine(events, e) 156 157 taskChan := make(chan *redoTask) 158 var wg sync.WaitGroup 159 wg.Add(1) 160 go func() { 161 defer wg.Done() 162 err := w.handleTasks(ctx, taskChan) 163 require.Equal(suite.T(), context.Canceled, err) 164 }() 165 166 callback := func(lastWritePos sorter.Position) { 167 require.Equal(suite.T(), sorter.Position{ 168 StartTs: 1, 169 CommitTs: 3, 170 }, lastWritePos, "we only write 3 events because of the memory quota") 171 require.Equal(suite.T(), sorter.Position{ 172 StartTs: 2, 173 CommitTs: 3, 174 }, lastWritePos.Next()) 175 cancel() 176 } 177 wrapper, _ := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) 178 taskChan <- &redoTask{ 179 span: suite.testSpan, 180 lowerBound: genLowerBound(), 181 getUpperBound: genUpperBoundGetter(4), 182 tableSink: wrapper, 183 callback: callback, 184 isCanceled: func() bool { return false }, 185 } 186 wg.Wait() 187 require.Len(suite.T(), m.getEvents(suite.testSpan), 3) 188 } 189 190 func (suite *redoLogWorkerSuite) TestHandleTaskAbortWhenNoMemAndBlocked() { 191 ctx, cancel := context.WithCancel(context.Background()) 192 events := []*model.PolymorphicEvent{ 193 genPolymorphicEvent(1, 10, suite.testSpan), 194 genPolymorphicEvent(1, 10, suite.testSpan), 195 genPolymorphicEvent(1, 10, suite.testSpan), 196 genPolymorphicEvent(1, 10, suite.testSpan), 197 genPolymorphicResolvedEvent(14), 198 } 199 // Only for three events. 200 eventSize := uint64(testEventSize * 3) 201 w, e, m := suite.createWorker(ctx, eventSize) 202 suite.addEventsToSortEngine(events, e) 203 204 taskChan := make(chan *redoTask) 205 var wg sync.WaitGroup 206 wg.Add(1) 207 go func() { 208 defer wg.Done() 209 err := w.handleTasks(ctx, taskChan) 210 require.ErrorIs(suite.T(), err, context.Canceled) 211 }() 212 213 callback := func(lastWritePos sorter.Position) { 214 require.Equal(suite.T(), sorter.Position{ 215 StartTs: 0, 216 CommitTs: 0, 217 }, lastWritePos) 218 } 219 wrapper, _ := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) 220 taskChan <- &redoTask{ 221 span: suite.testSpan, 222 lowerBound: genLowerBound(), 223 getUpperBound: genUpperBoundGetter(14), 224 tableSink: wrapper, 225 callback: callback, 226 isCanceled: func() bool { return false }, 227 } 228 require.Eventually(suite.T(), func() bool { 229 return len(m.getEvents(suite.testSpan)) == 2 230 }, 5*time.Second, 10*time.Millisecond) 231 // Abort the task when no memory quota and blocked. 232 w.memQuota.Close() 233 cancel() 234 wg.Wait() 235 require.Len(suite.T(), m.getEvents(suite.testSpan), 2, "Only two events should be sent to sink") 236 } 237 238 func (suite *redoLogWorkerSuite) TestHandleTaskWithSplitTxnAndAdvanceIfNoWorkload() { 239 ctx, cancel := context.WithCancel(context.Background()) 240 events := []*model.PolymorphicEvent{ 241 genPolymorphicResolvedEvent(4), 242 } 243 // Only for three events. 244 eventSize := uint64(testEventSize * 3) 245 w, e, m := suite.createWorker(ctx, eventSize) 246 defer w.memQuota.Close() 247 suite.addEventsToSortEngine(events, e) 248 249 taskChan := make(chan *redoTask) 250 var wg sync.WaitGroup 251 wg.Add(1) 252 go func() { 253 defer wg.Done() 254 err := w.handleTasks(ctx, taskChan) 255 require.ErrorIs(suite.T(), err, context.Canceled) 256 }() 257 258 callback := func(lastWritePos sorter.Position) { 259 require.Equal(suite.T(), sorter.Position{ 260 StartTs: 3, 261 CommitTs: 4, 262 }, lastWritePos) 263 require.Equal(suite.T(), sorter.Position{ 264 StartTs: 4, 265 CommitTs: 4, 266 }, lastWritePos.Next()) 267 } 268 wrapper, _ := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) 269 taskChan <- &redoTask{ 270 span: suite.testSpan, 271 lowerBound: genLowerBound(), 272 getUpperBound: genUpperBoundGetter(4), 273 tableSink: wrapper, 274 callback: callback, 275 isCanceled: func() bool { return false }, 276 } 277 require.Eventually(suite.T(), func() bool { 278 return m.GetResolvedTs(suite.testSpan) == 4 279 }, 5*time.Second, 10*time.Millisecond, "Directly advance resolved mark to 4") 280 cancel() 281 wg.Wait() 282 } 283 284 // When starts to handle a task, advancer.lastPos should be set to a correct position. 285 // Otherwise if advancer.lastPos isn't updated during scanning, callback will get an 286 // invalid `advancer.lastPos`. 287 func (suite *redoLogWorkerSuite) TestHandleTaskWithoutMemory() { 288 ctx, cancel := context.WithCancel(context.Background()) 289 events := []*model.PolymorphicEvent{ 290 genPolymorphicEvent(1, 3, suite.testSpan), 291 genPolymorphicResolvedEvent(4), 292 } 293 w, e, _ := suite.createWorker(ctx, 0) 294 defer w.memQuota.Close() 295 suite.addEventsToSortEngine(events, e) 296 297 taskChan := make(chan *redoTask) 298 var wg sync.WaitGroup 299 wg.Add(1) 300 go func() { 301 defer wg.Done() 302 err := w.handleTasks(ctx, taskChan) 303 require.Equal(suite.T(), context.Canceled, err) 304 }() 305 306 wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) 307 defer sink.Close() 308 309 chShouldBeClosed := make(chan struct{}, 1) 310 callback := func(lastWritePos sorter.Position) { 311 require.Equal(suite.T(), genLowerBound().Prev(), lastWritePos) 312 close(chShouldBeClosed) 313 } 314 taskChan <- &redoTask{ 315 span: suite.testSpan, 316 lowerBound: genLowerBound(), 317 getUpperBound: genUpperBoundGetter(4), 318 tableSink: wrapper, 319 callback: callback, 320 isCanceled: func() bool { return true }, 321 } 322 323 <-chShouldBeClosed 324 cancel() 325 wg.Wait() 326 }