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