risingwave_frontend/planner/
query.rs1use 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 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 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 assert!(!with_ties);
60 LogicalLimit::create(plan, limit, offset)
62 } else {
63 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 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}