From 02e61004b9542063bed5a584a9eb5914299ab1c8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 13 Oct 2018 14:17:49 +0800 Subject: [PATCH 1/2] improve the comment inside BooleanSimplification rule --- .../sql/catalyst/optimizer/expressions.scala | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 3c4b284b307f6..e40683ef2c34f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,31 +276,37 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - // The following optimization is applicable only when the operands are not nullable, + // The following optimizations are applicable only when the operands are not nullable, // since the three-value logic of AND and OR are different in NULL handling. // See the chart: // +---------+---------+---------+---------+ - // | p | q | p OR q | p AND q | + // | operand | operand | OR | AND | // +---------+---------+---------+---------+ // | TRUE | TRUE | TRUE | TRUE | // | TRUE | FALSE | TRUE | FALSE | - // | TRUE | UNKNOWN | TRUE | UNKNOWN | - // | FALSE | TRUE | TRUE | FALSE | // | FALSE | FALSE | FALSE | FALSE | - // | FALSE | UNKNOWN | UNKNOWN | FALSE | // | UNKNOWN | TRUE | TRUE | UNKNOWN | // | UNKNOWN | FALSE | UNKNOWN | FALSE | // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | // +---------+---------+---------+---------+ + + // This can break if a is null and c is false, so a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) + // This can break if a is null and b is false, so a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) - case (a Or b) And c if !a.nullable && a.semanticEquals(Not(c)) => And(b, c) - case (a Or b) And c if !b.nullable && b.semanticEquals(Not(c)) => And(a, c) + // This can break if c is null and b is false, so c can't be nullable. + case (a Or b) And c if !c.nullable && a.semanticEquals(Not(c)) => And(b, c) + // This can break if c is null and a is false, so c can't be nullable. + case (a Or b) And c if !c.nullable && b.semanticEquals(Not(c)) => And(a, c) + // This can break if a is null and c is true, so a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) + // This can break if a is null and b is true, so a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) - case (a And b) Or c if !a.nullable && a.semanticEquals(Not(c)) => Or(b, c) - case (a And b) Or c if !b.nullable && b.semanticEquals(Not(c)) => Or(a, c) + // This can break if c is null and b is true, so c can't be nullable. + case (a And b) Or c if !c.nullable && a.semanticEquals(Not(c)) => Or(b, c) + // This can break if c is null and a is true, so c can't be nullable. + case (a And b) Or c if !c.nullable && b.semanticEquals(Not(c)) => Or(a, c) // Common factor elimination for conjunction case and @ (left And right) => From 21dab84d2d54491a1cdd63f91ae5e83727f99526 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 13 Oct 2018 15:09:26 +0800 Subject: [PATCH 2/2] address comments --- .../sql/catalyst/optimizer/expressions.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index e40683ef2c34f..468a950fb1087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -290,22 +290,22 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | // +---------+---------+---------+---------+ - // This can break if a is null and c is false, so a can't be nullable. + // (NULL And (NULL Or FALSE)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) - // This can break if a is null and b is false, so a can't be nullable. + // (NULL And (FALSE Or NULL)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) - // This can break if c is null and b is false, so c can't be nullable. + // ((NULL Or FALSE) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. case (a Or b) And c if !c.nullable && a.semanticEquals(Not(c)) => And(b, c) - // This can break if c is null and a is false, so c can't be nullable. + // ((FALSE Or NULL) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. case (a Or b) And c if !c.nullable && b.semanticEquals(Not(c)) => And(a, c) - // This can break if a is null and c is true, so a can't be nullable. + // (NULL Or (NULL And TRUE)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) - // This can break if a is null and b is true, so a can't be nullable. + // (NULL Or (TRUE And NULL)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) - // This can break if c is null and b is true, so c can't be nullable. + // ((NULL And TRUE) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. case (a And b) Or c if !c.nullable && a.semanticEquals(Not(c)) => Or(b, c) - // This can break if c is null and a is true, so c can't be nullable. + // ((TRUE And NULL) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. case (a And b) Or c if !c.nullable && b.semanticEquals(Not(c)) => Or(a, c) // Common factor elimination for conjunction