risingwave_frontend/optimizer/plan_expr_rewriter/
cse_rewriter.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
// Copyright 2024 RisingWave Labs
//
// Licensed 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.

use std::collections::HashMap;

use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef};
use crate::optimizer::plan_expr_visitor::CseExprCounter;

#[derive(Default)]
pub struct CseRewriter {
    pub expr_counter: CseExprCounter,
    pub cse_input_ref_offset: usize,
    pub cse_mapping: HashMap<FunctionCall, InputRef>,
}

impl CseRewriter {
    pub fn new(expr_counter: CseExprCounter, cse_input_ref_offset: usize) -> Self {
        Self {
            expr_counter,
            cse_input_ref_offset,
            cse_mapping: HashMap::default(),
        }
    }
}

impl ExprRewriter for CseRewriter {
    fn rewrite_function_call(&mut self, func_call: FunctionCall) -> ExprImpl {
        if let Some(count) = self.expr_counter.counter.get(&func_call)
            && *count > 1
        {
            if let Some(expr) = self.cse_mapping.get(&func_call) {
                let expr: ExprImpl = ExprImpl::InputRef(expr.clone().into());
                return expr;
            }
            let input_ref = InputRef::new(self.cse_input_ref_offset, func_call.return_type());
            self.cse_input_ref_offset += 1;
            self.cse_mapping.insert(func_call, input_ref.clone());
            return ExprImpl::InputRef(input_ref.into());
        }

        let (func_type, inputs, ret) = func_call.decompose();
        let inputs = inputs
            .into_iter()
            .map(|expr| self.rewrite_expr(expr))
            .collect();
        FunctionCall::new_unchecked(func_type, inputs, ret).into()
    }
}