risingwave_frontend/planner/
mod.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::binder::{BoundStatement, ShareId};
18use crate::error::Result;
19use crate::optimizer::{OptimizerContextRef, PlanRoot};
20
21mod changelog;
22mod delete;
23mod insert;
24mod query;
25mod recursive_union;
26mod relation;
27mod select;
28mod set_expr;
29mod set_operation;
30mod statement;
31mod update;
32mod values;
33pub use query::LIMIT_ALL_COUNT;
34
35use crate::PlanRef;
36
37/// `Planner` converts a bound statement to a [`crate::optimizer::plan_node::PlanNode`] tree
38pub struct Planner {
39    ctx: OptimizerContextRef,
40    /// Mapping of `ShareId` to its share plan.
41    /// The share plan can be a CTE, a source, a view and so on.
42    share_cache: HashMap<ShareId, PlanRef>,
43    /// Plan for stream or batch.
44    plan_for: PlanFor,
45}
46
47#[derive(Debug, Copy, Clone)]
48pub enum PlanFor {
49    Stream,
50    /// Is the `Sink` in iceberg table engine.
51    /// It connects to the table node directly, while external stream jobs may connect to an iceberg source.
52    StreamIcebergEngineInternal,
53    /// Other batch queries, e.g., DML.
54    Batch,
55    /// Batch `SELECT` queries.
56    ///
57    /// ## Special handling
58    ///
59    /// Iceberg engine table will be converted to iceberg source based on this mode.
60    BatchDql,
61}
62
63impl Planner {
64    pub fn new_for_batch_dql(ctx: OptimizerContextRef) -> Planner {
65        Planner {
66            ctx,
67            share_cache: Default::default(),
68            plan_for: PlanFor::BatchDql,
69        }
70    }
71
72    pub fn new_for_batch(ctx: OptimizerContextRef) -> Planner {
73        Planner {
74            ctx,
75            share_cache: Default::default(),
76            plan_for: PlanFor::Batch,
77        }
78    }
79
80    pub fn new_for_stream(ctx: OptimizerContextRef) -> Planner {
81        Planner {
82            ctx,
83            share_cache: Default::default(),
84            plan_for: PlanFor::Stream,
85        }
86    }
87
88    pub fn new_for_iceberg_table_engine_sink(ctx: OptimizerContextRef) -> Planner {
89        Planner {
90            ctx,
91            share_cache: Default::default(),
92            plan_for: PlanFor::StreamIcebergEngineInternal,
93        }
94    }
95
96    /// Plan a [`BoundStatement`]. Need to bind a statement before plan.
97    pub fn plan(&mut self, stmt: BoundStatement) -> Result<PlanRoot> {
98        self.plan_statement(stmt)
99    }
100
101    pub fn ctx(&self) -> OptimizerContextRef {
102        self.ctx.clone()
103    }
104
105    pub fn plan_for(&self) -> PlanFor {
106        self.plan_for
107    }
108}