@@ -20,6 +20,7 @@ import (
20
20
"fmt"
21
21
"path"
22
22
"strconv"
23
+ "sync"
23
24
"sync/atomic"
24
25
"time"
25
26
@@ -41,6 +42,7 @@ import (
41
42
"github.com/pingcap/tidb/pkg/resourcemanager/pool/workerpool"
42
43
"github.com/pingcap/tidb/pkg/resourcemanager/util"
43
44
"github.com/pingcap/tidb/pkg/sessionctx"
45
+ "github.com/pingcap/tidb/pkg/sessionctx/variable"
44
46
"github.com/pingcap/tidb/pkg/table"
45
47
"github.com/pingcap/tidb/pkg/table/tables"
46
48
"github.com/pingcap/tidb/pkg/tablecodec"
@@ -170,11 +172,17 @@ func NewAddIndexIngestPipeline(
170
172
if err != nil {
171
173
return nil , err
172
174
}
173
- srcChkPool := createChunkPool (copCtx , concurrency , reorgMeta . BatchSize )
175
+ srcChkPool := createChunkPool (copCtx , reorgMeta )
174
176
readerCnt , writerCnt := expectedIngestWorkerCnt (concurrency , avgRowSize )
177
+ rm := reorgMeta
178
+ if rm .IsDistReorg {
179
+ // Currently, only the batch size of local ingest mode can be adjusted
180
+ rm = nil
181
+ }
175
182
176
183
srcOp := NewTableScanTaskSource (ctx , store , tbl , startKey , endKey , cpMgr )
177
- scanOp := NewTableScanOperator (ctx , sessPool , copCtx , srcChkPool , readerCnt , cpMgr , reorgMeta .BatchSize )
184
+ scanOp := NewTableScanOperator (ctx , sessPool , copCtx , srcChkPool , readerCnt , cpMgr ,
185
+ reorgMeta .GetBatchSizeOrDefault (int (variable .GetDDLReorgBatchSize ())), rm )
178
186
ingestOp := NewIndexIngestOperator (ctx , copCtx , backendCtx , sessPool ,
179
187
tbl , indexes , engines , srcChkPool , writerCnt , reorgMeta , cpMgr , rowCntListener )
180
188
sinkOp := newIndexWriteResultSink (ctx , backendCtx , tbl , indexes , cpMgr , rowCntListener )
@@ -221,7 +229,7 @@ func NewWriteIndexToExternalStoragePipeline(
221
229
if err != nil {
222
230
return nil , err
223
231
}
224
- srcChkPool := createChunkPool (copCtx , concurrency , reorgMeta . BatchSize )
232
+ srcChkPool := createChunkPool (copCtx , reorgMeta )
225
233
readerCnt , writerCnt := expectedIngestWorkerCnt (concurrency , avgRowSize )
226
234
227
235
backend , err := storage .ParseBackend (extStoreURI , nil )
@@ -239,7 +247,8 @@ func NewWriteIndexToExternalStoragePipeline(
239
247
})
240
248
241
249
srcOp := NewTableScanTaskSource (ctx , store , tbl , startKey , endKey , nil )
242
- scanOp := NewTableScanOperator (ctx , sessPool , copCtx , srcChkPool , readerCnt , nil , reorgMeta .BatchSize )
250
+ scanOp := NewTableScanOperator (ctx , sessPool , copCtx , srcChkPool , readerCnt , nil ,
251
+ reorgMeta .GetBatchSizeOrDefault (int (variable .GetDDLReorgBatchSize ())), nil )
243
252
writeOp := NewWriteExternalStoreOperator (
244
253
ctx , copCtx , sessPool , jobID , subtaskID ,
245
254
tbl , indexes , extStore , srcChkPool , writerCnt ,
@@ -264,14 +273,13 @@ func NewWriteIndexToExternalStoragePipeline(
264
273
), nil
265
274
}
266
275
267
- func createChunkPool (copCtx copr.CopContext , hintConc , hintBatchSize int ) chan * chunk. Chunk {
268
- poolSize := ingest . CopReadChunkPoolSize ( hintConc )
269
- batchSize := ingest . CopReadBatchSize ( hintBatchSize )
270
- srcChkPool := make ( chan * chunk.Chunk , poolSize )
271
- for i := 0 ; i < poolSize ; i ++ {
272
- srcChkPool <- chunk . NewChunkWithCapacity ( copCtx . GetBase (). FieldTypes , batchSize )
276
+ func createChunkPool (copCtx copr.CopContext , reorgMeta * model. DDLReorgMeta ) * sync. Pool {
277
+ return & sync. Pool {
278
+ New : func () any {
279
+ return chunk .NewChunkWithCapacity ( copCtx . GetBase (). FieldTypes ,
280
+ reorgMeta . GetBatchSizeOrDefault ( int ( variable . GetDDLReorgBatchSize ())))
281
+ },
273
282
}
274
- return srcChkPool
275
283
}
276
284
277
285
// TableScanTask contains the start key and the end key of a region.
@@ -478,10 +486,11 @@ func NewTableScanOperator(
478
486
ctx * OperatorCtx ,
479
487
sessPool opSessPool ,
480
488
copCtx copr.CopContext ,
481
- srcChkPool chan * chunk. Chunk ,
489
+ srcChkPool * sync. Pool ,
482
490
concurrency int ,
483
491
cpMgr * ingest.CheckpointManager ,
484
492
hintBatchSize int ,
493
+ reorgMeta * model.DDLReorgMeta ,
485
494
) * TableScanOperator {
486
495
totalCount := new (atomic.Int64 )
487
496
pool := workerpool .NewWorkerPool (
@@ -498,6 +507,7 @@ func NewTableScanOperator(
498
507
cpMgr : cpMgr ,
499
508
hintBatchSize : hintBatchSize ,
500
509
totalCount : totalCount ,
510
+ reorgMeta : reorgMeta ,
501
511
}
502
512
})
503
513
return & TableScanOperator {
@@ -518,9 +528,10 @@ type tableScanWorker struct {
518
528
copCtx copr.CopContext
519
529
sessPool opSessPool
520
530
se * session.Session
521
- srcChkPool chan * chunk. Chunk
531
+ srcChkPool * sync. Pool
522
532
523
533
cpMgr * ingest.CheckpointManager
534
+ reorgMeta * model.DDLReorgMeta
524
535
hintBatchSize int
525
536
totalCount * atomic.Int64
526
537
}
@@ -588,17 +599,21 @@ func (w *tableScanWorker) scanRecords(task TableScanTask, sender func(IndexRecor
588
599
}
589
600
590
601
func (w * tableScanWorker ) getChunk () * chunk.Chunk {
591
- chk := <- w .srcChkPool
592
- newCap := ingest .CopReadBatchSize (w .hintBatchSize )
593
- if chk .Capacity () != newCap {
594
- chk = chunk .NewChunkWithCapacity (w .copCtx .GetBase ().FieldTypes , newCap )
602
+ targetCap := ingest .CopReadBatchSize (w .hintBatchSize )
603
+ if w .reorgMeta != nil {
604
+ targetCap = ingest .CopReadBatchSize (w .reorgMeta .GetBatchSizeOrDefault (int (variable .GetDDLReorgBatchSize ())))
605
+ }
606
+ chk := w .srcChkPool .Get ().(* chunk.Chunk )
607
+ if chk .Capacity () != targetCap {
608
+ chk = chunk .NewChunkWithCapacity (w .copCtx .GetBase ().FieldTypes , targetCap )
609
+ logutil .Logger (w .ctx ).Info ("adjust ddl job config success" , zap .Int ("current batch size" , chk .Capacity ()))
595
610
}
596
611
chk .Reset ()
597
612
return chk
598
613
}
599
614
600
615
func (w * tableScanWorker ) recycleChunk (chk * chunk.Chunk ) {
601
- w .srcChkPool <- chk
616
+ w .srcChkPool . Put ( chk )
602
617
}
603
618
604
619
// WriteExternalStoreOperator writes index records to external storage.
@@ -618,7 +633,7 @@ func NewWriteExternalStoreOperator(
618
633
tbl table.PhysicalTable ,
619
634
indexes []table.Index ,
620
635
store storage.ExternalStorage ,
621
- srcChunkPool chan * chunk. Chunk ,
636
+ srcChunkPool * sync. Pool ,
622
637
concurrency int ,
623
638
onClose external.OnCloseFunc ,
624
639
memoryQuota uint64 ,
@@ -704,7 +719,7 @@ func NewIndexIngestOperator(
704
719
tbl table.PhysicalTable ,
705
720
indexes []table.Index ,
706
721
engines []ingest.Engine ,
707
- srcChunkPool chan * chunk. Chunk ,
722
+ srcChunkPool * sync. Pool ,
708
723
concurrency int ,
709
724
reorgMeta * model.DDLReorgMeta ,
710
725
cpMgr * ingest.CheckpointManager ,
@@ -765,7 +780,7 @@ type indexIngestExternalWorker struct {
765
780
func (w * indexIngestExternalWorker ) HandleTask (ck IndexRecordChunk , send func (IndexWriteResult )) {
766
781
defer func () {
767
782
if ck .Chunk != nil {
768
- w .srcChunkPool <- ck .Chunk
783
+ w .srcChunkPool . Put ( ck .Chunk )
769
784
}
770
785
}()
771
786
rs , err := w .indexIngestBaseWorker .HandleTask (ck )
@@ -787,7 +802,7 @@ type indexIngestLocalWorker struct {
787
802
func (w * indexIngestLocalWorker ) HandleTask (ck IndexRecordChunk , send func (IndexWriteResult )) {
788
803
defer func () {
789
804
if ck .Chunk != nil {
790
- w .srcChunkPool <- ck .Chunk
805
+ w .srcChunkPool . Put ( ck .Chunk )
791
806
}
792
807
}()
793
808
rs , err := w .indexIngestBaseWorker .HandleTask (ck )
@@ -827,7 +842,7 @@ type indexIngestBaseWorker struct {
827
842
restore func (sessionctx.Context )
828
843
829
844
writers []ingest.Writer
830
- srcChunkPool chan * chunk. Chunk
845
+ srcChunkPool * sync. Pool
831
846
// only available in global sort
832
847
totalCount * atomic.Int64
833
848
}
0 commit comments