You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/19 15:08:47 UTC

[GitHub] [arrow] pitrou commented on a diff in pull request #12891: ARROW-12659: [C++] Support is_valid as a guarantee

pitrou commented on code in PR #12891:
URL: https://github.com/apache/arrow/pull/12891#discussion_r853115172


##########
cpp/src/arrow/util/vector.h:
##########
@@ -77,9 +77,14 @@ std::vector<T> ReplaceVectorElement(const std::vector<T>& values, size_t index,
 }
 
 template <typename T, typename Predicate>
-std::vector<T> FilterVector(std::vector<T> values, Predicate&& predicate) {
-  auto new_end =
-      std::remove_if(values.begin(), values.end(), std::forward<Predicate>(predicate));
+std::vector<T> FilterVector(std::vector<T> values, Predicate&& predicate,
+                            std::vector<T>* filtered_out = NULLPTR) {

Review Comment:
   I may be missing something, but it does not seem this third argument is used anywhere?



##########
cpp/src/arrow/compute/kernels/scalar_validity_test.cc:
##########
@@ -48,6 +48,23 @@ TEST_F(TestBooleanValidityKernels, ArrayIsValid) {
                    "[false, true, true, false]");
 }
 
+TEST_F(TestBooleanValidityKernels, TrueUnlessNull) {
+  CheckScalarUnary("true_unless_null", type_singleton(), "[]", type_singleton(), "[]");
+  CheckScalarUnary("true_unless_null", type_singleton(), "[null]", type_singleton(),
+                   "[null]");
+  CheckScalarUnary("true_unless_null", type_singleton(), "[0, 1]", type_singleton(),
+                   "[true, true]");
+  CheckScalarUnary("true_unless_null", type_singleton(), "[null, 1, 0, null]",
+                   type_singleton(), "[null, true, true, null]");
+}
+
+TEST_F(TestBooleanValidityKernels, IsValidIsNullNullType) {
+  CheckScalarUnary("is_null", std::make_shared<NullArray>(5),
+                   ArrayFromJSON(boolean(), "[true, true, true, true, true]"));
+  CheckScalarUnary("is_valid", std::make_shared<NullArray>(5),
+                   ArrayFromJSON(boolean(), "[false, false, false, false, false]"));

Review Comment:
   Can we also test "true_unless_null" here?



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -305,19 +308,47 @@ bool Expression::IsNullLiteral() const {
   return false;
 }
 
-bool Expression::IsSatisfiable() const {
-  if (type() && type()->id() == Type::NA) {
-    return false;
+namespace {
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  DCHECK_NE(call.function, nullptr);
+  if (call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
   }
+  return util::nullopt;
+}
+}  // namespace
+
+bool Expression::IsSatisfiable() const {
+  if (!type()) return true;
+  if (type()->id() != Type::BOOL) return true;
 
   if (auto lit = literal()) {
     if (lit->null_count() == lit->length()) {
       return false;
     }
 
-    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+    if (lit->is_scalar()) {
       return lit->scalar_as<BooleanScalar>().value;
     }
+
+    return true;
+  }
+
+  if (field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  if (call->function_name == "invert") {

Review Comment:
   Can you add a comment explaining why this is useful?



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;

Review Comment:
   Would you like to add a comment explaining what the terms are? Is it `target <cmp> bound` or `bound <cmp> target`?
   



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;
+
+  // Extract an Inequality if possible, derived from "less",
+  // "greater", "less_equal", and "greater_equal" expressions,
+  // possibly disjuncted with an "is_null" Expression.
+  // cmp(a, 2)
+  // cmp(a, 2) or is_null(a)
+  static util::optional<Inequality> ExtractOne(const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
+
+    if (call->function_name == "or_kleene") {
+      // expect the LHS to be a usable field inequality
+      auto out = ExtractOneFromComparison(call->arguments[0]);
+      if (!out) return util::nullopt;
+
+      // expect the RHS to be an is_null expression
+      auto call_rhs = call->arguments[1].call();
+      if (!call_rhs) return util::nullopt;
+      if (call_rhs->function_name != "is_null") return util::nullopt;
+
+      // ... and that it references the same target
+      auto target = call_rhs->arguments[0].field_ref();
+      if (!target) return util::nullopt;
+      if (*target != out->target) return util::nullopt;
+
+      out->nullable = true;
+      return out;
+    }
 
-        // Ensure both calls are comparisons with equal LHS and scalar RHS
-        auto cmp = Comparison::Get(expr);
-        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+    // fall back to a simple comparison with no "is_null"
+    return ExtractOneFromComparison(guarantee);
+  }
 
-        if (!cmp) return expr;
-        if (!cmp_guarantee) return expr;
+  static util::optional<Inequality> ExtractOneFromComparison(
+      const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
 
-        const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
-        const auto& guarantee_lhs = guarantee.arguments[0];
-        if (lhs != guarantee_lhs) return expr;
+    if (auto cmp = Comparison::Get(call->function_name)) {
+      // not_equal comparisons are not very usable as guarantees
+      if (*cmp == Comparison::NOT_EQUAL) return util::nullopt;
 
-        auto rhs = call->arguments[1].literal();
-        auto guarantee_rhs = guarantee.arguments[1].literal();
+      auto target = call->arguments[0].field_ref();
+      if (!target) return util::nullopt;
 
-        if (!rhs) return expr;
-        if (!rhs->is_scalar()) return expr;
+      auto bound = call->arguments[1].literal();
+      if (!bound) return util::nullopt;
+      if (!bound->is_scalar()) return util::nullopt;
 
-        if (!guarantee_rhs) return expr;
-        if (!guarantee_rhs->is_scalar()) return expr;
+      return Inequality{*cmp, /*target=*/*target, *bound, /*nullable=*/false};
+    }
 
-        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
-                              Comparison::Execute(*rhs, *guarantee_rhs));
-        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+    return util::nullopt;
+  }
 
-        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
-          // RHS of filter is equal to RHS of guarantee
+  /// The given expression simplifies to `value` if the inequality
+  /// target is not nullable. Otherwise, it simplifies to either a
+  /// call to true_unless_null or !true_unless_null.
+  Result<Expression> simplified_to(const Expression& bound_target, bool value) const {
+    if (!nullable) return literal(value);
+
+    ExecContext exec_context;
+
+    // Data may be null, so comparison will yield `value` - or null IFF the data was null
+    //
+    // true_unless_null is cheap; it purely reuses the validity bitmap for the values
+    // buffer. Inversion is less cheap but we expect that term never to be evaluated
+    // since invert(true_unless_null(x)) is not satisfiable.
+    Expression::Call call;
+    call.function_name = "true_unless_null";
+    call.arguments = {bound_target};
+    ARROW_ASSIGN_OR_RAISE(
+        auto true_unless_null,
+        BindNonRecursive(std::move(call),
+                         /*insert_implicit_casts=*/false, &exec_context));
+    if (value) return true_unless_null;
+
+    Expression::Call invert;
+    invert.function_name = "invert";
+    invert.arguments = {std::move(true_unless_null)};
+    return BindNonRecursive(std::move(invert),
+                            /*insert_implicit_casts=*/false, &exec_context);
+  }
 
-          if ((*cmp & *cmp_guarantee) == *cmp_guarantee) {
-            // guarantee is a subset of filter, so all data will be included
-            // x > 1, x >= 1, x != 1 guaranteed by x > 1
-            return literal(true);
-          }
+  /// \brief Simplify the given expression given this inequality as a guarantee.
+  Result<Expression> Simplify(Expression expr) {
+    const auto& guarantee = *this;
 
-          if ((*cmp & *cmp_guarantee) == 0) {
-            // guarantee disjoint with filter, so all data will be excluded
-            // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
-            return literal(false);
-          }
+    auto call = expr.call();
+    if (!call) return expr;
 
-          return expr;
-        }
+    auto cmp = Comparison::Get(expr);
+    if (!cmp) return expr;
 
-        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
-          // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
-          return expr;
-        }
+    auto rhs = call->arguments[1].literal();
+    if (!rhs) return expr;
+    if (!rhs->is_scalar()) return expr;
 
-        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
-          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
-          return literal(true);
-        } else {
-          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
-          return literal(false);
-        }
+    const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
+    if (!lhs.field_ref()) return expr;
+    if (*lhs.field_ref() != guarantee.target) return expr;
+
+    ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_bound, Comparison::Execute(*rhs, guarantee.bound));
+    DCHECK_NE(cmp_rhs_bound, Comparison::NA);
+
+    if (cmp_rhs_bound == Comparison::EQUAL) {
+      // RHS of filter is equal to RHS of guarantee
+
+      if ((*cmp & guarantee.cmp) == guarantee.cmp) {
+        // guarantee is a subset of filter, so all data will be included
+        // x > 1, x >= 1, x != 1 guaranteed by x > 1
+        return simplified_to(lhs, true);
+      }
+
+      if ((*cmp & guarantee.cmp) == 0) {
+        // guarantee disjoint with filter, so all data will be excluded
+        // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
+        return simplified_to(lhs, false);
+      }
+
+      return expr;
+    }
+
+    if (guarantee.cmp & cmp_rhs_bound) {
+      // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3

Review Comment:
   Perhaps (with `rhs` being 1 and `bound` being 0):
   ```c++
   // x > 1, x >= 1, x != 1 cannot use guarantee x >= 0
   // (where `guarantee.cmp` is GREATER_EQUAL, `cmp_rhs_bound` is GREATER)
   ```
   



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;
+
+  // Extract an Inequality if possible, derived from "less",
+  // "greater", "less_equal", and "greater_equal" expressions,
+  // possibly disjuncted with an "is_null" Expression.
+  // cmp(a, 2)
+  // cmp(a, 2) or is_null(a)
+  static util::optional<Inequality> ExtractOne(const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
+
+    if (call->function_name == "or_kleene") {
+      // expect the LHS to be a usable field inequality
+      auto out = ExtractOneFromComparison(call->arguments[0]);
+      if (!out) return util::nullopt;
+
+      // expect the RHS to be an is_null expression
+      auto call_rhs = call->arguments[1].call();
+      if (!call_rhs) return util::nullopt;
+      if (call_rhs->function_name != "is_null") return util::nullopt;
+
+      // ... and that it references the same target
+      auto target = call_rhs->arguments[0].field_ref();
+      if (!target) return util::nullopt;
+      if (*target != out->target) return util::nullopt;
+
+      out->nullable = true;
+      return out;
+    }
 
-        // Ensure both calls are comparisons with equal LHS and scalar RHS
-        auto cmp = Comparison::Get(expr);
-        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+    // fall back to a simple comparison with no "is_null"
+    return ExtractOneFromComparison(guarantee);
+  }
 
-        if (!cmp) return expr;
-        if (!cmp_guarantee) return expr;
+  static util::optional<Inequality> ExtractOneFromComparison(
+      const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
 
-        const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
-        const auto& guarantee_lhs = guarantee.arguments[0];
-        if (lhs != guarantee_lhs) return expr;
+    if (auto cmp = Comparison::Get(call->function_name)) {
+      // not_equal comparisons are not very usable as guarantees
+      if (*cmp == Comparison::NOT_EQUAL) return util::nullopt;
 
-        auto rhs = call->arguments[1].literal();
-        auto guarantee_rhs = guarantee.arguments[1].literal();
+      auto target = call->arguments[0].field_ref();
+      if (!target) return util::nullopt;
 
-        if (!rhs) return expr;
-        if (!rhs->is_scalar()) return expr;
+      auto bound = call->arguments[1].literal();
+      if (!bound) return util::nullopt;
+      if (!bound->is_scalar()) return util::nullopt;
 
-        if (!guarantee_rhs) return expr;
-        if (!guarantee_rhs->is_scalar()) return expr;
+      return Inequality{*cmp, /*target=*/*target, *bound, /*nullable=*/false};
+    }
 
-        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
-                              Comparison::Execute(*rhs, *guarantee_rhs));
-        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+    return util::nullopt;
+  }
 
-        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
-          // RHS of filter is equal to RHS of guarantee
+  /// The given expression simplifies to `value` if the inequality
+  /// target is not nullable. Otherwise, it simplifies to either a
+  /// call to true_unless_null or !true_unless_null.
+  Result<Expression> simplified_to(const Expression& bound_target, bool value) const {
+    if (!nullable) return literal(value);
+
+    ExecContext exec_context;
+
+    // Data may be null, so comparison will yield `value` - or null IFF the data was null
+    //
+    // true_unless_null is cheap; it purely reuses the validity bitmap for the values
+    // buffer. Inversion is less cheap but we expect that term never to be evaluated
+    // since invert(true_unless_null(x)) is not satisfiable.
+    Expression::Call call;
+    call.function_name = "true_unless_null";
+    call.arguments = {bound_target};
+    ARROW_ASSIGN_OR_RAISE(
+        auto true_unless_null,
+        BindNonRecursive(std::move(call),
+                         /*insert_implicit_casts=*/false, &exec_context));
+    if (value) return true_unless_null;
+
+    Expression::Call invert;
+    invert.function_name = "invert";
+    invert.arguments = {std::move(true_unless_null)};
+    return BindNonRecursive(std::move(invert),
+                            /*insert_implicit_casts=*/false, &exec_context);
+  }
 
-          if ((*cmp & *cmp_guarantee) == *cmp_guarantee) {
-            // guarantee is a subset of filter, so all data will be included
-            // x > 1, x >= 1, x != 1 guaranteed by x > 1
-            return literal(true);
-          }
+  /// \brief Simplify the given expression given this inequality as a guarantee.
+  Result<Expression> Simplify(Expression expr) {
+    const auto& guarantee = *this;
 
-          if ((*cmp & *cmp_guarantee) == 0) {
-            // guarantee disjoint with filter, so all data will be excluded
-            // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
-            return literal(false);
-          }
+    auto call = expr.call();
+    if (!call) return expr;
 
-          return expr;
-        }
+    auto cmp = Comparison::Get(expr);
+    if (!cmp) return expr;
 
-        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
-          // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
-          return expr;
-        }
+    auto rhs = call->arguments[1].literal();
+    if (!rhs) return expr;
+    if (!rhs->is_scalar()) return expr;
 
-        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
-          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
-          return literal(true);
-        } else {
-          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
-          return literal(false);
-        }
+    const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
+    if (!lhs.field_ref()) return expr;
+    if (*lhs.field_ref() != guarantee.target) return expr;
+
+    ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_bound, Comparison::Execute(*rhs, guarantee.bound));
+    DCHECK_NE(cmp_rhs_bound, Comparison::NA);
+
+    if (cmp_rhs_bound == Comparison::EQUAL) {
+      // RHS of filter is equal to RHS of guarantee
+
+      if ((*cmp & guarantee.cmp) == guarantee.cmp) {
+        // guarantee is a subset of filter, so all data will be included
+        // x > 1, x >= 1, x != 1 guaranteed by x > 1
+        return simplified_to(lhs, true);
+      }
+
+      if ((*cmp & guarantee.cmp) == 0) {
+        // guarantee disjoint with filter, so all data will be excluded
+        // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
+        return simplified_to(lhs, false);
+      }
+
+      return expr;
+    }
+
+    if (guarantee.cmp & cmp_rhs_bound) {
+      // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
+      return expr;
+    }
+
+    if (*cmp & Comparison::GetFlipped(cmp_rhs_bound)) {
+      // x > 1, x >= 1, x != 1 guaranteed by x >= 3

Review Comment:
   Perhaps
   ```suggestion
         // x > 1, x >= 1, x != 1 guaranteed by x >= 3
         // (where `guarantee.cmp` is GREATER_EQUAL, `cmp_rhs_bound` is LESS)
   ```



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -305,19 +308,47 @@ bool Expression::IsNullLiteral() const {
   return false;
 }
 
-bool Expression::IsSatisfiable() const {
-  if (type() && type()->id() == Type::NA) {
-    return false;
+namespace {
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  DCHECK_NE(call.function, nullptr);
+  if (call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
   }
+  return util::nullopt;
+}
+}  // namespace
+
+bool Expression::IsSatisfiable() const {
+  if (!type()) return true;
+  if (type()->id() != Type::BOOL) return true;
 
   if (auto lit = literal()) {
     if (lit->null_count() == lit->length()) {
       return false;
     }
 
-    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+    if (lit->is_scalar()) {
       return lit->scalar_as<BooleanScalar>().value;
     }
+
+    return true;
+  }
+
+  if (field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  if (call->function_name == "invert") {
+    if (auto nested_call = call->arguments[0].call()) {
+      if (nested_call->function_name == "true_unless_null") return false;
+    }
+  }
+
+  if (call->function_name == "and_kleene") {

Review Comment:
   Why specifically "and_kleene" but not "and"? Can you add a comment?



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -128,17 +128,30 @@ util::optional<compute::Expression> ColumnChunkStatisticsAsExpression(
   auto maybe_min = min->CastTo(field->type());
   auto maybe_max = max->CastTo(field->type());
   if (maybe_min.ok() && maybe_max.ok()) {
-    auto col_min = maybe_min.MoveValueUnsafe();
-    auto col_max = maybe_max.MoveValueUnsafe();
-    if (col_min->Equals(col_max)) {
-      return compute::equal(std::move(field_expr), compute::literal(std::move(col_min)));
+    min = maybe_min.MoveValueUnsafe();
+    max = maybe_max.MoveValueUnsafe();
+
+    compute::Expression range;
+    if (min->Equals(max)) {
+      auto single_value = compute::equal(field_expr, compute::literal(std::move(min)));
+
+      if (statistics->null_count() == 0) {
+        return compute::and_(single_value, compute::is_valid(field_expr));

Review Comment:
   Is it useful to add `is_valid` here? If a value is equal to `min` it implies it is valid.



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -128,17 +128,30 @@ util::optional<compute::Expression> ColumnChunkStatisticsAsExpression(
   auto maybe_min = min->CastTo(field->type());
   auto maybe_max = max->CastTo(field->type());
   if (maybe_min.ok() && maybe_max.ok()) {
-    auto col_min = maybe_min.MoveValueUnsafe();
-    auto col_max = maybe_max.MoveValueUnsafe();
-    if (col_min->Equals(col_max)) {
-      return compute::equal(std::move(field_expr), compute::literal(std::move(col_min)));
+    min = maybe_min.MoveValueUnsafe();
+    max = maybe_max.MoveValueUnsafe();
+
+    compute::Expression range;
+    if (min->Equals(max)) {
+      auto single_value = compute::equal(field_expr, compute::literal(std::move(min)));
+
+      if (statistics->null_count() == 0) {
+        return compute::and_(single_value, compute::is_valid(field_expr));
+      }
+      return compute::or_(std::move(single_value), is_null(std::move(field_expr)));
     }
 
     auto lower_bound =
-        compute::greater_equal(field_expr, compute::literal(std::move(col_min)));
-    auto upper_bound =
-        compute::less_equal(std::move(field_expr), compute::literal(std::move(col_max)));
-    return compute::and_(std::move(lower_bound), std::move(upper_bound));
+        compute::greater_equal(field_expr, compute::literal(std::move(min)));
+    auto upper_bound = compute::less_equal(field_expr, compute::literal(std::move(max)));
+
+    if (statistics->null_count() != 0) {
+      lower_bound = compute::or_(std::move(lower_bound), is_null(field_expr));
+      upper_bound = compute::or_(std::move(upper_bound), is_null(std::move(field_expr)));
+      return compute::and_(std::move(lower_bound), std::move(upper_bound));
+    }
+    return compute::and_(compute::and_(std::move(lower_bound), std::move(upper_bound)),
+                         compute::is_valid(field_expr));

Review Comment:
   This seems a bit pointlessly complicated, or I'm missing something? Why not:
   ```suggestion
       auto in_range = compute::and_(std::move(lower_bound), std::move(upper_bound));
       if (statistics->null_count() != 0) {
         return compute::or_(std::move(in_range), compute::is_null(field_expr));
       }
       return in_range;
   ```
   



##########
cpp/src/arrow/dataset/file_parquet.cc:
##########
@@ -128,17 +128,30 @@ util::optional<compute::Expression> ColumnChunkStatisticsAsExpression(
   auto maybe_min = min->CastTo(field->type());
   auto maybe_max = max->CastTo(field->type());
   if (maybe_min.ok() && maybe_max.ok()) {
-    auto col_min = maybe_min.MoveValueUnsafe();
-    auto col_max = maybe_max.MoveValueUnsafe();
-    if (col_min->Equals(col_max)) {
-      return compute::equal(std::move(field_expr), compute::literal(std::move(col_min)));
+    min = maybe_min.MoveValueUnsafe();
+    max = maybe_max.MoveValueUnsafe();
+
+    compute::Expression range;

Review Comment:
   This variable doesn't seem used?



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;
+
+  // Extract an Inequality if possible, derived from "less",
+  // "greater", "less_equal", and "greater_equal" expressions,
+  // possibly disjuncted with an "is_null" Expression.
+  // cmp(a, 2)
+  // cmp(a, 2) or is_null(a)
+  static util::optional<Inequality> ExtractOne(const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
+
+    if (call->function_name == "or_kleene") {
+      // expect the LHS to be a usable field inequality
+      auto out = ExtractOneFromComparison(call->arguments[0]);
+      if (!out) return util::nullopt;
+
+      // expect the RHS to be an is_null expression
+      auto call_rhs = call->arguments[1].call();
+      if (!call_rhs) return util::nullopt;
+      if (call_rhs->function_name != "is_null") return util::nullopt;
+
+      // ... and that it references the same target
+      auto target = call_rhs->arguments[0].field_ref();
+      if (!target) return util::nullopt;
+      if (*target != out->target) return util::nullopt;
+
+      out->nullable = true;
+      return out;
+    }
 
-        // Ensure both calls are comparisons with equal LHS and scalar RHS
-        auto cmp = Comparison::Get(expr);
-        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+    // fall back to a simple comparison with no "is_null"
+    return ExtractOneFromComparison(guarantee);
+  }
 
-        if (!cmp) return expr;
-        if (!cmp_guarantee) return expr;
+  static util::optional<Inequality> ExtractOneFromComparison(
+      const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
 
-        const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
-        const auto& guarantee_lhs = guarantee.arguments[0];
-        if (lhs != guarantee_lhs) return expr;
+    if (auto cmp = Comparison::Get(call->function_name)) {
+      // not_equal comparisons are not very usable as guarantees
+      if (*cmp == Comparison::NOT_EQUAL) return util::nullopt;
 
-        auto rhs = call->arguments[1].literal();
-        auto guarantee_rhs = guarantee.arguments[1].literal();
+      auto target = call->arguments[0].field_ref();
+      if (!target) return util::nullopt;
 
-        if (!rhs) return expr;
-        if (!rhs->is_scalar()) return expr;
+      auto bound = call->arguments[1].literal();
+      if (!bound) return util::nullopt;
+      if (!bound->is_scalar()) return util::nullopt;
 
-        if (!guarantee_rhs) return expr;
-        if (!guarantee_rhs->is_scalar()) return expr;
+      return Inequality{*cmp, /*target=*/*target, *bound, /*nullable=*/false};
+    }
 
-        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
-                              Comparison::Execute(*rhs, *guarantee_rhs));
-        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+    return util::nullopt;
+  }
 
-        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
-          // RHS of filter is equal to RHS of guarantee
+  /// The given expression simplifies to `value` if the inequality
+  /// target is not nullable. Otherwise, it simplifies to either a
+  /// call to true_unless_null or !true_unless_null.
+  Result<Expression> simplified_to(const Expression& bound_target, bool value) const {
+    if (!nullable) return literal(value);
+
+    ExecContext exec_context;
+
+    // Data may be null, so comparison will yield `value` - or null IFF the data was null
+    //
+    // true_unless_null is cheap; it purely reuses the validity bitmap for the values
+    // buffer. Inversion is less cheap but we expect that term never to be evaluated
+    // since invert(true_unless_null(x)) is not satisfiable.
+    Expression::Call call;
+    call.function_name = "true_unless_null";
+    call.arguments = {bound_target};
+    ARROW_ASSIGN_OR_RAISE(
+        auto true_unless_null,
+        BindNonRecursive(std::move(call),
+                         /*insert_implicit_casts=*/false, &exec_context));
+    if (value) return true_unless_null;
+
+    Expression::Call invert;
+    invert.function_name = "invert";
+    invert.arguments = {std::move(true_unless_null)};
+    return BindNonRecursive(std::move(invert),
+                            /*insert_implicit_casts=*/false, &exec_context);
+  }
 
-          if ((*cmp & *cmp_guarantee) == *cmp_guarantee) {
-            // guarantee is a subset of filter, so all data will be included
-            // x > 1, x >= 1, x != 1 guaranteed by x > 1
-            return literal(true);
-          }
+  /// \brief Simplify the given expression given this inequality as a guarantee.
+  Result<Expression> Simplify(Expression expr) {
+    const auto& guarantee = *this;
 
-          if ((*cmp & *cmp_guarantee) == 0) {
-            // guarantee disjoint with filter, so all data will be excluded
-            // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
-            return literal(false);
-          }
+    auto call = expr.call();
+    if (!call) return expr;
 
-          return expr;
-        }
+    auto cmp = Comparison::Get(expr);
+    if (!cmp) return expr;
 
-        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
-          // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
-          return expr;
-        }
+    auto rhs = call->arguments[1].literal();
+    if (!rhs) return expr;
+    if (!rhs->is_scalar()) return expr;
 
-        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
-          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
-          return literal(true);
-        } else {
-          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
-          return literal(false);
-        }
+    const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
+    if (!lhs.field_ref()) return expr;
+    if (*lhs.field_ref() != guarantee.target) return expr;
+
+    ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_bound, Comparison::Execute(*rhs, guarantee.bound));
+    DCHECK_NE(cmp_rhs_bound, Comparison::NA);
+
+    if (cmp_rhs_bound == Comparison::EQUAL) {
+      // RHS of filter is equal to RHS of guarantee
+
+      if ((*cmp & guarantee.cmp) == guarantee.cmp) {
+        // guarantee is a subset of filter, so all data will be included
+        // x > 1, x >= 1, x != 1 guaranteed by x > 1
+        return simplified_to(lhs, true);
+      }
+
+      if ((*cmp & guarantee.cmp) == 0) {
+        // guarantee disjoint with filter, so all data will be excluded
+        // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
+        return simplified_to(lhs, false);
+      }
+
+      return expr;
+    }
+
+    if (guarantee.cmp & cmp_rhs_bound) {

Review Comment:
   This is cryptic, what is this condition supposed to imply?



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;
+
+  // Extract an Inequality if possible, derived from "less",
+  // "greater", "less_equal", and "greater_equal" expressions,
+  // possibly disjuncted with an "is_null" Expression.
+  // cmp(a, 2)
+  // cmp(a, 2) or is_null(a)
+  static util::optional<Inequality> ExtractOne(const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
+
+    if (call->function_name == "or_kleene") {
+      // expect the LHS to be a usable field inequality
+      auto out = ExtractOneFromComparison(call->arguments[0]);
+      if (!out) return util::nullopt;
+
+      // expect the RHS to be an is_null expression
+      auto call_rhs = call->arguments[1].call();
+      if (!call_rhs) return util::nullopt;
+      if (call_rhs->function_name != "is_null") return util::nullopt;
+
+      // ... and that it references the same target
+      auto target = call_rhs->arguments[0].field_ref();
+      if (!target) return util::nullopt;
+      if (*target != out->target) return util::nullopt;
+
+      out->nullable = true;
+      return out;
+    }
 
-        // Ensure both calls are comparisons with equal LHS and scalar RHS
-        auto cmp = Comparison::Get(expr);
-        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+    // fall back to a simple comparison with no "is_null"
+    return ExtractOneFromComparison(guarantee);
+  }
 
-        if (!cmp) return expr;
-        if (!cmp_guarantee) return expr;
+  static util::optional<Inequality> ExtractOneFromComparison(
+      const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
 
-        const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
-        const auto& guarantee_lhs = guarantee.arguments[0];
-        if (lhs != guarantee_lhs) return expr;
+    if (auto cmp = Comparison::Get(call->function_name)) {
+      // not_equal comparisons are not very usable as guarantees
+      if (*cmp == Comparison::NOT_EQUAL) return util::nullopt;
 
-        auto rhs = call->arguments[1].literal();
-        auto guarantee_rhs = guarantee.arguments[1].literal();
+      auto target = call->arguments[0].field_ref();
+      if (!target) return util::nullopt;
 
-        if (!rhs) return expr;
-        if (!rhs->is_scalar()) return expr;
+      auto bound = call->arguments[1].literal();
+      if (!bound) return util::nullopt;
+      if (!bound->is_scalar()) return util::nullopt;
 
-        if (!guarantee_rhs) return expr;
-        if (!guarantee_rhs->is_scalar()) return expr;
+      return Inequality{*cmp, /*target=*/*target, *bound, /*nullable=*/false};
+    }
 
-        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
-                              Comparison::Execute(*rhs, *guarantee_rhs));
-        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+    return util::nullopt;
+  }
 
-        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
-          // RHS of filter is equal to RHS of guarantee
+  /// The given expression simplifies to `value` if the inequality
+  /// target is not nullable. Otherwise, it simplifies to either a
+  /// call to true_unless_null or !true_unless_null.
+  Result<Expression> simplified_to(const Expression& bound_target, bool value) const {
+    if (!nullable) return literal(value);
+
+    ExecContext exec_context;
+
+    // Data may be null, so comparison will yield `value` - or null IFF the data was null
+    //
+    // true_unless_null is cheap; it purely reuses the validity bitmap for the values
+    // buffer. Inversion is less cheap but we expect that term never to be evaluated
+    // since invert(true_unless_null(x)) is not satisfiable.
+    Expression::Call call;
+    call.function_name = "true_unless_null";
+    call.arguments = {bound_target};
+    ARROW_ASSIGN_OR_RAISE(
+        auto true_unless_null,
+        BindNonRecursive(std::move(call),
+                         /*insert_implicit_casts=*/false, &exec_context));
+    if (value) return true_unless_null;
+
+    Expression::Call invert;
+    invert.function_name = "invert";
+    invert.arguments = {std::move(true_unless_null)};
+    return BindNonRecursive(std::move(invert),
+                            /*insert_implicit_casts=*/false, &exec_context);
+  }
 
-          if ((*cmp & *cmp_guarantee) == *cmp_guarantee) {
-            // guarantee is a subset of filter, so all data will be included
-            // x > 1, x >= 1, x != 1 guaranteed by x > 1
-            return literal(true);
-          }
+  /// \brief Simplify the given expression given this inequality as a guarantee.
+  Result<Expression> Simplify(Expression expr) {
+    const auto& guarantee = *this;
 
-          if ((*cmp & *cmp_guarantee) == 0) {
-            // guarantee disjoint with filter, so all data will be excluded
-            // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
-            return literal(false);
-          }
+    auto call = expr.call();
+    if (!call) return expr;
 
-          return expr;
-        }
+    auto cmp = Comparison::Get(expr);
+    if (!cmp) return expr;
 
-        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
-          // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
-          return expr;
-        }
+    auto rhs = call->arguments[1].literal();
+    if (!rhs) return expr;
+    if (!rhs->is_scalar()) return expr;
 
-        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
-          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
-          return literal(true);
-        } else {
-          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
-          return literal(false);
-        }
+    const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
+    if (!lhs.field_ref()) return expr;
+    if (*lhs.field_ref() != guarantee.target) return expr;
+
+    ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_bound, Comparison::Execute(*rhs, guarantee.bound));
+    DCHECK_NE(cmp_rhs_bound, Comparison::NA);
+
+    if (cmp_rhs_bound == Comparison::EQUAL) {
+      // RHS of filter is equal to RHS of guarantee
+
+      if ((*cmp & guarantee.cmp) == guarantee.cmp) {
+        // guarantee is a subset of filter, so all data will be included
+        // x > 1, x >= 1, x != 1 guaranteed by x > 1
+        return simplified_to(lhs, true);
+      }
+
+      if ((*cmp & guarantee.cmp) == 0) {
+        // guarantee disjoint with filter, so all data will be excluded
+        // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
+        return simplified_to(lhs, false);
+      }
+
+      return expr;
+    }
+
+    if (guarantee.cmp & cmp_rhs_bound) {
+      // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3

Review Comment:
   This is contradicted by the next comment below, did you make a mistake?



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;
+
+  // Extract an Inequality if possible, derived from "less",
+  // "greater", "less_equal", and "greater_equal" expressions,
+  // possibly disjuncted with an "is_null" Expression.
+  // cmp(a, 2)
+  // cmp(a, 2) or is_null(a)
+  static util::optional<Inequality> ExtractOne(const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
+
+    if (call->function_name == "or_kleene") {
+      // expect the LHS to be a usable field inequality
+      auto out = ExtractOneFromComparison(call->arguments[0]);
+      if (!out) return util::nullopt;
+
+      // expect the RHS to be an is_null expression
+      auto call_rhs = call->arguments[1].call();
+      if (!call_rhs) return util::nullopt;
+      if (call_rhs->function_name != "is_null") return util::nullopt;
+
+      // ... and that it references the same target
+      auto target = call_rhs->arguments[0].field_ref();
+      if (!target) return util::nullopt;
+      if (*target != out->target) return util::nullopt;
+
+      out->nullable = true;
+      return out;
+    }
 
-        // Ensure both calls are comparisons with equal LHS and scalar RHS
-        auto cmp = Comparison::Get(expr);
-        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+    // fall back to a simple comparison with no "is_null"
+    return ExtractOneFromComparison(guarantee);
+  }
 
-        if (!cmp) return expr;
-        if (!cmp_guarantee) return expr;
+  static util::optional<Inequality> ExtractOneFromComparison(
+      const Expression& guarantee) {
+    auto call = guarantee.call();
+    if (!call) return util::nullopt;
 
-        const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
-        const auto& guarantee_lhs = guarantee.arguments[0];
-        if (lhs != guarantee_lhs) return expr;
+    if (auto cmp = Comparison::Get(call->function_name)) {
+      // not_equal comparisons are not very usable as guarantees
+      if (*cmp == Comparison::NOT_EQUAL) return util::nullopt;
 
-        auto rhs = call->arguments[1].literal();
-        auto guarantee_rhs = guarantee.arguments[1].literal();
+      auto target = call->arguments[0].field_ref();
+      if (!target) return util::nullopt;
 
-        if (!rhs) return expr;
-        if (!rhs->is_scalar()) return expr;
+      auto bound = call->arguments[1].literal();
+      if (!bound) return util::nullopt;
+      if (!bound->is_scalar()) return util::nullopt;
 
-        if (!guarantee_rhs) return expr;
-        if (!guarantee_rhs->is_scalar()) return expr;
+      return Inequality{*cmp, /*target=*/*target, *bound, /*nullable=*/false};
+    }
 
-        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
-                              Comparison::Execute(*rhs, *guarantee_rhs));
-        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+    return util::nullopt;
+  }
 
-        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
-          // RHS of filter is equal to RHS of guarantee
+  /// The given expression simplifies to `value` if the inequality
+  /// target is not nullable. Otherwise, it simplifies to either a
+  /// call to true_unless_null or !true_unless_null.
+  Result<Expression> simplified_to(const Expression& bound_target, bool value) const {
+    if (!nullable) return literal(value);
+
+    ExecContext exec_context;
+
+    // Data may be null, so comparison will yield `value` - or null IFF the data was null
+    //
+    // true_unless_null is cheap; it purely reuses the validity bitmap for the values
+    // buffer. Inversion is less cheap but we expect that term never to be evaluated
+    // since invert(true_unless_null(x)) is not satisfiable.
+    Expression::Call call;
+    call.function_name = "true_unless_null";
+    call.arguments = {bound_target};
+    ARROW_ASSIGN_OR_RAISE(
+        auto true_unless_null,
+        BindNonRecursive(std::move(call),
+                         /*insert_implicit_casts=*/false, &exec_context));
+    if (value) return true_unless_null;
+
+    Expression::Call invert;
+    invert.function_name = "invert";
+    invert.arguments = {std::move(true_unless_null)};
+    return BindNonRecursive(std::move(invert),
+                            /*insert_implicit_casts=*/false, &exec_context);
+  }
 
-          if ((*cmp & *cmp_guarantee) == *cmp_guarantee) {
-            // guarantee is a subset of filter, so all data will be included
-            // x > 1, x >= 1, x != 1 guaranteed by x > 1
-            return literal(true);
-          }
+  /// \brief Simplify the given expression given this inequality as a guarantee.
+  Result<Expression> Simplify(Expression expr) {
+    const auto& guarantee = *this;
 
-          if ((*cmp & *cmp_guarantee) == 0) {
-            // guarantee disjoint with filter, so all data will be excluded
-            // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
-            return literal(false);
-          }
+    auto call = expr.call();
+    if (!call) return expr;
 
-          return expr;
-        }
+    auto cmp = Comparison::Get(expr);
+    if (!cmp) return expr;
 
-        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
-          // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
-          return expr;
-        }
+    auto rhs = call->arguments[1].literal();
+    if (!rhs) return expr;
+    if (!rhs->is_scalar()) return expr;
 
-        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
-          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
-          return literal(true);
-        } else {
-          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
-          return literal(false);
-        }
+    const auto& lhs = Comparison::StripOrderPreservingCasts(call->arguments[0]);
+    if (!lhs.field_ref()) return expr;
+    if (*lhs.field_ref() != guarantee.target) return expr;
+
+    ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_bound, Comparison::Execute(*rhs, guarantee.bound));
+    DCHECK_NE(cmp_rhs_bound, Comparison::NA);
+
+    if (cmp_rhs_bound == Comparison::EQUAL) {
+      // RHS of filter is equal to RHS of guarantee
+
+      if ((*cmp & guarantee.cmp) == guarantee.cmp) {
+        // guarantee is a subset of filter, so all data will be included
+        // x > 1, x >= 1, x != 1 guaranteed by x > 1
+        return simplified_to(lhs, true);
+      }
+
+      if ((*cmp & guarantee.cmp) == 0) {
+        // guarantee disjoint with filter, so all data will be excluded
+        // x > 1, x >= 1, x != 1 unsatisfiable if x == 1
+        return simplified_to(lhs, false);
+      }
+
+      return expr;
+    }
+
+    if (guarantee.cmp & cmp_rhs_bound) {
+      // x > 1, x >= 1, x != 1 cannot use guarantee x >= 3
+      return expr;
+    }
+
+    if (*cmp & Comparison::GetFlipped(cmp_rhs_bound)) {
+      // x > 1, x >= 1, x != 1 guaranteed by x >= 3
+      return simplified_to(lhs, true);
+    } else {
+      // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
+      return simplified_to(lhs, false);
+    }
+  }
+};
+
+/// \brief Simplify an expression given a guarantee, if the guarantee
+///   is is_valid().
+Result<Expression> IsValidSimplification(Expression expr,

Review Comment:
   Make this a verb, for example call it `SimplifyIsValid`.



##########
cpp/src/arrow/compute/exec/expression.cc:
##########
@@ -879,79 +918,183 @@ Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_cont
 
 namespace {
 
-Result<Expression> DirectComparisonSimplification(Expression expr,
-                                                  const Expression::Call& guarantee) {
-  return Modify(
-      std::move(expr), [](Expression expr) { return expr; },
-      [&guarantee](Expression expr, ...) -> Result<Expression> {
-        auto call = expr.call();
-        if (!call) return expr;
+// An inequality comparison which a target Expression is known to satisfy. If nullable,
+// the target may evaluate to null in addition to values satisfying the comparison.
+struct Inequality {
+  Comparison::type cmp;
+  const FieldRef& target;
+  const Datum& bound;
+  bool nullable;

Review Comment:
   Is this "the target can be 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org