github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/tablesink/progress_tracker_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 tablesink 15 16 import ( 17 "sync" 18 "testing" 19 "time" 20 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/pkg/spanz" 23 "github.com/stretchr/testify/require" 24 ) 25 26 // Only for test. 27 func (r *progressTracker) pendingResolvedTsEventsCount() int { 28 r.mu.Lock() 29 defer r.mu.Unlock() 30 return len(r.resolvedTsCache) 31 } 32 33 func TestNewProgressTracker(t *testing.T) { 34 t.Parallel() 35 36 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 37 require.Equal( 38 t, 39 uint64(0), 40 tracker.advance().Ts, 41 "init lastMinResolvedTs should be 0", 42 ) 43 } 44 45 func TestAddEvent(t *testing.T) { 46 t.Parallel() 47 48 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 49 tracker.addEvent() 50 tracker.addEvent() 51 tracker.addEvent() 52 require.Equal(t, 3, tracker.trackingCount(), "event should be added") 53 } 54 55 func TestAddResolvedTs(t *testing.T) { 56 t.Parallel() 57 58 // There is no event in the tracker. 59 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 60 tracker.addResolvedTs(model.NewResolvedTs(1)) 61 tracker.addResolvedTs(model.NewResolvedTs(2)) 62 tracker.addResolvedTs(model.NewResolvedTs(3)) 63 require.Equal(t, 0, tracker.trackingCount(), "resolved ts should not be added") 64 require.Equal(t, uint64(3), tracker.advance().Ts, "lastMinResolvedTs should be 3") 65 66 // There is an event in the tracker. 67 tracker = newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 68 tracker.addEvent() 69 tracker.addResolvedTs(model.NewResolvedTs(2)) 70 tracker.addResolvedTs(model.NewResolvedTs(3)) 71 require.Equal(t, 1, tracker.trackingCount(), "resolved ts should be added") 72 require.Equal(t, uint64(0), tracker.advance().Ts, "lastMinResolvedTs should not be updated") 73 } 74 75 func TestRemove(t *testing.T) { 76 t.Parallel() 77 var cb1, cb2, cb4, cb5 func() 78 79 // Only event. 80 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 81 tracker.addEvent() 82 cb2 = tracker.addEvent() 83 tracker.addEvent() 84 cb2() 85 tracker.advance() 86 require.Equal(t, 3, tracker.trackingCount(), "not advanced") 87 88 // Both event and resolved ts. 89 tracker = newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 90 cb1 = tracker.addEvent() 91 cb2 = tracker.addEvent() 92 tracker.addResolvedTs(model.NewResolvedTs(3)) 93 cb4 = tracker.addEvent() 94 cb5 = tracker.addEvent() 95 tracker.addResolvedTs(model.NewResolvedTs(6)) 96 tracker.addResolvedTs(model.NewResolvedTs(7)) 97 tracker.addResolvedTs(model.NewResolvedTs(8)) 98 // Remove one event. 99 cb2() 100 tracker.advance() 101 require.Equal(t, 4, tracker.trackingCount()) 102 require.Equal(t, uint64(0), tracker.advance().Ts, "lastMinResolvedTs should not be updated") 103 // Remove one more event. 104 cb4() 105 tracker.advance() 106 require.Equal(t, 4, tracker.trackingCount()) 107 require.Equal(t, uint64(0), tracker.advance().Ts, "lastMinResolvedTs should not be updated") 108 // Remove one more event. 109 cb1() 110 tracker.advance() 111 require.Equal(t, 1, tracker.trackingCount()) 112 require.Equal(t, uint64(3), tracker.advance().Ts, "lastMinResolvedTs should be advanced") 113 // Remove the last event. 114 cb5() 115 tracker.advance() 116 require.Equal(t, 0, tracker.trackingCount()) 117 require.Equal(t, uint64(8), tracker.advance().Ts, "lastMinResolvedTs should be 8") 118 } 119 120 func TestCloseTracker(t *testing.T) { 121 t.Parallel() 122 123 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 124 cb1 := tracker.addEvent() 125 tracker.addResolvedTs(model.NewResolvedTs(1)) 126 cb2 := tracker.addEvent() 127 tracker.addResolvedTs(model.NewResolvedTs(2)) 128 cb3 := tracker.addEvent() 129 tracker.addResolvedTs(model.NewResolvedTs(3)) 130 require.Equal(t, 3, tracker.trackingCount(), "event should be added") 131 132 var wg sync.WaitGroup 133 wg.Add(1) 134 go func() { 135 tracker.freezeProcess() 136 tracker.waitClosed(make(chan struct{})) 137 wg.Done() 138 }() 139 140 cb1() 141 cb2() 142 cb3() 143 wg.Wait() 144 require.Eventually(t, func() bool { 145 return tracker.trackingCount() == 0 146 }, 3*time.Second, 100*time.Millisecond, "all events should be removed") 147 } 148 149 func TestCloseTrackerCancellable(t *testing.T) { 150 t.Parallel() 151 152 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 153 tracker.addEvent() 154 tracker.addResolvedTs(model.NewResolvedTs(1)) 155 tracker.addEvent() 156 tracker.addResolvedTs(model.NewResolvedTs(2)) 157 tracker.addEvent() 158 tracker.addResolvedTs(model.NewResolvedTs(3)) 159 require.Equal(t, 3, tracker.trackingCount(), "event should be added") 160 161 dead := make(chan struct{}) 162 go func() { 163 time.Sleep(time.Millisecond * 10) 164 close(dead) 165 }() 166 167 var wg sync.WaitGroup 168 wg.Add(1) 169 go func() { 170 tracker.freezeProcess() 171 tracker.waitClosed(dead) 172 wg.Done() 173 }() 174 wg.Wait() 175 } 176 177 func TestTrackerBufferBoundary(t *testing.T) { 178 t.Parallel() 179 180 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), 8) 181 182 cbs := make([]func(), 0) 183 for i := 0; i < 65; i++ { 184 cbs = append(cbs, tracker.addEvent()) 185 } 186 require.Equal(t, 2, len(tracker.pendingEvents)) 187 require.Equal(t, 1, len(tracker.pendingEvents[0])) 188 require.Equal(t, 1, len(tracker.pendingEvents[1])) 189 for i, cb := range cbs { 190 cb() 191 tracker.advance() 192 require.Equal(t, 65-i-1, tracker.trackingCount()) 193 } 194 require.Equal(t, 1, len(tracker.pendingEvents)) 195 196 cbs = nil 197 for i := 65; i < 128; i++ { 198 cbs = append(cbs, tracker.addEvent()) 199 require.Equal(t, 1, len(tracker.pendingEvents)) 200 require.Equal(t, 1, len(tracker.pendingEvents[0])) 201 } 202 for i, cb := range cbs { 203 cb() 204 tracker.advance() 205 require.Equal(t, 63-i-1, tracker.trackingCount()) 206 } 207 require.Equal(t, 0, len(tracker.pendingEvents)) 208 } 209 210 func TestClosedTrackerDoNotAdvanceCheckpointTs(t *testing.T) { 211 t.Parallel() 212 213 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 214 cb1 := tracker.addEvent() 215 tracker.addResolvedTs(model.NewResolvedTs(1)) 216 cb2 := tracker.addEvent() 217 tracker.addResolvedTs(model.NewResolvedTs(2)) 218 cb3 := tracker.addEvent() 219 tracker.addResolvedTs(model.NewResolvedTs(3)) 220 require.Equal(t, 3, tracker.trackingCount(), "event should be added") 221 222 var wg sync.WaitGroup 223 wg.Add(1) 224 freezed := make(chan struct{}) 225 go func() { 226 tracker.freezeProcess() 227 close(freezed) 228 tracker.waitClosed(make(chan struct{})) 229 wg.Done() 230 }() 231 <-freezed 232 currentTs := tracker.advance() 233 cb1() 234 cb2() 235 cb3() 236 wg.Wait() 237 require.Eventually(t, func() bool { 238 return tracker.trackingCount() == 0 239 }, 3*time.Second, 100*time.Millisecond, "all events should be removed") 240 require.Equal(t, currentTs, tracker.advance(), "checkpointTs should not be advanced") 241 } 242 243 func TestOnlyResolvedTsShouldDirectlyAdvanceCheckpointTs(t *testing.T) { 244 t.Parallel() 245 246 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 247 cb1 := tracker.addEvent() 248 tracker.addResolvedTs(model.NewResolvedTs(1)) 249 cb2 := tracker.addEvent() 250 tracker.addResolvedTs(model.NewResolvedTs(2)) 251 tracker.addResolvedTs(model.NewResolvedTs(3)) 252 cb3 := tracker.addEvent() 253 tracker.addResolvedTs(model.NewResolvedTs(4)) 254 tracker.addResolvedTs(model.NewResolvedTs(5)) 255 require.Equal(t, 3, tracker.trackingCount(), "Events should be added") 256 cb1() 257 cb2() 258 tracker.addResolvedTs(model.NewResolvedTs(6)) 259 require.Equal(t, uint64(3), tracker.advance().Ts, "CheckpointTs should be advanced") 260 require.Equal(t, 1, tracker.trackingCount(), "Only one event should be left") 261 require.Equal(t, uint64(3), tracker.advance().Ts, "CheckpointTs still should be 3") 262 cb3() 263 require.Equal(t, uint64(6), tracker.advance().Ts, "CheckpointTs should be advanced") 264 tracker.addResolvedTs(model.NewResolvedTs(7)) 265 tracker.addResolvedTs(model.NewResolvedTs(8)) 266 tracker.addResolvedTs(model.NewResolvedTs(9)) 267 require.Equal(t, 0, tracker.pendingResolvedTsEventsCount(), "ResolvedTsCache should be empty") 268 require.Equal(t, uint64(9), tracker.advance().Ts, "CheckpointTs should be advanced") 269 } 270 271 func TestShouldDirectlyUpdateResolvedTsIfNoMoreEvents(t *testing.T) { 272 t.Parallel() 273 274 tracker := newProgressTracker(spanz.TableIDToComparableSpan(1), defaultBufferSize) 275 cb1 := tracker.addEvent() 276 tracker.addResolvedTs(model.NewResolvedTs(1)) 277 cb2 := tracker.addEvent() 278 tracker.addResolvedTs(model.NewResolvedTs(2)) 279 tracker.addResolvedTs(model.NewResolvedTs(3)) 280 require.Equal(t, 2, tracker.pendingResolvedTsEventsCount(), "ResolvedTsCache should only have 2 events") 281 cb3 := tracker.addEvent() 282 tracker.addResolvedTs(model.NewResolvedTs(4)) 283 tracker.addResolvedTs(model.NewResolvedTs(5)) 284 tracker.addResolvedTs(model.NewResolvedTs(6)) 285 cb1() 286 cb2() 287 require.Equal(t, uint64(3), tracker.advance().Ts, "CheckpointTs should be advanced") 288 require.Equal(t, 1, tracker.pendingResolvedTsEventsCount(), "ResolvedTsCache should only have one event") 289 cb3() 290 require.Equal(t, uint64(6), tracker.advance().Ts, "CheckpointTs should be advanced") 291 require.Equal(t, 0, tracker.pendingResolvedTsEventsCount(), "ResolvedTsCache should be empty") 292 }