risingwave_frontend/planner/
query.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 fixedbitset::FixedBitSet;
16use risingwave_common::catalog::PROJECTED_ROW_ID_COLUMN_NAME;
17
18use crate::binder::BoundQuery;
19use crate::error::Result;
20use crate::optimizer::plan_node::{LogicalLimit, LogicalTopN};
21use crate::optimizer::property::{Order, RequiredDist};
22use crate::optimizer::{LogicalPlanRoot, PlanRoot};
23use crate::planner::Planner;
24
25pub const LIMIT_ALL_COUNT: u64 = u64::MAX / 2;
26
27impl Planner {
28    /// Plan a [`BoundQuery`]. Need to bind before planning.
29    ///
30    /// Works for both batch query and streaming query (`CREATE MATERIALIZED VIEW`).
31    pub fn plan_query(&mut self, query: BoundQuery) -> Result<LogicalPlanRoot> {
32        let out_names = query.schema().names();
33        let BoundQuery {
34            body,
35            order,
36            limit,
37            offset,
38            with_ties,
39            extra_order_exprs,
40        } = query;
41
42        let extra_order_exprs_len = extra_order_exprs.len();
43        let mut plan = self.plan_set_expr(body, extra_order_exprs, &order)?;
44        let mut order = Order {
45            column_orders: order,
46        };
47
48        if limit.is_some() || offset.is_some() {
49            // Optimize order key if using it for TopN / Limit.
50            // Both are singleton dist, so we can leave dist_key_indices as empty.
51            let func_dep = plan.functional_dependency();
52            order = func_dep.minimize_order_key(order, &[]);
53
54            let limit = limit.unwrap_or(LIMIT_ALL_COUNT);
55
56            let offset = offset.unwrap_or_default();
57            plan = if order.column_orders.is_empty() {
58                // Should be rejected by parser.
59                assert!(!with_ties);
60                // Create a logical limit if with limit/offset but without order-by
61                LogicalLimit::create(plan, limit, offset)
62            } else {
63                // Create a logical top-n if with limit/offset and order-by
64                LogicalTopN::create(plan, limit, offset, order.clone(), with_ties, vec![])?
65            }
66        }
67        let mut out_fields = FixedBitSet::with_capacity(plan.schema().len());
68        out_fields.insert_range(..plan.schema().len() - extra_order_exprs_len);
69        if let Some(field) = plan.schema().fields.first()
70            && field.name == PROJECTED_ROW_ID_COLUMN_NAME
71        {
72            // Do not output projected_row_id hidden column.
73            out_fields.set(0, false);
74        }
75        let root =
76            PlanRoot::new_with_logical_plan(plan, RequiredDist::Any, order, out_fields, out_names);
77        Ok(root)
78    }
79}