cloud-fan commented on a change in pull request #27252: [SPARK-29231][SQL] 
Constraints should be inferred from cast equality constraint
URL: https://github.com/apache/spark/pull/27252#discussion_r369556787
 
 

 ##########
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala
 ##########
 @@ -62,11 +62,22 @@ trait ConstraintHelper {
    */
   def inferAdditionalConstraints(constraints: Set[Expression]): 
Set[Expression] = {
     var inferredConstraints = Set.empty[Expression]
-    constraints.foreach {
+    val binaryComparisons = 
constraints.filter(_.isInstanceOf[BinaryComparison])
+    binaryComparisons.foreach {
       case eq @ EqualTo(l: Attribute, r: Attribute) =>
-        val candidateConstraints = constraints - eq
+        val candidateConstraints = binaryComparisons - eq
         inferredConstraints ++= replaceConstraints(candidateConstraints, l, r)
         inferredConstraints ++= replaceConstraints(candidateConstraints, r, l)
+      case eq @ EqualTo(l @ Cast(lc: Attribute, _, tz), r: Attribute) =>
+        val candidateConstraints = binaryComparisons - eq
+        val bridge = Cast(r, lc.dataType, tz)
+        inferredConstraints ++= replaceConstraints(candidateConstraints, r, l)
+        inferredConstraints ++= replaceConstraints(candidateConstraints, lc, 
bridge)
 
 Review comment:
   is this safe? cast is not reversible. `cast(int_col as long) = long_col` can 
have different result than `int_col = cast(long_col as int)`.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to