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 2022/06/21 07:16:18 UTC

[GitHub] [spark] cloud-fan commented on a diff in pull request #36850: [SPARK-39069][SQL] Pushing EqualTo with Literal to other conditions

cloud-fan commented on code in PR #36850:
URL: https://github.com/apache/spark/pull/36850#discussion_r902241229


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -1502,19 +1502,47 @@ object EliminateSorts extends Rule[LogicalPlan] {
  * Removes filters that can be evaluated trivially.  This can be done through the following ways:
  * 1) by eliding the filter for cases where it will always evaluate to `true`.
  * 2) by substituting a dummy empty relation when the filter will always evaluate to `false`.
- * 3) by eliminating the always-true conditions given the constraints on the child's output.
+ * 3) by pushing EqualTo with Literal to other conditions.
+ * 4) by eliminating the always-true conditions given the constraints on the child's output.
  */
 object PruneFilters extends Rule[LogicalPlan] with PredicateHelper {
   def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(
     _.containsPattern(FILTER), ruleId) {
     // If the filter condition always evaluate to true, remove the filter.
-    case Filter(Literal(true, BooleanType), child) => child
+    case Filter(Literal.TrueLiteral, child) => child
     // If the filter condition always evaluate to null or false,
     // replace the input with an empty relation.
     case Filter(Literal(null, _), child) =>
       LocalRelation(child.output, data = Seq.empty, isStreaming = plan.isStreaming)
-    case Filter(Literal(false, BooleanType), child) =>
+    case Filter(Literal.FalseLiteral, child) =>
       LocalRelation(child.output, data = Seq.empty, isStreaming = plan.isStreaming)
+    case f @ Filter(condition, _: LeafNode) =>
+      val predicates = splitConjunctivePredicates(condition)
+      val equalToWithLiterals = predicates.collect {
+        case eq @ EqualTo(a: Attribute, l: Literal) => eq -> (a -> l)
+        case eq @ EqualTo(l: Literal, a: Attribute) => eq -> (a -> l)
+      }
+      if (equalToWithLiterals.nonEmpty) {
+        val defaultEqualKeyValues = AttributeMap(equalToWithLiterals.map(_._2))
+        val newCondition = predicates.map {
+          // Don't push to IsNotNull because InferFiltersFromConstraints may infer another IsNotNull
+          case isNotNull: IsNotNull => isNotNull
+          // Exclude the current EqualTo, and push. e.g.: a = 1 and a = 2 ==> 2 = 1 and 1 = 2

Review Comment:
   can you explain the high-level idea? are you trying to detect conflicting equal to predicates and turn them into false literal?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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