github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/mydump/region_test.go (about) 1 // Copyright 2019 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 mydump_test 15 16 import ( 17 "context" 18 "log" 19 "os" 20 21 "github.com/pingcap/br/pkg/storage" 22 23 . "github.com/pingcap/check" 24 25 "github.com/pingcap/tidb-lightning/lightning/config" 26 . "github.com/pingcap/tidb-lightning/lightning/mydump" 27 "github.com/pingcap/tidb-lightning/lightning/worker" 28 ) 29 30 var _ = Suite(&testMydumpRegionSuite{}) 31 32 type testMydumpRegionSuite struct{} 33 34 func (s *testMydumpRegionSuite) SetUpSuite(c *C) {} 35 func (s *testMydumpRegionSuite) TearDownSuite(c *C) {} 36 37 // var expectedTuplesCount = map[string]int64{ 38 // "i": 1, 39 // "report_case_high_risk": 1, 40 // "tbl_autoid": 10000, 41 // "tbl_multi_index": 10000, 42 // } 43 44 func getFileSize(file string) (int64, error) { 45 fd, err := os.Open(file) 46 if err != nil { 47 return -1, err 48 } 49 defer fd.Close() 50 51 fstat, err := fd.Stat() 52 if err != nil { 53 return -1, err 54 } 55 56 return fstat.Size(), nil 57 } 58 59 /* 60 TODO : test with specified 'regionBlockSize' ... 61 */ 62 func (s *testMydumpRegionSuite) TestTableRegion(c *C) { 63 cfg := newConfigWithSourceDir("./examples") 64 loader, _ := NewMyDumpLoader(context.Background(), cfg) 65 dbMeta := loader.GetDatabases()[0] 66 67 ioWorkers := worker.NewPool(context.Background(), 1, "io") 68 for _, meta := range dbMeta.Tables { 69 regions, err := MakeTableRegions(context.Background(), meta, 1, cfg, ioWorkers, loader.GetStore()) 70 c.Assert(err, IsNil) 71 72 // check - region-size vs file-size 73 var tolFileSize int64 = 0 74 for _, file := range meta.DataFiles { 75 tolFileSize += file.FileMeta.FileSize 76 } 77 var tolRegionSize int64 = 0 78 for _, region := range regions { 79 tolRegionSize += region.Size() 80 } 81 c.Assert(tolRegionSize, Equals, tolFileSize) 82 83 // // check - rows num 84 // var tolRows int64 = 0 85 // for _, region := range regions { 86 // tolRows += region.Rows() 87 // } 88 // c.Assert(tolRows, Equals, expectedTuplesCount[table]) 89 90 // check - range 91 regionNum := len(regions) 92 preReg := regions[0] 93 for i := 1; i < regionNum; i++ { 94 reg := regions[i] 95 if preReg.FileMeta.Path == reg.FileMeta.Path { 96 c.Assert(reg.Offset(), Equals, preReg.Offset()+preReg.Size()) 97 c.Assert(reg.RowIDMin(), Equals, preReg.RowIDMin()+preReg.Rows()) 98 } else { 99 c.Assert(reg.Offset, Equals, 0) 100 c.Assert(reg.RowIDMin(), Equals, 1) 101 } 102 preReg = reg 103 } 104 } 105 } 106 107 func (s *testMydumpRegionSuite) TestAllocateEngineIDs(c *C) { 108 dataFileSizes := make([]float64, 700) 109 for i := range dataFileSizes { 110 dataFileSizes[i] = 1.0 111 } 112 filesRegions := make([]*TableRegion, 0, len(dataFileSizes)) 113 for range dataFileSizes { 114 filesRegions = append(filesRegions, new(TableRegion)) 115 } 116 117 checkEngineSizes := func(what string, expected map[int32]int) { 118 actual := make(map[int32]int) 119 for _, region := range filesRegions { 120 actual[region.EngineID]++ 121 } 122 c.Assert(actual, DeepEquals, expected, Commentf("%s", what)) 123 } 124 125 // Batch size > Total size => Everything in the zero batch. 126 AllocateEngineIDs(filesRegions, dataFileSizes, 1000, 0.5, 1000) 127 checkEngineSizes("no batching", map[int32]int{ 128 0: 700, 129 }) 130 131 // Allocate 3 engines. 132 AllocateEngineIDs(filesRegions, dataFileSizes, 200, 0.5, 1000) 133 checkEngineSizes("batch size = 200", map[int32]int{ 134 0: 170, 135 1: 213, 136 2: 317, 137 }) 138 139 // Allocate 3 engines with an alternative ratio 140 AllocateEngineIDs(filesRegions, dataFileSizes, 200, 0.6, 1000) 141 checkEngineSizes("batch size = 200, ratio = 0.6", map[int32]int{ 142 0: 160, 143 1: 208, 144 2: 332, 145 }) 146 147 // Allocate 5 engines. 148 AllocateEngineIDs(filesRegions, dataFileSizes, 100, 0.5, 1000) 149 checkEngineSizes("batch size = 100", map[int32]int{ 150 0: 93, 151 1: 105, 152 2: 122, 153 3: 153, 154 4: 227, 155 }) 156 157 // Number of engines > table concurrency 158 AllocateEngineIDs(filesRegions, dataFileSizes, 50, 0.5, 4) 159 checkEngineSizes("batch size = 50, limit table conc = 4", map[int32]int{ 160 0: 50, 161 1: 59, 162 2: 73, 163 3: 110, 164 4: 50, 165 5: 50, 166 6: 50, 167 7: 50, 168 8: 50, 169 9: 50, 170 10: 50, 171 11: 50, 172 12: 8, 173 }) 174 175 // Zero ratio = Uniform 176 AllocateEngineIDs(filesRegions, dataFileSizes, 100, 0.0, 1000) 177 checkEngineSizes("batch size = 100, ratio = 0", map[int32]int{ 178 0: 100, 179 1: 100, 180 2: 100, 181 3: 100, 182 4: 100, 183 5: 100, 184 6: 100, 185 }) 186 } 187 188 func (s *testMydumpRegionSuite) TestSplitLargeFile(c *C) { 189 meta := &MDTableMeta{ 190 DB: "csv", 191 Name: "large_csv_file", 192 } 193 cfg := &config.Config{ 194 Mydumper: config.MydumperRuntime{ 195 ReadBlockSize: config.ReadBlockSize, 196 CSV: config.CSVConfig{ 197 Separator: ",", 198 Delimiter: "", 199 Header: true, 200 TrimLastSep: false, 201 NotNull: false, 202 Null: "NULL", 203 BackslashEscape: true, 204 }, 205 StrictFormat: true, 206 Filter: []string{"*.*"}, 207 }, 208 } 209 filePath := "./csv/split_large_file.csv" 210 dataFileInfo, err := os.Stat(filePath) 211 if err != nil { 212 log.Fatal(err) 213 } 214 fileSize := dataFileInfo.Size() 215 fileInfo := FileInfo{FileMeta: SourceFileMeta{Path: filePath, Type: SourceTypeCSV, FileSize: fileSize}} 216 colCnt := int64(3) 217 columns := []string{"a", "b", "c"} 218 for _, tc := range []struct { 219 maxRegionSize config.ByteSize 220 chkCnt int 221 offsets [][]int64 222 }{ 223 {1, 4, [][]int64{{6, 12}, {12, 18}, {18, 24}, {24, 30}}}, 224 {6, 2, [][]int64{{6, 18}, {18, 30}}}, 225 {8, 2, [][]int64{{6, 18}, {18, 30}}}, 226 {12, 2, [][]int64{{6, 24}, {24, 30}}}, 227 {13, 2, [][]int64{{6, 24}, {24, 30}}}, 228 {18, 1, [][]int64{{6, 30}}}, 229 {19, 1, [][]int64{{6, 30}}}, 230 } { 231 cfg.Mydumper.MaxRegionSize = tc.maxRegionSize 232 prevRowIdxMax := int64(0) 233 ioWorker := worker.NewPool(context.Background(), 4, "io") 234 235 store, err := storage.NewLocalStorage(".") 236 c.Assert(err, IsNil) 237 238 _, regions, _, err := SplitLargeFile(context.Background(), meta, cfg, fileInfo, colCnt, prevRowIdxMax, ioWorker, store) 239 c.Assert(err, IsNil) 240 c.Assert(len(regions), Equals, tc.chkCnt) 241 for i := range tc.offsets { 242 c.Assert(regions[i].Chunk.Offset, Equals, tc.offsets[i][0]) 243 c.Assert(regions[i].Chunk.EndOffset, Equals, tc.offsets[i][1]) 244 c.Assert(len(regions[i].Chunk.Columns), Equals, len(columns)) 245 c.Assert(regions[i].Chunk.Columns, DeepEquals, columns) 246 } 247 } 248 }