github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/mydump/region.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 15 16 import ( 17 "context" 18 "math" 19 "sync" 20 "time" 21 22 "github.com/pingcap/br/pkg/utils" 23 24 "github.com/pingcap/br/pkg/storage" 25 "github.com/pingcap/errors" 26 "go.uber.org/zap" 27 28 "github.com/pingcap/tidb-lightning/lightning/config" 29 "github.com/pingcap/tidb-lightning/lightning/log" 30 "github.com/pingcap/tidb-lightning/lightning/worker" 31 ) 32 33 const tableRegionSizeWarningThreshold int64 = 1024 * 1024 * 1024 34 35 type TableRegion struct { 36 EngineID int32 37 38 DB string 39 Table string 40 FileMeta SourceFileMeta 41 42 Chunk Chunk 43 } 44 45 func (reg *TableRegion) RowIDMin() int64 { 46 return reg.Chunk.PrevRowIDMax + 1 47 } 48 func (reg *TableRegion) Rows() int64 { 49 return reg.Chunk.RowIDMax - reg.Chunk.PrevRowIDMax 50 } 51 func (reg *TableRegion) Offset() int64 { 52 return reg.Chunk.Offset 53 } 54 func (reg *TableRegion) Size() int64 { 55 return reg.Chunk.EndOffset - reg.Chunk.Offset 56 } 57 58 //////////////////////////////////////////////////////////////// 59 60 func AllocateEngineIDs( 61 filesRegions []*TableRegion, 62 dataFileSizes []float64, 63 batchSize float64, 64 batchImportRatio float64, 65 tableConcurrency float64, 66 ) { 67 totalDataFileSize := 0.0 68 for _, dataFileSize := range dataFileSizes { 69 totalDataFileSize += dataFileSize 70 } 71 72 // No need to batch if the size is too small :) 73 if totalDataFileSize <= batchSize { 74 return 75 } 76 77 curEngineID := int32(0) 78 curEngineSize := 0.0 79 curBatchSize := batchSize 80 81 // import() step will not be concurrent. 82 // If multiple Batch end times are close, it will result in multiple 83 // Batch import serials. We need use a non-uniform batch size to create a pipeline effect. 84 // Here we calculate the total number of engines, which is needed to compute the scale up 85 // 86 // Total/B1 = 1/(1-R) * (N - 1/beta(N, R)) 87 // ≲ N/(1-R) 88 // 89 // We use a simple brute force search since the search space is extremely small. 90 ratio := totalDataFileSize * (1 - batchImportRatio) / batchSize 91 n := math.Ceil(ratio) 92 logGammaNPlusR, _ := math.Lgamma(n + batchImportRatio) 93 logGammaN, _ := math.Lgamma(n) 94 logGammaR, _ := math.Lgamma(batchImportRatio) 95 invBetaNR := math.Exp(logGammaNPlusR - logGammaN - logGammaR) // 1/B(N, R) = Γ(N+R)/Γ(N)Γ(R) 96 for { 97 if n <= 0 || n > tableConcurrency { 98 n = tableConcurrency 99 break 100 } 101 realRatio := n - invBetaNR 102 if realRatio >= ratio { 103 // we don't have enough engines. reduce the batch size to keep the pipeline smooth. 104 curBatchSize = totalDataFileSize * (1 - batchImportRatio) / realRatio 105 break 106 } 107 invBetaNR *= 1 + batchImportRatio/n // Γ(X+1) = X * Γ(X) 108 n += 1.0 109 } 110 111 for i, dataFileSize := range dataFileSizes { 112 filesRegions[i].EngineID = curEngineID 113 curEngineSize += dataFileSize 114 115 if curEngineSize >= curBatchSize { 116 curEngineSize = 0 117 curEngineID++ 118 119 i := float64(curEngineID) 120 // calculate the non-uniform batch size 121 if i >= n { 122 curBatchSize = batchSize 123 } else { 124 // B_(i+1) = B_i * (I/W/(N-i) + 1) 125 curBatchSize *= batchImportRatio/(n-i) + 1.0 126 } 127 } 128 } 129 } 130 131 func MakeTableRegions( 132 ctx context.Context, 133 meta *MDTableMeta, 134 columns int, 135 cfg *config.Config, 136 ioWorkers *worker.Pool, 137 store storage.ExternalStorage, 138 ) ([]*TableRegion, error) { 139 // Split files into regions 140 type fileRegionRes struct { 141 info FileInfo 142 regions []*TableRegion 143 sizes []float64 144 err error 145 } 146 147 start := time.Now() 148 149 execCtx, cancel := context.WithCancel(ctx) 150 defer cancel() 151 152 concurrency := utils.MaxInt(cfg.App.RegionConcurrency, 2) 153 fileChan := make(chan FileInfo, concurrency) 154 resultChan := make(chan fileRegionRes, concurrency) 155 var wg sync.WaitGroup 156 for i := 0; i < concurrency; i++ { 157 wg.Add(1) 158 go func() { 159 for info := range fileChan { 160 regions, sizes, err := makeSourceFileRegion(execCtx, meta, info, columns, cfg, ioWorkers, store) 161 select { 162 case resultChan <- fileRegionRes{info: info, regions: regions, sizes: sizes, err: err}: 163 case <-ctx.Done(): 164 break 165 } 166 if err != nil { 167 break 168 } 169 } 170 wg.Done() 171 }() 172 } 173 174 go func() { 175 wg.Wait() 176 close(resultChan) 177 }() 178 179 errChan := make(chan error, 1) 180 fileRegionsMap := make(map[string]fileRegionRes, len(meta.DataFiles)) 181 go func() { 182 for res := range resultChan { 183 if res.err != nil { 184 errChan <- res.err 185 return 186 } 187 fileRegionsMap[res.info.FileMeta.Path] = res 188 } 189 errChan <- nil 190 }() 191 192 for _, dataFile := range meta.DataFiles { 193 select { 194 case fileChan <- dataFile: 195 case <-ctx.Done(): 196 close(fileChan) 197 return nil, ctx.Err() 198 case err := <-errChan: 199 return nil, err 200 } 201 } 202 close(fileChan) 203 err := <-errChan 204 if err != nil { 205 return nil, err 206 } 207 208 filesRegions := make([]*TableRegion, 0, len(meta.DataFiles)) 209 dataFileSizes := make([]float64, 0, len(meta.DataFiles)) 210 prevRowIDMax := int64(0) 211 for _, dataFile := range meta.DataFiles { 212 fileRegionsRes := fileRegionsMap[dataFile.FileMeta.Path] 213 var delta int64 214 if len(fileRegionsRes.regions) > 0 { 215 delta = prevRowIDMax - fileRegionsRes.regions[0].Chunk.PrevRowIDMax 216 } 217 218 for _, region := range fileRegionsRes.regions { 219 region.Chunk.PrevRowIDMax += delta 220 region.Chunk.RowIDMax += delta 221 } 222 filesRegions = append(filesRegions, fileRegionsRes.regions...) 223 dataFileSizes = append(dataFileSizes, fileRegionsRes.sizes...) 224 prevRowIDMax = fileRegionsRes.regions[len(fileRegionsRes.regions)-1].Chunk.RowIDMax 225 } 226 227 log.L().Info("makeTableRegions", zap.Int("filesCount", len(meta.DataFiles)), 228 zap.Int64("maxRegionSize", int64(cfg.Mydumper.MaxRegionSize)), 229 zap.Int("RegionsCount", len(filesRegions)), 230 zap.Duration("cost", time.Since(start))) 231 232 AllocateEngineIDs(filesRegions, dataFileSizes, float64(cfg.Mydumper.BatchSize), cfg.Mydumper.BatchImportRatio, float64(cfg.App.TableConcurrency)) 233 return filesRegions, nil 234 } 235 236 func makeSourceFileRegion( 237 ctx context.Context, 238 meta *MDTableMeta, 239 fi FileInfo, 240 columns int, 241 cfg *config.Config, 242 ioWorkers *worker.Pool, 243 store storage.ExternalStorage, 244 ) ([]*TableRegion, []float64, error) { 245 if fi.FileMeta.Type == SourceTypeParquet { 246 _, region, err := makeParquetFileRegion(ctx, store, meta, fi, 0) 247 if err != nil { 248 return nil, nil, err 249 } 250 return []*TableRegion{region}, []float64{float64(fi.FileMeta.FileSize)}, nil 251 } 252 253 dataFileSize := fi.FileMeta.FileSize 254 divisor := int64(columns) 255 isCsvFile := fi.FileMeta.Type == SourceTypeCSV 256 if !isCsvFile { 257 divisor += 2 258 } 259 // If a csv file is overlarge, we need to split it into multiple regions. 260 // Note: We can only split a csv file whose format is strict. 261 if isCsvFile && dataFileSize > int64(cfg.Mydumper.MaxRegionSize) && cfg.Mydumper.StrictFormat { 262 263 _, regions, subFileSizes, err := SplitLargeFile(ctx, meta, cfg, fi, divisor, 0, ioWorkers, store) 264 return regions, subFileSizes, err 265 } 266 267 tableRegion := &TableRegion{ 268 DB: meta.DB, 269 Table: meta.Name, 270 FileMeta: fi.FileMeta, 271 Chunk: Chunk{ 272 Offset: 0, 273 EndOffset: fi.FileMeta.FileSize, 274 PrevRowIDMax: 0, 275 RowIDMax: fi.FileMeta.FileSize / divisor, 276 }, 277 } 278 279 if tableRegion.Size() > tableRegionSizeWarningThreshold { 280 log.L().Warn( 281 "file is too big to be processed efficiently; we suggest splitting it at 256 MB each", 282 zap.String("file", fi.FileMeta.Path), 283 zap.Int64("size", dataFileSize)) 284 } 285 return []*TableRegion{tableRegion}, []float64{float64(fi.FileMeta.FileSize)}, nil 286 } 287 288 // because parquet files can't seek efficiently, there is no benefit in split. 289 // parquet file are column orient, so the offset is read line number 290 func makeParquetFileRegion( 291 ctx context.Context, 292 store storage.ExternalStorage, 293 meta *MDTableMeta, 294 dataFile FileInfo, 295 prevRowIdxMax int64, 296 ) (int64, *TableRegion, error) { 297 r, err := store.Open(ctx, dataFile.FileMeta.Path) 298 if err != nil { 299 return prevRowIdxMax, nil, errors.Trace(err) 300 } 301 numberRows, err := ReadParquetFileRowCount(ctx, store, r, dataFile.FileMeta.Path) 302 rowIDMax := prevRowIdxMax + numberRows 303 region := &TableRegion{ 304 DB: meta.DB, 305 Table: meta.Name, 306 FileMeta: dataFile.FileMeta, 307 Chunk: Chunk{ 308 Offset: 0, 309 EndOffset: numberRows, 310 PrevRowIDMax: prevRowIdxMax, 311 RowIDMax: rowIDMax, 312 }, 313 } 314 return rowIDMax, region, nil 315 } 316 317 // SplitLargeFile splits a large csv file into multiple regions, the size of 318 // each regions is specified by `config.MaxRegionSize`. 319 // Note: We split the file coarsely, thus the format of csv file is needed to be 320 // strict. 321 // e.g. 322 // - CSV file with header is invalid 323 // - a complete tuple split into multiple lines is invalid 324 func SplitLargeFile( 325 ctx context.Context, 326 meta *MDTableMeta, 327 cfg *config.Config, 328 dataFile FileInfo, 329 divisor int64, 330 prevRowIdxMax int64, 331 ioWorker *worker.Pool, 332 store storage.ExternalStorage, 333 ) (prevRowIdMax int64, regions []*TableRegion, dataFileSizes []float64, err error) { 334 maxRegionSize := int64(cfg.Mydumper.MaxRegionSize) 335 dataFileSizes = make([]float64, 0, dataFile.FileMeta.FileSize/maxRegionSize+1) 336 startOffset, endOffset := int64(0), maxRegionSize 337 var columns []string 338 if cfg.Mydumper.CSV.Header { 339 r, err := store.Open(ctx, dataFile.FileMeta.Path) 340 if err != nil { 341 return 0, nil, nil, err 342 } 343 parser := NewCSVParser(&cfg.Mydumper.CSV, r, int64(cfg.Mydumper.ReadBlockSize), ioWorker, true) 344 if err = parser.ReadColumns(); err != nil { 345 return 0, nil, nil, err 346 } 347 columns = parser.Columns() 348 startOffset, _ = parser.Pos() 349 endOffset = startOffset + maxRegionSize 350 } 351 for { 352 curRowsCnt := (endOffset - startOffset) / divisor 353 rowIDMax := prevRowIdxMax + curRowsCnt 354 if endOffset != dataFile.FileMeta.FileSize { 355 r, err := store.Open(ctx, dataFile.FileMeta.Path) 356 if err != nil { 357 return 0, nil, nil, err 358 } 359 parser := NewCSVParser(&cfg.Mydumper.CSV, r, int64(cfg.Mydumper.ReadBlockSize), ioWorker, false) 360 if err = parser.SetPos(endOffset, prevRowIdMax); err != nil { 361 return 0, nil, nil, err 362 } 363 pos, err := parser.ReadUntilTokNewLine() 364 if err != nil { 365 return 0, nil, nil, err 366 } 367 endOffset = pos 368 parser.Close() 369 } 370 regions = append(regions, 371 &TableRegion{ 372 DB: meta.DB, 373 Table: meta.Name, 374 FileMeta: dataFile.FileMeta, 375 Chunk: Chunk{ 376 Offset: startOffset, 377 EndOffset: endOffset, 378 PrevRowIDMax: prevRowIdxMax, 379 RowIDMax: rowIDMax, 380 Columns: columns, 381 }, 382 }) 383 dataFileSizes = append(dataFileSizes, float64(endOffset-startOffset)) 384 prevRowIdxMax = rowIDMax 385 if endOffset == dataFile.FileMeta.FileSize { 386 break 387 } 388 startOffset = endOffset 389 if endOffset += maxRegionSize; endOffset > dataFile.FileMeta.FileSize { 390 endOffset = dataFile.FileMeta.FileSize 391 } 392 } 393 return prevRowIdxMax, regions, dataFileSizes, nil 394 }