You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/04 21:38:51 UTC

[GitHub] [spark] c21 commented on a change in pull request #31739: [SPARK-34622][SQL] Fix push down limit through join

c21 commented on a change in pull request #31739:
URL: https://github.com/apache/spark/pull/31739#discussion_r587844932



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -597,31 +623,11 @@ object LimitPushDown extends Rule[LogicalPlan] {
     // pushdown Limit.
     case LocalLimit(exp, u: Union) =>
       LocalLimit(exp, u.copy(children = u.children.map(maybePushLocalLimit(exp, _))))
-    // Add extra limits below JOIN:
-    // 1. For LEFT OUTER and RIGHT OUTER JOIN, we push limits to the left and right sides,
-    //    respectively.
-    // 2. For INNER and CROSS JOIN, we push limits to both the left and right sides if join
-    //    condition is empty.
-    // 3. For LEFT SEMI and LEFT ANTI JOIN, we push limits to the left side if join condition
-    //    is empty.
-    // 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, conditionOpt, _)) =>
-      val newJoin = joinType match {
-        case RightOuter => join.copy(right = maybePushLocalLimit(exp, right))
-        case LeftOuter => join.copy(left = maybePushLocalLimit(exp, left))
-        case _: InnerLike if conditionOpt.isEmpty =>
-          join.copy(
-            left = maybePushLocalLimit(exp, left),
-            right = maybePushLocalLimit(exp, right))
-        case LeftSemi | LeftAnti if conditionOpt.isEmpty =>
-          join.copy(left = maybePushLocalLimit(exp, left))
-        case _ => join
-      }
-      LocalLimit(exp, newJoin)
+
+    case LocalLimit(exp, join: Join) =>
+      LocalLimit(exp, pushLocalLimitThroughJoin(exp, join))
+    case LocalLimit(exp, project @ Project(_, join: Join)) =>

Review comment:
       Thanks @wangyum for adding this. I think there might be a list of operators safe to push through besides `Project` - e.g. `Sort`, `RepartitionByExpression`, `ScriptTransformation`, etc.
   
   Shall we add push down through `Project` separately? It should not be only restricted to `Project(Join)` right? It can be `Project(OtherOperator)` as well?
   
   ```
   case LocalLimit(exp, p: Project) =>
     LocalLimit(exp, p.copy(maybePushLocalLimit(exp, _)))
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org