risingwave_frontend/optimizer/plan_expr_rewriter/
cse_rewriter.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 std::collections::HashMap;
16
17use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef};
18use crate::optimizer::plan_expr_visitor::CseExprCounter;
19
20#[derive(Default)]
21pub struct CseRewriter {
22    pub expr_counter: CseExprCounter,
23    pub cse_input_ref_offset: usize,
24    pub cse_mapping: HashMap<FunctionCall, InputRef>,
25}
26
27impl CseRewriter {
28    pub fn new(expr_counter: CseExprCounter, cse_input_ref_offset: usize) -> Self {
29        Self {
30            expr_counter,
31            cse_input_ref_offset,
32            cse_mapping: HashMap::default(),
33        }
34    }
35}
36
37impl ExprRewriter for CseRewriter {
38    fn rewrite_function_call(&mut self, func_call: FunctionCall) -> ExprImpl {
39        if let Some(count) = self.expr_counter.counter.get(&func_call)
40            && *count > 1
41        {
42            if let Some(expr) = self.cse_mapping.get(&func_call) {
43                let expr: ExprImpl = ExprImpl::InputRef(expr.clone().into());
44                return expr;
45            }
46            let input_ref = InputRef::new(self.cse_input_ref_offset, func_call.return_type());
47            self.cse_input_ref_offset += 1;
48            self.cse_mapping.insert(func_call, input_ref.clone());
49            return ExprImpl::InputRef(input_ref.into());
50        }
51
52        let (func_type, inputs, ret) = func_call.decompose();
53        let inputs = inputs
54            .into_iter()
55            .map(|expr| self.rewrite_expr(expr))
56            .collect();
57        FunctionCall::new_unchecked(func_type, inputs, ret).into()
58    }
59}