/
Optimizer.scala
1852 lines (1685 loc) · 78.6 KB
/
Optimizer.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst.optimizer
import scala.collection.mutable
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
/**
* Abstract class all optimizers should inherit of, contains the standard batches (extending
* Optimizers can override this.
*/
abstract class Optimizer(catalogManager: CatalogManager)
extends RuleExecutor[LogicalPlan] {
// Check for structural integrity of the plan in test mode.
// Currently we check after the execution of each rule if a plan:
// - is still resolved
// - only host special expressions in supported operators
override protected def isPlanIntegral(plan: LogicalPlan): Boolean = {
!Utils.isTesting || (plan.resolved &&
plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty)
}
override protected val excludedOnceBatches: Set[String] =
Set(
"PartitionPruning",
"Extract Python UDFs")
protected def fixedPoint =
FixedPoint(
SQLConf.get.optimizerMaxIterations,
maxIterationsSetting = SQLConf.OPTIMIZER_MAX_ITERATIONS.key)
/**
* Defines the default rule batches in the Optimizer.
*
* Implementations of this class should override this method, and [[nonExcludableRules]] if
* necessary, instead of [[batches]]. The rule batches that eventually run in the Optimizer,
* i.e., returned by [[batches]], will be (defaultBatches - (excludedRules - nonExcludableRules)).
*/
def defaultBatches: Seq[Batch] = {
val operatorOptimizationRuleSet =
Seq(
// Operator push down
PushProjectionThroughUnion,
ReorderJoin,
EliminateOuterJoin,
PushDownPredicates,
PushDownLeftSemiAntiJoin,
PushLeftSemiLeftAntiThroughJoin,
LimitPushDown,
ColumnPruning,
InferFiltersFromConstraints,
// Operator combine
CollapseRepartition,
CollapseProject,
CollapseWindow,
CombineFilters,
CombineLimits,
CombineUnions,
// Constant folding and strength reduction
TransposeWindow,
NullPropagation,
ConstantPropagation,
FoldablePropagation,
OptimizeIn,
ConstantFolding,
EliminateAggregateFilter,
ReorderAssociativeOperator,
LikeSimplification,
BooleanSimplification,
SimplifyConditionals,
RemoveDispensableExpressions,
SimplifyBinaryComparison,
ReplaceNullWithFalseInPredicate,
PruneFilters,
SimplifyCasts,
SimplifyCaseConversionExpressions,
RewriteCorrelatedScalarSubquery,
EliminateSerialization,
RemoveRedundantAliases,
UnwrapCastInBinaryComparison,
RemoveNoopOperators,
CombineWithFields,
SimplifyExtractValueOps,
CombineConcats) ++
extendedOperatorOptimizationRules
val operatorOptimizationBatch: Seq[Batch] = {
val rulesWithoutInferFiltersFromConstraints =
operatorOptimizationRuleSet.filterNot(_ == InferFiltersFromConstraints)
Batch("Operator Optimization before Inferring Filters", fixedPoint,
rulesWithoutInferFiltersFromConstraints: _*) ::
Batch("Infer Filters", Once,
InferFiltersFromConstraints) ::
Batch("Operator Optimization after Inferring Filters", fixedPoint,
rulesWithoutInferFiltersFromConstraints: _*) ::
// Set strategy to Once to avoid pushing filter every time because we do not change the
// join condition.
Batch("Push extra predicate through join", fixedPoint,
PushExtraPredicateThroughJoin,
PushDownPredicates) :: Nil
}
val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) ::
// Technically some of the rules in Finish Analysis are not optimizer rules and belong more
// in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime).
// However, because we also use the analyzer to canonicalized queries (for view definition),
// we do not eliminate subqueries or compute current time in the analyzer.
Batch("Finish Analysis", Once,
EliminateResolvedHint,
EliminateSubqueryAliases,
EliminateView,
ReplaceExpressions,
RewriteNonCorrelatedExists,
ComputeCurrentTime,
GetCurrentDatabaseAndCatalog(catalogManager),
ReplaceDeduplicateWithAggregate) ::
//////////////////////////////////////////////////////////////////////////////////////////
// Optimizer rules start here
//////////////////////////////////////////////////////////////////////////////////////////
// - Do the first call of CombineUnions before starting the major Optimizer rules,
// since it can reduce the number of iteration and the other rules could add/move
// extra operators between two adjacent Union operators.
// - Call CombineUnions again in Batch("Operator Optimizations"),
// since the other rules might make two separate Unions operators adjacent.
Batch("Union", Once,
CombineUnions) ::
Batch("OptimizeLimitZero", Once,
OptimizeLimitZero) ::
// Run this once earlier. This might simplify the plan and reduce cost of optimizer.
// For example, a query such as Filter(LocalRelation) would go through all the heavy
// optimizer rules that are triggered when there is a filter
// (e.g. InferFiltersFromConstraints). If we run this batch earlier, the query becomes just
// LocalRelation and does not trigger many rules.
Batch("LocalRelation early", fixedPoint,
ConvertToLocalRelation,
PropagateEmptyRelation,
// PropagateEmptyRelation can change the nullability of an attribute from nullable to
// non-nullable when an empty relation child of a Union is removed
UpdateAttributeNullability) ::
Batch("Pullup Correlated Expressions", Once,
PullupCorrelatedPredicates) ::
// Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense
// to enforce idempotence on it and we change this batch from Once to FixedPoint(1).
Batch("Subquery", FixedPoint(1),
OptimizeSubqueries) ::
Batch("Replace Operators", fixedPoint,
RewriteExceptAll,
RewriteIntersectAll,
ReplaceIntersectWithSemiJoin,
ReplaceExceptWithFilter,
ReplaceExceptWithAntiJoin,
ReplaceDistinctWithAggregate) ::
Batch("Aggregate", fixedPoint,
RemoveLiteralFromGroupExpressions,
RemoveRepetitionFromGroupExpressions) :: Nil ++
operatorOptimizationBatch) :+
// This batch pushes filters and projections into scan nodes. Before this batch, the logical
// plan may contain nodes that do not report stats. Anything that uses stats must run after
// this batch.
Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+
// Since join costs in AQP can change between multiple runs, there is no reason that we have an
// idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once.
Batch("Join Reorder", FixedPoint(1),
CostBasedJoinReorder) :+
Batch("Eliminate Sorts", Once,
EliminateSorts) :+
Batch("Decimal Optimizations", fixedPoint,
DecimalAggregates) :+
// This batch must run after "Decimal Optimizations", as that one may change the
// aggregate distinct column
Batch("Distinct Aggregate Rewrite", Once,
RewriteDistinctAggregates) :+
Batch("Object Expressions Optimization", fixedPoint,
EliminateMapObjects,
CombineTypedFilters,
ObjectSerializerPruning,
ReassignLambdaVariableID) :+
Batch("LocalRelation", fixedPoint,
ConvertToLocalRelation,
PropagateEmptyRelation,
// PropagateEmptyRelation can change the nullability of an attribute from nullable to
// non-nullable when an empty relation child of a Union is removed
UpdateAttributeNullability) :+
// The following batch should be executed after batch "Join Reorder" and "LocalRelation".
Batch("Check Cartesian Products", Once,
CheckCartesianProducts) :+
Batch("RewriteSubquery", Once,
RewritePredicateSubquery,
ColumnPruning,
CollapseProject,
RemoveNoopOperators) :+
// This batch must be executed after the `RewriteSubquery` batch, which creates joins.
Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+
Batch("ReplaceWithFieldsExpression", Once, ReplaceWithFieldsExpression)
// remove any batches with no rules. this may happen when subclasses do not add optional rules.
batches.filter(_.rules.nonEmpty)
}
/**
* Defines rules that cannot be excluded from the Optimizer even if they are specified in
* SQL config "excludedRules".
*
* Implementations of this class can override this method if necessary. The rule batches
* that eventually run in the Optimizer, i.e., returned by [[batches]], will be
* (defaultBatches - (excludedRules - nonExcludableRules)).
*/
def nonExcludableRules: Seq[String] =
EliminateDistinct.ruleName ::
EliminateResolvedHint.ruleName ::
EliminateSubqueryAliases.ruleName ::
EliminateView.ruleName ::
ReplaceExpressions.ruleName ::
ComputeCurrentTime.ruleName ::
GetCurrentDatabaseAndCatalog(catalogManager).ruleName ::
RewriteDistinctAggregates.ruleName ::
ReplaceDeduplicateWithAggregate.ruleName ::
ReplaceIntersectWithSemiJoin.ruleName ::
ReplaceExceptWithFilter.ruleName ::
ReplaceExceptWithAntiJoin.ruleName ::
RewriteExceptAll.ruleName ::
RewriteIntersectAll.ruleName ::
ReplaceDistinctWithAggregate.ruleName ::
PullupCorrelatedPredicates.ruleName ::
RewriteCorrelatedScalarSubquery.ruleName ::
RewritePredicateSubquery.ruleName ::
NormalizeFloatingNumbers.ruleName ::
ReplaceWithFieldsExpression.ruleName :: Nil
/**
* Optimize all the subqueries inside expression.
*/
object OptimizeSubqueries extends Rule[LogicalPlan] {
private def removeTopLevelSort(plan: LogicalPlan): LogicalPlan = {
plan match {
case Sort(_, _, child) => child
case Project(fields, child) => Project(fields, removeTopLevelSort(child))
case other => other
}
}
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case s: SubqueryExpression =>
val Subquery(newPlan, _) = Optimizer.this.execute(Subquery.fromExpression(s))
// At this point we have an optimized subquery plan that we are going to attach
// to this subquery expression. Here we can safely remove any top level sort
// in the plan as tuples produced by a subquery are un-ordered.
s.withNewPlan(removeTopLevelSort(newPlan))
}
}
/**
* Override to provide additional rules for the operator optimization batch.
*/
def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
/**
* Override to provide additional rules for early projection and filter pushdown to scans.
*/
def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil
/**
* Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that
* eventually run in the Optimizer.
*
* Implementations of this class should override [[defaultBatches]], and [[nonExcludableRules]]
* if necessary, instead of this method.
*/
final override def batches: Seq[Batch] = {
val excludedRulesConf =
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(Utils.stringToSeq)
val excludedRules = excludedRulesConf.filter { ruleName =>
val nonExcludable = nonExcludableRules.contains(ruleName)
if (nonExcludable) {
logWarning(s"Optimization rule '${ruleName}' was not excluded from the optimizer " +
s"because this rule is a non-excludable rule.")
}
!nonExcludable
}
if (excludedRules.isEmpty) {
defaultBatches
} else {
defaultBatches.flatMap { batch =>
val filteredRules = batch.rules.filter { rule =>
val exclude = excludedRules.contains(rule.ruleName)
if (exclude) {
logInfo(s"Optimization rule '${rule.ruleName}' is excluded from the optimizer.")
}
!exclude
}
if (batch.rules == filteredRules) {
Some(batch)
} else if (filteredRules.nonEmpty) {
Some(Batch(batch.name, batch.strategy, filteredRules: _*))
} else {
logInfo(s"Optimization batch '${batch.name}' is excluded from the optimizer " +
s"as all enclosed rules have been excluded.")
None
}
}
}
}
}
/**
* Remove useless DISTINCT for MAX and MIN.
* This rule should be applied before RewriteDistinctAggregates.
*/
object EliminateDistinct extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan transformExpressions {
case ae: AggregateExpression if ae.isDistinct =>
ae.aggregateFunction match {
case _: Max | _: Min => ae.copy(isDistinct = false)
case _ => ae
}
}
}
/**
* Remove useless FILTER clause for aggregate expressions.
* This rule should be applied before RewriteDistinctAggregates.
*/
object EliminateAggregateFilter extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan transformExpressions {
case ae @ AggregateExpression(_, _, _, Some(Literal.TrueLiteral), _) =>
ae.copy(filter = None)
case AggregateExpression(af: DeclarativeAggregate, _, _, Some(Literal.FalseLiteral), _) =>
val initialProject = SafeProjection.create(af.initialValues)
val evalProject = SafeProjection.create(af.evaluateExpression :: Nil, af.aggBufferAttributes)
val initialBuffer = initialProject(EmptyRow)
val internalRow = evalProject(initialBuffer)
Literal.create(internalRow.get(0, af.dataType), af.dataType)
case AggregateExpression(af: ImperativeAggregate, _, _, Some(Literal.FalseLiteral), _) =>
val buffer = new SpecificInternalRow(af.aggBufferAttributes.map(_.dataType))
af.initialize(buffer)
Literal.create(af.eval(buffer), af.dataType)
}
}
/**
* An optimizer used in test code.
*
* To ensure extendability, we leave the standard rules in the abstract optimizer rules, while
* specific rules go to the subclasses
*/
object SimpleTestOptimizer extends SimpleTestOptimizer
class SimpleTestOptimizer extends Optimizer(
new CatalogManager(
new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true),
FakeV2SessionCatalog,
new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, new SQLConf())))
/**
* Remove redundant aliases from a query plan. A redundant alias is an alias that does not change
* the name or metadata of a column, and does not deduplicate it.
*/
object RemoveRedundantAliases extends Rule[LogicalPlan] {
/**
* Create an attribute mapping from the old to the new attributes. This function will only
* return the attribute pairs that have changed.
*/
private def createAttributeMapping(current: LogicalPlan, next: LogicalPlan)
: Seq[(Attribute, Attribute)] = {
current.output.zip(next.output).filterNot {
case (a1, a2) => a1.semanticEquals(a2)
}
}
/**
* Remove the top-level alias from an expression when it is redundant.
*/
private def removeRedundantAlias(e: Expression, excludeList: AttributeSet): Expression = e match {
// Alias with metadata can not be stripped, or the metadata will be lost.
// If the alias name is different from attribute name, we can't strip it either, or we
// may accidentally change the output schema name of the root plan.
case a @ Alias(attr: Attribute, name)
if a.metadata == Metadata.empty &&
name == attr.name &&
!excludeList.contains(attr) &&
!excludeList.contains(a) =>
attr
case a => a
}
/**
* Remove redundant alias expression from a LogicalPlan and its subtree. A set of excludes is used
* to prevent the removal of seemingly redundant aliases used to deduplicate the input for a
* (self) join or to prevent the removal of top-level subquery attributes.
*/
private def removeRedundantAliases(plan: LogicalPlan, excluded: AttributeSet): LogicalPlan = {
plan match {
// We want to keep the same output attributes for subqueries. This means we cannot remove
// the aliases that produce these attributes
case Subquery(child, correlated) =>
Subquery(removeRedundantAliases(child, excluded ++ child.outputSet), correlated)
// A join has to be treated differently, because the left and the right side of the join are
// not allowed to use the same attributes. We use an exclude list to prevent us from creating
// a situation in which this happens; the rule will only remove an alias if its child
// attribute is not on the black list.
case Join(left, right, joinType, condition, hint) =>
val newLeft = removeRedundantAliases(left, excluded ++ right.outputSet)
val newRight = removeRedundantAliases(right, excluded ++ newLeft.outputSet)
val mapping = AttributeMap(
createAttributeMapping(left, newLeft) ++
createAttributeMapping(right, newRight))
val newCondition = condition.map(_.transform {
case a: Attribute => mapping.getOrElse(a, a)
})
Join(newLeft, newRight, joinType, newCondition, hint)
case _ =>
// Remove redundant aliases in the subtree(s).
val currentNextAttrPairs = mutable.Buffer.empty[(Attribute, Attribute)]
val newNode = plan.mapChildren { child =>
val newChild = removeRedundantAliases(child, excluded)
currentNextAttrPairs ++= createAttributeMapping(child, newChild)
newChild
}
// Create the attribute mapping. Note that the currentNextAttrPairs can contain duplicate
// keys in case of Union (this is caused by the PushProjectionThroughUnion rule); in this
// case we use the first mapping (which should be provided by the first child).
val mapping = AttributeMap(currentNextAttrPairs.toSeq)
// Create a an expression cleaning function for nodes that can actually produce redundant
// aliases, use identity otherwise.
val clean: Expression => Expression = plan match {
case _: Project => removeRedundantAlias(_, excluded)
case _: Aggregate => removeRedundantAlias(_, excluded)
case _: Window => removeRedundantAlias(_, excluded)
case _ => identity[Expression]
}
// Transform the expressions.
newNode.mapExpressions { expr =>
clean(expr.transform {
case a: Attribute => mapping.getOrElse(a, a)
})
}
}
}
def apply(plan: LogicalPlan): LogicalPlan = removeRedundantAliases(plan, AttributeSet.empty)
}
/**
* Remove no-op operators from the query plan that do not make any modifications.
*/
object RemoveNoopOperators extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Eliminate no-op Projects
case p @ Project(_, child) if child.sameOutput(p) => child
// Eliminate no-op Window
case w: Window if w.windowExpressions.isEmpty => w.child
}
}
/**
* Pushes down [[LocalLimit]] beneath UNION ALL and beneath the streamed inputs of outer joins.
*/
object LimitPushDown extends Rule[LogicalPlan] {
private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = {
plan match {
case GlobalLimit(_, child) => child
case _ => plan
}
}
private def maybePushLocalLimit(limitExp: Expression, plan: LogicalPlan): LogicalPlan = {
(limitExp, plan.maxRowsPerPartition) match {
case (IntegerLiteral(newLimit), Some(childMaxRows)) if newLimit < childMaxRows =>
// If the child has a cap on max rows per partition and the cap is larger than
// the new limit, put a new LocalLimit there.
LocalLimit(limitExp, stripGlobalLimitIfPresent(plan))
case (_, None) =>
// If the child has no cap, put the new LocalLimit.
LocalLimit(limitExp, stripGlobalLimitIfPresent(plan))
case _ =>
// Otherwise, don't put a new LocalLimit.
plan
}
}
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Adding extra Limits below UNION ALL for children which are not Limit or do not have Limit
// descendants whose maxRow is larger. This heuristic is valid assuming there does not exist any
// Limit push-down rule that is unable to infer the value of maxRows.
// Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to
// pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to
// pushdown Limit.
case LocalLimit(exp, u: Union) =>
LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _))))
// Add extra limits below OUTER JOIN. For LEFT OUTER and RIGHT OUTER JOIN we push limits to
// the left and right sides, respectively. It's not safe to push limits below FULL OUTER
// JOIN in the general case without a more invasive rewrite.
// We also need to ensure that this limit pushdown rule will not eventually introduce limits
// on both sides if it is applied multiple times. Therefore:
// - If one side is already limited, stack another limit on top if the new limit is smaller.
// The redundant limit will be collapsed by the CombineLimits rule.
case LocalLimit(exp, join @ Join(left, right, joinType, _, _)) =>
val newJoin = joinType match {
case RightOuter => join.copy(right = maybePushLocalLimit(exp, right))
case LeftOuter => join.copy(left = maybePushLocalLimit(exp, left))
case _ => join
}
LocalLimit(exp, newJoin)
}
}
/**
* Pushes Project operator to both sides of a Union operator.
* Operations that are safe to pushdown are listed as follows.
* Union:
* Right now, Union means UNION ALL, which does not de-duplicate rows. So, it is
* safe to pushdown Filters and Projections through it. Filter pushdown is handled by another
* rule PushDownPredicates. Once we add UNION DISTINCT, we will not be able to pushdown Projections.
*/
object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper {
/**
* Maps Attributes from the left side to the corresponding Attribute on the right side.
*/
private def buildRewrites(left: LogicalPlan, right: LogicalPlan): AttributeMap[Attribute] = {
assert(left.output.size == right.output.size)
AttributeMap(left.output.zip(right.output))
}
/**
* Rewrites an expression so that it can be pushed to the right side of a
* Union or Except operator. This method relies on the fact that the output attributes
* of a union/intersect/except are always equal to the left child's output.
*/
private def pushToRight[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = {
val result = e transform {
case a: Attribute => rewrites(a)
} match {
// Make sure exprId is unique in each child of Union.
case Alias(child, alias) => Alias(child, alias)()
case other => other
}
// We must promise the compiler that we did not discard the names in the case of project
// expressions. This is safe since the only transformation is from Attribute => Attribute.
result.asInstanceOf[A]
}
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Push down deterministic projection through UNION ALL
case p @ Project(projectList, u: Union) =>
assert(u.children.nonEmpty)
if (projectList.forall(_.deterministic)) {
val newFirstChild = Project(projectList, u.children.head)
val newOtherChildren = u.children.tail.map { child =>
val rewrites = buildRewrites(u.children.head, child)
Project(projectList.map(pushToRight(_, rewrites)), child)
}
u.copy(children = newFirstChild +: newOtherChildren)
} else {
p
}
}
}
/**
* Attempts to eliminate the reading of unneeded columns from the query plan.
*
* Since adding Project before Filter conflicts with PushPredicatesThroughProject, this rule will
* remove the Project p2 in the following pattern:
*
* p1 @ Project(_, Filter(_, p2 @ Project(_, child))) if p2.outputSet.subsetOf(p2.inputSet)
*
* p2 is usually inserted by this rule and useless, p1 could prune the columns anyway.
*/
object ColumnPruning extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = removeProjectBeforeFilter(plan transform {
// Prunes the unused columns from project list of Project/Aggregate/Expand
case p @ Project(_, p2: Project) if !p2.outputSet.subsetOf(p.references) =>
p.copy(child = p2.copy(projectList = p2.projectList.filter(p.references.contains)))
case p @ Project(_, a: Aggregate) if !a.outputSet.subsetOf(p.references) =>
p.copy(
child = a.copy(aggregateExpressions = a.aggregateExpressions.filter(p.references.contains)))
case a @ Project(_, e @ Expand(_, _, grandChild)) if !e.outputSet.subsetOf(a.references) =>
val newOutput = e.output.filter(a.references.contains(_))
val newProjects = e.projections.map { proj =>
proj.zip(e.output).filter { case (_, a) =>
newOutput.contains(a)
}.unzip._1
}
a.copy(child = Expand(newProjects, newOutput, grandChild))
// Prunes the unused columns from child of `DeserializeToObject`
case d @ DeserializeToObject(_, _, child) if !child.outputSet.subsetOf(d.references) =>
d.copy(child = prunedChild(child, d.references))
// Prunes the unused columns from child of Aggregate/Expand/Generate/ScriptTransformation
case a @ Aggregate(_, _, child) if !child.outputSet.subsetOf(a.references) =>
a.copy(child = prunedChild(child, a.references))
case f @ FlatMapGroupsInPandas(_, _, _, child) if !child.outputSet.subsetOf(f.references) =>
f.copy(child = prunedChild(child, f.references))
case e @ Expand(_, _, child) if !child.outputSet.subsetOf(e.references) =>
e.copy(child = prunedChild(child, e.references))
case s @ ScriptTransformation(_, _, _, child, _)
if !child.outputSet.subsetOf(s.references) =>
s.copy(child = prunedChild(child, s.references))
// prune unrequired references
case p @ Project(_, g: Generate) if p.references != g.outputSet =>
val requiredAttrs = p.references -- g.producedAttributes ++ g.generator.references
val newChild = prunedChild(g.child, requiredAttrs)
val unrequired = g.generator.references -- p.references
val unrequiredIndices = newChild.output.zipWithIndex.filter(t => unrequired.contains(t._1))
.map(_._2)
p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices))
// prune unrequired nested fields from `Generate`.
case GeneratorNestedColumnAliasing(p) => p
// Eliminate unneeded attributes from right side of a Left Existence Join.
case j @ Join(_, right, LeftExistence(_), _, _) =>
j.copy(right = prunedChild(right, j.references))
// all the columns will be used to compare, so we can't prune them
case p @ Project(_, _: SetOperation) => p
case p @ Project(_, _: Distinct) => p
// Eliminate unneeded attributes from children of Union.
case p @ Project(_, u: Union) =>
if (!u.outputSet.subsetOf(p.references)) {
val firstChild = u.children.head
val newOutput = prunedChild(firstChild, p.references).output
// pruning the columns of all children based on the pruned first child.
val newChildren = u.children.map { p =>
val selected = p.output.zipWithIndex.filter { case (a, i) =>
newOutput.contains(firstChild.output(i))
}.map(_._1)
Project(selected, p)
}
p.copy(child = u.withNewChildren(newChildren))
} else {
p
}
// Prune unnecessary window expressions
case p @ Project(_, w: Window) if !w.windowOutputSet.subsetOf(p.references) =>
p.copy(child = w.copy(
windowExpressions = w.windowExpressions.filter(p.references.contains)))
// Can't prune the columns on LeafNode
case p @ Project(_, _: LeafNode) => p
case NestedColumnAliasing(p) => p
// for all other logical plans that inherits the output from it's children
// Project over project is handled by the first case, skip it here.
case p @ Project(_, child) if !child.isInstanceOf[Project] =>
val required = child.references ++ p.references
if (!child.inputSet.subsetOf(required)) {
val newChildren = child.children.map(c => prunedChild(c, required))
p.copy(child = child.withNewChildren(newChildren))
} else {
p
}
})
/** Applies a projection only when the child is producing unnecessary attributes */
private def prunedChild(c: LogicalPlan, allReferences: AttributeSet) =
if (!c.outputSet.subsetOf(allReferences)) {
Project(c.output.filter(allReferences.contains), c)
} else {
c
}
/**
* The Project before Filter is not necessary but conflict with PushPredicatesThroughProject,
* so remove it. Since the Projects have been added top-down, we need to remove in bottom-up
* order, otherwise lower Projects can be missed.
*/
private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transformUp {
case p1 @ Project(_, f @ Filter(_, p2 @ Project(_, child)))
if p2.outputSet.subsetOf(child.outputSet) &&
// We only remove attribute-only project.
p2.projectList.forall(_.isInstanceOf[AttributeReference]) =>
p1.copy(child = f.copy(child = child))
}
}
/**
* Combines two [[Project]] operators into one and perform alias substitution,
* merging the expressions into one single expression for the following cases.
* 1. When two [[Project]] operators are adjacent.
* 2. When two [[Project]] operators have LocalLimit/Sample/Repartition operator between them
* and the upper project consists of the same number of columns which is equal or aliasing.
* `GlobalLimit(LocalLimit)` pattern is also considered.
*/
object CollapseProject extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case p1 @ Project(_, p2: Project) =>
if (haveCommonNonDeterministicOutput(p1.projectList, p2.projectList)) {
p1
} else {
p2.copy(projectList = buildCleanedProjectList(p1.projectList, p2.projectList))
}
case p @ Project(_, agg: Aggregate) =>
if (haveCommonNonDeterministicOutput(p.projectList, agg.aggregateExpressions)) {
p
} else {
agg.copy(aggregateExpressions = buildCleanedProjectList(
p.projectList, agg.aggregateExpressions))
}
case Project(l1, g @ GlobalLimit(_, limit @ LocalLimit(_, p2 @ Project(l2, _))))
if isRenaming(l1, l2) =>
val newProjectList = buildCleanedProjectList(l1, l2)
g.copy(child = limit.copy(child = p2.copy(projectList = newProjectList)))
case Project(l1, limit @ LocalLimit(_, p2 @ Project(l2, _))) if isRenaming(l1, l2) =>
val newProjectList = buildCleanedProjectList(l1, l2)
limit.copy(child = p2.copy(projectList = newProjectList))
case Project(l1, r @ Repartition(_, _, p @ Project(l2, _))) if isRenaming(l1, l2) =>
r.copy(child = p.copy(projectList = buildCleanedProjectList(l1, p.projectList)))
case Project(l1, s @ Sample(_, _, _, _, p2 @ Project(l2, _))) if isRenaming(l1, l2) =>
s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList)))
}
private def collectAliases(projectList: Seq[NamedExpression]): AttributeMap[Alias] = {
AttributeMap(projectList.collect {
case a: Alias => a.toAttribute -> a
})
}
private def haveCommonNonDeterministicOutput(
upper: Seq[NamedExpression], lower: Seq[NamedExpression]): Boolean = {
// Create a map of Aliases to their values from the lower projection.
// e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)).
val aliases = collectAliases(lower)
// Collapse upper and lower Projects if and only if their overlapped expressions are all
// deterministic.
upper.exists(_.collect {
case a: Attribute if aliases.contains(a) => aliases(a).child
}.exists(!_.deterministic))
}
private def buildCleanedProjectList(
upper: Seq[NamedExpression],
lower: Seq[NamedExpression]): Seq[NamedExpression] = {
// Create a map of Aliases to their values from the lower projection.
// e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)).
val aliases = collectAliases(lower)
// Substitute any attributes that are produced by the lower projection, so that we safely
// eliminate it.
// e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...'
// Use transformUp to prevent infinite recursion.
val rewrittenUpper = upper.map(_.transformUp {
case a: Attribute => aliases.getOrElse(a, a)
})
// collapse upper and lower Projects may introduce unnecessary Aliases, trim them here.
rewrittenUpper.map { p =>
CleanupAliases.trimNonTopLevelAliases(p).asInstanceOf[NamedExpression]
}
}
private def isRenaming(list1: Seq[NamedExpression], list2: Seq[NamedExpression]): Boolean = {
list1.length == list2.length && list1.zip(list2).forall {
case (e1, e2) if e1.semanticEquals(e2) => true
case (Alias(a: Attribute, _), b) if a.metadata == Metadata.empty && a.name == b.name => true
case _ => false
}
}
}
/**
* Combines adjacent [[RepartitionOperation]] operators
*/
object CollapseRepartition extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
// Case 1: When a Repartition has a child of Repartition or RepartitionByExpression,
// 1) When the top node does not enable the shuffle (i.e., coalesce API), but the child
// enables the shuffle. Returns the child node if the last numPartitions is bigger;
// otherwise, keep unchanged.
// 2) In the other cases, returns the top node with the child's child
case r @ Repartition(_, _, child: RepartitionOperation) => (r.shuffle, child.shuffle) match {
case (false, true) => if (r.numPartitions >= child.numPartitions) child else r
case _ => r.copy(child = child.child)
}
// Case 2: When a RepartitionByExpression has a child of Repartition or RepartitionByExpression
// we can remove the child.
case r @ RepartitionByExpression(_, child: RepartitionOperation, _) =>
r.copy(child = child.child)
}
}
/**
* Collapse Adjacent Window Expression.
* - If the partition specs and order specs are the same and the window expression are
* independent and are of the same window function type, collapse into the parent.
*/
object CollapseWindow extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case w1 @ Window(we1, ps1, os1, w2 @ Window(we2, ps2, os2, grandChild))
if ps1 == ps2 && os1 == os2 && w1.references.intersect(w2.windowOutputSet).isEmpty &&
we1.nonEmpty && we2.nonEmpty &&
// This assumes Window contains the same type of window expressions. This is ensured
// by ExtractWindowFunctions.
WindowFunctionType.functionType(we1.head) == WindowFunctionType.functionType(we2.head) =>
w1.copy(windowExpressions = we2 ++ we1, child = grandChild)
}
}
/**
* Transpose Adjacent Window Expressions.
* - If the partition spec of the parent Window expression is compatible with the partition spec
* of the child window expression, transpose them.
*/
object TransposeWindow extends Rule[LogicalPlan] {
private def compatibleParititions(ps1 : Seq[Expression], ps2: Seq[Expression]): Boolean = {
ps1.length < ps2.length && ps2.take(ps1.length).permutations.exists(ps1.zip(_).forall {
case (l, r) => l.semanticEquals(r)
})
}
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case w1 @ Window(we1, ps1, os1, w2 @ Window(we2, ps2, os2, grandChild))
if w1.references.intersect(w2.windowOutputSet).isEmpty &&
w1.expressions.forall(_.deterministic) &&
w2.expressions.forall(_.deterministic) &&
compatibleParititions(ps1, ps2) =>
Project(w1.output, Window(we2, ps2, os2, Window(we1, ps1, os1, grandChild)))
}
}
/**
* Generate a list of additional filters from an operator's existing constraint but remove those
* that are either already part of the operator's condition or are part of the operator's child
* constraints. These filters are currently inserted to the existing conditions in the Filter
* operators and on either side of Join operators.
*
* Note: While this optimization is applicable to a lot of types of join, it primarily benefits
* Inner and LeftSemi joins.
*/
object InferFiltersFromConstraints extends Rule[LogicalPlan]
with PredicateHelper with ConstraintHelper {
def apply(plan: LogicalPlan): LogicalPlan = {
if (SQLConf.get.constraintPropagationEnabled) {
inferFilters(plan)
} else {
plan
}
}
private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform {
case filter @ Filter(condition, child) =>
val newFilters = filter.constraints --
(child.constraints ++ splitConjunctivePredicates(condition))
if (newFilters.nonEmpty) {
Filter(And(newFilters.reduce(And), condition), child)
} else {
filter
}
case join @ Join(left, right, joinType, conditionOpt, _) =>
joinType match {
// For inner join, we can infer additional filters for both sides. LeftSemi is kind of an
// inner join, it just drops the right side in the final output.
case _: InnerLike | LeftSemi =>
val allConstraints = getAllConstraints(left, right, conditionOpt)
val newLeft = inferNewFilter(left, allConstraints)
val newRight = inferNewFilter(right, allConstraints)
join.copy(left = newLeft, right = newRight)
// For right outer join, we can only infer additional filters for left side.
case RightOuter =>
val allConstraints = getAllConstraints(left, right, conditionOpt)
val newLeft = inferNewFilter(left, allConstraints)
join.copy(left = newLeft)
// For left join, we can only infer additional filters for right side.
case LeftOuter | LeftAnti =>
val allConstraints = getAllConstraints(left, right, conditionOpt)
val newRight = inferNewFilter(right, allConstraints)
join.copy(right = newRight)
case _ => join
}
}
private def getAllConstraints(
left: LogicalPlan,
right: LogicalPlan,
conditionOpt: Option[Expression]): ExpressionSet = {
val baseConstraints = left.constraints.union(right.constraints)
.union(ExpressionSet(conditionOpt.map(splitConjunctivePredicates).getOrElse(Nil)))
baseConstraints.union(inferAdditionalConstraints(baseConstraints))
}
private def inferNewFilter(plan: LogicalPlan, constraints: ExpressionSet): LogicalPlan = {
val newPredicates = constraints
.union(constructIsNotNullConstraints(constraints, plan.output))
.filter { c =>
c.references.nonEmpty && c.references.subsetOf(plan.outputSet) && c.deterministic
} -- plan.constraints
if (newPredicates.isEmpty) {
plan
} else {
Filter(newPredicates.reduce(And), plan)
}
}
}
/**
* Combines all adjacent [[Union]] operators into a single [[Union]].
*/
object CombineUnions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
case u: Union => flattenUnion(u, false)
case Distinct(u: Union) => Distinct(flattenUnion(u, true))
}
private def flattenUnion(union: Union, flattenDistinct: Boolean): Union = {
val topByName = union.byName
val topAllowMissingCol = union.allowMissingCol
val stack = mutable.Stack[LogicalPlan](union)
val flattened = mutable.ArrayBuffer.empty[LogicalPlan]
// Note that we should only flatten the unions with same byName and allowMissingCol.
// Although we do `UnionCoercion` at analysis phase, we manually run `CombineUnions`
// in some places like `Dataset.union`. Flattening unions with different resolution
// rules (by position and by name) could cause incorrect results.
while (stack.nonEmpty) {
stack.pop() match {
case Distinct(Union(children, byName, allowMissingCol))
if flattenDistinct && byName == topByName && allowMissingCol == topAllowMissingCol =>
stack.pushAll(children.reverse)
case Union(children, byName, allowMissingCol)
if byName == topByName && allowMissingCol == topAllowMissingCol =>
stack.pushAll(children.reverse)
case child =>
flattened += child
}
}
union.copy(children = flattened.toSeq)
}
}
/**
* Combines two adjacent [[Filter]] operators into one, merging the non-redundant conditions into
* one conjunctive predicate.
*/
object CombineFilters extends Rule[LogicalPlan] with PredicateHelper {
def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally
val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = {
// The query execution/optimization does not guarantee the expressions are evaluated in order.
// We only can combine them if and only if both are deterministic.
case Filter(fc, nf @ Filter(nc, grandChild)) if fc.deterministic && nc.deterministic =>
(ExpressionSet(splitConjunctivePredicates(fc)) --
ExpressionSet(splitConjunctivePredicates(nc))).reduceOption(And) match {
case Some(ac) =>
Filter(And(nc, ac), grandChild)
case None =>
nf
}