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//
15// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
16// This source code is licensed under both the GPLv2 (found in the
17// COPYING file in the root directory) and Apache 2.0 License
18// (found in the LICENSE.Apache file in the root directory).
19
20use risingwave_pb::batch_plan::iceberg_scan_node::IcebergScanType;
21
22use super::prelude::*;
23use crate::optimizer::plan_node::{BatchIcebergScan, PlanAggCall};
24
25pub struct BatchIcebergCountStar {}
26
27impl Rule<Batch> for BatchIcebergCountStar {
28 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
29 let agg = plan.as_batch_simple_agg()?;
30 if agg.core.group_key.is_empty()
31 && agg.agg_calls().len() == 1
32 && agg.agg_calls()[0].eq(&PlanAggCall::count_star())
33 {
34 let batch_iceberg = agg.core.input.as_batch_iceberg_scan()?;
35 if batch_iceberg.iceberg_scan_type() != IcebergScanType::DataScan {
36 return None;
37 }
38 return Some(
39 BatchIcebergScan::new_count_star_with_batch_iceberg_scan(batch_iceberg).into(),
40 );
41 }
42 None
43 }
44}
45
46impl BatchIcebergCountStar {
47 pub fn create() -> BoxedRule {
48 Box::new(BatchIcebergCountStar {})
49 }
50}