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 2020/06/06 08:03:31 UTC

[GitHub] [spark] gengliangwang commented on a change in pull request #28741: [WIP][SPARK-31919][SQL] Push down more predicates through Join

gengliangwang commented on a change in pull request #28741:
URL: https://github.com/apache/spark/pull/28741#discussion_r436249183



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushPredicateThroughJoin.scala
##########
@@ -0,0 +1,248 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, Not, Or, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+trait PushPredicateThroughJoinBase extends Rule[LogicalPlan] with PredicateHelper {
+  protected def enablePushingExtraPredicates: Boolean
+  /**
+   * Splits join condition expressions or filter predicates (on a given join's output) into three
+   * categories based on the attributes required to evaluate them. Note that we explicitly exclude
+   * non-deterministic (i.e., stateful) condition expressions in canEvaluateInLeft or
+   * canEvaluateInRight to prevent pushing these predicates on either side of the join.
+   *
+   * @return (canEvaluateInLeft, canEvaluateInRight, haveToEvaluateInBoth)
+   */
+  private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = {
+    val (pushDownCandidates, nonDeterministic) = condition.partition(_.deterministic)
+    val (leftEvaluateCondition, rest) =
+      pushDownCandidates.partition(_.references.subsetOf(left.outputSet))
+    val (rightEvaluateCondition, commonCondition) =
+        rest.partition(expr => expr.references.subsetOf(right.outputSet))
+
+    // For the predicates in `commonCondition`, it is still possible to find sub-predicates which
+    // are able to be pushed down.
+    val leftExtraCondition = if (enablePushingExtraPredicates) {
+      commonCondition.flatMap(convertibleFilter(_, left.outputSet, canPartialPushDown = true))
+    } else {
+      Seq.empty
+    }
+    val rightExtraCondition = if (enablePushingExtraPredicates) {
+      commonCondition.flatMap(convertibleFilter(_, right.outputSet, canPartialPushDown = true))
+    } else {
+      Seq.empty
+    }
+
+    // To avoid expanding the join condition into conjunctive normal form and making the size
+    // of codegen much larger, `commonCondition` will be kept as original form in the new join
+    // condition.
+    (leftEvaluateCondition ++ leftExtraCondition, rightEvaluateCondition ++ rightExtraCondition,
+      commonCondition ++ nonDeterministic)
+  }
+
+  private def convertibleFilter(
+    condition: Expression,
+    outputSet: AttributeSet,
+    canPartialPushDown: Boolean): Option[Expression] = condition match {
+    // At here, it is not safe to just convert one side and remove the other side
+    // if we do not understand what the parent filters are.
+    //
+    // Here is an example used to explain the reason.
+    // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to
+    // convert b in ('1'). If we only convert a = 2, we will end up with a filter
+    // NOT(a = 2), which will generate wrong results.
+    //
+    // Pushing one side of AND down is only safe to do at the top level or in the child
+    // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate
+    // can be safely removed.
+    case And(left, right) =>
+      val leftResultOptional = convertibleFilter(left, outputSet, canPartialPushDown)
+      val rightResultOptional = convertibleFilter(right, outputSet, canPartialPushDown)
+      (leftResultOptional, rightResultOptional) match {
+        case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult))
+        case (Some(leftResult), None) if canPartialPushDown => Some(leftResult)
+        case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult)
+        case _ => None
+      }
+
+    // The Or predicate is convertible when both of its children can be pushed down.
+    // That is to say, if one/both of the children can be partially pushed down, the Or
+    // predicate can be partially pushed down as well.
+    //
+    // Here is an example used to explain the reason.
+    // Let's say we have
+    // (a1 AND a2) OR (b1 AND b2),
+    // a1 and b1 is convertible, while a2 and b2 is not.
+    // The predicate can be converted as
+    // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2)
+    // As per the logical in And predicate, we can push down (a1 OR b1).
+    case Or(left, right) =>
+      for {
+        lhs <- convertibleFilter(left, outputSet, canPartialPushDown)
+        rhs <- convertibleFilter(right, outputSet, canPartialPushDown)
+      } yield Or(lhs, rhs)
+
+    case Not(pred) =>
+      val childResultOptional = convertibleFilter(pred, outputSet, canPartialPushDown = false)
+      childResultOptional.map(Not)
+
+    case other =>
+      if (other.references.subsetOf(outputSet)) {
+        Some(other)
+      } else {
+        None
+      }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally
+
+  val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = {

Review comment:
       This method is copied from the original one, except that the join condition is not changed when `enablePushingExtraPredicates` is true




----------------------------------------------------------------
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