risingwave_frontend/optimizer/rule/batch/
batch_push_limit_to_scan_rule.rs1use itertools::Itertools;
21
22use crate::optimizer::PlanRef;
23use crate::optimizer::plan_node::generic::PhysicalPlanRef;
24use crate::optimizer::plan_node::{BatchLimit, BatchSeqScan, PlanTreeNodeUnary};
25use crate::optimizer::rule::{BoxedRule, Rule};
26
27pub struct BatchPushLimitToScanRule {}
28
29impl Rule for BatchPushLimitToScanRule {
30 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
31 let limit: &BatchLimit = plan.as_batch_limit()?;
32 let limit_input = limit.input();
33 let scan: &BatchSeqScan = limit_input.as_batch_seq_scan()?;
34 if scan.limit().is_some() {
35 return None;
36 }
37 let pushed_limit = limit.limit() + limit.offset();
38 let new_scan = BatchSeqScan::new_with_dist(
39 scan.core().clone(),
40 scan.base.distribution().clone(),
41 scan.scan_ranges().iter().cloned().collect_vec(),
42 Some(pushed_limit),
43 );
44 Some(limit.clone_with_input(new_scan.into()).into())
45 }
46}
47
48impl BatchPushLimitToScanRule {
49 pub fn create() -> BoxedRule {
50 Box::new(BatchPushLimitToScanRule {})
51 }
52}