[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-02-10 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r376925624
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,27 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = {
+val isNullIntolerant = ifNotNullExpr.find { x =>
+  !x.isInstanceOf[NullIntolerant] && x.find(e => 
e.semanticEquals(checkedExpr)).nonEmpty
 
 Review comment:
   Hmmm, that still looks complicated.. If we cannot avoid the complexity for 
the stronger condition, as another option, I think we can cover the simple case 
(`FiterExec.isNullIntolerant(ifNotNullExpr)`) only in this pr. If necessary, we 
might be able to optimize the condition in future work. I think keeping the 
code simple is more important. WDYT?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-02-06 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r376237241
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,27 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = {
+val isNullIntolerant = ifNotNullExpr.find { x =>
+  !x.isInstanceOf[NullIntolerant] && x.find(e => 
e.semanticEquals(checkedExpr)).nonEmpty
 
 Review comment:
   Ah, I see. For better code readability, could you split the condition into 
the two parts as I suggested above? Also, I think its better to leave some 
comments about why we need more checks there.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-02-04 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r375086787
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,27 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = {
+val isNullIntolerant = ifNotNullExpr.find { x =>
+  !x.isInstanceOf[NullIntolerant] && x.find(e => 
e.semanticEquals(checkedExpr)).nonEmpty
 
 Review comment:
   Probably, you meant `FiterExec.isNullIntolerant(ifNotNullExpr) || additional 
checks for the case having null-tolerant exprs inside ifNotNullExpr`? 
(`FiterExec.isNullIntolerant` is private though...)


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-27 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r371564147
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -442,6 +459,13 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
   case If(cond, trueValue, falseValue)
 if cond.deterministic && trueValue.semanticEquals(falseValue) => 
trueValue
 
+  case i @ If(cond, trueValue, falseValue) => cond match {
+// If the null-check is redundant, remove it
+case IsNull(child) if isRedundantNullCheck(trueValue, falseValue, 
child) => falseValue
+case IsNotNull(child) if isRedundantNullCheck(falseValue, trueValue, 
child) => trueValue
+case _ => i
+  }
 
 Review comment:
   Why did you add the inner pattern-matching (`cond match {` )? I think its 
better to avoid unnecessary pattern matching (In the current fix, all the cases 
for `If` exprs can be matched in the line 466).


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-27 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r371563039
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,27 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = {
+val isNullIntolerant = ifNotNullExpr.find { x =>
+  !x.isInstanceOf[NullIntolerant] && x.find(e => 
e.semanticEquals(checkedExpr)).nonEmpty
 
 Review comment:
   The same logic? 
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala#L105-L109


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-21 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r369317359
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -35,18 +35,24 @@ class SimplifyConditionalSuite extends PlanTest with 
PredicateHelper {
   }
 
   protected def assertEquivalent(e1: Expression, e2: Expression): Unit = {
-val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
OneRowRelation()).analyze
-val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, 
OneRowRelation()).analyze)
+val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
LocalRelation('a.int)).analyze
+val actual = Optimize.execute(
+  Project(Alias(e1, "out")() :: Nil, LocalRelation('a.int)).analyze)
 comparePlans(actual, correctAnswer)
   }
 
   private val trueBranch = (TrueLiteral, Literal(5))
   private val normalBranch = (NonFoldableLiteral(true), Literal(10))
   private val unreachableBranch = (FalseLiteral, Literal(20))
-  private val nullBranch = (Literal.create(null, NullType), Literal(30))
-
-  val isNotNullCond = IsNotNull(UnresolvedAttribute(Seq("a")))
-  val isNullCond = IsNull(UnresolvedAttribute("b"))
+  private val nullValue = Literal.create(null, IntegerType)
 
 Review comment:
   Yea, I think its better to avoid the behaviour changes in the existing tests.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368785396
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -128,7 +192,7 @@ class SimplifyConditionalSuite extends PlanTest with 
PredicateHelper {
 // When the conditions in `CaseWhen` are all deterministic, `CaseWhen` can 
be removed.
 assertEquivalent(
   CaseWhen((isNotNullCond, Subtract(Literal(3), Literal(2))) ::
-(isNullCond, Literal(1)) ::
+(isNullCondB, Literal(1)) ::
 
 Review comment:
   You don't need to change the existing tests where possible.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368785148
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -35,18 +35,24 @@ class SimplifyConditionalSuite extends PlanTest with 
PredicateHelper {
   }
 
   protected def assertEquivalent(e1: Expression, e2: Expression): Unit = {
-val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
OneRowRelation()).analyze
-val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, 
OneRowRelation()).analyze)
+val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
LocalRelation('a.int)).analyze
+val actual = Optimize.execute(
+  Project(Alias(e1, "out")() :: Nil, LocalRelation('a.int)).analyze)
 comparePlans(actual, correctAnswer)
   }
 
   private val trueBranch = (TrueLiteral, Literal(5))
   private val normalBranch = (NonFoldableLiteral(true), Literal(10))
   private val unreachableBranch = (FalseLiteral, Literal(20))
-  private val nullBranch = (Literal.create(null, NullType), Literal(30))
-
-  val isNotNullCond = IsNotNull(UnresolvedAttribute(Seq("a")))
-  val isNullCond = IsNull(UnresolvedAttribute("b"))
+  private val nullValue = Literal.create(null, IntegerType)
 
 Review comment:
   Why did you change from `NullType` to `IntegerType` here?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368784740
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -483,6 +507,19 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 } else {
   e.copy(branches = branches.take(i).map(branch => (branch._1, 
elseValue)))
 }
+
+  case e @ CaseWhen(branches, elseValue) if branches.length == 1 =>
+// remove redundant null checks for CaseWhen with one branch
+branches(0)._1 match {
+  case IsNotNull(child) if isRedundantNullCheck(
+elseValue.getOrElse(Literal.create(null, child.dataType)),
 
 Review comment:
   `child.dataType` -> `e.dataType`?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368783215
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -18,15 +18,15 @@
 package org.apache.spark.sql.catalyst.optimizer
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, 
TrueLiteral}
 import org.apache.spark.sql.catalyst.plans.PlanTest
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.apache.spark.sql.types.{IntegerType, NullType}
 
-
 
 Review comment:
   nit: You need to avoid unnecessary changes like this.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368783122
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -35,18 +35,24 @@ class SimplifyConditionalSuite extends PlanTest with 
PredicateHelper {
   }
 
   protected def assertEquivalent(e1: Expression, e2: Expression): Unit = {
-val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
OneRowRelation()).analyze
-val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, 
OneRowRelation()).analyze)
+val correctAnswer = Project(Alias(e2, "out")() :: Nil, 
LocalRelation('a.int)).analyze
+val actual = Optimize.execute(
+  Project(Alias(e1, "out")() :: Nil, LocalRelation('a.int)).analyze)
 
 Review comment:
   nit: you don't need to break this line.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368782709
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -483,6 +507,19 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 } else {
   e.copy(branches = branches.take(i).map(branch => (branch._1, 
elseValue)))
 }
+
+  case e @ CaseWhen(branches, elseValue) if branches.length == 1 =>
+// remove redundant null checks for CaseWhen with one branch
+branches(0)._1 match {
+  case IsNotNull(child) if isRedundantNullCheck(
+elseValue.getOrElse(Literal.create(null, child.dataType)),
+branches(0)._2, child) => branches(0)._2
+  case IsNull(child) if isRedundantNullCheck(
+branches(0)._2,
+elseValue.getOrElse(Literal.create(null, child.dataType)),
+child) => elseValue.getOrElse(Literal.create(null, child.dataType))
+  case _ => e
+}
 
 Review comment:
   How about this?
   ```
 // remove redundant null checks for CaseWhen with one branch
 case CaseWhen(Seq((IsNotNull(child), trueValue)), Some(falseValue))
   if isRedundantNullCheck(falseValue, trueValue, child) => trueValue
 case CaseWhen(Seq((IsNull(child), trueValue)), Some(falseValue))
   if isRedundantNullCheck(trueValue, falseValue, child) => falseValue
 case CaseWhen(Seq((IsNotNull(child), trueValue)), None)
   if isRedundantNullCheck(Literal.create(null, child.dataType), 
trueValue, child) => trueValue
 case e @ CaseWhen(Seq((IsNull(child), trueValue)), None) =>
   val nullValue = Literal.create(null, child.dataType)
   if (isRedundantNullCheck(trueValue, nullValue, child)) {
 nullValue
   } else {
 e
   }
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368776586
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -442,6 +459,13 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
   case If(cond, trueValue, falseValue)
 if cond.deterministic && trueValue.semanticEquals(falseValue) => 
trueValue
 
+  case i @ If(cond, trueValue, falseValue) => cond match {
+// If the null-check is redundant, remove it
+case IsNull(child) if isRedundantNullCheck(trueValue, falseValue, 
child) => falseValue
+case IsNotNull(child) if isRedundantNullCheck(falseValue, trueValue, 
child) => trueValue
+case _ => i
+  }
 
 Review comment:
   How about this format?;
   ```
 // If the null-check is redundant, remove it
 case If(IsNull(child), trueValue, falseValue)
   if isRedundantNullCheck(trueValue, falseValue, child) => falseValue
 case If(IsNotNull(child), trueValue, falseValue)
   if isRedundantNullCheck(falseValue, trueValue, child) => trueValue
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-20 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368775725
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,22 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = ifNotNullExpr match {
+case e: NullIntolerant if (
+  (ifNullExpr == checkedExpr || ifNullExpr == Literal.create(null, 
checkedExpr.dataType))
+  && e.children.contains(checkedExpr)) => true
+case _ => false
+  }
 
 Review comment:
   Can you generalize the last condition more? e.g., how about the case, 
`substring(other_func(title#5), 0, 3)` in the example you described?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-19 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368369477
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala
 ##
 @@ -18,15 +18,15 @@
 package org.apache.spark.sql.catalyst.optimizer
 
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, 
TrueLiteral}
-import org.apache.spark.sql.catalyst.plans.PlanTest
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.catalyst.plans.{PlanTest}
 
 Review comment:
   nit: `{PlanTest}` -> `PlanTest`


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-19 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368369817
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,22 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = ifNotNullExpr match {
+case e: NullIntolerant if (
+  (ifNullExpr == checkedExpr || ifNullExpr == Literal.create(null, 
checkedExpr.dataType))
+  && e.children.contains(checkedExpr)) => true
+case _ => false
+  }
 
 Review comment:
   The first condition `ifNullExpr == checkedExpr` -> 
`ifNullExpr.semanticEquals(checkedExpr)`? e.g., `if isnull(a + b)  b + a else 
xxx`


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-19 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368369402
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,22 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = ifNotNullExpr match {
+case e: NullIntolerant if (
+  (ifNullExpr == checkedExpr || ifNullExpr == Literal.create(null, 
checkedExpr.dataType))
+  && e.children.contains(checkedExpr)) => true
+case _ => false
+  }
 
 Review comment:
   nit: How about this style?
   ```
 private def isRedundantNullCheck(
 ifNullExpr: Expression,
 ifNotNullExpr: Expression,
 checkedExpr: Expression): Boolean = {
   ifNotNullExpr.isInstanceOf[NullIntolerant] && {
 (ifNullExpr == checkedExpr || ifNullExpr == Literal.create(null, 
checkedExpr.dataType)) &&
   ifNotNullExpr.children.contains(checkedExpr)
   }
 }
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove redundant null checks

2020-01-19 Thread GitBox
maropu commented on a change in pull request #27231: [SPARK-28478][SQL] Remove 
redundant null checks
URL: https://github.com/apache/spark/pull/27231#discussion_r368370049
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ##
 @@ -434,6 +434,22 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 case _ => false
   }
 
+  /**
+   * Condition for redundant null check based on intolerant expressions.
+   * @param ifNullExpr expression that takes place if checkedExpr is null
+   * @param ifNotNullExpr expression that takes place if checkedExpr is not 
null
+   * @param checkedExpr expression that is checked for null value
+   */
+  private def isRedundantNullCheck(
+  ifNullExpr: Expression,
+  ifNotNullExpr: Expression,
+  checkedExpr: Expression): Boolean = ifNotNullExpr match {
+case e: NullIntolerant if (
+  (ifNullExpr == checkedExpr || ifNullExpr == Literal.create(null, 
checkedExpr.dataType))
+  && e.children.contains(checkedExpr)) => true
+case _ => false
+  }
 
 Review comment:
   The second condition `ifNullExpr == Literal.create(null, 
checkedExpr.dataType)` -> `ifNullExpr.foldable && ifNullExpr.eval() == null`?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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