wgtmac commented on code in PR #402:
URL: https://github.com/apache/iceberg-cpp/pull/402#discussion_r2605309544


##########
src/iceberg/expression/residual_evaluator.h:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+/// \file iceberg/expression/residual_evaluator.h
+/// Residual evaluator for finding residual expressions after partition 
evaluation.
+
+#include <memory>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class Expression;
+class StructLike;
+
+/// \brief Finds the residuals for an Expression using the partitions in the 
given
+/// PartitionSpec.
+///
+/// A residual expression is made by partially evaluating an expression using 
partition
+/// values. For example, if a table is partitioned by day(utc_timestamp) and 
is read
+/// with a filter expression utc_timestamp >= a and utc_timestamp <= b, then 
there are
+/// 4 possible residual expressions for the partition data, d:
+///
+/// - If d > day(a) and d < day(b), the residual is always true
+/// - If d == day(a) and d != day(b), the residual is utc_timestamp >= a
+/// - If d == day(b) and d != day(a), the residual is utc_timestamp <= b
+/// - If d == day(a) == day(b), the residual is utc_timestamp >= a and 
utc_timestamp <= b
+///
+/// Partition data is passed using StructLike. Residuals are returned by 
ResidualFor().
+class ICEBERG_EXPORT ResidualEvaluator {
+ public:
+  /// \brief Return a residual evaluator for an unpartitioned PartitionSpec.
+  ///
+  /// \param expr An expression
+  /// \return A residual evaluator that always returns the expression
+  static Result<std::unique_ptr<ResidualEvaluator>> Unpartitioned(
+      std::shared_ptr<Expression> expr);
+
+  /// \brief Return a residual evaluator for a PartitionSpec and Expression.
+  ///
+  /// \param spec A partition spec
+  /// \param schema The schema to bind expressions against
+  /// \param expr An expression
+  /// \param case_sensitive Whether field name matching is case-sensitive
+  /// \return A residual evaluator for the expression
+  static Result<std::unique_ptr<ResidualEvaluator>> Make(
+      const std::shared_ptr<PartitionSpec>& spec, const 
std::shared_ptr<Schema>& schema,
+      std::shared_ptr<Expression> expr, bool case_sensitive = true);
+
+  ~ResidualEvaluator();
+
+  /// \brief Returns a residual expression for the given partition values.
+  ///
+  /// \param partition_data Partition data values
+  /// \return The residual of this evaluator's expression from the partition 
values
+  virtual Result<std::shared_ptr<Expression>> ResidualFor(
+      const StructLike& partition_data) const;
+
+ protected:
+  ResidualEvaluator(std::shared_ptr<Expression> expr,
+                    const std::shared_ptr<PartitionSpec>& spec,
+                    const std::shared_ptr<Schema>& schema, bool 
case_sensitive);

Review Comment:
   Should we just use `const PartitionSpec& spec` and `const Schema& schema`?



##########
src/iceberg/expression/residual_evaluator.h:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+/// \file iceberg/expression/residual_evaluator.h
+/// Residual evaluator for finding residual expressions after partition 
evaluation.
+
+#include <memory>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class Expression;
+class StructLike;

Review Comment:
   ```suggestion
   ```



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));

Review Comment:
   We shouldn't throw in the ctor. Please check this in the `Make` function and 
make sure all inputs are valid to the ctor.



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));

Review Comment:
   ```suggestion
       ICEBERG_ASSIGN_OR_THROW(auto partition_type, 
spec_->PartitionType(*schema));
   ```



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));
+    partition_schema_ = FromStructType(std::move(*partition_type_), 
std::nullopt);
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysTrue() override {
+    return Expressions::AlwaysTrue();
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysFalse() override {
+    return Expressions::AlwaysFalse();
+  }
+
+  Result<std::shared_ptr<Expression>> Not(
+      const std::shared_ptr<Expression>& child_result) override {
+    return Expressions::Not(child_result);
+  }
+
+  Result<std::shared_ptr<Expression>> And(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::And(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> Or(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::Or(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return IsNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNaN(const std::shared_ptr<Bound>& 
expr) override {
+    return IsNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNaN(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> Lt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return LtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> LtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return LtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Gt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return GtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> GtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return GtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Eq(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return EqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotEq(const std::shared_ptr<Bound>& expr,
+                                            const Literal& lit) override {
+    return NotEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> StartsWith(const std::shared_ptr<Bound>& 
expr,
+                                                 const Literal& lit) override {
+    return StartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotStartsWith(const 
std::shared_ptr<Bound>& expr,
+                                                    const Literal& lit) 
override {
+    return NotStartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> In(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return InImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> NotIn(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return NotInImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<BoundPredicate>& pred) override;
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<UnboundPredicate>& pred) override {
+    ICEBERG_ASSIGN_OR_RAISE(auto bound_predicate, pred->Bind(*schema_, 
case_sensitive_));
+    if (bound_predicate->is_bound_predicate()) {
+      ICEBERG_ASSIGN_OR_RAISE(
+          auto residual,
+          
Predicate(std::dynamic_pointer_cast<BoundPredicate>(bound_predicate)));
+      if (residual->is_bound_predicate()) {
+        // replace inclusive original unbound predicate
+        return pred;
+      }
+      return residual;
+    }
+    // if binding didn't result in a Predicate, return the expression
+    return bound_predicate;
+  }
+
+ private:
+  // Helper methods for bound predicates
+  Result<std::shared_ptr<Expression>> EvaluateBoundPredicate(
+      const std::shared_ptr<BoundPredicate>& pred);
+
+  Result<std::shared_ptr<Expression>> IsNullImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNullImpl(const 
std::shared_ptr<Bound>& expr);
+  Result<std::shared_ptr<Expression>> IsNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> LtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> LtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> EqImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                                const Literal& lit);
+  Result<std::shared_ptr<Expression>> InImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> NotInImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> StartsWithImpl(const 
std::shared_ptr<Bound>& expr,
+                                                     const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotStartsWithImpl(
+      const std::shared_ptr<Bound>& expr, const Literal& lit);
+
+  std::shared_ptr<PartitionSpec> spec_;
+  std::shared_ptr<Schema> schema_;
+  std::shared_ptr<Schema> partition_schema_;
+  const StructLike& partition_data_;
+  bool case_sensitive_;
+};
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::IsNullImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNull()) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();

Review Comment:
   We can make all these functions are a one-liner if we do:
   
   ```cpp
   namespace {
   std::shared_ptr<Expression> always_true() { return True::Instance(); }
   std::shared_ptr<Expression> always_false() { return False::Instance(); }
   }  // namespace
   
     Result<std::shared_ptr<Expression>> IsNull(
         const std::shared_ptr<Bound>& expr) override {
       return expr->Evaluate(partition_data_).transform([](const auto& value) {
         return value.IsNull() ? always_true() : always_false();
       });
     }
   
   ```



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));
+    partition_schema_ = FromStructType(std::move(*partition_type_), 
std::nullopt);
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysTrue() override {
+    return Expressions::AlwaysTrue();
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysFalse() override {
+    return Expressions::AlwaysFalse();
+  }
+
+  Result<std::shared_ptr<Expression>> Not(
+      const std::shared_ptr<Expression>& child_result) override {
+    return Expressions::Not(child_result);
+  }
+
+  Result<std::shared_ptr<Expression>> And(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::And(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> Or(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::Or(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return IsNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNaN(const std::shared_ptr<Bound>& 
expr) override {
+    return IsNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNaN(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> Lt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return LtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> LtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return LtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Gt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return GtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> GtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return GtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Eq(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return EqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotEq(const std::shared_ptr<Bound>& expr,
+                                            const Literal& lit) override {
+    return NotEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> StartsWith(const std::shared_ptr<Bound>& 
expr,
+                                                 const Literal& lit) override {
+    return StartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotStartsWith(const 
std::shared_ptr<Bound>& expr,
+                                                    const Literal& lit) 
override {
+    return NotStartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> In(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return InImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> NotIn(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return NotInImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<BoundPredicate>& pred) override;
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<UnboundPredicate>& pred) override {
+    ICEBERG_ASSIGN_OR_RAISE(auto bound_predicate, pred->Bind(*schema_, 
case_sensitive_));
+    if (bound_predicate->is_bound_predicate()) {
+      ICEBERG_ASSIGN_OR_RAISE(
+          auto residual,
+          
Predicate(std::dynamic_pointer_cast<BoundPredicate>(bound_predicate)));
+      if (residual->is_bound_predicate()) {
+        // replace inclusive original unbound predicate
+        return pred;
+      }
+      return residual;
+    }
+    // if binding didn't result in a Predicate, return the expression
+    return bound_predicate;
+  }
+
+ private:
+  // Helper methods for bound predicates
+  Result<std::shared_ptr<Expression>> EvaluateBoundPredicate(
+      const std::shared_ptr<BoundPredicate>& pred);
+
+  Result<std::shared_ptr<Expression>> IsNullImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNullImpl(const 
std::shared_ptr<Bound>& expr);
+  Result<std::shared_ptr<Expression>> IsNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> LtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> LtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> EqImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                                const Literal& lit);
+  Result<std::shared_ptr<Expression>> InImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> NotInImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> StartsWithImpl(const 
std::shared_ptr<Bound>& expr,
+                                                     const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotStartsWithImpl(
+      const std::shared_ptr<Bound>& expr, const Literal& lit);
+
+  std::shared_ptr<PartitionSpec> spec_;
+  std::shared_ptr<Schema> schema_;
+  std::shared_ptr<Schema> partition_schema_;
+  const StructLike& partition_data_;
+  bool case_sensitive_;
+};
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::IsNullImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNull()) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotNullImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNull()) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::IsNaNImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNaN()) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotNaNImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNaN()) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::LtImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value < lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::LtEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value <= lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::GtImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value > lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::GtEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value >= lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::EqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value == lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value != lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::InImpl(
+    const std::shared_ptr<Bound>& expr,
+    const BoundSetPredicate::LiteralSet& literal_set) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (literal_set.contains(value)) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotInImpl(
+    const std::shared_ptr<Bound>& expr,
+    const BoundSetPredicate::LiteralSet& literal_set) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (literal_set.contains(value)) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::StartsWithImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+
+  // Both value and literal should be strings
+  if (!std::holds_alternative<std::string>(value.value()) ||
+      !std::holds_alternative<std::string>(lit.value())) {
+    return Expressions::AlwaysFalse();
+  }
+
+  const auto& str_value = std::get<std::string>(value.value());
+  const auto& str_prefix = std::get<std::string>(lit.value());
+  if (str_value.starts_with(str_prefix)) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotStartsWithImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+
+  // Both value and literal should be strings
+  if (!std::holds_alternative<std::string>(value.value()) ||
+      !std::holds_alternative<std::string>(lit.value())) {
+    return Expressions::AlwaysTrue();
+  }
+
+  const auto& str_value = std::get<std::string>(value.value());
+  const auto& str_prefix = std::get<std::string>(lit.value());
+  if (str_value.starts_with(str_prefix)) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::EvaluateBoundPredicate(

Review Comment:
   Why do you reinvent `BoundVisitor::Predicate(const 
std::shared_ptr<BoundPredicate>& pred)` again?



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));
+    partition_schema_ = FromStructType(std::move(*partition_type_), 
std::nullopt);
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysTrue() override {
+    return Expressions::AlwaysTrue();

Review Comment:
   We cannot use `Expressions` family in this file because they may throw. 
Please use corresponding `Make` functions.



##########
src/iceberg/expression/residual_evaluator.cc:
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/expression/residual_evaluator.h"
+
+#include "iceberg/expression/binder.h"
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/expression_visitor.h"
+#include "iceberg/expression/expressions.h"
+#include "iceberg/expression/predicate.h"
+#include "iceberg/partition_spec.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/schema.h"
+#include "iceberg/schema_internal.h"
+#include "iceberg/transform.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+class ResidualVisitor : public BoundVisitor<std::shared_ptr<Expression>> {
+ public:
+  ResidualVisitor(const std::shared_ptr<PartitionSpec>& spec,
+                  const std::shared_ptr<Schema>& schema, const StructLike& 
partition_data,
+                  bool case_sensitive)
+      : spec_(spec),
+        schema_(schema),
+        partition_data_(partition_data),
+        case_sensitive_(case_sensitive) {
+    ICEBERG_ASSIGN_OR_THROW(auto partition_type_, 
spec_->PartitionType(*schema));
+    partition_schema_ = FromStructType(std::move(*partition_type_), 
std::nullopt);
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysTrue() override {
+    return Expressions::AlwaysTrue();
+  }
+
+  Result<std::shared_ptr<Expression>> AlwaysFalse() override {
+    return Expressions::AlwaysFalse();
+  }
+
+  Result<std::shared_ptr<Expression>> Not(
+      const std::shared_ptr<Expression>& child_result) override {
+    return Expressions::Not(child_result);
+  }
+
+  Result<std::shared_ptr<Expression>> And(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::And(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> Or(
+      const std::shared_ptr<Expression>& left_result,
+      const std::shared_ptr<Expression>& right_result) override {
+    return Expressions::Or(left_result, right_result);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return IsNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNull(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNullImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> IsNaN(const std::shared_ptr<Bound>& 
expr) override {
+    return IsNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> NotNaN(
+      const std::shared_ptr<Bound>& expr) override {
+    return NotNaNImpl(expr);
+  }
+
+  Result<std::shared_ptr<Expression>> Lt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return LtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> LtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return LtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Gt(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return GtImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> GtEq(const std::shared_ptr<Bound>& expr,
+                                           const Literal& lit) override {
+    return GtEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> Eq(const std::shared_ptr<Bound>& expr,
+                                         const Literal& lit) override {
+    return EqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotEq(const std::shared_ptr<Bound>& expr,
+                                            const Literal& lit) override {
+    return NotEqImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> StartsWith(const std::shared_ptr<Bound>& 
expr,
+                                                 const Literal& lit) override {
+    return StartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> NotStartsWith(const 
std::shared_ptr<Bound>& expr,
+                                                    const Literal& lit) 
override {
+    return NotStartsWithImpl(expr, lit);
+  }
+
+  Result<std::shared_ptr<Expression>> In(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return InImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> NotIn(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set) override {
+    return NotInImpl(expr, literal_set);
+  }
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<BoundPredicate>& pred) override;
+
+  Result<std::shared_ptr<Expression>> Predicate(
+      const std::shared_ptr<UnboundPredicate>& pred) override {
+    ICEBERG_ASSIGN_OR_RAISE(auto bound_predicate, pred->Bind(*schema_, 
case_sensitive_));
+    if (bound_predicate->is_bound_predicate()) {
+      ICEBERG_ASSIGN_OR_RAISE(
+          auto residual,
+          
Predicate(std::dynamic_pointer_cast<BoundPredicate>(bound_predicate)));
+      if (residual->is_bound_predicate()) {
+        // replace inclusive original unbound predicate
+        return pred;
+      }
+      return residual;
+    }
+    // if binding didn't result in a Predicate, return the expression
+    return bound_predicate;
+  }
+
+ private:
+  // Helper methods for bound predicates
+  Result<std::shared_ptr<Expression>> EvaluateBoundPredicate(
+      const std::shared_ptr<BoundPredicate>& pred);
+
+  Result<std::shared_ptr<Expression>> IsNullImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNullImpl(const 
std::shared_ptr<Bound>& expr);
+  Result<std::shared_ptr<Expression>> IsNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> NotNaNImpl(const std::shared_ptr<Bound>& 
expr);
+  Result<std::shared_ptr<Expression>> LtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> LtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> GtEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                               const Literal& lit);
+  Result<std::shared_ptr<Expression>> EqImpl(const std::shared_ptr<Bound>& 
expr,
+                                             const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotEqImpl(const std::shared_ptr<Bound>& 
expr,
+                                                const Literal& lit);
+  Result<std::shared_ptr<Expression>> InImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> NotInImpl(
+      const std::shared_ptr<Bound>& expr,
+      const BoundSetPredicate::LiteralSet& literal_set);
+  Result<std::shared_ptr<Expression>> StartsWithImpl(const 
std::shared_ptr<Bound>& expr,
+                                                     const Literal& lit);
+  Result<std::shared_ptr<Expression>> NotStartsWithImpl(
+      const std::shared_ptr<Bound>& expr, const Literal& lit);
+
+  std::shared_ptr<PartitionSpec> spec_;
+  std::shared_ptr<Schema> schema_;
+  std::shared_ptr<Schema> partition_schema_;
+  const StructLike& partition_data_;
+  bool case_sensitive_;
+};
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::IsNullImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNull()) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotNullImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNull()) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::IsNaNImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNaN()) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotNaNImpl(
+    const std::shared_ptr<Bound>& expr) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value.IsNaN()) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::LtImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value < lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::LtEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value <= lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::GtImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value > lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::GtEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value >= lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::EqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value == lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotEqImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (value != lit) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::InImpl(
+    const std::shared_ptr<Bound>& expr,
+    const BoundSetPredicate::LiteralSet& literal_set) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (literal_set.contains(value)) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotInImpl(
+    const std::shared_ptr<Bound>& expr,
+    const BoundSetPredicate::LiteralSet& literal_set) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+  if (literal_set.contains(value)) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::StartsWithImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+
+  // Both value and literal should be strings
+  if (!std::holds_alternative<std::string>(value.value()) ||
+      !std::holds_alternative<std::string>(lit.value())) {
+    return Expressions::AlwaysFalse();
+  }
+
+  const auto& str_value = std::get<std::string>(value.value());
+  const auto& str_prefix = std::get<std::string>(lit.value());
+  if (str_value.starts_with(str_prefix)) {
+    return Expressions::AlwaysTrue();
+  }
+  return Expressions::AlwaysFalse();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::NotStartsWithImpl(
+    const std::shared_ptr<Bound>& expr, const Literal& lit) {
+  ICEBERG_ASSIGN_OR_RAISE(auto value, expr->Evaluate(partition_data_));
+
+  // Both value and literal should be strings
+  if (!std::holds_alternative<std::string>(value.value()) ||
+      !std::holds_alternative<std::string>(lit.value())) {
+    return Expressions::AlwaysTrue();
+  }
+
+  const auto& str_value = std::get<std::string>(value.value());
+  const auto& str_prefix = std::get<std::string>(lit.value());
+  if (str_value.starts_with(str_prefix)) {
+    return Expressions::AlwaysFalse();
+  }
+  return Expressions::AlwaysTrue();
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::EvaluateBoundPredicate(
+    const std::shared_ptr<BoundPredicate>& pred) {
+  ICEBERG_DCHECK(pred != nullptr, "BoundPredicate cannot be null");
+
+  switch (pred->kind()) {
+    case BoundPredicate::Kind::kUnary: {
+      switch (pred->op()) {
+        case Expression::Operation::kIsNull:
+          return IsNullImpl(pred->term());
+        case Expression::Operation::kNotNull:
+          return NotNullImpl(pred->term());
+        case Expression::Operation::kIsNan:
+          return IsNaNImpl(pred->term());
+        case Expression::Operation::kNotNan:
+          return NotNaNImpl(pred->term());
+        default:
+          return InvalidExpression("Invalid operation for BoundUnaryPredicate: 
{}",
+                                   ToString(pred->op()));
+      }
+    }
+    case BoundPredicate::Kind::kLiteral: {
+      const auto& literal_pred =
+          internal::checked_cast<const BoundLiteralPredicate&>(*pred);
+      switch (pred->op()) {
+        case Expression::Operation::kLt:
+          return LtImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kLtEq:
+          return LtEqImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kGt:
+          return GtImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kGtEq:
+          return GtEqImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kEq:
+          return EqImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kNotEq:
+          return NotEqImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kStartsWith:
+          return StartsWithImpl(pred->term(), literal_pred.literal());
+        case Expression::Operation::kNotStartsWith:
+          return NotStartsWithImpl(pred->term(), literal_pred.literal());
+        default:
+          return InvalidExpression("Invalid operation for 
BoundLiteralPredicate: {}",
+                                   ToString(pred->op()));
+      }
+    }
+    case BoundPredicate::Kind::kSet: {
+      const auto& set_pred = internal::checked_cast<const 
BoundSetPredicate&>(*pred);
+      switch (pred->op()) {
+        case Expression::Operation::kIn:
+          return InImpl(pred->term(), set_pred.literal_set());
+        case Expression::Operation::kNotIn:
+          return NotInImpl(pred->term(), set_pred.literal_set());
+        default:
+          return InvalidExpression("Invalid operation for BoundSetPredicate: 
{}",
+                                   ToString(pred->op()));
+      }
+    }
+  }
+
+  return InvalidExpression("Unsupported bound predicate: {}", 
pred->ToString());
+}
+
+Result<std::shared_ptr<Expression>> ResidualVisitor::Predicate(
+    const std::shared_ptr<BoundPredicate>& pred) {
+  // Get the strict projection and inclusive projection of this predicate in 
partition
+  // data, then use them to determine whether to return the original 
predicate. The
+  // strict projection returns true iff the original predicate would have 
returned true,
+  // so the predicate can be eliminated if the strict projection evaluates to 
true.
+  // Similarly the inclusive projection returns false iff the original 
predicate would
+  // have returned false, so the predicate can also be eliminated if the 
inclusive
+  // projection evaluates to false.
+
+  // Get the field ID from the predicate's reference
+  const auto& ref = pred->reference();
+  int32_t field_id = ref->field().field_id();
+
+  // Find partition fields that match this source field ID
+  std::vector<const PartitionField*> matching_fields;

Review Comment:
   It seems worth adding a `PartitionSpec::GetFieldsBySourceId` using `lazy.h` 
to avoid repeated search?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to