github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/puller/memorysorter/entry_sorter_test.go (about) 1 // Copyright 2020 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 memorysorter 15 16 import ( 17 "context" 18 "math/rand" 19 "sort" 20 "sync" 21 "testing" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/tiflow/cdc/model" 25 "github.com/stretchr/testify/require" 26 ) 27 28 func TestEntrySorter(t *testing.T) { 29 t.Parallel() 30 testCases := []struct { 31 input []*model.RawKVEntry 32 resolvedTs uint64 33 expect []*model.RawKVEntry 34 }{ 35 { 36 input: []*model.RawKVEntry{ 37 {CRTs: 1, OpType: model.OpTypePut}, 38 {CRTs: 2, OpType: model.OpTypePut}, 39 {CRTs: 4, OpType: model.OpTypeDelete}, 40 {CRTs: 2, OpType: model.OpTypeDelete}, 41 }, 42 resolvedTs: 0, 43 expect: []*model.RawKVEntry{ 44 {CRTs: 0, OpType: model.OpTypeResolved}, 45 }, 46 }, 47 { 48 input: []*model.RawKVEntry{ 49 {CRTs: 3, OpType: model.OpTypePut}, 50 {CRTs: 2, OpType: model.OpTypePut}, 51 {CRTs: 5, OpType: model.OpTypePut}, 52 }, 53 resolvedTs: 3, 54 expect: []*model.RawKVEntry{ 55 {CRTs: 1, OpType: model.OpTypePut}, 56 {CRTs: 2, OpType: model.OpTypeDelete}, 57 {CRTs: 2, OpType: model.OpTypePut}, 58 {CRTs: 2, OpType: model.OpTypePut}, 59 {CRTs: 3, OpType: model.OpTypePut}, 60 {CRTs: 3, OpType: model.OpTypeResolved}, 61 }, 62 }, 63 { 64 input: []*model.RawKVEntry{}, 65 resolvedTs: 3, 66 expect: []*model.RawKVEntry{{CRTs: 3, OpType: model.OpTypeResolved}}, 67 }, 68 { 69 input: []*model.RawKVEntry{ 70 {CRTs: 7, OpType: model.OpTypePut}, 71 }, 72 resolvedTs: 6, 73 expect: []*model.RawKVEntry{ 74 {CRTs: 4, OpType: model.OpTypeDelete}, 75 {CRTs: 5, OpType: model.OpTypePut}, 76 {CRTs: 6, OpType: model.OpTypeResolved}, 77 }, 78 }, 79 { 80 input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, 81 resolvedTs: 6, 82 expect: []*model.RawKVEntry{ 83 {CRTs: 6, OpType: model.OpTypeResolved}, 84 }, 85 }, 86 { 87 input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, 88 resolvedTs: 8, 89 expect: []*model.RawKVEntry{ 90 {CRTs: 7, OpType: model.OpTypeDelete}, 91 {CRTs: 7, OpType: model.OpTypeDelete}, 92 {CRTs: 7, OpType: model.OpTypePut}, 93 {CRTs: 8, OpType: model.OpTypeResolved}, 94 }, 95 }, 96 { 97 input: []*model.RawKVEntry{}, 98 resolvedTs: 15, 99 expect: []*model.RawKVEntry{ 100 {CRTs: 15, OpType: model.OpTypeResolved}, 101 }, 102 }, 103 } 104 es := NewEntrySorter(model.ChangeFeedID4Test("test", "test-cf")) 105 ctx, cancel := context.WithCancel(context.Background()) 106 var wg sync.WaitGroup 107 wg.Add(1) 108 go func() { 109 defer wg.Done() 110 err := es.Run(ctx) 111 require.Equal(t, context.Canceled, errors.Cause(err)) 112 }() 113 for _, tc := range testCases { 114 for _, entry := range tc.input { 115 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 116 } 117 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, tc.resolvedTs)) 118 for i := 0; i < len(tc.expect); i++ { 119 e := <-es.Output() 120 require.Equal(t, tc.expect[i], e.RawKV) 121 } 122 } 123 cancel() 124 wg.Wait() 125 } 126 127 func TestEntrySorterRandomly(t *testing.T) { 128 t.Parallel() 129 es := NewEntrySorter(model.ChangeFeedID4Test("test", "test-cf")) 130 ctx, cancel := context.WithCancel(context.Background()) 131 132 var wg sync.WaitGroup 133 wg.Add(1) 134 go func() { 135 defer wg.Done() 136 err := es.Run(ctx) 137 require.Equal(t, context.Canceled, errors.Cause(err)) 138 }() 139 140 maxTs := uint64(1000000) 141 wg.Add(1) 142 go func() { 143 defer wg.Done() 144 for resolvedTs := uint64(1); resolvedTs <= maxTs; resolvedTs += 400 { 145 var opType model.OpType 146 if rand.Intn(2) == 0 { 147 opType = model.OpTypePut 148 } else { 149 opType = model.OpTypeDelete 150 } 151 for i := 0; i < 1000; i++ { 152 entry := &model.RawKVEntry{ 153 CRTs: uint64(int64(resolvedTs) + rand.Int63n(int64(maxTs-resolvedTs))), 154 OpType: opType, 155 } 156 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 157 } 158 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, resolvedTs)) 159 } 160 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, maxTs)) 161 }() 162 var lastTs uint64 163 var resolvedTs uint64 164 lastOpType := model.OpTypePut 165 for entry := range es.Output() { 166 require.GreaterOrEqual(t, entry.CRTs, lastTs) 167 require.Greater(t, entry.CRTs, resolvedTs) 168 if lastOpType == model.OpTypePut && entry.RawKV.OpType == model.OpTypeDelete { 169 require.Greater(t, entry.CRTs, lastTs) 170 } 171 lastTs = entry.CRTs 172 lastOpType = entry.RawKV.OpType 173 if entry.IsResolved() { 174 resolvedTs = entry.CRTs 175 } 176 if resolvedTs == maxTs { 177 break 178 } 179 } 180 cancel() 181 wg.Wait() 182 } 183 184 func TestEventLess(t *testing.T) { 185 t.Parallel() 186 testCases := []struct { 187 order int 188 i *model.PolymorphicEvent 189 j *model.PolymorphicEvent 190 expected bool 191 }{ 192 { 193 0, 194 &model.PolymorphicEvent{ 195 CRTs: 1, 196 RawKV: &model.RawKVEntry{ 197 OpType: model.OpTypePut, 198 }, 199 }, 200 &model.PolymorphicEvent{ 201 CRTs: 2, 202 RawKV: &model.RawKVEntry{ 203 OpType: model.OpTypePut, 204 }, 205 }, 206 true, 207 }, 208 { 209 1, 210 &model.PolymorphicEvent{ 211 CRTs: 2, 212 RawKV: &model.RawKVEntry{ 213 OpType: model.OpTypeDelete, 214 }, 215 }, 216 &model.PolymorphicEvent{ 217 CRTs: 2, 218 RawKV: &model.RawKVEntry{ 219 OpType: model.OpTypeDelete, 220 }, 221 }, 222 false, 223 }, 224 { 225 2, 226 &model.PolymorphicEvent{ 227 CRTs: 2, 228 RawKV: &model.RawKVEntry{ 229 OpType: model.OpTypeResolved, 230 }, 231 }, 232 &model.PolymorphicEvent{ 233 CRTs: 2, 234 RawKV: &model.RawKVEntry{ 235 OpType: model.OpTypeResolved, 236 }, 237 }, 238 false, 239 }, 240 { 241 3, 242 &model.PolymorphicEvent{ 243 CRTs: 2, 244 RawKV: &model.RawKVEntry{ 245 OpType: model.OpTypeResolved, 246 }, 247 }, 248 &model.PolymorphicEvent{ 249 CRTs: 2, 250 RawKV: &model.RawKVEntry{ 251 OpType: model.OpTypeDelete, 252 }, 253 }, 254 false, 255 }, 256 { 257 4, 258 &model.PolymorphicEvent{ 259 CRTs: 3, 260 RawKV: &model.RawKVEntry{ 261 OpType: model.OpTypeDelete, 262 }, 263 }, 264 &model.PolymorphicEvent{ 265 CRTs: 2, 266 RawKV: &model.RawKVEntry{ 267 OpType: model.OpTypeResolved, 268 }, 269 }, 270 false, 271 }, 272 } 273 274 for i, tc := range testCases { 275 require.Equal(t, tc.expected, eventLess(tc.i, tc.j), "case %d", i) 276 } 277 } 278 279 func TestMergeEvents(t *testing.T) { 280 t.Parallel() 281 events1 := []*model.PolymorphicEvent{ 282 { 283 CRTs: 1, 284 RawKV: &model.RawKVEntry{ 285 OpType: model.OpTypeDelete, 286 }, 287 }, 288 { 289 CRTs: 2, 290 RawKV: &model.RawKVEntry{ 291 OpType: model.OpTypePut, 292 }, 293 }, 294 { 295 CRTs: 3, 296 RawKV: &model.RawKVEntry{ 297 OpType: model.OpTypePut, 298 }, 299 }, 300 { 301 CRTs: 4, 302 RawKV: &model.RawKVEntry{ 303 OpType: model.OpTypePut, 304 }, 305 }, 306 { 307 CRTs: 5, 308 RawKV: &model.RawKVEntry{ 309 OpType: model.OpTypeDelete, 310 }, 311 }, 312 } 313 events2 := []*model.PolymorphicEvent{ 314 { 315 CRTs: 3, 316 RawKV: &model.RawKVEntry{ 317 OpType: model.OpTypeResolved, 318 }, 319 }, 320 { 321 CRTs: 4, 322 RawKV: &model.RawKVEntry{ 323 OpType: model.OpTypePut, 324 }, 325 }, 326 { 327 CRTs: 4, 328 RawKV: &model.RawKVEntry{ 329 OpType: model.OpTypeResolved, 330 }, 331 }, 332 { 333 CRTs: 7, 334 RawKV: &model.RawKVEntry{ 335 OpType: model.OpTypePut, 336 }, 337 }, 338 { 339 CRTs: 9, 340 RawKV: &model.RawKVEntry{ 341 OpType: model.OpTypeDelete, 342 }, 343 }, 344 } 345 346 var outputResults []*model.PolymorphicEvent 347 output := func(event *model.PolymorphicEvent) { 348 outputResults = append(outputResults, event) 349 } 350 351 expectedResults := append(events1, events2...) 352 sort.Slice(expectedResults, func(i, j int) bool { 353 return eventLess(expectedResults[i], expectedResults[j]) 354 }) 355 356 mergeEvents(events1, events2, output) 357 require.Equal(t, expectedResults, outputResults) 358 } 359 360 func BenchmarkSorter(b *testing.B) { 361 es := NewEntrySorter(model.ChangeFeedID4Test("test", "test-cf")) 362 ctx, cancel := context.WithCancel(context.Background()) 363 var wg sync.WaitGroup 364 wg.Add(1) 365 go func() { 366 defer wg.Done() 367 err := es.Run(ctx) 368 if errors.Cause(err) != context.Canceled { 369 panic(errors.Annotate(err, "unexpected error")) 370 } 371 }() 372 373 maxTs := uint64(10000000) 374 b.ResetTimer() 375 wg.Add(1) 376 go func() { 377 defer wg.Done() 378 for resolvedTs := uint64(1); resolvedTs <= maxTs; resolvedTs += 400 { 379 var opType model.OpType 380 if rand.Intn(2) == 0 { 381 opType = model.OpTypePut 382 } else { 383 opType = model.OpTypeDelete 384 } 385 for i := 0; i < 100000; i++ { 386 entry := &model.RawKVEntry{ 387 CRTs: uint64(int64(resolvedTs) + rand.Int63n(1000)), 388 OpType: opType, 389 } 390 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 391 } 392 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, resolvedTs)) 393 } 394 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, maxTs)) 395 }() 396 var resolvedTs uint64 397 for entry := range es.Output() { 398 if entry.IsResolved() { 399 resolvedTs = entry.CRTs 400 } 401 if resolvedTs == maxTs { 402 break 403 } 404 } 405 cancel() 406 wg.Wait() 407 }