@@ -301,10 +301,7 @@ func doOptimize(
301
301
if err != nil {
302
302
return nil , nil , 0 , err
303
303
}
304
- finalPlan , err := postOptimize (ctx , sctx , physical )
305
- if err != nil {
306
- return nil , nil , 0 , err
307
- }
304
+ finalPlan := postOptimize (ctx , sctx , physical )
308
305
309
306
if sessVars .StmtCtx .EnableOptimizerCETrace {
310
307
refineCETrace (sctx )
@@ -412,13 +409,9 @@ func mergeContinuousSelections(p base.PhysicalPlan) {
412
409
}
413
410
}
414
411
415
- func postOptimize (ctx context.Context , sctx base.PlanContext , plan base.PhysicalPlan ) ( base.PhysicalPlan , error ) {
412
+ func postOptimize (ctx context.Context , sctx base.PlanContext , plan base.PhysicalPlan ) base.PhysicalPlan {
416
413
// some cases from update optimize will require avoiding projection elimination.
417
414
// see comments ahead of call of DoOptimize in function of buildUpdate().
418
- err := prunePhysicalColumns (sctx , plan )
419
- if err != nil {
420
- return nil , err
421
- }
422
415
plan = eliminatePhysicalProjection (plan )
423
416
plan = InjectExtraProjection (plan )
424
417
mergeContinuousSelections (plan )
@@ -430,7 +423,7 @@ func postOptimize(ctx context.Context, sctx base.PlanContext, plan base.Physical
430
423
disableReuseChunkIfNeeded (sctx , plan )
431
424
tryEnableLateMaterialization (sctx , plan )
432
425
generateRuntimeFilter (sctx , plan )
433
- return plan , nil
426
+ return plan
434
427
}
435
428
436
429
func generateRuntimeFilter (sctx base.PlanContext , plan base.PhysicalPlan ) {
@@ -449,145 +442,6 @@ func generateRuntimeFilter(sctx base.PlanContext, plan base.PhysicalPlan) {
449
442
zap .Duration ("Cost" , time .Since (startRFGenerator )))
450
443
}
451
444
452
- // prunePhysicalColumns currently only work for MPP(HashJoin<-Exchange).
453
- // Here add projection instead of pruning columns directly for safety considerations.
454
- // And projection is cheap here for it saves the network cost and work in memory.
455
- func prunePhysicalColumns (sctx base.PlanContext , plan base.PhysicalPlan ) error {
456
- if tableReader , ok := plan .(* PhysicalTableReader ); ok {
457
- if _ , isExchangeSender := tableReader .tablePlan .(* PhysicalExchangeSender ); isExchangeSender {
458
- err := prunePhysicalColumnsInternal (sctx , tableReader .tablePlan )
459
- if err != nil {
460
- return err
461
- }
462
- }
463
- } else {
464
- for _ , child := range plan .Children () {
465
- return prunePhysicalColumns (sctx , child )
466
- }
467
- }
468
- return nil
469
- }
470
-
471
- func (p * PhysicalHashJoin ) extractUsedCols (parentUsedCols []* expression.Column ) (leftCols []* expression.Column , rightCols []* expression.Column ) {
472
- for _ , eqCond := range p .EqualConditions {
473
- parentUsedCols = append (parentUsedCols , expression .ExtractColumns (eqCond )... )
474
- }
475
- for _ , neCond := range p .NAEqualConditions {
476
- parentUsedCols = append (parentUsedCols , expression .ExtractColumns (neCond )... )
477
- }
478
- for _ , leftCond := range p .LeftConditions {
479
- parentUsedCols = append (parentUsedCols , expression .ExtractColumns (leftCond )... )
480
- }
481
- for _ , rightCond := range p .RightConditions {
482
- parentUsedCols = append (parentUsedCols , expression .ExtractColumns (rightCond )... )
483
- }
484
- for _ , otherCond := range p .OtherConditions {
485
- parentUsedCols = append (parentUsedCols , expression .ExtractColumns (otherCond )... )
486
- }
487
- lChild := p .children [0 ]
488
- rChild := p .children [1 ]
489
- for _ , col := range parentUsedCols {
490
- if lChild .Schema ().Contains (col ) {
491
- leftCols = append (leftCols , col )
492
- } else if rChild .Schema ().Contains (col ) {
493
- rightCols = append (rightCols , col )
494
- }
495
- }
496
- return leftCols , rightCols
497
- }
498
-
499
- func prunePhysicalColumnForHashJoinChild (sctx base.PlanContext , hashJoin * PhysicalHashJoin , joinUsedCols []* expression.Column , sender * PhysicalExchangeSender ) error {
500
- var err error
501
- evalCtx := sctx .GetExprCtx ().GetEvalCtx ()
502
- joinUsed := expression .GetUsedList (evalCtx , joinUsedCols , sender .Schema ())
503
- hashCols := make ([]* expression.Column , len (sender .HashCols ))
504
- for i , mppCol := range sender .HashCols {
505
- hashCols [i ] = mppCol .Col
506
- }
507
- hashUsed := expression .GetUsedList (evalCtx , hashCols , sender .Schema ())
508
-
509
- needPrune := false
510
- usedExprs := make ([]expression.Expression , len (sender .Schema ().Columns ))
511
- prunedSchema := sender .Schema ().Clone ()
512
- for i := len (joinUsed ) - 1 ; i >= 0 ; i -- {
513
- usedExprs [i ] = sender .Schema ().Columns [i ]
514
- if ! joinUsed [i ] && ! hashUsed [i ] {
515
- needPrune = true
516
- usedExprs = append (usedExprs [:i ], usedExprs [i + 1 :]... )
517
- prunedSchema .Columns = append (prunedSchema .Columns [:i ], prunedSchema .Columns [i + 1 :]... )
518
- }
519
- }
520
-
521
- if needPrune && len (sender .children ) > 0 {
522
- ch := sender .children [0 ]
523
- proj := PhysicalProjection {
524
- Exprs : usedExprs ,
525
- }.Init (sctx , ch .StatsInfo (), ch .QueryBlockOffset ())
526
-
527
- proj .SetSchema (prunedSchema )
528
- proj .SetChildren (ch )
529
- sender .children [0 ] = proj
530
-
531
- // Resolve Indices from bottom to up
532
- err = proj .ResolveIndicesItself ()
533
- if err != nil {
534
- return err
535
- }
536
- err = sender .ResolveIndicesItself ()
537
- if err != nil {
538
- return err
539
- }
540
- err = hashJoin .ResolveIndicesItself ()
541
- if err != nil {
542
- return err
543
- }
544
- }
545
- return err
546
- }
547
-
548
- func prunePhysicalColumnsInternal (sctx base.PlanContext , plan base.PhysicalPlan ) error {
549
- var err error
550
- switch x := plan .(type ) {
551
- case * PhysicalHashJoin :
552
- schemaColumns := x .Schema ().Clone ().Columns
553
- leftCols , rightCols := x .extractUsedCols (schemaColumns )
554
- matchPattern := false
555
- for i := 0 ; i <= 1 ; i ++ {
556
- // Pattern: HashJoin <- ExchangeReceiver <- ExchangeSender
557
- matchPattern = false
558
- var exchangeSender * PhysicalExchangeSender
559
- if receiver , ok := x .children [i ].(* PhysicalExchangeReceiver ); ok {
560
- exchangeSender , matchPattern = receiver .children [0 ].(* PhysicalExchangeSender )
561
- }
562
-
563
- if matchPattern {
564
- if i == 0 {
565
- err = prunePhysicalColumnForHashJoinChild (sctx , x , leftCols , exchangeSender )
566
- } else {
567
- err = prunePhysicalColumnForHashJoinChild (sctx , x , rightCols , exchangeSender )
568
- }
569
- if err != nil {
570
- return nil
571
- }
572
- }
573
-
574
- /// recursively travel the physical plan
575
- err = prunePhysicalColumnsInternal (sctx , x .children [i ])
576
- if err != nil {
577
- return nil
578
- }
579
- }
580
- default :
581
- for _ , child := range x .Children () {
582
- err = prunePhysicalColumnsInternal (sctx , child )
583
- if err != nil {
584
- return err
585
- }
586
- }
587
- }
588
- return nil
589
- }
590
-
591
445
// tryEnableLateMaterialization tries to push down some filter conditions to the table scan operator
592
446
// @brief: push down some filter conditions to the table scan operator
593
447
// @param: sctx: session context
0 commit comments