@@ -54,10 +54,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
5454 val aliasMap = AttributeMap (duplicates.map { dup =>
5555 dup -> Alias (dup, dup.toString)()
5656 }.toSeq)
57- val aliasedExpressions = right.output.map { ref =>
58- aliasMap.getOrElse(ref, ref)
59- }
60- val newRight = Project (aliasedExpressions, right)
57+ val newRight = rewriteDedupPlan(right, aliasMap)
6158 val newJoinCond = joinCond.map { condExpr =>
6259 condExpr transform {
6360 case a : Attribute => aliasMap.getOrElse(a, a).toAttribute
@@ -70,6 +67,27 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
7067 case _ => joinPlan
7168 }
7269
70+ private def rewriteDedupPlan (plan : LogicalPlan , rewrites : AttributeMap [Alias ]): LogicalPlan = {
71+ val aliasedExpressions = plan.output.map { ref =>
72+ rewrites.getOrElse(ref, ref)
73+ }
74+ Project (aliasedExpressions, plan)
75+ }
76+
77+ private def dedupSubqueryOnSelfJoin (values : Seq [Expression ], sub : LogicalPlan ): LogicalPlan = {
78+ val leftRefs = AttributeSet .fromAttributeSets(values.map(_.references))
79+ val rightRefs = AttributeSet (sub.output)
80+ val duplicates = leftRefs.intersect(rightRefs)
81+ if (duplicates.isEmpty) {
82+ sub
83+ } else {
84+ val aliasMap = AttributeMap (duplicates.map { dup =>
85+ dup -> Alias (dup, dup.toString)()
86+ }.toSeq)
87+ rewriteDedupPlan(sub, aliasMap)
88+ }
89+ }
90+
7391 def apply (plan : LogicalPlan ): LogicalPlan = plan transform {
7492 case Filter (condition, child) =>
7593 val (withSubquery, withoutSubquery) =
@@ -92,18 +110,20 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
92110 // Deduplicate conflicting attributes if any.
93111 dedupJoin(Join (outerPlan, sub, LeftAnti , joinCond))
94112 case (p, InSubquery (values, ListQuery (sub, conditions, _, _))) =>
95- val inConditions = values.zip(sub.output).map(EqualTo .tupled)
113+ val newSub = dedupSubqueryOnSelfJoin(values, sub)
114+ val inConditions = values.zip(newSub.output).map(EqualTo .tupled)
96115 val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p)
97116 // Deduplicate conflicting attributes if any.
98- dedupJoin(Join (outerPlan, sub , LeftSemi , joinCond))
117+ dedupJoin(Join (outerPlan, newSub , LeftSemi , joinCond))
99118 case (p, Not (InSubquery (values, ListQuery (sub, conditions, _, _)))) =>
100119 // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr
101120 // Construct the condition. A NULL in one of the conditions is regarded as a positive
102121 // result; such a row will be filtered out by the Anti-Join operator.
103122
104123 // Note that will almost certainly be planned as a Broadcast Nested Loop join.
105124 // Use EXISTS if performance matters to you.
106- val inConditions = values.zip(sub.output).map(EqualTo .tupled)
125+ val newSub = dedupSubqueryOnSelfJoin(values, sub)
126+ val inConditions = values.zip(newSub.output).map(EqualTo .tupled)
107127 val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions, p)
108128 // Expand the NOT IN expression with the NULL-aware semantic
109129 // to its full form. That is from:
@@ -119,7 +139,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
119139 // (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1
120140 val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And )
121141 // Deduplicate conflicting attributes if any.
122- dedupJoin(Join (outerPlan, sub , LeftAnti , Option (finalJoinCond)))
142+ dedupJoin(Join (outerPlan, newSub , LeftAnti , Option (finalJoinCond)))
123143 case (p, predicate) =>
124144 val (newCond, inputPlan) = rewriteExistentialExpr(Seq (predicate), p)
125145 Project (p.output, Filter (newCond.get, inputPlan))
0 commit comments