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


##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+

Review Comment:
   Please also add a `std::string ToString() const` similar to the Java impl.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}

Review Comment:
   nit: add `ICEBERG_DCHECK(term != nullptr)` when `op` is not count star.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {
+ public:
+  enum class Mode { kNonNull, kNull, kStar };
+
+  BoundCountAggregate(Expression::Operation op, Mode mode,
+                      std::shared_ptr<BoundTerm> term);
+
+  Mode mode() const { return mode_; }
+
+  Kind kind() const override { return Kind::kCount; }
+
+  std::string ToString() const override;
+  Result<Literal> Evaluate(const StructLike& data) const override;

Review Comment:
   ```suggestion
   
     Result<Literal> Evaluate(const StructLike& data) const override;
   ```



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {

Review Comment:
   ```suggestion
   class ICEBERG_EXPORT CountAggregate : public BoundAggregate {
   ```



##########
src/iceberg/expression/expression_visitor.h:
##########
@@ -77,6 +78,22 @@ class ICEBERG_EXPORT ExpressionVisitor {
   /// \brief Visit an unbound predicate.
   /// \param pred The unbound predicate to visit
   virtual Result<R> Predicate(const std::shared_ptr<UnboundPredicate>& pred) = 
0;
+
+  /// \brief Visit a bound aggregate.
+  /// \param aggregate The bound aggregate to visit.
+  virtual Result<R> Aggregate(const std::shared_ptr<BoundAggregate>& 
aggregate) {
+    ICEBERG_DCHECK(aggregate != nullptr, "Bound aggregate cannot be null");
+    return NotSupported("Bound aggregate is not supported by this visitor: {}",

Review Comment:
   The error message is misleading since we are printing the aggregate as the 
visitor name.



##########
src/iceberg/expression/expressions.cc:
##########
@@ -81,6 +82,76 @@ std::shared_ptr<UnboundTransform> Expressions::Transform(
   return unbound_transform;
 }
 
+// Aggregates
+
+std::shared_ptr<UnboundAggregateImpl<BoundReference>> Expressions::Count(
+    std::string name) {
+  return Count(Ref(std::move(name)));
+}
+
+std::shared_ptr<UnboundAggregateImpl<BoundReference>> Expressions::Count(
+    std::shared_ptr<UnboundTerm<BoundReference>> expr) {
+  auto agg = std::make_shared<UnboundAggregateImpl<BoundReference>>(

Review Comment:
   See my other comments. Let's add `Make` functions to these types and throw 
when there is any error. Same for other functions in this file.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {
+ public:
+  enum class Mode { kNonNull, kNull, kStar };
+
+  BoundCountAggregate(Expression::Operation op, Mode mode,
+                      std::shared_ptr<BoundTerm> term);
+
+  Mode mode() const { return mode_; }
+
+  Kind kind() const override { return Kind::kCount; }
+
+  std::string ToString() const override;
+  Result<Literal> Evaluate(const StructLike& data) const override;
+
+ private:
+  Mode mode_;
+};
+
+/// \brief Bound MAX/MIN aggregate.
+class ICEBERG_EXPORT BoundValueAggregate : public BoundAggregate {
+ public:
+  BoundValueAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> 
term);
+
+  Kind kind() const override { return Kind::kValue; }
+
+  std::string ToString() const override;
+  Result<Literal> Evaluate(const StructLike& data) const override;
+};
+
+/// \brief Evaluates bound aggregates over StructLike rows.
+class ICEBERG_EXPORT AggregateEvaluator {
+ public:
+  virtual ~AggregateEvaluator() = default;
+
+  /// \brief Create an evaluator for a single bound aggregate.
+  /// \param aggregate The bound aggregate to evaluate across rows.
+  static Result<std::unique_ptr<AggregateEvaluator>> Make(
+      std::shared_ptr<BoundAggregate> aggregate);
+
+  /// \brief Create an evaluator for multiple bound aggregates.
+  /// \param aggregates Aggregates to evaluate in one pass; order is preserved 
in
+  /// Results().
+  static Result<std::unique_ptr<AggregateEvaluator>> MakeList(
+      std::vector<std::shared_ptr<BoundAggregate>> aggregates);
+
+  /// \brief Add a row to the aggregate.
+  virtual Status Add(const StructLike& row) = 0;

Review Comment:
   Update?



##########
src/iceberg/expression/aggregate.cc:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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/aggregate.h"
+
+#include <format>
+#include <optional>
+#include <vector>
+
+#include "iceberg/exception.h"
+#include "iceberg/expression/binder.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/type.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+namespace {
+
+Result<std::shared_ptr<PrimitiveType>> GetPrimitiveType(const BoundTerm& term) 
{
+  auto primitive = std::dynamic_pointer_cast<PrimitiveType>(term.type());
+  if (primitive == nullptr) {
+    return InvalidExpression("Aggregate requires primitive type, got {}",
+                             term.type()->ToString());
+  }
+  return primitive;
+}
+
+}  // namespace
+
+// -------------------- Bound aggregates --------------------
+
+BoundCountAggregate::BoundCountAggregate(Expression::Operation op, Mode mode,
+                                         std::shared_ptr<BoundTerm> term)
+    : BoundAggregate(op, std::move(term)), mode_(mode) {}
+
+std::string BoundCountAggregate::ToString() const {
+  if (mode_ == Mode::kStar) {
+    return "count(*)";
+  }
+  ICEBERG_DCHECK(term() != nullptr, "Bound count aggregate should have term");
+  switch (mode_) {
+    case Mode::kNull:
+      return std::format("count_null({})", term()->reference()->name());
+    case Mode::kNonNull:
+      return std::format("count({})", term()->reference()->name());
+    case Mode::kStar:
+      break;
+  }
+  std::unreachable();
+}
+
+Result<Literal> BoundCountAggregate::Evaluate(const StructLike& data) const {

Review Comment:
   Personally I prefer to mimic the Java class design if there is no compelling 
reason to deviate. 



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;

Review Comment:
   ditto for ICEBERG_DCHECK



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;

Review Comment:
   nit: add a `ICEBERG_DCHECK` for expected `op`



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}

Review Comment:
   Please add a static `Make` function and use private constructor so we can 
make sure that created expression does not have invalid fields.



##########
src/iceberg/expression/expressions.h:
##########
@@ -101,6 +102,48 @@ class ICEBERG_EXPORT Expressions {
   static std::shared_ptr<UnboundTransform> Transform(
       std::string name, std::shared_ptr<Transform> transform);
 
+  // Aggregates
+
+  /// \brief Create COUNT(col) aggregate.

Review Comment:
   ```suggestion
     /// \brief Create a COUNT aggregate for a field name.
   ```
   
   Let's be consistent with the above comment style. Same for all below.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {
+ public:
+  enum class Mode { kNonNull, kNull, kStar };
+
+  BoundCountAggregate(Expression::Operation op, Mode mode,
+                      std::shared_ptr<BoundTerm> term);
+
+  Mode mode() const { return mode_; }
+
+  Kind kind() const override { return Kind::kCount; }
+
+  std::string ToString() const override;
+  Result<Literal> Evaluate(const StructLike& data) const override;
+
+ private:
+  Mode mode_;
+};
+
+/// \brief Bound MAX/MIN aggregate.
+class ICEBERG_EXPORT BoundValueAggregate : public BoundAggregate {
+ public:
+  BoundValueAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> 
term);

Review Comment:
   ditto



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };

Review Comment:
   What is `kNone` for? We don't even need `CountMode`, do we? 
`UnboundAggregateImpl` should also support value mode.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;

Review Comment:
   nit: `predicate.h` has defined a duplicate `concept TermType`. Perhaps we 
can move it to `term.h` and reuse it here?



##########
src/iceberg/expression/aggregate.cc:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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/aggregate.h"
+
+#include <format>
+#include <optional>
+#include <vector>
+
+#include "iceberg/exception.h"
+#include "iceberg/expression/binder.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/type.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+namespace {
+
+Result<std::shared_ptr<PrimitiveType>> GetPrimitiveType(const BoundTerm& term) 
{
+  auto primitive = std::dynamic_pointer_cast<PrimitiveType>(term.type());
+  if (primitive == nullptr) {
+    return InvalidExpression("Aggregate requires primitive type, got {}",
+                             term.type()->ToString());
+  }
+  return primitive;

Review Comment:
   ```suggestion
     if (!term.type().is_primitive()) {
       return InvalidExpression("Aggregate requires primitive type, got {}",
                                term.type()->ToString());
     }
     return internal::checked_pointer_cast<PrimitiveType>(term.type());
   ```



##########
src/iceberg/expression/aggregate.cc:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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/aggregate.h"
+
+#include <format>
+#include <optional>
+#include <vector>
+
+#include "iceberg/exception.h"
+#include "iceberg/expression/binder.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/type.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+namespace {
+
+Result<std::shared_ptr<PrimitiveType>> GetPrimitiveType(const BoundTerm& term) 
{
+  auto primitive = std::dynamic_pointer_cast<PrimitiveType>(term.type());
+  if (primitive == nullptr) {
+    return InvalidExpression("Aggregate requires primitive type, got {}",
+                             term.type()->ToString());
+  }
+  return primitive;
+}
+
+}  // namespace
+
+// -------------------- Bound aggregates --------------------
+
+BoundCountAggregate::BoundCountAggregate(Expression::Operation op, Mode mode,
+                                         std::shared_ptr<BoundTerm> term)
+    : BoundAggregate(op, std::move(term)), mode_(mode) {}
+
+std::string BoundCountAggregate::ToString() const {
+  if (mode_ == Mode::kStar) {
+    return "count(*)";
+  }
+  ICEBERG_DCHECK(term() != nullptr, "Bound count aggregate should have term");
+  switch (mode_) {
+    case Mode::kNull:
+      return std::format("count_null({})", term()->reference()->name());
+    case Mode::kNonNull:
+      return std::format("count({})", term()->reference()->name());
+    case Mode::kStar:
+      break;
+  }
+  std::unreachable();
+}
+
+Result<Literal> BoundCountAggregate::Evaluate(const StructLike& data) const {

Review Comment:
   What's the tradeoff between these two options:
   - Current approach that uses a single `CountAggregate` for all kinds.
   - Add dedicated subclass for CountNull, CountNotNull and CountStar.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {
+ public:
+  enum class Mode { kNonNull, kNull, kStar };
+
+  BoundCountAggregate(Expression::Operation op, Mode mode,
+                      std::shared_ptr<BoundTerm> term);
+
+  Mode mode() const { return mode_; }
+
+  Kind kind() const override { return Kind::kCount; }
+
+  std::string ToString() const override;
+  Result<Literal> Evaluate(const StructLike& data) const override;
+
+ private:
+  Mode mode_;
+};
+
+/// \brief Bound MAX/MIN aggregate.
+class ICEBERG_EXPORT BoundValueAggregate : public BoundAggregate {

Review Comment:
   ```suggestion
   class ICEBERG_EXPORT ValueAggregate : public BoundAggregate {
   ```



##########
src/iceberg/expression/aggregate.cc:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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/aggregate.h"
+
+#include <format>
+#include <optional>
+#include <vector>
+
+#include "iceberg/exception.h"
+#include "iceberg/expression/binder.h"
+#include "iceberg/row/struct_like.h"
+#include "iceberg/type.h"
+#include "iceberg/util/checked_cast.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+namespace {
+
+Result<std::shared_ptr<PrimitiveType>> GetPrimitiveType(const BoundTerm& term) 
{
+  auto primitive = std::dynamic_pointer_cast<PrimitiveType>(term.type());
+  if (primitive == nullptr) {
+    return InvalidExpression("Aggregate requires primitive type, got {}",
+                             term.type()->ToString());
+  }
+  return primitive;
+}
+
+}  // namespace
+
+// -------------------- Bound aggregates --------------------
+
+BoundCountAggregate::BoundCountAggregate(Expression::Operation op, Mode mode,
+                                         std::shared_ptr<BoundTerm> term)
+    : BoundAggregate(op, std::move(term)), mode_(mode) {}
+
+std::string BoundCountAggregate::ToString() const {
+  if (mode_ == Mode::kStar) {
+    return "count(*)";
+  }
+  ICEBERG_DCHECK(term() != nullptr, "Bound count aggregate should have term");
+  switch (mode_) {
+    case Mode::kNull:
+      return std::format("count_null({})", term()->reference()->name());
+    case Mode::kNonNull:
+      return std::format("count({})", term()->reference()->name());
+    case Mode::kStar:
+      break;
+  }
+  std::unreachable();
+}
+
+Result<Literal> BoundCountAggregate::Evaluate(const StructLike& data) const {
+  switch (mode_) {
+    case Mode::kStar:
+      return Literal::Long(1);
+    case Mode::kNonNull: {
+      ICEBERG_ASSIGN_OR_RAISE(auto literal, term()->Evaluate(data));
+      return Literal::Long(literal.IsNull() ? 0 : 1);
+    }
+    case Mode::kNull: {
+      ICEBERG_ASSIGN_OR_RAISE(auto literal, term()->Evaluate(data));
+      return Literal::Long(literal.IsNull() ? 1 : 0);
+    }
+  }
+  std::unreachable();
+}
+
+BoundValueAggregate::BoundValueAggregate(Expression::Operation op,
+                                         std::shared_ptr<BoundTerm> term)
+    : BoundAggregate(op, std::move(term)) {}
+
+std::string BoundValueAggregate::ToString() const {
+  ICEBERG_DCHECK(term() != nullptr, "Bound value aggregate should have term");
+  auto prefix = op() == Expression::Operation::kMax ? "max" : "min";
+  return std::format("{}({})", prefix, term()->reference()->name());
+}
+
+Result<Literal> BoundValueAggregate::Evaluate(const StructLike& data) const {
+  ICEBERG_ASSIGN_OR_RAISE(auto literal, term()->Evaluate(data));
+  return literal;
+}
+
+// -------------------- Unbound binding --------------------
+
+template <typename B>
+Result<std::shared_ptr<Expression>> UnboundAggregateImpl<B>::Bind(
+    const Schema& schema, bool case_sensitive) const {
+  std::shared_ptr<B> bound_term;
+  if (this->term()) {
+    ICEBERG_ASSIGN_OR_THROW(bound_term, this->term()->Bind(schema, 
case_sensitive));
+  }
+
+  switch (count_mode_) {
+    case CountMode::kStar:
+    case CountMode::kNull:
+    case CountMode::kNonNull: {
+      auto op = this->op() == Expression::Operation::kCountStar
+                    ? Expression::Operation::kCountStar
+                    : Expression::Operation::kCount;
+      auto mode =
+          count_mode_ == CountMode::kNull
+              ? BoundCountAggregate::Mode::kNull
+              : (count_mode_ == CountMode::kStar ? 
BoundCountAggregate::Mode::kStar
+                                                 : 
BoundCountAggregate::Mode::kNonNull);
+      auto aggregate =
+          std::make_shared<BoundCountAggregate>(op, mode, 
std::move(bound_term));
+      return aggregate;
+    }
+    case CountMode::kNone: {
+      if (this->op() != Expression::Operation::kMax &&
+          this->op() != Expression::Operation::kMin) {
+        return NotSupported("Unsupported aggregate operation");
+      }
+      if (!bound_term) {
+        return InvalidExpression("Aggregate requires a term");
+      }
+      auto aggregate =
+          std::make_shared<BoundValueAggregate>(this->op(), 
std::move(bound_term));
+      return aggregate;
+    }
+  }
+  std::unreachable();
+}
+
+template class UnboundAggregateImpl<BoundReference>;
+
+// -------------------- AggregateEvaluator --------------------
+
+namespace {
+
+class AggregateEvaluatorImpl : public AggregateEvaluator {

Review Comment:
   Why not following the Java impl of aggregator class hierarchy? If we need to 
add support for `DataFile`, these lines will be doubled with a lot of duplicate 
code.



##########
src/iceberg/expression/aggregate.h:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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/aggregate.h
+/// Aggregate expression definitions.
+
+#include <concepts>
+#include <memory>
+#include <vector>
+
+#include "iceberg/expression/expression.h"
+#include "iceberg/expression/term.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+template <typename T>
+concept AggregateTermType = std::derived_from<T, Term>;
+
+/// \brief Base aggregate holding an operation and a term.
+template <AggregateTermType T>
+class ICEBERG_EXPORT Aggregate : public virtual Expression {
+ public:
+  ~Aggregate() override = default;
+
+  Expression::Operation op() const override { return operation_; }
+
+  const std::shared_ptr<T>& term() const { return term_; }
+
+ protected:
+  Aggregate(Expression::Operation op, std::shared_ptr<T> term)
+      : operation_(op), term_(std::move(term)) {}
+
+  Expression::Operation operation_;
+  std::shared_ptr<T> term_;
+};
+
+/// \brief Base class for unbound aggregates.
+class ICEBERG_EXPORT UnboundAggregate : public virtual Expression,
+                                        public Unbound<Expression> {
+ public:
+  ~UnboundAggregate() override = default;
+
+  bool is_unbound_aggregate() const override { return true; }
+};
+
+/// \brief Template for unbound aggregates that carry a term and operation.
+template <typename B>
+class ICEBERG_EXPORT UnboundAggregateImpl : public UnboundAggregate,
+                                            public Aggregate<UnboundTerm<B>> {
+  using BASE = Aggregate<UnboundTerm<B>>;
+
+ public:
+  enum class CountMode { kNonNull, kNull, kStar, kNone };
+
+  UnboundAggregateImpl(Expression::Operation op, 
std::shared_ptr<UnboundTerm<B>> term,
+                       CountMode count_mode = CountMode::kNone)
+      : BASE(op, std::move(term)), count_mode_(count_mode) {}
+
+  std::shared_ptr<NamedReference> reference() override {
+    return BASE::term() ? BASE::term()->reference() : nullptr;
+  }
+
+  Result<std::shared_ptr<Expression>> Bind(const Schema& schema,
+                                           bool case_sensitive) const override;
+
+  CountMode count_mode() const { return count_mode_; }
+
+ private:
+  CountMode count_mode_;
+};
+
+/// \brief Base class for bound aggregates.
+class ICEBERG_EXPORT BoundAggregate : public Aggregate<BoundTerm>, public 
Bound {
+ public:
+  using Aggregate<BoundTerm>::op;
+  using Aggregate<BoundTerm>::term;
+
+  std::shared_ptr<BoundReference> reference() override {
+    return term_ ? term_->reference() : nullptr;
+  }
+
+  Result<Literal> Evaluate(const StructLike& data) const override = 0;
+
+  bool is_bound_aggregate() const override { return true; }
+
+  enum class Kind : int8_t {
+    // Count aggregates (COUNT, COUNT_STAR, COUNT_NULL)
+    kCount = 0,
+    // Value aggregates (MIN, MAX)
+    kValue,
+  };
+
+  virtual Kind kind() const = 0;
+
+ protected:
+  BoundAggregate(Expression::Operation op, std::shared_ptr<BoundTerm> term)
+      : Aggregate<BoundTerm>(op, std::move(term)) {}
+};
+
+/// \brief Bound COUNT aggregate.
+class ICEBERG_EXPORT BoundCountAggregate : public BoundAggregate {
+ public:
+  enum class Mode { kNonNull, kNull, kStar };
+
+  BoundCountAggregate(Expression::Operation op, Mode mode,

Review Comment:
   Same here. Make the ctor private and add a static `Make` function.



-- 
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