[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-29 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1151346831


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -113,15 +114,13 @@ object ConstantPropagation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(
 _.containsAllPatterns(LITERAL, FILTER), ruleId) {
 case f: Filter =>
-  val (newCondition, _) = traverse(f.condition, replaceChildren = true, 
nullIsFalse = true)
-  if (newCondition.isDefined) {
-f.copy(condition = newCondition.get)
-  } else {
-f
-  }
+  f.mapExpressions(traverse(_, nullIsFalse = true, None))

Review Comment:
   This is our internal change:
   ```scala
   object ConstantPropagation extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(
   _.containsAllPatterns(LITERAL, FILTER), ruleId) {
   case f: Filter =>
 val (newCondition, _) = traverse(f.condition, replaceChildren = true, 
nullIsFalse = true)
 if (newCondition.isDefined) {
   f.copy(condition = newCondition.get)
 } else {
   f
 }
 }
   
 /**
  * Traverse a condition as a tree and replace attributes with constant 
values.
  * - On matching [[And]], recursively traverse each children and get 
propagated mappings.
  *   If the current node is not child of another [[And]], replace all 
occurrences of the
  *   attributes with the corresponding constant values.
  * - If a child of [[And]] is [[EqualTo]] or [[EqualNullSafe]], propagate 
the mapping
  *   of attribute => constant.
  * - On matching [[Or]] or [[Not]], recursively traverse each children, 
propagate empty mapping.
  * - Otherwise, stop traversal and propagate empty mapping.
  * @param condition condition to be traversed
  * @param replaceChildren whether to replace attributes with constant 
values in children
  * @param nullIsFalse whether a boolean expression result can be 
considered to false e.g. in the
  *case of `WHERE e`, null result of expression `e` 
means the same as if it
  *resulted false
  * @return A tuple including:
  * 1. Option[Expression]: optional changed condition after 
traversal
  * 2. AttributeMap: propagated mapping of attribute => constant
  */
 private def traverse(condition: Expression, replaceChildren: Boolean, 
nullIsFalse: Boolean)
   : (Option[Expression], AttributeMap[(Literal, BinaryComparison)]) =
   condition match {
 case e @ EqualTo(left: AttributeReference, right: Literal)
   if safeToReplace(left, nullIsFalse) =>
   (None, AttributeMap(Map(left -> (right, e
 case e @ EqualTo(left: Literal, right: AttributeReference)
   if safeToReplace(right, nullIsFalse) =>
   (None, AttributeMap(Map(right -> (left, e
 case e @ EqualNullSafe(left: AttributeReference, right: Literal)
   if safeToReplace(left, nullIsFalse) =>
   (None, AttributeMap(Map(left -> (right, e
 case e @ EqualNullSafe(left: Literal, right: AttributeReference)
   if safeToReplace(right, nullIsFalse) =>
   (None, AttributeMap(Map(right -> (left, e
 case a: And =>
   val (newLeft, equalityPredicatesLeft) =
 traverse(a.left, replaceChildren = false, nullIsFalse)
   val (newRight, equalityPredicatesRight) =
 traverse(a.right, replaceChildren = false, nullIsFalse)
   val equalityPredicates = equalityPredicatesLeft ++ 
equalityPredicatesRight
   val newSelf = if (equalityPredicates.nonEmpty && replaceChildren) {
 Some(And(replaceConstants(newLeft.getOrElse(a.left), 
equalityPredicates),
   replaceConstants(newRight.getOrElse(a.right), 
equalityPredicates)))
   } else {
 if (newLeft.isDefined || newRight.isDefined) {
   Some(And(newLeft.getOrElse(a.left), newRight.getOrElse(a.right)))
 } else {
   None
 }
   }
   (newSelf, equalityPredicates)
 case o: Or =>
   // Ignore the EqualityPredicates from children since they are only 
propagated through And.
   val (newLeft, _) = traverse(o.left, replaceChildren = true, 
nullIsFalse)
   val (newRight, _) = traverse(o.right, replaceChildren = true, 
nullIsFalse)
   val newSelf = if (newLeft.isDefined || newRight.isDefined) {
 Some(Or(left = newLeft.getOrElse(o.left), right = 
newRight.getOrElse((o.right
   } else {
 None
   }
   (newSelf, AttributeMap.empty)
 case n: Not =>
   // Ignore the EqualityPredicates from children since they are only 
propagated through And.
   val (newChild, _) = 

[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-28 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1151346831


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -113,15 +114,13 @@ object ConstantPropagation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(
 _.containsAllPatterns(LITERAL, FILTER), ruleId) {
 case f: Filter =>
-  val (newCondition, _) = traverse(f.condition, replaceChildren = true, 
nullIsFalse = true)
-  if (newCondition.isDefined) {
-f.copy(condition = newCondition.get)
-  } else {
-f
-  }
+  f.mapExpressions(traverse(_, nullIsFalse = true, None))

Review Comment:
   This is our internal change:
   ```scala
   object ConstantPropagation extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(
   _.containsAllPatterns(LITERAL, FILTER), ruleId) {
   case f: Filter =>
 val (newCondition, _) = traverse(f.condition, replaceChildren = true, 
nullIsFalse = true)
 if (newCondition.isDefined) {
   f.copy(condition = newCondition.get)
 } else {
   f
 }
 }
   
 /**
  * Traverse a condition as a tree and replace attributes with constant 
values.
  * - On matching [[And]], recursively traverse each children and get 
propagated mappings.
  *   If the current node is not child of another [[And]], replace all 
occurrences of the
  *   attributes with the corresponding constant values.
  * - If a child of [[And]] is [[EqualTo]] or [[EqualNullSafe]], propagate 
the mapping
  *   of attribute => constant.
  * - On matching [[Or]] or [[Not]], recursively traverse each children, 
propagate empty mapping.
  * - Otherwise, stop traversal and propagate empty mapping.
  * @param condition condition to be traversed
  * @param replaceChildren whether to replace attributes with constant 
values in children
  * @param nullIsFalse whether a boolean expression result can be 
considered to false e.g. in the
  *case of `WHERE e`, null result of expression `e` 
means the same as if it
  *resulted false
  * @return A tuple including:
  * 1. Option[Expression]: optional changed condition after 
traversal
  * 2. EqualityPredicates: propagated mapping of attribute => 
constant
  */
 private def traverse(condition: Expression, replaceChildren: Boolean, 
nullIsFalse: Boolean)
   : (Option[Expression], AttributeMap[(Literal, BinaryComparison)]) =
   condition match {
 case e @ EqualTo(left: AttributeReference, right: Literal)
   if safeToReplace(left, nullIsFalse) =>
   (None, AttributeMap(Map(left -> (right, e
 case e @ EqualTo(left: Literal, right: AttributeReference)
   if safeToReplace(right, nullIsFalse) =>
   (None, AttributeMap(Map(right -> (left, e
 case e @ EqualNullSafe(left: AttributeReference, right: Literal)
   if safeToReplace(left, nullIsFalse) =>
   (None, AttributeMap(Map(left -> (right, e
 case e @ EqualNullSafe(left: Literal, right: AttributeReference)
   if safeToReplace(right, nullIsFalse) =>
   (None, AttributeMap(Map(right -> (left, e
 case a: And =>
   val (newLeft, equalityPredicatesLeft) =
 traverse(a.left, replaceChildren = false, nullIsFalse)
   val (newRight, equalityPredicatesRight) =
 traverse(a.right, replaceChildren = false, nullIsFalse)
   val equalityPredicates = equalityPredicatesLeft ++ 
equalityPredicatesRight
   val newSelf = if (equalityPredicates.nonEmpty && replaceChildren) {
 Some(And(replaceConstants(newLeft.getOrElse(a.left), 
equalityPredicates),
   replaceConstants(newRight.getOrElse(a.right), 
equalityPredicates)))
   } else {
 if (newLeft.isDefined || newRight.isDefined) {
   Some(And(newLeft.getOrElse(a.left), newRight.getOrElse(a.right)))
 } else {
   None
 }
   }
   (newSelf, equalityPredicates)
 case o: Or =>
   // Ignore the EqualityPredicates from children since they are only 
propagated through And.
   val (newLeft, _) = traverse(o.left, replaceChildren = true, 
nullIsFalse)
   val (newRight, _) = traverse(o.right, replaceChildren = true, 
nullIsFalse)
   val newSelf = if (newLeft.isDefined || newRight.isDefined) {
 Some(Or(left = newLeft.getOrElse(o.left), right = 
newRight.getOrElse((o.right
   } else {
 None
   }
   (newSelf, AttributeMap.empty)
 case n: Not =>
   // Ignore the EqualityPredicates from children since they are only 
propagated through And.
   val (newChild, 

[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1127193046


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -138,56 +136,53 @@ object ConstantPropagation extends Rule[LogicalPlan] {
*case of `WHERE e`, null result of expression `e` means 
the same as if it
*resulted false
* @return A tuple including:
-   * 1. Option[Expression]: optional changed condition after traversal
+   * 1. Expression: optional changed condition after traversal

Review Comment:
   Remove the `optional`?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1126255588


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -198,16 +192,15 @@ object ConstantPropagation extends Rule[LogicalPlan] {
   private def safeToReplace(ar: AttributeReference, nullIsFalse: Boolean) =
 !ar.nullable || nullIsFalse
 
-  private def replaceConstants(condition: Expression, equalityPredicates: 
EqualityPredicates)
-: Expression = {
-val constantsMap = AttributeMap(equalityPredicates.map(_._1))
-val predicates = equalityPredicates.map(_._2).toSet
-def replaceConstants0(expression: Expression) = expression transform {
-  case a: AttributeReference => constantsMap.getOrElse(a, a)
-}
+  private def replaceConstants(
+  condition: Expression,
+  equalityPredicates: EqualityPredicates): Expression = {
+val predicates = equalityPredicates.values.map(_._2).toSet
 condition transform {
-  case e @ EqualTo(_, _) if !predicates.contains(e) => replaceConstants0(e)
-  case e @ EqualNullSafe(_, _) if !predicates.contains(e) => 
replaceConstants0(e)
+  case b: BinaryComparison if !predicates.contains(b) => b transform {
+case a: AttributeReference if 
equalityPredicates.contains(a.canonicalized) =>
+  equalityPredicates(a.canonicalized)._1

Review Comment:
   ```scala
   case a: AttributeReference => constantsMap.getOrElse(a, a)
   ```
   ?



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1126254824


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -198,16 +192,15 @@ object ConstantPropagation extends Rule[LogicalPlan] {
   private def safeToReplace(ar: AttributeReference, nullIsFalse: Boolean) =
 !ar.nullable || nullIsFalse
 
-  private def replaceConstants(condition: Expression, equalityPredicates: 
EqualityPredicates)
-: Expression = {
-val constantsMap = AttributeMap(equalityPredicates.map(_._1))
-val predicates = equalityPredicates.map(_._2).toSet
-def replaceConstants0(expression: Expression) = expression transform {
-  case a: AttributeReference => constantsMap.getOrElse(a, a)
-}
+  private def replaceConstants(
+  condition: Expression,
+  equalityPredicates: EqualityPredicates): Expression = {
+val predicates = equalityPredicates.values.map(_._2).toSet

Review Comment:
   Can we keep the `constantsMap`?
   ```scala
   val constantsMap = AttributeMap(equalityPredicates.map { case (attr, (lit, 
_)) => attr -> lit })
   ```



-- 
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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1126250472


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -138,56 +136,52 @@ object ConstantPropagation extends Rule[LogicalPlan] {
*case of `WHERE e`, null result of expression `e` means 
the same as if it
*resulted false
* @return A tuple including:
-   * 1. Option[Expression]: optional changed condition after traversal
+   * 1. Expression: optional changed condition after traversal
* 2. EqualityPredicates: propagated mapping of attribute => constant
*/
-  private def traverse(condition: Expression, replaceChildren: Boolean, 
nullIsFalse: Boolean)
-: (Option[Expression], EqualityPredicates) =
+  private def traverse(
+  condition: Expression,
+  replaceChildren: Boolean,
+  nullIsFalse: Boolean): (Expression, EqualityPredicates) =
 condition match {
   case e @ EqualTo(left: AttributeReference, right: Literal)
 if safeToReplace(left, nullIsFalse) =>
-(None, Seq(((left, right), e)))
+e -> mutable.Map(left.canonicalized -> (right, e))
   case e @ EqualTo(left: Literal, right: AttributeReference)
 if safeToReplace(right, nullIsFalse) =>
-(None, Seq(((right, left), e)))
+e -> mutable.Map(right.canonicalized -> (left, e))
   case e @ EqualNullSafe(left: AttributeReference, right: Literal)
 if safeToReplace(left, nullIsFalse) =>
-(None, Seq(((left, right), e)))
+e -> mutable.Map(left.canonicalized -> (right, e))
   case e @ EqualNullSafe(left: Literal, right: AttributeReference)
 if safeToReplace(right, nullIsFalse) =>
-(None, Seq(((right, left), e)))
-  case a: And =>
-val (newLeft, equalityPredicatesLeft) =
-  traverse(a.left, replaceChildren = false, nullIsFalse)
+e -> mutable.Map(right.canonicalized -> (left, e))
+  case a @ And(left, right) =>
+val (newLeft, equalityPredicates) =
+  traverse(left, replaceChildren = false, nullIsFalse)
 val (newRight, equalityPredicatesRight) =
-  traverse(a.right, replaceChildren = false, nullIsFalse)
-val equalityPredicates = equalityPredicatesLeft ++ 
equalityPredicatesRight
-val newSelf = if (equalityPredicates.nonEmpty && replaceChildren) {
-  Some(And(replaceConstants(newLeft.getOrElse(a.left), 
equalityPredicates),
-replaceConstants(newRight.getOrElse(a.right), equalityPredicates)))
+  traverse(right, replaceChildren = false, nullIsFalse)
+// We could recognize when conflicting constants are coming from the 
left and right sides
+// and immediately shortcut the `And` expression to 
`Literal.FalseLiteral`, but that case is
+// not so common and actually it is the job of `ConstantFolding` and 
`BooleanSimplification`
+// rules to deal with those optimizations.
+equalityPredicates ++= equalityPredicatesRight
+val newAnd = a.withNewChildren(if (equalityPredicates.nonEmpty && 
replaceChildren) {
+  val replacedNewLeft = replaceConstants(newLeft, equalityPredicates)
+  val replacedNewRight = replaceConstants(newRight, equalityPredicates)
+  Seq(replacedNewLeft, replacedNewRight)
 } else {
-  if (newLeft.isDefined || newRight.isDefined) {
-Some(And(newLeft.getOrElse(a.left), newRight.getOrElse(a.right)))
-  } else {
-None
-  }
-}
-(newSelf, equalityPredicates)
+  Seq(newLeft, newRight)
+})
+newAnd -> equalityPredicates
   case o: Or =>
 // Ignore the EqualityPredicates from children since they are only 
propagated through And.
-val (newLeft, _) = traverse(o.left, replaceChildren = true, 
nullIsFalse)
-val (newRight, _) = traverse(o.right, replaceChildren = true, 
nullIsFalse)
-val newSelf = if (newLeft.isDefined || newRight.isDefined) {
-  Some(Or(left = newLeft.getOrElse(o.left), right = 
newRight.getOrElse((o.right
-} else {
-  None
-}
-(newSelf, Seq.empty)
+o.mapChildren(traverse(_, replaceChildren = true, nullIsFalse)._1) -> 
mutable.Map.empty
   case n: Not =>
 // Ignore the EqualityPredicates from children since they are only 
propagated through And.
-val (newChild, _) = traverse(n.child, replaceChildren = true, 
nullIsFalse = false)
-(newChild.map(Not), Seq.empty)
-  case _ => (None, Seq.empty)
+n.mapChildren(traverse(_, replaceChildren = true, nullIsFalse = 
false)._1) ->

Review Comment:
   ```scala
   val newNot = n.mapChildren(traverse(_, replaceChildren = true, nullIsFalse = 
false)._1)
   newNot -> AttributeMap.empty
   ```
   ?



-- 
This is an automated message from the Apache Git Service.

[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1126249818


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -138,56 +136,52 @@ object ConstantPropagation extends Rule[LogicalPlan] {
*case of `WHERE e`, null result of expression `e` means 
the same as if it
*resulted false
* @return A tuple including:
-   * 1. Option[Expression]: optional changed condition after traversal
+   * 1. Expression: optional changed condition after traversal
* 2. EqualityPredicates: propagated mapping of attribute => constant
*/
-  private def traverse(condition: Expression, replaceChildren: Boolean, 
nullIsFalse: Boolean)
-: (Option[Expression], EqualityPredicates) =
+  private def traverse(
+  condition: Expression,
+  replaceChildren: Boolean,
+  nullIsFalse: Boolean): (Expression, EqualityPredicates) =
 condition match {
   case e @ EqualTo(left: AttributeReference, right: Literal)
 if safeToReplace(left, nullIsFalse) =>
-(None, Seq(((left, right), e)))
+e -> mutable.Map(left.canonicalized -> (right, e))
   case e @ EqualTo(left: Literal, right: AttributeReference)
 if safeToReplace(right, nullIsFalse) =>
-(None, Seq(((right, left), e)))
+e -> mutable.Map(right.canonicalized -> (left, e))
   case e @ EqualNullSafe(left: AttributeReference, right: Literal)
 if safeToReplace(left, nullIsFalse) =>
-(None, Seq(((left, right), e)))
+e -> mutable.Map(left.canonicalized -> (right, e))
   case e @ EqualNullSafe(left: Literal, right: AttributeReference)
 if safeToReplace(right, nullIsFalse) =>
-(None, Seq(((right, left), e)))
-  case a: And =>
-val (newLeft, equalityPredicatesLeft) =
-  traverse(a.left, replaceChildren = false, nullIsFalse)
+e -> mutable.Map(right.canonicalized -> (left, e))
+  case a @ And(left, right) =>
+val (newLeft, equalityPredicates) =
+  traverse(left, replaceChildren = false, nullIsFalse)
 val (newRight, equalityPredicatesRight) =
-  traverse(a.right, replaceChildren = false, nullIsFalse)
-val equalityPredicates = equalityPredicatesLeft ++ 
equalityPredicatesRight
-val newSelf = if (equalityPredicates.nonEmpty && replaceChildren) {
-  Some(And(replaceConstants(newLeft.getOrElse(a.left), 
equalityPredicates),
-replaceConstants(newRight.getOrElse(a.right), equalityPredicates)))
+  traverse(right, replaceChildren = false, nullIsFalse)
+// We could recognize when conflicting constants are coming from the 
left and right sides
+// and immediately shortcut the `And` expression to 
`Literal.FalseLiteral`, but that case is
+// not so common and actually it is the job of `ConstantFolding` and 
`BooleanSimplification`
+// rules to deal with those optimizations.
+equalityPredicates ++= equalityPredicatesRight
+val newAnd = a.withNewChildren(if (equalityPredicates.nonEmpty && 
replaceChildren) {
+  val replacedNewLeft = replaceConstants(newLeft, equalityPredicates)
+  val replacedNewRight = replaceConstants(newRight, equalityPredicates)
+  Seq(replacedNewLeft, replacedNewRight)
 } else {
-  if (newLeft.isDefined || newRight.isDefined) {
-Some(And(newLeft.getOrElse(a.left), newRight.getOrElse(a.right)))
-  } else {
-None
-  }
-}
-(newSelf, equalityPredicates)
+  Seq(newLeft, newRight)
+})
+newAnd -> equalityPredicates
   case o: Or =>
 // Ignore the EqualityPredicates from children since they are only 
propagated through And.
-val (newLeft, _) = traverse(o.left, replaceChildren = true, 
nullIsFalse)
-val (newRight, _) = traverse(o.right, replaceChildren = true, 
nullIsFalse)
-val newSelf = if (newLeft.isDefined || newRight.isDefined) {
-  Some(Or(left = newLeft.getOrElse(o.left), right = 
newRight.getOrElse((o.right
-} else {
-  None
-}
-(newSelf, Seq.empty)
+o.mapChildren(traverse(_, replaceChildren = true, nullIsFalse)._1) -> 
mutable.Map.empty

Review Comment:
   ```scala
   val newOr = o.mapChildren(traverse(_, replaceChildren = true, 
nullIsFalse)._1)
   newOr -> AttributeMap.empty
   ```
   ?



-- 
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-unsubscr...@spark.apache.org

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


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

[GitHub] [spark] wangyum commented on a diff in pull request #40268: [SPARK-42500][SQL] ConstantPropagation support more cases

2023-03-06 Thread via GitHub


wangyum commented on code in PR #40268:
URL: https://github.com/apache/spark/pull/40268#discussion_r1126248967


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala:
##
@@ -112,16 +113,13 @@ object ConstantFolding extends Rule[LogicalPlan] {
 object ConstantPropagation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(
 _.containsAllPatterns(LITERAL, FILTER), ruleId) {
-case f: Filter =>
-  val (newCondition, _) = traverse(f.condition, replaceChildren = true, 
nullIsFalse = true)
-  if (newCondition.isDefined) {
-f.copy(condition = newCondition.get)
-  } else {
-f
-  }
+case f: Filter => f.mapExpressions(traverse(_, replaceChildren = true, 
nullIsFalse = true)._1)
   }
 
-  type EqualityPredicates = Seq[((AttributeReference, Literal), 
BinaryComparison)]
+  // The keys are always canonicalized `AttributeReference`s, but it is easier 
to use `Expression`
+  // type keys here instead of casting `AttributeReference.canonicalized` to 
`AttributeReference` at
+  // the calling sites.
+  type EqualityPredicates = mutable.Map[Expression, (Literal, 
BinaryComparison)]

Review Comment:
   I think `AttributeReference` is enough. 
   
   Can we use `AttributeMap` ? In order to avoid the use of `x.canonicalized` 
later:
   ```scala
   type EqualityPredicates = AttributeMap[(Literal, BinaryComparison)]
   ```



-- 
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-unsubscr...@spark.apache.org

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


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