@@ -110,7 +110,7 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
110
110
e .producer .resTbl .Lock ()
111
111
defer e .producer .resTbl .Unlock ()
112
112
if ! e .producer .resTbl .Done () {
113
- if err = e .producer .produce (ctx , e ); err != nil {
113
+ if err = e .producer .produce (ctx ); err != nil {
114
114
return err
115
115
}
116
116
}
@@ -338,15 +338,15 @@ func (p *cteProducer) nextChunkLimit(cteExec *CTEExec, req *chunk.Chunk) error {
338
338
return nil
339
339
}
340
340
341
- func (p * cteProducer ) produce (ctx context.Context , cteExec * CTEExec ) (err error ) {
341
+ func (p * cteProducer ) produce (ctx context.Context ) (err error ) {
342
342
if p .resTbl .Error () != nil {
343
343
return p .resTbl .Error ()
344
344
}
345
- resAction := setupCTEStorageTracker (p .resTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
346
- iterInAction := setupCTEStorageTracker (p .iterInTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
345
+ resAction := setupCTEStorageTracker (p .resTbl , p . ctx , p .memTracker , p .diskTracker )
346
+ iterInAction := setupCTEStorageTracker (p .iterInTbl , p . ctx , p .memTracker , p .diskTracker )
347
347
var iterOutAction * chunk.SpillDiskAction
348
348
if p .iterOutTbl != nil {
349
- iterOutAction = setupCTEStorageTracker (p .iterOutTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
349
+ iterOutAction = setupCTEStorageTracker (p .iterOutTbl , p . ctx , p .memTracker , p .diskTracker )
350
350
}
351
351
352
352
failpoint .Inject ("testCTEStorageSpill" , func (val failpoint.Value ) {
@@ -429,12 +429,29 @@ func (p *cteProducer) computeRecursivePart(ctx context.Context) (err error) {
429
429
return
430
430
}
431
431
432
+ var iterNum uint64
432
433
for {
433
434
chk := exec .TryNewCacheChunk (p .recursiveExec )
434
435
if err = exec .Next (ctx , p .recursiveExec , chk ); err != nil {
435
436
return
436
437
}
437
438
if chk .NumRows () == 0 {
439
+ if iterNum % 1000 == 0 {
440
+ // To avoid too many logs.
441
+ p .logTbls (ctx , err , iterNum )
442
+ }
443
+ iterNum ++
444
+ failpoint .Inject ("assertIterTableSpillToDisk" , func (maxIter failpoint.Value ) {
445
+ if iterNum > 0 && iterNum < uint64 (maxIter .(int )) && err == nil {
446
+ if p .iterInTbl .GetMemBytes () != 0 || p .iterInTbl .GetDiskBytes () == 0 ||
447
+ p .iterOutTbl .GetMemBytes () != 0 || p .iterOutTbl .GetDiskBytes () == 0 ||
448
+ p .resTbl .GetMemBytes () != 0 || p .resTbl .GetDiskBytes () == 0 {
449
+ p .logTbls (ctx , err , iterNum )
450
+ panic ("assert row container spill disk failed" )
451
+ }
452
+ }
453
+ })
454
+
438
455
if err = p .setupTblsForNewIteration (); err != nil {
439
456
return
440
457
}
@@ -493,6 +510,8 @@ func (p *cteProducer) setupTblsForNewIteration() (err error) {
493
510
if err = p .iterInTbl .Reopen (); err != nil {
494
511
return err
495
512
}
513
+ setupCTEStorageTracker (p .iterInTbl , p .ctx , p .memTracker , p .diskTracker )
514
+
496
515
if p .isDistinct {
497
516
// Already deduplicated by resTbl, adding directly is ok.
498
517
for _ , chk := range chks {
@@ -507,7 +526,11 @@ func (p *cteProducer) setupTblsForNewIteration() (err error) {
507
526
}
508
527
509
528
// Clear data in iterOutTbl.
510
- return p .iterOutTbl .Reopen ()
529
+ if err = p .iterOutTbl .Reopen (); err != nil {
530
+ return err
531
+ }
532
+ setupCTEStorageTracker (p .iterOutTbl , p .ctx , p .memTracker , p .diskTracker )
533
+ return nil
511
534
}
512
535
513
536
func (p * cteProducer ) reset () {
@@ -535,6 +558,8 @@ func (p *cteProducer) reopenTbls() (err error) {
535
558
if p .isDistinct {
536
559
p .hashTbl = join .NewConcurrentMapHashTable ()
537
560
}
561
+ // Normally we need to setup tracker after calling Reopen(),
562
+ // But reopen resTbl means we need to call produce() again, it will setup tracker.
538
563
if err := p .resTbl .Reopen (); err != nil {
539
564
return err
540
565
}
@@ -736,3 +761,11 @@ func (p *cteProducer) checkAndUpdateCorColHashCode() bool {
736
761
}
737
762
return changed
738
763
}
764
+
765
+ func (p * cteProducer ) logTbls (ctx context.Context , err error , iterNum uint64 ) {
766
+ logutil .Logger (ctx ).Debug ("cte iteration info" ,
767
+ zap .Any ("iterInTbl mem usage" , p .iterInTbl .GetMemBytes ()), zap .Any ("iterInTbl disk usage" , p .iterInTbl .GetDiskBytes ()),
768
+ zap .Any ("iterOutTbl mem usage" , p .iterOutTbl .GetMemBytes ()), zap .Any ("iterOutTbl disk usage" , p .iterOutTbl .GetDiskBytes ()),
769
+ zap .Any ("resTbl mem usage" , p .resTbl .GetMemBytes ()), zap .Any ("resTbl disk usage" , p .resTbl .GetDiskBytes ()),
770
+ zap .Any ("resTbl rows" , p .resTbl .NumRows ()), zap .Any ("iteration num" , iterNum ), zap .Error (err ))
771
+ }
0 commit comments