github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/scheduler/internal/v3/keyspan/splitter_region_count_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 keyspan 15 16 import ( 17 "context" 18 "fmt" 19 "testing" 20 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/cdc/processor/tablepb" 23 "github.com/pingcap/tiflow/pkg/config" 24 "github.com/stretchr/testify/require" 25 ) 26 27 func TestRegionCountSplitSpan(t *testing.T) { 28 t.Parallel() 29 30 cache := NewMockRegionCache() 31 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_0"), EndKey: []byte("t1_1")}, 1) 32 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, 2) 33 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, 3) 34 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, 4) 35 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_4"), EndKey: []byte("t2_2")}, 5) 36 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t2_2"), EndKey: []byte("t2_3")}, 6) 37 38 cases := []struct { 39 totalCaptures int 40 span tablepb.Span 41 expectSpans []tablepb.Span 42 }{ 43 { 44 totalCaptures: 7, 45 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 46 expectSpans: []tablepb.Span{ 47 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 48 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 49 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 50 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 51 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 52 }, 53 }, 54 { 55 totalCaptures: 6, 56 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 57 expectSpans: []tablepb.Span{ 58 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 59 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 60 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 61 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 62 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 63 }, 64 }, 65 { 66 totalCaptures: 5, 67 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 68 expectSpans: []tablepb.Span{ 69 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 70 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 71 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 72 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 73 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 74 }, 75 }, 76 { 77 totalCaptures: 4, 78 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 79 expectSpans: []tablepb.Span{ 80 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 81 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 82 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 83 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 84 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 85 }, 86 }, 87 { 88 totalCaptures: 3, 89 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 90 expectSpans: []tablepb.Span{ 91 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 92 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 93 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 94 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 95 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 96 }, 97 }, 98 { 99 totalCaptures: 2, 100 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 101 expectSpans: []tablepb.Span{ 102 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region 103 {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region 104 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region 105 {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region 106 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 107 }, 108 }, 109 { 110 totalCaptures: 1, 111 span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 112 expectSpans: []tablepb.Span{ 113 {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_2")}, // 2 region 114 {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_4")}, // 2 region 115 {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region 116 }, 117 }, 118 } 119 120 for i, cs := range cases { 121 cfg := &config.ChangefeedSchedulerConfig{ 122 EnableTableAcrossNodes: true, 123 RegionThreshold: 1, 124 } 125 splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) 126 spans := splitter.split(context.Background(), cs.span, cs.totalCaptures) 127 require.Equalf(t, cs.expectSpans, spans, "%d %s", i, &cs.span) 128 } 129 } 130 131 func TestRegionCountEvenlySplitSpan(t *testing.T) { 132 t.Parallel() 133 134 cache := NewMockRegionCache() 135 totalRegion := 1000 136 for i := 0; i < totalRegion; i++ { 137 cache.regions.ReplaceOrInsert(tablepb.Span{ 138 StartKey: []byte(fmt.Sprintf("t1_%09d", i)), 139 EndKey: []byte(fmt.Sprintf("t1_%09d", i+1)), 140 }, uint64(i+1)) 141 } 142 143 cases := []struct { 144 totalCaptures int 145 expectedSpans int 146 expectSpansMin int 147 expectSpansMax int 148 }{ 149 { 150 totalCaptures: 0, 151 expectedSpans: 1, 152 expectSpansMin: 1000, 153 expectSpansMax: 1000, 154 }, 155 { 156 totalCaptures: 1, 157 expectedSpans: 3, 158 expectSpansMin: 333, 159 expectSpansMax: 334, 160 }, 161 { 162 totalCaptures: 3, 163 expectedSpans: 9, 164 expectSpansMin: 111, 165 expectSpansMax: 113, 166 }, 167 { 168 totalCaptures: 7, 169 expectedSpans: 42, 170 expectSpansMin: 23, 171 expectSpansMax: 24, 172 }, 173 { 174 totalCaptures: 999, 175 expectedSpans: 100, 176 expectSpansMin: 1, 177 expectSpansMax: 10, 178 }, 179 { 180 totalCaptures: 1000, 181 expectedSpans: 100, 182 expectSpansMin: 1, 183 expectSpansMax: 10, 184 }, 185 { 186 totalCaptures: 2000, 187 expectedSpans: 100, 188 expectSpansMin: 1, 189 expectSpansMax: 10, 190 }, 191 } 192 for i, cs := range cases { 193 cfg := &config.ChangefeedSchedulerConfig{ 194 EnableTableAcrossNodes: true, 195 RegionThreshold: 1, 196 } 197 splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) 198 spans := splitter.split( 199 context.Background(), 200 tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, 201 cs.totalCaptures, 202 ) 203 204 require.Equalf(t, cs.expectedSpans, len(spans), "%d %v", i, cs) 205 206 for _, span := range spans { 207 start, end := 0, 1000 208 if len(span.StartKey) > len("t1") { 209 _, err := fmt.Sscanf(string(span.StartKey), "t1_%d", &start) 210 require.Nil(t, err, "%d %v %s", i, cs, span.StartKey) 211 } 212 if len(span.EndKey) > len("t2") { 213 _, err := fmt.Sscanf(string(span.EndKey), "t1_%d", &end) 214 require.Nil(t, err, "%d %v %s", i, cs, span.EndKey) 215 } 216 require.GreaterOrEqual(t, end-start, cs.expectSpansMin, "%d %v", i, cs) 217 require.LessOrEqual(t, end-start, cs.expectSpansMax, "%d %v", i, cs) 218 } 219 } 220 } 221 222 func TestSplitSpanRegionOutOfOrder(t *testing.T) { 223 t.Parallel() 224 225 cache := NewMockRegionCache() 226 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_0"), EndKey: []byte("t1_1")}, 1) 227 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_1"), EndKey: []byte("t1_4")}, 2) 228 cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, 3) 229 230 cfg := &config.ChangefeedSchedulerConfig{ 231 EnableTableAcrossNodes: true, 232 RegionThreshold: 1, 233 } 234 splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) 235 span := tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")} 236 spans := splitter.split(context.Background(), span, 1) 237 require.Equal( 238 t, []tablepb.Span{{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}}, spans) 239 }