risingwave_frontend/planner/
mod.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
// Copyright 2025 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::binder::{BoundStatement, ShareId};
use crate::error::Result;
use crate::optimizer::{OptimizerContextRef, PlanRoot};

mod changelog;
mod delete;
mod insert;
mod query;
mod recursive_union;
mod relation;
mod select;
mod set_expr;
mod set_operation;
mod statement;
mod update;
mod values;
pub use query::LIMIT_ALL_COUNT;

use crate::PlanRef;

/// `Planner` converts a bound statement to a [`crate::optimizer::plan_node::PlanNode`] tree
pub struct Planner {
    ctx: OptimizerContextRef,
    /// Mapping of `ShareId` to its share plan.
    /// The share plan can be a CTE, a source, a view and so on.
    share_cache: HashMap<ShareId, PlanRef>,
    /// Plan for stream or batch.
    plan_for: PlanFor,
}

#[derive(Debug, Copy, Clone)]
pub enum PlanFor {
    Stream,
    Batch,
    /// `BatchDql` is a special mode for batch.
    /// Iceberg engine table will be converted to iceberg source based on this mode.
    BatchDql,
}

impl Planner {
    pub fn new_for_batch_dql(ctx: OptimizerContextRef) -> Planner {
        Planner {
            ctx,
            share_cache: Default::default(),
            plan_for: PlanFor::BatchDql,
        }
    }

    pub fn new_for_batch(ctx: OptimizerContextRef) -> Planner {
        Planner {
            ctx,
            share_cache: Default::default(),
            plan_for: PlanFor::Batch,
        }
    }

    pub fn new_for_stream(ctx: OptimizerContextRef) -> Planner {
        Planner {
            ctx,
            share_cache: Default::default(),
            plan_for: PlanFor::Stream,
        }
    }

    /// Plan a [`BoundStatement`]. Need to bind a statement before plan.
    pub fn plan(&mut self, stmt: BoundStatement) -> Result<PlanRoot> {
        self.plan_statement(stmt)
    }

    pub fn ctx(&self) -> OptimizerContextRef {
        self.ctx.clone()
    }

    pub fn plan_for(&self) -> PlanFor {
        self.plan_for
    }
}