risingwave_frontend/optimizer/plan_node/
logical_cte_ref.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
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
// 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 risingwave_common::bail_not_implemented;
use risingwave_common::util::column_index_mapping::ColIndexMapping;

use super::expr_visitable::ExprVisitable;
use super::utils::impl_distill_by_unit;
use super::{
    generic, ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalProject, PlanBase,
    PredicatePushdown, PredicatePushdownContext, RewriteStreamContext, ToBatch, ToStream,
    ToStreamContext,
};
use crate::binder::ShareId;
use crate::error::Result;
use crate::utils::Condition;
use crate::PlanRef;

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct LogicalCteRef {
    pub base: PlanBase<Logical>,
    core: generic::CteRef<PlanRef>,
}

impl LogicalCteRef {
    pub fn new(share_id: ShareId, base_plan: PlanRef) -> Self {
        let core = generic::CteRef::new(share_id, base_plan);
        let base = PlanBase::new_logical_with_core(&core);
        Self { base, core }
    }

    pub fn create(share_id: ShareId, base_plan: PlanRef) -> PlanRef {
        Self::new(share_id, base_plan).into()
    }
}

impl_plan_tree_node_for_leaf! {LogicalCteRef}

impl_distill_by_unit! {LogicalCteRef, core, "LogicalCteRef"}

impl ExprRewritable for LogicalCteRef {}

impl ExprVisitable for LogicalCteRef {}

impl ColPrunable for LogicalCteRef {
    fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
        LogicalProject::with_out_col_idx(self.clone().into(), required_cols.iter().copied()).into()
    }
}

impl PredicatePushdown for LogicalCteRef {
    fn predicate_pushdown(
        &self,
        _predicate: Condition,
        _ctx: &mut PredicatePushdownContext,
    ) -> PlanRef {
        self.clone().into()
    }
}

impl ToBatch for LogicalCteRef {
    fn to_batch(&self) -> Result<PlanRef> {
        bail_not_implemented!(
            issue = 15135,
            "recursive CTE not supported for to_batch of LogicalCteRef"
        )
    }
}

impl ToStream for LogicalCteRef {
    fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result<PlanRef> {
        bail_not_implemented!(
            issue = 15135,
            "recursive CTE not supported for to_stream of LogicalCteRef"
        )
    }

    fn logical_rewrite_for_stream(
        &self,
        _ctx: &mut RewriteStreamContext,
    ) -> Result<(PlanRef, ColIndexMapping)> {
        bail_not_implemented!(
            issue = 15135,
            "recursive CTE not supported for logical_rewrite_for_stream of LogicalCteRef"
        )
    }
}