risingwave_frontend/planner/
mod.rs

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