github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/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 puller 15 16 import ( 17 "context" 18 "math/rand" 19 "sync" 20 "testing" 21 22 "github.com/pingcap/check" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/pkg/util/testleak" 26 ) 27 28 type mockEntrySorterSuite struct{} 29 30 var _ = check.Suite(&mockEntrySorterSuite{}) 31 32 func TestSuite(t *testing.T) { 33 check.TestingT(t) 34 } 35 36 func (s *mockEntrySorterSuite) TestEntrySorter(c *check.C) { 37 defer testleak.AfterTest(c)() 38 testCases := []struct { 39 input []*model.RawKVEntry 40 resolvedTs uint64 41 expect []*model.RawKVEntry 42 }{ 43 { 44 input: []*model.RawKVEntry{ 45 {CRTs: 1, OpType: model.OpTypePut}, 46 {CRTs: 2, OpType: model.OpTypePut}, 47 {CRTs: 4, OpType: model.OpTypeDelete}, 48 {CRTs: 2, OpType: model.OpTypeDelete}, 49 }, 50 resolvedTs: 0, 51 expect: []*model.RawKVEntry{ 52 {CRTs: 0, OpType: model.OpTypeResolved}, 53 }, 54 }, 55 { 56 input: []*model.RawKVEntry{ 57 {CRTs: 3, OpType: model.OpTypePut}, 58 {CRTs: 2, OpType: model.OpTypePut}, 59 {CRTs: 5, OpType: model.OpTypePut}, 60 }, 61 resolvedTs: 3, 62 expect: []*model.RawKVEntry{ 63 {CRTs: 1, OpType: model.OpTypePut}, 64 {CRTs: 2, OpType: model.OpTypeDelete}, 65 {CRTs: 2, OpType: model.OpTypePut}, 66 {CRTs: 2, OpType: model.OpTypePut}, 67 {CRTs: 3, OpType: model.OpTypePut}, 68 {CRTs: 3, OpType: model.OpTypeResolved}, 69 }, 70 }, 71 { 72 input: []*model.RawKVEntry{}, 73 resolvedTs: 3, 74 expect: []*model.RawKVEntry{{CRTs: 3, OpType: model.OpTypeResolved}}, 75 }, 76 { 77 input: []*model.RawKVEntry{ 78 {CRTs: 7, OpType: model.OpTypePut}, 79 }, 80 resolvedTs: 6, 81 expect: []*model.RawKVEntry{ 82 {CRTs: 4, OpType: model.OpTypeDelete}, 83 {CRTs: 5, OpType: model.OpTypePut}, 84 {CRTs: 6, OpType: model.OpTypeResolved}, 85 }, 86 }, 87 { 88 input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, 89 resolvedTs: 6, 90 expect: []*model.RawKVEntry{ 91 {CRTs: 6, OpType: model.OpTypeResolved}, 92 }, 93 }, 94 { 95 input: []*model.RawKVEntry{{CRTs: 7, OpType: model.OpTypeDelete}}, 96 resolvedTs: 8, 97 expect: []*model.RawKVEntry{ 98 {CRTs: 7, OpType: model.OpTypeDelete}, 99 {CRTs: 7, OpType: model.OpTypeDelete}, 100 {CRTs: 7, OpType: model.OpTypePut}, 101 {CRTs: 8, OpType: model.OpTypeResolved}, 102 }, 103 }, 104 { 105 input: []*model.RawKVEntry{}, 106 resolvedTs: 15, 107 expect: []*model.RawKVEntry{ 108 {CRTs: 15, OpType: model.OpTypeResolved}, 109 }, 110 }, 111 } 112 es := NewEntrySorter() 113 ctx, cancel := context.WithCancel(context.Background()) 114 var wg sync.WaitGroup 115 wg.Add(1) 116 go func() { 117 defer wg.Done() 118 err := es.Run(ctx) 119 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 120 }() 121 for _, tc := range testCases { 122 for _, entry := range tc.input { 123 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 124 } 125 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, tc.resolvedTs)) 126 for i := 0; i < len(tc.expect); i++ { 127 e := <-es.Output() 128 c.Check(e.RawKV, check.DeepEquals, tc.expect[i]) 129 } 130 } 131 cancel() 132 wg.Wait() 133 } 134 135 func (s *mockEntrySorterSuite) TestEntrySorterRandomly(c *check.C) { 136 defer testleak.AfterTest(c)() 137 es := NewEntrySorter() 138 ctx, cancel := context.WithCancel(context.Background()) 139 140 var wg sync.WaitGroup 141 wg.Add(1) 142 go func() { 143 defer wg.Done() 144 err := es.Run(ctx) 145 c.Assert(errors.Cause(err), check.Equals, context.Canceled) 146 }() 147 148 maxTs := uint64(1000000) 149 wg.Add(1) 150 go func() { 151 defer wg.Done() 152 for resolvedTs := uint64(1); resolvedTs <= maxTs; resolvedTs += 400 { 153 var opType model.OpType 154 if rand.Intn(2) == 0 { 155 opType = model.OpTypePut 156 } else { 157 opType = model.OpTypeDelete 158 } 159 for i := 0; i < 1000; i++ { 160 entry := &model.RawKVEntry{ 161 CRTs: uint64(int64(resolvedTs) + rand.Int63n(int64(maxTs-resolvedTs))), 162 OpType: opType, 163 } 164 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 165 } 166 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, resolvedTs)) 167 } 168 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, maxTs)) 169 }() 170 var lastTs uint64 171 var resolvedTs uint64 172 lastOpType := model.OpTypePut 173 for entry := range es.Output() { 174 c.Assert(entry.CRTs, check.GreaterEqual, lastTs) 175 c.Assert(entry.CRTs, check.Greater, resolvedTs) 176 if lastOpType == model.OpTypePut && entry.RawKV.OpType == model.OpTypeDelete { 177 c.Assert(entry.CRTs, check.Greater, lastTs) 178 } 179 lastTs = entry.CRTs 180 lastOpType = entry.RawKV.OpType 181 if entry.RawKV.OpType == model.OpTypeResolved { 182 resolvedTs = entry.CRTs 183 } 184 if resolvedTs == maxTs { 185 break 186 } 187 } 188 cancel() 189 wg.Wait() 190 } 191 192 func BenchmarkSorter(b *testing.B) { 193 es := NewEntrySorter() 194 ctx, cancel := context.WithCancel(context.Background()) 195 var wg sync.WaitGroup 196 wg.Add(1) 197 go func() { 198 defer wg.Done() 199 err := es.Run(ctx) 200 if errors.Cause(err) != context.Canceled { 201 panic(errors.Annotate(err, "unexpected error")) 202 } 203 }() 204 205 maxTs := uint64(10000000) 206 b.ResetTimer() 207 wg.Add(1) 208 go func() { 209 defer wg.Done() 210 for resolvedTs := uint64(1); resolvedTs <= maxTs; resolvedTs += 400 { 211 var opType model.OpType 212 if rand.Intn(2) == 0 { 213 opType = model.OpTypePut 214 } else { 215 opType = model.OpTypeDelete 216 } 217 for i := 0; i < 100000; i++ { 218 entry := &model.RawKVEntry{ 219 CRTs: uint64(int64(resolvedTs) + rand.Int63n(1000)), 220 OpType: opType, 221 } 222 es.AddEntry(ctx, model.NewPolymorphicEvent(entry)) 223 } 224 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, resolvedTs)) 225 } 226 es.AddEntry(ctx, model.NewResolvedPolymorphicEvent(0, maxTs)) 227 }() 228 var resolvedTs uint64 229 for entry := range es.Output() { 230 if entry.RawKV.OpType == model.OpTypeResolved { 231 resolvedTs = entry.CRTs 232 } 233 if resolvedTs == maxTs { 234 break 235 } 236 } 237 cancel() 238 wg.Wait() 239 }