risingwave_frontend/optimizer/rule/batch/
batch_iceberg_count_star.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 risingwave_pb::batch_plan::iceberg_scan_node::IcebergScanType;
16
17use super::prelude::*;
18use crate::optimizer::plan_node::{BatchIcebergScan, PlanAggCall};
19
20pub struct BatchIcebergCountStar {}
21
22impl Rule<Batch> for BatchIcebergCountStar {
23    fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
24        let agg = plan.as_batch_simple_agg()?;
25        if agg.core.group_key.is_empty()
26            && agg.agg_calls().len() == 1
27            && agg.agg_calls()[0].eq(&PlanAggCall::count_star())
28        {
29            let batch_iceberg = agg.core.input.as_batch_iceberg_scan()?;
30            if batch_iceberg.iceberg_scan_type() != IcebergScanType::DataScan {
31                return None;
32            }
33            return Some(
34                BatchIcebergScan::new_count_star_with_batch_iceberg_scan(batch_iceberg).into(),
35            );
36        }
37        None
38    }
39}
40
41impl BatchIcebergCountStar {
42    pub fn create() -> BoxedRule {
43        Box::new(BatchIcebergCountStar {})
44    }
45}