@@ -108,7 +108,7 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
108
108
e .producer .resTbl .Lock ()
109
109
defer e .producer .resTbl .Unlock ()
110
110
if ! e .producer .resTbl .Done () {
111
- if err = e .producer .produce (ctx , e ); err != nil {
111
+ if err = e .producer .produce (ctx ); err != nil {
112
112
return err
113
113
}
114
114
}
@@ -335,15 +335,15 @@ func (p *cteProducer) nextChunkLimit(cteExec *CTEExec, req *chunk.Chunk) error {
335
335
return nil
336
336
}
337
337
338
- func (p * cteProducer ) produce (ctx context.Context , cteExec * CTEExec ) (err error ) {
338
+ func (p * cteProducer ) produce (ctx context.Context ) (err error ) {
339
339
if p .resTbl .Error () != nil {
340
340
return p .resTbl .Error ()
341
341
}
342
- resAction := setupCTEStorageTracker (p .resTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
343
- iterInAction := setupCTEStorageTracker (p .iterInTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
342
+ resAction := setupCTEStorageTracker (p .resTbl , p . ctx , p .memTracker , p .diskTracker )
343
+ iterInAction := setupCTEStorageTracker (p .iterInTbl , p . ctx , p .memTracker , p .diskTracker )
344
344
var iterOutAction * chunk.SpillDiskAction
345
345
if p .iterOutTbl != nil {
346
- iterOutAction = setupCTEStorageTracker (p .iterOutTbl , cteExec . Ctx () , p .memTracker , p .diskTracker )
346
+ iterOutAction = setupCTEStorageTracker (p .iterOutTbl , p . ctx , p .memTracker , p .diskTracker )
347
347
}
348
348
349
349
failpoint .Inject ("testCTEStorageSpill" , func (val failpoint.Value ) {
@@ -426,12 +426,29 @@ func (p *cteProducer) computeRecursivePart(ctx context.Context) (err error) {
426
426
return
427
427
}
428
428
429
+ var iterNum uint64
429
430
for {
430
431
chk := exec .TryNewCacheChunk (p .recursiveExec )
431
432
if err = exec .Next (ctx , p .recursiveExec , chk ); err != nil {
432
433
return
433
434
}
434
435
if chk .NumRows () == 0 {
436
+ if iterNum % 1000 == 0 {
437
+ // To avoid too many logs.
438
+ p .logTbls (ctx , err , iterNum )
439
+ }
440
+ iterNum ++
441
+ failpoint .Inject ("assertIterTableSpillToDisk" , func (maxIter failpoint.Value ) {
442
+ if iterNum > 0 && iterNum < uint64 (maxIter .(int )) && err == nil {
443
+ if p .iterInTbl .GetMemBytes () != 0 || p .iterInTbl .GetDiskBytes () == 0 ||
444
+ p .iterOutTbl .GetMemBytes () != 0 || p .iterOutTbl .GetDiskBytes () == 0 ||
445
+ p .resTbl .GetMemBytes () != 0 || p .resTbl .GetDiskBytes () == 0 {
446
+ p .logTbls (ctx , err , iterNum )
447
+ panic ("assert row container spill disk failed" )
448
+ }
449
+ }
450
+ })
451
+
435
452
if err = p .setupTblsForNewIteration (); err != nil {
436
453
return
437
454
}
@@ -490,6 +507,8 @@ func (p *cteProducer) setupTblsForNewIteration() (err error) {
490
507
if err = p .iterInTbl .Reopen (); err != nil {
491
508
return err
492
509
}
510
+ setupCTEStorageTracker (p .iterInTbl , p .ctx , p .memTracker , p .diskTracker )
511
+
493
512
if p .isDistinct {
494
513
// Already deduplicated by resTbl, adding directly is ok.
495
514
for _ , chk := range chks {
@@ -504,7 +523,11 @@ func (p *cteProducer) setupTblsForNewIteration() (err error) {
504
523
}
505
524
506
525
// Clear data in iterOutTbl.
507
- return p .iterOutTbl .Reopen ()
526
+ if err = p .iterOutTbl .Reopen (); err != nil {
527
+ return err
528
+ }
529
+ setupCTEStorageTracker (p .iterOutTbl , p .ctx , p .memTracker , p .diskTracker )
530
+ return nil
508
531
}
509
532
510
533
func (p * cteProducer ) reset () {
@@ -532,6 +555,8 @@ func (p *cteProducer) reopenTbls() (err error) {
532
555
if p .isDistinct {
533
556
p .hashTbl = newConcurrentMapHashTable ()
534
557
}
558
+ // Normally we need to setup tracker after calling Reopen(),
559
+ // But reopen resTbl means we need to call produce() again, it will setup tracker.
535
560
if err := p .resTbl .Reopen (); err != nil {
536
561
return err
537
562
}
@@ -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