risingwave_frontend/optimizer/plan_node/
logical_agg.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use fixedbitset::FixedBitSet;
16use itertools::Itertools;
17use risingwave_common::types::{DataType, ScalarImpl};
18use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
19use risingwave_common::{bail, bail_not_implemented, not_implemented};
20use risingwave_expr::aggregate::{AggType, PbAggKind, agg_types};
21
22use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder};
23use super::utils::impl_distill_by_unit;
24use super::{
25    BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, LogicalPlanRef as PlanRef,
26    PlanBase, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamPlanRef, StreamProject,
27    StreamShare, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream,
28};
29use crate::error::{ErrorCode, Result, RwError};
30use crate::expr::{
31    AggCall, Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef, Literal,
32    OrderBy, WindowFunction,
33};
34use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
35use crate::optimizer::plan_node::generic::GenericPlanNode;
36use crate::optimizer::plan_node::stream_global_approx_percentile::StreamGlobalApproxPercentile;
37use crate::optimizer::plan_node::stream_local_approx_percentile::StreamLocalApproxPercentile;
38use crate::optimizer::plan_node::stream_row_merge::StreamRowMerge;
39use crate::optimizer::plan_node::{
40    BatchSortAgg, ColumnPruningContext, LogicalDedup, LogicalProject, PredicatePushdownContext,
41    RewriteStreamContext, ToStreamContext, gen_filter_and_pushdown,
42};
43use crate::optimizer::property::{Distribution, Order, RequiredDist};
44use crate::utils::{
45    ColIndexMapping, ColIndexMappingRewriteExt, Condition, GroupBy, IndexSet, Substitute,
46};
47
48pub struct AggInfo {
49    pub calls: Vec<PlanAggCall>,
50    pub col_mapping: ColIndexMapping,
51}
52
53/// `SeparatedAggInfo` is used to separate normal and approx percentile aggs.
54pub struct SeparatedAggInfo {
55    normal: AggInfo,
56    approx: AggInfo,
57}
58
59/// `LogicalAgg` groups input data by their group key and computes aggregation functions.
60///
61/// It corresponds to the `GROUP BY` operator in a SQL query statement together with the aggregate
62/// functions in the `SELECT` clause.
63///
64/// The output schema will first include the group key and then the aggregation calls.
65#[derive(Clone, Debug, PartialEq, Eq, Hash)]
66pub struct LogicalAgg {
67    pub base: PlanBase<Logical>,
68    core: Agg<PlanRef>,
69}
70
71impl LogicalAgg {
72    /// Generate plan for stateless 2-phase streaming agg.
73    /// Should only be used iff input is distributed. Input must be converted to stream form.
74    fn gen_stateless_two_phase_streaming_agg_plan(
75        &self,
76        stream_input: StreamPlanRef,
77    ) -> Result<StreamPlanRef> {
78        debug_assert!(self.group_key().is_empty());
79
80        // ====== Handle approx percentile aggs
81        let (
82            non_approx_percentile_col_mapping,
83            approx_percentile_col_mapping,
84            approx_percentile,
85            core,
86        ) = self.prepare_approx_percentile(stream_input.clone())?;
87
88        if core.agg_calls.is_empty() {
89            if let Some(approx_percentile) = approx_percentile {
90                return Ok(approx_percentile);
91            };
92            bail!("expected at least one agg call");
93        }
94
95        let need_row_merge: bool = Self::need_row_merge(&approx_percentile);
96
97        // ====== Handle normal aggs
98        let total_agg_calls = core
99            .agg_calls
100            .iter()
101            .enumerate()
102            .map(|(partial_output_idx, agg_call)| {
103                agg_call.partial_to_total_agg_call(partial_output_idx)
104            })
105            .collect_vec();
106        let local_agg = StreamStatelessSimpleAgg::new(core)?;
107        let exchange =
108            RequiredDist::single().streaming_enforce_if_not_satisfies(local_agg.into())?;
109
110        let must_output_per_barrier = need_row_merge;
111        let global_agg = new_stream_simple_agg(
112            Agg::new(total_agg_calls, IndexSet::empty(), exchange),
113            must_output_per_barrier,
114        )?;
115
116        // ====== Merge approx percentile and normal aggs
117        Self::add_row_merge_if_needed(
118            approx_percentile,
119            global_agg.into(),
120            approx_percentile_col_mapping,
121            non_approx_percentile_col_mapping,
122        )
123    }
124
125    /// Generate plan for stateless/stateful 2-phase streaming agg.
126    /// Should only be used iff input is distributed.
127    /// Input must be converted to stream form.
128    fn gen_vnode_two_phase_streaming_agg_plan(
129        &self,
130        stream_input: StreamPlanRef,
131        dist_key: &[usize],
132    ) -> Result<StreamPlanRef> {
133        let (
134            non_approx_percentile_col_mapping,
135            approx_percentile_col_mapping,
136            approx_percentile,
137            core,
138        ) = self.prepare_approx_percentile(stream_input.clone())?;
139
140        if core.agg_calls.is_empty() {
141            if let Some(approx_percentile) = approx_percentile {
142                return Ok(approx_percentile);
143            };
144            bail!("expected at least one agg call");
145        }
146        let need_row_merge = Self::need_row_merge(&approx_percentile);
147
148        // Generate vnode via project
149        // TODO(kwannoel): We should apply Project optimization rules here.
150        let input_col_num = stream_input.schema().len(); // get schema len before moving `stream_input`.
151        let project = StreamProject::new(generic::Project::with_vnode_col(stream_input, dist_key));
152        let vnode_col_idx = project.base.schema().len() - 1;
153
154        // Generate local agg step
155        let mut local_group_key = self.group_key().clone();
156        local_group_key.insert(vnode_col_idx);
157        let n_local_group_key = local_group_key.len();
158        let local_agg = new_stream_hash_agg(
159            Agg::new(core.agg_calls.to_vec(), local_group_key, project.into()),
160            Some(vnode_col_idx),
161        )?;
162        // Global group key excludes vnode.
163        let local_agg_group_key_cardinality = local_agg.group_key().len();
164        let local_group_key_without_vnode =
165            &local_agg.group_key().to_vec()[..local_agg_group_key_cardinality - 1];
166        let global_group_key = local_agg
167            .i2o_col_mapping()
168            .rewrite_dist_key(local_group_key_without_vnode)
169            .expect("some input group key could not be mapped");
170
171        // Generate global agg step
172        let global_agg = if self.group_key().is_empty() {
173            let exchange =
174                RequiredDist::single().streaming_enforce_if_not_satisfies(local_agg.into())?;
175            let must_output_per_barrier = need_row_merge;
176            let global_agg = new_stream_simple_agg(
177                Agg::new(
178                    core.agg_calls
179                        .iter()
180                        .enumerate()
181                        .map(|(partial_output_idx, agg_call)| {
182                            agg_call
183                                .partial_to_total_agg_call(n_local_group_key + partial_output_idx)
184                        })
185                        .collect(),
186                    global_group_key.into_iter().collect(),
187                    exchange,
188                ),
189                must_output_per_barrier,
190            )?;
191            global_agg.into()
192        } else {
193            // the `RowMergeExec` has not supported keyed merge
194            assert!(!need_row_merge);
195            let exchange = RequiredDist::shard_by_key(input_col_num, &global_group_key)
196                .streaming_enforce_if_not_satisfies(local_agg.into())?;
197            // Local phase should have reordered the group keys into their required order.
198            // we can just follow it.
199            let global_agg = new_stream_hash_agg(
200                Agg::new(
201                    core.agg_calls
202                        .iter()
203                        .enumerate()
204                        .map(|(partial_output_idx, agg_call)| {
205                            agg_call
206                                .partial_to_total_agg_call(n_local_group_key + partial_output_idx)
207                        })
208                        .collect(),
209                    global_group_key.into_iter().collect(),
210                    exchange,
211                ),
212                None,
213            )?;
214            global_agg.into()
215        };
216        Self::add_row_merge_if_needed(
217            approx_percentile,
218            global_agg,
219            approx_percentile_col_mapping,
220            non_approx_percentile_col_mapping,
221        )
222    }
223
224    fn gen_single_plan(&self, stream_input: StreamPlanRef) -> Result<StreamPlanRef> {
225        let input = RequiredDist::single().streaming_enforce_if_not_satisfies(stream_input)?;
226        let core = self.core.clone_with_input(input);
227        Ok(new_stream_simple_agg(core, false)?.into())
228    }
229
230    fn gen_shuffle_plan(&self, stream_input: StreamPlanRef) -> Result<StreamPlanRef> {
231        let input =
232            RequiredDist::shard_by_key(stream_input.schema().len(), &self.group_key().to_vec())
233                .streaming_enforce_if_not_satisfies(stream_input)?;
234        let core = self.core.clone_with_input(input);
235        Ok(new_stream_hash_agg(core, None)?.into())
236    }
237
238    /// Generates distributed stream plan.
239    fn gen_dist_stream_agg_plan(&self, stream_input: StreamPlanRef) -> Result<StreamPlanRef> {
240        use super::stream::prelude::*;
241
242        let input_dist = stream_input.distribution();
243        debug_assert!(*input_dist != Distribution::Broadcast);
244
245        // Shuffle agg
246        // If we have group key, and we won't try two phase agg optimization at all,
247        // we will always choose shuffle agg over single agg.
248        if !self.group_key().is_empty() && !self.core.must_try_two_phase_agg() {
249            return self.gen_shuffle_plan(stream_input);
250        }
251
252        // Standalone agg
253        // If no group key, and cannot two phase agg, we have to use single plan.
254        if self.group_key().is_empty() && !self.core.can_two_phase_agg() {
255            return self.gen_single_plan(stream_input);
256        }
257
258        debug_assert!(if !self.group_key().is_empty() {
259            self.core.must_try_two_phase_agg()
260        } else {
261            self.core.can_two_phase_agg()
262        });
263
264        // Stateless 2-phase simple agg
265        // can be applied on stateless simple agg calls,
266        // with input distributed by [`Distribution::AnyShard`]
267        if self.group_key().is_empty()
268            && self
269                .core
270                .all_local_aggs_are_stateless(stream_input.append_only())
271            && input_dist.satisfies(&RequiredDist::AnyShard)
272        {
273            return self.gen_stateless_two_phase_streaming_agg_plan(stream_input);
274        }
275
276        // If input is [`Distribution::SomeShard`] and we must try to use two phase agg,
277        // The only remaining strategy is Vnode-based 2-phase agg.
278        // We shall first distribute it by PK,
279        // so it obeys consistent hash strategy via [`Distribution::HashShard`].
280        let stream_input =
281            if *input_dist == Distribution::SomeShard && self.core.must_try_two_phase_agg() {
282                RequiredDist::shard_by_key(
283                    stream_input.schema().len(),
284                    stream_input.expect_stream_key(),
285                )
286                .streaming_enforce_if_not_satisfies(stream_input)?
287            } else {
288                stream_input
289            };
290        let input_dist = stream_input.distribution();
291
292        // Vnode-based 2-phase agg
293        // can be applied on agg calls not affected by order,
294        // with input distributed by dist_key.
295        match input_dist {
296            Distribution::HashShard(dist_key) | Distribution::UpstreamHashShard(dist_key, _)
297                if (self.group_key().is_empty()
298                    || !self.core.hash_agg_dist_satisfied_by_input_dist(input_dist)) =>
299            {
300                let dist_key = dist_key.clone();
301                return self.gen_vnode_two_phase_streaming_agg_plan(stream_input, &dist_key);
302            }
303            _ => {}
304        }
305
306        // Fallback to shuffle or single, if we can't generate any 2-phase plans.
307        if !self.group_key().is_empty() {
308            self.gen_shuffle_plan(stream_input)
309        } else {
310            self.gen_single_plan(stream_input)
311        }
312    }
313
314    /// Prepares metadata and the `approx_percentile` plan, if there's one present.
315    /// It may modify `core.agg_calls` to separate normal agg and approx percentile agg,
316    /// and `core.input` to share the input via `StreamShare`,
317    /// to both approx percentile agg and normal agg.
318    fn prepare_approx_percentile(
319        &self,
320        stream_input: StreamPlanRef,
321    ) -> Result<(
322        ColIndexMapping,
323        ColIndexMapping,
324        Option<StreamPlanRef>,
325        Agg<StreamPlanRef>,
326    )> {
327        let SeparatedAggInfo { normal, approx } = self.separate_normal_and_special_agg();
328
329        let AggInfo {
330            calls: non_approx_percentile_agg_calls,
331            col_mapping: non_approx_percentile_col_mapping,
332        } = normal;
333        let AggInfo {
334            calls: approx_percentile_agg_calls,
335            col_mapping: approx_percentile_col_mapping,
336        } = approx;
337        if !self.group_key().is_empty() && !approx_percentile_agg_calls.is_empty() {
338            bail_not_implemented!(
339                "two-phase streaming approx percentile aggregation with group key, \
340             please use single phase aggregation instead"
341            );
342        }
343
344        // Either we have approx percentile aggs and non_approx percentile aggs,
345        // or we have at least 2 approx percentile aggs.
346        let needs_row_merge = (!non_approx_percentile_agg_calls.is_empty()
347            && !approx_percentile_agg_calls.is_empty())
348            || approx_percentile_agg_calls.len() >= 2;
349        let input = if needs_row_merge {
350            // If there's row merge, we need to share the input.
351            StreamShare::new_from_input(stream_input.clone()).into()
352        } else {
353            stream_input
354        };
355        let mut core = self.core.clone_with_input(input);
356        core.agg_calls = non_approx_percentile_agg_calls;
357
358        let approx_percentile =
359            self.build_approx_percentile_aggs(core.input.clone(), &approx_percentile_agg_calls)?;
360        Ok((
361            non_approx_percentile_col_mapping,
362            approx_percentile_col_mapping,
363            approx_percentile,
364            core,
365        ))
366    }
367
368    fn need_row_merge(approx_percentile: &Option<StreamPlanRef>) -> bool {
369        approx_percentile.is_some()
370    }
371
372    /// Add `RowMerge` if needed
373    fn add_row_merge_if_needed(
374        approx_percentile: Option<StreamPlanRef>,
375        global_agg: StreamPlanRef,
376        approx_percentile_col_mapping: ColIndexMapping,
377        non_approx_percentile_col_mapping: ColIndexMapping,
378    ) -> Result<StreamPlanRef> {
379        // just for assert
380        let need_row_merge = Self::need_row_merge(&approx_percentile);
381
382        if let Some(approx_percentile) = approx_percentile {
383            assert!(need_row_merge);
384            let row_merge = StreamRowMerge::new(
385                approx_percentile,
386                global_agg,
387                approx_percentile_col_mapping,
388                non_approx_percentile_col_mapping,
389            )?;
390            Ok(row_merge.into())
391        } else {
392            assert!(!need_row_merge);
393            Ok(global_agg)
394        }
395    }
396
397    fn separate_normal_and_special_agg(&self) -> SeparatedAggInfo {
398        let estimated_len = self.agg_calls().len() - 1;
399        let mut approx_percentile_agg_calls = Vec::with_capacity(estimated_len);
400        let mut non_approx_percentile_agg_calls = Vec::with_capacity(estimated_len);
401        let mut approx_percentile_col_mapping = Vec::with_capacity(estimated_len);
402        let mut non_approx_percentile_col_mapping = Vec::with_capacity(estimated_len);
403        for (output_idx, agg_call) in self.agg_calls().iter().enumerate() {
404            if agg_call.agg_type == AggType::Builtin(PbAggKind::ApproxPercentile) {
405                approx_percentile_agg_calls.push(agg_call.clone());
406                approx_percentile_col_mapping.push(Some(output_idx));
407            } else {
408                non_approx_percentile_agg_calls.push(agg_call.clone());
409                non_approx_percentile_col_mapping.push(Some(output_idx));
410            }
411        }
412        let normal = AggInfo {
413            calls: non_approx_percentile_agg_calls,
414            col_mapping: ColIndexMapping::new(
415                non_approx_percentile_col_mapping,
416                self.agg_calls().len(),
417            ),
418        };
419        let approx = AggInfo {
420            calls: approx_percentile_agg_calls,
421            col_mapping: ColIndexMapping::new(
422                approx_percentile_col_mapping,
423                self.agg_calls().len(),
424            ),
425        };
426        SeparatedAggInfo { normal, approx }
427    }
428
429    fn build_approx_percentile_agg(
430        &self,
431        input: StreamPlanRef,
432        approx_percentile_agg_call: &PlanAggCall,
433    ) -> Result<StreamPlanRef> {
434        let local_approx_percentile =
435            StreamLocalApproxPercentile::new(input, approx_percentile_agg_call)?;
436        let exchange = RequiredDist::single()
437            .streaming_enforce_if_not_satisfies(local_approx_percentile.into())?;
438        let global_approx_percentile =
439            StreamGlobalApproxPercentile::new(exchange, approx_percentile_agg_call);
440        Ok(global_approx_percentile.into())
441    }
442
443    /// If only 1 approx percentile, just return it.
444    /// Otherwise build a tree of approx percentile with `MergeProject`.
445    /// e.g.
446    /// ApproxPercentile(col1, 0.5) as x,
447    /// ApproxPercentile(col2, 0.5) as y,
448    /// ApproxPercentile(col3, 0.5) as z
449    /// will be built as
450    ///        `MergeProject`
451    ///       /          \
452    ///  `MergeProject`       z
453    ///  /        \
454    /// x          y
455    fn build_approx_percentile_aggs(
456        &self,
457        input: StreamPlanRef,
458        approx_percentile_agg_call: &[PlanAggCall],
459    ) -> Result<Option<StreamPlanRef>> {
460        if approx_percentile_agg_call.is_empty() {
461            return Ok(None);
462        }
463        let approx_percentile_plans: Vec<_> = approx_percentile_agg_call
464            .iter()
465            .map(|agg_call| self.build_approx_percentile_agg(input.clone(), agg_call))
466            .try_collect()?;
467        assert!(!approx_percentile_plans.is_empty());
468        let mut iter = approx_percentile_plans.into_iter();
469        let mut acc = iter.next().unwrap();
470        for (current_size, plan) in iter.enumerate().map(|(i, p)| (i + 1, p)) {
471            let new_size = current_size + 1;
472            let row_merge = StreamRowMerge::new(
473                acc,
474                plan,
475                ColIndexMapping::identity_or_none(current_size, new_size),
476                ColIndexMapping::new(vec![Some(current_size)], new_size),
477            )?;
478            acc = row_merge.into();
479        }
480        Ok(Some(acc))
481    }
482
483    pub fn core(&self) -> &Agg<PlanRef> {
484        &self.core
485    }
486}
487
488/// `LogicalAggBuilder` extracts agg calls and references to group columns from select list and
489/// build the plan like `LogicalAgg - LogicalProject`.
490/// it is constructed by `group_exprs` and collect and rewrite the expression in selection and
491/// having clause.
492pub struct LogicalAggBuilder {
493    /// the builder of the input Project
494    input_proj_builder: ProjectBuilder,
495    /// the group key column indices in the project's output
496    group_key: IndexSet,
497    /// the grouping sets
498    grouping_sets: Vec<IndexSet>,
499    /// the agg calls
500    agg_calls: Vec<PlanAggCall>,
501    /// the error during the expression rewriting
502    error: Option<RwError>,
503    /// If `is_in_filter_clause` is true, it means that
504    /// we are processing filter clause.
505    /// This field is needed because input refs in these clauses
506    /// are allowed to refer to any columns, while those not in filter
507    /// clause are only allowed to refer to group keys.
508    is_in_filter_clause: bool,
509}
510
511impl LogicalAggBuilder {
512    fn new(group_by: GroupBy, input_schema_len: usize) -> Result<Self> {
513        let mut input_proj_builder = ProjectBuilder::default();
514
515        let mut gen_group_key_and_grouping_sets =
516            |grouping_sets: Vec<Vec<ExprImpl>>| -> Result<(IndexSet, Vec<IndexSet>)> {
517                let grouping_sets: Vec<IndexSet> = grouping_sets
518                    .into_iter()
519                    .map(|set| {
520                        set.into_iter()
521                            .map(|expr| input_proj_builder.add_expr(&expr))
522                            .try_collect()
523                            .map_err(|err| not_implemented!("{err} inside GROUP BY"))
524                    })
525                    .try_collect()?;
526
527                // Construct group key based on grouping sets.
528                let group_key = grouping_sets
529                    .iter()
530                    .fold(FixedBitSet::with_capacity(input_schema_len), |acc, x| {
531                        acc.union(&x.to_bitset()).collect()
532                    });
533
534                Ok((IndexSet::from_iter(group_key.ones()), grouping_sets))
535            };
536
537        let (group_key, grouping_sets) = match group_by {
538            GroupBy::GroupKey(group_key) => {
539                let group_key = group_key
540                    .into_iter()
541                    .map(|expr| input_proj_builder.add_expr(&expr))
542                    .try_collect()
543                    .map_err(|err| not_implemented!("{err} inside GROUP BY"))?;
544                (group_key, vec![])
545            }
546            GroupBy::GroupingSets(grouping_sets) => gen_group_key_and_grouping_sets(grouping_sets)?,
547            GroupBy::Rollup(rollup) => {
548                // Convert rollup to grouping sets.
549                let grouping_sets = (0..=rollup.len())
550                    .map(|n| {
551                        rollup
552                            .iter()
553                            .take(n)
554                            .flat_map(|x| x.iter().cloned())
555                            .collect_vec()
556                    })
557                    .collect_vec();
558                gen_group_key_and_grouping_sets(grouping_sets)?
559            }
560            GroupBy::Cube(cube) => {
561                // Convert cube to grouping sets.
562                let grouping_sets = cube
563                    .into_iter()
564                    .powerset()
565                    .map(|x| x.into_iter().flatten().collect_vec())
566                    .collect_vec();
567                gen_group_key_and_grouping_sets(grouping_sets)?
568            }
569        };
570
571        Ok(LogicalAggBuilder {
572            group_key,
573            grouping_sets,
574            agg_calls: vec![],
575            error: None,
576            input_proj_builder,
577            is_in_filter_clause: false,
578        })
579    }
580
581    pub fn build(self, input: PlanRef) -> LogicalAgg {
582        // This LogicalProject focuses on the exprs in aggregates and GROUP BY clause.
583        let logical_project = LogicalProject::with_core(self.input_proj_builder.build(input));
584
585        // This LogicalAgg focuses on calculating the aggregates and grouping.
586        Agg::new(self.agg_calls, self.group_key, logical_project.into())
587            .with_grouping_sets(self.grouping_sets)
588            .into()
589    }
590
591    fn rewrite_with_error(&mut self, expr: ExprImpl) -> Result<ExprImpl> {
592        let rewritten_expr = self.rewrite_expr(expr);
593        if let Some(error) = self.error.take() {
594            return Err(error);
595        }
596        Ok(rewritten_expr)
597    }
598
599    /// check if the expression is a group by key, and try to return the group key
600    pub fn try_as_group_expr(&self, expr: &ExprImpl) -> Option<usize> {
601        if let Some(input_index) = self.input_proj_builder.expr_index(expr)
602            && let Some(index) = self
603                .group_key
604                .indices()
605                .position(|group_key| group_key == input_index)
606        {
607            return Some(index);
608        }
609        None
610    }
611
612    fn schema_agg_start_offset(&self) -> usize {
613        self.group_key.len()
614    }
615
616    /// Rewrite [`AggCall`] if needed, and push it into the builder using `push_agg_call`.
617    /// This is shared by [`LogicalAggBuilder`] and `LogicalOverWindowBuilder`.
618    pub(crate) fn general_rewrite_agg_call(
619        agg_call: AggCall,
620        mut push_agg_call: impl FnMut(AggCall) -> Result<InputRef>,
621    ) -> Result<ExprImpl> {
622        match agg_call.agg_type {
623            // Rewrite avg to cast(sum as avg_return_type) / count.
624            AggType::Builtin(PbAggKind::Avg) => {
625                assert_eq!(agg_call.args.len(), 1);
626
627                let sum = ExprImpl::from(push_agg_call(AggCall::new(
628                    PbAggKind::Sum.into(),
629                    agg_call.args.clone(),
630                    agg_call.distinct,
631                    agg_call.order_by.clone(),
632                    agg_call.filter.clone(),
633                    agg_call.direct_args.clone(),
634                )?)?)
635                .cast_explicit(&agg_call.return_type())?;
636
637                let count = ExprImpl::from(push_agg_call(AggCall::new(
638                    PbAggKind::Count.into(),
639                    agg_call.args.clone(),
640                    agg_call.distinct,
641                    agg_call.order_by.clone(),
642                    agg_call.filter.clone(),
643                    agg_call.direct_args.clone(),
644                )?)?);
645
646                Ok(FunctionCall::new(ExprType::Divide, Vec::from([sum, count]))?.into())
647            }
648            // We compute `var_samp` as
649            // (sum(sq) - sum * sum / count) / (count - 1)
650            // and `var_pop` as
651            // (sum(sq) - sum * sum / count) / count
652            // Since we don't have the square function, we use the plain Multiply for squaring,
653            // which is in a sense more general than the pow function, especially when calculating
654            // covariances in the future. Also we don't have the sqrt function for rooting, so we
655            // use pow(x, 0.5) to simulate
656            AggType::Builtin(
657                kind @ (PbAggKind::StddevPop
658                | PbAggKind::StddevSamp
659                | PbAggKind::VarPop
660                | PbAggKind::VarSamp),
661            ) => {
662                let arg = agg_call.args().iter().exactly_one().unwrap();
663                let squared_arg = ExprImpl::from(FunctionCall::new(
664                    ExprType::Multiply,
665                    vec![arg.clone(), arg.clone()],
666                )?);
667
668                let sum_of_sq = ExprImpl::from(push_agg_call(AggCall::new(
669                    PbAggKind::Sum.into(),
670                    vec![squared_arg],
671                    agg_call.distinct,
672                    agg_call.order_by.clone(),
673                    agg_call.filter.clone(),
674                    agg_call.direct_args.clone(),
675                )?)?)
676                .cast_explicit(&agg_call.return_type())?;
677
678                let sum = ExprImpl::from(push_agg_call(AggCall::new(
679                    PbAggKind::Sum.into(),
680                    agg_call.args.clone(),
681                    agg_call.distinct,
682                    agg_call.order_by.clone(),
683                    agg_call.filter.clone(),
684                    agg_call.direct_args.clone(),
685                )?)?)
686                .cast_explicit(&agg_call.return_type())?;
687
688                let count = ExprImpl::from(push_agg_call(AggCall::new(
689                    PbAggKind::Count.into(),
690                    agg_call.args.clone(),
691                    agg_call.distinct,
692                    agg_call.order_by.clone(),
693                    agg_call.filter.clone(),
694                    agg_call.direct_args.clone(),
695                )?)?);
696
697                let zero = ExprImpl::literal_int(0);
698                let one = ExprImpl::literal_int(1);
699
700                let squared_sum = ExprImpl::from(FunctionCall::new(
701                    ExprType::Multiply,
702                    vec![sum.clone(), sum],
703                )?);
704
705                let raw_numerator = ExprImpl::from(FunctionCall::new(
706                    ExprType::Subtract,
707                    vec![
708                        sum_of_sq,
709                        ExprImpl::from(FunctionCall::new(
710                            ExprType::Divide,
711                            vec![squared_sum, count.clone()],
712                        )?),
713                    ],
714                )?);
715
716                // We need to check for potential accuracy issues that may occasionally lead to results less than 0.
717                let numerator_type = raw_numerator.return_type();
718                let numerator = ExprImpl::from(FunctionCall::new(
719                    ExprType::Greatest,
720                    vec![raw_numerator, zero.clone().cast_explicit(&numerator_type)?],
721                )?);
722
723                let denominator = match kind {
724                    PbAggKind::VarPop | PbAggKind::StddevPop => count.clone(),
725                    PbAggKind::VarSamp | PbAggKind::StddevSamp => ExprImpl::from(
726                        FunctionCall::new(ExprType::Subtract, vec![count.clone(), one.clone()])?,
727                    ),
728                    _ => unreachable!(),
729                };
730
731                let mut target = ExprImpl::from(FunctionCall::new(
732                    ExprType::Divide,
733                    vec![numerator, denominator],
734                )?);
735
736                if matches!(kind, PbAggKind::StddevPop | PbAggKind::StddevSamp) {
737                    target = ExprImpl::from(FunctionCall::new(ExprType::Sqrt, vec![target])?);
738                }
739
740                let null = ExprImpl::from(Literal::new(None, agg_call.return_type()));
741                let case_cond = match kind {
742                    PbAggKind::VarPop | PbAggKind::StddevPop => {
743                        ExprImpl::from(FunctionCall::new(ExprType::Equal, vec![count, zero])?)
744                    }
745                    PbAggKind::VarSamp | PbAggKind::StddevSamp => ExprImpl::from(
746                        FunctionCall::new(ExprType::LessThanOrEqual, vec![count, one])?,
747                    ),
748                    _ => unreachable!(),
749                };
750
751                Ok(ExprImpl::from(FunctionCall::new(
752                    ExprType::Case,
753                    vec![case_cond, null, target],
754                )?))
755            }
756            AggType::Builtin(PbAggKind::ApproxPercentile) => {
757                if agg_call.order_by.sort_exprs[0].order_type == OrderType::descending() {
758                    // Rewrite DESC into 1.0-percentile for approx_percentile.
759                    let prev_percentile = agg_call.direct_args[0].clone();
760                    let new_percentile = 1.0
761                        - prev_percentile
762                            .get_data()
763                            .as_ref()
764                            .unwrap()
765                            .as_float64()
766                            .into_inner();
767                    let new_percentile = Some(ScalarImpl::Float64(new_percentile.into()));
768                    let new_percentile = Literal::new(new_percentile, DataType::Float64);
769                    let new_direct_args = vec![new_percentile, agg_call.direct_args[1].clone()];
770
771                    let new_agg_call = AggCall {
772                        order_by: OrderBy::any(),
773                        direct_args: new_direct_args,
774                        ..agg_call
775                    };
776                    Ok(push_agg_call(new_agg_call)?.into())
777                } else {
778                    let new_agg_call = AggCall {
779                        order_by: OrderBy::any(),
780                        ..agg_call
781                    };
782                    Ok(push_agg_call(new_agg_call)?.into())
783                }
784            }
785            _ => Ok(push_agg_call(agg_call)?.into()),
786        }
787    }
788
789    /// Push a new agg call into the builder.
790    /// Return an `InputRef` to that agg call.
791    /// For existing agg calls, return an `InputRef` to the existing one.
792    fn push_agg_call(&mut self, agg_call: AggCall) -> Result<InputRef> {
793        let AggCall {
794            agg_type,
795            return_type,
796            args,
797            distinct,
798            order_by,
799            filter,
800            direct_args,
801        } = agg_call;
802
803        self.is_in_filter_clause = true;
804        // filter expr is not added to `input_proj_builder` as a whole. Special exprs incl
805        // subquery/agg/table are rejected in `bind_agg`.
806        let filter = filter.rewrite_expr(self);
807        self.is_in_filter_clause = false;
808
809        let args: Vec<_> = args
810            .iter()
811            .map(|expr| {
812                let index = self.input_proj_builder.add_expr(expr)?;
813                Ok(InputRef::new(index, expr.return_type()))
814            })
815            .try_collect()
816            .map_err(|err: &'static str| not_implemented!("{err} inside aggregation calls"))?;
817
818        let order_by: Vec<_> = order_by
819            .sort_exprs
820            .iter()
821            .map(|e| {
822                let index = self.input_proj_builder.add_expr(&e.expr)?;
823                Ok(ColumnOrder::new(index, e.order_type))
824            })
825            .try_collect()
826            .map_err(|err: &'static str| {
827                not_implemented!("{err} inside aggregation calls order by")
828            })?;
829
830        let plan_agg_call = PlanAggCall {
831            agg_type,
832            return_type: return_type.clone(),
833            inputs: args,
834            distinct,
835            order_by,
836            filter,
837            direct_args,
838        };
839
840        if let Some((pos, existing)) = self
841            .agg_calls
842            .iter()
843            .find_position(|&c| c == &plan_agg_call)
844        {
845            return Ok(InputRef::new(
846                self.schema_agg_start_offset() + pos,
847                existing.return_type.clone(),
848            ));
849        }
850        let index = self.schema_agg_start_offset() + self.agg_calls.len();
851        self.agg_calls.push(plan_agg_call);
852        Ok(InputRef::new(index, return_type))
853    }
854
855    /// When there is an agg call, there are 3 things to do:
856    /// 1. Rewrite `avg`, `var_samp`, etc. into a combination of `sum`, `count`, etc.;
857    /// 2. Add exprs in arguments to input `Project`;
858    /// 2. Add the agg call to current `Agg`, and return an `InputRef` to it.
859    ///
860    /// Note that the rewriter does not traverse into inputs of agg calls.
861    fn try_rewrite_agg_call(&mut self, mut agg_call: AggCall) -> Result<ExprImpl> {
862        if matches!(agg_call.agg_type, agg_types::must_have_order_by!())
863            && agg_call.order_by.sort_exprs.is_empty()
864        {
865            return Err(ErrorCode::InvalidInputSyntax(format!(
866                "Aggregation function {} requires ORDER BY clause",
867                agg_call.agg_type
868            ))
869            .into());
870        }
871
872        // try ignore ORDER BY if it doesn't affect the result
873        if matches!(
874            agg_call.agg_type,
875            agg_types::result_unaffected_by_order_by!()
876        ) {
877            agg_call.order_by = OrderBy::any();
878        }
879        // try ignore DISTINCT if it doesn't affect the result
880        if matches!(
881            agg_call.agg_type,
882            agg_types::result_unaffected_by_distinct!()
883        ) {
884            agg_call.distinct = false;
885        }
886
887        if matches!(agg_call.agg_type, AggType::Builtin(PbAggKind::Grouping)) {
888            if self.grouping_sets.is_empty() {
889                return Err(ErrorCode::NotSupported(
890                    "GROUPING must be used in a query with grouping sets".into(),
891                    "try to use grouping sets instead".into(),
892                )
893                .into());
894            }
895            if agg_call.args.len() >= 32 {
896                return Err(ErrorCode::InvalidInputSyntax(
897                    "GROUPING must have fewer than 32 arguments".into(),
898                )
899                .into());
900            }
901            if agg_call
902                .args
903                .iter()
904                .any(|x| self.try_as_group_expr(x).is_none())
905            {
906                return Err(ErrorCode::InvalidInputSyntax(
907                    "arguments to GROUPING must be grouping expressions of the associated query level"
908                        .into(),
909                ).into());
910            }
911        }
912
913        Self::general_rewrite_agg_call(agg_call, |agg_call| self.push_agg_call(agg_call))
914    }
915}
916
917impl ExprRewriter for LogicalAggBuilder {
918    fn rewrite_agg_call(&mut self, agg_call: AggCall) -> ExprImpl {
919        let dummy = Literal::new(None, agg_call.return_type()).into();
920        match self.try_rewrite_agg_call(agg_call) {
921            Ok(expr) => expr,
922            Err(err) => {
923                self.error = Some(err);
924                dummy
925            }
926        }
927    }
928
929    /// When there is an `FunctionCall` (outside of agg call), it must refers to a group column.
930    /// Or all `InputRef`s appears in it must refer to a group column.
931    fn rewrite_function_call(&mut self, func_call: FunctionCall) -> ExprImpl {
932        let expr = func_call.into();
933        if let Some(group_key) = self.try_as_group_expr(&expr) {
934            InputRef::new(group_key, expr.return_type()).into()
935        } else {
936            let (func_type, inputs, ret) = expr.into_function_call().unwrap().decompose();
937            let inputs = inputs
938                .into_iter()
939                .map(|expr| self.rewrite_expr(expr))
940                .collect();
941            FunctionCall::new_unchecked(func_type, inputs, ret).into()
942        }
943    }
944
945    /// When there is an `WindowFunction` (outside of agg call), it must refers to a group column.
946    /// Or all `InputRef`s appears in it must refer to a group column.
947    fn rewrite_window_function(&mut self, window_func: WindowFunction) -> ExprImpl {
948        let WindowFunction {
949            args,
950            partition_by,
951            order_by,
952            ..
953        } = window_func;
954        let args = args
955            .into_iter()
956            .map(|expr| self.rewrite_expr(expr))
957            .collect();
958        let partition_by = partition_by
959            .into_iter()
960            .map(|expr| self.rewrite_expr(expr))
961            .collect();
962        let order_by = order_by.rewrite_expr(self);
963        WindowFunction {
964            args,
965            partition_by,
966            order_by,
967            ..window_func
968        }
969        .into()
970    }
971
972    /// When there is an `InputRef` (outside of agg call), it must refers to a group column.
973    fn rewrite_input_ref(&mut self, input_ref: InputRef) -> ExprImpl {
974        let expr = input_ref.into();
975        if let Some(group_key) = self.try_as_group_expr(&expr) {
976            InputRef::new(group_key, expr.return_type()).into()
977        } else if self.is_in_filter_clause {
978            InputRef::new(
979                self.input_proj_builder.add_expr(&expr).unwrap(),
980                expr.return_type(),
981            )
982            .into()
983        } else {
984            self.error = Some(
985                ErrorCode::InvalidInputSyntax(
986                    "column must appear in the GROUP BY clause or be used in an aggregate function"
987                        .into(),
988                )
989                .into(),
990            );
991            expr
992        }
993    }
994
995    fn rewrite_subquery(&mut self, subquery: crate::expr::Subquery) -> ExprImpl {
996        if subquery.is_correlated_by_depth(0) {
997            self.error = Some(
998                not_implemented!(
999                    issue = 2275,
1000                    "correlated subquery in HAVING or SELECT with agg",
1001                )
1002                .into(),
1003            );
1004        }
1005        subquery.into()
1006    }
1007}
1008
1009impl From<Agg<PlanRef>> for LogicalAgg {
1010    fn from(core: Agg<PlanRef>) -> Self {
1011        let base = PlanBase::new_logical_with_core(&core);
1012        Self { base, core }
1013    }
1014}
1015
1016/// Because `From`/`Into` are not transitive
1017impl From<Agg<PlanRef>> for PlanRef {
1018    fn from(core: Agg<PlanRef>) -> Self {
1019        LogicalAgg::from(core).into()
1020    }
1021}
1022
1023impl LogicalAgg {
1024    /// get the Mapping of columnIndex from input column index to out column index
1025    pub fn i2o_col_mapping(&self) -> ColIndexMapping {
1026        self.core.i2o_col_mapping()
1027    }
1028
1029    /// `create` will analyze select exprs, group exprs and having, and construct a plan like
1030    ///
1031    /// ```text
1032    /// LogicalAgg -> LogicalProject -> input
1033    /// ```
1034    ///
1035    /// It also returns the rewritten select exprs and having that reference into the aggregated
1036    /// results.
1037    pub fn create(
1038        select_exprs: Vec<ExprImpl>,
1039        group_by: GroupBy,
1040        having: Option<ExprImpl>,
1041        input: PlanRef,
1042    ) -> Result<(PlanRef, Vec<ExprImpl>, Option<ExprImpl>)> {
1043        let mut agg_builder = LogicalAggBuilder::new(group_by, input.schema().len())?;
1044
1045        let rewritten_select_exprs = select_exprs
1046            .into_iter()
1047            .map(|expr| agg_builder.rewrite_with_error(expr))
1048            .collect::<Result<_>>()?;
1049        let rewritten_having = having
1050            .map(|expr| agg_builder.rewrite_with_error(expr))
1051            .transpose()?;
1052
1053        Ok((
1054            agg_builder.build(input).into(),
1055            rewritten_select_exprs,
1056            rewritten_having,
1057        ))
1058    }
1059
1060    /// Get a reference to the logical agg's agg calls.
1061    pub fn agg_calls(&self) -> &Vec<PlanAggCall> {
1062        &self.core.agg_calls
1063    }
1064
1065    /// Get a reference to the logical agg's group key.
1066    pub fn group_key(&self) -> &IndexSet {
1067        &self.core.group_key
1068    }
1069
1070    pub fn grouping_sets(&self) -> &Vec<IndexSet> {
1071        &self.core.grouping_sets
1072    }
1073
1074    pub fn decompose(self) -> (Vec<PlanAggCall>, IndexSet, Vec<IndexSet>, PlanRef, bool) {
1075        self.core.decompose()
1076    }
1077
1078    #[must_use]
1079    pub fn rewrite_with_input_agg(
1080        &self,
1081        input: PlanRef,
1082        agg_calls: &[PlanAggCall],
1083        mut input_col_change: ColIndexMapping,
1084    ) -> (Self, ColIndexMapping) {
1085        let agg_calls = agg_calls
1086            .iter()
1087            .cloned()
1088            .map(|mut agg_call| {
1089                agg_call.inputs.iter_mut().for_each(|i| {
1090                    *i = InputRef::new(input_col_change.map(i.index()), i.return_type())
1091                });
1092                agg_call.order_by.iter_mut().for_each(|o| {
1093                    o.column_index = input_col_change.map(o.column_index);
1094                });
1095                agg_call.filter = agg_call.filter.rewrite_expr(&mut input_col_change);
1096                agg_call
1097            })
1098            .collect();
1099        // This is the group key order should be after rewriting.
1100        let group_key_in_vec: Vec<usize> = self
1101            .group_key()
1102            .indices()
1103            .map(|key| input_col_change.map(key))
1104            .collect();
1105        // This is the group key order we get after rewriting.
1106        let group_key: IndexSet = group_key_in_vec.iter().cloned().collect();
1107        let grouping_sets = self
1108            .grouping_sets()
1109            .iter()
1110            .map(|set| set.indices().map(|key| input_col_change.map(key)).collect())
1111            .collect();
1112
1113        let new_agg = Agg::new(agg_calls, group_key.clone(), input)
1114            .with_grouping_sets(grouping_sets)
1115            .with_enable_two_phase(self.core().enable_two_phase);
1116
1117        // group_key remapping might cause an output column change, since group key actually is a
1118        // `FixedBitSet`.
1119        let mut out_col_change = vec![];
1120        for idx in group_key_in_vec {
1121            let pos = group_key.indices().position(|x| x == idx).unwrap();
1122            out_col_change.push(pos);
1123        }
1124        for i in (group_key.len())..new_agg.schema().len() {
1125            out_col_change.push(i);
1126        }
1127        let out_col_change =
1128            ColIndexMapping::with_remaining_columns(&out_col_change, new_agg.schema().len());
1129
1130        (new_agg.into(), out_col_change)
1131    }
1132}
1133
1134impl PlanTreeNodeUnary<Logical> for LogicalAgg {
1135    fn input(&self) -> PlanRef {
1136        self.core.input.clone()
1137    }
1138
1139    fn clone_with_input(&self, input: PlanRef) -> Self {
1140        Agg::new(self.agg_calls().to_vec(), self.group_key().clone(), input)
1141            .with_grouping_sets(self.grouping_sets().clone())
1142            .with_enable_two_phase(self.core().enable_two_phase)
1143            .into()
1144    }
1145
1146    fn rewrite_with_input(
1147        &self,
1148        input: PlanRef,
1149        input_col_change: ColIndexMapping,
1150    ) -> (Self, ColIndexMapping) {
1151        self.rewrite_with_input_agg(input, self.agg_calls(), input_col_change)
1152    }
1153}
1154
1155impl_plan_tree_node_for_unary! { Logical, LogicalAgg }
1156impl_distill_by_unit!(LogicalAgg, core, "LogicalAgg");
1157
1158impl ExprRewritable<Logical> for LogicalAgg {
1159    fn has_rewritable_expr(&self) -> bool {
1160        true
1161    }
1162
1163    fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef {
1164        let mut core = self.core.clone();
1165        core.rewrite_exprs(r);
1166        Self {
1167            base: self.base.clone_with_new_plan_id(),
1168            core,
1169        }
1170        .into()
1171    }
1172}
1173
1174impl ExprVisitable for LogicalAgg {
1175    fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
1176        self.core.visit_exprs(v);
1177    }
1178}
1179
1180impl ColPrunable for LogicalAgg {
1181    fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
1182        let group_key_required_cols = self.group_key().to_bitset();
1183
1184        let (agg_call_required_cols, agg_calls) = {
1185            let input_cnt = self.input().schema().len();
1186            let mut tmp = FixedBitSet::with_capacity(input_cnt);
1187            let group_key_cardinality = self.group_key().len();
1188            let new_agg_calls = required_cols
1189                .iter()
1190                .filter(|&&index| index >= group_key_cardinality)
1191                .map(|&index| {
1192                    let index = index - group_key_cardinality;
1193                    let agg_call = self.agg_calls()[index].clone();
1194                    tmp.extend(agg_call.inputs.iter().map(|x| x.index()));
1195                    tmp.extend(agg_call.order_by.iter().map(|x| x.column_index));
1196                    // collect columns used in aggregate filter expressions
1197                    for i in &agg_call.filter.conjunctions {
1198                        tmp.union_with(&i.collect_input_refs(input_cnt));
1199                    }
1200                    agg_call
1201                })
1202                .collect_vec();
1203            (tmp, new_agg_calls)
1204        };
1205
1206        let input_required_cols = {
1207            let mut tmp = FixedBitSet::with_capacity(self.input().schema().len());
1208            tmp.union_with(&group_key_required_cols);
1209            tmp.union_with(&agg_call_required_cols);
1210            tmp.ones().collect_vec()
1211        };
1212        let input_col_change = ColIndexMapping::with_remaining_columns(
1213            &input_required_cols,
1214            self.input().schema().len(),
1215        );
1216        let agg = {
1217            let input = self.input().prune_col(&input_required_cols, ctx);
1218            let (agg, output_col_change) =
1219                self.rewrite_with_input_agg(input, &agg_calls, input_col_change);
1220            assert!(output_col_change.is_identity());
1221            agg
1222        };
1223        let new_output_cols = {
1224            // group key were never pruned or even re-ordered in current impl
1225            let group_key_cardinality = agg.group_key().len();
1226            let mut tmp = (0..group_key_cardinality).collect_vec();
1227            tmp.extend(
1228                required_cols
1229                    .iter()
1230                    .filter(|&&index| index >= group_key_cardinality),
1231            );
1232            tmp
1233        };
1234        if new_output_cols == required_cols {
1235            // current schema perfectly fit the required columns
1236            agg.into()
1237        } else {
1238            // some columns are not needed, or the order need to be adjusted.
1239            // so we did a projection to remove/reorder the columns.
1240            let mapping =
1241                &ColIndexMapping::with_remaining_columns(&new_output_cols, self.schema().len());
1242            let output_required_cols = required_cols
1243                .iter()
1244                .map(|&idx| mapping.map(idx))
1245                .collect_vec();
1246            let src_size = agg.schema().len();
1247            LogicalProject::with_mapping(
1248                agg.into(),
1249                ColIndexMapping::with_remaining_columns(&output_required_cols, src_size),
1250            )
1251            .into()
1252        }
1253    }
1254}
1255
1256impl PredicatePushdown for LogicalAgg {
1257    fn predicate_pushdown(
1258        &self,
1259        predicate: Condition,
1260        ctx: &mut PredicatePushdownContext,
1261    ) -> PlanRef {
1262        let num_group_key = self.group_key().len();
1263        let num_agg_calls = self.agg_calls().len();
1264        assert!(num_group_key + num_agg_calls == self.schema().len());
1265
1266        // SimpleAgg should be skipped because the predicate either references agg_calls
1267        // or is const.
1268        // If the filter references agg_calls, we can not push it.
1269        // When it is constantly true, pushing is useless and may actually cause more evaluation
1270        // cost of the predicate.
1271        // When it is constantly false, pushing is wrong - the old plan returns 0 rows but new one
1272        // returns 1 row.
1273        if num_group_key == 0 {
1274            return gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx);
1275        }
1276
1277        // If the filter references agg_calls, we can not push it.
1278        let mut agg_call_columns = FixedBitSet::with_capacity(num_group_key + num_agg_calls);
1279        agg_call_columns.insert_range(num_group_key..num_group_key + num_agg_calls);
1280        let (agg_call_pred, pushed_predicate) = predicate.split_disjoint(&agg_call_columns);
1281
1282        // convert the predicate to one that references the child of the agg
1283        let mut subst = Substitute {
1284            mapping: self
1285                .group_key()
1286                .indices()
1287                .enumerate()
1288                .map(|(i, group_key)| {
1289                    InputRef::new(group_key, self.schema().fields()[i].data_type()).into()
1290                })
1291                .collect(),
1292        };
1293        let pushed_predicate = pushed_predicate.rewrite_expr(&mut subst);
1294
1295        gen_filter_and_pushdown(self, agg_call_pred, pushed_predicate, ctx)
1296    }
1297}
1298
1299impl ToBatch for LogicalAgg {
1300    fn to_batch(&self) -> Result<crate::optimizer::plan_node::BatchPlanRef> {
1301        self.to_batch_with_order_required(&Order::any())
1302    }
1303
1304    // TODO(rc): `to_batch_with_order_required` seems to be useless after we decide to use
1305    // `BatchSortAgg` only when input is already sorted
1306    fn to_batch_with_order_required(
1307        &self,
1308        required_order: &Order,
1309    ) -> Result<crate::optimizer::plan_node::BatchPlanRef> {
1310        let input = self.input().to_batch()?;
1311        let new_logical = self.core.clone_with_input(input);
1312        let agg_plan = if self.group_key().is_empty() {
1313            BatchSimpleAgg::new(new_logical).into()
1314        } else if self.ctx().session_ctx().config().batch_enable_sort_agg()
1315            && new_logical.input_provides_order_on_group_keys()
1316        {
1317            BatchSortAgg::new(new_logical).into()
1318        } else {
1319            BatchHashAgg::new(new_logical).into()
1320        };
1321        required_order.enforce_if_not_satisfies(agg_plan)
1322    }
1323}
1324
1325fn find_or_append_row_count(mut logical: Agg<StreamPlanRef>) -> (Agg<StreamPlanRef>, usize) {
1326    // `HashAgg`/`SimpleAgg` executors require a `count(*)` to correctly build changes, so
1327    // append a `count(*)` if not exists.
1328    let count_star = PlanAggCall::count_star();
1329    let row_count_idx = if let Some((idx, _)) = logical
1330        .agg_calls
1331        .iter()
1332        .find_position(|&c| c == &count_star)
1333    {
1334        idx
1335    } else {
1336        let idx = logical.agg_calls.len();
1337        logical.agg_calls.push(count_star);
1338        idx
1339    };
1340    (logical, row_count_idx)
1341}
1342
1343fn new_stream_simple_agg(
1344    core: Agg<StreamPlanRef>,
1345    must_output_per_barrier: bool,
1346) -> Result<StreamSimpleAgg> {
1347    let (logical, row_count_idx) = find_or_append_row_count(core);
1348    StreamSimpleAgg::new(logical, row_count_idx, must_output_per_barrier)
1349}
1350
1351fn new_stream_hash_agg(
1352    core: Agg<StreamPlanRef>,
1353    vnode_col_idx: Option<usize>,
1354) -> Result<StreamHashAgg> {
1355    let (logical, row_count_idx) = find_or_append_row_count(core);
1356    StreamHashAgg::new(logical, vnode_col_idx, row_count_idx)
1357}
1358
1359impl ToStream for LogicalAgg {
1360    fn to_stream(&self, ctx: &mut ToStreamContext) -> Result<StreamPlanRef> {
1361        use super::stream::prelude::*;
1362
1363        for agg_call in self.agg_calls() {
1364            if matches!(agg_call.agg_type, agg_types::unimplemented_in_stream!()) {
1365                bail_not_implemented!("{} aggregation in materialized view", agg_call.agg_type);
1366            }
1367        }
1368        let eowc = ctx.emit_on_window_close();
1369        let stream_input = self.input().to_stream(ctx)?;
1370
1371        // Use Dedup operator, if possible.
1372        if stream_input.append_only() && self.agg_calls().is_empty() && !self.group_key().is_empty()
1373        {
1374            let input = if self.group_key().len() != self.input().schema().len() {
1375                let cols = &self.group_key().to_vec();
1376                LogicalProject::with_mapping(
1377                    self.input(),
1378                    ColIndexMapping::with_remaining_columns(cols, self.input().schema().len()),
1379                )
1380                .into()
1381            } else {
1382                self.input()
1383            };
1384            let input_schema_len = input.schema().len();
1385            let logical_dedup = LogicalDedup::new(input, (0..input_schema_len).collect());
1386            return logical_dedup.to_stream(ctx);
1387        }
1388
1389        if self.agg_calls().iter().any(|call| {
1390            matches!(
1391                call.agg_type,
1392                AggType::Builtin(PbAggKind::ApproxCountDistinct)
1393            )
1394        }) {
1395            if stream_input.append_only() {
1396                self.core.ctx().session_ctx().notice_to_user(
1397                    "Streaming `APPROX_COUNT_DISTINCT` is still a preview feature and subject to change. Please do not use it in production environment.",
1398                );
1399            } else {
1400                bail_not_implemented!(
1401                    "Streaming `APPROX_COUNT_DISTINCT` is only supported in append-only stream"
1402                );
1403            }
1404        }
1405
1406        let plan = self.gen_dist_stream_agg_plan(stream_input)?;
1407
1408        let (plan, n_final_agg_calls) = if let Some(final_agg) = plan.as_stream_simple_agg() {
1409            if eowc {
1410                return Err(ErrorCode::InvalidInputSyntax(
1411                    "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`"
1412                        .to_owned(),
1413                )
1414                .into());
1415            }
1416            (plan.clone(), final_agg.agg_calls().len())
1417        } else if let Some(final_agg) = plan.as_stream_hash_agg() {
1418            (
1419                if eowc {
1420                    final_agg.to_eowc_version()?
1421                } else {
1422                    plan.clone()
1423                },
1424                final_agg.agg_calls().len(),
1425            )
1426        } else if let Some(_approx_percentile_agg) = plan.as_stream_global_approx_percentile() {
1427            if eowc {
1428                return Err(ErrorCode::InvalidInputSyntax(
1429                    "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`"
1430                        .to_owned(),
1431                )
1432                .into());
1433            }
1434            (plan.clone(), 1)
1435        } else if let Some(stream_row_merge) = plan.as_stream_row_merge() {
1436            if eowc {
1437                return Err(ErrorCode::InvalidInputSyntax(
1438                    "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`"
1439                        .to_owned(),
1440                )
1441                .into());
1442            }
1443            (plan.clone(), stream_row_merge.base.schema().len())
1444        } else {
1445            panic!(
1446                "the root PlanNode must be StreamHashAgg, StreamSimpleAgg, StreamGlobalApproxPercentile, or StreamRowMerge"
1447            );
1448        };
1449
1450        if self.agg_calls().len() == n_final_agg_calls {
1451            // an existing `count(*)` is used as row count column in `StreamXxxAgg`
1452            Ok(plan)
1453        } else {
1454            // a `count(*)` is appended, should project the output
1455            assert_eq!(self.agg_calls().len() + 1, n_final_agg_calls);
1456            Ok(StreamProject::new(generic::Project::with_out_col_idx(
1457                plan,
1458                0..self.schema().len(),
1459            ))
1460            // If there's no agg call, then `count(*)` will be the only column in the output besides keys.
1461            // Since it'll be pruned immediately in `StreamProject`, the update records are likely to be
1462            // no-op. So we set the hint to instruct the executor to eliminate them.
1463            // See https://github.com/risingwavelabs/risingwave/issues/17030.
1464            .with_noop_update_hint(self.agg_calls().is_empty())
1465            .into())
1466        }
1467    }
1468
1469    fn logical_rewrite_for_stream(
1470        &self,
1471        ctx: &mut RewriteStreamContext,
1472    ) -> Result<(PlanRef, ColIndexMapping)> {
1473        let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?;
1474        let (agg, out_col_change) = self.rewrite_with_input(input, input_col_change);
1475        let (map, _) = out_col_change.into_parts();
1476        let out_col_change = ColIndexMapping::new(map, agg.schema().len());
1477        Ok((agg.into(), out_col_change))
1478    }
1479}
1480
1481#[cfg(test)]
1482mod tests {
1483    use risingwave_common::catalog::{Field, Schema};
1484
1485    use super::*;
1486    use crate::expr::{assert_eq_input_ref, input_ref_to_column_indices};
1487    use crate::optimizer::optimizer_context::OptimizerContext;
1488    use crate::optimizer::plan_node::LogicalValues;
1489
1490    #[tokio::test]
1491    async fn test_create() {
1492        let ty = DataType::Int32;
1493        let ctx = OptimizerContext::mock().await;
1494        let fields: Vec<Field> = vec![
1495            Field::with_name(ty.clone(), "v1"),
1496            Field::with_name(ty.clone(), "v2"),
1497            Field::with_name(ty.clone(), "v3"),
1498        ];
1499        let values = LogicalValues::new(vec![], Schema { fields }, ctx);
1500        let input = PlanRef::from(values);
1501        let input_ref_1 = InputRef::new(0, ty.clone());
1502        let input_ref_2 = InputRef::new(1, ty.clone());
1503        let input_ref_3 = InputRef::new(2, ty.clone());
1504
1505        let gen_internal_value = |select_exprs: Vec<ExprImpl>,
1506                                  group_exprs|
1507         -> (Vec<ExprImpl>, Vec<PlanAggCall>, IndexSet) {
1508            let (plan, exprs, _) = LogicalAgg::create(
1509                select_exprs,
1510                GroupBy::GroupKey(group_exprs),
1511                None,
1512                input.clone(),
1513            )
1514            .unwrap();
1515
1516            let logical_agg = plan.as_logical_agg().unwrap();
1517            let agg_calls = logical_agg.agg_calls().to_vec();
1518            let group_key = logical_agg.group_key().clone();
1519
1520            (exprs, agg_calls, group_key)
1521        };
1522
1523        // Test case: select v1 from test group by v1;
1524        {
1525            let select_exprs = vec![input_ref_1.clone().into()];
1526            let group_exprs = vec![input_ref_1.clone().into()];
1527
1528            let (exprs, agg_calls, group_key) = gen_internal_value(select_exprs, group_exprs);
1529
1530            assert_eq!(exprs.len(), 1);
1531            assert_eq_input_ref!(&exprs[0], 0);
1532
1533            assert_eq!(agg_calls.len(), 0);
1534            assert_eq!(group_key, vec![0].into());
1535        }
1536
1537        // Test case: select v1, min(v2) from test group by v1;
1538        {
1539            let min_v2 = AggCall::new(
1540                PbAggKind::Min.into(),
1541                vec![input_ref_2.clone().into()],
1542                false,
1543                OrderBy::any(),
1544                Condition::true_cond(),
1545                vec![],
1546            )
1547            .unwrap();
1548            let select_exprs = vec![input_ref_1.clone().into(), min_v2.into()];
1549            let group_exprs = vec![input_ref_1.clone().into()];
1550
1551            let (exprs, agg_calls, group_key) = gen_internal_value(select_exprs, group_exprs);
1552
1553            assert_eq!(exprs.len(), 2);
1554            assert_eq_input_ref!(&exprs[0], 0);
1555            assert_eq_input_ref!(&exprs[1], 1);
1556
1557            assert_eq!(agg_calls.len(), 1);
1558            assert_eq!(agg_calls[0].agg_type, PbAggKind::Min.into());
1559            assert_eq!(input_ref_to_column_indices(&agg_calls[0].inputs), vec![1]);
1560            assert_eq!(group_key, vec![0].into());
1561        }
1562
1563        // Test case: select v1, min(v2) + max(v3) from t group by v1;
1564        {
1565            let min_v2 = AggCall::new(
1566                PbAggKind::Min.into(),
1567                vec![input_ref_2.clone().into()],
1568                false,
1569                OrderBy::any(),
1570                Condition::true_cond(),
1571                vec![],
1572            )
1573            .unwrap();
1574            let max_v3 = AggCall::new(
1575                PbAggKind::Max.into(),
1576                vec![input_ref_3.clone().into()],
1577                false,
1578                OrderBy::any(),
1579                Condition::true_cond(),
1580                vec![],
1581            )
1582            .unwrap();
1583            let func_call =
1584                FunctionCall::new(ExprType::Add, vec![min_v2.into(), max_v3.into()]).unwrap();
1585            let select_exprs = vec![input_ref_1.clone().into(), ExprImpl::from(func_call)];
1586            let group_exprs = vec![input_ref_1.clone().into()];
1587
1588            let (exprs, agg_calls, group_key) = gen_internal_value(select_exprs, group_exprs);
1589
1590            assert_eq_input_ref!(&exprs[0], 0);
1591            if let ExprImpl::FunctionCall(func_call) = &exprs[1] {
1592                assert_eq!(func_call.func_type(), ExprType::Add);
1593                let inputs = func_call.inputs();
1594                assert_eq_input_ref!(&inputs[0], 1);
1595                assert_eq_input_ref!(&inputs[1], 2);
1596            } else {
1597                panic!("Wrong expression type!");
1598            }
1599
1600            assert_eq!(agg_calls.len(), 2);
1601            assert_eq!(agg_calls[0].agg_type, PbAggKind::Min.into());
1602            assert_eq!(input_ref_to_column_indices(&agg_calls[0].inputs), vec![1]);
1603            assert_eq!(agg_calls[1].agg_type, PbAggKind::Max.into());
1604            assert_eq!(input_ref_to_column_indices(&agg_calls[1].inputs), vec![2]);
1605            assert_eq!(group_key, vec![0].into());
1606        }
1607
1608        // Test case: select v2, min(v1 * v3) from test group by v2;
1609        {
1610            let v1_mult_v3 = FunctionCall::new(
1611                ExprType::Multiply,
1612                vec![input_ref_1.into(), input_ref_3.into()],
1613            )
1614            .unwrap();
1615            let agg_call = AggCall::new(
1616                PbAggKind::Min.into(),
1617                vec![v1_mult_v3.into()],
1618                false,
1619                OrderBy::any(),
1620                Condition::true_cond(),
1621                vec![],
1622            )
1623            .unwrap();
1624            let select_exprs = vec![input_ref_2.clone().into(), agg_call.into()];
1625            let group_exprs = vec![input_ref_2.into()];
1626
1627            let (exprs, agg_calls, group_key) = gen_internal_value(select_exprs, group_exprs);
1628
1629            assert_eq_input_ref!(&exprs[0], 0);
1630            assert_eq_input_ref!(&exprs[1], 1);
1631
1632            assert_eq!(agg_calls.len(), 1);
1633            assert_eq!(agg_calls[0].agg_type, PbAggKind::Min.into());
1634            assert_eq!(input_ref_to_column_indices(&agg_calls[0].inputs), vec![1]);
1635            assert_eq!(group_key, vec![0].into());
1636        }
1637    }
1638
1639    /// Generate a agg call node with given [`DataType`] and fields.
1640    /// For example, `generate_agg_call(Int32, [v1, v2, v3])` will result in:
1641    /// ```text
1642    /// Agg(min(input_ref(2))) group by (input_ref(1))
1643    ///   TableScan(v1, v2, v3)
1644    /// ```
1645    async fn generate_agg_call(ty: DataType, fields: Vec<Field>) -> LogicalAgg {
1646        let ctx = OptimizerContext::mock().await;
1647
1648        let values = LogicalValues::new(vec![], Schema { fields }, ctx);
1649        let agg_call = PlanAggCall {
1650            agg_type: PbAggKind::Min.into(),
1651            return_type: ty.clone(),
1652            inputs: vec![InputRef::new(2, ty.clone())],
1653            distinct: false,
1654            order_by: vec![],
1655            filter: Condition::true_cond(),
1656            direct_args: vec![],
1657        };
1658        Agg::new(vec![agg_call], vec![1].into(), values.into()).into()
1659    }
1660
1661    #[tokio::test]
1662    /// Pruning
1663    /// ```text
1664    /// Agg(min(input_ref(2))) group by (input_ref(1))
1665    ///   TableScan(v1, v2, v3)
1666    /// ```
1667    /// with required columns [0,1] (all columns) will result in
1668    /// ```text
1669    /// Agg(min(input_ref(1))) group by (input_ref(0))
1670    ///  TableScan(v2, v3)
1671    /// ```
1672    async fn test_prune_all() {
1673        let ty = DataType::Int32;
1674        let fields: Vec<Field> = vec![
1675            Field::with_name(ty.clone(), "v1"),
1676            Field::with_name(ty.clone(), "v2"),
1677            Field::with_name(ty.clone(), "v3"),
1678        ];
1679        let agg: PlanRef = generate_agg_call(ty.clone(), fields.clone()).await.into();
1680        // Perform the prune
1681        let required_cols = vec![0, 1];
1682        let plan = agg.prune_col(&required_cols, &mut ColumnPruningContext::new(agg.clone()));
1683
1684        // Check the result
1685        let agg_new = plan.as_logical_agg().unwrap();
1686        assert_eq!(agg_new.group_key(), &vec![0].into());
1687
1688        assert_eq!(agg_new.agg_calls().len(), 1);
1689        let agg_call_new = agg_new.agg_calls()[0].clone();
1690        assert_eq!(agg_call_new.agg_type, PbAggKind::Min.into());
1691        assert_eq!(input_ref_to_column_indices(&agg_call_new.inputs), vec![1]);
1692        assert_eq!(agg_call_new.return_type, ty);
1693
1694        let values = agg_new.input();
1695        let values = values.as_logical_values().unwrap();
1696        assert_eq!(values.schema().fields(), &fields[1..]);
1697    }
1698
1699    #[tokio::test]
1700    /// Pruning
1701    /// ```text
1702    /// Agg(min(input_ref(2))) group by (input_ref(1))
1703    ///   TableScan(v1, v2, v3)
1704    /// ```
1705    /// with required columns [1,0] (all columns, with reversed order) will result in
1706    /// ```text
1707    /// Project [input_ref(1), input_ref(0)]
1708    ///   Agg(min(input_ref(1))) group by (input_ref(0))
1709    ///     TableScan(v2, v3)
1710    /// ```
1711    async fn test_prune_all_with_order_required() {
1712        let ty = DataType::Int32;
1713        let fields: Vec<Field> = vec![
1714            Field::with_name(ty.clone(), "v1"),
1715            Field::with_name(ty.clone(), "v2"),
1716            Field::with_name(ty.clone(), "v3"),
1717        ];
1718        let agg: PlanRef = generate_agg_call(ty.clone(), fields.clone()).await.into();
1719        // Perform the prune
1720        let required_cols = vec![1, 0];
1721        let plan = agg.prune_col(&required_cols, &mut ColumnPruningContext::new(agg.clone()));
1722        // Check the result
1723        let proj = plan.as_logical_project().unwrap();
1724        assert_eq!(proj.exprs().len(), 2);
1725        assert_eq!(proj.exprs()[0].as_input_ref().unwrap().index(), 1);
1726        assert_eq!(proj.exprs()[1].as_input_ref().unwrap().index(), 0);
1727        let proj_input = proj.input();
1728        let agg_new = proj_input.as_logical_agg().unwrap();
1729        assert_eq!(agg_new.group_key(), &vec![0].into());
1730
1731        assert_eq!(agg_new.agg_calls().len(), 1);
1732        let agg_call_new = agg_new.agg_calls()[0].clone();
1733        assert_eq!(agg_call_new.agg_type, PbAggKind::Min.into());
1734        assert_eq!(input_ref_to_column_indices(&agg_call_new.inputs), vec![1]);
1735        assert_eq!(agg_call_new.return_type, ty);
1736
1737        let values = agg_new.input();
1738        let values = values.as_logical_values().unwrap();
1739        assert_eq!(values.schema().fields(), &fields[1..]);
1740    }
1741
1742    #[tokio::test]
1743    /// Pruning
1744    /// ```text
1745    /// Agg(min(input_ref(2))) group by (input_ref(1))
1746    ///   TableScan(v1, v2, v3)
1747    /// ```
1748    /// with required columns [1] (group key removed) will result in
1749    /// ```text
1750    /// Project(input_ref(1))
1751    ///   Agg(min(input_ref(1))) group by (input_ref(0))
1752    ///     TableScan(v2, v3)
1753    /// ```
1754    async fn test_prune_group_key() {
1755        let ctx = OptimizerContext::mock().await;
1756        let ty = DataType::Int32;
1757        let fields: Vec<Field> = vec![
1758            Field::with_name(ty.clone(), "v1"),
1759            Field::with_name(ty.clone(), "v2"),
1760            Field::with_name(ty.clone(), "v3"),
1761        ];
1762        let values: LogicalValues = LogicalValues::new(
1763            vec![],
1764            Schema {
1765                fields: fields.clone(),
1766            },
1767            ctx,
1768        );
1769        let agg_call = PlanAggCall {
1770            agg_type: PbAggKind::Min.into(),
1771            return_type: ty.clone(),
1772            inputs: vec![InputRef::new(2, ty.clone())],
1773            distinct: false,
1774            order_by: vec![],
1775            filter: Condition::true_cond(),
1776            direct_args: vec![],
1777        };
1778        let agg: PlanRef = Agg::new(vec![agg_call], vec![1].into(), values.into()).into();
1779
1780        // Perform the prune
1781        let required_cols = vec![1];
1782        let plan = agg.prune_col(&required_cols, &mut ColumnPruningContext::new(agg.clone()));
1783
1784        // Check the result
1785        let project = plan.as_logical_project().unwrap();
1786        assert_eq!(project.exprs().len(), 1);
1787        assert_eq_input_ref!(&project.exprs()[0], 1);
1788
1789        let agg_new = project.input();
1790        let agg_new = agg_new.as_logical_agg().unwrap();
1791        assert_eq!(agg_new.group_key(), &vec![0].into());
1792
1793        assert_eq!(agg_new.agg_calls().len(), 1);
1794        let agg_call_new = agg_new.agg_calls()[0].clone();
1795        assert_eq!(agg_call_new.agg_type, PbAggKind::Min.into());
1796        assert_eq!(input_ref_to_column_indices(&agg_call_new.inputs), vec![1]);
1797        assert_eq!(agg_call_new.return_type, ty);
1798
1799        let values = agg_new.input();
1800        let values = values.as_logical_values().unwrap();
1801        assert_eq!(values.schema().fields(), &fields[1..]);
1802    }
1803
1804    #[tokio::test]
1805    /// Pruning
1806    /// ```text
1807    /// Agg(min(input_ref(2)), max(input_ref(1))) group by (input_ref(1), input_ref(2))
1808    ///   TableScan(v1, v2, v3)
1809    /// ```
1810    /// with required columns [0,3] will result in
1811    /// ```text
1812    /// Project(input_ref(0), input_ref(2))
1813    ///   Agg(max(input_ref(0))) group by (input_ref(0), input_ref(1))
1814    ///     TableScan(v2, v3)
1815    /// ```
1816    async fn test_prune_agg() {
1817        let ty = DataType::Int32;
1818        let ctx = OptimizerContext::mock().await;
1819        let fields: Vec<Field> = vec![
1820            Field::with_name(ty.clone(), "v1"),
1821            Field::with_name(ty.clone(), "v2"),
1822            Field::with_name(ty.clone(), "v3"),
1823        ];
1824        let values = LogicalValues::new(
1825            vec![],
1826            Schema {
1827                fields: fields.clone(),
1828            },
1829            ctx,
1830        );
1831
1832        let agg_calls = vec![
1833            PlanAggCall {
1834                agg_type: PbAggKind::Min.into(),
1835                return_type: ty.clone(),
1836                inputs: vec![InputRef::new(2, ty.clone())],
1837                distinct: false,
1838                order_by: vec![],
1839                filter: Condition::true_cond(),
1840                direct_args: vec![],
1841            },
1842            PlanAggCall {
1843                agg_type: PbAggKind::Max.into(),
1844                return_type: ty.clone(),
1845                inputs: vec![InputRef::new(1, ty.clone())],
1846                distinct: false,
1847                order_by: vec![],
1848                filter: Condition::true_cond(),
1849                direct_args: vec![],
1850            },
1851        ];
1852        let agg: PlanRef = Agg::new(agg_calls, vec![1, 2].into(), values.into()).into();
1853
1854        // Perform the prune
1855        let required_cols = vec![0, 3];
1856        let plan = agg.prune_col(&required_cols, &mut ColumnPruningContext::new(agg.clone()));
1857        // Check the result
1858        let project = plan.as_logical_project().unwrap();
1859        assert_eq!(project.exprs().len(), 2);
1860        assert_eq_input_ref!(&project.exprs()[0], 0);
1861        assert_eq_input_ref!(&project.exprs()[1], 2);
1862
1863        let agg_new = project.input();
1864        let agg_new = agg_new.as_logical_agg().unwrap();
1865        assert_eq!(agg_new.group_key(), &vec![0, 1].into());
1866
1867        assert_eq!(agg_new.agg_calls().len(), 1);
1868        let agg_call_new = agg_new.agg_calls()[0].clone();
1869        assert_eq!(agg_call_new.agg_type, PbAggKind::Max.into());
1870        assert_eq!(input_ref_to_column_indices(&agg_call_new.inputs), vec![0]);
1871        assert_eq!(agg_call_new.return_type, ty);
1872
1873        let values = agg_new.input();
1874        let values = values.as_logical_values().unwrap();
1875        assert_eq!(values.schema().fields(), &fields[1..]);
1876    }
1877}