Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -373,6 +373,9 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper {
case Not(a And b) => Or(Not(a), Not(b))

case Not(Not(e)) => e

case Not(expr: IsNull) => IsNotNull(expr.child)
case Not(expr: IsNotNull) => IsNull(expr.child)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. expr -> e.

case Not(e: IsNull) => IsNotNull(e.child)
case Not(e: IsNotNull) => IsNull(e.child)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. expr -> e.

How about

      case Not(IsNull(e)) => IsNotNull(e)
      case Not(IsNotNull(e)) => IsNull(e)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better!

}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
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}
Expand All @@ -40,10 +41,19 @@ class SimplifyConditionalSuite extends PlanTest with PredicateHelper {
comparePlans(actual, correctAnswer)
}

private def assertFilter(originalExpr: Expression, expectedExpr: Expression): Unit = {
val originalPlan = testRelation.where(originalExpr).analyze
val optimizedPlan = Optimize.execute(originalPlan)
val expectedPlan = testRelation.where(expectedExpr).analyze
comparePlans(optimizedPlan, expectedPlan)
}

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))
private val testRelation =
LocalRelation('i.int, 'b.boolean, 'a.array(IntegerType), 'm.map(IntegerType, IntegerType))

val isNotNullCond = IsNotNull(UnresolvedAttribute(Seq("a")))
val isNullCond = IsNull(UnresolvedAttribute("b"))
Expand Down Expand Up @@ -166,4 +176,9 @@ class SimplifyConditionalSuite extends PlanTest with PredicateHelper {
Literal(1))
)
}

test("simplify NOT(IsNull(x)) and NOT(IsNotNull(x))") {
assertFilter(Not(IsNotNull(UnresolvedAttribute("b"))), IsNull(UnresolvedAttribute("b")))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We cannot use assertEquivalent here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We cannot use assertEquivalent here?

assertEquivalent use OneRowRelation, no column, so I add a testRelation.
I will remove it to BooleanSimplificationSuit .

assertFilter(Not(IsNull(UnresolvedAttribute("b"))), IsNotNull(UnresolvedAttribute("b")))
}
}
13 changes: 13 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3192,6 +3192,19 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession {
checkAnswer(df3, Array(Row(new java.math.BigDecimal("0.100000000000000000000000100"))))
}
}

test("SPARK-29152: Simplify NOT(IsNull(x)) and NOT(IsNotNull(x))") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I personally think the SimplifyConditionalSuite test is enough for this fix.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I personally think the SimplifyConditionalSuite test is enough for this fix.

Yeah, seems end-to-end test in SQLQuerySuit is redundant

withTempView("tbl1") {
val df: DataFrame =
Seq[java.lang.Boolean](true, false, true, null, false, null, true).toDF("id")
df.createOrReplaceTempView("tbl1")
val query1 = sql("select id from tbl1 where not(isnull(id) or id == false)")
val query2 = sql("select id from tbl1 where not(isnotnull(id) and id == true) ")

checkAnswer(query1, Row(true) :: Row(true) :: Row(true) :: Nil)
checkAnswer(query2, Row(false) :: Row(null) :: Row(false) :: Row(null) :: Nil)
}
}
}

case class Foo(bar: Option[String])