diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
index f14df93160b756ff6c02f4f54cff1def4ba6cc96..b32374c5742ef62ba3c387484f2fcf23953add8c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
@@ -24,6 +24,7 @@ import scala.math.BigDecimal.RoundingMode
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.CatalystConf
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Filter, LeafNode, Statistics}
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.types._
@@ -104,12 +105,23 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         val percent2 = calculateFilterSelectivity(cond2, update = false).getOrElse(1.0)
         Some(percent1 + percent2 - (percent1 * percent2))
 
+      // Not-operator pushdown
       case Not(And(cond1, cond2)) =>
         calculateFilterSelectivity(Or(Not(cond1), Not(cond2)), update = false)
 
+      // Not-operator pushdown
       case Not(Or(cond1, cond2)) =>
         calculateFilterSelectivity(And(Not(cond1), Not(cond2)), update = false)
 
+      // Collapse two consecutive Not operators which could be generated after Not-operator pushdown
+      case Not(Not(cond)) =>
+        calculateFilterSelectivity(cond, update = false)
+
+      // The foldable Not has been processed in the ConstantFolding rule
+      // This is a top-down traversal. The Not could be pushed down by the above two cases.
+      case Not(l @ Literal(null, _)) =>
+        calculateSingleCondition(l, update = false)
+
       case Not(cond) =>
         calculateFilterSelectivity(cond, update = false) match {
           case Some(percent) => Some(1.0 - percent)
@@ -134,13 +146,16 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
    */
   def calculateSingleCondition(condition: Expression, update: Boolean): Option[Double] = {
     condition match {
+      case l: Literal =>
+        evaluateLiteral(l)
+
       // For evaluateBinary method, we assume the literal on the right side of an operator.
       // So we will change the order if not.
 
       // EqualTo/EqualNullSafe does not care about the order
-      case op @ Equality(ar: Attribute, l: Literal) =>
+      case Equality(ar: Attribute, l: Literal) =>
         evaluateEquality(ar, l, update)
-      case op @ Equality(l: Literal, ar: Attribute) =>
+      case Equality(l: Literal, ar: Attribute) =>
         evaluateEquality(ar, l, update)
 
       case op @ LessThan(ar: Attribute, l: Literal) =>
@@ -342,6 +357,26 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
 
   }
 
+  /**
+   * Returns a percentage of rows meeting a Literal expression.
+   * This method evaluates all the possible literal cases in Filter.
+   *
+   * FalseLiteral and TrueLiteral should be eliminated by optimizer, but null literal might be added
+   * by optimizer rule NullPropagation. For safety, we handle all the cases here.
+   *
+   * @param literal a literal value (or constant)
+   * @return an optional double value to show the percentage of rows meeting a given condition
+   */
+  def evaluateLiteral(literal: Literal): Option[Double] = {
+    literal match {
+      case Literal(null, _) => Some(0.0)
+      case FalseLiteral => Some(0.0)
+      case TrueLiteral => Some(1.0)
+      // Ideally, we should not hit the following branch
+      case _ => None
+    }
+  }
+
   /**
    * Returns a percentage of rows meeting "IN" operator expression.
    * This method evaluates the equality predicate for all data types.
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
index 07abe1ed285335986cf45bb141964936e16a86ab..1966c96c0529498eca11bf1043fa6a2777a1328d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.statsEstimation
 import java.sql.Date
 
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
 import org.apache.spark.sql.catalyst.plans.LeftOuter
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Filter, Join, Statistics}
 import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
@@ -76,6 +77,82 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
     attrDouble -> colStatDouble,
     attrString -> colStatString))
 
+  test("true") {
+    validateEstimatedStats(
+      Filter(TrueLiteral, childStatsTestPlan(Seq(attrInt), 10L)),
+      Seq(attrInt -> colStatInt),
+      expectedRowCount = 10)
+  }
+
+  test("false") {
+    validateEstimatedStats(
+      Filter(FalseLiteral, childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("null") {
+    validateEstimatedStats(
+      Filter(Literal(null, IntegerType), childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("Not(null)") {
+    validateEstimatedStats(
+      Filter(Not(Literal(null, IntegerType)), childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("Not(Not(null))") {
+    validateEstimatedStats(
+      Filter(Not(Not(Literal(null, IntegerType))), childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("cint < 3 AND null") {
+    val condition = And(LessThan(attrInt, Literal(3)), Literal(null, IntegerType))
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("cint < 3 OR null") {
+    val condition = Or(LessThan(attrInt, Literal(3)), Literal(null, IntegerType))
+    val m = Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)).stats(conf)
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Seq(attrInt -> colStatInt),
+      expectedRowCount = 3)
+  }
+
+  test("Not(cint < 3 AND null)") {
+    val condition = Not(And(LessThan(attrInt, Literal(3)), Literal(null, IntegerType)))
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Seq(attrInt -> colStatInt),
+      expectedRowCount = 8)
+  }
+
+  test("Not(cint < 3 OR null)") {
+    val condition = Not(Or(LessThan(attrInt, Literal(3)), Literal(null, IntegerType)))
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
+  test("Not(cint < 3 AND Not(null))") {
+    val condition = Not(And(LessThan(attrInt, Literal(3)), Not(Literal(null, IntegerType))))
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Seq(attrInt -> colStatInt),
+      expectedRowCount = 8)
+  }
+
   test("cint = 2") {
     validateEstimatedStats(
       Filter(EqualTo(attrInt, Literal(2)), childStatsTestPlan(Seq(attrInt), 10L)),
@@ -163,6 +240,16 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
       expectedRowCount = 10)
   }
 
+  test("cint IS NOT NULL && null") {
+    // 'cint < null' will be optimized to 'cint IS NOT NULL && null'.
+    // More similar cases can be found in the Optimizer NullPropagation.
+    val condition = And(IsNotNull(attrInt), Literal(null, IntegerType))
+    validateEstimatedStats(
+      Filter(condition, childStatsTestPlan(Seq(attrInt), 10L)),
+      Nil,
+      expectedRowCount = 0)
+  }
+
   test("cint > 3 AND cint <= 6") {
     val condition = And(GreaterThan(attrInt, Literal(3)), LessThanOrEqual(attrInt, Literal(6)))
     validateEstimatedStats(